You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@samza.apache.org by Chris Riccomini <cr...@apache.org> on 2015/07/01 03:55:12 UTC

Thoughts and obesrvations on Samza

Hey all,

I have had some discussions with Samza engineers at LinkedIn and Confluent
and we came up with a few observations and would like to propose some
changes.

We've observed some things that I want to call out about Samza's design,
and I'd like to propose some changes.

* Samza is dependent upon a dynamic deployment system.
* Samza is too pluggable.
* Samza's SystemConsumer/SystemProducer and Kafka's consumer APIs are
trying to solve a lot of the same problems.

All three of these issues are related, but I'll address them in order.

Deployment

Samza strongly depends on the use of a dynamic deployment scheduler such as
YARN, Mesos, etc. When we initially built Samza, we bet that there would be
one or two winners in this area, and we could support them, and the rest
would go away. In reality, there are many variations. Furthermore, many
people still prefer to just start their processors like normal Java
processes, and use traditional deployment scripts such as Fabric, Chef,
Ansible, etc. Forcing a deployment system on users makes the Samza start-up
process really painful for first time users.

Dynamic deployment as a requirement was also a bit of a mis-fire because of
a fundamental misunderstanding between the nature of batch jobs and stream
processing jobs. Early on, we made conscious effort to favor the Hadoop
(Map/Reduce) way of doing things, since it worked and was well understood.
One thing that we missed was that batch jobs have a definite beginning, and
end, and stream processing jobs don't (usually). This leads to a much
simpler scheduling problem for stream processors. You basically just need
to find a place to start the processor, and start it. The way we run grids,
at LinkedIn, there's no concept of a cluster being "full". We always add
more machines. The problem with coupling Samza with a scheduler is that
Samza (as a framework) now has to handle deployment. This pulls in a bunch
of things such as configuration distribution (config stream), shell scrips
(bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.

Another reason for requiring dynamic deployment was to support data
locality. If you want to have locality, you need to put your processors
close to the data they're processing. Upon further investigation, though,
this feature is not that beneficial. There is some good discussion about
some problems with it on SAMZA-335. Again, we took the Map/Reduce path, but
there are some fundamental differences between HDFS and Kafka. HDFS has
blocks, while Kafka has partitions. This leads to less optimization
potential with stream processors on top of Kafka.

This feature is also used as a crutch. Samza doesn't have any built in
fault-tolerance logic. Instead, it depends on the dynamic deployment
scheduling system to handle restarts when a processor dies. This has made
it very difficult to write a standalone Samza container (SAMZA-516).

Pluggability

In some cases pluggability is good, but I think that we've gone too far
with it. Currently, Samza has:

* Pluggable config.
* Pluggable metrics.
* Pluggable deployment systems.
* Pluggable streaming systems (SystemConsumer, SystemProducer, etc).
* Pluggable serdes.
* Pluggable storage engines.
* Pluggable strategies for just about every component (MessageChooser,
SystemStreamPartitionGrouper, ConfigRewriter, etc).

There's probably more that I've forgotten, as well. Some of these are
useful, but some have proven not to be. This all comes at a cost:
complexity. This complexity is making it harder for our users to pick up
and use Samza out of the box. It also makes it difficult for Samza
developers to reason about what the characteristics of the container (since
the characteristics change depending on which plugins are use).

The issues with pluggability are most visible in the System APIs. What
Samza really requires to be functional is Kafka as its transport layer. But
we've conflated two unrelated use cases into one API:

1. Get data into/out of Kafka.
2. Process the data in Kafka.

The current System API supports both of these use cases. The problem is, we
actually want different features for each use case. By papering over these
two use cases, and providing a single API, we've introduced a ton of leaky
abstractions.

For example, what we'd really like in (2) is to have monotonically
increasing longs for offsets (like Kafka). This would be at odds with (1),
though, since different systems have different SCNs/Offsets/UUIDs/vectors.
There was discussion both on the mailing list and the SQL JIRAs about the
need for this.

The same thing holds true for replayability. Kafka allows us to rewind when
we have a failure. Many other systems don't. In some cases, systems return
null for their offsets (e.g. WikipediaSystemConsumer) because they have no
offsets.

Partitioning is another example. Kafka supports partitioning, but many
systems don't. We model this by having a single partition for those
systems. Still, other systems model partitioning differently (e.g. Kinesis).

The SystemAdmin interface is also a mess. Creating streams in a
system-agnostic way is almost impossible. As is modeling metadata for the
system (replication factor, partitions, location, etc). The list goes on.

Duplicate work

At the time that we began writing Samza, Kafka's consumer and producer APIs
had a relatively weak feature set. On the consumer-side, you had two
options: use the high level consumer, or the simple consumer. The problem
with the high-level consumer was that it controlled your offsets, partition
assignments, and the order in which you received messages. The problem with
the simple consumer is that it's not simple. It's basic. You end up having
to handle a lot of really low-level stuff that you shouldn't. We spent a
lot of time to make Samza's KafkaSystemConsumer very robust. It also allows
us to support some cool features:

* Per-partition message ordering and prioritization.
* Tight control over partition assignment to support joins, global state
(if we want to implement it :)), etc.
* Tight control over offset checkpointing.

What we didn't realize at the time is that these features should actually
be in Kafka. A lot of Kafka consumers (not just Samza stream processors)
end up wanting to do things like joins and partition assignment. The Kafka
community has come to the same conclusion. They're adding a ton of upgrades
into their new Kafka consumer implementation. To a large extent, it's
duplicate work to what we've already done in Samza.

On top of this, Kafka ended up taking a very similar approach to Samza's
KafkaCheckpointManager implementation for handling offset checkpointing.
Like Samza, Kafka's new offset management feature stores offset checkpoints
in a topic, and allows you to fetch them from the broker.

A lot of this seems like a waste, since we could have shared the work if it
had been done in Kafka from the get-go.

Vision

All of this leads me to a rather radical proposal. Samza is relatively
stable at this point. I'd venture to say that we're near a 1.0 release. I'd
like to propose that we take what we've learned, and begin thinking about
Samza beyond 1.0. What would we change if we were starting from scratch? My
proposal is to:

1. Make Samza standalone the *only* way to run Samza processors, and
eliminate all direct dependences on YARN, Mesos, etc.
2. Make a definitive call to support only Kafka as the stream processing
layer.
3. Eliminate Samza's metrics, logging, serialization, and config systems,
and simply use Kafka's instead.

This would fix all of the issues that I outlined above. It should also
shrink the Samza code base pretty dramatically. Supporting only a
standalone container will allow Samza to be executed on YARN (using
Slider), Mesos (using Marathon/Aurora), or most other in-house deployment
systems. This should make life a lot easier for new users. Imagine having
the hello-samza tutorial without YARN. The drop in mailing list traffic
will be pretty dramatic.

Coupling with Kafka seems long overdue to me. The reality is, everyone that
I'm aware of is using Samza with Kafka. We basically require it already in
order for most features to work. Those that are using other systems are
generally using it for ingest into Kafka (1), and then they do the
processing on top. There is already discussion (
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767)
in Kafka to make ingesting into Kafka extremely easy.

Once we make the call to couple with Kafka, we can leverage a ton of their
ecosystem. We no longer have to maintain our own config, metrics, etc. We
can all share the same libraries, and make them better. This will also
allow us to share the consumer/producer APIs, and will let us leverage
their offset management and partition management, rather than having our
own. All of the coordinator stream code would go away, as would most of the
YARN AppMaster code. We'd probably have to push some partition management
features into the Kafka broker, but they're already moving in that
direction with the new consumer API. The features we have for partition
assignment aren't unique to Samza, and seem like they should be in Kafka
anyway. There will always be some niche usages which will require extra
care and hence full control over partition assignments much like the Kafka
low level consumer api. These would continue to be supported.

These items will be good for the Samza community. They'll make Samza easier
to use, and make it easier for developers to add new features.

Obviously this is a fairly large (and somewhat backwards incompatible
change). If we choose to go this route, it's important that we openly
communicate how we're going to provide a migration path from the existing
APIs to the new ones (if we make incompatible changes). I think at a
minimum, we'd probably need to provide a wrapper to allow existing
StreamTask implementations to continue running on the new container. It's
also important that we openly communicate about timing, and stages of the
migration.

If you made it this far, I'm sure you have opinions. :) Please send your
thoughts and feedback.

Cheers,
Chris

RE: Thoughts and obesrvations on Samza

Posted by Kartik Paramasivam <kp...@linkedin.com.INVALID>.
Thanks Chris/Jay for sending this out. 

I wanted to chime in to reflect some of the motivations from LinkedIn POV.

Late last year we started observing that there is a class of event driven applications at LinkedIn which can't leverage some of the cool work that is happening in Samza pertaining stream processing (windowing, joins, sql, and off course the local state abstraction etc.). The main reason was that these applications wanted to embed stream  processing along with their own service instead of running their stream processing logic in a shared Samza service.  Running Samza on their own was not workable for them as operating YARN a big road block for them.   Currently such applications end up directly consuming using Kafka client API or Databus client API instead .  

The Samza standalone work (Samza-516) that Chris did was a result of this.   This effort underlined the fact that if we had a deeper relationship with Kafka whereby we could leverage the partition distribution functionality in the kafka client, then we could avoid a lot of complexity in Samza for supporting stand alone mode.  

Jay's prototype helped establish the fact that if we leverage kafka in a deeper way the resulting code base for the Samza core would be vastly simplified. 
A deeper relationship where we can align efforts with Kafka is a win-win situation for both Kafka and Samza.

There are scenarios where the current state of partition distribution capabilities in Kafka are not yet sufficient for 100% of Samza jobs.  An example of such jobs are stateful samza jobs where the state is huge.  In essence with such jobs, the current behavior of reseeding of state from the Kafka changeLog during a simple upgrade is un-workable. This is the main reason we focussed on adding host affinity (SAMZA-617) .   The %age of such jobs is fairly low.  Hence for many Samza users out there this would not even be a problem.   In the new world, there will be a period of transition where Kafka doesn't have all the partition distribution capabilities that Samza needs. During this time we should be able to use the new core with the existing partition distribution and fault tolerance abilities that we get from the current Samza YARN-AppMaster.

One of the common questions I get is about the support for other existing system-consumers. At LinkedIn, a bigger %age of samza jobs consume events from non-kafka system-consumers (Databus).  What we observed was that practically all of such applications had a samza job which would consume from Databus and write to Kafka and have the rest of the downstream samza jobs consume just from Kafka. This was being done to improve testability, better ability to replay etc.  etc.  With the ongoing CopyCat discussion in Kafka, the hope is that we will get a much better story around ingesting data in Kafka from 100s of sources.

The other common question is that if we go down this path then what will happen to the existing Samza jobs.   Like Chris mentions, given that the new core is going to support all the stream processing functionality of the current core, it should (in theory) be simple to have the current SamzaContainer act like a wrapper on top of this new  core.  Current applications will hopefully not see a significant change.

Overall I am very excited about this proposal. Like all big changes it does feel uncomfortable in the beginning, but I feel that if we are able get through this then there is a better together story with Kafka waiting for us.

cheers
Kartik 
________________________________________
From: Jay Kreps [jay@confluent.io]
Sent: Tuesday, June 30, 2015 11:33 PM
To: dev@samza.apache.org
Subject: Re: Thoughts and obesrvations on Samza

Looks like gmail mangled the code example, it was supposed to look like
this:

Properties props = new Properties();
props.put("bootstrap.servers", "localhost:4242");
StreamingConfig config = new StreamingConfig(props);
config.subscribe("test-topic-1", "test-topic-2");
config.processor(ExampleStreamProcessor.class);
config.serialization(new StringSerializer(), new StringDeserializer());
KafkaStreaming container = new KafkaStreaming(config);
container.run();

-Jay

On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io> wrote:

> Hey guys,
>
> This came out of some conversations Chris and I were having around whether
> it would make sense to use Samza as a kind of data ingestion framework for
> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of combined
> with complaints around config and YARN and the discussion around how to
> best do a standalone mode.
>
> So the thought experiment was, given that Samza was basically already
> totally Kafka specific, what if you just embraced that and turned it into
> something less like a heavyweight framework and more like a third Kafka
> client--a kind of "producing consumer" with state management facilities.
> Basically a library. Instead of a complex stream processing framework this
> would actually be a very simple thing, not much more complicated to use or
> operate than a Kafka consumer. As Chris said we thought about it a lot of
> what Samza (and the other stream processing systems were doing) seemed like
> kind of a hangover from MapReduce.
>
> Of course you need to ingest/output data to and from the stream
> processing. But when we actually looked into how that would work, Samza
> isn't really an ideal data ingestion framework for a bunch of reasons. To
> really do that right you need a pretty different internal data model and
> set of apis. So what if you split them and had an api for Kafka
> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> transformation (Samza).
>
> This would also allow really embracing the same terminology and
> conventions. One complaint about the current state is that the two systems
> kind of feel bolted on. Terminology like "stream" vs "topic" and different
> config and monitoring systems means you kind of have to learn Kafka's way,
> then learn Samza's slightly different way, then kind of understand how they
> map to each other, which having walked a few people through this is
> surprisingly tricky for folks to get.
>
> Since I have been spending a lot of time on airplanes I hacked up an
> ernest but still somewhat incomplete prototype of what this would look
> like. This is just unceremoniously dumped into Kafka as it required a few
> changes to the new consumer. Here is the code:
>
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org/apache/kafka/clients/streaming
>
> For the purpose of the prototype I just liberally renamed everything to
> try to align it with Kafka with no regard for compatibility.
>
> To use this would be something like this:
> Properties props = new Properties(); props.put("bootstrap.servers",
> "localhost:4242"); StreamingConfig config = new StreamingConfig(props); config.subscribe("test-topic-1",
> "test-topic-2"); config.processor(ExampleStreamProcessor.class); config.serialization(new
> StringSerializer(), new StringDeserializer()); KafkaStreaming container =
> new KafkaStreaming(config); container.run();
>
> KafkaStreaming is basically the SamzaContainer; StreamProcessor is
> basically StreamTask.
>
> So rather than putting all the class names in a file and then having the
> job assembled by reflection, you just instantiate the container
> programmatically. Work is balanced over however many instances of this are
> alive at any time (i.e. if an instance dies, new tasks are added to the
> existing containers without shutting them down).
>
> We would provide some glue for running this stuff in YARN via Slider,
> Mesos via Marathon, and AWS using some of their tools but from the point of
> view of these frameworks these stream processing jobs are just stateless
> services that can come and go and expand and contract at will. There is no
> more custom scheduler.
>
> Here are some relevant details:
>
>    1. It is only ~1300 lines of code, it would get larger if we
>    productionized but not vastly larger. We really do get a ton of leverage
>    out of Kafka.
>    2. Partition management is fully delegated to the new consumer. This
>    is nice since now any partition management strategy available to Kafka
>    consumer is also available to Samza (and vice versa) and with the exact
>    same configs.
>    3. It supports state as well as state reuse
>
> Anyhow take a look, hopefully it is thought provoking.
>
> -Jay
>
>
>
> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
>> Hey all,
>>
>> I have had some discussions with Samza engineers at LinkedIn and Confluent
>> and we came up with a few observations and would like to propose some
>> changes.
>>
>> We've observed some things that I want to call out about Samza's design,
>> and I'd like to propose some changes.
>>
>> * Samza is dependent upon a dynamic deployment system.
>> * Samza is too pluggable.
>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer APIs are
>> trying to solve a lot of the same problems.
>>
>> All three of these issues are related, but I'll address them in order.
>>
>> Deployment
>>
>> Samza strongly depends on the use of a dynamic deployment scheduler such
>> as
>> YARN, Mesos, etc. When we initially built Samza, we bet that there would
>> be
>> one or two winners in this area, and we could support them, and the rest
>> would go away. In reality, there are many variations. Furthermore, many
>> people still prefer to just start their processors like normal Java
>> processes, and use traditional deployment scripts such as Fabric, Chef,
>> Ansible, etc. Forcing a deployment system on users makes the Samza
>> start-up
>> process really painful for first time users.
>>
>> Dynamic deployment as a requirement was also a bit of a mis-fire because
>> of
>> a fundamental misunderstanding between the nature of batch jobs and stream
>> processing jobs. Early on, we made conscious effort to favor the Hadoop
>> (Map/Reduce) way of doing things, since it worked and was well understood.
>> One thing that we missed was that batch jobs have a definite beginning,
>> and
>> end, and stream processing jobs don't (usually). This leads to a much
>> simpler scheduling problem for stream processors. You basically just need
>> to find a place to start the processor, and start it. The way we run
>> grids,
>> at LinkedIn, there's no concept of a cluster being "full". We always add
>> more machines. The problem with coupling Samza with a scheduler is that
>> Samza (as a framework) now has to handle deployment. This pulls in a bunch
>> of things such as configuration distribution (config stream), shell scrips
>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
>>
>> Another reason for requiring dynamic deployment was to support data
>> locality. If you want to have locality, you need to put your processors
>> close to the data they're processing. Upon further investigation, though,
>> this feature is not that beneficial. There is some good discussion about
>> some problems with it on SAMZA-335. Again, we took the Map/Reduce path,
>> but
>> there are some fundamental differences between HDFS and Kafka. HDFS has
>> blocks, while Kafka has partitions. This leads to less optimization
>> potential with stream processors on top of Kafka.
>>
>> This feature is also used as a crutch. Samza doesn't have any built in
>> fault-tolerance logic. Instead, it depends on the dynamic deployment
>> scheduling system to handle restarts when a processor dies. This has made
>> it very difficult to write a standalone Samza container (SAMZA-516).
>>
>> Pluggability
>>
>> In some cases pluggability is good, but I think that we've gone too far
>> with it. Currently, Samza has:
>>
>> * Pluggable config.
>> * Pluggable metrics.
>> * Pluggable deployment systems.
>> * Pluggable streaming systems (SystemConsumer, SystemProducer, etc).
>> * Pluggable serdes.
>> * Pluggable storage engines.
>> * Pluggable strategies for just about every component (MessageChooser,
>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>>
>> There's probably more that I've forgotten, as well. Some of these are
>> useful, but some have proven not to be. This all comes at a cost:
>> complexity. This complexity is making it harder for our users to pick up
>> and use Samza out of the box. It also makes it difficult for Samza
>> developers to reason about what the characteristics of the container
>> (since
>> the characteristics change depending on which plugins are use).
>>
>> The issues with pluggability are most visible in the System APIs. What
>> Samza really requires to be functional is Kafka as its transport layer.
>> But
>> we've conflated two unrelated use cases into one API:
>>
>> 1. Get data into/out of Kafka.
>> 2. Process the data in Kafka.
>>
>> The current System API supports both of these use cases. The problem is,
>> we
>> actually want different features for each use case. By papering over these
>> two use cases, and providing a single API, we've introduced a ton of leaky
>> abstractions.
>>
>> For example, what we'd really like in (2) is to have monotonically
>> increasing longs for offsets (like Kafka). This would be at odds with (1),
>> though, since different systems have different SCNs/Offsets/UUIDs/vectors.
>> There was discussion both on the mailing list and the SQL JIRAs about the
>> need for this.
>>
>> The same thing holds true for replayability. Kafka allows us to rewind
>> when
>> we have a failure. Many other systems don't. In some cases, systems return
>> null for their offsets (e.g. WikipediaSystemConsumer) because they have no
>> offsets.
>>
>> Partitioning is another example. Kafka supports partitioning, but many
>> systems don't. We model this by having a single partition for those
>> systems. Still, other systems model partitioning differently (e.g.
>> Kinesis).
>>
>> The SystemAdmin interface is also a mess. Creating streams in a
>> system-agnostic way is almost impossible. As is modeling metadata for the
>> system (replication factor, partitions, location, etc). The list goes on.
>>
>> Duplicate work
>>
>> At the time that we began writing Samza, Kafka's consumer and producer
>> APIs
>> had a relatively weak feature set. On the consumer-side, you had two
>> options: use the high level consumer, or the simple consumer. The problem
>> with the high-level consumer was that it controlled your offsets,
>> partition
>> assignments, and the order in which you received messages. The problem
>> with
>> the simple consumer is that it's not simple. It's basic. You end up having
>> to handle a lot of really low-level stuff that you shouldn't. We spent a
>> lot of time to make Samza's KafkaSystemConsumer very robust. It also
>> allows
>> us to support some cool features:
>>
>> * Per-partition message ordering and prioritization.
>> * Tight control over partition assignment to support joins, global state
>> (if we want to implement it :)), etc.
>> * Tight control over offset checkpointing.
>>
>> What we didn't realize at the time is that these features should actually
>> be in Kafka. A lot of Kafka consumers (not just Samza stream processors)
>> end up wanting to do things like joins and partition assignment. The Kafka
>> community has come to the same conclusion. They're adding a ton of
>> upgrades
>> into their new Kafka consumer implementation. To a large extent, it's
>> duplicate work to what we've already done in Samza.
>>
>> On top of this, Kafka ended up taking a very similar approach to Samza's
>> KafkaCheckpointManager implementation for handling offset checkpointing.
>> Like Samza, Kafka's new offset management feature stores offset
>> checkpoints
>> in a topic, and allows you to fetch them from the broker.
>>
>> A lot of this seems like a waste, since we could have shared the work if
>> it
>> had been done in Kafka from the get-go.
>>
>> Vision
>>
>> All of this leads me to a rather radical proposal. Samza is relatively
>> stable at this point. I'd venture to say that we're near a 1.0 release.
>> I'd
>> like to propose that we take what we've learned, and begin thinking about
>> Samza beyond 1.0. What would we change if we were starting from scratch?
>> My
>> proposal is to:
>>
>> 1. Make Samza standalone the *only* way to run Samza processors, and
>> eliminate all direct dependences on YARN, Mesos, etc.
>> 2. Make a definitive call to support only Kafka as the stream processing
>> layer.
>> 3. Eliminate Samza's metrics, logging, serialization, and config systems,
>> and simply use Kafka's instead.
>>
>> This would fix all of the issues that I outlined above. It should also
>> shrink the Samza code base pretty dramatically. Supporting only a
>> standalone container will allow Samza to be executed on YARN (using
>> Slider), Mesos (using Marathon/Aurora), or most other in-house deployment
>> systems. This should make life a lot easier for new users. Imagine having
>> the hello-samza tutorial without YARN. The drop in mailing list traffic
>> will be pretty dramatic.
>>
>> Coupling with Kafka seems long overdue to me. The reality is, everyone
>> that
>> I'm aware of is using Samza with Kafka. We basically require it already in
>> order for most features to work. Those that are using other systems are
>> generally using it for ingest into Kafka (1), and then they do the
>> processing on top. There is already discussion (
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
>> )
>> in Kafka to make ingesting into Kafka extremely easy.
>>
>> Once we make the call to couple with Kafka, we can leverage a ton of their
>> ecosystem. We no longer have to maintain our own config, metrics, etc. We
>> can all share the same libraries, and make them better. This will also
>> allow us to share the consumer/producer APIs, and will let us leverage
>> their offset management and partition management, rather than having our
>> own. All of the coordinator stream code would go away, as would most of
>> the
>> YARN AppMaster code. We'd probably have to push some partition management
>> features into the Kafka broker, but they're already moving in that
>> direction with the new consumer API. The features we have for partition
>> assignment aren't unique to Samza, and seem like they should be in Kafka
>> anyway. There will always be some niche usages which will require extra
>> care and hence full control over partition assignments much like the Kafka
>> low level consumer api. These would continue to be supported.
>>
>> These items will be good for the Samza community. They'll make Samza
>> easier
>> to use, and make it easier for developers to add new features.
>>
>> Obviously this is a fairly large (and somewhat backwards incompatible
>> change). If we choose to go this route, it's important that we openly
>> communicate how we're going to provide a migration path from the existing
>> APIs to the new ones (if we make incompatible changes). I think at a
>> minimum, we'd probably need to provide a wrapper to allow existing
>> StreamTask implementations to continue running on the new container. It's
>> also important that we openly communicate about timing, and stages of the
>> migration.
>>
>> If you made it this far, I'm sure you have opinions. :) Please send your
>> thoughts and feedback.
>>
>> Cheers,
>> Chris
>>
>
>

Re: Thoughts and obesrvations on Samza

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

I think Chris and I are proposing the same thing. I not really saying that
we should literally make Samza a Kafka client, but rather that
philosophically what we want to have is closer to a fancy client than it is
to map/reduce (but current samza is the reverse).

To answer your questions, both work exactly as the proposed standalone mode
works. The key observation is that the new Kafka consumer support on the
server provides what is basically a much more operationally sound version
of the standalone functionality we were going to bake into the container.
The idea is that if you are kafka-specific you can just use that.

So to answer your specific questions:
1. In both Chris and my proposal Samza gets out of the business of
deploying and starting processes. The rationale for this is that there are
soooo many ways of doing this well, and Samza currently forces a way that
isn't too great. Basically if you want to package your job in docker and
deploy with Mesos, fine. If you want to package it as a simple command line
program and start it with puppet/chef/salt/whatever, fine too.
2. However fault-tolerance remains. What this means is that although the
starting of processes is external all the processes work to balance
processing amongst themselves (just as in the proposed standalone mode).
All the state management stuff is preserved as is and is part of the
lifecycle of the tasks. To give a concrete example if you start a single
process it will consume all partitions and have all the tasks, as you start
more processes they will take over partitions, if some of them fail their
partitions will be given back to the remaining processes.

Shared state (assuming you are referring to broadcast topics?) is totally
orthogonal I think and would change with this proposal other than that the
partition assignment mechanism would have to have a "broadcast" assignment
strategy.

Either way I think the key idea is to kind of embrace Kafka--both in
naming, conventions, config, monitoring, etc so that you just have to
master one way of doing these things rather than having two layers.

-Jay



On Wed, Jul 1, 2015 at 10:59 PM, Yan Fang <ya...@gmail.com> wrote:

> Overall, I agree to couple with Kafka more tightly. Because Samza de facto
> is based on Kafka, and it should leverage what Kafka has. At the same time,
> Kafka does not need to reinvent what Samza already has. I also like the
> idea of separating the ingestion and transformation.
>
> But it is a little difficult for me to image how the Samza will look like.
> And I feel Chris and Jay have a little difference in terms of how Samza
> should look like.
>
> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
> user's application code calls this client?
>
> 1. If we make Samza be a library of Kafka (like what the code shows), how
> do we implement auto-balance and fault-tolerance? Are they taken care by
> the Kafka broker or other mechanism, such as "Samza worker" (just make up
> the name) ?
>
> 2. What about other features, such as auto-scaling, shared state,
> monitoring?
>
>
> *** If we have Samza standalone, (is this what Chris suggests?)
>
> 1. we still need to ingest data from Kakfa and produce to it. Then it
> becomes the same as what Samza looks like now, except it does not rely on
> Yarn anymore.
>
> 2. if it is standalone, how can it leverage Kafka's metrics, logs, etc? Use
> Kafka code as the dependency?
>
>
> Thanks,
>
> Fang, Yan
> yanfang724@gmail.com
>
> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Read through the code example and it looks good to me. A few thoughts
> > regarding deployment:
> >
> > Today Samza deploys as executable runnable like:
> >
> > deploy/samza/bin/run-job.sh --config-factory=... --config-path=file://...
> >
> > And this proposal advocate for deploying Samza more as embedded libraries
> > in user application code (ignoring the terminology since it is not the
> same
> > as the prototype code):
> >
> > StreamTask task = new MyStreamTask(configs);
> > Thread thread = new Thread(task);
> > thread.start();
> >
> > I think both of these deployment modes are important for different types
> of
> > users. That said, I think making Samza purely standalone is still
> > sufficient for either runnable or library modes.
> >
> > Guozhang
> >
> > On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Looks like gmail mangled the code example, it was supposed to look like
> > > this:
> > >
> > > Properties props = new Properties();
> > > props.put("bootstrap.servers", "localhost:4242");
> > > StreamingConfig config = new StreamingConfig(props);
> > > config.subscribe("test-topic-1", "test-topic-2");
> > > config.processor(ExampleStreamProcessor.class);
> > > config.serialization(new StringSerializer(), new StringDeserializer());
> > > KafkaStreaming container = new KafkaStreaming(config);
> > > container.run();
> > >
> > > -Jay
> > >
> > > On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Hey guys,
> > > >
> > > > This came out of some conversations Chris and I were having around
> > > whether
> > > > it would make sense to use Samza as a kind of data ingestion
> framework
> > > for
> > > > Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> > combined
> > > > with complaints around config and YARN and the discussion around how
> to
> > > > best do a standalone mode.
> > > >
> > > > So the thought experiment was, given that Samza was basically already
> > > > totally Kafka specific, what if you just embraced that and turned it
> > into
> > > > something less like a heavyweight framework and more like a third
> Kafka
> > > > client--a kind of "producing consumer" with state management
> > facilities.
> > > > Basically a library. Instead of a complex stream processing framework
> > > this
> > > > would actually be a very simple thing, not much more complicated to
> use
> > > or
> > > > operate than a Kafka consumer. As Chris said we thought about it a
> lot
> > of
> > > > what Samza (and the other stream processing systems were doing)
> seemed
> > > like
> > > > kind of a hangover from MapReduce.
> > > >
> > > > Of course you need to ingest/output data to and from the stream
> > > > processing. But when we actually looked into how that would work,
> Samza
> > > > isn't really an ideal data ingestion framework for a bunch of
> reasons.
> > To
> > > > really do that right you need a pretty different internal data model
> > and
> > > > set of apis. So what if you split them and had an api for Kafka
> > > > ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > > > transformation (Samza).
> > > >
> > > > This would also allow really embracing the same terminology and
> > > > conventions. One complaint about the current state is that the two
> > > systems
> > > > kind of feel bolted on. Terminology like "stream" vs "topic" and
> > > different
> > > > config and monitoring systems means you kind of have to learn Kafka's
> > > way,
> > > > then learn Samza's slightly different way, then kind of understand
> how
> > > they
> > > > map to each other, which having walked a few people through this is
> > > > surprisingly tricky for folks to get.
> > > >
> > > > Since I have been spending a lot of time on airplanes I hacked up an
> > > > ernest but still somewhat incomplete prototype of what this would
> look
> > > > like. This is just unceremoniously dumped into Kafka as it required a
> > few
> > > > changes to the new consumer. Here is the code:
> > > >
> > > >
> > >
> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org/apache/kafka/clients/streaming
> > > >
> > > > For the purpose of the prototype I just liberally renamed everything
> to
> > > > try to align it with Kafka with no regard for compatibility.
> > > >
> > > > To use this would be something like this:
> > > > Properties props = new Properties(); props.put("bootstrap.servers",
> > > > "localhost:4242"); StreamingConfig config = new
> StreamingConfig(props);
> > > config.subscribe("test-topic-1",
> > > > "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > > config.serialization(new
> > > > StringSerializer(), new StringDeserializer()); KafkaStreaming
> > container =
> > > > new KafkaStreaming(config); container.run();
> > > >
> > > > KafkaStreaming is basically the SamzaContainer; StreamProcessor is
> > > > basically StreamTask.
> > > >
> > > > So rather than putting all the class names in a file and then having
> > the
> > > > job assembled by reflection, you just instantiate the container
> > > > programmatically. Work is balanced over however many instances of
> this
> > > are
> > > > alive at any time (i.e. if an instance dies, new tasks are added to
> the
> > > > existing containers without shutting them down).
> > > >
> > > > We would provide some glue for running this stuff in YARN via Slider,
> > > > Mesos via Marathon, and AWS using some of their tools but from the
> > point
> > > of
> > > > view of these frameworks these stream processing jobs are just
> > stateless
> > > > services that can come and go and expand and contract at will. There
> is
> > > no
> > > > more custom scheduler.
> > > >
> > > > Here are some relevant details:
> > > >
> > > >    1. It is only ~1300 lines of code, it would get larger if we
> > > >    productionized but not vastly larger. We really do get a ton of
> > > leverage
> > > >    out of Kafka.
> > > >    2. Partition management is fully delegated to the new consumer.
> This
> > > >    is nice since now any partition management strategy available to
> > Kafka
> > > >    consumer is also available to Samza (and vice versa) and with the
> > > exact
> > > >    same configs.
> > > >    3. It supports state as well as state reuse
> > > >
> > > > Anyhow take a look, hopefully it is thought provoking.
> > > >
> > > > -Jay
> > > >
> > > >
> > > >
> > > > On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > criccomini@apache.org>
> > > > wrote:
> > > >
> > > >> Hey all,
> > > >>
> > > >> I have had some discussions with Samza engineers at LinkedIn and
> > > Confluent
> > > >> and we came up with a few observations and would like to propose
> some
> > > >> changes.
> > > >>
> > > >> We've observed some things that I want to call out about Samza's
> > design,
> > > >> and I'd like to propose some changes.
> > > >>
> > > >> * Samza is dependent upon a dynamic deployment system.
> > > >> * Samza is too pluggable.
> > > >> * Samza's SystemConsumer/SystemProducer and Kafka's consumer APIs
> are
> > > >> trying to solve a lot of the same problems.
> > > >>
> > > >> All three of these issues are related, but I'll address them in
> order.
> > > >>
> > > >> Deployment
> > > >>
> > > >> Samza strongly depends on the use of a dynamic deployment scheduler
> > such
> > > >> as
> > > >> YARN, Mesos, etc. When we initially built Samza, we bet that there
> > would
> > > >> be
> > > >> one or two winners in this area, and we could support them, and the
> > rest
> > > >> would go away. In reality, there are many variations. Furthermore,
> > many
> > > >> people still prefer to just start their processors like normal Java
> > > >> processes, and use traditional deployment scripts such as Fabric,
> > Chef,
> > > >> Ansible, etc. Forcing a deployment system on users makes the Samza
> > > >> start-up
> > > >> process really painful for first time users.
> > > >>
> > > >> Dynamic deployment as a requirement was also a bit of a mis-fire
> > because
> > > >> of
> > > >> a fundamental misunderstanding between the nature of batch jobs and
> > > stream
> > > >> processing jobs. Early on, we made conscious effort to favor the
> > Hadoop
> > > >> (Map/Reduce) way of doing things, since it worked and was well
> > > understood.
> > > >> One thing that we missed was that batch jobs have a definite
> > beginning,
> > > >> and
> > > >> end, and stream processing jobs don't (usually). This leads to a
> much
> > > >> simpler scheduling problem for stream processors. You basically just
> > > need
> > > >> to find a place to start the processor, and start it. The way we run
> > > >> grids,
> > > >> at LinkedIn, there's no concept of a cluster being "full". We always
> > add
> > > >> more machines. The problem with coupling Samza with a scheduler is
> > that
> > > >> Samza (as a framework) now has to handle deployment. This pulls in a
> > > bunch
> > > >> of things such as configuration distribution (config stream), shell
> > > scrips
> > > >> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> > > >>
> > > >> Another reason for requiring dynamic deployment was to support data
> > > >> locality. If you want to have locality, you need to put your
> > processors
> > > >> close to the data they're processing. Upon further investigation,
> > > though,
> > > >> this feature is not that beneficial. There is some good discussion
> > about
> > > >> some problems with it on SAMZA-335. Again, we took the Map/Reduce
> > path,
> > > >> but
> > > >> there are some fundamental differences between HDFS and Kafka. HDFS
> > has
> > > >> blocks, while Kafka has partitions. This leads to less optimization
> > > >> potential with stream processors on top of Kafka.
> > > >>
> > > >> This feature is also used as a crutch. Samza doesn't have any built
> in
> > > >> fault-tolerance logic. Instead, it depends on the dynamic deployment
> > > >> scheduling system to handle restarts when a processor dies. This has
> > > made
> > > >> it very difficult to write a standalone Samza container (SAMZA-516).
> > > >>
> > > >> Pluggability
> > > >>
> > > >> In some cases pluggability is good, but I think that we've gone too
> > far
> > > >> with it. Currently, Samza has:
> > > >>
> > > >> * Pluggable config.
> > > >> * Pluggable metrics.
> > > >> * Pluggable deployment systems.
> > > >> * Pluggable streaming systems (SystemConsumer, SystemProducer, etc).
> > > >> * Pluggable serdes.
> > > >> * Pluggable storage engines.
> > > >> * Pluggable strategies for just about every component
> (MessageChooser,
> > > >> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > >>
> > > >> There's probably more that I've forgotten, as well. Some of these
> are
> > > >> useful, but some have proven not to be. This all comes at a cost:
> > > >> complexity. This complexity is making it harder for our users to
> pick
> > up
> > > >> and use Samza out of the box. It also makes it difficult for Samza
> > > >> developers to reason about what the characteristics of the container
> > > >> (since
> > > >> the characteristics change depending on which plugins are use).
> > > >>
> > > >> The issues with pluggability are most visible in the System APIs.
> What
> > > >> Samza really requires to be functional is Kafka as its transport
> > layer.
> > > >> But
> > > >> we've conflated two unrelated use cases into one API:
> > > >>
> > > >> 1. Get data into/out of Kafka.
> > > >> 2. Process the data in Kafka.
> > > >>
> > > >> The current System API supports both of these use cases. The problem
> > is,
> > > >> we
> > > >> actually want different features for each use case. By papering over
> > > these
> > > >> two use cases, and providing a single API, we've introduced a ton of
> > > leaky
> > > >> abstractions.
> > > >>
> > > >> For example, what we'd really like in (2) is to have monotonically
> > > >> increasing longs for offsets (like Kafka). This would be at odds
> with
> > > (1),
> > > >> though, since different systems have different
> > > SCNs/Offsets/UUIDs/vectors.
> > > >> There was discussion both on the mailing list and the SQL JIRAs
> about
> > > the
> > > >> need for this.
> > > >>
> > > >> The same thing holds true for replayability. Kafka allows us to
> rewind
> > > >> when
> > > >> we have a failure. Many other systems don't. In some cases, systems
> > > return
> > > >> null for their offsets (e.g. WikipediaSystemConsumer) because they
> > have
> > > no
> > > >> offsets.
> > > >>
> > > >> Partitioning is another example. Kafka supports partitioning, but
> many
> > > >> systems don't. We model this by having a single partition for those
> > > >> systems. Still, other systems model partitioning differently (e.g.
> > > >> Kinesis).
> > > >>
> > > >> The SystemAdmin interface is also a mess. Creating streams in a
> > > >> system-agnostic way is almost impossible. As is modeling metadata
> for
> > > the
> > > >> system (replication factor, partitions, location, etc). The list
> goes
> > > on.
> > > >>
> > > >> Duplicate work
> > > >>
> > > >> At the time that we began writing Samza, Kafka's consumer and
> producer
> > > >> APIs
> > > >> had a relatively weak feature set. On the consumer-side, you had two
> > > >> options: use the high level consumer, or the simple consumer. The
> > > problem
> > > >> with the high-level consumer was that it controlled your offsets,
> > > >> partition
> > > >> assignments, and the order in which you received messages. The
> problem
> > > >> with
> > > >> the simple consumer is that it's not simple. It's basic. You end up
> > > having
> > > >> to handle a lot of really low-level stuff that you shouldn't. We
> > spent a
> > > >> lot of time to make Samza's KafkaSystemConsumer very robust. It also
> > > >> allows
> > > >> us to support some cool features:
> > > >>
> > > >> * Per-partition message ordering and prioritization.
> > > >> * Tight control over partition assignment to support joins, global
> > state
> > > >> (if we want to implement it :)), etc.
> > > >> * Tight control over offset checkpointing.
> > > >>
> > > >> What we didn't realize at the time is that these features should
> > > actually
> > > >> be in Kafka. A lot of Kafka consumers (not just Samza stream
> > processors)
> > > >> end up wanting to do things like joins and partition assignment. The
> > > Kafka
> > > >> community has come to the same conclusion. They're adding a ton of
> > > >> upgrades
> > > >> into their new Kafka consumer implementation. To a large extent,
> it's
> > > >> duplicate work to what we've already done in Samza.
> > > >>
> > > >> On top of this, Kafka ended up taking a very similar approach to
> > Samza's
> > > >> KafkaCheckpointManager implementation for handling offset
> > checkpointing.
> > > >> Like Samza, Kafka's new offset management feature stores offset
> > > >> checkpoints
> > > >> in a topic, and allows you to fetch them from the broker.
> > > >>
> > > >> A lot of this seems like a waste, since we could have shared the
> work
> > if
> > > >> it
> > > >> had been done in Kafka from the get-go.
> > > >>
> > > >> Vision
> > > >>
> > > >> All of this leads me to a rather radical proposal. Samza is
> relatively
> > > >> stable at this point. I'd venture to say that we're near a 1.0
> > release.
> > > >> I'd
> > > >> like to propose that we take what we've learned, and begin thinking
> > > about
> > > >> Samza beyond 1.0. What would we change if we were starting from
> > scratch?
> > > >> My
> > > >> proposal is to:
> > > >>
> > > >> 1. Make Samza standalone the *only* way to run Samza processors, and
> > > >> eliminate all direct dependences on YARN, Mesos, etc.
> > > >> 2. Make a definitive call to support only Kafka as the stream
> > processing
> > > >> layer.
> > > >> 3. Eliminate Samza's metrics, logging, serialization, and config
> > > systems,
> > > >> and simply use Kafka's instead.
> > > >>
> > > >> This would fix all of the issues that I outlined above. It should
> also
> > > >> shrink the Samza code base pretty dramatically. Supporting only a
> > > >> standalone container will allow Samza to be executed on YARN (using
> > > >> Slider), Mesos (using Marathon/Aurora), or most other in-house
> > > deployment
> > > >> systems. This should make life a lot easier for new users. Imagine
> > > having
> > > >> the hello-samza tutorial without YARN. The drop in mailing list
> > traffic
> > > >> will be pretty dramatic.
> > > >>
> > > >> Coupling with Kafka seems long overdue to me. The reality is,
> everyone
> > > >> that
> > > >> I'm aware of is using Samza with Kafka. We basically require it
> > already
> > > in
> > > >> order for most features to work. Those that are using other systems
> > are
> > > >> generally using it for ingest into Kafka (1), and then they do the
> > > >> processing on top. There is already discussion (
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> > > >> )
> > > >> in Kafka to make ingesting into Kafka extremely easy.
> > > >>
> > > >> Once we make the call to couple with Kafka, we can leverage a ton of
> > > their
> > > >> ecosystem. We no longer have to maintain our own config, metrics,
> etc.
> > > We
> > > >> can all share the same libraries, and make them better. This will
> also
> > > >> allow us to share the consumer/producer APIs, and will let us
> leverage
> > > >> their offset management and partition management, rather than having
> > our
> > > >> own. All of the coordinator stream code would go away, as would most
> > of
> > > >> the
> > > >> YARN AppMaster code. We'd probably have to push some partition
> > > management
> > > >> features into the Kafka broker, but they're already moving in that
> > > >> direction with the new consumer API. The features we have for
> > partition
> > > >> assignment aren't unique to Samza, and seem like they should be in
> > Kafka
> > > >> anyway. There will always be some niche usages which will require
> > extra
> > > >> care and hence full control over partition assignments much like the
> > > Kafka
> > > >> low level consumer api. These would continue to be supported.
> > > >>
> > > >> These items will be good for the Samza community. They'll make Samza
> > > >> easier
> > > >> to use, and make it easier for developers to add new features.
> > > >>
> > > >> Obviously this is a fairly large (and somewhat backwards
> incompatible
> > > >> change). If we choose to go this route, it's important that we
> openly
> > > >> communicate how we're going to provide a migration path from the
> > > existing
> > > >> APIs to the new ones (if we make incompatible changes). I think at a
> > > >> minimum, we'd probably need to provide a wrapper to allow existing
> > > >> StreamTask implementations to continue running on the new container.
> > > It's
> > > >> also important that we openly communicate about timing, and stages
> of
> > > the
> > > >> migration.
> > > >>
> > > >> If you made it this far, I'm sure you have opinions. :) Please send
> > your
> > > >> thoughts and feedback.
> > > >>
> > > >> Cheers,
> > > >> Chris
> > > >>
> > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
{quote}
There is a good chance that the different execution frameworks can be
abstract out
{quote}
Actually, I think that even we define an abstract layer, it won't need to
have Samza process be aware of the existence of the abstract layer. The
Samza container running as a process can be totally unaware of the
execution framework it uses. The job submission/configuration/launching
tools can be completely isolated from the Samza container as a process,
ideally.

On Mon, Jul 6, 2015 at 3:13 PM, Yi Pan <ni...@gmail.com> wrote:

> @Jay, you got my point.
>
> {quote}
> I think the question is whether for the "as a
> service" you are proposing actually trying to build some layer over
> YARN/Mesos/AWS that abstracts these away?
> {quote}
> I am not very strong on this, but I do see that as an option. The reason I
> had is: if all we need from YARN/Mesos/AWS is simply:
> 1. a set of resource to run Samza containers
> 2. launch Samza containers on the chosen set of resource
> 3. make the container processes fault tolerant (i.e. monitor and restart
> the failed processes)
> There is a good chance that the different execution frameworks can be
> abstract out since all cluster management systems would need to provide the
> above set of functionalities. I would need to spend more time on the Mesos
> patch for Samza to come up w/ more concrete idea on the abstract layer. If
> it turns out that the abstraction is not possible, Samza as a service may
> have to be implemented in many versions on top of Samza as a process.
>
> On Mon, Jul 6, 2015 at 12:02 PM, Jay Kreps <ja...@gmail.com> wrote:
>
>> Hey Yi/Martin,
>>
>> With respect to state reuse: Yeah that is a better explanation of what I
>> was trying to say. That prototype actually includes state reuse (each task
>> checkpoints it's offset for the various partitions during the commit and
>> uses that to determine if the state is valid and can be reused on
>> restart).
>> So it is just a matter of a framework being able to express a preference
>> for the recently used hosts--those that can will get the optimization and
>> those that can't won't.
>>
>> Yi, for the three layers I think we agree, but I may be unclear on what
>> you
>> are actually proposing. I think the question is whether for the "as a
>> service" you are proposing actually trying to build some layer over
>> YARN/Mesos/AWS that abstracts these away? Of course, there is nothing that
>> requires such an abstraction to run stream processing as a service on top
>> of (say) YARN. Can you be more specific about what would be in-scope for
>> that?
>>
>> I am generally skeptical of a layer that abstracts away these frameworks
>> for two reasons. First I think they are quite different and each is
>> advancing fairly rapidly so abstracting over them may be hard. Second I
>> think the layer of abstraction doesn't really help the user. If you think
>> about it you tend to just adopt one of the frameworks (say Mesos/Marathon)
>> and you learn the tooling associated with that and use it for all your
>> different systems. Building an abstraction over that doesn't really give
>> you any flexibility: you can't swap in another framework because all your
>> other stuff is tied to (say) Mesos/Marathon, and the layer of indirection
>> obscures the interface the user is already familar with from other
>> systems.
>> But I think I may actually be misunderstanding your proposal...
>>
>> -Jay
>>
>> On Mon, Jul 6, 2015 at 11:30 AM, Yi Pan <ni...@gmail.com> wrote:
>>
>> > Hi, Martin,
>> >
>> > Great to hear your voice! I will just try to focus on your questions
>> > regarding to "w/o YARN" part.
>> >
>> > {quote}
>> > For example, would host affinity (SAMZA-617) still be possible?
>> > {quote}
>> > It would be possible if we separate the job execution/process launching
>> > from the partition assignment among all Samza containers. Host-affinity
>> > could follow the model below:
>> > a. We still keeps a container-host mapping as in SAMZA-617
>> > b. It would be a best-effort approach try to get the same set of host
>> from
>> > the job execution framework (e.g. as we did w/ YARN in SamzaAppMaster
>> code.
>> > If Slider/Marathon comes up w/ this feature, we can migrate to that as
>> > well).
>> > c. When the container starts, it discovers the local state and try to
>> > retain the same partitions via the partition management APIs (i.e.
>> > explicitly ask for a specific set of partitions to be assigned to the
>> > container, instead of default to whatever broker decides to assign to
>> the
>> > container)
>> >
>> > {quote}
>> > a question about the proposed API for instantiating a job in code
>> (rather
>> > than a properties file): when submitting a job to a cluster, is the idea
>> > that the instantiation code runs on a client somewhere, which then pokes
>> > the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code run on
>> > each container that is part of the job (in which case, how does the job
>> > submission to the cluster work)?
>> > {quote}
>> > I want to chime in here to propose the three layered model:
>> > 1. Samza as a library. In this case, the client application should
>> actually
>> > implement the interface to submit its own job to a cluster, if the
>> client
>> > application chooses to run in a cluster. Samza as a library won't be
>> poking
>> > any endpoints on YARN/Mesos/AWS
>> > 2. Samza as a process. In this case, a properties file could be desired
>> > when starting the Samza process. Here again, Samza as a process should
>> not
>> > need to interact w/ any endpoints on YARN/Mesos/AWS.
>> > 3. Samza as a service. In this case, we definitely need some job
>> > configuration and a Samza implementation of interface to submit jobs and
>> > their configuration to a cluster (i.e. YARN/Mesos/AWS, mostly the same
>> as
>> > we have today, except that we should just ask for resources to run Samza
>> > containers and leave the partition management aside)
>> >
>> > Thanks!
>> >
>> > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <ma...@kleppmann.com>
>> > wrote:
>> >
>> > > Hi all,
>> > >
>> > > Lots of good thoughts here.
>> > >
>> > > I agree with the general philosophy of tying Samza more firmly to
>> Kafka.
>> > > After I spent a while looking at integrating other message brokers
>> (e.g.
>> > > Kinesis) with SystemConsumer, I came to the conclusion that
>> > SystemConsumer
>> > > tacitly assumes a model so much like Kafka's that pretty much nobody
>> but
>> > > Kafka actually implements it. (Databus is perhaps an exception, but it
>> > > isn't widely used outside of LinkedIn.) Thus, making Samza fully
>> > dependent
>> > > on Kafka acknowledges that the system-independence was never as real
>> as
>> > we
>> > > perhaps made it out to be. The gains of code reuse are real.
>> > >
>> > > The idea of decoupling Samza from YARN has also always been appealing
>> to
>> > > me, for various reasons already mentioned in this thread. Although
>> making
>> > > Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems
>> laudable, I
>> > am
>> > > a little concerned that it will restrict us to a lowest common
>> > denominator.
>> > > For example, would host affinity (SAMZA-617) still be possible? For
>> jobs
>> > > with large amounts of state, I think SAMZA-617 would be a big boon,
>> since
>> > > restoring state off the changelog on every single restart is painful,
>> due
>> > > to long recovery times. It would be a shame if the decoupling from
>> YARN
>> > > made host affinity impossible.
>> > >
>> > > Jay, a question about the proposed API for instantiating a job in code
>> > > (rather than a properties file): when submitting a job to a cluster,
>> is
>> > the
>> > > idea that the instantiation code runs on a client somewhere, which
>> then
>> > > pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code
>> > run
>> > > on each container that is part of the job (in which case, how does the
>> > job
>> > > submission to the cluster work)?
>> > >
>> > > I agree with Garry that it doesn't feel right to make a 1.0 release
>> with
>> > a
>> > > plan for it to be immediately obsolete. So if this is going to
>> happen, I
>> > > think it would be more honest to stick with 0.* version numbers until
>> the
>> > > library-ified Samza has been implemented, is stable and widely used.
>> > >
>> > > Should the new Samza be a subproject of Kafka? There is precedent for
>> > > tight coupling between different Apache projects (e.g. Curator and
>> > > Zookeeper, or Slider and YARN), so I think remaining separate would be
>> > ok.
>> > > Even if Samza is fully dependent on Kafka, there is enough substance
>> in
>> > > Samza that it warrants being a separate project. An argument in
>> favour of
>> > > merging would be if we think Kafka has a much stronger "brand
>> presence"
>> > > than Samza; I'm ambivalent on that one. If the Kafka project is
>> willing
>> > to
>> > > endorse Samza as the "official" way of doing stateful stream
>> > > transformations, that would probably have much the same effect as
>> > > re-branding Samza as "Kafka Stream Processors" or suchlike. Close
>> > > collaboration between the two projects will be needed in any case.
>> > >
>> > > From a project management perspective, I guess the "new Samza" would
>> have
>> > > to be developed on a branch alongside ongoing maintenance of the
>> current
>> > > line of development? I think it would be important to continue
>> supporting
>> > > existing users, and provide a graceful migration path to the new
>> version.
>> > > Leaving the current versions unsupported and forcing people to rewrite
>> > > their jobs would send a bad signal.
>> > >
>> > > Best,
>> > > Martin
>> > >
>> > > On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
>> > >
>> > > > Hey Garry,
>> > > >
>> > > > Yeah that's super frustrating. I'd be happy to chat more about this
>> if
>> > > > you'd be interested. I think Chris and I started with the idea of
>> "what
>> > > > would it take to make Samza a kick-ass ingestion tool" but
>> ultimately
>> > we
>> > > > kind of came around to the idea that ingestion and transformation
>> had
>> > > > pretty different needs and coupling the two made things hard.
>> > > >
>> > > > For what it's worth I think copycat (KIP-26) actually will do what
>> you
>> > > are
>> > > > looking for.
>> > > >
>> > > > With regard to your point about slider, I don't necessarily
>> disagree.
>> > > But I
>> > > > think getting good YARN support is quite doable and I think we can
>> make
>> > > > that work well. I think the issue this proposal solves is that
>> > > technically
>> > > > it is pretty hard to support multiple cluster management systems the
>> > way
>> > > > things are now, you need to write an "app master" or "framework" for
>> > each
>> > > > and they are all a little different so testing is really hard. In
>> the
>> > > > absence of this we have been stuck with just YARN which has
>> fantastic
>> > > > penetration in the Hadoopy part of the org, but zero penetration
>> > > elsewhere.
>> > > > Given the huge amount of work being put in to slider, marathon, aws
>> > > > tooling, not to mention the umpteen related packaging technologies
>> > people
>> > > > want to use (Docker, Kubernetes, various cloud-specific deploy
>> tools,
>> > > etc)
>> > > > I really think it is important to get this right.
>> > > >
>> > > > -Jay
>> > > >
>> > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>> > > > g.turkington@improvedigital.com> wrote:
>> > > >
>> > > >> Hi all,
>> > > >>
>> > > >> I think the question below re does Samza become a sub-project of
>> Kafka
>> > > >> highlights the broader point around migration. Chris mentions
>> Samza's
>> > > >> maturity is heading towards a v1 release but I'm not sure it feels
>> > > right to
>> > > >> launch a v1 then immediately plan to deprecate most of it.
>> > > >>
>> > > >> From a selfish perspective I have some guys who have started
>> working
>> > > with
>> > > >> Samza and building some new consumers/producers was next up. Sounds
>> > like
>> > > >> that is absolutely not the direction to go. I need to look into the
>> > KIP
>> > > in
>> > > >> more detail but for me the attractiveness of adding new Samza
>> > > >> consumer/producers -- even if yes all they were doing was really
>> > getting
>> > > >> data into and out of Kafka --  was to avoid  having to worry about
>> the
>> > > >> lifecycle management of external clients. If there is a generic
>> Kafka
>> > > >> ingress/egress layer that I can plug a new connector into and have
>> a
>> > > lot of
>> > > >> the heavy lifting re scale and reliability done for me then it
>> gives
>> > me
>> > > all
>> > > >> the pushing new consumers/producers would. If not then it
>> complicates
>> > my
>> > > >> operational deployments.
>> > > >>
>> > > >> Which is similar to my other question with the proposal -- if we
>> > build a
>> > > >> fully available/stand-alone Samza plus the requisite shims to
>> > integrate
>> > > >> with Slider etc I suspect the former may be a lot more work than we
>> > > think.
>> > > >> We may make it much easier for a newcomer to get something running
>> but
>> > > >> having them step up and get a reliable production deployment may
>> still
>> > > >> dominate mailing list  traffic, if for different reasons than
>> today.
>> > > >>
>> > > >> Don't get me wrong -- I'm comfortable with making the Samza
>> dependency
>> > > on
>> > > >> Kafka much more explicit and I absolutely see the benefits  in the
>> > > >> reduction of duplication and clashing terminologies/abstractions
>> that
>> > > >> Chris/Jay describe. Samza as a library would likely be a very nice
>> > tool
>> > > to
>> > > >> add to the Kafka ecosystem. I just have the concerns above re the
>> > > >> operational side.
>> > > >>
>> > > >> Garry
>> > > >>
>> > > >> -----Original Message-----
>> > > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
>> > > >> Sent: 02 July 2015 12:56
>> > > >> To: dev@samza.apache.org
>> > > >> Subject: Re: Thoughts and obesrvations on Samza
>> > > >>
>> > > >> Very interesting thoughts.
>> > > >> From outside, I have always perceived Samza as a computing layer
>> over
>> > > >> Kafka.
>> > > >>
>> > > >> The question, maybe a bit provocative, is "should Samza be a
>> > sub-project
>> > > >> of Kafka then?"
>> > > >> Or does it make sense to keep it as a separate project with a
>> separate
>> > > >> governance?
>> > > >>
>> > > >> Cheers,
>> > > >>
>> > > >> --
>> > > >> Gianmarco
>> > > >>
>> > > >> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
>> > > >>
>> > > >>> Overall, I agree to couple with Kafka more tightly. Because Samza
>> de
>> > > >>> facto is based on Kafka, and it should leverage what Kafka has. At
>> > the
>> > > >>> same time, Kafka does not need to reinvent what Samza already
>> has. I
>> > > >>> also like the idea of separating the ingestion and transformation.
>> > > >>>
>> > > >>> But it is a little difficult for me to image how the Samza will
>> look
>> > > >> like.
>> > > >>> And I feel Chris and Jay have a little difference in terms of how
>> > > >>> Samza should look like.
>> > > >>>
>> > > >>> *** Will it look like what Jay's code shows (A client of Kakfa) ?
>> And
>> > > >>> user's application code calls this client?
>> > > >>>
>> > > >>> 1. If we make Samza be a library of Kafka (like what the code
>> shows),
>> > > >>> how do we implement auto-balance and fault-tolerance? Are they
>> taken
>> > > >>> care by the Kafka broker or other mechanism, such as "Samza
>> worker"
>> > > >>> (just make up the name) ?
>> > > >>>
>> > > >>> 2. What about other features, such as auto-scaling, shared state,
>> > > >>> monitoring?
>> > > >>>
>> > > >>>
>> > > >>> *** If we have Samza standalone, (is this what Chris suggests?)
>> > > >>>
>> > > >>> 1. we still need to ingest data from Kakfa and produce to it.
>> Then it
>> > > >>> becomes the same as what Samza looks like now, except it does not
>> > rely
>> > > >>> on Yarn anymore.
>> > > >>>
>> > > >>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
>> > > >>> etc? Use Kafka code as the dependency?
>> > > >>>
>> > > >>>
>> > > >>> Thanks,
>> > > >>>
>> > > >>> Fang, Yan
>> > > >>> yanfang724@gmail.com
>> > > >>>
>> > > >>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wangguoz@gmail.com
>> >
>> > > >> wrote:
>> > > >>>
>> > > >>>> Read through the code example and it looks good to me. A few
>> > > >>>> thoughts regarding deployment:
>> > > >>>>
>> > > >>>> Today Samza deploys as executable runnable like:
>> > > >>>>
>> > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
>> > > >> --config-path=file://...
>> > > >>>>
>> > > >>>> And this proposal advocate for deploying Samza more as embedded
>> > > >>>> libraries in user application code (ignoring the terminology
>> since
>> > > >>>> it is not the
>> > > >>> same
>> > > >>>> as the prototype code):
>> > > >>>>
>> > > >>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
>> > > >>>> Thread(task); thread.start();
>> > > >>>>
>> > > >>>> I think both of these deployment modes are important for
>> different
>> > > >>>> types
>> > > >>> of
>> > > >>>> users. That said, I think making Samza purely standalone is still
>> > > >>>> sufficient for either runnable or library modes.
>> > > >>>>
>> > > >>>> Guozhang
>> > > >>>>
>> > > >>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
>> > wrote:
>> > > >>>>
>> > > >>>>> Looks like gmail mangled the code example, it was supposed to
>> look
>> > > >>>>> like
>> > > >>>>> this:
>> > > >>>>>
>> > > >>>>> Properties props = new Properties();
>> > > >>>>> props.put("bootstrap.servers", "localhost:4242");
>> StreamingConfig
>> > > >>>>> config = new StreamingConfig(props);
>> > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
>> > > >>>>> config.processor(ExampleStreamProcessor.class);
>> > > >>>>> config.serialization(new StringSerializer(), new
>> > > >>>>> StringDeserializer()); KafkaStreaming container = new
>> > > >>>>> KafkaStreaming(config); container.run();
>> > > >>>>>
>> > > >>>>> -Jay
>> > > >>>>>
>> > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
>> > > >> wrote:
>> > > >>>>>
>> > > >>>>>> Hey guys,
>> > > >>>>>>
>> > > >>>>>> This came out of some conversations Chris and I were having
>> > > >>>>>> around
>> > > >>>>> whether
>> > > >>>>>> it would make sense to use Samza as a kind of data ingestion
>> > > >>> framework
>> > > >>>>> for
>> > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
>> > > >>>> combined
>> > > >>>>>> with complaints around config and YARN and the discussion
>> around
>> > > >>>>>> how
>> > > >>> to
>> > > >>>>>> best do a standalone mode.
>> > > >>>>>>
>> > > >>>>>> So the thought experiment was, given that Samza was basically
>> > > >>>>>> already totally Kafka specific, what if you just embraced that
>> > > >>>>>> and turned it
>> > > >>>> into
>> > > >>>>>> something less like a heavyweight framework and more like a
>> > > >>>>>> third
>> > > >>> Kafka
>> > > >>>>>> client--a kind of "producing consumer" with state management
>> > > >>>> facilities.
>> > > >>>>>> Basically a library. Instead of a complex stream processing
>> > > >>>>>> framework
>> > > >>>>> this
>> > > >>>>>> would actually be a very simple thing, not much more
>> complicated
>> > > >>>>>> to
>> > > >>> use
>> > > >>>>> or
>> > > >>>>>> operate than a Kafka consumer. As Chris said we thought about
>> it
>> > > >>>>>> a
>> > > >>> lot
>> > > >>>> of
>> > > >>>>>> what Samza (and the other stream processing systems were doing)
>> > > >>> seemed
>> > > >>>>> like
>> > > >>>>>> kind of a hangover from MapReduce.
>> > > >>>>>>
>> > > >>>>>> Of course you need to ingest/output data to and from the stream
>> > > >>>>>> processing. But when we actually looked into how that would
>> > > >>>>>> work,
>> > > >>> Samza
>> > > >>>>>> isn't really an ideal data ingestion framework for a bunch of
>> > > >>> reasons.
>> > > >>>> To
>> > > >>>>>> really do that right you need a pretty different internal data
>> > > >>>>>> model
>> > > >>>> and
>> > > >>>>>> set of apis. So what if you split them and had an api for Kafka
>> > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
>> Kafka
>> > > >>>>>> transformation (Samza).
>> > > >>>>>>
>> > > >>>>>> This would also allow really embracing the same terminology and
>> > > >>>>>> conventions. One complaint about the current state is that the
>> > > >>>>>> two
>> > > >>>>> systems
>> > > >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic"
>> and
>> > > >>>>> different
>> > > >>>>>> config and monitoring systems means you kind of have to learn
>> > > >>>>>> Kafka's
>> > > >>>>> way,
>> > > >>>>>> then learn Samza's slightly different way, then kind of
>> > > >>>>>> understand
>> > > >>> how
>> > > >>>>> they
>> > > >>>>>> map to each other, which having walked a few people through
>> this
>> > > >>>>>> is surprisingly tricky for folks to get.
>> > > >>>>>>
>> > > >>>>>> Since I have been spending a lot of time on airplanes I hacked
>> > > >>>>>> up an ernest but still somewhat incomplete prototype of what
>> > > >>>>>> this would
>> > > >>> look
>> > > >>>>>> like. This is just unceremoniously dumped into Kafka as it
>> > > >>>>>> required a
>> > > >>>> few
>> > > >>>>>> changes to the new consumer. Here is the code:
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>
>> > > >>>>
>> > > >>>
>> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> > > >>> /apache/kafka/clients/streaming
>> > > >>>>>>
>> > > >>>>>> For the purpose of the prototype I just liberally renamed
>> > > >>>>>> everything
>> > > >>> to
>> > > >>>>>> try to align it with Kafka with no regard for compatibility.
>> > > >>>>>>
>> > > >>>>>> To use this would be something like this:
>> > > >>>>>> Properties props = new Properties();
>> > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
>> > > >>>>>> StreamingConfig config = new
>> > > >>> StreamingConfig(props);
>> > > >>>>> config.subscribe("test-topic-1",
>> > > >>>>>> "test-topic-2");
>> config.processor(ExampleStreamProcessor.class);
>> > > >>>>> config.serialization(new
>> > > >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
>> > > >>>> container =
>> > > >>>>>> new KafkaStreaming(config); container.run();
>> > > >>>>>>
>> > > >>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
>> > > >>>>>> is basically StreamTask.
>> > > >>>>>>
>> > > >>>>>> So rather than putting all the class names in a file and then
>> > > >>>>>> having
>> > > >>>> the
>> > > >>>>>> job assembled by reflection, you just instantiate the container
>> > > >>>>>> programmatically. Work is balanced over however many instances
>> > > >>>>>> of
>> > > >>> this
>> > > >>>>> are
>> > > >>>>>> alive at any time (i.e. if an instance dies, new tasks are
>> added
>> > > >>>>>> to
>> > > >>> the
>> > > >>>>>> existing containers without shutting them down).
>> > > >>>>>>
>> > > >>>>>> We would provide some glue for running this stuff in YARN via
>> > > >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
>> > > >>>>>> but from the
>> > > >>>> point
>> > > >>>>> of
>> > > >>>>>> view of these frameworks these stream processing jobs are just
>> > > >>>> stateless
>> > > >>>>>> services that can come and go and expand and contract at will.
>> > > >>>>>> There
>> > > >>> is
>> > > >>>>> no
>> > > >>>>>> more custom scheduler.
>> > > >>>>>>
>> > > >>>>>> Here are some relevant details:
>> > > >>>>>>
>> > > >>>>>>   1. It is only ~1300 lines of code, it would get larger if we
>> > > >>>>>>   productionized but not vastly larger. We really do get a ton
>> > > >>>>>> of
>> > > >>>>> leverage
>> > > >>>>>>   out of Kafka.
>> > > >>>>>>   2. Partition management is fully delegated to the new
>> consumer.
>> > > >>> This
>> > > >>>>>>   is nice since now any partition management strategy available
>> > > >>>>>> to
>> > > >>>> Kafka
>> > > >>>>>>   consumer is also available to Samza (and vice versa) and with
>> > > >>>>>> the
>> > > >>>>> exact
>> > > >>>>>>   same configs.
>> > > >>>>>>   3. It supports state as well as state reuse
>> > > >>>>>>
>> > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
>> > > >>>>>>
>> > > >>>>>> -Jay
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>> > > >>>> criccomini@apache.org>
>> > > >>>>>> wrote:
>> > > >>>>>>
>> > > >>>>>>> Hey all,
>> > > >>>>>>>
>> > > >>>>>>> I have had some discussions with Samza engineers at LinkedIn
>> > > >>>>>>> and
>> > > >>>>> Confluent
>> > > >>>>>>> and we came up with a few observations and would like to
>> > > >>>>>>> propose
>> > > >>> some
>> > > >>>>>>> changes.
>> > > >>>>>>>
>> > > >>>>>>> We've observed some things that I want to call out about
>> > > >>>>>>> Samza's
>> > > >>>> design,
>> > > >>>>>>> and I'd like to propose some changes.
>> > > >>>>>>>
>> > > >>>>>>> * Samza is dependent upon a dynamic deployment system.
>> > > >>>>>>> * Samza is too pluggable.
>> > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
>> > > >>>>>>> APIs
>> > > >>> are
>> > > >>>>>>> trying to solve a lot of the same problems.
>> > > >>>>>>>
>> > > >>>>>>> All three of these issues are related, but I'll address them
>> in
>> > > >>> order.
>> > > >>>>>>>
>> > > >>>>>>> Deployment
>> > > >>>>>>>
>> > > >>>>>>> Samza strongly depends on the use of a dynamic deployment
>> > > >>>>>>> scheduler
>> > > >>>> such
>> > > >>>>>>> as
>> > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
>> > > >>>>>>> there
>> > > >>>> would
>> > > >>>>>>> be
>> > > >>>>>>> one or two winners in this area, and we could support them,
>> and
>> > > >>>>>>> the
>> > > >>>> rest
>> > > >>>>>>> would go away. In reality, there are many variations.
>> > > >>>>>>> Furthermore,
>> > > >>>> many
>> > > >>>>>>> people still prefer to just start their processors like normal
>> > > >>>>>>> Java processes, and use traditional deployment scripts such as
>> > > >>>>>>> Fabric,
>> > > >>>> Chef,
>> > > >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
>> > > >>>>>>> Samza start-up process really painful for first time users.
>> > > >>>>>>>
>> > > >>>>>>> Dynamic deployment as a requirement was also a bit of a
>> > > >>>>>>> mis-fire
>> > > >>>> because
>> > > >>>>>>> of
>> > > >>>>>>> a fundamental misunderstanding between the nature of batch
>> jobs
>> > > >>>>>>> and
>> > > >>>>> stream
>> > > >>>>>>> processing jobs. Early on, we made conscious effort to favor
>> > > >>>>>>> the
>> > > >>>> Hadoop
>> > > >>>>>>> (Map/Reduce) way of doing things, since it worked and was well
>> > > >>>>> understood.
>> > > >>>>>>> One thing that we missed was that batch jobs have a definite
>> > > >>>> beginning,
>> > > >>>>>>> and
>> > > >>>>>>> end, and stream processing jobs don't (usually). This leads to
>> > > >>>>>>> a
>> > > >>> much
>> > > >>>>>>> simpler scheduling problem for stream processors. You
>> basically
>> > > >>>>>>> just
>> > > >>>>> need
>> > > >>>>>>> to find a place to start the processor, and start it. The way
>> > > >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
>> > > >>>>>>> being "full". We always
>> > > >>>> add
>> > > >>>>>>> more machines. The problem with coupling Samza with a
>> scheduler
>> > > >>>>>>> is
>> > > >>>> that
>> > > >>>>>>> Samza (as a framework) now has to handle deployment. This
>> pulls
>> > > >>>>>>> in a
>> > > >>>>> bunch
>> > > >>>>>>> of things such as configuration distribution (config stream),
>> > > >>>>>>> shell
>> > > >>>>> scrips
>> > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff),
>> etc.
>> > > >>>>>>>
>> > > >>>>>>> Another reason for requiring dynamic deployment was to support
>> > > >>>>>>> data locality. If you want to have locality, you need to put
>> > > >>>>>>> your
>> > > >>>> processors
>> > > >>>>>>> close to the data they're processing. Upon further
>> > > >>>>>>> investigation,
>> > > >>>>> though,
>> > > >>>>>>> this feature is not that beneficial. There is some good
>> > > >>>>>>> discussion
>> > > >>>> about
>> > > >>>>>>> some problems with it on SAMZA-335. Again, we took the
>> > > >>>>>>> Map/Reduce
>> > > >>>> path,
>> > > >>>>>>> but
>> > > >>>>>>> there are some fundamental differences between HDFS and Kafka.
>> > > >>>>>>> HDFS
>> > > >>>> has
>> > > >>>>>>> blocks, while Kafka has partitions. This leads to less
>> > > >>>>>>> optimization potential with stream processors on top of Kafka.
>> > > >>>>>>>
>> > > >>>>>>> This feature is also used as a crutch. Samza doesn't have any
>> > > >>>>>>> built
>> > > >>> in
>> > > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
>> > > >>>>>>> deployment scheduling system to handle restarts when a
>> > > >>>>>>> processor dies. This has
>> > > >>>>> made
>> > > >>>>>>> it very difficult to write a standalone Samza container
>> > > >> (SAMZA-516).
>> > > >>>>>>>
>> > > >>>>>>> Pluggability
>> > > >>>>>>>
>> > > >>>>>>> In some cases pluggability is good, but I think that we've
>> gone
>> > > >>>>>>> too
>> > > >>>> far
>> > > >>>>>>> with it. Currently, Samza has:
>> > > >>>>>>>
>> > > >>>>>>> * Pluggable config.
>> > > >>>>>>> * Pluggable metrics.
>> > > >>>>>>> * Pluggable deployment systems.
>> > > >>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
>> > > >> etc).
>> > > >>>>>>> * Pluggable serdes.
>> > > >>>>>>> * Pluggable storage engines.
>> > > >>>>>>> * Pluggable strategies for just about every component
>> > > >>> (MessageChooser,
>> > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>> > > >>>>>>>
>> > > >>>>>>> There's probably more that I've forgotten, as well. Some of
>> > > >>>>>>> these
>> > > >>> are
>> > > >>>>>>> useful, but some have proven not to be. This all comes at a
>> cost:
>> > > >>>>>>> complexity. This complexity is making it harder for our users
>> > > >>>>>>> to
>> > > >>> pick
>> > > >>>> up
>> > > >>>>>>> and use Samza out of the box. It also makes it difficult for
>> > > >>>>>>> Samza developers to reason about what the characteristics of
>> > > >>>>>>> the container (since the characteristics change depending on
>> > > >>>>>>> which plugins are use).
>> > > >>>>>>>
>> > > >>>>>>> The issues with pluggability are most visible in the System
>> APIs.
>> > > >>> What
>> > > >>>>>>> Samza really requires to be functional is Kafka as its
>> > > >>>>>>> transport
>> > > >>>> layer.
>> > > >>>>>>> But
>> > > >>>>>>> we've conflated two unrelated use cases into one API:
>> > > >>>>>>>
>> > > >>>>>>> 1. Get data into/out of Kafka.
>> > > >>>>>>> 2. Process the data in Kafka.
>> > > >>>>>>>
>> > > >>>>>>> The current System API supports both of these use cases. The
>> > > >>>>>>> problem
>> > > >>>> is,
>> > > >>>>>>> we
>> > > >>>>>>> actually want different features for each use case. By
>> papering
>> > > >>>>>>> over
>> > > >>>>> these
>> > > >>>>>>> two use cases, and providing a single API, we've introduced a
>> > > >>>>>>> ton of
>> > > >>>>> leaky
>> > > >>>>>>> abstractions.
>> > > >>>>>>>
>> > > >>>>>>> For example, what we'd really like in (2) is to have
>> > > >>>>>>> monotonically increasing longs for offsets (like Kafka). This
>> > > >>>>>>> would be at odds
>> > > >>> with
>> > > >>>>> (1),
>> > > >>>>>>> though, since different systems have different
>> > > >>>>> SCNs/Offsets/UUIDs/vectors.
>> > > >>>>>>> There was discussion both on the mailing list and the SQL
>> JIRAs
>> > > >>> about
>> > > >>>>> the
>> > > >>>>>>> need for this.
>> > > >>>>>>>
>> > > >>>>>>> The same thing holds true for replayability. Kafka allows us
>> to
>> > > >>> rewind
>> > > >>>>>>> when
>> > > >>>>>>> we have a failure. Many other systems don't. In some cases,
>> > > >>>>>>> systems
>> > > >>>>> return
>> > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
>> > > >>>>>>> they
>> > > >>>> have
>> > > >>>>> no
>> > > >>>>>>> offsets.
>> > > >>>>>>>
>> > > >>>>>>> Partitioning is another example. Kafka supports partitioning,
>> > > >>>>>>> but
>> > > >>> many
>> > > >>>>>>> systems don't. We model this by having a single partition for
>> > > >>>>>>> those systems. Still, other systems model partitioning
>> > > >> differently (e.g.
>> > > >>>>>>> Kinesis).
>> > > >>>>>>>
>> > > >>>>>>> The SystemAdmin interface is also a mess. Creating streams in
>> a
>> > > >>>>>>> system-agnostic way is almost impossible. As is modeling
>> > > >>>>>>> metadata
>> > > >>> for
>> > > >>>>> the
>> > > >>>>>>> system (replication factor, partitions, location, etc). The
>> > > >>>>>>> list
>> > > >>> goes
>> > > >>>>> on.
>> > > >>>>>>>
>> > > >>>>>>> Duplicate work
>> > > >>>>>>>
>> > > >>>>>>> At the time that we began writing Samza, Kafka's consumer and
>> > > >>> producer
>> > > >>>>>>> APIs
>> > > >>>>>>> had a relatively weak feature set. On the consumer-side, you
>> > > >>>>>>> had two
>> > > >>>>>>> options: use the high level consumer, or the simple consumer.
>> > > >>>>>>> The
>> > > >>>>> problem
>> > > >>>>>>> with the high-level consumer was that it controlled your
>> > > >>>>>>> offsets, partition assignments, and the order in which you
>> > > >>>>>>> received messages. The
>> > > >>> problem
>> > > >>>>>>> with
>> > > >>>>>>> the simple consumer is that it's not simple. It's basic. You
>> > > >>>>>>> end up
>> > > >>>>> having
>> > > >>>>>>> to handle a lot of really low-level stuff that you shouldn't.
>> > > >>>>>>> We
>> > > >>>> spent a
>> > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust.
>> It
>> > > >>>>>>> also allows us to support some cool features:
>> > > >>>>>>>
>> > > >>>>>>> * Per-partition message ordering and prioritization.
>> > > >>>>>>> * Tight control over partition assignment to support joins,
>> > > >>>>>>> global
>> > > >>>> state
>> > > >>>>>>> (if we want to implement it :)), etc.
>> > > >>>>>>> * Tight control over offset checkpointing.
>> > > >>>>>>>
>> > > >>>>>>> What we didn't realize at the time is that these features
>> > > >>>>>>> should
>> > > >>>>> actually
>> > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
>> > > >>>> processors)
>> > > >>>>>>> end up wanting to do things like joins and partition
>> > > >>>>>>> assignment. The
>> > > >>>>> Kafka
>> > > >>>>>>> community has come to the same conclusion. They're adding a
>> ton
>> > > >>>>>>> of upgrades into their new Kafka consumer implementation. To a
>> > > >>>>>>> large extent,
>> > > >>> it's
>> > > >>>>>>> duplicate work to what we've already done in Samza.
>> > > >>>>>>>
>> > > >>>>>>> On top of this, Kafka ended up taking a very similar approach
>> > > >>>>>>> to
>> > > >>>> Samza's
>> > > >>>>>>> KafkaCheckpointManager implementation for handling offset
>> > > >>>> checkpointing.
>> > > >>>>>>> Like Samza, Kafka's new offset management feature stores
>> offset
>> > > >>>>>>> checkpoints in a topic, and allows you to fetch them from the
>> > > >>>>>>> broker.
>> > > >>>>>>>
>> > > >>>>>>> A lot of this seems like a waste, since we could have shared
>> > > >>>>>>> the
>> > > >>> work
>> > > >>>> if
>> > > >>>>>>> it
>> > > >>>>>>> had been done in Kafka from the get-go.
>> > > >>>>>>>
>> > > >>>>>>> Vision
>> > > >>>>>>>
>> > > >>>>>>> All of this leads me to a rather radical proposal. Samza is
>> > > >>> relatively
>> > > >>>>>>> stable at this point. I'd venture to say that we're near a 1.0
>> > > >>>> release.
>> > > >>>>>>> I'd
>> > > >>>>>>> like to propose that we take what we've learned, and begin
>> > > >>>>>>> thinking
>> > > >>>>> about
>> > > >>>>>>> Samza beyond 1.0. What would we change if we were starting
>> from
>> > > >>>> scratch?
>> > > >>>>>>> My
>> > > >>>>>>> proposal is to:
>> > > >>>>>>>
>> > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
>> > > >>>>>>> processors, and eliminate all direct dependences on YARN,
>> Mesos,
>> > > >> etc.
>> > > >>>>>>> 2. Make a definitive call to support only Kafka as the stream
>> > > >>>> processing
>> > > >>>>>>> layer.
>> > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
>> > > >>>>>>> config
>> > > >>>>> systems,
>> > > >>>>>>> and simply use Kafka's instead.
>> > > >>>>>>>
>> > > >>>>>>> This would fix all of the issues that I outlined above. It
>> > > >>>>>>> should
>> > > >>> also
>> > > >>>>>>> shrink the Samza code base pretty dramatically. Supporting
>> only
>> > > >>>>>>> a standalone container will allow Samza to be executed on YARN
>> > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
>> > > >>>>>>> in-house
>> > > >>>>> deployment
>> > > >>>>>>> systems. This should make life a lot easier for new users.
>> > > >>>>>>> Imagine
>> > > >>>>> having
>> > > >>>>>>> the hello-samza tutorial without YARN. The drop in mailing
>> list
>> > > >>>> traffic
>> > > >>>>>>> will be pretty dramatic.
>> > > >>>>>>>
>> > > >>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
>> > > >>> everyone
>> > > >>>>>>> that
>> > > >>>>>>> I'm aware of is using Samza with Kafka. We basically require
>> it
>> > > >>>> already
>> > > >>>>> in
>> > > >>>>>>> order for most features to work. Those that are using other
>> > > >>>>>>> systems
>> > > >>>> are
>> > > >>>>>>> generally using it for ingest into Kafka (1), and then they do
>> > > >>>>>>> the processing on top. There is already discussion (
>> > > >>>>>>>
>> > > >>>>>
>> > > >>>>
>> > > >>>
>> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> > > >>> 767
>> > > >>>>>>> )
>> > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
>> > > >>>>>>>
>> > > >>>>>>> Once we make the call to couple with Kafka, we can leverage a
>> > > >>>>>>> ton of
>> > > >>>>> their
>> > > >>>>>>> ecosystem. We no longer have to maintain our own config,
>> > > >>>>>>> metrics,
>> > > >>> etc.
>> > > >>>>> We
>> > > >>>>>>> can all share the same libraries, and make them better. This
>> > > >>>>>>> will
>> > > >>> also
>> > > >>>>>>> allow us to share the consumer/producer APIs, and will let us
>> > > >>> leverage
>> > > >>>>>>> their offset management and partition management, rather than
>> > > >>>>>>> having
>> > > >>>> our
>> > > >>>>>>> own. All of the coordinator stream code would go away, as
>> would
>> > > >>>>>>> most
>> > > >>>> of
>> > > >>>>>>> the
>> > > >>>>>>> YARN AppMaster code. We'd probably have to push some partition
>> > > >>>>> management
>> > > >>>>>>> features into the Kafka broker, but they're already moving in
>> > > >>>>>>> that direction with the new consumer API. The features we have
>> > > >>>>>>> for
>> > > >>>> partition
>> > > >>>>>>> assignment aren't unique to Samza, and seem like they should
>> be
>> > > >>>>>>> in
>> > > >>>> Kafka
>> > > >>>>>>> anyway. There will always be some niche usages which will
>> > > >>>>>>> require
>> > > >>>> extra
>> > > >>>>>>> care and hence full control over partition assignments much
>> > > >>>>>>> like the
>> > > >>>>> Kafka
>> > > >>>>>>> low level consumer api. These would continue to be supported.
>> > > >>>>>>>
>> > > >>>>>>> These items will be good for the Samza community. They'll make
>> > > >>>>>>> Samza easier to use, and make it easier for developers to add
>> > > >>>>>>> new features.
>> > > >>>>>>>
>> > > >>>>>>> Obviously this is a fairly large (and somewhat backwards
>> > > >>> incompatible
>> > > >>>>>>> change). If we choose to go this route, it's important that we
>> > > >>> openly
>> > > >>>>>>> communicate how we're going to provide a migration path from
>> > > >>>>>>> the
>> > > >>>>> existing
>> > > >>>>>>> APIs to the new ones (if we make incompatible changes). I
>> think
>> > > >>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
>> > > >>>>>>> existing StreamTask implementations to continue running on the
>> > > >> new container.
>> > > >>>>> It's
>> > > >>>>>>> also important that we openly communicate about timing, and
>> > > >>>>>>> stages
>> > > >>> of
>> > > >>>>> the
>> > > >>>>>>> migration.
>> > > >>>>>>>
>> > > >>>>>>> If you made it this far, I'm sure you have opinions. :) Please
>> > > >>>>>>> send
>> > > >>>> your
>> > > >>>>>>> thoughts and feedback.
>> > > >>>>>>>
>> > > >>>>>>> Cheers,
>> > > >>>>>>> Chris
>> > > >>>>>>>
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>
>> > > >>>>
>> > > >>>>
>> > > >>>>
>> > > >>>> --
>> > > >>>> -- Guozhang
>> > > >>>>
>> > > >>>
>> > > >>
>> > >
>> > >
>> >
>>
>
>

Re: Thoughts and obesrvations on Samza

Posted by Timothy Chen <tn...@gmail.com>.
Hi all,

I remember there was a JIRA/patch about the Mesos integration long
time ago that was depending on some Samza feature to go in, and
haven't been tracking after that.

Not sure what the current status is, but as I see folks are discussing
about abstracting cluster manager and Mesos, bunch of different
frameworks has done this (Storm, Spark, etc). Let me know if there is
anything I can help out with on the Mesos integration side.

Tim

On Mon, Jul 6, 2015 at 3:13 PM, Yi Pan <ni...@gmail.com> wrote:
> @Jay, you got my point.
>
> {quote}
> I think the question is whether for the "as a
> service" you are proposing actually trying to build some layer over
> YARN/Mesos/AWS that abstracts these away?
> {quote}
> I am not very strong on this, but I do see that as an option. The reason I
> had is: if all we need from YARN/Mesos/AWS is simply:
> 1. a set of resource to run Samza containers
> 2. launch Samza containers on the chosen set of resource
> 3. make the container processes fault tolerant (i.e. monitor and restart
> the failed processes)
> There is a good chance that the different execution frameworks can be
> abstract out since all cluster management systems would need to provide the
> above set of functionalities. I would need to spend more time on the Mesos
> patch for Samza to come up w/ more concrete idea on the abstract layer. If
> it turns out that the abstraction is not possible, Samza as a service may
> have to be implemented in many versions on top of Samza as a process.
>
> On Mon, Jul 6, 2015 at 12:02 PM, Jay Kreps <ja...@gmail.com> wrote:
>
>> Hey Yi/Martin,
>>
>> With respect to state reuse: Yeah that is a better explanation of what I
>> was trying to say. That prototype actually includes state reuse (each task
>> checkpoints it's offset for the various partitions during the commit and
>> uses that to determine if the state is valid and can be reused on restart).
>> So it is just a matter of a framework being able to express a preference
>> for the recently used hosts--those that can will get the optimization and
>> those that can't won't.
>>
>> Yi, for the three layers I think we agree, but I may be unclear on what you
>> are actually proposing. I think the question is whether for the "as a
>> service" you are proposing actually trying to build some layer over
>> YARN/Mesos/AWS that abstracts these away? Of course, there is nothing that
>> requires such an abstraction to run stream processing as a service on top
>> of (say) YARN. Can you be more specific about what would be in-scope for
>> that?
>>
>> I am generally skeptical of a layer that abstracts away these frameworks
>> for two reasons. First I think they are quite different and each is
>> advancing fairly rapidly so abstracting over them may be hard. Second I
>> think the layer of abstraction doesn't really help the user. If you think
>> about it you tend to just adopt one of the frameworks (say Mesos/Marathon)
>> and you learn the tooling associated with that and use it for all your
>> different systems. Building an abstraction over that doesn't really give
>> you any flexibility: you can't swap in another framework because all your
>> other stuff is tied to (say) Mesos/Marathon, and the layer of indirection
>> obscures the interface the user is already familar with from other systems.
>> But I think I may actually be misunderstanding your proposal...
>>
>> -Jay
>>
>> On Mon, Jul 6, 2015 at 11:30 AM, Yi Pan <ni...@gmail.com> wrote:
>>
>> > Hi, Martin,
>> >
>> > Great to hear your voice! I will just try to focus on your questions
>> > regarding to "w/o YARN" part.
>> >
>> > {quote}
>> > For example, would host affinity (SAMZA-617) still be possible?
>> > {quote}
>> > It would be possible if we separate the job execution/process launching
>> > from the partition assignment among all Samza containers. Host-affinity
>> > could follow the model below:
>> > a. We still keeps a container-host mapping as in SAMZA-617
>> > b. It would be a best-effort approach try to get the same set of host
>> from
>> > the job execution framework (e.g. as we did w/ YARN in SamzaAppMaster
>> code.
>> > If Slider/Marathon comes up w/ this feature, we can migrate to that as
>> > well).
>> > c. When the container starts, it discovers the local state and try to
>> > retain the same partitions via the partition management APIs (i.e.
>> > explicitly ask for a specific set of partitions to be assigned to the
>> > container, instead of default to whatever broker decides to assign to the
>> > container)
>> >
>> > {quote}
>> > a question about the proposed API for instantiating a job in code (rather
>> > than a properties file): when submitting a job to a cluster, is the idea
>> > that the instantiation code runs on a client somewhere, which then pokes
>> > the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code run on
>> > each container that is part of the job (in which case, how does the job
>> > submission to the cluster work)?
>> > {quote}
>> > I want to chime in here to propose the three layered model:
>> > 1. Samza as a library. In this case, the client application should
>> actually
>> > implement the interface to submit its own job to a cluster, if the client
>> > application chooses to run in a cluster. Samza as a library won't be
>> poking
>> > any endpoints on YARN/Mesos/AWS
>> > 2. Samza as a process. In this case, a properties file could be desired
>> > when starting the Samza process. Here again, Samza as a process should
>> not
>> > need to interact w/ any endpoints on YARN/Mesos/AWS.
>> > 3. Samza as a service. In this case, we definitely need some job
>> > configuration and a Samza implementation of interface to submit jobs and
>> > their configuration to a cluster (i.e. YARN/Mesos/AWS, mostly the same as
>> > we have today, except that we should just ask for resources to run Samza
>> > containers and leave the partition management aside)
>> >
>> > Thanks!
>> >
>> > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <ma...@kleppmann.com>
>> > wrote:
>> >
>> > > Hi all,
>> > >
>> > > Lots of good thoughts here.
>> > >
>> > > I agree with the general philosophy of tying Samza more firmly to
>> Kafka.
>> > > After I spent a while looking at integrating other message brokers
>> (e.g.
>> > > Kinesis) with SystemConsumer, I came to the conclusion that
>> > SystemConsumer
>> > > tacitly assumes a model so much like Kafka's that pretty much nobody
>> but
>> > > Kafka actually implements it. (Databus is perhaps an exception, but it
>> > > isn't widely used outside of LinkedIn.) Thus, making Samza fully
>> > dependent
>> > > on Kafka acknowledges that the system-independence was never as real as
>> > we
>> > > perhaps made it out to be. The gains of code reuse are real.
>> > >
>> > > The idea of decoupling Samza from YARN has also always been appealing
>> to
>> > > me, for various reasons already mentioned in this thread. Although
>> making
>> > > Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems laudable,
>> I
>> > am
>> > > a little concerned that it will restrict us to a lowest common
>> > denominator.
>> > > For example, would host affinity (SAMZA-617) still be possible? For
>> jobs
>> > > with large amounts of state, I think SAMZA-617 would be a big boon,
>> since
>> > > restoring state off the changelog on every single restart is painful,
>> due
>> > > to long recovery times. It would be a shame if the decoupling from YARN
>> > > made host affinity impossible.
>> > >
>> > > Jay, a question about the proposed API for instantiating a job in code
>> > > (rather than a properties file): when submitting a job to a cluster, is
>> > the
>> > > idea that the instantiation code runs on a client somewhere, which then
>> > > pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code
>> > run
>> > > on each container that is part of the job (in which case, how does the
>> > job
>> > > submission to the cluster work)?
>> > >
>> > > I agree with Garry that it doesn't feel right to make a 1.0 release
>> with
>> > a
>> > > plan for it to be immediately obsolete. So if this is going to happen,
>> I
>> > > think it would be more honest to stick with 0.* version numbers until
>> the
>> > > library-ified Samza has been implemented, is stable and widely used.
>> > >
>> > > Should the new Samza be a subproject of Kafka? There is precedent for
>> > > tight coupling between different Apache projects (e.g. Curator and
>> > > Zookeeper, or Slider and YARN), so I think remaining separate would be
>> > ok.
>> > > Even if Samza is fully dependent on Kafka, there is enough substance in
>> > > Samza that it warrants being a separate project. An argument in favour
>> of
>> > > merging would be if we think Kafka has a much stronger "brand presence"
>> > > than Samza; I'm ambivalent on that one. If the Kafka project is willing
>> > to
>> > > endorse Samza as the "official" way of doing stateful stream
>> > > transformations, that would probably have much the same effect as
>> > > re-branding Samza as "Kafka Stream Processors" or suchlike. Close
>> > > collaboration between the two projects will be needed in any case.
>> > >
>> > > From a project management perspective, I guess the "new Samza" would
>> have
>> > > to be developed on a branch alongside ongoing maintenance of the
>> current
>> > > line of development? I think it would be important to continue
>> supporting
>> > > existing users, and provide a graceful migration path to the new
>> version.
>> > > Leaving the current versions unsupported and forcing people to rewrite
>> > > their jobs would send a bad signal.
>> > >
>> > > Best,
>> > > Martin
>> > >
>> > > On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
>> > >
>> > > > Hey Garry,
>> > > >
>> > > > Yeah that's super frustrating. I'd be happy to chat more about this
>> if
>> > > > you'd be interested. I think Chris and I started with the idea of
>> "what
>> > > > would it take to make Samza a kick-ass ingestion tool" but ultimately
>> > we
>> > > > kind of came around to the idea that ingestion and transformation had
>> > > > pretty different needs and coupling the two made things hard.
>> > > >
>> > > > For what it's worth I think copycat (KIP-26) actually will do what
>> you
>> > > are
>> > > > looking for.
>> > > >
>> > > > With regard to your point about slider, I don't necessarily disagree.
>> > > But I
>> > > > think getting good YARN support is quite doable and I think we can
>> make
>> > > > that work well. I think the issue this proposal solves is that
>> > > technically
>> > > > it is pretty hard to support multiple cluster management systems the
>> > way
>> > > > things are now, you need to write an "app master" or "framework" for
>> > each
>> > > > and they are all a little different so testing is really hard. In the
>> > > > absence of this we have been stuck with just YARN which has fantastic
>> > > > penetration in the Hadoopy part of the org, but zero penetration
>> > > elsewhere.
>> > > > Given the huge amount of work being put in to slider, marathon, aws
>> > > > tooling, not to mention the umpteen related packaging technologies
>> > people
>> > > > want to use (Docker, Kubernetes, various cloud-specific deploy tools,
>> > > etc)
>> > > > I really think it is important to get this right.
>> > > >
>> > > > -Jay
>> > > >
>> > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>> > > > g.turkington@improvedigital.com> wrote:
>> > > >
>> > > >> Hi all,
>> > > >>
>> > > >> I think the question below re does Samza become a sub-project of
>> Kafka
>> > > >> highlights the broader point around migration. Chris mentions
>> Samza's
>> > > >> maturity is heading towards a v1 release but I'm not sure it feels
>> > > right to
>> > > >> launch a v1 then immediately plan to deprecate most of it.
>> > > >>
>> > > >> From a selfish perspective I have some guys who have started working
>> > > with
>> > > >> Samza and building some new consumers/producers was next up. Sounds
>> > like
>> > > >> that is absolutely not the direction to go. I need to look into the
>> > KIP
>> > > in
>> > > >> more detail but for me the attractiveness of adding new Samza
>> > > >> consumer/producers -- even if yes all they were doing was really
>> > getting
>> > > >> data into and out of Kafka --  was to avoid  having to worry about
>> the
>> > > >> lifecycle management of external clients. If there is a generic
>> Kafka
>> > > >> ingress/egress layer that I can plug a new connector into and have a
>> > > lot of
>> > > >> the heavy lifting re scale and reliability done for me then it gives
>> > me
>> > > all
>> > > >> the pushing new consumers/producers would. If not then it
>> complicates
>> > my
>> > > >> operational deployments.
>> > > >>
>> > > >> Which is similar to my other question with the proposal -- if we
>> > build a
>> > > >> fully available/stand-alone Samza plus the requisite shims to
>> > integrate
>> > > >> with Slider etc I suspect the former may be a lot more work than we
>> > > think.
>> > > >> We may make it much easier for a newcomer to get something running
>> but
>> > > >> having them step up and get a reliable production deployment may
>> still
>> > > >> dominate mailing list  traffic, if for different reasons than today.
>> > > >>
>> > > >> Don't get me wrong -- I'm comfortable with making the Samza
>> dependency
>> > > on
>> > > >> Kafka much more explicit and I absolutely see the benefits  in the
>> > > >> reduction of duplication and clashing terminologies/abstractions
>> that
>> > > >> Chris/Jay describe. Samza as a library would likely be a very nice
>> > tool
>> > > to
>> > > >> add to the Kafka ecosystem. I just have the concerns above re the
>> > > >> operational side.
>> > > >>
>> > > >> Garry
>> > > >>
>> > > >> -----Original Message-----
>> > > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
>> > > >> Sent: 02 July 2015 12:56
>> > > >> To: dev@samza.apache.org
>> > > >> Subject: Re: Thoughts and obesrvations on Samza
>> > > >>
>> > > >> Very interesting thoughts.
>> > > >> From outside, I have always perceived Samza as a computing layer
>> over
>> > > >> Kafka.
>> > > >>
>> > > >> The question, maybe a bit provocative, is "should Samza be a
>> > sub-project
>> > > >> of Kafka then?"
>> > > >> Or does it make sense to keep it as a separate project with a
>> separate
>> > > >> governance?
>> > > >>
>> > > >> Cheers,
>> > > >>
>> > > >> --
>> > > >> Gianmarco
>> > > >>
>> > > >> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
>> > > >>
>> > > >>> Overall, I agree to couple with Kafka more tightly. Because Samza
>> de
>> > > >>> facto is based on Kafka, and it should leverage what Kafka has. At
>> > the
>> > > >>> same time, Kafka does not need to reinvent what Samza already has.
>> I
>> > > >>> also like the idea of separating the ingestion and transformation.
>> > > >>>
>> > > >>> But it is a little difficult for me to image how the Samza will
>> look
>> > > >> like.
>> > > >>> And I feel Chris and Jay have a little difference in terms of how
>> > > >>> Samza should look like.
>> > > >>>
>> > > >>> *** Will it look like what Jay's code shows (A client of Kakfa) ?
>> And
>> > > >>> user's application code calls this client?
>> > > >>>
>> > > >>> 1. If we make Samza be a library of Kafka (like what the code
>> shows),
>> > > >>> how do we implement auto-balance and fault-tolerance? Are they
>> taken
>> > > >>> care by the Kafka broker or other mechanism, such as "Samza worker"
>> > > >>> (just make up the name) ?
>> > > >>>
>> > > >>> 2. What about other features, such as auto-scaling, shared state,
>> > > >>> monitoring?
>> > > >>>
>> > > >>>
>> > > >>> *** If we have Samza standalone, (is this what Chris suggests?)
>> > > >>>
>> > > >>> 1. we still need to ingest data from Kakfa and produce to it. Then
>> it
>> > > >>> becomes the same as what Samza looks like now, except it does not
>> > rely
>> > > >>> on Yarn anymore.
>> > > >>>
>> > > >>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
>> > > >>> etc? Use Kafka code as the dependency?
>> > > >>>
>> > > >>>
>> > > >>> Thanks,
>> > > >>>
>> > > >>> Fang, Yan
>> > > >>> yanfang724@gmail.com
>> > > >>>
>> > > >>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
>> > > >> wrote:
>> > > >>>
>> > > >>>> Read through the code example and it looks good to me. A few
>> > > >>>> thoughts regarding deployment:
>> > > >>>>
>> > > >>>> Today Samza deploys as executable runnable like:
>> > > >>>>
>> > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
>> > > >> --config-path=file://...
>> > > >>>>
>> > > >>>> And this proposal advocate for deploying Samza more as embedded
>> > > >>>> libraries in user application code (ignoring the terminology since
>> > > >>>> it is not the
>> > > >>> same
>> > > >>>> as the prototype code):
>> > > >>>>
>> > > >>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
>> > > >>>> Thread(task); thread.start();
>> > > >>>>
>> > > >>>> I think both of these deployment modes are important for different
>> > > >>>> types
>> > > >>> of
>> > > >>>> users. That said, I think making Samza purely standalone is still
>> > > >>>> sufficient for either runnable or library modes.
>> > > >>>>
>> > > >>>> Guozhang
>> > > >>>>
>> > > >>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
>> > wrote:
>> > > >>>>
>> > > >>>>> Looks like gmail mangled the code example, it was supposed to
>> look
>> > > >>>>> like
>> > > >>>>> this:
>> > > >>>>>
>> > > >>>>> Properties props = new Properties();
>> > > >>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
>> > > >>>>> config = new StreamingConfig(props);
>> > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
>> > > >>>>> config.processor(ExampleStreamProcessor.class);
>> > > >>>>> config.serialization(new StringSerializer(), new
>> > > >>>>> StringDeserializer()); KafkaStreaming container = new
>> > > >>>>> KafkaStreaming(config); container.run();
>> > > >>>>>
>> > > >>>>> -Jay
>> > > >>>>>
>> > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
>> > > >> wrote:
>> > > >>>>>
>> > > >>>>>> Hey guys,
>> > > >>>>>>
>> > > >>>>>> This came out of some conversations Chris and I were having
>> > > >>>>>> around
>> > > >>>>> whether
>> > > >>>>>> it would make sense to use Samza as a kind of data ingestion
>> > > >>> framework
>> > > >>>>> for
>> > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
>> > > >>>> combined
>> > > >>>>>> with complaints around config and YARN and the discussion around
>> > > >>>>>> how
>> > > >>> to
>> > > >>>>>> best do a standalone mode.
>> > > >>>>>>
>> > > >>>>>> So the thought experiment was, given that Samza was basically
>> > > >>>>>> already totally Kafka specific, what if you just embraced that
>> > > >>>>>> and turned it
>> > > >>>> into
>> > > >>>>>> something less like a heavyweight framework and more like a
>> > > >>>>>> third
>> > > >>> Kafka
>> > > >>>>>> client--a kind of "producing consumer" with state management
>> > > >>>> facilities.
>> > > >>>>>> Basically a library. Instead of a complex stream processing
>> > > >>>>>> framework
>> > > >>>>> this
>> > > >>>>>> would actually be a very simple thing, not much more complicated
>> > > >>>>>> to
>> > > >>> use
>> > > >>>>> or
>> > > >>>>>> operate than a Kafka consumer. As Chris said we thought about it
>> > > >>>>>> a
>> > > >>> lot
>> > > >>>> of
>> > > >>>>>> what Samza (and the other stream processing systems were doing)
>> > > >>> seemed
>> > > >>>>> like
>> > > >>>>>> kind of a hangover from MapReduce.
>> > > >>>>>>
>> > > >>>>>> Of course you need to ingest/output data to and from the stream
>> > > >>>>>> processing. But when we actually looked into how that would
>> > > >>>>>> work,
>> > > >>> Samza
>> > > >>>>>> isn't really an ideal data ingestion framework for a bunch of
>> > > >>> reasons.
>> > > >>>> To
>> > > >>>>>> really do that right you need a pretty different internal data
>> > > >>>>>> model
>> > > >>>> and
>> > > >>>>>> set of apis. So what if you split them and had an api for Kafka
>> > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
>> > > >>>>>> transformation (Samza).
>> > > >>>>>>
>> > > >>>>>> This would also allow really embracing the same terminology and
>> > > >>>>>> conventions. One complaint about the current state is that the
>> > > >>>>>> two
>> > > >>>>> systems
>> > > >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
>> > > >>>>> different
>> > > >>>>>> config and monitoring systems means you kind of have to learn
>> > > >>>>>> Kafka's
>> > > >>>>> way,
>> > > >>>>>> then learn Samza's slightly different way, then kind of
>> > > >>>>>> understand
>> > > >>> how
>> > > >>>>> they
>> > > >>>>>> map to each other, which having walked a few people through this
>> > > >>>>>> is surprisingly tricky for folks to get.
>> > > >>>>>>
>> > > >>>>>> Since I have been spending a lot of time on airplanes I hacked
>> > > >>>>>> up an ernest but still somewhat incomplete prototype of what
>> > > >>>>>> this would
>> > > >>> look
>> > > >>>>>> like. This is just unceremoniously dumped into Kafka as it
>> > > >>>>>> required a
>> > > >>>> few
>> > > >>>>>> changes to the new consumer. Here is the code:
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>
>> > > >>>>
>> > > >>>
>> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> > > >>> /apache/kafka/clients/streaming
>> > > >>>>>>
>> > > >>>>>> For the purpose of the prototype I just liberally renamed
>> > > >>>>>> everything
>> > > >>> to
>> > > >>>>>> try to align it with Kafka with no regard for compatibility.
>> > > >>>>>>
>> > > >>>>>> To use this would be something like this:
>> > > >>>>>> Properties props = new Properties();
>> > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
>> > > >>>>>> StreamingConfig config = new
>> > > >>> StreamingConfig(props);
>> > > >>>>> config.subscribe("test-topic-1",
>> > > >>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
>> > > >>>>> config.serialization(new
>> > > >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
>> > > >>>> container =
>> > > >>>>>> new KafkaStreaming(config); container.run();
>> > > >>>>>>
>> > > >>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
>> > > >>>>>> is basically StreamTask.
>> > > >>>>>>
>> > > >>>>>> So rather than putting all the class names in a file and then
>> > > >>>>>> having
>> > > >>>> the
>> > > >>>>>> job assembled by reflection, you just instantiate the container
>> > > >>>>>> programmatically. Work is balanced over however many instances
>> > > >>>>>> of
>> > > >>> this
>> > > >>>>> are
>> > > >>>>>> alive at any time (i.e. if an instance dies, new tasks are added
>> > > >>>>>> to
>> > > >>> the
>> > > >>>>>> existing containers without shutting them down).
>> > > >>>>>>
>> > > >>>>>> We would provide some glue for running this stuff in YARN via
>> > > >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
>> > > >>>>>> but from the
>> > > >>>> point
>> > > >>>>> of
>> > > >>>>>> view of these frameworks these stream processing jobs are just
>> > > >>>> stateless
>> > > >>>>>> services that can come and go and expand and contract at will.
>> > > >>>>>> There
>> > > >>> is
>> > > >>>>> no
>> > > >>>>>> more custom scheduler.
>> > > >>>>>>
>> > > >>>>>> Here are some relevant details:
>> > > >>>>>>
>> > > >>>>>>   1. It is only ~1300 lines of code, it would get larger if we
>> > > >>>>>>   productionized but not vastly larger. We really do get a ton
>> > > >>>>>> of
>> > > >>>>> leverage
>> > > >>>>>>   out of Kafka.
>> > > >>>>>>   2. Partition management is fully delegated to the new
>> consumer.
>> > > >>> This
>> > > >>>>>>   is nice since now any partition management strategy available
>> > > >>>>>> to
>> > > >>>> Kafka
>> > > >>>>>>   consumer is also available to Samza (and vice versa) and with
>> > > >>>>>> the
>> > > >>>>> exact
>> > > >>>>>>   same configs.
>> > > >>>>>>   3. It supports state as well as state reuse
>> > > >>>>>>
>> > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
>> > > >>>>>>
>> > > >>>>>> -Jay
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>> > > >>>> criccomini@apache.org>
>> > > >>>>>> wrote:
>> > > >>>>>>
>> > > >>>>>>> Hey all,
>> > > >>>>>>>
>> > > >>>>>>> I have had some discussions with Samza engineers at LinkedIn
>> > > >>>>>>> and
>> > > >>>>> Confluent
>> > > >>>>>>> and we came up with a few observations and would like to
>> > > >>>>>>> propose
>> > > >>> some
>> > > >>>>>>> changes.
>> > > >>>>>>>
>> > > >>>>>>> We've observed some things that I want to call out about
>> > > >>>>>>> Samza's
>> > > >>>> design,
>> > > >>>>>>> and I'd like to propose some changes.
>> > > >>>>>>>
>> > > >>>>>>> * Samza is dependent upon a dynamic deployment system.
>> > > >>>>>>> * Samza is too pluggable.
>> > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
>> > > >>>>>>> APIs
>> > > >>> are
>> > > >>>>>>> trying to solve a lot of the same problems.
>> > > >>>>>>>
>> > > >>>>>>> All three of these issues are related, but I'll address them in
>> > > >>> order.
>> > > >>>>>>>
>> > > >>>>>>> Deployment
>> > > >>>>>>>
>> > > >>>>>>> Samza strongly depends on the use of a dynamic deployment
>> > > >>>>>>> scheduler
>> > > >>>> such
>> > > >>>>>>> as
>> > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
>> > > >>>>>>> there
>> > > >>>> would
>> > > >>>>>>> be
>> > > >>>>>>> one or two winners in this area, and we could support them, and
>> > > >>>>>>> the
>> > > >>>> rest
>> > > >>>>>>> would go away. In reality, there are many variations.
>> > > >>>>>>> Furthermore,
>> > > >>>> many
>> > > >>>>>>> people still prefer to just start their processors like normal
>> > > >>>>>>> Java processes, and use traditional deployment scripts such as
>> > > >>>>>>> Fabric,
>> > > >>>> Chef,
>> > > >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
>> > > >>>>>>> Samza start-up process really painful for first time users.
>> > > >>>>>>>
>> > > >>>>>>> Dynamic deployment as a requirement was also a bit of a
>> > > >>>>>>> mis-fire
>> > > >>>> because
>> > > >>>>>>> of
>> > > >>>>>>> a fundamental misunderstanding between the nature of batch jobs
>> > > >>>>>>> and
>> > > >>>>> stream
>> > > >>>>>>> processing jobs. Early on, we made conscious effort to favor
>> > > >>>>>>> the
>> > > >>>> Hadoop
>> > > >>>>>>> (Map/Reduce) way of doing things, since it worked and was well
>> > > >>>>> understood.
>> > > >>>>>>> One thing that we missed was that batch jobs have a definite
>> > > >>>> beginning,
>> > > >>>>>>> and
>> > > >>>>>>> end, and stream processing jobs don't (usually). This leads to
>> > > >>>>>>> a
>> > > >>> much
>> > > >>>>>>> simpler scheduling problem for stream processors. You basically
>> > > >>>>>>> just
>> > > >>>>> need
>> > > >>>>>>> to find a place to start the processor, and start it. The way
>> > > >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
>> > > >>>>>>> being "full". We always
>> > > >>>> add
>> > > >>>>>>> more machines. The problem with coupling Samza with a scheduler
>> > > >>>>>>> is
>> > > >>>> that
>> > > >>>>>>> Samza (as a framework) now has to handle deployment. This pulls
>> > > >>>>>>> in a
>> > > >>>>> bunch
>> > > >>>>>>> of things such as configuration distribution (config stream),
>> > > >>>>>>> shell
>> > > >>>>> scrips
>> > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff),
>> etc.
>> > > >>>>>>>
>> > > >>>>>>> Another reason for requiring dynamic deployment was to support
>> > > >>>>>>> data locality. If you want to have locality, you need to put
>> > > >>>>>>> your
>> > > >>>> processors
>> > > >>>>>>> close to the data they're processing. Upon further
>> > > >>>>>>> investigation,
>> > > >>>>> though,
>> > > >>>>>>> this feature is not that beneficial. There is some good
>> > > >>>>>>> discussion
>> > > >>>> about
>> > > >>>>>>> some problems with it on SAMZA-335. Again, we took the
>> > > >>>>>>> Map/Reduce
>> > > >>>> path,
>> > > >>>>>>> but
>> > > >>>>>>> there are some fundamental differences between HDFS and Kafka.
>> > > >>>>>>> HDFS
>> > > >>>> has
>> > > >>>>>>> blocks, while Kafka has partitions. This leads to less
>> > > >>>>>>> optimization potential with stream processors on top of Kafka.
>> > > >>>>>>>
>> > > >>>>>>> This feature is also used as a crutch. Samza doesn't have any
>> > > >>>>>>> built
>> > > >>> in
>> > > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
>> > > >>>>>>> deployment scheduling system to handle restarts when a
>> > > >>>>>>> processor dies. This has
>> > > >>>>> made
>> > > >>>>>>> it very difficult to write a standalone Samza container
>> > > >> (SAMZA-516).
>> > > >>>>>>>
>> > > >>>>>>> Pluggability
>> > > >>>>>>>
>> > > >>>>>>> In some cases pluggability is good, but I think that we've gone
>> > > >>>>>>> too
>> > > >>>> far
>> > > >>>>>>> with it. Currently, Samza has:
>> > > >>>>>>>
>> > > >>>>>>> * Pluggable config.
>> > > >>>>>>> * Pluggable metrics.
>> > > >>>>>>> * Pluggable deployment systems.
>> > > >>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
>> > > >> etc).
>> > > >>>>>>> * Pluggable serdes.
>> > > >>>>>>> * Pluggable storage engines.
>> > > >>>>>>> * Pluggable strategies for just about every component
>> > > >>> (MessageChooser,
>> > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>> > > >>>>>>>
>> > > >>>>>>> There's probably more that I've forgotten, as well. Some of
>> > > >>>>>>> these
>> > > >>> are
>> > > >>>>>>> useful, but some have proven not to be. This all comes at a
>> cost:
>> > > >>>>>>> complexity. This complexity is making it harder for our users
>> > > >>>>>>> to
>> > > >>> pick
>> > > >>>> up
>> > > >>>>>>> and use Samza out of the box. It also makes it difficult for
>> > > >>>>>>> Samza developers to reason about what the characteristics of
>> > > >>>>>>> the container (since the characteristics change depending on
>> > > >>>>>>> which plugins are use).
>> > > >>>>>>>
>> > > >>>>>>> The issues with pluggability are most visible in the System
>> APIs.
>> > > >>> What
>> > > >>>>>>> Samza really requires to be functional is Kafka as its
>> > > >>>>>>> transport
>> > > >>>> layer.
>> > > >>>>>>> But
>> > > >>>>>>> we've conflated two unrelated use cases into one API:
>> > > >>>>>>>
>> > > >>>>>>> 1. Get data into/out of Kafka.
>> > > >>>>>>> 2. Process the data in Kafka.
>> > > >>>>>>>
>> > > >>>>>>> The current System API supports both of these use cases. The
>> > > >>>>>>> problem
>> > > >>>> is,
>> > > >>>>>>> we
>> > > >>>>>>> actually want different features for each use case. By papering
>> > > >>>>>>> over
>> > > >>>>> these
>> > > >>>>>>> two use cases, and providing a single API, we've introduced a
>> > > >>>>>>> ton of
>> > > >>>>> leaky
>> > > >>>>>>> abstractions.
>> > > >>>>>>>
>> > > >>>>>>> For example, what we'd really like in (2) is to have
>> > > >>>>>>> monotonically increasing longs for offsets (like Kafka). This
>> > > >>>>>>> would be at odds
>> > > >>> with
>> > > >>>>> (1),
>> > > >>>>>>> though, since different systems have different
>> > > >>>>> SCNs/Offsets/UUIDs/vectors.
>> > > >>>>>>> There was discussion both on the mailing list and the SQL JIRAs
>> > > >>> about
>> > > >>>>> the
>> > > >>>>>>> need for this.
>> > > >>>>>>>
>> > > >>>>>>> The same thing holds true for replayability. Kafka allows us to
>> > > >>> rewind
>> > > >>>>>>> when
>> > > >>>>>>> we have a failure. Many other systems don't. In some cases,
>> > > >>>>>>> systems
>> > > >>>>> return
>> > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
>> > > >>>>>>> they
>> > > >>>> have
>> > > >>>>> no
>> > > >>>>>>> offsets.
>> > > >>>>>>>
>> > > >>>>>>> Partitioning is another example. Kafka supports partitioning,
>> > > >>>>>>> but
>> > > >>> many
>> > > >>>>>>> systems don't. We model this by having a single partition for
>> > > >>>>>>> those systems. Still, other systems model partitioning
>> > > >> differently (e.g.
>> > > >>>>>>> Kinesis).
>> > > >>>>>>>
>> > > >>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
>> > > >>>>>>> system-agnostic way is almost impossible. As is modeling
>> > > >>>>>>> metadata
>> > > >>> for
>> > > >>>>> the
>> > > >>>>>>> system (replication factor, partitions, location, etc). The
>> > > >>>>>>> list
>> > > >>> goes
>> > > >>>>> on.
>> > > >>>>>>>
>> > > >>>>>>> Duplicate work
>> > > >>>>>>>
>> > > >>>>>>> At the time that we began writing Samza, Kafka's consumer and
>> > > >>> producer
>> > > >>>>>>> APIs
>> > > >>>>>>> had a relatively weak feature set. On the consumer-side, you
>> > > >>>>>>> had two
>> > > >>>>>>> options: use the high level consumer, or the simple consumer.
>> > > >>>>>>> The
>> > > >>>>> problem
>> > > >>>>>>> with the high-level consumer was that it controlled your
>> > > >>>>>>> offsets, partition assignments, and the order in which you
>> > > >>>>>>> received messages. The
>> > > >>> problem
>> > > >>>>>>> with
>> > > >>>>>>> the simple consumer is that it's not simple. It's basic. You
>> > > >>>>>>> end up
>> > > >>>>> having
>> > > >>>>>>> to handle a lot of really low-level stuff that you shouldn't.
>> > > >>>>>>> We
>> > > >>>> spent a
>> > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
>> > > >>>>>>> also allows us to support some cool features:
>> > > >>>>>>>
>> > > >>>>>>> * Per-partition message ordering and prioritization.
>> > > >>>>>>> * Tight control over partition assignment to support joins,
>> > > >>>>>>> global
>> > > >>>> state
>> > > >>>>>>> (if we want to implement it :)), etc.
>> > > >>>>>>> * Tight control over offset checkpointing.
>> > > >>>>>>>
>> > > >>>>>>> What we didn't realize at the time is that these features
>> > > >>>>>>> should
>> > > >>>>> actually
>> > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
>> > > >>>> processors)
>> > > >>>>>>> end up wanting to do things like joins and partition
>> > > >>>>>>> assignment. The
>> > > >>>>> Kafka
>> > > >>>>>>> community has come to the same conclusion. They're adding a ton
>> > > >>>>>>> of upgrades into their new Kafka consumer implementation. To a
>> > > >>>>>>> large extent,
>> > > >>> it's
>> > > >>>>>>> duplicate work to what we've already done in Samza.
>> > > >>>>>>>
>> > > >>>>>>> On top of this, Kafka ended up taking a very similar approach
>> > > >>>>>>> to
>> > > >>>> Samza's
>> > > >>>>>>> KafkaCheckpointManager implementation for handling offset
>> > > >>>> checkpointing.
>> > > >>>>>>> Like Samza, Kafka's new offset management feature stores offset
>> > > >>>>>>> checkpoints in a topic, and allows you to fetch them from the
>> > > >>>>>>> broker.
>> > > >>>>>>>
>> > > >>>>>>> A lot of this seems like a waste, since we could have shared
>> > > >>>>>>> the
>> > > >>> work
>> > > >>>> if
>> > > >>>>>>> it
>> > > >>>>>>> had been done in Kafka from the get-go.
>> > > >>>>>>>
>> > > >>>>>>> Vision
>> > > >>>>>>>
>> > > >>>>>>> All of this leads me to a rather radical proposal. Samza is
>> > > >>> relatively
>> > > >>>>>>> stable at this point. I'd venture to say that we're near a 1.0
>> > > >>>> release.
>> > > >>>>>>> I'd
>> > > >>>>>>> like to propose that we take what we've learned, and begin
>> > > >>>>>>> thinking
>> > > >>>>> about
>> > > >>>>>>> Samza beyond 1.0. What would we change if we were starting from
>> > > >>>> scratch?
>> > > >>>>>>> My
>> > > >>>>>>> proposal is to:
>> > > >>>>>>>
>> > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
>> > > >>>>>>> processors, and eliminate all direct dependences on YARN,
>> Mesos,
>> > > >> etc.
>> > > >>>>>>> 2. Make a definitive call to support only Kafka as the stream
>> > > >>>> processing
>> > > >>>>>>> layer.
>> > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
>> > > >>>>>>> config
>> > > >>>>> systems,
>> > > >>>>>>> and simply use Kafka's instead.
>> > > >>>>>>>
>> > > >>>>>>> This would fix all of the issues that I outlined above. It
>> > > >>>>>>> should
>> > > >>> also
>> > > >>>>>>> shrink the Samza code base pretty dramatically. Supporting only
>> > > >>>>>>> a standalone container will allow Samza to be executed on YARN
>> > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
>> > > >>>>>>> in-house
>> > > >>>>> deployment
>> > > >>>>>>> systems. This should make life a lot easier for new users.
>> > > >>>>>>> Imagine
>> > > >>>>> having
>> > > >>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
>> > > >>>> traffic
>> > > >>>>>>> will be pretty dramatic.
>> > > >>>>>>>
>> > > >>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
>> > > >>> everyone
>> > > >>>>>>> that
>> > > >>>>>>> I'm aware of is using Samza with Kafka. We basically require it
>> > > >>>> already
>> > > >>>>> in
>> > > >>>>>>> order for most features to work. Those that are using other
>> > > >>>>>>> systems
>> > > >>>> are
>> > > >>>>>>> generally using it for ingest into Kafka (1), and then they do
>> > > >>>>>>> the processing on top. There is already discussion (
>> > > >>>>>>>
>> > > >>>>>
>> > > >>>>
>> > > >>>
>> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> > > >>> 767
>> > > >>>>>>> )
>> > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
>> > > >>>>>>>
>> > > >>>>>>> Once we make the call to couple with Kafka, we can leverage a
>> > > >>>>>>> ton of
>> > > >>>>> their
>> > > >>>>>>> ecosystem. We no longer have to maintain our own config,
>> > > >>>>>>> metrics,
>> > > >>> etc.
>> > > >>>>> We
>> > > >>>>>>> can all share the same libraries, and make them better. This
>> > > >>>>>>> will
>> > > >>> also
>> > > >>>>>>> allow us to share the consumer/producer APIs, and will let us
>> > > >>> leverage
>> > > >>>>>>> their offset management and partition management, rather than
>> > > >>>>>>> having
>> > > >>>> our
>> > > >>>>>>> own. All of the coordinator stream code would go away, as would
>> > > >>>>>>> most
>> > > >>>> of
>> > > >>>>>>> the
>> > > >>>>>>> YARN AppMaster code. We'd probably have to push some partition
>> > > >>>>> management
>> > > >>>>>>> features into the Kafka broker, but they're already moving in
>> > > >>>>>>> that direction with the new consumer API. The features we have
>> > > >>>>>>> for
>> > > >>>> partition
>> > > >>>>>>> assignment aren't unique to Samza, and seem like they should be
>> > > >>>>>>> in
>> > > >>>> Kafka
>> > > >>>>>>> anyway. There will always be some niche usages which will
>> > > >>>>>>> require
>> > > >>>> extra
>> > > >>>>>>> care and hence full control over partition assignments much
>> > > >>>>>>> like the
>> > > >>>>> Kafka
>> > > >>>>>>> low level consumer api. These would continue to be supported.
>> > > >>>>>>>
>> > > >>>>>>> These items will be good for the Samza community. They'll make
>> > > >>>>>>> Samza easier to use, and make it easier for developers to add
>> > > >>>>>>> new features.
>> > > >>>>>>>
>> > > >>>>>>> Obviously this is a fairly large (and somewhat backwards
>> > > >>> incompatible
>> > > >>>>>>> change). If we choose to go this route, it's important that we
>> > > >>> openly
>> > > >>>>>>> communicate how we're going to provide a migration path from
>> > > >>>>>>> the
>> > > >>>>> existing
>> > > >>>>>>> APIs to the new ones (if we make incompatible changes). I think
>> > > >>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
>> > > >>>>>>> existing StreamTask implementations to continue running on the
>> > > >> new container.
>> > > >>>>> It's
>> > > >>>>>>> also important that we openly communicate about timing, and
>> > > >>>>>>> stages
>> > > >>> of
>> > > >>>>> the
>> > > >>>>>>> migration.
>> > > >>>>>>>
>> > > >>>>>>> If you made it this far, I'm sure you have opinions. :) Please
>> > > >>>>>>> send
>> > > >>>> your
>> > > >>>>>>> thoughts and feedback.
>> > > >>>>>>>
>> > > >>>>>>> Cheers,
>> > > >>>>>>> Chris
>> > > >>>>>>>
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>
>> > > >>>>
>> > > >>>>
>> > > >>>>
>> > > >>>> --
>> > > >>>> -- Guozhang
>> > > >>>>
>> > > >>>
>> > > >>
>> > >
>> > >
>> >
>>


Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
@Jay, you got my point.

{quote}
I think the question is whether for the "as a
service" you are proposing actually trying to build some layer over
YARN/Mesos/AWS that abstracts these away?
{quote}
I am not very strong on this, but I do see that as an option. The reason I
had is: if all we need from YARN/Mesos/AWS is simply:
1. a set of resource to run Samza containers
2. launch Samza containers on the chosen set of resource
3. make the container processes fault tolerant (i.e. monitor and restart
the failed processes)
There is a good chance that the different execution frameworks can be
abstract out since all cluster management systems would need to provide the
above set of functionalities. I would need to spend more time on the Mesos
patch for Samza to come up w/ more concrete idea on the abstract layer. If
it turns out that the abstraction is not possible, Samza as a service may
have to be implemented in many versions on top of Samza as a process.

On Mon, Jul 6, 2015 at 12:02 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Yi/Martin,
>
> With respect to state reuse: Yeah that is a better explanation of what I
> was trying to say. That prototype actually includes state reuse (each task
> checkpoints it's offset for the various partitions during the commit and
> uses that to determine if the state is valid and can be reused on restart).
> So it is just a matter of a framework being able to express a preference
> for the recently used hosts--those that can will get the optimization and
> those that can't won't.
>
> Yi, for the three layers I think we agree, but I may be unclear on what you
> are actually proposing. I think the question is whether for the "as a
> service" you are proposing actually trying to build some layer over
> YARN/Mesos/AWS that abstracts these away? Of course, there is nothing that
> requires such an abstraction to run stream processing as a service on top
> of (say) YARN. Can you be more specific about what would be in-scope for
> that?
>
> I am generally skeptical of a layer that abstracts away these frameworks
> for two reasons. First I think they are quite different and each is
> advancing fairly rapidly so abstracting over them may be hard. Second I
> think the layer of abstraction doesn't really help the user. If you think
> about it you tend to just adopt one of the frameworks (say Mesos/Marathon)
> and you learn the tooling associated with that and use it for all your
> different systems. Building an abstraction over that doesn't really give
> you any flexibility: you can't swap in another framework because all your
> other stuff is tied to (say) Mesos/Marathon, and the layer of indirection
> obscures the interface the user is already familar with from other systems.
> But I think I may actually be misunderstanding your proposal...
>
> -Jay
>
> On Mon, Jul 6, 2015 at 11:30 AM, Yi Pan <ni...@gmail.com> wrote:
>
> > Hi, Martin,
> >
> > Great to hear your voice! I will just try to focus on your questions
> > regarding to "w/o YARN" part.
> >
> > {quote}
> > For example, would host affinity (SAMZA-617) still be possible?
> > {quote}
> > It would be possible if we separate the job execution/process launching
> > from the partition assignment among all Samza containers. Host-affinity
> > could follow the model below:
> > a. We still keeps a container-host mapping as in SAMZA-617
> > b. It would be a best-effort approach try to get the same set of host
> from
> > the job execution framework (e.g. as we did w/ YARN in SamzaAppMaster
> code.
> > If Slider/Marathon comes up w/ this feature, we can migrate to that as
> > well).
> > c. When the container starts, it discovers the local state and try to
> > retain the same partitions via the partition management APIs (i.e.
> > explicitly ask for a specific set of partitions to be assigned to the
> > container, instead of default to whatever broker decides to assign to the
> > container)
> >
> > {quote}
> > a question about the proposed API for instantiating a job in code (rather
> > than a properties file): when submitting a job to a cluster, is the idea
> > that the instantiation code runs on a client somewhere, which then pokes
> > the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code run on
> > each container that is part of the job (in which case, how does the job
> > submission to the cluster work)?
> > {quote}
> > I want to chime in here to propose the three layered model:
> > 1. Samza as a library. In this case, the client application should
> actually
> > implement the interface to submit its own job to a cluster, if the client
> > application chooses to run in a cluster. Samza as a library won't be
> poking
> > any endpoints on YARN/Mesos/AWS
> > 2. Samza as a process. In this case, a properties file could be desired
> > when starting the Samza process. Here again, Samza as a process should
> not
> > need to interact w/ any endpoints on YARN/Mesos/AWS.
> > 3. Samza as a service. In this case, we definitely need some job
> > configuration and a Samza implementation of interface to submit jobs and
> > their configuration to a cluster (i.e. YARN/Mesos/AWS, mostly the same as
> > we have today, except that we should just ask for resources to run Samza
> > containers and leave the partition management aside)
> >
> > Thanks!
> >
> > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <ma...@kleppmann.com>
> > wrote:
> >
> > > Hi all,
> > >
> > > Lots of good thoughts here.
> > >
> > > I agree with the general philosophy of tying Samza more firmly to
> Kafka.
> > > After I spent a while looking at integrating other message brokers
> (e.g.
> > > Kinesis) with SystemConsumer, I came to the conclusion that
> > SystemConsumer
> > > tacitly assumes a model so much like Kafka's that pretty much nobody
> but
> > > Kafka actually implements it. (Databus is perhaps an exception, but it
> > > isn't widely used outside of LinkedIn.) Thus, making Samza fully
> > dependent
> > > on Kafka acknowledges that the system-independence was never as real as
> > we
> > > perhaps made it out to be. The gains of code reuse are real.
> > >
> > > The idea of decoupling Samza from YARN has also always been appealing
> to
> > > me, for various reasons already mentioned in this thread. Although
> making
> > > Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems laudable,
> I
> > am
> > > a little concerned that it will restrict us to a lowest common
> > denominator.
> > > For example, would host affinity (SAMZA-617) still be possible? For
> jobs
> > > with large amounts of state, I think SAMZA-617 would be a big boon,
> since
> > > restoring state off the changelog on every single restart is painful,
> due
> > > to long recovery times. It would be a shame if the decoupling from YARN
> > > made host affinity impossible.
> > >
> > > Jay, a question about the proposed API for instantiating a job in code
> > > (rather than a properties file): when submitting a job to a cluster, is
> > the
> > > idea that the instantiation code runs on a client somewhere, which then
> > > pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code
> > run
> > > on each container that is part of the job (in which case, how does the
> > job
> > > submission to the cluster work)?
> > >
> > > I agree with Garry that it doesn't feel right to make a 1.0 release
> with
> > a
> > > plan for it to be immediately obsolete. So if this is going to happen,
> I
> > > think it would be more honest to stick with 0.* version numbers until
> the
> > > library-ified Samza has been implemented, is stable and widely used.
> > >
> > > Should the new Samza be a subproject of Kafka? There is precedent for
> > > tight coupling between different Apache projects (e.g. Curator and
> > > Zookeeper, or Slider and YARN), so I think remaining separate would be
> > ok.
> > > Even if Samza is fully dependent on Kafka, there is enough substance in
> > > Samza that it warrants being a separate project. An argument in favour
> of
> > > merging would be if we think Kafka has a much stronger "brand presence"
> > > than Samza; I'm ambivalent on that one. If the Kafka project is willing
> > to
> > > endorse Samza as the "official" way of doing stateful stream
> > > transformations, that would probably have much the same effect as
> > > re-branding Samza as "Kafka Stream Processors" or suchlike. Close
> > > collaboration between the two projects will be needed in any case.
> > >
> > > From a project management perspective, I guess the "new Samza" would
> have
> > > to be developed on a branch alongside ongoing maintenance of the
> current
> > > line of development? I think it would be important to continue
> supporting
> > > existing users, and provide a graceful migration path to the new
> version.
> > > Leaving the current versions unsupported and forcing people to rewrite
> > > their jobs would send a bad signal.
> > >
> > > Best,
> > > Martin
> > >
> > > On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Hey Garry,
> > > >
> > > > Yeah that's super frustrating. I'd be happy to chat more about this
> if
> > > > you'd be interested. I think Chris and I started with the idea of
> "what
> > > > would it take to make Samza a kick-ass ingestion tool" but ultimately
> > we
> > > > kind of came around to the idea that ingestion and transformation had
> > > > pretty different needs and coupling the two made things hard.
> > > >
> > > > For what it's worth I think copycat (KIP-26) actually will do what
> you
> > > are
> > > > looking for.
> > > >
> > > > With regard to your point about slider, I don't necessarily disagree.
> > > But I
> > > > think getting good YARN support is quite doable and I think we can
> make
> > > > that work well. I think the issue this proposal solves is that
> > > technically
> > > > it is pretty hard to support multiple cluster management systems the
> > way
> > > > things are now, you need to write an "app master" or "framework" for
> > each
> > > > and they are all a little different so testing is really hard. In the
> > > > absence of this we have been stuck with just YARN which has fantastic
> > > > penetration in the Hadoopy part of the org, but zero penetration
> > > elsewhere.
> > > > Given the huge amount of work being put in to slider, marathon, aws
> > > > tooling, not to mention the umpteen related packaging technologies
> > people
> > > > want to use (Docker, Kubernetes, various cloud-specific deploy tools,
> > > etc)
> > > > I really think it is important to get this right.
> > > >
> > > > -Jay
> > > >
> > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > > g.turkington@improvedigital.com> wrote:
> > > >
> > > >> Hi all,
> > > >>
> > > >> I think the question below re does Samza become a sub-project of
> Kafka
> > > >> highlights the broader point around migration. Chris mentions
> Samza's
> > > >> maturity is heading towards a v1 release but I'm not sure it feels
> > > right to
> > > >> launch a v1 then immediately plan to deprecate most of it.
> > > >>
> > > >> From a selfish perspective I have some guys who have started working
> > > with
> > > >> Samza and building some new consumers/producers was next up. Sounds
> > like
> > > >> that is absolutely not the direction to go. I need to look into the
> > KIP
> > > in
> > > >> more detail but for me the attractiveness of adding new Samza
> > > >> consumer/producers -- even if yes all they were doing was really
> > getting
> > > >> data into and out of Kafka --  was to avoid  having to worry about
> the
> > > >> lifecycle management of external clients. If there is a generic
> Kafka
> > > >> ingress/egress layer that I can plug a new connector into and have a
> > > lot of
> > > >> the heavy lifting re scale and reliability done for me then it gives
> > me
> > > all
> > > >> the pushing new consumers/producers would. If not then it
> complicates
> > my
> > > >> operational deployments.
> > > >>
> > > >> Which is similar to my other question with the proposal -- if we
> > build a
> > > >> fully available/stand-alone Samza plus the requisite shims to
> > integrate
> > > >> with Slider etc I suspect the former may be a lot more work than we
> > > think.
> > > >> We may make it much easier for a newcomer to get something running
> but
> > > >> having them step up and get a reliable production deployment may
> still
> > > >> dominate mailing list  traffic, if for different reasons than today.
> > > >>
> > > >> Don't get me wrong -- I'm comfortable with making the Samza
> dependency
> > > on
> > > >> Kafka much more explicit and I absolutely see the benefits  in the
> > > >> reduction of duplication and clashing terminologies/abstractions
> that
> > > >> Chris/Jay describe. Samza as a library would likely be a very nice
> > tool
> > > to
> > > >> add to the Kafka ecosystem. I just have the concerns above re the
> > > >> operational side.
> > > >>
> > > >> Garry
> > > >>
> > > >> -----Original Message-----
> > > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> > > >> Sent: 02 July 2015 12:56
> > > >> To: dev@samza.apache.org
> > > >> Subject: Re: Thoughts and obesrvations on Samza
> > > >>
> > > >> Very interesting thoughts.
> > > >> From outside, I have always perceived Samza as a computing layer
> over
> > > >> Kafka.
> > > >>
> > > >> The question, maybe a bit provocative, is "should Samza be a
> > sub-project
> > > >> of Kafka then?"
> > > >> Or does it make sense to keep it as a separate project with a
> separate
> > > >> governance?
> > > >>
> > > >> Cheers,
> > > >>
> > > >> --
> > > >> Gianmarco
> > > >>
> > > >> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> > > >>
> > > >>> Overall, I agree to couple with Kafka more tightly. Because Samza
> de
> > > >>> facto is based on Kafka, and it should leverage what Kafka has. At
> > the
> > > >>> same time, Kafka does not need to reinvent what Samza already has.
> I
> > > >>> also like the idea of separating the ingestion and transformation.
> > > >>>
> > > >>> But it is a little difficult for me to image how the Samza will
> look
> > > >> like.
> > > >>> And I feel Chris and Jay have a little difference in terms of how
> > > >>> Samza should look like.
> > > >>>
> > > >>> *** Will it look like what Jay's code shows (A client of Kakfa) ?
> And
> > > >>> user's application code calls this client?
> > > >>>
> > > >>> 1. If we make Samza be a library of Kafka (like what the code
> shows),
> > > >>> how do we implement auto-balance and fault-tolerance? Are they
> taken
> > > >>> care by the Kafka broker or other mechanism, such as "Samza worker"
> > > >>> (just make up the name) ?
> > > >>>
> > > >>> 2. What about other features, such as auto-scaling, shared state,
> > > >>> monitoring?
> > > >>>
> > > >>>
> > > >>> *** If we have Samza standalone, (is this what Chris suggests?)
> > > >>>
> > > >>> 1. we still need to ingest data from Kakfa and produce to it. Then
> it
> > > >>> becomes the same as what Samza looks like now, except it does not
> > rely
> > > >>> on Yarn anymore.
> > > >>>
> > > >>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
> > > >>> etc? Use Kafka code as the dependency?
> > > >>>
> > > >>>
> > > >>> Thanks,
> > > >>>
> > > >>> Fang, Yan
> > > >>> yanfang724@gmail.com
> > > >>>
> > > >>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
> > > >> wrote:
> > > >>>
> > > >>>> Read through the code example and it looks good to me. A few
> > > >>>> thoughts regarding deployment:
> > > >>>>
> > > >>>> Today Samza deploys as executable runnable like:
> > > >>>>
> > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
> > > >> --config-path=file://...
> > > >>>>
> > > >>>> And this proposal advocate for deploying Samza more as embedded
> > > >>>> libraries in user application code (ignoring the terminology since
> > > >>>> it is not the
> > > >>> same
> > > >>>> as the prototype code):
> > > >>>>
> > > >>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
> > > >>>> Thread(task); thread.start();
> > > >>>>
> > > >>>> I think both of these deployment modes are important for different
> > > >>>> types
> > > >>> of
> > > >>>> users. That said, I think making Samza purely standalone is still
> > > >>>> sufficient for either runnable or library modes.
> > > >>>>
> > > >>>> Guozhang
> > > >>>>
> > > >>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
> > wrote:
> > > >>>>
> > > >>>>> Looks like gmail mangled the code example, it was supposed to
> look
> > > >>>>> like
> > > >>>>> this:
> > > >>>>>
> > > >>>>> Properties props = new Properties();
> > > >>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
> > > >>>>> config = new StreamingConfig(props);
> > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
> > > >>>>> config.processor(ExampleStreamProcessor.class);
> > > >>>>> config.serialization(new StringSerializer(), new
> > > >>>>> StringDeserializer()); KafkaStreaming container = new
> > > >>>>> KafkaStreaming(config); container.run();
> > > >>>>>
> > > >>>>> -Jay
> > > >>>>>
> > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> > > >> wrote:
> > > >>>>>
> > > >>>>>> Hey guys,
> > > >>>>>>
> > > >>>>>> This came out of some conversations Chris and I were having
> > > >>>>>> around
> > > >>>>> whether
> > > >>>>>> it would make sense to use Samza as a kind of data ingestion
> > > >>> framework
> > > >>>>> for
> > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> > > >>>> combined
> > > >>>>>> with complaints around config and YARN and the discussion around
> > > >>>>>> how
> > > >>> to
> > > >>>>>> best do a standalone mode.
> > > >>>>>>
> > > >>>>>> So the thought experiment was, given that Samza was basically
> > > >>>>>> already totally Kafka specific, what if you just embraced that
> > > >>>>>> and turned it
> > > >>>> into
> > > >>>>>> something less like a heavyweight framework and more like a
> > > >>>>>> third
> > > >>> Kafka
> > > >>>>>> client--a kind of "producing consumer" with state management
> > > >>>> facilities.
> > > >>>>>> Basically a library. Instead of a complex stream processing
> > > >>>>>> framework
> > > >>>>> this
> > > >>>>>> would actually be a very simple thing, not much more complicated
> > > >>>>>> to
> > > >>> use
> > > >>>>> or
> > > >>>>>> operate than a Kafka consumer. As Chris said we thought about it
> > > >>>>>> a
> > > >>> lot
> > > >>>> of
> > > >>>>>> what Samza (and the other stream processing systems were doing)
> > > >>> seemed
> > > >>>>> like
> > > >>>>>> kind of a hangover from MapReduce.
> > > >>>>>>
> > > >>>>>> Of course you need to ingest/output data to and from the stream
> > > >>>>>> processing. But when we actually looked into how that would
> > > >>>>>> work,
> > > >>> Samza
> > > >>>>>> isn't really an ideal data ingestion framework for a bunch of
> > > >>> reasons.
> > > >>>> To
> > > >>>>>> really do that right you need a pretty different internal data
> > > >>>>>> model
> > > >>>> and
> > > >>>>>> set of apis. So what if you split them and had an api for Kafka
> > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > > >>>>>> transformation (Samza).
> > > >>>>>>
> > > >>>>>> This would also allow really embracing the same terminology and
> > > >>>>>> conventions. One complaint about the current state is that the
> > > >>>>>> two
> > > >>>>> systems
> > > >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
> > > >>>>> different
> > > >>>>>> config and monitoring systems means you kind of have to learn
> > > >>>>>> Kafka's
> > > >>>>> way,
> > > >>>>>> then learn Samza's slightly different way, then kind of
> > > >>>>>> understand
> > > >>> how
> > > >>>>> they
> > > >>>>>> map to each other, which having walked a few people through this
> > > >>>>>> is surprisingly tricky for folks to get.
> > > >>>>>>
> > > >>>>>> Since I have been spending a lot of time on airplanes I hacked
> > > >>>>>> up an ernest but still somewhat incomplete prototype of what
> > > >>>>>> this would
> > > >>> look
> > > >>>>>> like. This is just unceremoniously dumped into Kafka as it
> > > >>>>>> required a
> > > >>>> few
> > > >>>>>> changes to the new consumer. Here is the code:
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>
> > > >>>
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > >>> /apache/kafka/clients/streaming
> > > >>>>>>
> > > >>>>>> For the purpose of the prototype I just liberally renamed
> > > >>>>>> everything
> > > >>> to
> > > >>>>>> try to align it with Kafka with no regard for compatibility.
> > > >>>>>>
> > > >>>>>> To use this would be something like this:
> > > >>>>>> Properties props = new Properties();
> > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > >>>>>> StreamingConfig config = new
> > > >>> StreamingConfig(props);
> > > >>>>> config.subscribe("test-topic-1",
> > > >>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > > >>>>> config.serialization(new
> > > >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> > > >>>> container =
> > > >>>>>> new KafkaStreaming(config); container.run();
> > > >>>>>>
> > > >>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
> > > >>>>>> is basically StreamTask.
> > > >>>>>>
> > > >>>>>> So rather than putting all the class names in a file and then
> > > >>>>>> having
> > > >>>> the
> > > >>>>>> job assembled by reflection, you just instantiate the container
> > > >>>>>> programmatically. Work is balanced over however many instances
> > > >>>>>> of
> > > >>> this
> > > >>>>> are
> > > >>>>>> alive at any time (i.e. if an instance dies, new tasks are added
> > > >>>>>> to
> > > >>> the
> > > >>>>>> existing containers without shutting them down).
> > > >>>>>>
> > > >>>>>> We would provide some glue for running this stuff in YARN via
> > > >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> > > >>>>>> but from the
> > > >>>> point
> > > >>>>> of
> > > >>>>>> view of these frameworks these stream processing jobs are just
> > > >>>> stateless
> > > >>>>>> services that can come and go and expand and contract at will.
> > > >>>>>> There
> > > >>> is
> > > >>>>> no
> > > >>>>>> more custom scheduler.
> > > >>>>>>
> > > >>>>>> Here are some relevant details:
> > > >>>>>>
> > > >>>>>>   1. It is only ~1300 lines of code, it would get larger if we
> > > >>>>>>   productionized but not vastly larger. We really do get a ton
> > > >>>>>> of
> > > >>>>> leverage
> > > >>>>>>   out of Kafka.
> > > >>>>>>   2. Partition management is fully delegated to the new
> consumer.
> > > >>> This
> > > >>>>>>   is nice since now any partition management strategy available
> > > >>>>>> to
> > > >>>> Kafka
> > > >>>>>>   consumer is also available to Samza (and vice versa) and with
> > > >>>>>> the
> > > >>>>> exact
> > > >>>>>>   same configs.
> > > >>>>>>   3. It supports state as well as state reuse
> > > >>>>>>
> > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
> > > >>>>>>
> > > >>>>>> -Jay
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > >>>> criccomini@apache.org>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>>> Hey all,
> > > >>>>>>>
> > > >>>>>>> I have had some discussions with Samza engineers at LinkedIn
> > > >>>>>>> and
> > > >>>>> Confluent
> > > >>>>>>> and we came up with a few observations and would like to
> > > >>>>>>> propose
> > > >>> some
> > > >>>>>>> changes.
> > > >>>>>>>
> > > >>>>>>> We've observed some things that I want to call out about
> > > >>>>>>> Samza's
> > > >>>> design,
> > > >>>>>>> and I'd like to propose some changes.
> > > >>>>>>>
> > > >>>>>>> * Samza is dependent upon a dynamic deployment system.
> > > >>>>>>> * Samza is too pluggable.
> > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> > > >>>>>>> APIs
> > > >>> are
> > > >>>>>>> trying to solve a lot of the same problems.
> > > >>>>>>>
> > > >>>>>>> All three of these issues are related, but I'll address them in
> > > >>> order.
> > > >>>>>>>
> > > >>>>>>> Deployment
> > > >>>>>>>
> > > >>>>>>> Samza strongly depends on the use of a dynamic deployment
> > > >>>>>>> scheduler
> > > >>>> such
> > > >>>>>>> as
> > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> > > >>>>>>> there
> > > >>>> would
> > > >>>>>>> be
> > > >>>>>>> one or two winners in this area, and we could support them, and
> > > >>>>>>> the
> > > >>>> rest
> > > >>>>>>> would go away. In reality, there are many variations.
> > > >>>>>>> Furthermore,
> > > >>>> many
> > > >>>>>>> people still prefer to just start their processors like normal
> > > >>>>>>> Java processes, and use traditional deployment scripts such as
> > > >>>>>>> Fabric,
> > > >>>> Chef,
> > > >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> > > >>>>>>> Samza start-up process really painful for first time users.
> > > >>>>>>>
> > > >>>>>>> Dynamic deployment as a requirement was also a bit of a
> > > >>>>>>> mis-fire
> > > >>>> because
> > > >>>>>>> of
> > > >>>>>>> a fundamental misunderstanding between the nature of batch jobs
> > > >>>>>>> and
> > > >>>>> stream
> > > >>>>>>> processing jobs. Early on, we made conscious effort to favor
> > > >>>>>>> the
> > > >>>> Hadoop
> > > >>>>>>> (Map/Reduce) way of doing things, since it worked and was well
> > > >>>>> understood.
> > > >>>>>>> One thing that we missed was that batch jobs have a definite
> > > >>>> beginning,
> > > >>>>>>> and
> > > >>>>>>> end, and stream processing jobs don't (usually). This leads to
> > > >>>>>>> a
> > > >>> much
> > > >>>>>>> simpler scheduling problem for stream processors. You basically
> > > >>>>>>> just
> > > >>>>> need
> > > >>>>>>> to find a place to start the processor, and start it. The way
> > > >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> > > >>>>>>> being "full". We always
> > > >>>> add
> > > >>>>>>> more machines. The problem with coupling Samza with a scheduler
> > > >>>>>>> is
> > > >>>> that
> > > >>>>>>> Samza (as a framework) now has to handle deployment. This pulls
> > > >>>>>>> in a
> > > >>>>> bunch
> > > >>>>>>> of things such as configuration distribution (config stream),
> > > >>>>>>> shell
> > > >>>>> scrips
> > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff),
> etc.
> > > >>>>>>>
> > > >>>>>>> Another reason for requiring dynamic deployment was to support
> > > >>>>>>> data locality. If you want to have locality, you need to put
> > > >>>>>>> your
> > > >>>> processors
> > > >>>>>>> close to the data they're processing. Upon further
> > > >>>>>>> investigation,
> > > >>>>> though,
> > > >>>>>>> this feature is not that beneficial. There is some good
> > > >>>>>>> discussion
> > > >>>> about
> > > >>>>>>> some problems with it on SAMZA-335. Again, we took the
> > > >>>>>>> Map/Reduce
> > > >>>> path,
> > > >>>>>>> but
> > > >>>>>>> there are some fundamental differences between HDFS and Kafka.
> > > >>>>>>> HDFS
> > > >>>> has
> > > >>>>>>> blocks, while Kafka has partitions. This leads to less
> > > >>>>>>> optimization potential with stream processors on top of Kafka.
> > > >>>>>>>
> > > >>>>>>> This feature is also used as a crutch. Samza doesn't have any
> > > >>>>>>> built
> > > >>> in
> > > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> > > >>>>>>> deployment scheduling system to handle restarts when a
> > > >>>>>>> processor dies. This has
> > > >>>>> made
> > > >>>>>>> it very difficult to write a standalone Samza container
> > > >> (SAMZA-516).
> > > >>>>>>>
> > > >>>>>>> Pluggability
> > > >>>>>>>
> > > >>>>>>> In some cases pluggability is good, but I think that we've gone
> > > >>>>>>> too
> > > >>>> far
> > > >>>>>>> with it. Currently, Samza has:
> > > >>>>>>>
> > > >>>>>>> * Pluggable config.
> > > >>>>>>> * Pluggable metrics.
> > > >>>>>>> * Pluggable deployment systems.
> > > >>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
> > > >> etc).
> > > >>>>>>> * Pluggable serdes.
> > > >>>>>>> * Pluggable storage engines.
> > > >>>>>>> * Pluggable strategies for just about every component
> > > >>> (MessageChooser,
> > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > >>>>>>>
> > > >>>>>>> There's probably more that I've forgotten, as well. Some of
> > > >>>>>>> these
> > > >>> are
> > > >>>>>>> useful, but some have proven not to be. This all comes at a
> cost:
> > > >>>>>>> complexity. This complexity is making it harder for our users
> > > >>>>>>> to
> > > >>> pick
> > > >>>> up
> > > >>>>>>> and use Samza out of the box. It also makes it difficult for
> > > >>>>>>> Samza developers to reason about what the characteristics of
> > > >>>>>>> the container (since the characteristics change depending on
> > > >>>>>>> which plugins are use).
> > > >>>>>>>
> > > >>>>>>> The issues with pluggability are most visible in the System
> APIs.
> > > >>> What
> > > >>>>>>> Samza really requires to be functional is Kafka as its
> > > >>>>>>> transport
> > > >>>> layer.
> > > >>>>>>> But
> > > >>>>>>> we've conflated two unrelated use cases into one API:
> > > >>>>>>>
> > > >>>>>>> 1. Get data into/out of Kafka.
> > > >>>>>>> 2. Process the data in Kafka.
> > > >>>>>>>
> > > >>>>>>> The current System API supports both of these use cases. The
> > > >>>>>>> problem
> > > >>>> is,
> > > >>>>>>> we
> > > >>>>>>> actually want different features for each use case. By papering
> > > >>>>>>> over
> > > >>>>> these
> > > >>>>>>> two use cases, and providing a single API, we've introduced a
> > > >>>>>>> ton of
> > > >>>>> leaky
> > > >>>>>>> abstractions.
> > > >>>>>>>
> > > >>>>>>> For example, what we'd really like in (2) is to have
> > > >>>>>>> monotonically increasing longs for offsets (like Kafka). This
> > > >>>>>>> would be at odds
> > > >>> with
> > > >>>>> (1),
> > > >>>>>>> though, since different systems have different
> > > >>>>> SCNs/Offsets/UUIDs/vectors.
> > > >>>>>>> There was discussion both on the mailing list and the SQL JIRAs
> > > >>> about
> > > >>>>> the
> > > >>>>>>> need for this.
> > > >>>>>>>
> > > >>>>>>> The same thing holds true for replayability. Kafka allows us to
> > > >>> rewind
> > > >>>>>>> when
> > > >>>>>>> we have a failure. Many other systems don't. In some cases,
> > > >>>>>>> systems
> > > >>>>> return
> > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
> > > >>>>>>> they
> > > >>>> have
> > > >>>>> no
> > > >>>>>>> offsets.
> > > >>>>>>>
> > > >>>>>>> Partitioning is another example. Kafka supports partitioning,
> > > >>>>>>> but
> > > >>> many
> > > >>>>>>> systems don't. We model this by having a single partition for
> > > >>>>>>> those systems. Still, other systems model partitioning
> > > >> differently (e.g.
> > > >>>>>>> Kinesis).
> > > >>>>>>>
> > > >>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
> > > >>>>>>> system-agnostic way is almost impossible. As is modeling
> > > >>>>>>> metadata
> > > >>> for
> > > >>>>> the
> > > >>>>>>> system (replication factor, partitions, location, etc). The
> > > >>>>>>> list
> > > >>> goes
> > > >>>>> on.
> > > >>>>>>>
> > > >>>>>>> Duplicate work
> > > >>>>>>>
> > > >>>>>>> At the time that we began writing Samza, Kafka's consumer and
> > > >>> producer
> > > >>>>>>> APIs
> > > >>>>>>> had a relatively weak feature set. On the consumer-side, you
> > > >>>>>>> had two
> > > >>>>>>> options: use the high level consumer, or the simple consumer.
> > > >>>>>>> The
> > > >>>>> problem
> > > >>>>>>> with the high-level consumer was that it controlled your
> > > >>>>>>> offsets, partition assignments, and the order in which you
> > > >>>>>>> received messages. The
> > > >>> problem
> > > >>>>>>> with
> > > >>>>>>> the simple consumer is that it's not simple. It's basic. You
> > > >>>>>>> end up
> > > >>>>> having
> > > >>>>>>> to handle a lot of really low-level stuff that you shouldn't.
> > > >>>>>>> We
> > > >>>> spent a
> > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
> > > >>>>>>> also allows us to support some cool features:
> > > >>>>>>>
> > > >>>>>>> * Per-partition message ordering and prioritization.
> > > >>>>>>> * Tight control over partition assignment to support joins,
> > > >>>>>>> global
> > > >>>> state
> > > >>>>>>> (if we want to implement it :)), etc.
> > > >>>>>>> * Tight control over offset checkpointing.
> > > >>>>>>>
> > > >>>>>>> What we didn't realize at the time is that these features
> > > >>>>>>> should
> > > >>>>> actually
> > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> > > >>>> processors)
> > > >>>>>>> end up wanting to do things like joins and partition
> > > >>>>>>> assignment. The
> > > >>>>> Kafka
> > > >>>>>>> community has come to the same conclusion. They're adding a ton
> > > >>>>>>> of upgrades into their new Kafka consumer implementation. To a
> > > >>>>>>> large extent,
> > > >>> it's
> > > >>>>>>> duplicate work to what we've already done in Samza.
> > > >>>>>>>
> > > >>>>>>> On top of this, Kafka ended up taking a very similar approach
> > > >>>>>>> to
> > > >>>> Samza's
> > > >>>>>>> KafkaCheckpointManager implementation for handling offset
> > > >>>> checkpointing.
> > > >>>>>>> Like Samza, Kafka's new offset management feature stores offset
> > > >>>>>>> checkpoints in a topic, and allows you to fetch them from the
> > > >>>>>>> broker.
> > > >>>>>>>
> > > >>>>>>> A lot of this seems like a waste, since we could have shared
> > > >>>>>>> the
> > > >>> work
> > > >>>> if
> > > >>>>>>> it
> > > >>>>>>> had been done in Kafka from the get-go.
> > > >>>>>>>
> > > >>>>>>> Vision
> > > >>>>>>>
> > > >>>>>>> All of this leads me to a rather radical proposal. Samza is
> > > >>> relatively
> > > >>>>>>> stable at this point. I'd venture to say that we're near a 1.0
> > > >>>> release.
> > > >>>>>>> I'd
> > > >>>>>>> like to propose that we take what we've learned, and begin
> > > >>>>>>> thinking
> > > >>>>> about
> > > >>>>>>> Samza beyond 1.0. What would we change if we were starting from
> > > >>>> scratch?
> > > >>>>>>> My
> > > >>>>>>> proposal is to:
> > > >>>>>>>
> > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > > >>>>>>> processors, and eliminate all direct dependences on YARN,
> Mesos,
> > > >> etc.
> > > >>>>>>> 2. Make a definitive call to support only Kafka as the stream
> > > >>>> processing
> > > >>>>>>> layer.
> > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> > > >>>>>>> config
> > > >>>>> systems,
> > > >>>>>>> and simply use Kafka's instead.
> > > >>>>>>>
> > > >>>>>>> This would fix all of the issues that I outlined above. It
> > > >>>>>>> should
> > > >>> also
> > > >>>>>>> shrink the Samza code base pretty dramatically. Supporting only
> > > >>>>>>> a standalone container will allow Samza to be executed on YARN
> > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> > > >>>>>>> in-house
> > > >>>>> deployment
> > > >>>>>>> systems. This should make life a lot easier for new users.
> > > >>>>>>> Imagine
> > > >>>>> having
> > > >>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
> > > >>>> traffic
> > > >>>>>>> will be pretty dramatic.
> > > >>>>>>>
> > > >>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
> > > >>> everyone
> > > >>>>>>> that
> > > >>>>>>> I'm aware of is using Samza with Kafka. We basically require it
> > > >>>> already
> > > >>>>> in
> > > >>>>>>> order for most features to work. Those that are using other
> > > >>>>>>> systems
> > > >>>> are
> > > >>>>>>> generally using it for ingest into Kafka (1), and then they do
> > > >>>>>>> the processing on top. There is already discussion (
> > > >>>>>>>
> > > >>>>>
> > > >>>>
> > > >>>
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > >>> 767
> > > >>>>>>> )
> > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > > >>>>>>>
> > > >>>>>>> Once we make the call to couple with Kafka, we can leverage a
> > > >>>>>>> ton of
> > > >>>>> their
> > > >>>>>>> ecosystem. We no longer have to maintain our own config,
> > > >>>>>>> metrics,
> > > >>> etc.
> > > >>>>> We
> > > >>>>>>> can all share the same libraries, and make them better. This
> > > >>>>>>> will
> > > >>> also
> > > >>>>>>> allow us to share the consumer/producer APIs, and will let us
> > > >>> leverage
> > > >>>>>>> their offset management and partition management, rather than
> > > >>>>>>> having
> > > >>>> our
> > > >>>>>>> own. All of the coordinator stream code would go away, as would
> > > >>>>>>> most
> > > >>>> of
> > > >>>>>>> the
> > > >>>>>>> YARN AppMaster code. We'd probably have to push some partition
> > > >>>>> management
> > > >>>>>>> features into the Kafka broker, but they're already moving in
> > > >>>>>>> that direction with the new consumer API. The features we have
> > > >>>>>>> for
> > > >>>> partition
> > > >>>>>>> assignment aren't unique to Samza, and seem like they should be
> > > >>>>>>> in
> > > >>>> Kafka
> > > >>>>>>> anyway. There will always be some niche usages which will
> > > >>>>>>> require
> > > >>>> extra
> > > >>>>>>> care and hence full control over partition assignments much
> > > >>>>>>> like the
> > > >>>>> Kafka
> > > >>>>>>> low level consumer api. These would continue to be supported.
> > > >>>>>>>
> > > >>>>>>> These items will be good for the Samza community. They'll make
> > > >>>>>>> Samza easier to use, and make it easier for developers to add
> > > >>>>>>> new features.
> > > >>>>>>>
> > > >>>>>>> Obviously this is a fairly large (and somewhat backwards
> > > >>> incompatible
> > > >>>>>>> change). If we choose to go this route, it's important that we
> > > >>> openly
> > > >>>>>>> communicate how we're going to provide a migration path from
> > > >>>>>>> the
> > > >>>>> existing
> > > >>>>>>> APIs to the new ones (if we make incompatible changes). I think
> > > >>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
> > > >>>>>>> existing StreamTask implementations to continue running on the
> > > >> new container.
> > > >>>>> It's
> > > >>>>>>> also important that we openly communicate about timing, and
> > > >>>>>>> stages
> > > >>> of
> > > >>>>> the
> > > >>>>>>> migration.
> > > >>>>>>>
> > > >>>>>>> If you made it this far, I'm sure you have opinions. :) Please
> > > >>>>>>> send
> > > >>>> your
> > > >>>>>>> thoughts and feedback.
> > > >>>>>>>
> > > >>>>>>> Cheers,
> > > >>>>>>> Chris
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>> --
> > > >>>> -- Guozhang
> > > >>>>
> > > >>>
> > > >>
> > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@gmail.com>.
Hey Yi/Martin,

With respect to state reuse: Yeah that is a better explanation of what I
was trying to say. That prototype actually includes state reuse (each task
checkpoints it's offset for the various partitions during the commit and
uses that to determine if the state is valid and can be reused on restart).
So it is just a matter of a framework being able to express a preference
for the recently used hosts--those that can will get the optimization and
those that can't won't.

Yi, for the three layers I think we agree, but I may be unclear on what you
are actually proposing. I think the question is whether for the "as a
service" you are proposing actually trying to build some layer over
YARN/Mesos/AWS that abstracts these away? Of course, there is nothing that
requires such an abstraction to run stream processing as a service on top
of (say) YARN. Can you be more specific about what would be in-scope for
that?

I am generally skeptical of a layer that abstracts away these frameworks
for two reasons. First I think they are quite different and each is
advancing fairly rapidly so abstracting over them may be hard. Second I
think the layer of abstraction doesn't really help the user. If you think
about it you tend to just adopt one of the frameworks (say Mesos/Marathon)
and you learn the tooling associated with that and use it for all your
different systems. Building an abstraction over that doesn't really give
you any flexibility: you can't swap in another framework because all your
other stuff is tied to (say) Mesos/Marathon, and the layer of indirection
obscures the interface the user is already familar with from other systems.
But I think I may actually be misunderstanding your proposal...

-Jay

On Mon, Jul 6, 2015 at 11:30 AM, Yi Pan <ni...@gmail.com> wrote:

> Hi, Martin,
>
> Great to hear your voice! I will just try to focus on your questions
> regarding to "w/o YARN" part.
>
> {quote}
> For example, would host affinity (SAMZA-617) still be possible?
> {quote}
> It would be possible if we separate the job execution/process launching
> from the partition assignment among all Samza containers. Host-affinity
> could follow the model below:
> a. We still keeps a container-host mapping as in SAMZA-617
> b. It would be a best-effort approach try to get the same set of host from
> the job execution framework (e.g. as we did w/ YARN in SamzaAppMaster code.
> If Slider/Marathon comes up w/ this feature, we can migrate to that as
> well).
> c. When the container starts, it discovers the local state and try to
> retain the same partitions via the partition management APIs (i.e.
> explicitly ask for a specific set of partitions to be assigned to the
> container, instead of default to whatever broker decides to assign to the
> container)
>
> {quote}
> a question about the proposed API for instantiating a job in code (rather
> than a properties file): when submitting a job to a cluster, is the idea
> that the instantiation code runs on a client somewhere, which then pokes
> the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code run on
> each container that is part of the job (in which case, how does the job
> submission to the cluster work)?
> {quote}
> I want to chime in here to propose the three layered model:
> 1. Samza as a library. In this case, the client application should actually
> implement the interface to submit its own job to a cluster, if the client
> application chooses to run in a cluster. Samza as a library won't be poking
> any endpoints on YARN/Mesos/AWS
> 2. Samza as a process. In this case, a properties file could be desired
> when starting the Samza process. Here again, Samza as a process should not
> need to interact w/ any endpoints on YARN/Mesos/AWS.
> 3. Samza as a service. In this case, we definitely need some job
> configuration and a Samza implementation of interface to submit jobs and
> their configuration to a cluster (i.e. YARN/Mesos/AWS, mostly the same as
> we have today, except that we should just ask for resources to run Samza
> containers and leave the partition management aside)
>
> Thanks!
>
> On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <ma...@kleppmann.com>
> wrote:
>
> > Hi all,
> >
> > Lots of good thoughts here.
> >
> > I agree with the general philosophy of tying Samza more firmly to Kafka.
> > After I spent a while looking at integrating other message brokers (e.g.
> > Kinesis) with SystemConsumer, I came to the conclusion that
> SystemConsumer
> > tacitly assumes a model so much like Kafka's that pretty much nobody but
> > Kafka actually implements it. (Databus is perhaps an exception, but it
> > isn't widely used outside of LinkedIn.) Thus, making Samza fully
> dependent
> > on Kafka acknowledges that the system-independence was never as real as
> we
> > perhaps made it out to be. The gains of code reuse are real.
> >
> > The idea of decoupling Samza from YARN has also always been appealing to
> > me, for various reasons already mentioned in this thread. Although making
> > Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems laudable, I
> am
> > a little concerned that it will restrict us to a lowest common
> denominator.
> > For example, would host affinity (SAMZA-617) still be possible? For jobs
> > with large amounts of state, I think SAMZA-617 would be a big boon, since
> > restoring state off the changelog on every single restart is painful, due
> > to long recovery times. It would be a shame if the decoupling from YARN
> > made host affinity impossible.
> >
> > Jay, a question about the proposed API for instantiating a job in code
> > (rather than a properties file): when submitting a job to a cluster, is
> the
> > idea that the instantiation code runs on a client somewhere, which then
> > pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code
> run
> > on each container that is part of the job (in which case, how does the
> job
> > submission to the cluster work)?
> >
> > I agree with Garry that it doesn't feel right to make a 1.0 release with
> a
> > plan for it to be immediately obsolete. So if this is going to happen, I
> > think it would be more honest to stick with 0.* version numbers until the
> > library-ified Samza has been implemented, is stable and widely used.
> >
> > Should the new Samza be a subproject of Kafka? There is precedent for
> > tight coupling between different Apache projects (e.g. Curator and
> > Zookeeper, or Slider and YARN), so I think remaining separate would be
> ok.
> > Even if Samza is fully dependent on Kafka, there is enough substance in
> > Samza that it warrants being a separate project. An argument in favour of
> > merging would be if we think Kafka has a much stronger "brand presence"
> > than Samza; I'm ambivalent on that one. If the Kafka project is willing
> to
> > endorse Samza as the "official" way of doing stateful stream
> > transformations, that would probably have much the same effect as
> > re-branding Samza as "Kafka Stream Processors" or suchlike. Close
> > collaboration between the two projects will be needed in any case.
> >
> > From a project management perspective, I guess the "new Samza" would have
> > to be developed on a branch alongside ongoing maintenance of the current
> > line of development? I think it would be important to continue supporting
> > existing users, and provide a graceful migration path to the new version.
> > Leaving the current versions unsupported and forcing people to rewrite
> > their jobs would send a bad signal.
> >
> > Best,
> > Martin
> >
> > On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Hey Garry,
> > >
> > > Yeah that's super frustrating. I'd be happy to chat more about this if
> > > you'd be interested. I think Chris and I started with the idea of "what
> > > would it take to make Samza a kick-ass ingestion tool" but ultimately
> we
> > > kind of came around to the idea that ingestion and transformation had
> > > pretty different needs and coupling the two made things hard.
> > >
> > > For what it's worth I think copycat (KIP-26) actually will do what you
> > are
> > > looking for.
> > >
> > > With regard to your point about slider, I don't necessarily disagree.
> > But I
> > > think getting good YARN support is quite doable and I think we can make
> > > that work well. I think the issue this proposal solves is that
> > technically
> > > it is pretty hard to support multiple cluster management systems the
> way
> > > things are now, you need to write an "app master" or "framework" for
> each
> > > and they are all a little different so testing is really hard. In the
> > > absence of this we have been stuck with just YARN which has fantastic
> > > penetration in the Hadoopy part of the org, but zero penetration
> > elsewhere.
> > > Given the huge amount of work being put in to slider, marathon, aws
> > > tooling, not to mention the umpteen related packaging technologies
> people
> > > want to use (Docker, Kubernetes, various cloud-specific deploy tools,
> > etc)
> > > I really think it is important to get this right.
> > >
> > > -Jay
> > >
> > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > g.turkington@improvedigital.com> wrote:
> > >
> > >> Hi all,
> > >>
> > >> I think the question below re does Samza become a sub-project of Kafka
> > >> highlights the broader point around migration. Chris mentions Samza's
> > >> maturity is heading towards a v1 release but I'm not sure it feels
> > right to
> > >> launch a v1 then immediately plan to deprecate most of it.
> > >>
> > >> From a selfish perspective I have some guys who have started working
> > with
> > >> Samza and building some new consumers/producers was next up. Sounds
> like
> > >> that is absolutely not the direction to go. I need to look into the
> KIP
> > in
> > >> more detail but for me the attractiveness of adding new Samza
> > >> consumer/producers -- even if yes all they were doing was really
> getting
> > >> data into and out of Kafka --  was to avoid  having to worry about the
> > >> lifecycle management of external clients. If there is a generic Kafka
> > >> ingress/egress layer that I can plug a new connector into and have a
> > lot of
> > >> the heavy lifting re scale and reliability done for me then it gives
> me
> > all
> > >> the pushing new consumers/producers would. If not then it complicates
> my
> > >> operational deployments.
> > >>
> > >> Which is similar to my other question with the proposal -- if we
> build a
> > >> fully available/stand-alone Samza plus the requisite shims to
> integrate
> > >> with Slider etc I suspect the former may be a lot more work than we
> > think.
> > >> We may make it much easier for a newcomer to get something running but
> > >> having them step up and get a reliable production deployment may still
> > >> dominate mailing list  traffic, if for different reasons than today.
> > >>
> > >> Don't get me wrong -- I'm comfortable with making the Samza dependency
> > on
> > >> Kafka much more explicit and I absolutely see the benefits  in the
> > >> reduction of duplication and clashing terminologies/abstractions that
> > >> Chris/Jay describe. Samza as a library would likely be a very nice
> tool
> > to
> > >> add to the Kafka ecosystem. I just have the concerns above re the
> > >> operational side.
> > >>
> > >> Garry
> > >>
> > >> -----Original Message-----
> > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> > >> Sent: 02 July 2015 12:56
> > >> To: dev@samza.apache.org
> > >> Subject: Re: Thoughts and obesrvations on Samza
> > >>
> > >> Very interesting thoughts.
> > >> From outside, I have always perceived Samza as a computing layer over
> > >> Kafka.
> > >>
> > >> The question, maybe a bit provocative, is "should Samza be a
> sub-project
> > >> of Kafka then?"
> > >> Or does it make sense to keep it as a separate project with a separate
> > >> governance?
> > >>
> > >> Cheers,
> > >>
> > >> --
> > >> Gianmarco
> > >>
> > >> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> > >>
> > >>> Overall, I agree to couple with Kafka more tightly. Because Samza de
> > >>> facto is based on Kafka, and it should leverage what Kafka has. At
> the
> > >>> same time, Kafka does not need to reinvent what Samza already has. I
> > >>> also like the idea of separating the ingestion and transformation.
> > >>>
> > >>> But it is a little difficult for me to image how the Samza will look
> > >> like.
> > >>> And I feel Chris and Jay have a little difference in terms of how
> > >>> Samza should look like.
> > >>>
> > >>> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
> > >>> user's application code calls this client?
> > >>>
> > >>> 1. If we make Samza be a library of Kafka (like what the code shows),
> > >>> how do we implement auto-balance and fault-tolerance? Are they taken
> > >>> care by the Kafka broker or other mechanism, such as "Samza worker"
> > >>> (just make up the name) ?
> > >>>
> > >>> 2. What about other features, such as auto-scaling, shared state,
> > >>> monitoring?
> > >>>
> > >>>
> > >>> *** If we have Samza standalone, (is this what Chris suggests?)
> > >>>
> > >>> 1. we still need to ingest data from Kakfa and produce to it. Then it
> > >>> becomes the same as what Samza looks like now, except it does not
> rely
> > >>> on Yarn anymore.
> > >>>
> > >>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
> > >>> etc? Use Kafka code as the dependency?
> > >>>
> > >>>
> > >>> Thanks,
> > >>>
> > >>> Fang, Yan
> > >>> yanfang724@gmail.com
> > >>>
> > >>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
> > >> wrote:
> > >>>
> > >>>> Read through the code example and it looks good to me. A few
> > >>>> thoughts regarding deployment:
> > >>>>
> > >>>> Today Samza deploys as executable runnable like:
> > >>>>
> > >>>> deploy/samza/bin/run-job.sh --config-factory=...
> > >> --config-path=file://...
> > >>>>
> > >>>> And this proposal advocate for deploying Samza more as embedded
> > >>>> libraries in user application code (ignoring the terminology since
> > >>>> it is not the
> > >>> same
> > >>>> as the prototype code):
> > >>>>
> > >>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
> > >>>> Thread(task); thread.start();
> > >>>>
> > >>>> I think both of these deployment modes are important for different
> > >>>> types
> > >>> of
> > >>>> users. That said, I think making Samza purely standalone is still
> > >>>> sufficient for either runnable or library modes.
> > >>>>
> > >>>> Guozhang
> > >>>>
> > >>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > >>>>
> > >>>>> Looks like gmail mangled the code example, it was supposed to look
> > >>>>> like
> > >>>>> this:
> > >>>>>
> > >>>>> Properties props = new Properties();
> > >>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
> > >>>>> config = new StreamingConfig(props);
> > >>>>> config.subscribe("test-topic-1", "test-topic-2");
> > >>>>> config.processor(ExampleStreamProcessor.class);
> > >>>>> config.serialization(new StringSerializer(), new
> > >>>>> StringDeserializer()); KafkaStreaming container = new
> > >>>>> KafkaStreaming(config); container.run();
> > >>>>>
> > >>>>> -Jay
> > >>>>>
> > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> > >> wrote:
> > >>>>>
> > >>>>>> Hey guys,
> > >>>>>>
> > >>>>>> This came out of some conversations Chris and I were having
> > >>>>>> around
> > >>>>> whether
> > >>>>>> it would make sense to use Samza as a kind of data ingestion
> > >>> framework
> > >>>>> for
> > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> > >>>> combined
> > >>>>>> with complaints around config and YARN and the discussion around
> > >>>>>> how
> > >>> to
> > >>>>>> best do a standalone mode.
> > >>>>>>
> > >>>>>> So the thought experiment was, given that Samza was basically
> > >>>>>> already totally Kafka specific, what if you just embraced that
> > >>>>>> and turned it
> > >>>> into
> > >>>>>> something less like a heavyweight framework and more like a
> > >>>>>> third
> > >>> Kafka
> > >>>>>> client--a kind of "producing consumer" with state management
> > >>>> facilities.
> > >>>>>> Basically a library. Instead of a complex stream processing
> > >>>>>> framework
> > >>>>> this
> > >>>>>> would actually be a very simple thing, not much more complicated
> > >>>>>> to
> > >>> use
> > >>>>> or
> > >>>>>> operate than a Kafka consumer. As Chris said we thought about it
> > >>>>>> a
> > >>> lot
> > >>>> of
> > >>>>>> what Samza (and the other stream processing systems were doing)
> > >>> seemed
> > >>>>> like
> > >>>>>> kind of a hangover from MapReduce.
> > >>>>>>
> > >>>>>> Of course you need to ingest/output data to and from the stream
> > >>>>>> processing. But when we actually looked into how that would
> > >>>>>> work,
> > >>> Samza
> > >>>>>> isn't really an ideal data ingestion framework for a bunch of
> > >>> reasons.
> > >>>> To
> > >>>>>> really do that right you need a pretty different internal data
> > >>>>>> model
> > >>>> and
> > >>>>>> set of apis. So what if you split them and had an api for Kafka
> > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > >>>>>> transformation (Samza).
> > >>>>>>
> > >>>>>> This would also allow really embracing the same terminology and
> > >>>>>> conventions. One complaint about the current state is that the
> > >>>>>> two
> > >>>>> systems
> > >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
> > >>>>> different
> > >>>>>> config and monitoring systems means you kind of have to learn
> > >>>>>> Kafka's
> > >>>>> way,
> > >>>>>> then learn Samza's slightly different way, then kind of
> > >>>>>> understand
> > >>> how
> > >>>>> they
> > >>>>>> map to each other, which having walked a few people through this
> > >>>>>> is surprisingly tricky for folks to get.
> > >>>>>>
> > >>>>>> Since I have been spending a lot of time on airplanes I hacked
> > >>>>>> up an ernest but still somewhat incomplete prototype of what
> > >>>>>> this would
> > >>> look
> > >>>>>> like. This is just unceremoniously dumped into Kafka as it
> > >>>>>> required a
> > >>>> few
> > >>>>>> changes to the new consumer. Here is the code:
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>
> > >>>
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >>> /apache/kafka/clients/streaming
> > >>>>>>
> > >>>>>> For the purpose of the prototype I just liberally renamed
> > >>>>>> everything
> > >>> to
> > >>>>>> try to align it with Kafka with no regard for compatibility.
> > >>>>>>
> > >>>>>> To use this would be something like this:
> > >>>>>> Properties props = new Properties();
> > >>>>>> props.put("bootstrap.servers", "localhost:4242");
> > >>>>>> StreamingConfig config = new
> > >>> StreamingConfig(props);
> > >>>>> config.subscribe("test-topic-1",
> > >>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > >>>>> config.serialization(new
> > >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> > >>>> container =
> > >>>>>> new KafkaStreaming(config); container.run();
> > >>>>>>
> > >>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
> > >>>>>> is basically StreamTask.
> > >>>>>>
> > >>>>>> So rather than putting all the class names in a file and then
> > >>>>>> having
> > >>>> the
> > >>>>>> job assembled by reflection, you just instantiate the container
> > >>>>>> programmatically. Work is balanced over however many instances
> > >>>>>> of
> > >>> this
> > >>>>> are
> > >>>>>> alive at any time (i.e. if an instance dies, new tasks are added
> > >>>>>> to
> > >>> the
> > >>>>>> existing containers without shutting them down).
> > >>>>>>
> > >>>>>> We would provide some glue for running this stuff in YARN via
> > >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> > >>>>>> but from the
> > >>>> point
> > >>>>> of
> > >>>>>> view of these frameworks these stream processing jobs are just
> > >>>> stateless
> > >>>>>> services that can come and go and expand and contract at will.
> > >>>>>> There
> > >>> is
> > >>>>> no
> > >>>>>> more custom scheduler.
> > >>>>>>
> > >>>>>> Here are some relevant details:
> > >>>>>>
> > >>>>>>   1. It is only ~1300 lines of code, it would get larger if we
> > >>>>>>   productionized but not vastly larger. We really do get a ton
> > >>>>>> of
> > >>>>> leverage
> > >>>>>>   out of Kafka.
> > >>>>>>   2. Partition management is fully delegated to the new consumer.
> > >>> This
> > >>>>>>   is nice since now any partition management strategy available
> > >>>>>> to
> > >>>> Kafka
> > >>>>>>   consumer is also available to Samza (and vice versa) and with
> > >>>>>> the
> > >>>>> exact
> > >>>>>>   same configs.
> > >>>>>>   3. It supports state as well as state reuse
> > >>>>>>
> > >>>>>> Anyhow take a look, hopefully it is thought provoking.
> > >>>>>>
> > >>>>>> -Jay
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > >>>> criccomini@apache.org>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>> Hey all,
> > >>>>>>>
> > >>>>>>> I have had some discussions with Samza engineers at LinkedIn
> > >>>>>>> and
> > >>>>> Confluent
> > >>>>>>> and we came up with a few observations and would like to
> > >>>>>>> propose
> > >>> some
> > >>>>>>> changes.
> > >>>>>>>
> > >>>>>>> We've observed some things that I want to call out about
> > >>>>>>> Samza's
> > >>>> design,
> > >>>>>>> and I'd like to propose some changes.
> > >>>>>>>
> > >>>>>>> * Samza is dependent upon a dynamic deployment system.
> > >>>>>>> * Samza is too pluggable.
> > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> > >>>>>>> APIs
> > >>> are
> > >>>>>>> trying to solve a lot of the same problems.
> > >>>>>>>
> > >>>>>>> All three of these issues are related, but I'll address them in
> > >>> order.
> > >>>>>>>
> > >>>>>>> Deployment
> > >>>>>>>
> > >>>>>>> Samza strongly depends on the use of a dynamic deployment
> > >>>>>>> scheduler
> > >>>> such
> > >>>>>>> as
> > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> > >>>>>>> there
> > >>>> would
> > >>>>>>> be
> > >>>>>>> one or two winners in this area, and we could support them, and
> > >>>>>>> the
> > >>>> rest
> > >>>>>>> would go away. In reality, there are many variations.
> > >>>>>>> Furthermore,
> > >>>> many
> > >>>>>>> people still prefer to just start their processors like normal
> > >>>>>>> Java processes, and use traditional deployment scripts such as
> > >>>>>>> Fabric,
> > >>>> Chef,
> > >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> > >>>>>>> Samza start-up process really painful for first time users.
> > >>>>>>>
> > >>>>>>> Dynamic deployment as a requirement was also a bit of a
> > >>>>>>> mis-fire
> > >>>> because
> > >>>>>>> of
> > >>>>>>> a fundamental misunderstanding between the nature of batch jobs
> > >>>>>>> and
> > >>>>> stream
> > >>>>>>> processing jobs. Early on, we made conscious effort to favor
> > >>>>>>> the
> > >>>> Hadoop
> > >>>>>>> (Map/Reduce) way of doing things, since it worked and was well
> > >>>>> understood.
> > >>>>>>> One thing that we missed was that batch jobs have a definite
> > >>>> beginning,
> > >>>>>>> and
> > >>>>>>> end, and stream processing jobs don't (usually). This leads to
> > >>>>>>> a
> > >>> much
> > >>>>>>> simpler scheduling problem for stream processors. You basically
> > >>>>>>> just
> > >>>>> need
> > >>>>>>> to find a place to start the processor, and start it. The way
> > >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> > >>>>>>> being "full". We always
> > >>>> add
> > >>>>>>> more machines. The problem with coupling Samza with a scheduler
> > >>>>>>> is
> > >>>> that
> > >>>>>>> Samza (as a framework) now has to handle deployment. This pulls
> > >>>>>>> in a
> > >>>>> bunch
> > >>>>>>> of things such as configuration distribution (config stream),
> > >>>>>>> shell
> > >>>>> scrips
> > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> > >>>>>>>
> > >>>>>>> Another reason for requiring dynamic deployment was to support
> > >>>>>>> data locality. If you want to have locality, you need to put
> > >>>>>>> your
> > >>>> processors
> > >>>>>>> close to the data they're processing. Upon further
> > >>>>>>> investigation,
> > >>>>> though,
> > >>>>>>> this feature is not that beneficial. There is some good
> > >>>>>>> discussion
> > >>>> about
> > >>>>>>> some problems with it on SAMZA-335. Again, we took the
> > >>>>>>> Map/Reduce
> > >>>> path,
> > >>>>>>> but
> > >>>>>>> there are some fundamental differences between HDFS and Kafka.
> > >>>>>>> HDFS
> > >>>> has
> > >>>>>>> blocks, while Kafka has partitions. This leads to less
> > >>>>>>> optimization potential with stream processors on top of Kafka.
> > >>>>>>>
> > >>>>>>> This feature is also used as a crutch. Samza doesn't have any
> > >>>>>>> built
> > >>> in
> > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> > >>>>>>> deployment scheduling system to handle restarts when a
> > >>>>>>> processor dies. This has
> > >>>>> made
> > >>>>>>> it very difficult to write a standalone Samza container
> > >> (SAMZA-516).
> > >>>>>>>
> > >>>>>>> Pluggability
> > >>>>>>>
> > >>>>>>> In some cases pluggability is good, but I think that we've gone
> > >>>>>>> too
> > >>>> far
> > >>>>>>> with it. Currently, Samza has:
> > >>>>>>>
> > >>>>>>> * Pluggable config.
> > >>>>>>> * Pluggable metrics.
> > >>>>>>> * Pluggable deployment systems.
> > >>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
> > >> etc).
> > >>>>>>> * Pluggable serdes.
> > >>>>>>> * Pluggable storage engines.
> > >>>>>>> * Pluggable strategies for just about every component
> > >>> (MessageChooser,
> > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > >>>>>>>
> > >>>>>>> There's probably more that I've forgotten, as well. Some of
> > >>>>>>> these
> > >>> are
> > >>>>>>> useful, but some have proven not to be. This all comes at a cost:
> > >>>>>>> complexity. This complexity is making it harder for our users
> > >>>>>>> to
> > >>> pick
> > >>>> up
> > >>>>>>> and use Samza out of the box. It also makes it difficult for
> > >>>>>>> Samza developers to reason about what the characteristics of
> > >>>>>>> the container (since the characteristics change depending on
> > >>>>>>> which plugins are use).
> > >>>>>>>
> > >>>>>>> The issues with pluggability are most visible in the System APIs.
> > >>> What
> > >>>>>>> Samza really requires to be functional is Kafka as its
> > >>>>>>> transport
> > >>>> layer.
> > >>>>>>> But
> > >>>>>>> we've conflated two unrelated use cases into one API:
> > >>>>>>>
> > >>>>>>> 1. Get data into/out of Kafka.
> > >>>>>>> 2. Process the data in Kafka.
> > >>>>>>>
> > >>>>>>> The current System API supports both of these use cases. The
> > >>>>>>> problem
> > >>>> is,
> > >>>>>>> we
> > >>>>>>> actually want different features for each use case. By papering
> > >>>>>>> over
> > >>>>> these
> > >>>>>>> two use cases, and providing a single API, we've introduced a
> > >>>>>>> ton of
> > >>>>> leaky
> > >>>>>>> abstractions.
> > >>>>>>>
> > >>>>>>> For example, what we'd really like in (2) is to have
> > >>>>>>> monotonically increasing longs for offsets (like Kafka). This
> > >>>>>>> would be at odds
> > >>> with
> > >>>>> (1),
> > >>>>>>> though, since different systems have different
> > >>>>> SCNs/Offsets/UUIDs/vectors.
> > >>>>>>> There was discussion both on the mailing list and the SQL JIRAs
> > >>> about
> > >>>>> the
> > >>>>>>> need for this.
> > >>>>>>>
> > >>>>>>> The same thing holds true for replayability. Kafka allows us to
> > >>> rewind
> > >>>>>>> when
> > >>>>>>> we have a failure. Many other systems don't. In some cases,
> > >>>>>>> systems
> > >>>>> return
> > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
> > >>>>>>> they
> > >>>> have
> > >>>>> no
> > >>>>>>> offsets.
> > >>>>>>>
> > >>>>>>> Partitioning is another example. Kafka supports partitioning,
> > >>>>>>> but
> > >>> many
> > >>>>>>> systems don't. We model this by having a single partition for
> > >>>>>>> those systems. Still, other systems model partitioning
> > >> differently (e.g.
> > >>>>>>> Kinesis).
> > >>>>>>>
> > >>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
> > >>>>>>> system-agnostic way is almost impossible. As is modeling
> > >>>>>>> metadata
> > >>> for
> > >>>>> the
> > >>>>>>> system (replication factor, partitions, location, etc). The
> > >>>>>>> list
> > >>> goes
> > >>>>> on.
> > >>>>>>>
> > >>>>>>> Duplicate work
> > >>>>>>>
> > >>>>>>> At the time that we began writing Samza, Kafka's consumer and
> > >>> producer
> > >>>>>>> APIs
> > >>>>>>> had a relatively weak feature set. On the consumer-side, you
> > >>>>>>> had two
> > >>>>>>> options: use the high level consumer, or the simple consumer.
> > >>>>>>> The
> > >>>>> problem
> > >>>>>>> with the high-level consumer was that it controlled your
> > >>>>>>> offsets, partition assignments, and the order in which you
> > >>>>>>> received messages. The
> > >>> problem
> > >>>>>>> with
> > >>>>>>> the simple consumer is that it's not simple. It's basic. You
> > >>>>>>> end up
> > >>>>> having
> > >>>>>>> to handle a lot of really low-level stuff that you shouldn't.
> > >>>>>>> We
> > >>>> spent a
> > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
> > >>>>>>> also allows us to support some cool features:
> > >>>>>>>
> > >>>>>>> * Per-partition message ordering and prioritization.
> > >>>>>>> * Tight control over partition assignment to support joins,
> > >>>>>>> global
> > >>>> state
> > >>>>>>> (if we want to implement it :)), etc.
> > >>>>>>> * Tight control over offset checkpointing.
> > >>>>>>>
> > >>>>>>> What we didn't realize at the time is that these features
> > >>>>>>> should
> > >>>>> actually
> > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> > >>>> processors)
> > >>>>>>> end up wanting to do things like joins and partition
> > >>>>>>> assignment. The
> > >>>>> Kafka
> > >>>>>>> community has come to the same conclusion. They're adding a ton
> > >>>>>>> of upgrades into their new Kafka consumer implementation. To a
> > >>>>>>> large extent,
> > >>> it's
> > >>>>>>> duplicate work to what we've already done in Samza.
> > >>>>>>>
> > >>>>>>> On top of this, Kafka ended up taking a very similar approach
> > >>>>>>> to
> > >>>> Samza's
> > >>>>>>> KafkaCheckpointManager implementation for handling offset
> > >>>> checkpointing.
> > >>>>>>> Like Samza, Kafka's new offset management feature stores offset
> > >>>>>>> checkpoints in a topic, and allows you to fetch them from the
> > >>>>>>> broker.
> > >>>>>>>
> > >>>>>>> A lot of this seems like a waste, since we could have shared
> > >>>>>>> the
> > >>> work
> > >>>> if
> > >>>>>>> it
> > >>>>>>> had been done in Kafka from the get-go.
> > >>>>>>>
> > >>>>>>> Vision
> > >>>>>>>
> > >>>>>>> All of this leads me to a rather radical proposal. Samza is
> > >>> relatively
> > >>>>>>> stable at this point. I'd venture to say that we're near a 1.0
> > >>>> release.
> > >>>>>>> I'd
> > >>>>>>> like to propose that we take what we've learned, and begin
> > >>>>>>> thinking
> > >>>>> about
> > >>>>>>> Samza beyond 1.0. What would we change if we were starting from
> > >>>> scratch?
> > >>>>>>> My
> > >>>>>>> proposal is to:
> > >>>>>>>
> > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > >>>>>>> processors, and eliminate all direct dependences on YARN, Mesos,
> > >> etc.
> > >>>>>>> 2. Make a definitive call to support only Kafka as the stream
> > >>>> processing
> > >>>>>>> layer.
> > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> > >>>>>>> config
> > >>>>> systems,
> > >>>>>>> and simply use Kafka's instead.
> > >>>>>>>
> > >>>>>>> This would fix all of the issues that I outlined above. It
> > >>>>>>> should
> > >>> also
> > >>>>>>> shrink the Samza code base pretty dramatically. Supporting only
> > >>>>>>> a standalone container will allow Samza to be executed on YARN
> > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> > >>>>>>> in-house
> > >>>>> deployment
> > >>>>>>> systems. This should make life a lot easier for new users.
> > >>>>>>> Imagine
> > >>>>> having
> > >>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
> > >>>> traffic
> > >>>>>>> will be pretty dramatic.
> > >>>>>>>
> > >>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
> > >>> everyone
> > >>>>>>> that
> > >>>>>>> I'm aware of is using Samza with Kafka. We basically require it
> > >>>> already
> > >>>>> in
> > >>>>>>> order for most features to work. Those that are using other
> > >>>>>>> systems
> > >>>> are
> > >>>>>>> generally using it for ingest into Kafka (1), and then they do
> > >>>>>>> the processing on top. There is already discussion (
> > >>>>>>>
> > >>>>>
> > >>>>
> > >>>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >>> 767
> > >>>>>>> )
> > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > >>>>>>>
> > >>>>>>> Once we make the call to couple with Kafka, we can leverage a
> > >>>>>>> ton of
> > >>>>> their
> > >>>>>>> ecosystem. We no longer have to maintain our own config,
> > >>>>>>> metrics,
> > >>> etc.
> > >>>>> We
> > >>>>>>> can all share the same libraries, and make them better. This
> > >>>>>>> will
> > >>> also
> > >>>>>>> allow us to share the consumer/producer APIs, and will let us
> > >>> leverage
> > >>>>>>> their offset management and partition management, rather than
> > >>>>>>> having
> > >>>> our
> > >>>>>>> own. All of the coordinator stream code would go away, as would
> > >>>>>>> most
> > >>>> of
> > >>>>>>> the
> > >>>>>>> YARN AppMaster code. We'd probably have to push some partition
> > >>>>> management
> > >>>>>>> features into the Kafka broker, but they're already moving in
> > >>>>>>> that direction with the new consumer API. The features we have
> > >>>>>>> for
> > >>>> partition
> > >>>>>>> assignment aren't unique to Samza, and seem like they should be
> > >>>>>>> in
> > >>>> Kafka
> > >>>>>>> anyway. There will always be some niche usages which will
> > >>>>>>> require
> > >>>> extra
> > >>>>>>> care and hence full control over partition assignments much
> > >>>>>>> like the
> > >>>>> Kafka
> > >>>>>>> low level consumer api. These would continue to be supported.
> > >>>>>>>
> > >>>>>>> These items will be good for the Samza community. They'll make
> > >>>>>>> Samza easier to use, and make it easier for developers to add
> > >>>>>>> new features.
> > >>>>>>>
> > >>>>>>> Obviously this is a fairly large (and somewhat backwards
> > >>> incompatible
> > >>>>>>> change). If we choose to go this route, it's important that we
> > >>> openly
> > >>>>>>> communicate how we're going to provide a migration path from
> > >>>>>>> the
> > >>>>> existing
> > >>>>>>> APIs to the new ones (if we make incompatible changes). I think
> > >>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
> > >>>>>>> existing StreamTask implementations to continue running on the
> > >> new container.
> > >>>>> It's
> > >>>>>>> also important that we openly communicate about timing, and
> > >>>>>>> stages
> > >>> of
> > >>>>> the
> > >>>>>>> migration.
> > >>>>>>>
> > >>>>>>> If you made it this far, I'm sure you have opinions. :) Please
> > >>>>>>> send
> > >>>> your
> > >>>>>>> thoughts and feedback.
> > >>>>>>>
> > >>>>>>> Cheers,
> > >>>>>>> Chris
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>
> > >>>>
> > >>>>
> > >>>>
> > >>>> --
> > >>>> -- Guozhang
> > >>>>
> > >>>
> > >>
> >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
Hi, Martin,

Great to hear your voice! I will just try to focus on your questions
regarding to "w/o YARN" part.

{quote}
For example, would host affinity (SAMZA-617) still be possible?
{quote}
It would be possible if we separate the job execution/process launching
from the partition assignment among all Samza containers. Host-affinity
could follow the model below:
a. We still keeps a container-host mapping as in SAMZA-617
b. It would be a best-effort approach try to get the same set of host from
the job execution framework (e.g. as we did w/ YARN in SamzaAppMaster code.
If Slider/Marathon comes up w/ this feature, we can migrate to that as
well).
c. When the container starts, it discovers the local state and try to
retain the same partitions via the partition management APIs (i.e.
explicitly ask for a specific set of partitions to be assigned to the
container, instead of default to whatever broker decides to assign to the
container)

{quote}
a question about the proposed API for instantiating a job in code (rather
than a properties file): when submitting a job to a cluster, is the idea
that the instantiation code runs on a client somewhere, which then pokes
the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code run on
each container that is part of the job (in which case, how does the job
submission to the cluster work)?
{quote}
I want to chime in here to propose the three layered model:
1. Samza as a library. In this case, the client application should actually
implement the interface to submit its own job to a cluster, if the client
application chooses to run in a cluster. Samza as a library won't be poking
any endpoints on YARN/Mesos/AWS
2. Samza as a process. In this case, a properties file could be desired
when starting the Samza process. Here again, Samza as a process should not
need to interact w/ any endpoints on YARN/Mesos/AWS.
3. Samza as a service. In this case, we definitely need some job
configuration and a Samza implementation of interface to submit jobs and
their configuration to a cluster (i.e. YARN/Mesos/AWS, mostly the same as
we have today, except that we should just ask for resources to run Samza
containers and leave the partition management aside)

Thanks!

On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <ma...@kleppmann.com>
wrote:

> Hi all,
>
> Lots of good thoughts here.
>
> I agree with the general philosophy of tying Samza more firmly to Kafka.
> After I spent a while looking at integrating other message brokers (e.g.
> Kinesis) with SystemConsumer, I came to the conclusion that SystemConsumer
> tacitly assumes a model so much like Kafka's that pretty much nobody but
> Kafka actually implements it. (Databus is perhaps an exception, but it
> isn't widely used outside of LinkedIn.) Thus, making Samza fully dependent
> on Kafka acknowledges that the system-independence was never as real as we
> perhaps made it out to be. The gains of code reuse are real.
>
> The idea of decoupling Samza from YARN has also always been appealing to
> me, for various reasons already mentioned in this thread. Although making
> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems laudable, I am
> a little concerned that it will restrict us to a lowest common denominator.
> For example, would host affinity (SAMZA-617) still be possible? For jobs
> with large amounts of state, I think SAMZA-617 would be a big boon, since
> restoring state off the changelog on every single restart is painful, due
> to long recovery times. It would be a shame if the decoupling from YARN
> made host affinity impossible.
>
> Jay, a question about the proposed API for instantiating a job in code
> (rather than a properties file): when submitting a job to a cluster, is the
> idea that the instantiation code runs on a client somewhere, which then
> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code run
> on each container that is part of the job (in which case, how does the job
> submission to the cluster work)?
>
> I agree with Garry that it doesn't feel right to make a 1.0 release with a
> plan for it to be immediately obsolete. So if this is going to happen, I
> think it would be more honest to stick with 0.* version numbers until the
> library-ified Samza has been implemented, is stable and widely used.
>
> Should the new Samza be a subproject of Kafka? There is precedent for
> tight coupling between different Apache projects (e.g. Curator and
> Zookeeper, or Slider and YARN), so I think remaining separate would be ok.
> Even if Samza is fully dependent on Kafka, there is enough substance in
> Samza that it warrants being a separate project. An argument in favour of
> merging would be if we think Kafka has a much stronger "brand presence"
> than Samza; I'm ambivalent on that one. If the Kafka project is willing to
> endorse Samza as the "official" way of doing stateful stream
> transformations, that would probably have much the same effect as
> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
> collaboration between the two projects will be needed in any case.
>
> From a project management perspective, I guess the "new Samza" would have
> to be developed on a branch alongside ongoing maintenance of the current
> line of development? I think it would be important to continue supporting
> existing users, and provide a graceful migration path to the new version.
> Leaving the current versions unsupported and forcing people to rewrite
> their jobs would send a bad signal.
>
> Best,
> Martin
>
> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey Garry,
> >
> > Yeah that's super frustrating. I'd be happy to chat more about this if
> > you'd be interested. I think Chris and I started with the idea of "what
> > would it take to make Samza a kick-ass ingestion tool" but ultimately we
> > kind of came around to the idea that ingestion and transformation had
> > pretty different needs and coupling the two made things hard.
> >
> > For what it's worth I think copycat (KIP-26) actually will do what you
> are
> > looking for.
> >
> > With regard to your point about slider, I don't necessarily disagree.
> But I
> > think getting good YARN support is quite doable and I think we can make
> > that work well. I think the issue this proposal solves is that
> technically
> > it is pretty hard to support multiple cluster management systems the way
> > things are now, you need to write an "app master" or "framework" for each
> > and they are all a little different so testing is really hard. In the
> > absence of this we have been stuck with just YARN which has fantastic
> > penetration in the Hadoopy part of the org, but zero penetration
> elsewhere.
> > Given the huge amount of work being put in to slider, marathon, aws
> > tooling, not to mention the umpteen related packaging technologies people
> > want to use (Docker, Kubernetes, various cloud-specific deploy tools,
> etc)
> > I really think it is important to get this right.
> >
> > -Jay
> >
> > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > g.turkington@improvedigital.com> wrote:
> >
> >> Hi all,
> >>
> >> I think the question below re does Samza become a sub-project of Kafka
> >> highlights the broader point around migration. Chris mentions Samza's
> >> maturity is heading towards a v1 release but I'm not sure it feels
> right to
> >> launch a v1 then immediately plan to deprecate most of it.
> >>
> >> From a selfish perspective I have some guys who have started working
> with
> >> Samza and building some new consumers/producers was next up. Sounds like
> >> that is absolutely not the direction to go. I need to look into the KIP
> in
> >> more detail but for me the attractiveness of adding new Samza
> >> consumer/producers -- even if yes all they were doing was really getting
> >> data into and out of Kafka --  was to avoid  having to worry about the
> >> lifecycle management of external clients. If there is a generic Kafka
> >> ingress/egress layer that I can plug a new connector into and have a
> lot of
> >> the heavy lifting re scale and reliability done for me then it gives me
> all
> >> the pushing new consumers/producers would. If not then it complicates my
> >> operational deployments.
> >>
> >> Which is similar to my other question with the proposal -- if we build a
> >> fully available/stand-alone Samza plus the requisite shims to integrate
> >> with Slider etc I suspect the former may be a lot more work than we
> think.
> >> We may make it much easier for a newcomer to get something running but
> >> having them step up and get a reliable production deployment may still
> >> dominate mailing list  traffic, if for different reasons than today.
> >>
> >> Don't get me wrong -- I'm comfortable with making the Samza dependency
> on
> >> Kafka much more explicit and I absolutely see the benefits  in the
> >> reduction of duplication and clashing terminologies/abstractions that
> >> Chris/Jay describe. Samza as a library would likely be a very nice tool
> to
> >> add to the Kafka ecosystem. I just have the concerns above re the
> >> operational side.
> >>
> >> Garry
> >>
> >> -----Original Message-----
> >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> >> Sent: 02 July 2015 12:56
> >> To: dev@samza.apache.org
> >> Subject: Re: Thoughts and obesrvations on Samza
> >>
> >> Very interesting thoughts.
> >> From outside, I have always perceived Samza as a computing layer over
> >> Kafka.
> >>
> >> The question, maybe a bit provocative, is "should Samza be a sub-project
> >> of Kafka then?"
> >> Or does it make sense to keep it as a separate project with a separate
> >> governance?
> >>
> >> Cheers,
> >>
> >> --
> >> Gianmarco
> >>
> >> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> >>
> >>> Overall, I agree to couple with Kafka more tightly. Because Samza de
> >>> facto is based on Kafka, and it should leverage what Kafka has. At the
> >>> same time, Kafka does not need to reinvent what Samza already has. I
> >>> also like the idea of separating the ingestion and transformation.
> >>>
> >>> But it is a little difficult for me to image how the Samza will look
> >> like.
> >>> And I feel Chris and Jay have a little difference in terms of how
> >>> Samza should look like.
> >>>
> >>> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
> >>> user's application code calls this client?
> >>>
> >>> 1. If we make Samza be a library of Kafka (like what the code shows),
> >>> how do we implement auto-balance and fault-tolerance? Are they taken
> >>> care by the Kafka broker or other mechanism, such as "Samza worker"
> >>> (just make up the name) ?
> >>>
> >>> 2. What about other features, such as auto-scaling, shared state,
> >>> monitoring?
> >>>
> >>>
> >>> *** If we have Samza standalone, (is this what Chris suggests?)
> >>>
> >>> 1. we still need to ingest data from Kakfa and produce to it. Then it
> >>> becomes the same as what Samza looks like now, except it does not rely
> >>> on Yarn anymore.
> >>>
> >>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
> >>> etc? Use Kafka code as the dependency?
> >>>
> >>>
> >>> Thanks,
> >>>
> >>> Fang, Yan
> >>> yanfang724@gmail.com
> >>>
> >>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >>>
> >>>> Read through the code example and it looks good to me. A few
> >>>> thoughts regarding deployment:
> >>>>
> >>>> Today Samza deploys as executable runnable like:
> >>>>
> >>>> deploy/samza/bin/run-job.sh --config-factory=...
> >> --config-path=file://...
> >>>>
> >>>> And this proposal advocate for deploying Samza more as embedded
> >>>> libraries in user application code (ignoring the terminology since
> >>>> it is not the
> >>> same
> >>>> as the prototype code):
> >>>>
> >>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
> >>>> Thread(task); thread.start();
> >>>>
> >>>> I think both of these deployment modes are important for different
> >>>> types
> >>> of
> >>>> users. That said, I think making Samza purely standalone is still
> >>>> sufficient for either runnable or library modes.
> >>>>
> >>>> Guozhang
> >>>>
> >>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> >>>>
> >>>>> Looks like gmail mangled the code example, it was supposed to look
> >>>>> like
> >>>>> this:
> >>>>>
> >>>>> Properties props = new Properties();
> >>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
> >>>>> config = new StreamingConfig(props);
> >>>>> config.subscribe("test-topic-1", "test-topic-2");
> >>>>> config.processor(ExampleStreamProcessor.class);
> >>>>> config.serialization(new StringSerializer(), new
> >>>>> StringDeserializer()); KafkaStreaming container = new
> >>>>> KafkaStreaming(config); container.run();
> >>>>>
> >>>>> -Jay
> >>>>>
> >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> >> wrote:
> >>>>>
> >>>>>> Hey guys,
> >>>>>>
> >>>>>> This came out of some conversations Chris and I were having
> >>>>>> around
> >>>>> whether
> >>>>>> it would make sense to use Samza as a kind of data ingestion
> >>> framework
> >>>>> for
> >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> >>>> combined
> >>>>>> with complaints around config and YARN and the discussion around
> >>>>>> how
> >>> to
> >>>>>> best do a standalone mode.
> >>>>>>
> >>>>>> So the thought experiment was, given that Samza was basically
> >>>>>> already totally Kafka specific, what if you just embraced that
> >>>>>> and turned it
> >>>> into
> >>>>>> something less like a heavyweight framework and more like a
> >>>>>> third
> >>> Kafka
> >>>>>> client--a kind of "producing consumer" with state management
> >>>> facilities.
> >>>>>> Basically a library. Instead of a complex stream processing
> >>>>>> framework
> >>>>> this
> >>>>>> would actually be a very simple thing, not much more complicated
> >>>>>> to
> >>> use
> >>>>> or
> >>>>>> operate than a Kafka consumer. As Chris said we thought about it
> >>>>>> a
> >>> lot
> >>>> of
> >>>>>> what Samza (and the other stream processing systems were doing)
> >>> seemed
> >>>>> like
> >>>>>> kind of a hangover from MapReduce.
> >>>>>>
> >>>>>> Of course you need to ingest/output data to and from the stream
> >>>>>> processing. But when we actually looked into how that would
> >>>>>> work,
> >>> Samza
> >>>>>> isn't really an ideal data ingestion framework for a bunch of
> >>> reasons.
> >>>> To
> >>>>>> really do that right you need a pretty different internal data
> >>>>>> model
> >>>> and
> >>>>>> set of apis. So what if you split them and had an api for Kafka
> >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> >>>>>> transformation (Samza).
> >>>>>>
> >>>>>> This would also allow really embracing the same terminology and
> >>>>>> conventions. One complaint about the current state is that the
> >>>>>> two
> >>>>> systems
> >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
> >>>>> different
> >>>>>> config and monitoring systems means you kind of have to learn
> >>>>>> Kafka's
> >>>>> way,
> >>>>>> then learn Samza's slightly different way, then kind of
> >>>>>> understand
> >>> how
> >>>>> they
> >>>>>> map to each other, which having walked a few people through this
> >>>>>> is surprisingly tricky for folks to get.
> >>>>>>
> >>>>>> Since I have been spending a lot of time on airplanes I hacked
> >>>>>> up an ernest but still somewhat incomplete prototype of what
> >>>>>> this would
> >>> look
> >>>>>> like. This is just unceremoniously dumped into Kafka as it
> >>>>>> required a
> >>>> few
> >>>>>> changes to the new consumer. Here is the code:
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >>> /apache/kafka/clients/streaming
> >>>>>>
> >>>>>> For the purpose of the prototype I just liberally renamed
> >>>>>> everything
> >>> to
> >>>>>> try to align it with Kafka with no regard for compatibility.
> >>>>>>
> >>>>>> To use this would be something like this:
> >>>>>> Properties props = new Properties();
> >>>>>> props.put("bootstrap.servers", "localhost:4242");
> >>>>>> StreamingConfig config = new
> >>> StreamingConfig(props);
> >>>>> config.subscribe("test-topic-1",
> >>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> >>>>> config.serialization(new
> >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> >>>> container =
> >>>>>> new KafkaStreaming(config); container.run();
> >>>>>>
> >>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
> >>>>>> is basically StreamTask.
> >>>>>>
> >>>>>> So rather than putting all the class names in a file and then
> >>>>>> having
> >>>> the
> >>>>>> job assembled by reflection, you just instantiate the container
> >>>>>> programmatically. Work is balanced over however many instances
> >>>>>> of
> >>> this
> >>>>> are
> >>>>>> alive at any time (i.e. if an instance dies, new tasks are added
> >>>>>> to
> >>> the
> >>>>>> existing containers without shutting them down).
> >>>>>>
> >>>>>> We would provide some glue for running this stuff in YARN via
> >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> >>>>>> but from the
> >>>> point
> >>>>> of
> >>>>>> view of these frameworks these stream processing jobs are just
> >>>> stateless
> >>>>>> services that can come and go and expand and contract at will.
> >>>>>> There
> >>> is
> >>>>> no
> >>>>>> more custom scheduler.
> >>>>>>
> >>>>>> Here are some relevant details:
> >>>>>>
> >>>>>>   1. It is only ~1300 lines of code, it would get larger if we
> >>>>>>   productionized but not vastly larger. We really do get a ton
> >>>>>> of
> >>>>> leverage
> >>>>>>   out of Kafka.
> >>>>>>   2. Partition management is fully delegated to the new consumer.
> >>> This
> >>>>>>   is nice since now any partition management strategy available
> >>>>>> to
> >>>> Kafka
> >>>>>>   consumer is also available to Samza (and vice versa) and with
> >>>>>> the
> >>>>> exact
> >>>>>>   same configs.
> >>>>>>   3. It supports state as well as state reuse
> >>>>>>
> >>>>>> Anyhow take a look, hopefully it is thought provoking.
> >>>>>>
> >>>>>> -Jay
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> >>>> criccomini@apache.org>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Hey all,
> >>>>>>>
> >>>>>>> I have had some discussions with Samza engineers at LinkedIn
> >>>>>>> and
> >>>>> Confluent
> >>>>>>> and we came up with a few observations and would like to
> >>>>>>> propose
> >>> some
> >>>>>>> changes.
> >>>>>>>
> >>>>>>> We've observed some things that I want to call out about
> >>>>>>> Samza's
> >>>> design,
> >>>>>>> and I'd like to propose some changes.
> >>>>>>>
> >>>>>>> * Samza is dependent upon a dynamic deployment system.
> >>>>>>> * Samza is too pluggable.
> >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> >>>>>>> APIs
> >>> are
> >>>>>>> trying to solve a lot of the same problems.
> >>>>>>>
> >>>>>>> All three of these issues are related, but I'll address them in
> >>> order.
> >>>>>>>
> >>>>>>> Deployment
> >>>>>>>
> >>>>>>> Samza strongly depends on the use of a dynamic deployment
> >>>>>>> scheduler
> >>>> such
> >>>>>>> as
> >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> >>>>>>> there
> >>>> would
> >>>>>>> be
> >>>>>>> one or two winners in this area, and we could support them, and
> >>>>>>> the
> >>>> rest
> >>>>>>> would go away. In reality, there are many variations.
> >>>>>>> Furthermore,
> >>>> many
> >>>>>>> people still prefer to just start their processors like normal
> >>>>>>> Java processes, and use traditional deployment scripts such as
> >>>>>>> Fabric,
> >>>> Chef,
> >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> >>>>>>> Samza start-up process really painful for first time users.
> >>>>>>>
> >>>>>>> Dynamic deployment as a requirement was also a bit of a
> >>>>>>> mis-fire
> >>>> because
> >>>>>>> of
> >>>>>>> a fundamental misunderstanding between the nature of batch jobs
> >>>>>>> and
> >>>>> stream
> >>>>>>> processing jobs. Early on, we made conscious effort to favor
> >>>>>>> the
> >>>> Hadoop
> >>>>>>> (Map/Reduce) way of doing things, since it worked and was well
> >>>>> understood.
> >>>>>>> One thing that we missed was that batch jobs have a definite
> >>>> beginning,
> >>>>>>> and
> >>>>>>> end, and stream processing jobs don't (usually). This leads to
> >>>>>>> a
> >>> much
> >>>>>>> simpler scheduling problem for stream processors. You basically
> >>>>>>> just
> >>>>> need
> >>>>>>> to find a place to start the processor, and start it. The way
> >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> >>>>>>> being "full". We always
> >>>> add
> >>>>>>> more machines. The problem with coupling Samza with a scheduler
> >>>>>>> is
> >>>> that
> >>>>>>> Samza (as a framework) now has to handle deployment. This pulls
> >>>>>>> in a
> >>>>> bunch
> >>>>>>> of things such as configuration distribution (config stream),
> >>>>>>> shell
> >>>>> scrips
> >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> >>>>>>>
> >>>>>>> Another reason for requiring dynamic deployment was to support
> >>>>>>> data locality. If you want to have locality, you need to put
> >>>>>>> your
> >>>> processors
> >>>>>>> close to the data they're processing. Upon further
> >>>>>>> investigation,
> >>>>> though,
> >>>>>>> this feature is not that beneficial. There is some good
> >>>>>>> discussion
> >>>> about
> >>>>>>> some problems with it on SAMZA-335. Again, we took the
> >>>>>>> Map/Reduce
> >>>> path,
> >>>>>>> but
> >>>>>>> there are some fundamental differences between HDFS and Kafka.
> >>>>>>> HDFS
> >>>> has
> >>>>>>> blocks, while Kafka has partitions. This leads to less
> >>>>>>> optimization potential with stream processors on top of Kafka.
> >>>>>>>
> >>>>>>> This feature is also used as a crutch. Samza doesn't have any
> >>>>>>> built
> >>> in
> >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> >>>>>>> deployment scheduling system to handle restarts when a
> >>>>>>> processor dies. This has
> >>>>> made
> >>>>>>> it very difficult to write a standalone Samza container
> >> (SAMZA-516).
> >>>>>>>
> >>>>>>> Pluggability
> >>>>>>>
> >>>>>>> In some cases pluggability is good, but I think that we've gone
> >>>>>>> too
> >>>> far
> >>>>>>> with it. Currently, Samza has:
> >>>>>>>
> >>>>>>> * Pluggable config.
> >>>>>>> * Pluggable metrics.
> >>>>>>> * Pluggable deployment systems.
> >>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
> >> etc).
> >>>>>>> * Pluggable serdes.
> >>>>>>> * Pluggable storage engines.
> >>>>>>> * Pluggable strategies for just about every component
> >>> (MessageChooser,
> >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> >>>>>>>
> >>>>>>> There's probably more that I've forgotten, as well. Some of
> >>>>>>> these
> >>> are
> >>>>>>> useful, but some have proven not to be. This all comes at a cost:
> >>>>>>> complexity. This complexity is making it harder for our users
> >>>>>>> to
> >>> pick
> >>>> up
> >>>>>>> and use Samza out of the box. It also makes it difficult for
> >>>>>>> Samza developers to reason about what the characteristics of
> >>>>>>> the container (since the characteristics change depending on
> >>>>>>> which plugins are use).
> >>>>>>>
> >>>>>>> The issues with pluggability are most visible in the System APIs.
> >>> What
> >>>>>>> Samza really requires to be functional is Kafka as its
> >>>>>>> transport
> >>>> layer.
> >>>>>>> But
> >>>>>>> we've conflated two unrelated use cases into one API:
> >>>>>>>
> >>>>>>> 1. Get data into/out of Kafka.
> >>>>>>> 2. Process the data in Kafka.
> >>>>>>>
> >>>>>>> The current System API supports both of these use cases. The
> >>>>>>> problem
> >>>> is,
> >>>>>>> we
> >>>>>>> actually want different features for each use case. By papering
> >>>>>>> over
> >>>>> these
> >>>>>>> two use cases, and providing a single API, we've introduced a
> >>>>>>> ton of
> >>>>> leaky
> >>>>>>> abstractions.
> >>>>>>>
> >>>>>>> For example, what we'd really like in (2) is to have
> >>>>>>> monotonically increasing longs for offsets (like Kafka). This
> >>>>>>> would be at odds
> >>> with
> >>>>> (1),
> >>>>>>> though, since different systems have different
> >>>>> SCNs/Offsets/UUIDs/vectors.
> >>>>>>> There was discussion both on the mailing list and the SQL JIRAs
> >>> about
> >>>>> the
> >>>>>>> need for this.
> >>>>>>>
> >>>>>>> The same thing holds true for replayability. Kafka allows us to
> >>> rewind
> >>>>>>> when
> >>>>>>> we have a failure. Many other systems don't. In some cases,
> >>>>>>> systems
> >>>>> return
> >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
> >>>>>>> they
> >>>> have
> >>>>> no
> >>>>>>> offsets.
> >>>>>>>
> >>>>>>> Partitioning is another example. Kafka supports partitioning,
> >>>>>>> but
> >>> many
> >>>>>>> systems don't. We model this by having a single partition for
> >>>>>>> those systems. Still, other systems model partitioning
> >> differently (e.g.
> >>>>>>> Kinesis).
> >>>>>>>
> >>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
> >>>>>>> system-agnostic way is almost impossible. As is modeling
> >>>>>>> metadata
> >>> for
> >>>>> the
> >>>>>>> system (replication factor, partitions, location, etc). The
> >>>>>>> list
> >>> goes
> >>>>> on.
> >>>>>>>
> >>>>>>> Duplicate work
> >>>>>>>
> >>>>>>> At the time that we began writing Samza, Kafka's consumer and
> >>> producer
> >>>>>>> APIs
> >>>>>>> had a relatively weak feature set. On the consumer-side, you
> >>>>>>> had two
> >>>>>>> options: use the high level consumer, or the simple consumer.
> >>>>>>> The
> >>>>> problem
> >>>>>>> with the high-level consumer was that it controlled your
> >>>>>>> offsets, partition assignments, and the order in which you
> >>>>>>> received messages. The
> >>> problem
> >>>>>>> with
> >>>>>>> the simple consumer is that it's not simple. It's basic. You
> >>>>>>> end up
> >>>>> having
> >>>>>>> to handle a lot of really low-level stuff that you shouldn't.
> >>>>>>> We
> >>>> spent a
> >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
> >>>>>>> also allows us to support some cool features:
> >>>>>>>
> >>>>>>> * Per-partition message ordering and prioritization.
> >>>>>>> * Tight control over partition assignment to support joins,
> >>>>>>> global
> >>>> state
> >>>>>>> (if we want to implement it :)), etc.
> >>>>>>> * Tight control over offset checkpointing.
> >>>>>>>
> >>>>>>> What we didn't realize at the time is that these features
> >>>>>>> should
> >>>>> actually
> >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> >>>> processors)
> >>>>>>> end up wanting to do things like joins and partition
> >>>>>>> assignment. The
> >>>>> Kafka
> >>>>>>> community has come to the same conclusion. They're adding a ton
> >>>>>>> of upgrades into their new Kafka consumer implementation. To a
> >>>>>>> large extent,
> >>> it's
> >>>>>>> duplicate work to what we've already done in Samza.
> >>>>>>>
> >>>>>>> On top of this, Kafka ended up taking a very similar approach
> >>>>>>> to
> >>>> Samza's
> >>>>>>> KafkaCheckpointManager implementation for handling offset
> >>>> checkpointing.
> >>>>>>> Like Samza, Kafka's new offset management feature stores offset
> >>>>>>> checkpoints in a topic, and allows you to fetch them from the
> >>>>>>> broker.
> >>>>>>>
> >>>>>>> A lot of this seems like a waste, since we could have shared
> >>>>>>> the
> >>> work
> >>>> if
> >>>>>>> it
> >>>>>>> had been done in Kafka from the get-go.
> >>>>>>>
> >>>>>>> Vision
> >>>>>>>
> >>>>>>> All of this leads me to a rather radical proposal. Samza is
> >>> relatively
> >>>>>>> stable at this point. I'd venture to say that we're near a 1.0
> >>>> release.
> >>>>>>> I'd
> >>>>>>> like to propose that we take what we've learned, and begin
> >>>>>>> thinking
> >>>>> about
> >>>>>>> Samza beyond 1.0. What would we change if we were starting from
> >>>> scratch?
> >>>>>>> My
> >>>>>>> proposal is to:
> >>>>>>>
> >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> >>>>>>> processors, and eliminate all direct dependences on YARN, Mesos,
> >> etc.
> >>>>>>> 2. Make a definitive call to support only Kafka as the stream
> >>>> processing
> >>>>>>> layer.
> >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> >>>>>>> config
> >>>>> systems,
> >>>>>>> and simply use Kafka's instead.
> >>>>>>>
> >>>>>>> This would fix all of the issues that I outlined above. It
> >>>>>>> should
> >>> also
> >>>>>>> shrink the Samza code base pretty dramatically. Supporting only
> >>>>>>> a standalone container will allow Samza to be executed on YARN
> >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> >>>>>>> in-house
> >>>>> deployment
> >>>>>>> systems. This should make life a lot easier for new users.
> >>>>>>> Imagine
> >>>>> having
> >>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
> >>>> traffic
> >>>>>>> will be pretty dramatic.
> >>>>>>>
> >>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
> >>> everyone
> >>>>>>> that
> >>>>>>> I'm aware of is using Samza with Kafka. We basically require it
> >>>> already
> >>>>> in
> >>>>>>> order for most features to work. Those that are using other
> >>>>>>> systems
> >>>> are
> >>>>>>> generally using it for ingest into Kafka (1), and then they do
> >>>>>>> the processing on top. There is already discussion (
> >>>>>>>
> >>>>>
> >>>>
> >>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >>> 767
> >>>>>>> )
> >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> >>>>>>>
> >>>>>>> Once we make the call to couple with Kafka, we can leverage a
> >>>>>>> ton of
> >>>>> their
> >>>>>>> ecosystem. We no longer have to maintain our own config,
> >>>>>>> metrics,
> >>> etc.
> >>>>> We
> >>>>>>> can all share the same libraries, and make them better. This
> >>>>>>> will
> >>> also
> >>>>>>> allow us to share the consumer/producer APIs, and will let us
> >>> leverage
> >>>>>>> their offset management and partition management, rather than
> >>>>>>> having
> >>>> our
> >>>>>>> own. All of the coordinator stream code would go away, as would
> >>>>>>> most
> >>>> of
> >>>>>>> the
> >>>>>>> YARN AppMaster code. We'd probably have to push some partition
> >>>>> management
> >>>>>>> features into the Kafka broker, but they're already moving in
> >>>>>>> that direction with the new consumer API. The features we have
> >>>>>>> for
> >>>> partition
> >>>>>>> assignment aren't unique to Samza, and seem like they should be
> >>>>>>> in
> >>>> Kafka
> >>>>>>> anyway. There will always be some niche usages which will
> >>>>>>> require
> >>>> extra
> >>>>>>> care and hence full control over partition assignments much
> >>>>>>> like the
> >>>>> Kafka
> >>>>>>> low level consumer api. These would continue to be supported.
> >>>>>>>
> >>>>>>> These items will be good for the Samza community. They'll make
> >>>>>>> Samza easier to use, and make it easier for developers to add
> >>>>>>> new features.
> >>>>>>>
> >>>>>>> Obviously this is a fairly large (and somewhat backwards
> >>> incompatible
> >>>>>>> change). If we choose to go this route, it's important that we
> >>> openly
> >>>>>>> communicate how we're going to provide a migration path from
> >>>>>>> the
> >>>>> existing
> >>>>>>> APIs to the new ones (if we make incompatible changes). I think
> >>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
> >>>>>>> existing StreamTask implementations to continue running on the
> >> new container.
> >>>>> It's
> >>>>>>> also important that we openly communicate about timing, and
> >>>>>>> stages
> >>> of
> >>>>> the
> >>>>>>> migration.
> >>>>>>>
> >>>>>>> If you made it this far, I'm sure you have opinions. :) Please
> >>>>>>> send
> >>>> your
> >>>>>>> thoughts and feedback.
> >>>>>>>
> >>>>>>> Cheers,
> >>>>>>> Chris
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>>
> >>>> --
> >>>> -- Guozhang
> >>>>
> >>>
> >>
>
>

RE: Thoughts and obesrvations on Samza

Posted by Ken Krugler <kk...@transpac.com>.
Hi Martin,

As a lurker here, this has been a very interesting thread.

I would suggest talking to one of the Solr committers about their experience in merging with Lucene, as that's got many similarities to what you're discussing here…though Solr was more mature when that happened.

From what I've seen externally, it ultimately was a win, though not without a lot of teething issues. Many of these seemed to come from personality conflicts between the groups, versus technical/admin/operational issues.

One additional comment inline below.

-- Ken

PS - As a counter example, Elasticsearch is doing fine as a separate project :)



> From: Martin Kleppmann
> Sent: July 6, 2015 1:18:29pm PDT
> To: dev@samza.apache.org
> Subject: Re: Thoughts and obesrvations on Samza
> 
> Ok, thanks for the clarifications. Just a few follow-up comments.
> 
> - I see the appeal of merging with Kafka or becoming a subproject: the reasons you mention are good. The risk I see is that release schedules become coupled to each other, which can slow everyone down, and large projects with many contributors are harder to manage. (Jakob, can you speak from experience, having seen a wider range of Hadoop ecosystem projects?)

Solr releases eventually came faster, as the Lucene release cycle improved, so in the end it wasn't the slow-down that everyone was expecting.

And it certainly reduced the continuous, painful debates over what belonged in Solr vs. Lucene.

> Some of the goals of a better unified developer experience could also be solved by integrating Samza nicely into a Kafka distribution (such as Confluent's). I'm not against merging projects if we decide that's the way to go, just pointing out the same goals can perhaps also be achieved in other ways.
> 
> - With regard to dropping the YARN dependency: are you proposing that Samza doesn't give any help to people wanting to run on YARN/Mesos/AWS/etc? So the docs would basically have a link to Slider and nothing else? Or would we maintain integrations with a bunch of popular deployment methods (e.g. the necessary glue and shell scripts to make Samza work with Slider)?
> 
> I absolutely think it's a good idea to have the "as a library" and "as a process" (using Yi's taxonomy) options for people who want them, but I think there should also be a low-friction path for common "as a service" deployment methods, for which we probably need to maintain integrations.
> 
> - Project naming: "Kafka Streams" seems odd to me, because Kafka is all about streams already. Perhaps "Kafka Transformers" or "Kafka Filters" would be more apt?
> 
> One suggestion: perhaps the core of Samza (stream transformation with state management -- i.e. the "Samza as a library" bit) could become part of Kafka, while higher-level tools such as streaming SQL and integrations with deployment frameworks remain in a separate project? In other words, Kafka would absorb the proven, stable core of Samza, which would become the "third Kafka client" mentioned early in this thread. The Samza project would then target that third Kafka client as its base API, and the project would be freed up to explore more experimental new horizons.
> 
> Martin

> 
> On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
> 
>> Hey Martin,
>> 
>> For the YARN/Mesos/etc decoupling I actually don't think it ties our hands
>> at all, all it does is refactor things. The division of responsibility is
>> that Samza core is responsible for task lifecycle, state, and partition
>> management (using the Kafka co-ordinator) but it is NOT responsible for
>> packaging, configuration deployment or execution of processes. The problem
>> of packaging and starting these processes is
>> framework/environment-specific. This leaves individual frameworks to be as
>> fancy or vanilla as they like. So you can get simple stateless support in
>> YARN, Mesos, etc using their off-the-shelf app framework (Slider, Marathon,
>> etc). These are well known by people and have nice UIs and a lot of
>> flexibility. I don't think they have node affinity as a built in option
>> (though I could be wrong). So if we want that we can either wait for them
>> to add it or do a custom framework to add that feature (as now). Obviously
>> if you manage things with old-school ops tools (puppet/chef/etc) you get
>> locality easily. The nice thing, though, is that all the samza "business
>> logic" around partition management and fault tolerance is in Samza core so
>> it is shared across frameworks and the framework specific bit is just
>> whether it is smart enough to try to get the same host when a job is
>> restarted.
>> 
>> With respect to the Kafka-alignment, yeah I think the goal would be (a)
>> actually get better alignment in user experience, and (b) express this in
>> the naming and project branding. Specifically:
>> 1. Website/docs, it would be nice for the "transformation" api to be
>> discoverable in the main Kafka docs--i.e. be able to explain when to use
>> the consumer and when to use the stream processing functionality and lead
>> people into that experience.
>> 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that has both
>> Kafka and the stream processing part and they actually work together.
>> 3. Unify the programming experience so the client and Samza api share
>> config/monitoring/naming/packaging/etc.
>> 
>> I think sub-projects keep separate committers and can have a separate repo,
>> but I'm actually not really sure (I can't find a definition of a subproject
>> in Apache).
>> 
>> Basically at a high-level you want the experience to "feel" like a single
>> system, not to relatively independent things that are kind of awkwardly
>> glued together.
>> 
>> I think if we did that they having naming or branding like "kafka
>> streaming" or "kafka streams" or something like that would actually do a
>> good job of conveying what it is. I do that this would help adoption quite
>> a lot as it would correctly convey that using Kafka Streaming with Kafka is
>> a fairly seamless experience and Kafka is pretty heavily adopted at this
>> point.
>> 
>> Fwiw we actually considered this model originally when open sourcing Samza,
>> however at that time Kafka was relatively unknown and we decided not to do
>> it since we felt it would be limiting. From my point of view the three
>> things have changed (1) Kafka is now really heavily used for stream
>> processing, (2) we learned that abstracting out the stream well is
>> basically impossible, (3) we learned it is really hard to keep the two
>> things feeling like a single product.
>> 
>> -Jay
>> 
>> 
>> On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <ma...@kleppmann.com>
>> wrote:
>> 
>>> Hi all,
>>> 
>>> Lots of good thoughts here.
>>> 
>>> I agree with the general philosophy of tying Samza more firmly to Kafka.
>>> After I spent a while looking at integrating other message brokers (e.g.
>>> Kinesis) with SystemConsumer, I came to the conclusion that SystemConsumer
>>> tacitly assumes a model so much like Kafka's that pretty much nobody but
>>> Kafka actually implements it. (Databus is perhaps an exception, but it
>>> isn't widely used outside of LinkedIn.) Thus, making Samza fully dependent
>>> on Kafka acknowledges that the system-independence was never as real as we
>>> perhaps made it out to be. The gains of code reuse are real.
>>> 
>>> The idea of decoupling Samza from YARN has also always been appealing to
>>> me, for various reasons already mentioned in this thread. Although making
>>> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems laudable, I am
>>> a little concerned that it will restrict us to a lowest common denominator.
>>> For example, would host affinity (SAMZA-617) still be possible? For jobs
>>> with large amounts of state, I think SAMZA-617 would be a big boon, since
>>> restoring state off the changelog on every single restart is painful, due
>>> to long recovery times. It would be a shame if the decoupling from YARN
>>> made host affinity impossible.
>>> 
>>> Jay, a question about the proposed API for instantiating a job in code
>>> (rather than a properties file): when submitting a job to a cluster, is the
>>> idea that the instantiation code runs on a client somewhere, which then
>>> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code run
>>> on each container that is part of the job (in which case, how does the job
>>> submission to the cluster work)?
>>> 
>>> I agree with Garry that it doesn't feel right to make a 1.0 release with a
>>> plan for it to be immediately obsolete. So if this is going to happen, I
>>> think it would be more honest to stick with 0.* version numbers until the
>>> library-ified Samza has been implemented, is stable and widely used.
>>> 
>>> Should the new Samza be a subproject of Kafka? There is precedent for
>>> tight coupling between different Apache projects (e.g. Curator and
>>> Zookeeper, or Slider and YARN), so I think remaining separate would be ok.
>>> Even if Samza is fully dependent on Kafka, there is enough substance in
>>> Samza that it warrants being a separate project. An argument in favour of
>>> merging would be if we think Kafka has a much stronger "brand presence"
>>> than Samza; I'm ambivalent on that one. If the Kafka project is willing to
>>> endorse Samza as the "official" way of doing stateful stream
>>> transformations, that would probably have much the same effect as
>>> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
>>> collaboration between the two projects will be needed in any case.
>>> 
>>> From a project management perspective, I guess the "new Samza" would have
>>> to be developed on a branch alongside ongoing maintenance of the current
>>> line of development? I think it would be important to continue supporting
>>> existing users, and provide a graceful migration path to the new version.
>>> Leaving the current versions unsupported and forcing people to rewrite
>>> their jobs would send a bad signal.
>>> 
>>> Best,
>>> Martin
>>> 
>>> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
>>> 
>>>> Hey Garry,
>>>> 
>>>> Yeah that's super frustrating. I'd be happy to chat more about this if
>>>> you'd be interested. I think Chris and I started with the idea of "what
>>>> would it take to make Samza a kick-ass ingestion tool" but ultimately we
>>>> kind of came around to the idea that ingestion and transformation had
>>>> pretty different needs and coupling the two made things hard.
>>>> 
>>>> For what it's worth I think copycat (KIP-26) actually will do what you
>>> are
>>>> looking for.
>>>> 
>>>> With regard to your point about slider, I don't necessarily disagree.
>>> But I
>>>> think getting good YARN support is quite doable and I think we can make
>>>> that work well. I think the issue this proposal solves is that
>>> technically
>>>> it is pretty hard to support multiple cluster management systems the way
>>>> things are now, you need to write an "app master" or "framework" for each
>>>> and they are all a little different so testing is really hard. In the
>>>> absence of this we have been stuck with just YARN which has fantastic
>>>> penetration in the Hadoopy part of the org, but zero penetration
>>> elsewhere.
>>>> Given the huge amount of work being put in to slider, marathon, aws
>>>> tooling, not to mention the umpteen related packaging technologies people
>>>> want to use (Docker, Kubernetes, various cloud-specific deploy tools,
>>> etc)
>>>> I really think it is important to get this right.
>>>> 
>>>> -Jay
>>>> 
>>>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>>>> g.turkington@improvedigital.com> wrote:
>>>> 
>>>>> Hi all,
>>>>> 
>>>>> I think the question below re does Samza become a sub-project of Kafka
>>>>> highlights the broader point around migration. Chris mentions Samza's
>>>>> maturity is heading towards a v1 release but I'm not sure it feels
>>> right to
>>>>> launch a v1 then immediately plan to deprecate most of it.
>>>>> 
>>>>> From a selfish perspective I have some guys who have started working
>>> with
>>>>> Samza and building some new consumers/producers was next up. Sounds like
>>>>> that is absolutely not the direction to go. I need to look into the KIP
>>> in
>>>>> more detail but for me the attractiveness of adding new Samza
>>>>> consumer/producers -- even if yes all they were doing was really getting
>>>>> data into and out of Kafka --  was to avoid  having to worry about the
>>>>> lifecycle management of external clients. If there is a generic Kafka
>>>>> ingress/egress layer that I can plug a new connector into and have a
>>> lot of
>>>>> the heavy lifting re scale and reliability done for me then it gives me
>>> all
>>>>> the pushing new consumers/producers would. If not then it complicates my
>>>>> operational deployments.
>>>>> 
>>>>> Which is similar to my other question with the proposal -- if we build a
>>>>> fully available/stand-alone Samza plus the requisite shims to integrate
>>>>> with Slider etc I suspect the former may be a lot more work than we
>>> think.
>>>>> We may make it much easier for a newcomer to get something running but
>>>>> having them step up and get a reliable production deployment may still
>>>>> dominate mailing list  traffic, if for different reasons than today.
>>>>> 
>>>>> Don't get me wrong -- I'm comfortable with making the Samza dependency
>>> on
>>>>> Kafka much more explicit and I absolutely see the benefits  in the
>>>>> reduction of duplication and clashing terminologies/abstractions that
>>>>> Chris/Jay describe. Samza as a library would likely be a very nice tool
>>> to
>>>>> add to the Kafka ecosystem. I just have the concerns above re the
>>>>> operational side.
>>>>> 
>>>>> Garry
>>>>> 
>>>>> -----Original Message-----
>>>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
>>>>> Sent: 02 July 2015 12:56
>>>>> To: dev@samza.apache.org
>>>>> Subject: Re: Thoughts and obesrvations on Samza
>>>>> 
>>>>> Very interesting thoughts.
>>>>> From outside, I have always perceived Samza as a computing layer over
>>>>> Kafka.
>>>>> 
>>>>> The question, maybe a bit provocative, is "should Samza be a sub-project
>>>>> of Kafka then?"
>>>>> Or does it make sense to keep it as a separate project with a separate
>>>>> governance?
>>>>> 
>>>>> Cheers,
>>>>> 
>>>>> --
>>>>> Gianmarco
>>>>> 
>>>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
>>>>> 
>>>>>> Overall, I agree to couple with Kafka more tightly. Because Samza de
>>>>>> facto is based on Kafka, and it should leverage what Kafka has. At the
>>>>>> same time, Kafka does not need to reinvent what Samza already has. I
>>>>>> also like the idea of separating the ingestion and transformation.
>>>>>> 
>>>>>> But it is a little difficult for me to image how the Samza will look
>>>>> like.
>>>>>> And I feel Chris and Jay have a little difference in terms of how
>>>>>> Samza should look like.
>>>>>> 
>>>>>> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
>>>>>> user's application code calls this client?
>>>>>> 
>>>>>> 1. If we make Samza be a library of Kafka (like what the code shows),
>>>>>> how do we implement auto-balance and fault-tolerance? Are they taken
>>>>>> care by the Kafka broker or other mechanism, such as "Samza worker"
>>>>>> (just make up the name) ?
>>>>>> 
>>>>>> 2. What about other features, such as auto-scaling, shared state,
>>>>>> monitoring?
>>>>>> 
>>>>>> 
>>>>>> *** If we have Samza standalone, (is this what Chris suggests?)
>>>>>> 
>>>>>> 1. we still need to ingest data from Kakfa and produce to it. Then it
>>>>>> becomes the same as what Samza looks like now, except it does not rely
>>>>>> on Yarn anymore.
>>>>>> 
>>>>>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
>>>>>> etc? Use Kafka code as the dependency?
>>>>>> 
>>>>>> 
>>>>>> Thanks,
>>>>>> 
>>>>>> Fang, Yan
>>>>>> yanfang724@gmail.com
>>>>>> 
>>>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
>>>>> wrote:
>>>>>> 
>>>>>>> Read through the code example and it looks good to me. A few
>>>>>>> thoughts regarding deployment:
>>>>>>> 
>>>>>>> Today Samza deploys as executable runnable like:
>>>>>>> 
>>>>>>> deploy/samza/bin/run-job.sh --config-factory=...
>>>>> --config-path=file://...
>>>>>>> 
>>>>>>> And this proposal advocate for deploying Samza more as embedded
>>>>>>> libraries in user application code (ignoring the terminology since
>>>>>>> it is not the
>>>>>> same
>>>>>>> as the prototype code):
>>>>>>> 
>>>>>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
>>>>>>> Thread(task); thread.start();
>>>>>>> 
>>>>>>> I think both of these deployment modes are important for different
>>>>>>> types
>>>>>> of
>>>>>>> users. That said, I think making Samza purely standalone is still
>>>>>>> sufficient for either runnable or library modes.
>>>>>>> 
>>>>>>> Guozhang
>>>>>>> 
>>>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
>>>>>>> 
>>>>>>>> Looks like gmail mangled the code example, it was supposed to look
>>>>>>>> like
>>>>>>>> this:
>>>>>>>> 
>>>>>>>> Properties props = new Properties();
>>>>>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
>>>>>>>> config = new StreamingConfig(props);
>>>>>>>> config.subscribe("test-topic-1", "test-topic-2");
>>>>>>>> config.processor(ExampleStreamProcessor.class);
>>>>>>>> config.serialization(new StringSerializer(), new
>>>>>>>> StringDeserializer()); KafkaStreaming container = new
>>>>>>>> KafkaStreaming(config); container.run();
>>>>>>>> 
>>>>>>>> -Jay
>>>>>>>> 
>>>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
>>>>> wrote:
>>>>>>>> 
>>>>>>>>> Hey guys,
>>>>>>>>> 
>>>>>>>>> This came out of some conversations Chris and I were having
>>>>>>>>> around
>>>>>>>> whether
>>>>>>>>> it would make sense to use Samza as a kind of data ingestion
>>>>>> framework
>>>>>>>> for
>>>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
>>>>>>> combined
>>>>>>>>> with complaints around config and YARN and the discussion around
>>>>>>>>> how
>>>>>> to
>>>>>>>>> best do a standalone mode.
>>>>>>>>> 
>>>>>>>>> So the thought experiment was, given that Samza was basically
>>>>>>>>> already totally Kafka specific, what if you just embraced that
>>>>>>>>> and turned it
>>>>>>> into
>>>>>>>>> something less like a heavyweight framework and more like a
>>>>>>>>> third
>>>>>> Kafka
>>>>>>>>> client--a kind of "producing consumer" with state management
>>>>>>> facilities.
>>>>>>>>> Basically a library. Instead of a complex stream processing
>>>>>>>>> framework
>>>>>>>> this
>>>>>>>>> would actually be a very simple thing, not much more complicated
>>>>>>>>> to
>>>>>> use
>>>>>>>> or
>>>>>>>>> operate than a Kafka consumer. As Chris said we thought about it
>>>>>>>>> a
>>>>>> lot
>>>>>>> of
>>>>>>>>> what Samza (and the other stream processing systems were doing)
>>>>>> seemed
>>>>>>>> like
>>>>>>>>> kind of a hangover from MapReduce.
>>>>>>>>> 
>>>>>>>>> Of course you need to ingest/output data to and from the stream
>>>>>>>>> processing. But when we actually looked into how that would
>>>>>>>>> work,
>>>>>> Samza
>>>>>>>>> isn't really an ideal data ingestion framework for a bunch of
>>>>>> reasons.
>>>>>>> To
>>>>>>>>> really do that right you need a pretty different internal data
>>>>>>>>> model
>>>>>>> and
>>>>>>>>> set of apis. So what if you split them and had an api for Kafka
>>>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
>>>>>>>>> transformation (Samza).
>>>>>>>>> 
>>>>>>>>> This would also allow really embracing the same terminology and
>>>>>>>>> conventions. One complaint about the current state is that the
>>>>>>>>> two
>>>>>>>> systems
>>>>>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
>>>>>>>> different
>>>>>>>>> config and monitoring systems means you kind of have to learn
>>>>>>>>> Kafka's
>>>>>>>> way,
>>>>>>>>> then learn Samza's slightly different way, then kind of
>>>>>>>>> understand
>>>>>> how
>>>>>>>> they
>>>>>>>>> map to each other, which having walked a few people through this
>>>>>>>>> is surprisingly tricky for folks to get.
>>>>>>>>> 
>>>>>>>>> Since I have been spending a lot of time on airplanes I hacked
>>>>>>>>> up an ernest but still somewhat incomplete prototype of what
>>>>>>>>> this would
>>>>>> look
>>>>>>>>> like. This is just unceremoniously dumped into Kafka as it
>>>>>>>>> required a
>>>>>>> few
>>>>>>>>> changes to the new consumer. Here is the code:
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>>>>>> /apache/kafka/clients/streaming
>>>>>>>>> 
>>>>>>>>> For the purpose of the prototype I just liberally renamed
>>>>>>>>> everything
>>>>>> to
>>>>>>>>> try to align it with Kafka with no regard for compatibility.
>>>>>>>>> 
>>>>>>>>> To use this would be something like this:
>>>>>>>>> Properties props = new Properties();
>>>>>>>>> props.put("bootstrap.servers", "localhost:4242");
>>>>>>>>> StreamingConfig config = new
>>>>>> StreamingConfig(props);
>>>>>>>> config.subscribe("test-topic-1",
>>>>>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
>>>>>>>> config.serialization(new
>>>>>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
>>>>>>> container =
>>>>>>>>> new KafkaStreaming(config); container.run();
>>>>>>>>> 
>>>>>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
>>>>>>>>> is basically StreamTask.
>>>>>>>>> 
>>>>>>>>> So rather than putting all the class names in a file and then
>>>>>>>>> having
>>>>>>> the
>>>>>>>>> job assembled by reflection, you just instantiate the container
>>>>>>>>> programmatically. Work is balanced over however many instances
>>>>>>>>> of
>>>>>> this
>>>>>>>> are
>>>>>>>>> alive at any time (i.e. if an instance dies, new tasks are added
>>>>>>>>> to
>>>>>> the
>>>>>>>>> existing containers without shutting them down).
>>>>>>>>> 
>>>>>>>>> We would provide some glue for running this stuff in YARN via
>>>>>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
>>>>>>>>> but from the
>>>>>>> point
>>>>>>>> of
>>>>>>>>> view of these frameworks these stream processing jobs are just
>>>>>>> stateless
>>>>>>>>> services that can come and go and expand and contract at will.
>>>>>>>>> There
>>>>>> is
>>>>>>>> no
>>>>>>>>> more custom scheduler.
>>>>>>>>> 
>>>>>>>>> Here are some relevant details:
>>>>>>>>> 
>>>>>>>>> 1. It is only ~1300 lines of code, it would get larger if we
>>>>>>>>> productionized but not vastly larger. We really do get a ton
>>>>>>>>> of
>>>>>>>> leverage
>>>>>>>>> out of Kafka.
>>>>>>>>> 2. Partition management is fully delegated to the new consumer.
>>>>>> This
>>>>>>>>> is nice since now any partition management strategy available
>>>>>>>>> to
>>>>>>> Kafka
>>>>>>>>> consumer is also available to Samza (and vice versa) and with
>>>>>>>>> the
>>>>>>>> exact
>>>>>>>>> same configs.
>>>>>>>>> 3. It supports state as well as state reuse
>>>>>>>>> 
>>>>>>>>> Anyhow take a look, hopefully it is thought provoking.
>>>>>>>>> 
>>>>>>>>> -Jay
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>>>>>>> criccomini@apache.org>
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Hey all,
>>>>>>>>>> 
>>>>>>>>>> I have had some discussions with Samza engineers at LinkedIn
>>>>>>>>>> and
>>>>>>>> Confluent
>>>>>>>>>> and we came up with a few observations and would like to
>>>>>>>>>> propose
>>>>>> some
>>>>>>>>>> changes.
>>>>>>>>>> 
>>>>>>>>>> We've observed some things that I want to call out about
>>>>>>>>>> Samza's
>>>>>>> design,
>>>>>>>>>> and I'd like to propose some changes.
>>>>>>>>>> 
>>>>>>>>>> * Samza is dependent upon a dynamic deployment system.
>>>>>>>>>> * Samza is too pluggable.
>>>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
>>>>>>>>>> APIs
>>>>>> are
>>>>>>>>>> trying to solve a lot of the same problems.
>>>>>>>>>> 
>>>>>>>>>> All three of these issues are related, but I'll address them in
>>>>>> order.
>>>>>>>>>> 
>>>>>>>>>> Deployment
>>>>>>>>>> 
>>>>>>>>>> Samza strongly depends on the use of a dynamic deployment
>>>>>>>>>> scheduler
>>>>>>> such
>>>>>>>>>> as
>>>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
>>>>>>>>>> there
>>>>>>> would
>>>>>>>>>> be
>>>>>>>>>> one or two winners in this area, and we could support them, and
>>>>>>>>>> the
>>>>>>> rest
>>>>>>>>>> would go away. In reality, there are many variations.
>>>>>>>>>> Furthermore,
>>>>>>> many
>>>>>>>>>> people still prefer to just start their processors like normal
>>>>>>>>>> Java processes, and use traditional deployment scripts such as
>>>>>>>>>> Fabric,
>>>>>>> Chef,
>>>>>>>>>> Ansible, etc. Forcing a deployment system on users makes the
>>>>>>>>>> Samza start-up process really painful for first time users.
>>>>>>>>>> 
>>>>>>>>>> Dynamic deployment as a requirement was also a bit of a
>>>>>>>>>> mis-fire
>>>>>>> because
>>>>>>>>>> of
>>>>>>>>>> a fundamental misunderstanding between the nature of batch jobs
>>>>>>>>>> and
>>>>>>>> stream
>>>>>>>>>> processing jobs. Early on, we made conscious effort to favor
>>>>>>>>>> the
>>>>>>> Hadoop
>>>>>>>>>> (Map/Reduce) way of doing things, since it worked and was well
>>>>>>>> understood.
>>>>>>>>>> One thing that we missed was that batch jobs have a definite
>>>>>>> beginning,
>>>>>>>>>> and
>>>>>>>>>> end, and stream processing jobs don't (usually). This leads to
>>>>>>>>>> a
>>>>>> much
>>>>>>>>>> simpler scheduling problem for stream processors. You basically
>>>>>>>>>> just
>>>>>>>> need
>>>>>>>>>> to find a place to start the processor, and start it. The way
>>>>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
>>>>>>>>>> being "full". We always
>>>>>>> add
>>>>>>>>>> more machines. The problem with coupling Samza with a scheduler
>>>>>>>>>> is
>>>>>>> that
>>>>>>>>>> Samza (as a framework) now has to handle deployment. This pulls
>>>>>>>>>> in a
>>>>>>>> bunch
>>>>>>>>>> of things such as configuration distribution (config stream),
>>>>>>>>>> shell
>>>>>>>> scrips
>>>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
>>>>>>>>>> 
>>>>>>>>>> Another reason for requiring dynamic deployment was to support
>>>>>>>>>> data locality. If you want to have locality, you need to put
>>>>>>>>>> your
>>>>>>> processors
>>>>>>>>>> close to the data they're processing. Upon further
>>>>>>>>>> investigation,
>>>>>>>> though,
>>>>>>>>>> this feature is not that beneficial. There is some good
>>>>>>>>>> discussion
>>>>>>> about
>>>>>>>>>> some problems with it on SAMZA-335. Again, we took the
>>>>>>>>>> Map/Reduce
>>>>>>> path,
>>>>>>>>>> but
>>>>>>>>>> there are some fundamental differences between HDFS and Kafka.
>>>>>>>>>> HDFS
>>>>>>> has
>>>>>>>>>> blocks, while Kafka has partitions. This leads to less
>>>>>>>>>> optimization potential with stream processors on top of Kafka.
>>>>>>>>>> 
>>>>>>>>>> This feature is also used as a crutch. Samza doesn't have any
>>>>>>>>>> built
>>>>>> in
>>>>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
>>>>>>>>>> deployment scheduling system to handle restarts when a
>>>>>>>>>> processor dies. This has
>>>>>>>> made
>>>>>>>>>> it very difficult to write a standalone Samza container
>>>>> (SAMZA-516).
>>>>>>>>>> 
>>>>>>>>>> Pluggability
>>>>>>>>>> 
>>>>>>>>>> In some cases pluggability is good, but I think that we've gone
>>>>>>>>>> too
>>>>>>> far
>>>>>>>>>> with it. Currently, Samza has:
>>>>>>>>>> 
>>>>>>>>>> * Pluggable config.
>>>>>>>>>> * Pluggable metrics.
>>>>>>>>>> * Pluggable deployment systems.
>>>>>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
>>>>> etc).
>>>>>>>>>> * Pluggable serdes.
>>>>>>>>>> * Pluggable storage engines.
>>>>>>>>>> * Pluggable strategies for just about every component
>>>>>> (MessageChooser,
>>>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>>>>>>>>>> 
>>>>>>>>>> There's probably more that I've forgotten, as well. Some of
>>>>>>>>>> these
>>>>>> are
>>>>>>>>>> useful, but some have proven not to be. This all comes at a cost:
>>>>>>>>>> complexity. This complexity is making it harder for our users
>>>>>>>>>> to
>>>>>> pick
>>>>>>> up
>>>>>>>>>> and use Samza out of the box. It also makes it difficult for
>>>>>>>>>> Samza developers to reason about what the characteristics of
>>>>>>>>>> the container (since the characteristics change depending on
>>>>>>>>>> which plugins are use).
>>>>>>>>>> 
>>>>>>>>>> The issues with pluggability are most visible in the System APIs.
>>>>>> What
>>>>>>>>>> Samza really requires to be functional is Kafka as its
>>>>>>>>>> transport
>>>>>>> layer.
>>>>>>>>>> But
>>>>>>>>>> we've conflated two unrelated use cases into one API:
>>>>>>>>>> 
>>>>>>>>>> 1. Get data into/out of Kafka.
>>>>>>>>>> 2. Process the data in Kafka.
>>>>>>>>>> 
>>>>>>>>>> The current System API supports both of these use cases. The
>>>>>>>>>> problem
>>>>>>> is,
>>>>>>>>>> we
>>>>>>>>>> actually want different features for each use case. By papering
>>>>>>>>>> over
>>>>>>>> these
>>>>>>>>>> two use cases, and providing a single API, we've introduced a
>>>>>>>>>> ton of
>>>>>>>> leaky
>>>>>>>>>> abstractions.
>>>>>>>>>> 
>>>>>>>>>> For example, what we'd really like in (2) is to have
>>>>>>>>>> monotonically increasing longs for offsets (like Kafka). This
>>>>>>>>>> would be at odds
>>>>>> with
>>>>>>>> (1),
>>>>>>>>>> though, since different systems have different
>>>>>>>> SCNs/Offsets/UUIDs/vectors.
>>>>>>>>>> There was discussion both on the mailing list and the SQL JIRAs
>>>>>> about
>>>>>>>> the
>>>>>>>>>> need for this.
>>>>>>>>>> 
>>>>>>>>>> The same thing holds true for replayability. Kafka allows us to
>>>>>> rewind
>>>>>>>>>> when
>>>>>>>>>> we have a failure. Many other systems don't. In some cases,
>>>>>>>>>> systems
>>>>>>>> return
>>>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
>>>>>>>>>> they
>>>>>>> have
>>>>>>>> no
>>>>>>>>>> offsets.
>>>>>>>>>> 
>>>>>>>>>> Partitioning is another example. Kafka supports partitioning,
>>>>>>>>>> but
>>>>>> many
>>>>>>>>>> systems don't. We model this by having a single partition for
>>>>>>>>>> those systems. Still, other systems model partitioning
>>>>> differently (e.g.
>>>>>>>>>> Kinesis).
>>>>>>>>>> 
>>>>>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
>>>>>>>>>> system-agnostic way is almost impossible. As is modeling
>>>>>>>>>> metadata
>>>>>> for
>>>>>>>> the
>>>>>>>>>> system (replication factor, partitions, location, etc). The
>>>>>>>>>> list
>>>>>> goes
>>>>>>>> on.
>>>>>>>>>> 
>>>>>>>>>> Duplicate work
>>>>>>>>>> 
>>>>>>>>>> At the time that we began writing Samza, Kafka's consumer and
>>>>>> producer
>>>>>>>>>> APIs
>>>>>>>>>> had a relatively weak feature set. On the consumer-side, you
>>>>>>>>>> had two
>>>>>>>>>> options: use the high level consumer, or the simple consumer.
>>>>>>>>>> The
>>>>>>>> problem
>>>>>>>>>> with the high-level consumer was that it controlled your
>>>>>>>>>> offsets, partition assignments, and the order in which you
>>>>>>>>>> received messages. The
>>>>>> problem
>>>>>>>>>> with
>>>>>>>>>> the simple consumer is that it's not simple. It's basic. You
>>>>>>>>>> end up
>>>>>>>> having
>>>>>>>>>> to handle a lot of really low-level stuff that you shouldn't.
>>>>>>>>>> We
>>>>>>> spent a
>>>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
>>>>>>>>>> also allows us to support some cool features:
>>>>>>>>>> 
>>>>>>>>>> * Per-partition message ordering and prioritization.
>>>>>>>>>> * Tight control over partition assignment to support joins,
>>>>>>>>>> global
>>>>>>> state
>>>>>>>>>> (if we want to implement it :)), etc.
>>>>>>>>>> * Tight control over offset checkpointing.
>>>>>>>>>> 
>>>>>>>>>> What we didn't realize at the time is that these features
>>>>>>>>>> should
>>>>>>>> actually
>>>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
>>>>>>> processors)
>>>>>>>>>> end up wanting to do things like joins and partition
>>>>>>>>>> assignment. The
>>>>>>>> Kafka
>>>>>>>>>> community has come to the same conclusion. They're adding a ton
>>>>>>>>>> of upgrades into their new Kafka consumer implementation. To a
>>>>>>>>>> large extent,
>>>>>> it's
>>>>>>>>>> duplicate work to what we've already done in Samza.
>>>>>>>>>> 
>>>>>>>>>> On top of this, Kafka ended up taking a very similar approach
>>>>>>>>>> to
>>>>>>> Samza's
>>>>>>>>>> KafkaCheckpointManager implementation for handling offset
>>>>>>> checkpointing.
>>>>>>>>>> Like Samza, Kafka's new offset management feature stores offset
>>>>>>>>>> checkpoints in a topic, and allows you to fetch them from the
>>>>>>>>>> broker.
>>>>>>>>>> 
>>>>>>>>>> A lot of this seems like a waste, since we could have shared
>>>>>>>>>> the
>>>>>> work
>>>>>>> if
>>>>>>>>>> it
>>>>>>>>>> had been done in Kafka from the get-go.
>>>>>>>>>> 
>>>>>>>>>> Vision
>>>>>>>>>> 
>>>>>>>>>> All of this leads me to a rather radical proposal. Samza is
>>>>>> relatively
>>>>>>>>>> stable at this point. I'd venture to say that we're near a 1.0
>>>>>>> release.
>>>>>>>>>> I'd
>>>>>>>>>> like to propose that we take what we've learned, and begin
>>>>>>>>>> thinking
>>>>>>>> about
>>>>>>>>>> Samza beyond 1.0. What would we change if we were starting from
>>>>>>> scratch?
>>>>>>>>>> My
>>>>>>>>>> proposal is to:
>>>>>>>>>> 
>>>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
>>>>>>>>>> processors, and eliminate all direct dependences on YARN, Mesos,
>>>>> etc.
>>>>>>>>>> 2. Make a definitive call to support only Kafka as the stream
>>>>>>> processing
>>>>>>>>>> layer.
>>>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
>>>>>>>>>> config
>>>>>>>> systems,
>>>>>>>>>> and simply use Kafka's instead.
>>>>>>>>>> 
>>>>>>>>>> This would fix all of the issues that I outlined above. It
>>>>>>>>>> should
>>>>>> also
>>>>>>>>>> shrink the Samza code base pretty dramatically. Supporting only
>>>>>>>>>> a standalone container will allow Samza to be executed on YARN
>>>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
>>>>>>>>>> in-house
>>>>>>>> deployment
>>>>>>>>>> systems. This should make life a lot easier for new users.
>>>>>>>>>> Imagine
>>>>>>>> having
>>>>>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
>>>>>>> traffic
>>>>>>>>>> will be pretty dramatic.
>>>>>>>>>> 
>>>>>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
>>>>>> everyone
>>>>>>>>>> that
>>>>>>>>>> I'm aware of is using Samza with Kafka. We basically require it
>>>>>>> already
>>>>>>>> in
>>>>>>>>>> order for most features to work. Those that are using other
>>>>>>>>>> systems
>>>>>>> are
>>>>>>>>>> generally using it for ingest into Kafka (1), and then they do
>>>>>>>>>> the processing on top. There is already discussion (
>>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>>>>>> 767
>>>>>>>>>> )
>>>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
>>>>>>>>>> 
>>>>>>>>>> Once we make the call to couple with Kafka, we can leverage a
>>>>>>>>>> ton of
>>>>>>>> their
>>>>>>>>>> ecosystem. We no longer have to maintain our own config,
>>>>>>>>>> metrics,
>>>>>> etc.
>>>>>>>> We
>>>>>>>>>> can all share the same libraries, and make them better. This
>>>>>>>>>> will
>>>>>> also
>>>>>>>>>> allow us to share the consumer/producer APIs, and will let us
>>>>>> leverage
>>>>>>>>>> their offset management and partition management, rather than
>>>>>>>>>> having
>>>>>>> our
>>>>>>>>>> own. All of the coordinator stream code would go away, as would
>>>>>>>>>> most
>>>>>>> of
>>>>>>>>>> the
>>>>>>>>>> YARN AppMaster code. We'd probably have to push some partition
>>>>>>>> management
>>>>>>>>>> features into the Kafka broker, but they're already moving in
>>>>>>>>>> that direction with the new consumer API. The features we have
>>>>>>>>>> for
>>>>>>> partition
>>>>>>>>>> assignment aren't unique to Samza, and seem like they should be
>>>>>>>>>> in
>>>>>>> Kafka
>>>>>>>>>> anyway. There will always be some niche usages which will
>>>>>>>>>> require
>>>>>>> extra
>>>>>>>>>> care and hence full control over partition assignments much
>>>>>>>>>> like the
>>>>>>>> Kafka
>>>>>>>>>> low level consumer api. These would continue to be supported.
>>>>>>>>>> 
>>>>>>>>>> These items will be good for the Samza community. They'll make
>>>>>>>>>> Samza easier to use, and make it easier for developers to add
>>>>>>>>>> new features.
>>>>>>>>>> 
>>>>>>>>>> Obviously this is a fairly large (and somewhat backwards
>>>>>> incompatible
>>>>>>>>>> change). If we choose to go this route, it's important that we
>>>>>> openly
>>>>>>>>>> communicate how we're going to provide a migration path from
>>>>>>>>>> the
>>>>>>>> existing
>>>>>>>>>> APIs to the new ones (if we make incompatible changes). I think
>>>>>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
>>>>>>>>>> existing StreamTask implementations to continue running on the
>>>>> new container.
>>>>>>>> It's
>>>>>>>>>> also important that we openly communicate about timing, and
>>>>>>>>>> stages
>>>>>> of
>>>>>>>> the
>>>>>>>>>> migration.
>>>>>>>>>> 
>>>>>>>>>> If you made it this far, I'm sure you have opinions. :) Please
>>>>>>>>>> send
>>>>>>> your
>>>>>>>>>> thoughts and feedback.
>>>>>>>>>> 
>>>>>>>>>> Cheers,
>>>>>>>>>> Chris
>>>>>>>>>> 





--------------------------
Ken Krugler
+1 530-210-6378
http://www.scaleunlimited.com
custom big data solutions & training
Hadoop, Cascading, Cassandra & Solr






Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
Hi, Julian and Martin,

Good point on community-merging vs project-merging and good summary!

For Julian's point #2, I think that he was referring to the support to
integrate w/ a cluster job execution framework, like YARN/Mesos/AWS. And
who (i.e. the community) and which project (i.e. code) would support this
integration layer. My personal preference is this should be considered as a
sub- or separate project (i.e. code-wise) on top of Samza, and supported by
Samza community (or at least, a good overlap w/ the Samza community).
Personally, I view it as a distributed job execution framework for
streaming processing, just like YARN+Slider for MapReduce jobs, if it makes
sense.


On Thu, Jul 9, 2015 at 10:14 AM, Martin Kleppmann <ma...@kleppmann.com>
wrote:

> Thanks Julian for calling out the principle of community over code, which
> is super important. If it was just a matter of code, the Kafka project
> could simply pull in the Samza code (or write a new stream processor)
> without asking permission -- but they wouldn't get the Samza community.
> Thus, I think the community aspect is the most important part of this
> discussion. If we're talking about merging projects, it's really about
> merging communities.
>
> I had a chat with a friend who is a Lucene/Solr committer: those were also
> originally two separate projects, which merged into one. He said the merge
> was not always easy, but probably a net win for both projects and
> communities overall. In their community people tend to specialise on either
> the Lucene part or the Solr part, but that's ok -- it's still a cohesive
> community nevertheless, and it benefits from close collaboration due to
> having everyone in the same project. Releases didn't slow down; in fact,
> they perhaps got faster due to less cross-project coordination overhead. So
> that allayed my concerns about a big project becoming slow.
>
> Besides community and code/architecture, another consideration is our user
> base (including those who are not on this mailing list). What is good for
> our users? I've thought about this more over the last few days:
>
> - Reducing users' confusion is good. If someone is adopting Kafka, they
> will also need some way of processing their data in Kafka. At the moment,
> the Kafka docs give you consumer APIs but nothing more. Having to choose a
> separate stream processing framework is a burden on users, especially if
> that framework uses terminology that is inconsistent with Kafka. If we make
> Samza a part of Kafka and unify the terminology, it would become a coherent
> part of the documentation, and be much less confusing for users.
>
> - Making it easy for users to get started is good. Simplifying the API and
> configuration is part of it. Making YARN optional is also good. It would
> also help to be part of the same package that people download, and part of
> the same documentation. (Simplifying API/config and decoupling from YARN
> can be done as a separate project; becoming part of the same package would
> require merging projects.)
>
> - Supporting users' choice of programming language is good. I used to work
> with Ruby, and in the Ruby community there are plenty of people with an
> irrational hatred of the JVM. I imagine other language communities are
> likely similar. If Samza becomes a fairly thin client library to Kafka
> (using partition assignment etc provided by the Kafka brokers), then it
> becomes much more feasible to implement the same interface in other
> languages too, giving true multi-language support.
>
> Having thought about this, I am coming to the conclusion that a stream
> processor that is part of the Kafka project would be good for users, and
> thus a more successful project. However, the people with experience in
> stream processing systems are in the Samza community. This leads me to
> thinking that merging projects and communities might be a good idea: with
> the union of experience from both communities, we will probably build a
> better system that is better for users.
>
> Jakob advocated maintaining support for input sources other than Kafka.
> While I can totally see the need for a framework that does this, I think
> the need is pretty well satisfied by Storm, which already has spouts for
> Kafka, Kestrel, JMS, AMQP, Redis and beanstalkd (and perhaps more). I don't
> see much value in Samza attempting to catch up here, especially if Copycat
> will provide connectors to many systems by different means. On the other
> hand, my failed attempts to implement SystemConsumers for Kinesis and
> Postgres make me think that a stream processor that supports many different
> inputs is limited to a lowest-common-denominator model; if Samza supports
> only Kafka, I think it could support Kafka better than any other framework
> (by doing one thing and doing it well).
>
> Julian: not sure I understand your point 2 about departing from the vision
> of distributed processing. A library-ified Samza would still allow
> distributed processing, and (with a small amount of glue) could still be
> deployed to YARN or other cluster.
>
> So, in conclusion, I'm starting to agree with the approach that Jay has
> been advocating in this thread.
>
> Martin
>
>
> On 9 Jul 2015, at 15:32, Julian Hyde <jh...@apache.org> wrote:
>
> > Wow, what a great discussion. A brave discussion, since no project
> > wants to reduce its scope. And important, because "right-sizing"
> > technology components can help them win in the long run.
> >
> > I have a couple of let-me-play-devil's-advocate questions.
> >
> > 1. Community over code
> >
> > Let's look at this in terms of the Apache Way. The Apache Way
> > advocates "community over code", and as Jakob points out, the Samza
> > community is distinct from the Kafka community. It seems that we are
> > talking here about Samza-the-code.
> >
> > According to the Apache Way, what Samza-the-project should be doing is
> > what Samza-the-community is good at. Samza-the-code-as-it-is-today can
> > move to Kafka, stay in Samza, or be deleted if it has been superseded.
> >
> > Architectural discussions are important to have, and the Apache Way
> > gets in the way of good architecture sometimes. When we're thinking
> > about moving code, let's also think about the community of people
> > working on the code.
> >
> > Apache Phoenix is a good analogy. Phoenix is technically very closely
> > tied to HBase, but a distinct community, with different skill-sets.
> > (HBase, like Kafka, is "hard core", and not for everyone.) They have
> > also been good at re-examining their project scope and re-scoping
> > where necessary.
> >
> > 2. Architecture
> >
> > This proposal retreats from the grand vision of "distributed stream
> > management system" where not only storage is distributed but also
> > processing. There is no architectural piece that says "I need 10 JVMs
> > to process this CPU intensive standing query and I currently only have
> > 6." What projects, current or envisioned, would fit that gap? Is that
> > work a good fit for the Samza community?
> >
> > Julian
> >
> >
> >
> > On Wed, Jul 8, 2015 at 10:47 PM, Jordan Shaw <jo...@pubnub.com> wrote:
> >> I'm all for any optimizations that can be made to the Yarn workflow.
> >>
> >> I actually agree with Jakob in regard to the producers/consumers. I have
> >> spent sometime writing consumers and producers for other transport
> >> abstractions and overall the current api abstractions in Samza I feel
> are
> >> pretty good. There are some things that are sort of anomalous and
> catered
> >> more toward the Kafka model but easy enough to work around and I've been
> >> able to make other Producers and Consumers work that are no where near
> the
> >> same paradigm as Kafka.
> >>
> >> To Jay's point although Kafka is great and does the streaming data
> paradigm
> >> very well there is really no reason why a different transport
> application
> >> implemented properly wouldn't be able to stream data with the same
> >> effectiveness as Kafka and that transport may suite the user's use case
> >> better or be more cost effective than Kafka. For example we had to
> decide
> >> if Kafka was worth the extra cost of running a zookeeper cluster and if
> the
> >> scaling through partitioning was worth the operational overhead vs
> having a
> >> mesh network over ZeroMQ. After deciding that our use case would fit
> with
> >> Kafka fine there were other challenges like understanding how AWS EC2
> SSD's
> >> behaved (AWS amortizes all disk into into Random I/O this is bad for
> Kafka).
> >>
> >> Thus, I would lend of the side of transport flexibility for a framework
> >> like Samza over binding to a transport medium like Kafka.
> >>
> >>
> >> On Wed, Jul 8, 2015 at 1:39 PM, Jay Kreps <ja...@gmail.com> wrote:
> >>
> >>> Good summary Jakob.
> >>>
> >>> WRT to the general purpose vs Kafka-specific, I actually see it
> slightly
> >>> differently. Consider how Storm works as an example, there is a data
> source
> >>> (spout) which could be Kafka, Database, etc, and then there is a
> transport
> >>> (a netty TCP thing iiuc). Storm allows you to process data from any
> source,
> >>> but when it comes from a source they always funnel it through their
> >>> transport to get to the tasks/bolts. It is natural to think of Kafka
> as the
> >>> Spout, but I think the better analogy is actually that Kafka is the
> >>> transport.
> >>>
> >>> It is really hard to make the transport truly pluggable because this is
> >>> what the tasks interact with and you need to have guarantees about
> delivery
> >>> (and reprocessing), partitioning, atomicity of output, ordering, etc so
> >>> your stream processing can get the right answer. From my point of view
> what
> >>> this proposal says is that Kafka would be non-pluggable as the
> *transport*.
> >>>
> >>> So in this proposal data would still come into and out of Kafka from a
> wide
> >>> variety of sources, but by requiring Kafka as the transport the
> interaction
> >>> with the tasks will always look the same (a persistent, partitioned,
> log).
> >>> So going back to the Storm analogy it is something like
> >>>  Spout interface = copy cat
> >>>  Bolt interface = samza
> >>>
> >>> This does obviously make Samza dependent on Kafka but it doesn't mean
> you
> >>> wouldn't be processing data from all kinds of sources--indeed that is
> the
> >>> whole purpose. It just means that each of these data streams would be
> >>> available as a multi-subscriber Kafka topic to other systems,
> applications,
> >>> etc, not just for your job.
> >>>
> >>> If you think about how things are now Samza already depends on a
> >>> partitioned, persistent, offset addressable log with log
> >>> compaction...which, unsurprisingly, so I don't think this is really a
> new
> >>> dependency.
> >>>
> >>> Philosophically I think this makes sense too. To make a bunch of
> programs
> >>> fit together you have to standardize something. In this proposal what
> you
> >>> are standardizing around is really Kafka's protocol for streaming data
> and
> >>> your data format. The transformations that connect these streams can be
> >>> done via Samza, Storm, Spark, standalone java or python programs, etc
> but
> >>> the ultimate output and contract to the rest of the organization/world
> will
> >>> be the resulting Kafka topic. Philosophically I think this kind of
> data and
> >>> protocol based contract is the right way to go rather than saying that
> the
> >>> contract is a particular java api and the stream/data is what is
> pluggable.
> >>>
> >>> -Jay
> >>>
> >>>
> >>>
> >>> On Wed, Jul 8, 2015 at 11:03 AM, Jakob Homan <jg...@gmail.com>
> wrote:
> >>>
> >>>> Rewinding back to the beginning of this topic, there are effectively
> >>>> three proposals on the table:
> >>>>
> >>>> 1) Chris' ideas for a direction towards a 2.0 release with an emphasis
> >>>> on API and configuration simplification.  This ideas are based on lots
> >>>> of lessons learned from the 0.x branch and are worthy of a 2.0 label
> >>>> and breaking backwards compability.  I'm not sure I agree with all of
> >>>> them, but they're definitely worth pursuing.
> >>>>
> >>>> 2) Chris' alternative proposal, which goes beyond his first and is
> >>>> essentially a reboot of Samza to a more limited, entirely
> >>>> Kafka-focused approach.  Samza would cease being a general purpose
> >>>> stream processing framework, akin to and an alternative to say, Apache
> >>>> Storm, and would instead become a standalone complement to the Kafka
> >>>> project.
> >>>>
> >>>> 3) Jay's proposal, which goes even further, and suggests that the
> >>>> Kafka community would be better served by adding stream processing as
> >>>> a module to Kafka.  This is a perfectly valid approach, but since it's
> >>>> entirely confined to the Kafka project, doesn't really involve Samza.
> >>>> If the Kafka team were to go this route, there would be no obligation
> >>>> on the Samza team to shut down, disband, etc.
> >>>>
> >>>> This last bit is important because Samza and Kafka, while closely
> >>>> linked, are distinct communities.  The intersection of committers on
> >>>> both Kafka and Samza is three people out of a combined 18 committers
> >>>> across both projects.   Samza is a distinct community that shares
> >>>> quite a few users with Kafka, but is able to chart its own course.
> >>>>
> >>>> My own view is that Samza has had an amazing year and is taking off at
> >>>> a rapid rate.  It was only proposed for Incubator two years ago and is
> >>>> still very young. The original team at LinkedIn has left that company
> >>>> but the project has continued to grow via contributions both from
> >>>> LinkedIn and from without.  We've recently seen a significant uptake
> >>>> in discussion and bug reports.
> >>>>
> >>>> The API, deployment and configuration changes Chris suggests are good
> >>>> ideas, but I think there is still serious value in having a
> >>>> stand-alone general stream processing framework that supports other
> >>>> input sources than Kafka.  We've already had contributions for adding
> >>>> producer support to ElasticSearch and HDFS.  As more users come on
> >>>> board, I would expect them to contribute more consumers and producers.
> >>>>
> >>>> It's a bit of chicken-and-the-egg problem; since the original team
> >>>> didn't have cycles to prioritize support for non-Kafka systems
> >>>> (kinesis, eventhub, twitter, flume, zeromq, etc.), Samza was less
> >>>> compelling than other stream processing frameworks that did have
> >>>> support and was therefore not used in those situations.  I'd love to
> >>>> see those added and the SystemConsumer/Producer APIs improved to
> >>>> fluently support them as well as Kafka.
> >>>> Martin had a question regarding the tight coupling between Hadoop HDFS
> >>>> and MapReduce (and YARN and Common).  This has been a problem for
> >>>> years and there have been several aborted attempts to split the
> >>>> projects out.  Each time there turned out to be a strong need for
> >>>> cross-cutting collaboration and so the effort was dropped.  Absent the
> >>>> third option above (Kafka adding stream support to itself directly), I
> >>>> would imagine something similar would play out here.
> >>>>
> >>>> We should get a feeling for which of the three proposals the Samza
> >>>> community is behind, technical details of each notwithstanding.  This
> >>>> would include not just the committers/PMC members, but also the users,
> >>>> contributors and lurkers.
> >>>>
> >>>> -Jakob
> >>>>
> >>>> On 8 July 2015 at 07:41, Ben Kirwin <be...@kirw.in> wrote:
> >>>>> Hi all,
> >>>>>
> >>>>> Interesting stuff! Jumping in a bit late, but here goes...
> >>>>>
> >>>>> I'd definitely be excited about a slimmed-down and more
> Kafka-specific
> >>>>> Samza -- you don't seem to lose much functionality that people
> >>>>> actually use, and the gains in simplicity / code sharing seem
> >>>>> potentially very large. (I've spent a bunch of time peeling back
> those
> >>>>> layers of abstraction to get eg. more control over message send
> order,
> >>>>> and working directly against Kafka's APIs would have been much
> >>>>> easier.) I also like the approach of letting Kafka code do the heavy
> >>>>> lifting and letting stream processing systems build on those -- good,
> >>>>> reusable implementations would be great for the whole
> >>>>> stream-processing ecosystem, and Samza in particular.
> >>>>>
> >>>>> On the other hand, I do hope that using Kafka's group membership /
> >>>>> partition assignment / etc. stays optional. As far as I can tell,
> >>>>> ~every major stream processing system that uses Kafka has chosen (or
> >>>>> switched to) 'static' partitioning, where each logical task consumes
> a
> >>>>> fixed set of partitions. When 'dynamic deploying' (a la Storm / Mesos
> >>>>> / Yarn) the underlying system is already doing failure detection and
> >>>>> transferring work between hosts when machines go down, so using
> >>>>> Kafka's implementation is redundant at best -- and at worst, the
> >>>>> interaction between the two systems can make outages worse.
> >>>>>
> >>>>> And thanks to Chris / Jay for getting this ball rolling. Exciting
> >>>> times...
> >>>>>
> >>>>> On Tue, Jul 7, 2015 at 2:35 PM, Jay Kreps <ja...@confluent.io> wrote:
> >>>>>> Hey Roger,
> >>>>>>
> >>>>>> I couldn't agree more. We spent a bunch of time talking to people
> and
> >>>> that
> >>>>>> is exactly the stuff we heard time and again. What makes it hard, of
> >>>>>> course, is that there is some tension between compatibility with
> >>> what's
> >>>>>> there now and making things better for new users.
> >>>>>>
> >>>>>> I also strongly agree with the importance of multi-language support.
> >>> We
> >>>> are
> >>>>>> talking now about Java, but for application development use cases
> >>> people
> >>>>>> want to work in whatever language they are using elsewhere. I think
> >>>> moving
> >>>>>> to a model where Kafka itself does the group membership, lifecycle
> >>>> control,
> >>>>>> and partition assignment has the advantage of putting all that
> complex
> >>>>>> stuff behind a clean api that the clients are already going to be
> >>>>>> implementing for their consumer, so the added functionality for
> stream
> >>>>>> processing beyond a consumer becomes very minor.
> >>>>>>
> >>>>>> -Jay
> >>>>>>
> >>>>>> On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> roger.hoover@gmail.com
> >>>>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Metamorphosis...nice. :)
> >>>>>>>
> >>>>>>> This has been a great discussion.  As a user of Samza who's
> recently
> >>>>>>> integrated it into a relatively large organization, I just want to
> >>> add
> >>>>>>> support to a few points already made.
> >>>>>>>
> >>>>>>> The biggest hurdles to adoption of Samza as it currently exists
> that
> >>>> I've
> >>>>>>> experienced are:
> >>>>>>> 1) YARN - YARN is overly complex in many environments where Puppet
> >>>> would do
> >>>>>>> just fine but it was the only mechanism to get fault tolerance.
> >>>>>>> 2) Configuration - I think I like the idea of configuring most of
> the
> >>>> job
> >>>>>>> in code rather than config files.  In general, I think the goal
> >>> should
> >>>> be
> >>>>>>> to make it harder to make mistakes, especially of the kind where
> the
> >>>> code
> >>>>>>> expects something and the config doesn't match.  The current config
> >>> is
> >>>>>>> quite intricate and error-prone.  For example, the application
> logic
> >>>> may
> >>>>>>> depend on bootstrapping a topic but rather than asserting that in
> the
> >>>> code,
> >>>>>>> you have to rely on getting the config right.  Likewise with
> serdes,
> >>>> the
> >>>>>>> Java representations produced by various serdes (JSON, Avro, etc.)
> >>> are
> >>>> not
> >>>>>>> equivalent so you cannot just reconfigure a serde without changing
> >>> the
> >>>>>>> code.   It would be nice for jobs to be able to assert what they
> >>> expect
> >>>>>>> from their input topics in terms of partitioning.  This is getting
> a
> >>>> little
> >>>>>>> off topic but I was even thinking about creating a "Samza config
> >>>> linter"
> >>>>>>> that would sanity check a set of configs.  Especially in
> >>> organizations
> >>>>>>> where config is managed by a different team than the application
> >>>> developer,
> >>>>>>> it's very hard to get avoid config mistakes.
> >>>>>>> 3) Java/Scala centric - for many teams (especially DevOps-type
> >>> folks),
> >>>> the
> >>>>>>> pain of the Java toolchain (maven, slow builds, weak command line
> >>>> support,
> >>>>>>> configuration over convention) really inhibits productivity.  As
> more
> >>>> and
> >>>>>>> more high-quality clients become available for Kafka, I hope
> they'll
> >>>> follow
> >>>>>>> Samza's model.  Not sure how much it affects the proposals in this
> >>>> thread
> >>>>>>> but please consider other languages in the ecosystem as well.  From
> >>>> what
> >>>>>>> I've heard, Spark has more Python users than Java/Scala.
> >>>>>>> (FYI, we added a Jython wrapper for the Samza API
> >>>>>>>
> >>>>>>>
> >>>>
> >>>
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> >>>>>>> and are working on a Yeoman generator
> >>>>>>> https://github.com/Quantiply/generator-rico for Jython/Samza
> >>> projects
> >>>> to
> >>>>>>> alleviate some of the pain)
> >>>>>>>
> >>>>>>> I also want to underscore Jay's point about improving the user
> >>>> experience.
> >>>>>>> That's a very important factor for adoption.  I think the goal
> should
> >>>> be to
> >>>>>>> make Samza as easy to get started with as something like Logstash.
> >>>>>>> Logstash is vastly inferior in terms of capabilities to Samza but
> >>> it's
> >>>> easy
> >>>>>>> to get started and that makes a big difference.
> >>>>>>>
> >>>>>>> Cheers,
> >>>>>>>
> >>>>>>> Roger
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
> >>>>>>> gdfm@apache.org> wrote:
> >>>>>>>
> >>>>>>>> Forgot to add. On the naming issues, Kafka Metamorphosis is a
> clear
> >>>>>>> winner
> >>>>>>>> :)
> >>>>>>>>
> >>>>>>>> --
> >>>>>>>> Gianmarco
> >>>>>>>>
> >>>>>>>> On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
> >>>> gdfm@apache.org
> >>>>>>>>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi,
> >>>>>>>>>
> >>>>>>>>> @Martin, thanks for you comments.
> >>>>>>>>> Maybe I'm missing some important point, but I think coupling the
> >>>>>>> releases
> >>>>>>>>> is actually a *good* thing.
> >>>>>>>>> To make an example, would it be better if the MR and HDFS
> >>>> components of
> >>>>>>>>> Hadoop had different release schedules?
> >>>>>>>>>
> >>>>>>>>> Actually, keeping the discussion in a single place would make
> >>>> agreeing
> >>>>>>> on
> >>>>>>>>> releases (and backwards compatibility) much easier, as everybody
> >>>> would
> >>>>>>> be
> >>>>>>>>> responsible for the whole codebase.
> >>>>>>>>>
> >>>>>>>>> That said, I like the idea of absorbing samza-core as a
> >>>> sub-project,
> >>>>>>> and
> >>>>>>>>> leave the fancy stuff separate.
> >>>>>>>>> It probably gives 90% of the benefits we have been discussing
> >>> here.
> >>>>>>>>>
> >>>>>>>>> Cheers,
> >>>>>>>>>
> >>>>>>>>> --
> >>>>>>>>> Gianmarco
> >>>>>>>>>
> >>>>>>>>> On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hey Martin,
> >>>>>>>>>>
> >>>>>>>>>> I agree coupling release schedules is a downside.
> >>>>>>>>>>
> >>>>>>>>>> Definitely we can try to solve some of the integration problems
> >>> in
> >>>>>>>>>> Confluent Platform or in other distributions. But I think this
> >>>> ends up
> >>>>>>>>>> being really shallow. I guess I feel to really get a good user
> >>>>>>>> experience
> >>>>>>>>>> the two systems have to kind of feel like part of the same thing
> >>>> and
> >>>>>>> you
> >>>>>>>>>> can't really add that in later--you can put both in the same
> >>>>>>>> downloadable
> >>>>>>>>>> tar file but it doesn't really give a very cohesive feeling. I
> >>>> agree
> >>>>>>>> that
> >>>>>>>>>> ultimately any of the project stuff is as much social and naming
> >>>> as
> >>>>>>>>>> anything else--theoretically two totally independent projects
> >>>> could
> >>>>>>> work
> >>>>>>>>>> to
> >>>>>>>>>> tightly align. In practice this seems to be quite difficult
> >>>> though.
> >>>>>>>>>>
> >>>>>>>>>> For the frameworks--totally agree it would be good to maintain
> >>> the
> >>>>>>>>>> framework support with the project. In some cases there may not
> >>>> be too
> >>>>>>>>>> much
> >>>>>>>>>> there since the integration gets lighter but I think whatever
> >>>> stubs
> >>>>>>> you
> >>>>>>>>>> need should be included. So no I definitely wasn't trying to
> >>> imply
> >>>>>>>>>> dropping
> >>>>>>>>>> support for these frameworks, just making the integration
> >>> lighter
> >>>> by
> >>>>>>>>>> separating process management from partition management.
> >>>>>>>>>>
> >>>>>>>>>> You raise two good points we would have to figure out if we went
> >>>> down
> >>>>>>>> the
> >>>>>>>>>> alignment path:
> >>>>>>>>>> 1. With respect to the name, yeah I think the first question is
> >>>>>>> whether
> >>>>>>>>>> some "re-branding" would be worth it. If so then I think we can
> >>>> have a
> >>>>>>>> big
> >>>>>>>>>> thread on the name. I'm definitely not set on Kafka Streaming or
> >>>> Kafka
> >>>>>>>>>> Streams I was just using them to be kind of illustrative. I
> >>> agree
> >>>> with
> >>>>>>>>>> your
> >>>>>>>>>> critique of these names, though I think people would get the
> >>> idea.
> >>>>>>>>>> 2. Yeah you also raise a good point about how to "factor" it.
> >>>> Here are
> >>>>>>>> the
> >>>>>>>>>> options I see (I could get enthusiastic about any of them):
> >>>>>>>>>>   a. One repo for both Kafka and Samza
> >>>>>>>>>>   b. Two repos, retaining the current seperation
> >>>>>>>>>>   c. Two repos, the equivalent of samza-api and samza-core is
> >>>>>>> absorbed
> >>>>>>>>>> almost like a third client
> >>>>>>>>>>
> >>>>>>>>>> Cheers,
> >>>>>>>>>>
> >>>>>>>>>> -Jay
> >>>>>>>>>>
> >>>>>>>>>> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> >>>>>>> martin@kleppmann.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Ok, thanks for the clarifications. Just a few follow-up
> >>>> comments.
> >>>>>>>>>>>
> >>>>>>>>>>> - I see the appeal of merging with Kafka or becoming a
> >>>> subproject:
> >>>>>>> the
> >>>>>>>>>>> reasons you mention are good. The risk I see is that release
> >>>>>>> schedules
> >>>>>>>>>>> become coupled to each other, which can slow everyone down,
> >>> and
> >>>>>>> large
> >>>>>>>>>>> projects with many contributors are harder to manage. (Jakob,
> >>>> can
> >>>>>>> you
> >>>>>>>>>> speak
> >>>>>>>>>>> from experience, having seen a wider range of Hadoop ecosystem
> >>>>>>>>>> projects?)
> >>>>>>>>>>>
> >>>>>>>>>>> Some of the goals of a better unified developer experience
> >>> could
> >>>>>>> also
> >>>>>>>> be
> >>>>>>>>>>> solved by integrating Samza nicely into a Kafka distribution
> >>>> (such
> >>>>>>> as
> >>>>>>>>>>> Confluent's). I'm not against merging projects if we decide
> >>>> that's
> >>>>>>> the
> >>>>>>>>>> way
> >>>>>>>>>>> to go, just pointing out the same goals can perhaps also be
> >>>> achieved
> >>>>>>>> in
> >>>>>>>>>>> other ways.
> >>>>>>>>>>>
> >>>>>>>>>>> - With regard to dropping the YARN dependency: are you
> >>> proposing
> >>>>>>> that
> >>>>>>>>>>> Samza doesn't give any help to people wanting to run on
> >>>>>>>>>> YARN/Mesos/AWS/etc?
> >>>>>>>>>>> So the docs would basically have a link to Slider and nothing
> >>>> else?
> >>>>>>> Or
> >>>>>>>>>>> would we maintain integrations with a bunch of popular
> >>>> deployment
> >>>>>>>>>> methods
> >>>>>>>>>>> (e.g. the necessary glue and shell scripts to make Samza work
> >>>> with
> >>>>>>>>>> Slider)?
> >>>>>>>>>>>
> >>>>>>>>>>> I absolutely think it's a good idea to have the "as a library"
> >>>> and
> >>>>>>>> "as a
> >>>>>>>>>>> process" (using Yi's taxonomy) options for people who want
> >>> them,
> >>>>>>> but I
> >>>>>>>>>>> think there should also be a low-friction path for common "as
> >>> a
> >>>>>>>> service"
> >>>>>>>>>>> deployment methods, for which we probably need to maintain
> >>>>>>>> integrations.
> >>>>>>>>>>>
> >>>>>>>>>>> - Project naming: "Kafka Streams" seems odd to me, because
> >>>> Kafka is
> >>>>>>>> all
> >>>>>>>>>>> about streams already. Perhaps "Kafka Transformers" or "Kafka
> >>>>>>> Filters"
> >>>>>>>>>>> would be more apt?
> >>>>>>>>>>>
> >>>>>>>>>>> One suggestion: perhaps the core of Samza (stream
> >>> transformation
> >>>>>>> with
> >>>>>>>>>>> state management -- i.e. the "Samza as a library" bit) could
> >>>> become
> >>>>>>>>>> part of
> >>>>>>>>>>> Kafka, while higher-level tools such as streaming SQL and
> >>>>>>> integrations
> >>>>>>>>>> with
> >>>>>>>>>>> deployment frameworks remain in a separate project? In other
> >>>> words,
> >>>>>>>>>> Kafka
> >>>>>>>>>>> would absorb the proven, stable core of Samza, which would
> >>>> become
> >>>>>>> the
> >>>>>>>>>>> "third Kafka client" mentioned early in this thread. The Samza
> >>>>>>> project
> >>>>>>>>>>> would then target that third Kafka client as its base API, and
> >>>> the
> >>>>>>>>>> project
> >>>>>>>>>>> would be freed up to explore more experimental new horizons.
> >>>>>>>>>>>
> >>>>>>>>>>> Martin
> >>>>>>>>>>>
> >>>>>>>>>>> On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com>
> >>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hey Martin,
> >>>>>>>>>>>>
> >>>>>>>>>>>> For the YARN/Mesos/etc decoupling I actually don't think it
> >>>> ties
> >>>>>>> our
> >>>>>>>>>>> hands
> >>>>>>>>>>>> at all, all it does is refactor things. The division of
> >>>>>>>>>> responsibility is
> >>>>>>>>>>>> that Samza core is responsible for task lifecycle, state,
> >>> and
> >>>>>>>>>> partition
> >>>>>>>>>>>> management (using the Kafka co-ordinator) but it is NOT
> >>>>>>> responsible
> >>>>>>>>>> for
> >>>>>>>>>>>> packaging, configuration deployment or execution of
> >>>> processes. The
> >>>>>>>>>>> problem
> >>>>>>>>>>>> of packaging and starting these processes is
> >>>>>>>>>>>> framework/environment-specific. This leaves individual
> >>>> frameworks
> >>>>>>> to
> >>>>>>>>>> be
> >>>>>>>>>>> as
> >>>>>>>>>>>> fancy or vanilla as they like. So you can get simple
> >>> stateless
> >>>>>>>>>> support in
> >>>>>>>>>>>> YARN, Mesos, etc using their off-the-shelf app framework
> >>>> (Slider,
> >>>>>>>>>>> Marathon,
> >>>>>>>>>>>> etc). These are well known by people and have nice UIs and a
> >>>> lot
> >>>>>>> of
> >>>>>>>>>>>> flexibility. I don't think they have node affinity as a
> >>> built
> >>>> in
> >>>>>>>>>> option
> >>>>>>>>>>>> (though I could be wrong). So if we want that we can either
> >>>> wait
> >>>>>>> for
> >>>>>>>>>> them
> >>>>>>>>>>>> to add it or do a custom framework to add that feature (as
> >>>> now).
> >>>>>>>>>>> Obviously
> >>>>>>>>>>>> if you manage things with old-school ops tools
> >>>> (puppet/chef/etc)
> >>>>>>> you
> >>>>>>>>>> get
> >>>>>>>>>>>> locality easily. The nice thing, though, is that all the
> >>> samza
> >>>>>>>>>> "business
> >>>>>>>>>>>> logic" around partition management and fault tolerance is in
> >>>> Samza
> >>>>>>>>>> core
> >>>>>>>>>>> so
> >>>>>>>>>>>> it is shared across frameworks and the framework specific
> >>> bit
> >>>> is
> >>>>>>>> just
> >>>>>>>>>>>> whether it is smart enough to try to get the same host when
> >>> a
> >>>> job
> >>>>>>> is
> >>>>>>>>>>>> restarted.
> >>>>>>>>>>>>
> >>>>>>>>>>>> With respect to the Kafka-alignment, yeah I think the goal
> >>>> would
> >>>>>>> be
> >>>>>>>>>> (a)
> >>>>>>>>>>>> actually get better alignment in user experience, and (b)
> >>>> express
> >>>>>>>>>> this in
> >>>>>>>>>>>> the naming and project branding. Specifically:
> >>>>>>>>>>>> 1. Website/docs, it would be nice for the "transformation"
> >>>> api to
> >>>>>>> be
> >>>>>>>>>>>> discoverable in the main Kafka docs--i.e. be able to explain
> >>>> when
> >>>>>>> to
> >>>>>>>>>> use
> >>>>>>>>>>>> the consumer and when to use the stream processing
> >>>> functionality
> >>>>>>> and
> >>>>>>>>>> lead
> >>>>>>>>>>>> people into that experience.
> >>>>>>>>>>>> 2. Align releases so if you get Kafkza 1.4.2 (or whatever)
> >>>> that
> >>>>>>> has
> >>>>>>>>>> both
> >>>>>>>>>>>> Kafka and the stream processing part and they actually work
> >>>>>>>> together.
> >>>>>>>>>>>> 3. Unify the programming experience so the client and Samza
> >>>> api
> >>>>>>>> share
> >>>>>>>>>>>> config/monitoring/naming/packaging/etc.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I think sub-projects keep separate committers and can have a
> >>>>>>>> separate
> >>>>>>>>>>> repo,
> >>>>>>>>>>>> but I'm actually not really sure (I can't find a definition
> >>>> of a
> >>>>>>>>>>> subproject
> >>>>>>>>>>>> in Apache).
> >>>>>>>>>>>>
> >>>>>>>>>>>> Basically at a high-level you want the experience to "feel"
> >>>> like a
> >>>>>>>>>> single
> >>>>>>>>>>>> system, not to relatively independent things that are kind
> >>> of
> >>>>>>>>>> awkwardly
> >>>>>>>>>>>> glued together.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I think if we did that they having naming or branding like
> >>>> "kafka
> >>>>>>>>>>>> streaming" or "kafka streams" or something like that would
> >>>>>>> actually
> >>>>>>>>>> do a
> >>>>>>>>>>>> good job of conveying what it is. I do that this would help
> >>>>>>> adoption
> >>>>>>>>>>> quite
> >>>>>>>>>>>> a lot as it would correctly convey that using Kafka
> >>> Streaming
> >>>> with
> >>>>>>>>>> Kafka
> >>>>>>>>>>> is
> >>>>>>>>>>>> a fairly seamless experience and Kafka is pretty heavily
> >>>> adopted
> >>>>>>> at
> >>>>>>>>>> this
> >>>>>>>>>>>> point.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Fwiw we actually considered this model originally when open
> >>>>>>> sourcing
> >>>>>>>>>>> Samza,
> >>>>>>>>>>>> however at that time Kafka was relatively unknown and we
> >>>> decided
> >>>>>>> not
> >>>>>>>>>> to
> >>>>>>>>>>> do
> >>>>>>>>>>>> it since we felt it would be limiting. From my point of view
> >>>> the
> >>>>>>>> three
> >>>>>>>>>>>> things have changed (1) Kafka is now really heavily used for
> >>>>>>> stream
> >>>>>>>>>>>> processing, (2) we learned that abstracting out the stream
> >>>> well is
> >>>>>>>>>>>> basically impossible, (3) we learned it is really hard to
> >>>> keep the
> >>>>>>>> two
> >>>>>>>>>>>> things feeling like a single product.
> >>>>>>>>>>>>
> >>>>>>>>>>>> -Jay
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> >>>>>>>>>> martin@kleppmann.com>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Lots of good thoughts here.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I agree with the general philosophy of tying Samza more
> >>>> firmly to
> >>>>>>>>>> Kafka.
> >>>>>>>>>>>>> After I spent a while looking at integrating other message
> >>>>>>> brokers
> >>>>>>>>>> (e.g.
> >>>>>>>>>>>>> Kinesis) with SystemConsumer, I came to the conclusion that
> >>>>>>>>>>> SystemConsumer
> >>>>>>>>>>>>> tacitly assumes a model so much like Kafka's that pretty
> >>> much
> >>>>>>>> nobody
> >>>>>>>>>> but
> >>>>>>>>>>>>> Kafka actually implements it. (Databus is perhaps an
> >>>> exception,
> >>>>>>> but
> >>>>>>>>>> it
> >>>>>>>>>>>>> isn't widely used outside of LinkedIn.) Thus, making Samza
> >>>> fully
> >>>>>>>>>>> dependent
> >>>>>>>>>>>>> on Kafka acknowledges that the system-independence was
> >>> never
> >>>> as
> >>>>>>>> real
> >>>>>>>>>> as
> >>>>>>>>>>> we
> >>>>>>>>>>>>> perhaps made it out to be. The gains of code reuse are
> >>> real.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> The idea of decoupling Samza from YARN has also always been
> >>>>>>>>>> appealing to
> >>>>>>>>>>>>> me, for various reasons already mentioned in this thread.
> >>>>>>> Although
> >>>>>>>>>>> making
> >>>>>>>>>>>>> Samza jobs deployable on anything (YARN/Mesos/AWS/etc)
> >>> seems
> >>>>>>>>>> laudable,
> >>>>>>>>>>> I am
> >>>>>>>>>>>>> a little concerned that it will restrict us to a lowest
> >>>> common
> >>>>>>>>>>> denominator.
> >>>>>>>>>>>>> For example, would host affinity (SAMZA-617) still be
> >>>> possible?
> >>>>>>> For
> >>>>>>>>>> jobs
> >>>>>>>>>>>>> with large amounts of state, I think SAMZA-617 would be a
> >>> big
> >>>>>>> boon,
> >>>>>>>>>>> since
> >>>>>>>>>>>>> restoring state off the changelog on every single restart
> >>> is
> >>>>>>>> painful,
> >>>>>>>>>>> due
> >>>>>>>>>>>>> to long recovery times. It would be a shame if the
> >>> decoupling
> >>>>>>> from
> >>>>>>>>>> YARN
> >>>>>>>>>>>>> made host affinity impossible.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Jay, a question about the proposed API for instantiating a
> >>>> job in
> >>>>>>>>>> code
> >>>>>>>>>>>>> (rather than a properties file): when submitting a job to a
> >>>>>>>> cluster,
> >>>>>>>>>> is
> >>>>>>>>>>> the
> >>>>>>>>>>>>> idea that the instantiation code runs on a client
> >>> somewhere,
> >>>>>>> which
> >>>>>>>>>> then
> >>>>>>>>>>>>> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or
> >>> does
> >>>> that
> >>>>>>>>>> code
> >>>>>>>>>>> run
> >>>>>>>>>>>>> on each container that is part of the job (in which case,
> >>> how
> >>>>>>> does
> >>>>>>>>>> the
> >>>>>>>>>>> job
> >>>>>>>>>>>>> submission to the cluster work)?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I agree with Garry that it doesn't feel right to make a 1.0
> >>>>>>> release
> >>>>>>>>>>> with a
> >>>>>>>>>>>>> plan for it to be immediately obsolete. So if this is going
> >>>> to
> >>>>>>>>>> happen, I
> >>>>>>>>>>>>> think it would be more honest to stick with 0.* version
> >>>> numbers
> >>>>>>>> until
> >>>>>>>>>>> the
> >>>>>>>>>>>>> library-ified Samza has been implemented, is stable and
> >>>> widely
> >>>>>>>> used.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Should the new Samza be a subproject of Kafka? There is
> >>>> precedent
> >>>>>>>> for
> >>>>>>>>>>>>> tight coupling between different Apache projects (e.g.
> >>>> Curator
> >>>>>>> and
> >>>>>>>>>>>>> Zookeeper, or Slider and YARN), so I think remaining
> >>> separate
> >>>>>>> would
> >>>>>>>>>> be
> >>>>>>>>>>> ok.
> >>>>>>>>>>>>> Even if Samza is fully dependent on Kafka, there is enough
> >>>>>>>> substance
> >>>>>>>>>> in
> >>>>>>>>>>>>> Samza that it warrants being a separate project. An
> >>> argument
> >>>> in
> >>>>>>>>>> favour
> >>>>>>>>>>> of
> >>>>>>>>>>>>> merging would be if we think Kafka has a much stronger
> >>> "brand
> >>>>>>>>>> presence"
> >>>>>>>>>>>>> than Samza; I'm ambivalent on that one. If the Kafka
> >>> project
> >>>> is
> >>>>>>>>>> willing
> >>>>>>>>>>> to
> >>>>>>>>>>>>> endorse Samza as the "official" way of doing stateful
> >>> stream
> >>>>>>>>>>>>> transformations, that would probably have much the same
> >>>> effect as
> >>>>>>>>>>>>> re-branding Samza as "Kafka Stream Processors" or suchlike.
> >>>> Close
> >>>>>>>>>>>>> collaboration between the two projects will be needed in
> >>> any
> >>>>>>> case.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> From a project management perspective, I guess the "new
> >>>> Samza"
> >>>>>>>> would
> >>>>>>>>>>> have
> >>>>>>>>>>>>> to be developed on a branch alongside ongoing maintenance
> >>> of
> >>>> the
> >>>>>>>>>> current
> >>>>>>>>>>>>> line of development? I think it would be important to
> >>>> continue
> >>>>>>>>>>> supporting
> >>>>>>>>>>>>> existing users, and provide a graceful migration path to
> >>> the
> >>>> new
> >>>>>>>>>>> version.
> >>>>>>>>>>>>> Leaving the current versions unsupported and forcing people
> >>>> to
> >>>>>>>>>> rewrite
> >>>>>>>>>>>>> their jobs would send a bad signal.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Best,
> >>>>>>>>>>>>> Martin
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io>
> >>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hey Garry,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Yeah that's super frustrating. I'd be happy to chat more
> >>>> about
> >>>>>>>> this
> >>>>>>>>>> if
> >>>>>>>>>>>>>> you'd be interested. I think Chris and I started with the
> >>>> idea
> >>>>>>> of
> >>>>>>>>>> "what
> >>>>>>>>>>>>>> would it take to make Samza a kick-ass ingestion tool" but
> >>>>>>>>>> ultimately
> >>>>>>>>>>> we
> >>>>>>>>>>>>>> kind of came around to the idea that ingestion and
> >>>>>>> transformation
> >>>>>>>>>> had
> >>>>>>>>>>>>>> pretty different needs and coupling the two made things
> >>>> hard.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> For what it's worth I think copycat (KIP-26) actually will
> >>>> do
> >>>>>>> what
> >>>>>>>>>> you
> >>>>>>>>>>>>> are
> >>>>>>>>>>>>>> looking for.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> With regard to your point about slider, I don't
> >>> necessarily
> >>>>>>>>>> disagree.
> >>>>>>>>>>>>> But I
> >>>>>>>>>>>>>> think getting good YARN support is quite doable and I
> >>> think
> >>>> we
> >>>>>>> can
> >>>>>>>>>> make
> >>>>>>>>>>>>>> that work well. I think the issue this proposal solves is
> >>>> that
> >>>>>>>>>>>>> technically
> >>>>>>>>>>>>>> it is pretty hard to support multiple cluster management
> >>>> systems
> >>>>>>>> the
> >>>>>>>>>>> way
> >>>>>>>>>>>>>> things are now, you need to write an "app master" or
> >>>> "framework"
> >>>>>>>> for
> >>>>>>>>>>> each
> >>>>>>>>>>>>>> and they are all a little different so testing is really
> >>>> hard.
> >>>>>>> In
> >>>>>>>>>> the
> >>>>>>>>>>>>>> absence of this we have been stuck with just YARN which
> >>> has
> >>>>>>>>>> fantastic
> >>>>>>>>>>>>>> penetration in the Hadoopy part of the org, but zero
> >>>> penetration
> >>>>>>>>>>>>> elsewhere.
> >>>>>>>>>>>>>> Given the huge amount of work being put in to slider,
> >>>> marathon,
> >>>>>>>> aws
> >>>>>>>>>>>>>> tooling, not to mention the umpteen related packaging
> >>>>>>> technologies
> >>>>>>>>>>> people
> >>>>>>>>>>>>>> want to use (Docker, Kubernetes, various cloud-specific
> >>>> deploy
> >>>>>>>>>> tools,
> >>>>>>>>>>>>> etc)
> >>>>>>>>>>>>>> I really think it is important to get this right.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -Jay
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> >>>>>>>>>>>>>> g.turkington@improvedigital.com> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I think the question below re does Samza become a
> >>>> sub-project
> >>>>>>> of
> >>>>>>>>>> Kafka
> >>>>>>>>>>>>>>> highlights the broader point around migration. Chris
> >>>> mentions
> >>>>>>>>>> Samza's
> >>>>>>>>>>>>>>> maturity is heading towards a v1 release but I'm not sure
> >>>> it
> >>>>>>>> feels
> >>>>>>>>>>>>> right to
> >>>>>>>>>>>>>>> launch a v1 then immediately plan to deprecate most of
> >>> it.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> From a selfish perspective I have some guys who have
> >>>> started
> >>>>>>>>>> working
> >>>>>>>>>>>>> with
> >>>>>>>>>>>>>>> Samza and building some new consumers/producers was next
> >>>> up.
> >>>>>>>> Sounds
> >>>>>>>>>>> like
> >>>>>>>>>>>>>>> that is absolutely not the direction to go. I need to
> >>> look
> >>>> into
> >>>>>>>> the
> >>>>>>>>>>> KIP
> >>>>>>>>>>>>> in
> >>>>>>>>>>>>>>> more detail but for me the attractiveness of adding new
> >>>> Samza
> >>>>>>>>>>>>>>> consumer/producers -- even if yes all they were doing was
> >>>>>>> really
> >>>>>>>>>>> getting
> >>>>>>>>>>>>>>> data into and out of Kafka --  was to avoid  having to
> >>>> worry
> >>>>>>>> about
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> lifecycle management of external clients. If there is a
> >>>> generic
> >>>>>>>>>> Kafka
> >>>>>>>>>>>>>>> ingress/egress layer that I can plug a new connector into
> >>>> and
> >>>>>>>> have
> >>>>>>>>>> a
> >>>>>>>>>>>>> lot of
> >>>>>>>>>>>>>>> the heavy lifting re scale and reliability done for me
> >>>> then it
> >>>>>>>>>> gives
> >>>>>>>>>>> me
> >>>>>>>>>>>>> all
> >>>>>>>>>>>>>>> the pushing new consumers/producers would. If not then it
> >>>>>>>>>> complicates
> >>>>>>>>>>> my
> >>>>>>>>>>>>>>> operational deployments.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Which is similar to my other question with the proposal
> >>> --
> >>>> if
> >>>>>>> we
> >>>>>>>>>>> build a
> >>>>>>>>>>>>>>> fully available/stand-alone Samza plus the requisite
> >>> shims
> >>>> to
> >>>>>>>>>>> integrate
> >>>>>>>>>>>>>>> with Slider etc I suspect the former may be a lot more
> >>> work
> >>>>>>> than
> >>>>>>>> we
> >>>>>>>>>>>>> think.
> >>>>>>>>>>>>>>> We may make it much easier for a newcomer to get
> >>> something
> >>>>>>>> running
> >>>>>>>>>> but
> >>>>>>>>>>>>>>> having them step up and get a reliable production
> >>>> deployment
> >>>>>>> may
> >>>>>>>>>> still
> >>>>>>>>>>>>>>> dominate mailing list  traffic, if for different reasons
> >>>> than
> >>>>>>>>>> today.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Don't get me wrong -- I'm comfortable with making the
> >>> Samza
> >>>>>>>>>> dependency
> >>>>>>>>>>>>> on
> >>>>>>>>>>>>>>> Kafka much more explicit and I absolutely see the
> >>>> benefits  in
> >>>>>>>> the
> >>>>>>>>>>>>>>> reduction of duplication and clashing
> >>>>>>> terminologies/abstractions
> >>>>>>>>>> that
> >>>>>>>>>>>>>>> Chris/Jay describe. Samza as a library would likely be a
> >>>> very
> >>>>>>>> nice
> >>>>>>>>>>> tool
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>> add to the Kafka ecosystem. I just have the concerns
> >>> above
> >>>> re
> >>>>>>> the
> >>>>>>>>>>>>>>> operational side.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Garry
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> -----Original Message-----
> >>>>>>>>>>>>>>> From: Gianmarco De Francisci Morales [mailto:
> >>>> gdfm@apache.org]
> >>>>>>>>>>>>>>> Sent: 02 July 2015 12:56
> >>>>>>>>>>>>>>> To: dev@samza.apache.org
> >>>>>>>>>>>>>>> Subject: Re: Thoughts and obesrvations on Samza
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Very interesting thoughts.
> >>>>>>>>>>>>>>> From outside, I have always perceived Samza as a
> >>> computing
> >>>>>>> layer
> >>>>>>>>>> over
> >>>>>>>>>>>>>>> Kafka.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The question, maybe a bit provocative, is "should Samza
> >>> be
> >>>> a
> >>>>>>>>>>> sub-project
> >>>>>>>>>>>>>>> of Kafka then?"
> >>>>>>>>>>>>>>> Or does it make sense to keep it as a separate project
> >>>> with a
> >>>>>>>>>> separate
> >>>>>>>>>>>>>>> governance?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>> Gianmarco
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> >>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Overall, I agree to couple with Kafka more tightly.
> >>>> Because
> >>>>>>>> Samza
> >>>>>>>>>> de
> >>>>>>>>>>>>>>>> facto is based on Kafka, and it should leverage what
> >>> Kafka
> >>>>>>> has.
> >>>>>>>> At
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> same time, Kafka does not need to reinvent what Samza
> >>>> already
> >>>>>>>>>> has. I
> >>>>>>>>>>>>>>>> also like the idea of separating the ingestion and
> >>>>>>>> transformation.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> But it is a little difficult for me to image how the
> >>> Samza
> >>>>>>> will
> >>>>>>>>>> look
> >>>>>>>>>>>>>>> like.
> >>>>>>>>>>>>>>>> And I feel Chris and Jay have a little difference in
> >>>> terms of
> >>>>>>>> how
> >>>>>>>>>>>>>>>> Samza should look like.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> *** Will it look like what Jay's code shows (A client of
> >>>>>>> Kakfa)
> >>>>>>>> ?
> >>>>>>>>>> And
> >>>>>>>>>>>>>>>> user's application code calls this client?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 1. If we make Samza be a library of Kafka (like what the
> >>>> code
> >>>>>>>>>> shows),
> >>>>>>>>>>>>>>>> how do we implement auto-balance and fault-tolerance?
> >>> Are
> >>>> they
> >>>>>>>>>> taken
> >>>>>>>>>>>>>>>> care by the Kafka broker or other mechanism, such as
> >>>> "Samza
> >>>>>>>>>> worker"
> >>>>>>>>>>>>>>>> (just make up the name) ?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 2. What about other features, such as auto-scaling,
> >>> shared
> >>>>>>>> state,
> >>>>>>>>>>>>>>>> monitoring?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> *** If we have Samza standalone, (is this what Chris
> >>>>>>> suggests?)
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 1. we still need to ingest data from Kakfa and produce
> >>> to
> >>>> it.
> >>>>>>>>>> Then it
> >>>>>>>>>>>>>>>> becomes the same as what Samza looks like now, except it
> >>>> does
> >>>>>>>> not
> >>>>>>>>>>> rely
> >>>>>>>>>>>>>>>> on Yarn anymore.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 2. if it is standalone, how can it leverage Kafka's
> >>>> metrics,
> >>>>>>>> logs,
> >>>>>>>>>>>>>>>> etc? Use Kafka code as the dependency?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Fang, Yan
> >>>>>>>>>>>>>>>> yanfang724@gmail.com
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> >>>>>>>> wangguoz@gmail.com
> >>>>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Read through the code example and it looks good to me.
> >>> A
> >>>> few
> >>>>>>>>>>>>>>>>> thoughts regarding deployment:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Today Samza deploys as executable runnable like:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> >>>>>>>>>>>>>>> --config-path=file://...
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> And this proposal advocate for deploying Samza more as
> >>>>>>> embedded
> >>>>>>>>>>>>>>>>> libraries in user application code (ignoring the
> >>>> terminology
> >>>>>>>>>> since
> >>>>>>>>>>>>>>>>> it is not the
> >>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>> as the prototype code):
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> StreamTask task = new MyStreamTask(configs); Thread
> >>>> thread =
> >>>>>>>> new
> >>>>>>>>>>>>>>>>> Thread(task); thread.start();
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I think both of these deployment modes are important
> >>> for
> >>>>>>>>>> different
> >>>>>>>>>>>>>>>>> types
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> users. That said, I think making Samza purely
> >>> standalone
> >>>> is
> >>>>>>>> still
> >>>>>>>>>>>>>>>>> sufficient for either runnable or library modes.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Guozhang
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> >>>>>>> jay@confluent.io>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Looks like gmail mangled the code example, it was
> >>>> supposed
> >>>>>>> to
> >>>>>>>>>> look
> >>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Properties props = new Properties();
> >>>>>>>>>>>>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> >>>>>>>>>> StreamingConfig
> >>>>>>>>>>>>>>>>>> config = new StreamingConfig(props);
> >>>>>>>>>>>>>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> >>>>>>>>>>>>>>>>>> config.processor(ExampleStreamProcessor.class);
> >>>>>>>>>>>>>>>>>> config.serialization(new StringSerializer(), new
> >>>>>>>>>>>>>>>>>> StringDeserializer()); KafkaStreaming container = new
> >>>>>>>>>>>>>>>>>> KafkaStreaming(config); container.run();
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> -Jay
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> >>>>>>> jay@confluent.io
> >>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hey guys,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> This came out of some conversations Chris and I were
> >>>> having
> >>>>>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>> whether
> >>>>>>>>>>>>>>>>>>> it would make sense to use Samza as a kind of data
> >>>>>>> ingestion
> >>>>>>>>>>>>>>>> framework
> >>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat").
> >>> This
> >>>>>>> kind
> >>>>>>>> of
> >>>>>>>>>>>>>>>>> combined
> >>>>>>>>>>>>>>>>>>> with complaints around config and YARN and the
> >>>> discussion
> >>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>> how
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> best do a standalone mode.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> So the thought experiment was, given that Samza was
> >>>>>>> basically
> >>>>>>>>>>>>>>>>>>> already totally Kafka specific, what if you just
> >>>> embraced
> >>>>>>>> that
> >>>>>>>>>>>>>>>>>>> and turned it
> >>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>> something less like a heavyweight framework and more
> >>>> like a
> >>>>>>>>>>>>>>>>>>> third
> >>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>> client--a kind of "producing consumer" with state
> >>>>>>> management
> >>>>>>>>>>>>>>>>> facilities.
> >>>>>>>>>>>>>>>>>>> Basically a library. Instead of a complex stream
> >>>> processing
> >>>>>>>>>>>>>>>>>>> framework
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> would actually be a very simple thing, not much more
> >>>>>>>>>> complicated
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>> operate than a Kafka consumer. As Chris said we
> >>> thought
> >>>>>>> about
> >>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>> lot
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>> what Samza (and the other stream processing systems
> >>>> were
> >>>>>>>> doing)
> >>>>>>>>>>>>>>>> seemed
> >>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>> kind of a hangover from MapReduce.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Of course you need to ingest/output data to and from
> >>>> the
> >>>>>>>> stream
> >>>>>>>>>>>>>>>>>>> processing. But when we actually looked into how that
> >>>> would
> >>>>>>>>>>>>>>>>>>> work,
> >>>>>>>>>>>>>>>> Samza
> >>>>>>>>>>>>>>>>>>> isn't really an ideal data ingestion framework for a
> >>>> bunch
> >>>>>>> of
> >>>>>>>>>>>>>>>> reasons.
> >>>>>>>>>>>>>>>>> To
> >>>>>>>>>>>>>>>>>>> really do that right you need a pretty different
> >>>> internal
> >>>>>>>> data
> >>>>>>>>>>>>>>>>>>> model
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> set of apis. So what if you split them and had an api
> >>>> for
> >>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate
> >>> api
> >>>> for
> >>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>> transformation (Samza).
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> This would also allow really embracing the same
> >>>> terminology
> >>>>>>>> and
> >>>>>>>>>>>>>>>>>>> conventions. One complaint about the current state is
> >>>> that
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>> two
> >>>>>>>>>>>>>>>>>> systems
> >>>>>>>>>>>>>>>>>>> kind of feel bolted on. Terminology like "stream" vs
> >>>>>>> "topic"
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> different
> >>>>>>>>>>>>>>>>>>> config and monitoring systems means you kind of have
> >>> to
> >>>>>>> learn
> >>>>>>>>>>>>>>>>>>> Kafka's
> >>>>>>>>>>>>>>>>>> way,
> >>>>>>>>>>>>>>>>>>> then learn Samza's slightly different way, then kind
> >>> of
> >>>>>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>> how
> >>>>>>>>>>>>>>>>>> they
> >>>>>>>>>>>>>>>>>>> map to each other, which having walked a few people
> >>>> through
> >>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> is surprisingly tricky for folks to get.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Since I have been spending a lot of time on
> >>> airplanes I
> >>>>>>>> hacked
> >>>>>>>>>>>>>>>>>>> up an ernest but still somewhat incomplete prototype
> >>> of
> >>>>>>> what
> >>>>>>>>>>>>>>>>>>> this would
> >>>>>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>>>> like. This is just unceremoniously dumped into Kafka
> >>>> as it
> >>>>>>>>>>>>>>>>>>> required a
> >>>>>>>>>>>>>>>>> few
> >>>>>>>>>>>>>>>>>>> changes to the new consumer. Here is the code:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >>>>>>>>>>>>>>>> /apache/kafka/clients/streaming
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> For the purpose of the prototype I just liberally
> >>>> renamed
> >>>>>>>>>>>>>>>>>>> everything
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> try to align it with Kafka with no regard for
> >>>>>>> compatibility.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> To use this would be something like this:
> >>>>>>>>>>>>>>>>>>> Properties props = new Properties();
> >>>>>>>>>>>>>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> >>>>>>>>>>>>>>>>>>> StreamingConfig config = new
> >>>>>>>>>>>>>>>> StreamingConfig(props);
> >>>>>>>>>>>>>>>>>> config.subscribe("test-topic-1",
> >>>>>>>>>>>>>>>>>>> "test-topic-2");
> >>>>>>>>>> config.processor(ExampleStreamProcessor.class);
> >>>>>>>>>>>>>>>>>> config.serialization(new
> >>>>>>>>>>>>>>>>>>> StringSerializer(), new StringDeserializer());
> >>>>>>> KafkaStreaming
> >>>>>>>>>>>>>>>>> container =
> >>>>>>>>>>>>>>>>>>> new KafkaStreaming(config); container.run();
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> KafkaStreaming is basically the SamzaContainer;
> >>>>>>>> StreamProcessor
> >>>>>>>>>>>>>>>>>>> is basically StreamTask.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> So rather than putting all the class names in a file
> >>>> and
> >>>>>>> then
> >>>>>>>>>>>>>>>>>>> having
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> job assembled by reflection, you just instantiate the
> >>>>>>>> container
> >>>>>>>>>>>>>>>>>>> programmatically. Work is balanced over however many
> >>>>>>>> instances
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>> alive at any time (i.e. if an instance dies, new
> >>> tasks
> >>>> are
> >>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> existing containers without shutting them down).
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> We would provide some glue for running this stuff in
> >>>> YARN
> >>>>>>> via
> >>>>>>>>>>>>>>>>>>> Slider, Mesos via Marathon, and AWS using some of
> >>> their
> >>>>>>> tools
> >>>>>>>>>>>>>>>>>>> but from the
> >>>>>>>>>>>>>>>>> point
> >>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>> view of these frameworks these stream processing jobs
> >>>> are
> >>>>>>>> just
> >>>>>>>>>>>>>>>>> stateless
> >>>>>>>>>>>>>>>>>>> services that can come and go and expand and contract
> >>>> at
> >>>>>>>> will.
> >>>>>>>>>>>>>>>>>>> There
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> no
> >>>>>>>>>>>>>>>>>>> more custom scheduler.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Here are some relevant details:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 1. It is only ~1300 lines of code, it would get
> >>>> larger if
> >>>>>>> we
> >>>>>>>>>>>>>>>>>>> productionized but not vastly larger. We really do
> >>>> get a
> >>>>>>> ton
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>> leverage
> >>>>>>>>>>>>>>>>>>> out of Kafka.
> >>>>>>>>>>>>>>>>>>> 2. Partition management is fully delegated to the
> >>> new
> >>>>>>>>>> consumer.
> >>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>> is nice since now any partition management strategy
> >>>>>>>> available
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>> consumer is also available to Samza (and vice versa)
> >>>> and
> >>>>>>>> with
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> exact
> >>>>>>>>>>>>>>>>>>> same configs.
> >>>>>>>>>>>>>>>>>>> 3. It supports state as well as state reuse
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Anyhow take a look, hopefully it is thought
> >>> provoking.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> -Jay
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> >>>>>>>>>>>>>>>>> criccomini@apache.org>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hey all,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I have had some discussions with Samza engineers at
> >>>>>>> LinkedIn
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> Confluent
> >>>>>>>>>>>>>>>>>>>> and we came up with a few observations and would
> >>> like
> >>>> to
> >>>>>>>>>>>>>>>>>>>> propose
> >>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>> changes.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> We've observed some things that I want to call out
> >>>> about
> >>>>>>>>>>>>>>>>>>>> Samza's
> >>>>>>>>>>>>>>>>> design,
> >>>>>>>>>>>>>>>>>>>> and I'd like to propose some changes.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> * Samza is dependent upon a dynamic deployment
> >>> system.
> >>>>>>>>>>>>>>>>>>>> * Samza is too pluggable.
> >>>>>>>>>>>>>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> >>>>>>> consumer
> >>>>>>>>>>>>>>>>>>>> APIs
> >>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>> trying to solve a lot of the same problems.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> All three of these issues are related, but I'll
> >>>> address
> >>>>>>> them
> >>>>>>>>>> in
> >>>>>>>>>>>>>>>> order.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Deployment
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Samza strongly depends on the use of a dynamic
> >>>> deployment
> >>>>>>>>>>>>>>>>>>>> scheduler
> >>>>>>>>>>>>>>>>> such
> >>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we
> >>>> bet
> >>>>>>> that
> >>>>>>>>>>>>>>>>>>>> there
> >>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>> one or two winners in this area, and we could
> >>> support
> >>>>>>> them,
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> rest
> >>>>>>>>>>>>>>>>>>>> would go away. In reality, there are many
> >>> variations.
> >>>>>>>>>>>>>>>>>>>> Furthermore,
> >>>>>>>>>>>>>>>>> many
> >>>>>>>>>>>>>>>>>>>> people still prefer to just start their processors
> >>>> like
> >>>>>>>> normal
> >>>>>>>>>>>>>>>>>>>> Java processes, and use traditional deployment
> >>> scripts
> >>>>>>> such
> >>>>>>>> as
> >>>>>>>>>>>>>>>>>>>> Fabric,
> >>>>>>>>>>>>>>>>> Chef,
> >>>>>>>>>>>>>>>>>>>> Ansible, etc. Forcing a deployment system on users
> >>>> makes
> >>>>>>> the
> >>>>>>>>>>>>>>>>>>>> Samza start-up process really painful for first time
> >>>>>>> users.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Dynamic deployment as a requirement was also a bit
> >>> of
> >>>> a
> >>>>>>>>>>>>>>>>>>>> mis-fire
> >>>>>>>>>>>>>>>>> because
> >>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> a fundamental misunderstanding between the nature of
> >>>> batch
> >>>>>>>>>> jobs
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>>> processing jobs. Early on, we made conscious effort
> >>> to
> >>>>>>> favor
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> Hadoop
> >>>>>>>>>>>>>>>>>>>> (Map/Reduce) way of doing things, since it worked
> >>> and
> >>>> was
> >>>>>>>> well
> >>>>>>>>>>>>>>>>>> understood.
> >>>>>>>>>>>>>>>>>>>> One thing that we missed was that batch jobs have a
> >>>>>>> definite
> >>>>>>>>>>>>>>>>> beginning,
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> end, and stream processing jobs don't (usually).
> >>> This
> >>>>>>> leads
> >>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>> simpler scheduling problem for stream processors.
> >>> You
> >>>>>>>>>> basically
> >>>>>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>>>> to find a place to start the processor, and start
> >>> it.
> >>>> The
> >>>>>>>> way
> >>>>>>>>>>>>>>>>>>>> we run grids, at LinkedIn, there's no concept of a
> >>>> cluster
> >>>>>>>>>>>>>>>>>>>> being "full". We always
> >>>>>>>>>>>>>>>>> add
> >>>>>>>>>>>>>>>>>>>> more machines. The problem with coupling Samza with
> >>> a
> >>>>>>>>>> scheduler
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> Samza (as a framework) now has to handle deployment.
> >>>> This
> >>>>>>>>>> pulls
> >>>>>>>>>>>>>>>>>>>> in a
> >>>>>>>>>>>>>>>>>> bunch
> >>>>>>>>>>>>>>>>>>>> of things such as configuration distribution (config
> >>>>>>>> stream),
> >>>>>>>>>>>>>>>>>>>> shell
> >>>>>>>>>>>>>>>>>> scrips
> >>>>>>>>>>>>>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> >>>>>>> stuff),
> >>>>>>>>>> etc.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Another reason for requiring dynamic deployment was
> >>> to
> >>>>>>>> support
> >>>>>>>>>>>>>>>>>>>> data locality. If you want to have locality, you
> >>> need
> >>>> to
> >>>>>>> put
> >>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>> processors
> >>>>>>>>>>>>>>>>>>>> close to the data they're processing. Upon further
> >>>>>>>>>>>>>>>>>>>> investigation,
> >>>>>>>>>>>>>>>>>> though,
> >>>>>>>>>>>>>>>>>>>> this feature is not that beneficial. There is some
> >>>> good
> >>>>>>>>>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>> some problems with it on SAMZA-335. Again, we took
> >>> the
> >>>>>>>>>>>>>>>>>>>> Map/Reduce
> >>>>>>>>>>>>>>>>> path,
> >>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>> there are some fundamental differences between HDFS
> >>>> and
> >>>>>>>> Kafka.
> >>>>>>>>>>>>>>>>>>>> HDFS
> >>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>> blocks, while Kafka has partitions. This leads to
> >>> less
> >>>>>>>>>>>>>>>>>>>> optimization potential with stream processors on top
> >>>> of
> >>>>>>>> Kafka.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> This feature is also used as a crutch. Samza doesn't
> >>>> have
> >>>>>>>> any
> >>>>>>>>>>>>>>>>>>>> built
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>> fault-tolerance logic. Instead, it depends on the
> >>>> dynamic
> >>>>>>>>>>>>>>>>>>>> deployment scheduling system to handle restarts
> >>> when a
> >>>>>>>>>>>>>>>>>>>> processor dies. This has
> >>>>>>>>>>>>>>>>>> made
> >>>>>>>>>>>>>>>>>>>> it very difficult to write a standalone Samza
> >>>> container
> >>>>>>>>>>>>>>> (SAMZA-516).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Pluggability
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> In some cases pluggability is good, but I think that
> >>>> we've
> >>>>>>>>>> gone
> >>>>>>>>>>>>>>>>>>>> too
> >>>>>>>>>>>>>>>>> far
> >>>>>>>>>>>>>>>>>>>> with it. Currently, Samza has:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> * Pluggable config.
> >>>>>>>>>>>>>>>>>>>> * Pluggable metrics.
> >>>>>>>>>>>>>>>>>>>> * Pluggable deployment systems.
> >>>>>>>>>>>>>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> >>>>>>>> SystemProducer,
> >>>>>>>>>>>>>>> etc).
> >>>>>>>>>>>>>>>>>>>> * Pluggable serdes.
> >>>>>>>>>>>>>>>>>>>> * Pluggable storage engines.
> >>>>>>>>>>>>>>>>>>>> * Pluggable strategies for just about every
> >>> component
> >>>>>>>>>>>>>>>> (MessageChooser,
> >>>>>>>>>>>>>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> There's probably more that I've forgotten, as well.
> >>>> Some
> >>>>>>> of
> >>>>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>> useful, but some have proven not to be. This all
> >>>> comes at
> >>>>>>> a
> >>>>>>>>>> cost:
> >>>>>>>>>>>>>>>>>>>> complexity. This complexity is making it harder for
> >>>> our
> >>>>>>>> users
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> pick
> >>>>>>>>>>>>>>>>> up
> >>>>>>>>>>>>>>>>>>>> and use Samza out of the box. It also makes it
> >>>> difficult
> >>>>>>> for
> >>>>>>>>>>>>>>>>>>>> Samza developers to reason about what the
> >>>> characteristics
> >>>>>>> of
> >>>>>>>>>>>>>>>>>>>> the container (since the characteristics change
> >>>> depending
> >>>>>>> on
> >>>>>>>>>>>>>>>>>>>> which plugins are use).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The issues with pluggability are most visible in the
> >>>>>>> System
> >>>>>>>>>> APIs.
> >>>>>>>>>>>>>>>> What
> >>>>>>>>>>>>>>>>>>>> Samza really requires to be functional is Kafka as
> >>> its
> >>>>>>>>>>>>>>>>>>>> transport
> >>>>>>>>>>>>>>>>> layer.
> >>>>>>>>>>>>>>>>>>>> But
> >>>>>>>>>>>>>>>>>>>> we've conflated two unrelated use cases into one
> >>> API:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 1. Get data into/out of Kafka.
> >>>>>>>>>>>>>>>>>>>> 2. Process the data in Kafka.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The current System API supports both of these use
> >>>> cases.
> >>>>>>> The
> >>>>>>>>>>>>>>>>>>>> problem
> >>>>>>>>>>>>>>>>> is,
> >>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> actually want different features for each use case.
> >>> By
> >>>>>>>>>> papering
> >>>>>>>>>>>>>>>>>>>> over
> >>>>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>> two use cases, and providing a single API, we've
> >>>>>>> introduced
> >>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> ton of
> >>>>>>>>>>>>>>>>>> leaky
> >>>>>>>>>>>>>>>>>>>> abstractions.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> For example, what we'd really like in (2) is to have
> >>>>>>>>>>>>>>>>>>>> monotonically increasing longs for offsets (like
> >>>> Kafka).
> >>>>>>>> This
> >>>>>>>>>>>>>>>>>>>> would be at odds
> >>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>> (1),
> >>>>>>>>>>>>>>>>>>>> though, since different systems have different
> >>>>>>>>>>>>>>>>>> SCNs/Offsets/UUIDs/vectors.
> >>>>>>>>>>>>>>>>>>>> There was discussion both on the mailing list and
> >>> the
> >>>> SQL
> >>>>>>>>>> JIRAs
> >>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> need for this.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The same thing holds true for replayability. Kafka
> >>>> allows
> >>>>>>> us
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>> rewind
> >>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>> we have a failure. Many other systems don't. In some
> >>>>>>> cases,
> >>>>>>>>>>>>>>>>>>>> systems
> >>>>>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>>>> null for their offsets (e.g.
> >>> WikipediaSystemConsumer)
> >>>>>>>> because
> >>>>>>>>>>>>>>>>>>>> they
> >>>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>> no
> >>>>>>>>>>>>>>>>>>>> offsets.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Partitioning is another example. Kafka supports
> >>>>>>>> partitioning,
> >>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>> many
> >>>>>>>>>>>>>>>>>>>> systems don't. We model this by having a single
> >>>> partition
> >>>>>>>> for
> >>>>>>>>>>>>>>>>>>>> those systems. Still, other systems model
> >>> partitioning
> >>>>>>>>>>>>>>> differently (e.g.
> >>>>>>>>>>>>>>>>>>>> Kinesis).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The SystemAdmin interface is also a mess. Creating
> >>>> streams
> >>>>>>>> in
> >>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> system-agnostic way is almost impossible. As is
> >>>> modeling
> >>>>>>>>>>>>>>>>>>>> metadata
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> system (replication factor, partitions, location,
> >>>> etc).
> >>>>>>> The
> >>>>>>>>>>>>>>>>>>>> list
> >>>>>>>>>>>>>>>> goes
> >>>>>>>>>>>>>>>>>> on.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Duplicate work
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> At the time that we began writing Samza, Kafka's
> >>>> consumer
> >>>>>>>> and
> >>>>>>>>>>>>>>>> producer
> >>>>>>>>>>>>>>>>>>>> APIs
> >>>>>>>>>>>>>>>>>>>> had a relatively weak feature set. On the
> >>>> consumer-side,
> >>>>>>> you
> >>>>>>>>>>>>>>>>>>>> had two
> >>>>>>>>>>>>>>>>>>>> options: use the high level consumer, or the simple
> >>>>>>>> consumer.
> >>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>> problem
> >>>>>>>>>>>>>>>>>>>> with the high-level consumer was that it controlled
> >>>> your
> >>>>>>>>>>>>>>>>>>>> offsets, partition assignments, and the order in
> >>>> which you
> >>>>>>>>>>>>>>>>>>>> received messages. The
> >>>>>>>>>>>>>>>> problem
> >>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>> the simple consumer is that it's not simple. It's
> >>>> basic.
> >>>>>>> You
> >>>>>>>>>>>>>>>>>>>> end up
> >>>>>>>>>>>>>>>>>> having
> >>>>>>>>>>>>>>>>>>>> to handle a lot of really low-level stuff that you
> >>>>>>>> shouldn't.
> >>>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>> spent a
> >>>>>>>>>>>>>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> >>>>>>> robust.
> >>>>>>>>>> It
> >>>>>>>>>>>>>>>>>>>> also allows us to support some cool features:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> * Per-partition message ordering and prioritization.
> >>>>>>>>>>>>>>>>>>>> * Tight control over partition assignment to support
> >>>>>>> joins,
> >>>>>>>>>>>>>>>>>>>> global
> >>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>> (if we want to implement it :)), etc.
> >>>>>>>>>>>>>>>>>>>> * Tight control over offset checkpointing.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> What we didn't realize at the time is that these
> >>>> features
> >>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>> be in Kafka. A lot of Kafka consumers (not just
> >>> Samza
> >>>>>>> stream
> >>>>>>>>>>>>>>>>> processors)
> >>>>>>>>>>>>>>>>>>>> end up wanting to do things like joins and partition
> >>>>>>>>>>>>>>>>>>>> assignment. The
> >>>>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>> community has come to the same conclusion. They're
> >>>> adding
> >>>>>>> a
> >>>>>>>>>> ton
> >>>>>>>>>>>>>>>>>>>> of upgrades into their new Kafka consumer
> >>>> implementation.
> >>>>>>>> To a
> >>>>>>>>>>>>>>>>>>>> large extent,
> >>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>> duplicate work to what we've already done in Samza.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On top of this, Kafka ended up taking a very similar
> >>>>>>>> approach
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> Samza's
> >>>>>>>>>>>>>>>>>>>> KafkaCheckpointManager implementation for handling
> >>>> offset
> >>>>>>>>>>>>>>>>> checkpointing.
> >>>>>>>>>>>>>>>>>>>> Like Samza, Kafka's new offset management feature
> >>>> stores
> >>>>>>>>>> offset
> >>>>>>>>>>>>>>>>>>>> checkpoints in a topic, and allows you to fetch them
> >>>> from
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> broker.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> A lot of this seems like a waste, since we could
> >>> have
> >>>>>>> shared
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> work
> >>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>> had been done in Kafka from the get-go.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Vision
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> All of this leads me to a rather radical proposal.
> >>>> Samza
> >>>>>>> is
> >>>>>>>>>>>>>>>> relatively
> >>>>>>>>>>>>>>>>>>>> stable at this point. I'd venture to say that we're
> >>>> near a
> >>>>>>>> 1.0
> >>>>>>>>>>>>>>>>> release.
> >>>>>>>>>>>>>>>>>>>> I'd
> >>>>>>>>>>>>>>>>>>>> like to propose that we take what we've learned, and
> >>>> begin
> >>>>>>>>>>>>>>>>>>>> thinking
> >>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>> Samza beyond 1.0. What would we change if we were
> >>>> starting
> >>>>>>>>>> from
> >>>>>>>>>>>>>>>>> scratch?
> >>>>>>>>>>>>>>>>>>>> My
> >>>>>>>>>>>>>>>>>>>> proposal is to:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> >>>>>>>>>>>>>>>>>>>> processors, and eliminate all direct dependences on
> >>>> YARN,
> >>>>>>>>>> Mesos,
> >>>>>>>>>>>>>>> etc.
> >>>>>>>>>>>>>>>>>>>> 2. Make a definitive call to support only Kafka as
> >>> the
> >>>>>>>> stream
> >>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>> layer.
> >>>>>>>>>>>>>>>>>>>> 3. Eliminate Samza's metrics, logging,
> >>> serialization,
> >>>> and
> >>>>>>>>>>>>>>>>>>>> config
> >>>>>>>>>>>>>>>>>> systems,
> >>>>>>>>>>>>>>>>>>>> and simply use Kafka's instead.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> This would fix all of the issues that I outlined
> >>>> above. It
> >>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>> shrink the Samza code base pretty dramatically.
> >>>> Supporting
> >>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>> a standalone container will allow Samza to be
> >>>> executed on
> >>>>>>>> YARN
> >>>>>>>>>>>>>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or
> >>> most
> >>>>>>> other
> >>>>>>>>>>>>>>>>>>>> in-house
> >>>>>>>>>>>>>>>>>> deployment
> >>>>>>>>>>>>>>>>>>>> systems. This should make life a lot easier for new
> >>>> users.
> >>>>>>>>>>>>>>>>>>>> Imagine
> >>>>>>>>>>>>>>>>>> having
> >>>>>>>>>>>>>>>>>>>> the hello-samza tutorial without YARN. The drop in
> >>>> mailing
> >>>>>>>>>> list
> >>>>>>>>>>>>>>>>> traffic
> >>>>>>>>>>>>>>>>>>>> will be pretty dramatic.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Coupling with Kafka seems long overdue to me. The
> >>>> reality
> >>>>>>>> is,
> >>>>>>>>>>>>>>>> everyone
> >>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> I'm aware of is using Samza with Kafka. We basically
> >>>>>>> require
> >>>>>>>>>> it
> >>>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>> order for most features to work. Those that are
> >>> using
> >>>>>>> other
> >>>>>>>>>>>>>>>>>>>> systems
> >>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>> generally using it for ingest into Kafka (1), and
> >>> then
> >>>>>>> they
> >>>>>>>> do
> >>>>>>>>>>>>>>>>>>>> the processing on top. There is already discussion (
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >>>>>>>>>>>>>>>> 767
> >>>>>>>>>>>>>>>>>>>> )
> >>>>>>>>>>>>>>>>>>>> in Kafka to make ingesting into Kafka extremely
> >>> easy.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Once we make the call to couple with Kafka, we can
> >>>>>>> leverage
> >>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> ton of
> >>>>>>>>>>>>>>>>>> their
> >>>>>>>>>>>>>>>>>>>> ecosystem. We no longer have to maintain our own
> >>>> config,
> >>>>>>>>>>>>>>>>>>>> metrics,
> >>>>>>>>>>>>>>>> etc.
> >>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>> can all share the same libraries, and make them
> >>>> better.
> >>>>>>> This
> >>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>>>>> allow us to share the consumer/producer APIs, and
> >>>> will let
> >>>>>>>> us
> >>>>>>>>>>>>>>>> leverage
> >>>>>>>>>>>>>>>>>>>> their offset management and partition management,
> >>>> rather
> >>>>>>>> than
> >>>>>>>>>>>>>>>>>>>> having
> >>>>>>>>>>>>>>>>> our
> >>>>>>>>>>>>>>>>>>>> own. All of the coordinator stream code would go
> >>>> away, as
> >>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>> most
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> YARN AppMaster code. We'd probably have to push some
> >>>>>>>> partition
> >>>>>>>>>>>>>>>>>> management
> >>>>>>>>>>>>>>>>>>>> features into the Kafka broker, but they're already
> >>>> moving
> >>>>>>>> in
> >>>>>>>>>>>>>>>>>>>> that direction with the new consumer API. The
> >>>> features we
> >>>>>>>> have
> >>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>> partition
> >>>>>>>>>>>>>>>>>>>> assignment aren't unique to Samza, and seem like
> >>> they
> >>>>>>> should
> >>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>> anyway. There will always be some niche usages which
> >>>> will
> >>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>> extra
> >>>>>>>>>>>>>>>>>>>> care and hence full control over partition
> >>> assignments
> >>>>>>> much
> >>>>>>>>>>>>>>>>>>>> like the
> >>>>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>> low level consumer api. These would continue to be
> >>>>>>>> supported.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> These items will be good for the Samza community.
> >>>> They'll
> >>>>>>>> make
> >>>>>>>>>>>>>>>>>>>> Samza easier to use, and make it easier for
> >>>> developers to
> >>>>>>>> add
> >>>>>>>>>>>>>>>>>>>> new features.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Obviously this is a fairly large (and somewhat
> >>>> backwards
> >>>>>>>>>>>>>>>> incompatible
> >>>>>>>>>>>>>>>>>>>> change). If we choose to go this route, it's
> >>> important
> >>>>>>> that
> >>>>>>>> we
> >>>>>>>>>>>>>>>> openly
> >>>>>>>>>>>>>>>>>>>> communicate how we're going to provide a migration
> >>>> path
> >>>>>>> from
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>>> APIs to the new ones (if we make incompatible
> >>>> changes). I
> >>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>> at a minimum, we'd probably need to provide a
> >>> wrapper
> >>>> to
> >>>>>>>> allow
> >>>>>>>>>>>>>>>>>>>> existing StreamTask implementations to continue
> >>>> running on
> >>>>>>>> the
> >>>>>>>>>>>>>>> new container.
> >>>>>>>>>>>>>>>>>> It's
> >>>>>>>>>>>>>>>>>>>> also important that we openly communicate about
> >>>> timing,
> >>>>>>> and
> >>>>>>>>>>>>>>>>>>>> stages
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> migration.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> If you made it this far, I'm sure you have opinions.
> >>>> :)
> >>>>>>>> Please
> >>>>>>>>>>>>>>>>>>>> send
> >>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>> thoughts and feedback.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>>> Chris
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>> -- Guozhang
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>
> >>>
> >>
> >>
> >>
> >> --
> >> Jordan Shaw
> >> Full Stack Software Engineer
> >> PubNub Inc
> >> 1045 17th St
> >> San Francisco, CA 94107
>
>

Re: Thoughts and obesrvations on Samza

Posted by Tim Williams <wi...@gmail.com>.
Some thoughts from the peanut gallery...

On Thu, Jul 9, 2015 at 5:14 PM, Martin Kleppmann <ma...@kleppmann.com> wrote:
> Thanks Julian for calling out the principle of community over code, which is super important. If it was just a matter of code, the Kafka project could simply pull in the Samza code (or write a new stream processor) without asking permission -- but they wouldn't get the Samza community. Thus, I think the community aspect is the most important part of this discussion. If we're talking about merging projects, it's really about merging communities.
>
> I had a chat with a friend who is a Lucene/Solr committer: those were also originally two separate projects, which merged into one. He said the merge was not always easy, but probably a net win for both projects and communities overall. In their community people tend to specialise on either the Lucene part or the Solr part, but that's ok -- it's still a cohesive community nevertheless, and it benefits from close collaboration due to having everyone in the same project. Releases didn't slow down; in fact, they perhaps got faster due to less cross-project coordination overhead. So that allayed my concerns about a big project becoming slow.

It seems to me that looking at the Lucene/Solr merge is only helpful
if you're experiencing the same pain points.  In that case,
enhancements would occur "downstream" (ie. in Solr) that either
wouldn't or would take a long time to make it upstream to Lucene core.
I haven't lurked here long enough to know if that's the case here or
not. In any case, I reckon it'd be best to consider these two things
(community-future/code-future) independently.  Also, around the same
time, Tika and Mahout went the opposite (TLP) direction and have
flourished... I'd also just say that a "subproject" is an anti-pattern
around here...

> Besides community and code/architecture, another consideration is our user base (including those who are not on this mailing list). What is good for our users? I've thought about this more over the last few days:

I'm a new, dumb user so I'm happy to help think through what's good for me:)

> - Reducing users' confusion is good. If someone is adopting Kafka, they will also need some way of processing their data in Kafka. At the moment, the Kafka docs give you consumer APIs but nothing more. Having to choose a separate stream processing framework is a burden on users, especially if that framework uses terminology that is inconsistent with Kafka. If we make Samza a part of Kafka and unify the terminology, it would become a coherent part of the documentation, and be much less confusing for users.

I don't think "having to choose.." is a burden - I simply didn't know
Samza existed until a friend pointed me, but that can be fixed by
convincing Kafka to give a more prominent link.  So far, the
terminology hasn't confused me but maybe that's because my usage so
far is really unsophisticated.

> - Making it easy for users to get started is good. Simplifying the API and configuration is part of it. Making YARN optional is also good. It would also help to be part of the same package that people download, and part of the same documentation. (Simplifying API/config and decoupling from YARN can be done as a separate project; becoming part of the same package would require merging projects.)

FWIW, YARN was one of the compelling aspects of Samza because I didn't
have to wonder about resilience vs. what one has to do with the
Flume-Kafka stuffs. Plus, your grid/bootstrap stuff makes it really
sweet to get started.  Of course, all of YARN has the downside of
being extremely difficult to debug and that has been really annoying
but other than debug logging to a kafka topic or something I'm not how
ya'll can improve that.

Anyway, thanks for Samza, it's been really nice so far...

Thanks,
--tim

Re: Thoughts and obesrvations on Samza

Posted by Martin Kleppmann <ma...@kleppmann.com>.
Thanks Julian for calling out the principle of community over code, which is super important. If it was just a matter of code, the Kafka project could simply pull in the Samza code (or write a new stream processor) without asking permission -- but they wouldn't get the Samza community. Thus, I think the community aspect is the most important part of this discussion. If we're talking about merging projects, it's really about merging communities.

I had a chat with a friend who is a Lucene/Solr committer: those were also originally two separate projects, which merged into one. He said the merge was not always easy, but probably a net win for both projects and communities overall. In their community people tend to specialise on either the Lucene part or the Solr part, but that's ok -- it's still a cohesive community nevertheless, and it benefits from close collaboration due to having everyone in the same project. Releases didn't slow down; in fact, they perhaps got faster due to less cross-project coordination overhead. So that allayed my concerns about a big project becoming slow.

Besides community and code/architecture, another consideration is our user base (including those who are not on this mailing list). What is good for our users? I've thought about this more over the last few days:

- Reducing users' confusion is good. If someone is adopting Kafka, they will also need some way of processing their data in Kafka. At the moment, the Kafka docs give you consumer APIs but nothing more. Having to choose a separate stream processing framework is a burden on users, especially if that framework uses terminology that is inconsistent with Kafka. If we make Samza a part of Kafka and unify the terminology, it would become a coherent part of the documentation, and be much less confusing for users.

- Making it easy for users to get started is good. Simplifying the API and configuration is part of it. Making YARN optional is also good. It would also help to be part of the same package that people download, and part of the same documentation. (Simplifying API/config and decoupling from YARN can be done as a separate project; becoming part of the same package would require merging projects.)

- Supporting users' choice of programming language is good. I used to work with Ruby, and in the Ruby community there are plenty of people with an irrational hatred of the JVM. I imagine other language communities are likely similar. If Samza becomes a fairly thin client library to Kafka (using partition assignment etc provided by the Kafka brokers), then it becomes much more feasible to implement the same interface in other languages too, giving true multi-language support.

Having thought about this, I am coming to the conclusion that a stream processor that is part of the Kafka project would be good for users, and thus a more successful project. However, the people with experience in stream processing systems are in the Samza community. This leads me to thinking that merging projects and communities might be a good idea: with the union of experience from both communities, we will probably build a better system that is better for users.

Jakob advocated maintaining support for input sources other than Kafka. While I can totally see the need for a framework that does this, I think the need is pretty well satisfied by Storm, which already has spouts for Kafka, Kestrel, JMS, AMQP, Redis and beanstalkd (and perhaps more). I don't see much value in Samza attempting to catch up here, especially if Copycat will provide connectors to many systems by different means. On the other hand, my failed attempts to implement SystemConsumers for Kinesis and Postgres make me think that a stream processor that supports many different inputs is limited to a lowest-common-denominator model; if Samza supports only Kafka, I think it could support Kafka better than any other framework (by doing one thing and doing it well).

Julian: not sure I understand your point 2 about departing from the vision of distributed processing. A library-ified Samza would still allow distributed processing, and (with a small amount of glue) could still be deployed to YARN or other cluster.

So, in conclusion, I'm starting to agree with the approach that Jay has been advocating in this thread.

Martin


On 9 Jul 2015, at 15:32, Julian Hyde <jh...@apache.org> wrote:

> Wow, what a great discussion. A brave discussion, since no project
> wants to reduce its scope. And important, because "right-sizing"
> technology components can help them win in the long run.
> 
> I have a couple of let-me-play-devil's-advocate questions.
> 
> 1. Community over code
> 
> Let's look at this in terms of the Apache Way. The Apache Way
> advocates "community over code", and as Jakob points out, the Samza
> community is distinct from the Kafka community. It seems that we are
> talking here about Samza-the-code.
> 
> According to the Apache Way, what Samza-the-project should be doing is
> what Samza-the-community is good at. Samza-the-code-as-it-is-today can
> move to Kafka, stay in Samza, or be deleted if it has been superseded.
> 
> Architectural discussions are important to have, and the Apache Way
> gets in the way of good architecture sometimes. When we're thinking
> about moving code, let's also think about the community of people
> working on the code.
> 
> Apache Phoenix is a good analogy. Phoenix is technically very closely
> tied to HBase, but a distinct community, with different skill-sets.
> (HBase, like Kafka, is "hard core", and not for everyone.) They have
> also been good at re-examining their project scope and re-scoping
> where necessary.
> 
> 2. Architecture
> 
> This proposal retreats from the grand vision of "distributed stream
> management system" where not only storage is distributed but also
> processing. There is no architectural piece that says "I need 10 JVMs
> to process this CPU intensive standing query and I currently only have
> 6." What projects, current or envisioned, would fit that gap? Is that
> work a good fit for the Samza community?
> 
> Julian
> 
> 
> 
> On Wed, Jul 8, 2015 at 10:47 PM, Jordan Shaw <jo...@pubnub.com> wrote:
>> I'm all for any optimizations that can be made to the Yarn workflow.
>> 
>> I actually agree with Jakob in regard to the producers/consumers. I have
>> spent sometime writing consumers and producers for other transport
>> abstractions and overall the current api abstractions in Samza I feel are
>> pretty good. There are some things that are sort of anomalous and catered
>> more toward the Kafka model but easy enough to work around and I've been
>> able to make other Producers and Consumers work that are no where near the
>> same paradigm as Kafka.
>> 
>> To Jay's point although Kafka is great and does the streaming data paradigm
>> very well there is really no reason why a different transport application
>> implemented properly wouldn't be able to stream data with the same
>> effectiveness as Kafka and that transport may suite the user's use case
>> better or be more cost effective than Kafka. For example we had to decide
>> if Kafka was worth the extra cost of running a zookeeper cluster and if the
>> scaling through partitioning was worth the operational overhead vs having a
>> mesh network over ZeroMQ. After deciding that our use case would fit with
>> Kafka fine there were other challenges like understanding how AWS EC2 SSD's
>> behaved (AWS amortizes all disk into into Random I/O this is bad for Kafka).
>> 
>> Thus, I would lend of the side of transport flexibility for a framework
>> like Samza over binding to a transport medium like Kafka.
>> 
>> 
>> On Wed, Jul 8, 2015 at 1:39 PM, Jay Kreps <ja...@gmail.com> wrote:
>> 
>>> Good summary Jakob.
>>> 
>>> WRT to the general purpose vs Kafka-specific, I actually see it slightly
>>> differently. Consider how Storm works as an example, there is a data source
>>> (spout) which could be Kafka, Database, etc, and then there is a transport
>>> (a netty TCP thing iiuc). Storm allows you to process data from any source,
>>> but when it comes from a source they always funnel it through their
>>> transport to get to the tasks/bolts. It is natural to think of Kafka as the
>>> Spout, but I think the better analogy is actually that Kafka is the
>>> transport.
>>> 
>>> It is really hard to make the transport truly pluggable because this is
>>> what the tasks interact with and you need to have guarantees about delivery
>>> (and reprocessing), partitioning, atomicity of output, ordering, etc so
>>> your stream processing can get the right answer. From my point of view what
>>> this proposal says is that Kafka would be non-pluggable as the *transport*.
>>> 
>>> So in this proposal data would still come into and out of Kafka from a wide
>>> variety of sources, but by requiring Kafka as the transport the interaction
>>> with the tasks will always look the same (a persistent, partitioned, log).
>>> So going back to the Storm analogy it is something like
>>>  Spout interface = copy cat
>>>  Bolt interface = samza
>>> 
>>> This does obviously make Samza dependent on Kafka but it doesn't mean you
>>> wouldn't be processing data from all kinds of sources--indeed that is the
>>> whole purpose. It just means that each of these data streams would be
>>> available as a multi-subscriber Kafka topic to other systems, applications,
>>> etc, not just for your job.
>>> 
>>> If you think about how things are now Samza already depends on a
>>> partitioned, persistent, offset addressable log with log
>>> compaction...which, unsurprisingly, so I don't think this is really a new
>>> dependency.
>>> 
>>> Philosophically I think this makes sense too. To make a bunch of programs
>>> fit together you have to standardize something. In this proposal what you
>>> are standardizing around is really Kafka's protocol for streaming data and
>>> your data format. The transformations that connect these streams can be
>>> done via Samza, Storm, Spark, standalone java or python programs, etc but
>>> the ultimate output and contract to the rest of the organization/world will
>>> be the resulting Kafka topic. Philosophically I think this kind of data and
>>> protocol based contract is the right way to go rather than saying that the
>>> contract is a particular java api and the stream/data is what is pluggable.
>>> 
>>> -Jay
>>> 
>>> 
>>> 
>>> On Wed, Jul 8, 2015 at 11:03 AM, Jakob Homan <jg...@gmail.com> wrote:
>>> 
>>>> Rewinding back to the beginning of this topic, there are effectively
>>>> three proposals on the table:
>>>> 
>>>> 1) Chris' ideas for a direction towards a 2.0 release with an emphasis
>>>> on API and configuration simplification.  This ideas are based on lots
>>>> of lessons learned from the 0.x branch and are worthy of a 2.0 label
>>>> and breaking backwards compability.  I'm not sure I agree with all of
>>>> them, but they're definitely worth pursuing.
>>>> 
>>>> 2) Chris' alternative proposal, which goes beyond his first and is
>>>> essentially a reboot of Samza to a more limited, entirely
>>>> Kafka-focused approach.  Samza would cease being a general purpose
>>>> stream processing framework, akin to and an alternative to say, Apache
>>>> Storm, and would instead become a standalone complement to the Kafka
>>>> project.
>>>> 
>>>> 3) Jay's proposal, which goes even further, and suggests that the
>>>> Kafka community would be better served by adding stream processing as
>>>> a module to Kafka.  This is a perfectly valid approach, but since it's
>>>> entirely confined to the Kafka project, doesn't really involve Samza.
>>>> If the Kafka team were to go this route, there would be no obligation
>>>> on the Samza team to shut down, disband, etc.
>>>> 
>>>> This last bit is important because Samza and Kafka, while closely
>>>> linked, are distinct communities.  The intersection of committers on
>>>> both Kafka and Samza is three people out of a combined 18 committers
>>>> across both projects.   Samza is a distinct community that shares
>>>> quite a few users with Kafka, but is able to chart its own course.
>>>> 
>>>> My own view is that Samza has had an amazing year and is taking off at
>>>> a rapid rate.  It was only proposed for Incubator two years ago and is
>>>> still very young. The original team at LinkedIn has left that company
>>>> but the project has continued to grow via contributions both from
>>>> LinkedIn and from without.  We've recently seen a significant uptake
>>>> in discussion and bug reports.
>>>> 
>>>> The API, deployment and configuration changes Chris suggests are good
>>>> ideas, but I think there is still serious value in having a
>>>> stand-alone general stream processing framework that supports other
>>>> input sources than Kafka.  We've already had contributions for adding
>>>> producer support to ElasticSearch and HDFS.  As more users come on
>>>> board, I would expect them to contribute more consumers and producers.
>>>> 
>>>> It's a bit of chicken-and-the-egg problem; since the original team
>>>> didn't have cycles to prioritize support for non-Kafka systems
>>>> (kinesis, eventhub, twitter, flume, zeromq, etc.), Samza was less
>>>> compelling than other stream processing frameworks that did have
>>>> support and was therefore not used in those situations.  I'd love to
>>>> see those added and the SystemConsumer/Producer APIs improved to
>>>> fluently support them as well as Kafka.
>>>> Martin had a question regarding the tight coupling between Hadoop HDFS
>>>> and MapReduce (and YARN and Common).  This has been a problem for
>>>> years and there have been several aborted attempts to split the
>>>> projects out.  Each time there turned out to be a strong need for
>>>> cross-cutting collaboration and so the effort was dropped.  Absent the
>>>> third option above (Kafka adding stream support to itself directly), I
>>>> would imagine something similar would play out here.
>>>> 
>>>> We should get a feeling for which of the three proposals the Samza
>>>> community is behind, technical details of each notwithstanding.  This
>>>> would include not just the committers/PMC members, but also the users,
>>>> contributors and lurkers.
>>>> 
>>>> -Jakob
>>>> 
>>>> On 8 July 2015 at 07:41, Ben Kirwin <be...@kirw.in> wrote:
>>>>> Hi all,
>>>>> 
>>>>> Interesting stuff! Jumping in a bit late, but here goes...
>>>>> 
>>>>> I'd definitely be excited about a slimmed-down and more Kafka-specific
>>>>> Samza -- you don't seem to lose much functionality that people
>>>>> actually use, and the gains in simplicity / code sharing seem
>>>>> potentially very large. (I've spent a bunch of time peeling back those
>>>>> layers of abstraction to get eg. more control over message send order,
>>>>> and working directly against Kafka's APIs would have been much
>>>>> easier.) I also like the approach of letting Kafka code do the heavy
>>>>> lifting and letting stream processing systems build on those -- good,
>>>>> reusable implementations would be great for the whole
>>>>> stream-processing ecosystem, and Samza in particular.
>>>>> 
>>>>> On the other hand, I do hope that using Kafka's group membership /
>>>>> partition assignment / etc. stays optional. As far as I can tell,
>>>>> ~every major stream processing system that uses Kafka has chosen (or
>>>>> switched to) 'static' partitioning, where each logical task consumes a
>>>>> fixed set of partitions. When 'dynamic deploying' (a la Storm / Mesos
>>>>> / Yarn) the underlying system is already doing failure detection and
>>>>> transferring work between hosts when machines go down, so using
>>>>> Kafka's implementation is redundant at best -- and at worst, the
>>>>> interaction between the two systems can make outages worse.
>>>>> 
>>>>> And thanks to Chris / Jay for getting this ball rolling. Exciting
>>>> times...
>>>>> 
>>>>> On Tue, Jul 7, 2015 at 2:35 PM, Jay Kreps <ja...@confluent.io> wrote:
>>>>>> Hey Roger,
>>>>>> 
>>>>>> I couldn't agree more. We spent a bunch of time talking to people and
>>>> that
>>>>>> is exactly the stuff we heard time and again. What makes it hard, of
>>>>>> course, is that there is some tension between compatibility with
>>> what's
>>>>>> there now and making things better for new users.
>>>>>> 
>>>>>> I also strongly agree with the importance of multi-language support.
>>> We
>>>> are
>>>>>> talking now about Java, but for application development use cases
>>> people
>>>>>> want to work in whatever language they are using elsewhere. I think
>>>> moving
>>>>>> to a model where Kafka itself does the group membership, lifecycle
>>>> control,
>>>>>> and partition assignment has the advantage of putting all that complex
>>>>>> stuff behind a clean api that the clients are already going to be
>>>>>> implementing for their consumer, so the added functionality for stream
>>>>>> processing beyond a consumer becomes very minor.
>>>>>> 
>>>>>> -Jay
>>>>>> 
>>>>>> On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <roger.hoover@gmail.com
>>>> 
>>>>>> wrote:
>>>>>> 
>>>>>>> Metamorphosis...nice. :)
>>>>>>> 
>>>>>>> This has been a great discussion.  As a user of Samza who's recently
>>>>>>> integrated it into a relatively large organization, I just want to
>>> add
>>>>>>> support to a few points already made.
>>>>>>> 
>>>>>>> The biggest hurdles to adoption of Samza as it currently exists that
>>>> I've
>>>>>>> experienced are:
>>>>>>> 1) YARN - YARN is overly complex in many environments where Puppet
>>>> would do
>>>>>>> just fine but it was the only mechanism to get fault tolerance.
>>>>>>> 2) Configuration - I think I like the idea of configuring most of the
>>>> job
>>>>>>> in code rather than config files.  In general, I think the goal
>>> should
>>>> be
>>>>>>> to make it harder to make mistakes, especially of the kind where the
>>>> code
>>>>>>> expects something and the config doesn't match.  The current config
>>> is
>>>>>>> quite intricate and error-prone.  For example, the application logic
>>>> may
>>>>>>> depend on bootstrapping a topic but rather than asserting that in the
>>>> code,
>>>>>>> you have to rely on getting the config right.  Likewise with serdes,
>>>> the
>>>>>>> Java representations produced by various serdes (JSON, Avro, etc.)
>>> are
>>>> not
>>>>>>> equivalent so you cannot just reconfigure a serde without changing
>>> the
>>>>>>> code.   It would be nice for jobs to be able to assert what they
>>> expect
>>>>>>> from their input topics in terms of partitioning.  This is getting a
>>>> little
>>>>>>> off topic but I was even thinking about creating a "Samza config
>>>> linter"
>>>>>>> that would sanity check a set of configs.  Especially in
>>> organizations
>>>>>>> where config is managed by a different team than the application
>>>> developer,
>>>>>>> it's very hard to get avoid config mistakes.
>>>>>>> 3) Java/Scala centric - for many teams (especially DevOps-type
>>> folks),
>>>> the
>>>>>>> pain of the Java toolchain (maven, slow builds, weak command line
>>>> support,
>>>>>>> configuration over convention) really inhibits productivity.  As more
>>>> and
>>>>>>> more high-quality clients become available for Kafka, I hope they'll
>>>> follow
>>>>>>> Samza's model.  Not sure how much it affects the proposals in this
>>>> thread
>>>>>>> but please consider other languages in the ecosystem as well.  From
>>>> what
>>>>>>> I've heard, Spark has more Python users than Java/Scala.
>>>>>>> (FYI, we added a Jython wrapper for the Samza API
>>>>>>> 
>>>>>>> 
>>>> 
>>> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
>>>>>>> and are working on a Yeoman generator
>>>>>>> https://github.com/Quantiply/generator-rico for Jython/Samza
>>> projects
>>>> to
>>>>>>> alleviate some of the pain)
>>>>>>> 
>>>>>>> I also want to underscore Jay's point about improving the user
>>>> experience.
>>>>>>> That's a very important factor for adoption.  I think the goal should
>>>> be to
>>>>>>> make Samza as easy to get started with as something like Logstash.
>>>>>>> Logstash is vastly inferior in terms of capabilities to Samza but
>>> it's
>>>> easy
>>>>>>> to get started and that makes a big difference.
>>>>>>> 
>>>>>>> Cheers,
>>>>>>> 
>>>>>>> Roger
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
>>>>>>> gdfm@apache.org> wrote:
>>>>>>> 
>>>>>>>> Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
>>>>>>> winner
>>>>>>>> :)
>>>>>>>> 
>>>>>>>> --
>>>>>>>> Gianmarco
>>>>>>>> 
>>>>>>>> On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
>>>> gdfm@apache.org
>>>>>>>> 
>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>>> Hi,
>>>>>>>>> 
>>>>>>>>> @Martin, thanks for you comments.
>>>>>>>>> Maybe I'm missing some important point, but I think coupling the
>>>>>>> releases
>>>>>>>>> is actually a *good* thing.
>>>>>>>>> To make an example, would it be better if the MR and HDFS
>>>> components of
>>>>>>>>> Hadoop had different release schedules?
>>>>>>>>> 
>>>>>>>>> Actually, keeping the discussion in a single place would make
>>>> agreeing
>>>>>>> on
>>>>>>>>> releases (and backwards compatibility) much easier, as everybody
>>>> would
>>>>>>> be
>>>>>>>>> responsible for the whole codebase.
>>>>>>>>> 
>>>>>>>>> That said, I like the idea of absorbing samza-core as a
>>>> sub-project,
>>>>>>> and
>>>>>>>>> leave the fancy stuff separate.
>>>>>>>>> It probably gives 90% of the benefits we have been discussing
>>> here.
>>>>>>>>> 
>>>>>>>>> Cheers,
>>>>>>>>> 
>>>>>>>>> --
>>>>>>>>> Gianmarco
>>>>>>>>> 
>>>>>>>>> On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
>>>>>>>>> 
>>>>>>>>>> Hey Martin,
>>>>>>>>>> 
>>>>>>>>>> I agree coupling release schedules is a downside.
>>>>>>>>>> 
>>>>>>>>>> Definitely we can try to solve some of the integration problems
>>> in
>>>>>>>>>> Confluent Platform or in other distributions. But I think this
>>>> ends up
>>>>>>>>>> being really shallow. I guess I feel to really get a good user
>>>>>>>> experience
>>>>>>>>>> the two systems have to kind of feel like part of the same thing
>>>> and
>>>>>>> you
>>>>>>>>>> can't really add that in later--you can put both in the same
>>>>>>>> downloadable
>>>>>>>>>> tar file but it doesn't really give a very cohesive feeling. I
>>>> agree
>>>>>>>> that
>>>>>>>>>> ultimately any of the project stuff is as much social and naming
>>>> as
>>>>>>>>>> anything else--theoretically two totally independent projects
>>>> could
>>>>>>> work
>>>>>>>>>> to
>>>>>>>>>> tightly align. In practice this seems to be quite difficult
>>>> though.
>>>>>>>>>> 
>>>>>>>>>> For the frameworks--totally agree it would be good to maintain
>>> the
>>>>>>>>>> framework support with the project. In some cases there may not
>>>> be too
>>>>>>>>>> much
>>>>>>>>>> there since the integration gets lighter but I think whatever
>>>> stubs
>>>>>>> you
>>>>>>>>>> need should be included. So no I definitely wasn't trying to
>>> imply
>>>>>>>>>> dropping
>>>>>>>>>> support for these frameworks, just making the integration
>>> lighter
>>>> by
>>>>>>>>>> separating process management from partition management.
>>>>>>>>>> 
>>>>>>>>>> You raise two good points we would have to figure out if we went
>>>> down
>>>>>>>> the
>>>>>>>>>> alignment path:
>>>>>>>>>> 1. With respect to the name, yeah I think the first question is
>>>>>>> whether
>>>>>>>>>> some "re-branding" would be worth it. If so then I think we can
>>>> have a
>>>>>>>> big
>>>>>>>>>> thread on the name. I'm definitely not set on Kafka Streaming or
>>>> Kafka
>>>>>>>>>> Streams I was just using them to be kind of illustrative. I
>>> agree
>>>> with
>>>>>>>>>> your
>>>>>>>>>> critique of these names, though I think people would get the
>>> idea.
>>>>>>>>>> 2. Yeah you also raise a good point about how to "factor" it.
>>>> Here are
>>>>>>>> the
>>>>>>>>>> options I see (I could get enthusiastic about any of them):
>>>>>>>>>>   a. One repo for both Kafka and Samza
>>>>>>>>>>   b. Two repos, retaining the current seperation
>>>>>>>>>>   c. Two repos, the equivalent of samza-api and samza-core is
>>>>>>> absorbed
>>>>>>>>>> almost like a third client
>>>>>>>>>> 
>>>>>>>>>> Cheers,
>>>>>>>>>> 
>>>>>>>>>> -Jay
>>>>>>>>>> 
>>>>>>>>>> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
>>>>>>> martin@kleppmann.com>
>>>>>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>>> Ok, thanks for the clarifications. Just a few follow-up
>>>> comments.
>>>>>>>>>>> 
>>>>>>>>>>> - I see the appeal of merging with Kafka or becoming a
>>>> subproject:
>>>>>>> the
>>>>>>>>>>> reasons you mention are good. The risk I see is that release
>>>>>>> schedules
>>>>>>>>>>> become coupled to each other, which can slow everyone down,
>>> and
>>>>>>> large
>>>>>>>>>>> projects with many contributors are harder to manage. (Jakob,
>>>> can
>>>>>>> you
>>>>>>>>>> speak
>>>>>>>>>>> from experience, having seen a wider range of Hadoop ecosystem
>>>>>>>>>> projects?)
>>>>>>>>>>> 
>>>>>>>>>>> Some of the goals of a better unified developer experience
>>> could
>>>>>>> also
>>>>>>>> be
>>>>>>>>>>> solved by integrating Samza nicely into a Kafka distribution
>>>> (such
>>>>>>> as
>>>>>>>>>>> Confluent's). I'm not against merging projects if we decide
>>>> that's
>>>>>>> the
>>>>>>>>>> way
>>>>>>>>>>> to go, just pointing out the same goals can perhaps also be
>>>> achieved
>>>>>>>> in
>>>>>>>>>>> other ways.
>>>>>>>>>>> 
>>>>>>>>>>> - With regard to dropping the YARN dependency: are you
>>> proposing
>>>>>>> that
>>>>>>>>>>> Samza doesn't give any help to people wanting to run on
>>>>>>>>>> YARN/Mesos/AWS/etc?
>>>>>>>>>>> So the docs would basically have a link to Slider and nothing
>>>> else?
>>>>>>> Or
>>>>>>>>>>> would we maintain integrations with a bunch of popular
>>>> deployment
>>>>>>>>>> methods
>>>>>>>>>>> (e.g. the necessary glue and shell scripts to make Samza work
>>>> with
>>>>>>>>>> Slider)?
>>>>>>>>>>> 
>>>>>>>>>>> I absolutely think it's a good idea to have the "as a library"
>>>> and
>>>>>>>> "as a
>>>>>>>>>>> process" (using Yi's taxonomy) options for people who want
>>> them,
>>>>>>> but I
>>>>>>>>>>> think there should also be a low-friction path for common "as
>>> a
>>>>>>>> service"
>>>>>>>>>>> deployment methods, for which we probably need to maintain
>>>>>>>> integrations.
>>>>>>>>>>> 
>>>>>>>>>>> - Project naming: "Kafka Streams" seems odd to me, because
>>>> Kafka is
>>>>>>>> all
>>>>>>>>>>> about streams already. Perhaps "Kafka Transformers" or "Kafka
>>>>>>> Filters"
>>>>>>>>>>> would be more apt?
>>>>>>>>>>> 
>>>>>>>>>>> One suggestion: perhaps the core of Samza (stream
>>> transformation
>>>>>>> with
>>>>>>>>>>> state management -- i.e. the "Samza as a library" bit) could
>>>> become
>>>>>>>>>> part of
>>>>>>>>>>> Kafka, while higher-level tools such as streaming SQL and
>>>>>>> integrations
>>>>>>>>>> with
>>>>>>>>>>> deployment frameworks remain in a separate project? In other
>>>> words,
>>>>>>>>>> Kafka
>>>>>>>>>>> would absorb the proven, stable core of Samza, which would
>>>> become
>>>>>>> the
>>>>>>>>>>> "third Kafka client" mentioned early in this thread. The Samza
>>>>>>> project
>>>>>>>>>>> would then target that third Kafka client as its base API, and
>>>> the
>>>>>>>>>> project
>>>>>>>>>>> would be freed up to explore more experimental new horizons.
>>>>>>>>>>> 
>>>>>>>>>>> Martin
>>>>>>>>>>> 
>>>>>>>>>>> On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com>
>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Hey Martin,
>>>>>>>>>>>> 
>>>>>>>>>>>> For the YARN/Mesos/etc decoupling I actually don't think it
>>>> ties
>>>>>>> our
>>>>>>>>>>> hands
>>>>>>>>>>>> at all, all it does is refactor things. The division of
>>>>>>>>>> responsibility is
>>>>>>>>>>>> that Samza core is responsible for task lifecycle, state,
>>> and
>>>>>>>>>> partition
>>>>>>>>>>>> management (using the Kafka co-ordinator) but it is NOT
>>>>>>> responsible
>>>>>>>>>> for
>>>>>>>>>>>> packaging, configuration deployment or execution of
>>>> processes. The
>>>>>>>>>>> problem
>>>>>>>>>>>> of packaging and starting these processes is
>>>>>>>>>>>> framework/environment-specific. This leaves individual
>>>> frameworks
>>>>>>> to
>>>>>>>>>> be
>>>>>>>>>>> as
>>>>>>>>>>>> fancy or vanilla as they like. So you can get simple
>>> stateless
>>>>>>>>>> support in
>>>>>>>>>>>> YARN, Mesos, etc using their off-the-shelf app framework
>>>> (Slider,
>>>>>>>>>>> Marathon,
>>>>>>>>>>>> etc). These are well known by people and have nice UIs and a
>>>> lot
>>>>>>> of
>>>>>>>>>>>> flexibility. I don't think they have node affinity as a
>>> built
>>>> in
>>>>>>>>>> option
>>>>>>>>>>>> (though I could be wrong). So if we want that we can either
>>>> wait
>>>>>>> for
>>>>>>>>>> them
>>>>>>>>>>>> to add it or do a custom framework to add that feature (as
>>>> now).
>>>>>>>>>>> Obviously
>>>>>>>>>>>> if you manage things with old-school ops tools
>>>> (puppet/chef/etc)
>>>>>>> you
>>>>>>>>>> get
>>>>>>>>>>>> locality easily. The nice thing, though, is that all the
>>> samza
>>>>>>>>>> "business
>>>>>>>>>>>> logic" around partition management and fault tolerance is in
>>>> Samza
>>>>>>>>>> core
>>>>>>>>>>> so
>>>>>>>>>>>> it is shared across frameworks and the framework specific
>>> bit
>>>> is
>>>>>>>> just
>>>>>>>>>>>> whether it is smart enough to try to get the same host when
>>> a
>>>> job
>>>>>>> is
>>>>>>>>>>>> restarted.
>>>>>>>>>>>> 
>>>>>>>>>>>> With respect to the Kafka-alignment, yeah I think the goal
>>>> would
>>>>>>> be
>>>>>>>>>> (a)
>>>>>>>>>>>> actually get better alignment in user experience, and (b)
>>>> express
>>>>>>>>>> this in
>>>>>>>>>>>> the naming and project branding. Specifically:
>>>>>>>>>>>> 1. Website/docs, it would be nice for the "transformation"
>>>> api to
>>>>>>> be
>>>>>>>>>>>> discoverable in the main Kafka docs--i.e. be able to explain
>>>> when
>>>>>>> to
>>>>>>>>>> use
>>>>>>>>>>>> the consumer and when to use the stream processing
>>>> functionality
>>>>>>> and
>>>>>>>>>> lead
>>>>>>>>>>>> people into that experience.
>>>>>>>>>>>> 2. Align releases so if you get Kafkza 1.4.2 (or whatever)
>>>> that
>>>>>>> has
>>>>>>>>>> both
>>>>>>>>>>>> Kafka and the stream processing part and they actually work
>>>>>>>> together.
>>>>>>>>>>>> 3. Unify the programming experience so the client and Samza
>>>> api
>>>>>>>> share
>>>>>>>>>>>> config/monitoring/naming/packaging/etc.
>>>>>>>>>>>> 
>>>>>>>>>>>> I think sub-projects keep separate committers and can have a
>>>>>>>> separate
>>>>>>>>>>> repo,
>>>>>>>>>>>> but I'm actually not really sure (I can't find a definition
>>>> of a
>>>>>>>>>>> subproject
>>>>>>>>>>>> in Apache).
>>>>>>>>>>>> 
>>>>>>>>>>>> Basically at a high-level you want the experience to "feel"
>>>> like a
>>>>>>>>>> single
>>>>>>>>>>>> system, not to relatively independent things that are kind
>>> of
>>>>>>>>>> awkwardly
>>>>>>>>>>>> glued together.
>>>>>>>>>>>> 
>>>>>>>>>>>> I think if we did that they having naming or branding like
>>>> "kafka
>>>>>>>>>>>> streaming" or "kafka streams" or something like that would
>>>>>>> actually
>>>>>>>>>> do a
>>>>>>>>>>>> good job of conveying what it is. I do that this would help
>>>>>>> adoption
>>>>>>>>>>> quite
>>>>>>>>>>>> a lot as it would correctly convey that using Kafka
>>> Streaming
>>>> with
>>>>>>>>>> Kafka
>>>>>>>>>>> is
>>>>>>>>>>>> a fairly seamless experience and Kafka is pretty heavily
>>>> adopted
>>>>>>> at
>>>>>>>>>> this
>>>>>>>>>>>> point.
>>>>>>>>>>>> 
>>>>>>>>>>>> Fwiw we actually considered this model originally when open
>>>>>>> sourcing
>>>>>>>>>>> Samza,
>>>>>>>>>>>> however at that time Kafka was relatively unknown and we
>>>> decided
>>>>>>> not
>>>>>>>>>> to
>>>>>>>>>>> do
>>>>>>>>>>>> it since we felt it would be limiting. From my point of view
>>>> the
>>>>>>>> three
>>>>>>>>>>>> things have changed (1) Kafka is now really heavily used for
>>>>>>> stream
>>>>>>>>>>>> processing, (2) we learned that abstracting out the stream
>>>> well is
>>>>>>>>>>>> basically impossible, (3) we learned it is really hard to
>>>> keep the
>>>>>>>> two
>>>>>>>>>>>> things feeling like a single product.
>>>>>>>>>>>> 
>>>>>>>>>>>> -Jay
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
>>>>>>>>>> martin@kleppmann.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Lots of good thoughts here.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I agree with the general philosophy of tying Samza more
>>>> firmly to
>>>>>>>>>> Kafka.
>>>>>>>>>>>>> After I spent a while looking at integrating other message
>>>>>>> brokers
>>>>>>>>>> (e.g.
>>>>>>>>>>>>> Kinesis) with SystemConsumer, I came to the conclusion that
>>>>>>>>>>> SystemConsumer
>>>>>>>>>>>>> tacitly assumes a model so much like Kafka's that pretty
>>> much
>>>>>>>> nobody
>>>>>>>>>> but
>>>>>>>>>>>>> Kafka actually implements it. (Databus is perhaps an
>>>> exception,
>>>>>>> but
>>>>>>>>>> it
>>>>>>>>>>>>> isn't widely used outside of LinkedIn.) Thus, making Samza
>>>> fully
>>>>>>>>>>> dependent
>>>>>>>>>>>>> on Kafka acknowledges that the system-independence was
>>> never
>>>> as
>>>>>>>> real
>>>>>>>>>> as
>>>>>>>>>>> we
>>>>>>>>>>>>> perhaps made it out to be. The gains of code reuse are
>>> real.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> The idea of decoupling Samza from YARN has also always been
>>>>>>>>>> appealing to
>>>>>>>>>>>>> me, for various reasons already mentioned in this thread.
>>>>>>> Although
>>>>>>>>>>> making
>>>>>>>>>>>>> Samza jobs deployable on anything (YARN/Mesos/AWS/etc)
>>> seems
>>>>>>>>>> laudable,
>>>>>>>>>>> I am
>>>>>>>>>>>>> a little concerned that it will restrict us to a lowest
>>>> common
>>>>>>>>>>> denominator.
>>>>>>>>>>>>> For example, would host affinity (SAMZA-617) still be
>>>> possible?
>>>>>>> For
>>>>>>>>>> jobs
>>>>>>>>>>>>> with large amounts of state, I think SAMZA-617 would be a
>>> big
>>>>>>> boon,
>>>>>>>>>>> since
>>>>>>>>>>>>> restoring state off the changelog on every single restart
>>> is
>>>>>>>> painful,
>>>>>>>>>>> due
>>>>>>>>>>>>> to long recovery times. It would be a shame if the
>>> decoupling
>>>>>>> from
>>>>>>>>>> YARN
>>>>>>>>>>>>> made host affinity impossible.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Jay, a question about the proposed API for instantiating a
>>>> job in
>>>>>>>>>> code
>>>>>>>>>>>>> (rather than a properties file): when submitting a job to a
>>>>>>>> cluster,
>>>>>>>>>> is
>>>>>>>>>>> the
>>>>>>>>>>>>> idea that the instantiation code runs on a client
>>> somewhere,
>>>>>>> which
>>>>>>>>>> then
>>>>>>>>>>>>> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or
>>> does
>>>> that
>>>>>>>>>> code
>>>>>>>>>>> run
>>>>>>>>>>>>> on each container that is part of the job (in which case,
>>> how
>>>>>>> does
>>>>>>>>>> the
>>>>>>>>>>> job
>>>>>>>>>>>>> submission to the cluster work)?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I agree with Garry that it doesn't feel right to make a 1.0
>>>>>>> release
>>>>>>>>>>> with a
>>>>>>>>>>>>> plan for it to be immediately obsolete. So if this is going
>>>> to
>>>>>>>>>> happen, I
>>>>>>>>>>>>> think it would be more honest to stick with 0.* version
>>>> numbers
>>>>>>>> until
>>>>>>>>>>> the
>>>>>>>>>>>>> library-ified Samza has been implemented, is stable and
>>>> widely
>>>>>>>> used.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Should the new Samza be a subproject of Kafka? There is
>>>> precedent
>>>>>>>> for
>>>>>>>>>>>>> tight coupling between different Apache projects (e.g.
>>>> Curator
>>>>>>> and
>>>>>>>>>>>>> Zookeeper, or Slider and YARN), so I think remaining
>>> separate
>>>>>>> would
>>>>>>>>>> be
>>>>>>>>>>> ok.
>>>>>>>>>>>>> Even if Samza is fully dependent on Kafka, there is enough
>>>>>>>> substance
>>>>>>>>>> in
>>>>>>>>>>>>> Samza that it warrants being a separate project. An
>>> argument
>>>> in
>>>>>>>>>> favour
>>>>>>>>>>> of
>>>>>>>>>>>>> merging would be if we think Kafka has a much stronger
>>> "brand
>>>>>>>>>> presence"
>>>>>>>>>>>>> than Samza; I'm ambivalent on that one. If the Kafka
>>> project
>>>> is
>>>>>>>>>> willing
>>>>>>>>>>> to
>>>>>>>>>>>>> endorse Samza as the "official" way of doing stateful
>>> stream
>>>>>>>>>>>>> transformations, that would probably have much the same
>>>> effect as
>>>>>>>>>>>>> re-branding Samza as "Kafka Stream Processors" or suchlike.
>>>> Close
>>>>>>>>>>>>> collaboration between the two projects will be needed in
>>> any
>>>>>>> case.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> From a project management perspective, I guess the "new
>>>> Samza"
>>>>>>>> would
>>>>>>>>>>> have
>>>>>>>>>>>>> to be developed on a branch alongside ongoing maintenance
>>> of
>>>> the
>>>>>>>>>> current
>>>>>>>>>>>>> line of development? I think it would be important to
>>>> continue
>>>>>>>>>>> supporting
>>>>>>>>>>>>> existing users, and provide a graceful migration path to
>>> the
>>>> new
>>>>>>>>>>> version.
>>>>>>>>>>>>> Leaving the current versions unsupported and forcing people
>>>> to
>>>>>>>>>> rewrite
>>>>>>>>>>>>> their jobs would send a bad signal.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> Martin
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io>
>>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Hey Garry,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Yeah that's super frustrating. I'd be happy to chat more
>>>> about
>>>>>>>> this
>>>>>>>>>> if
>>>>>>>>>>>>>> you'd be interested. I think Chris and I started with the
>>>> idea
>>>>>>> of
>>>>>>>>>> "what
>>>>>>>>>>>>>> would it take to make Samza a kick-ass ingestion tool" but
>>>>>>>>>> ultimately
>>>>>>>>>>> we
>>>>>>>>>>>>>> kind of came around to the idea that ingestion and
>>>>>>> transformation
>>>>>>>>>> had
>>>>>>>>>>>>>> pretty different needs and coupling the two made things
>>>> hard.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> For what it's worth I think copycat (KIP-26) actually will
>>>> do
>>>>>>> what
>>>>>>>>>> you
>>>>>>>>>>>>> are
>>>>>>>>>>>>>> looking for.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> With regard to your point about slider, I don't
>>> necessarily
>>>>>>>>>> disagree.
>>>>>>>>>>>>> But I
>>>>>>>>>>>>>> think getting good YARN support is quite doable and I
>>> think
>>>> we
>>>>>>> can
>>>>>>>>>> make
>>>>>>>>>>>>>> that work well. I think the issue this proposal solves is
>>>> that
>>>>>>>>>>>>> technically
>>>>>>>>>>>>>> it is pretty hard to support multiple cluster management
>>>> systems
>>>>>>>> the
>>>>>>>>>>> way
>>>>>>>>>>>>>> things are now, you need to write an "app master" or
>>>> "framework"
>>>>>>>> for
>>>>>>>>>>> each
>>>>>>>>>>>>>> and they are all a little different so testing is really
>>>> hard.
>>>>>>> In
>>>>>>>>>> the
>>>>>>>>>>>>>> absence of this we have been stuck with just YARN which
>>> has
>>>>>>>>>> fantastic
>>>>>>>>>>>>>> penetration in the Hadoopy part of the org, but zero
>>>> penetration
>>>>>>>>>>>>> elsewhere.
>>>>>>>>>>>>>> Given the huge amount of work being put in to slider,
>>>> marathon,
>>>>>>>> aws
>>>>>>>>>>>>>> tooling, not to mention the umpteen related packaging
>>>>>>> technologies
>>>>>>>>>>> people
>>>>>>>>>>>>>> want to use (Docker, Kubernetes, various cloud-specific
>>>> deploy
>>>>>>>>>> tools,
>>>>>>>>>>>>> etc)
>>>>>>>>>>>>>> I really think it is important to get this right.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>>>>>>>>>>>>>> g.turkington@improvedigital.com> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> I think the question below re does Samza become a
>>>> sub-project
>>>>>>> of
>>>>>>>>>> Kafka
>>>>>>>>>>>>>>> highlights the broader point around migration. Chris
>>>> mentions
>>>>>>>>>> Samza's
>>>>>>>>>>>>>>> maturity is heading towards a v1 release but I'm not sure
>>>> it
>>>>>>>> feels
>>>>>>>>>>>>> right to
>>>>>>>>>>>>>>> launch a v1 then immediately plan to deprecate most of
>>> it.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> From a selfish perspective I have some guys who have
>>>> started
>>>>>>>>>> working
>>>>>>>>>>>>> with
>>>>>>>>>>>>>>> Samza and building some new consumers/producers was next
>>>> up.
>>>>>>>> Sounds
>>>>>>>>>>> like
>>>>>>>>>>>>>>> that is absolutely not the direction to go. I need to
>>> look
>>>> into
>>>>>>>> the
>>>>>>>>>>> KIP
>>>>>>>>>>>>> in
>>>>>>>>>>>>>>> more detail but for me the attractiveness of adding new
>>>> Samza
>>>>>>>>>>>>>>> consumer/producers -- even if yes all they were doing was
>>>>>>> really
>>>>>>>>>>> getting
>>>>>>>>>>>>>>> data into and out of Kafka --  was to avoid  having to
>>>> worry
>>>>>>>> about
>>>>>>>>>> the
>>>>>>>>>>>>>>> lifecycle management of external clients. If there is a
>>>> generic
>>>>>>>>>> Kafka
>>>>>>>>>>>>>>> ingress/egress layer that I can plug a new connector into
>>>> and
>>>>>>>> have
>>>>>>>>>> a
>>>>>>>>>>>>> lot of
>>>>>>>>>>>>>>> the heavy lifting re scale and reliability done for me
>>>> then it
>>>>>>>>>> gives
>>>>>>>>>>> me
>>>>>>>>>>>>> all
>>>>>>>>>>>>>>> the pushing new consumers/producers would. If not then it
>>>>>>>>>> complicates
>>>>>>>>>>> my
>>>>>>>>>>>>>>> operational deployments.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Which is similar to my other question with the proposal
>>> --
>>>> if
>>>>>>> we
>>>>>>>>>>> build a
>>>>>>>>>>>>>>> fully available/stand-alone Samza plus the requisite
>>> shims
>>>> to
>>>>>>>>>>> integrate
>>>>>>>>>>>>>>> with Slider etc I suspect the former may be a lot more
>>> work
>>>>>>> than
>>>>>>>> we
>>>>>>>>>>>>> think.
>>>>>>>>>>>>>>> We may make it much easier for a newcomer to get
>>> something
>>>>>>>> running
>>>>>>>>>> but
>>>>>>>>>>>>>>> having them step up and get a reliable production
>>>> deployment
>>>>>>> may
>>>>>>>>>> still
>>>>>>>>>>>>>>> dominate mailing list  traffic, if for different reasons
>>>> than
>>>>>>>>>> today.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Don't get me wrong -- I'm comfortable with making the
>>> Samza
>>>>>>>>>> dependency
>>>>>>>>>>>>> on
>>>>>>>>>>>>>>> Kafka much more explicit and I absolutely see the
>>>> benefits  in
>>>>>>>> the
>>>>>>>>>>>>>>> reduction of duplication and clashing
>>>>>>> terminologies/abstractions
>>>>>>>>>> that
>>>>>>>>>>>>>>> Chris/Jay describe. Samza as a library would likely be a
>>>> very
>>>>>>>> nice
>>>>>>>>>>> tool
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> add to the Kafka ecosystem. I just have the concerns
>>> above
>>>> re
>>>>>>> the
>>>>>>>>>>>>>>> operational side.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Garry
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> -----Original Message-----
>>>>>>>>>>>>>>> From: Gianmarco De Francisci Morales [mailto:
>>>> gdfm@apache.org]
>>>>>>>>>>>>>>> Sent: 02 July 2015 12:56
>>>>>>>>>>>>>>> To: dev@samza.apache.org
>>>>>>>>>>>>>>> Subject: Re: Thoughts and obesrvations on Samza
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Very interesting thoughts.
>>>>>>>>>>>>>>> From outside, I have always perceived Samza as a
>>> computing
>>>>>>> layer
>>>>>>>>>> over
>>>>>>>>>>>>>>> Kafka.
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> The question, maybe a bit provocative, is "should Samza
>>> be
>>>> a
>>>>>>>>>>> sub-project
>>>>>>>>>>>>>>> of Kafka then?"
>>>>>>>>>>>>>>> Or does it make sense to keep it as a separate project
>>>> with a
>>>>>>>>>> separate
>>>>>>>>>>>>>>> governance?
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>> Gianmarco
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
>>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Overall, I agree to couple with Kafka more tightly.
>>>> Because
>>>>>>>> Samza
>>>>>>>>>> de
>>>>>>>>>>>>>>>> facto is based on Kafka, and it should leverage what
>>> Kafka
>>>>>>> has.
>>>>>>>> At
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> same time, Kafka does not need to reinvent what Samza
>>>> already
>>>>>>>>>> has. I
>>>>>>>>>>>>>>>> also like the idea of separating the ingestion and
>>>>>>>> transformation.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> But it is a little difficult for me to image how the
>>> Samza
>>>>>>> will
>>>>>>>>>> look
>>>>>>>>>>>>>>> like.
>>>>>>>>>>>>>>>> And I feel Chris and Jay have a little difference in
>>>> terms of
>>>>>>>> how
>>>>>>>>>>>>>>>> Samza should look like.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> *** Will it look like what Jay's code shows (A client of
>>>>>>> Kakfa)
>>>>>>>> ?
>>>>>>>>>> And
>>>>>>>>>>>>>>>> user's application code calls this client?
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1. If we make Samza be a library of Kafka (like what the
>>>> code
>>>>>>>>>> shows),
>>>>>>>>>>>>>>>> how do we implement auto-balance and fault-tolerance?
>>> Are
>>>> they
>>>>>>>>>> taken
>>>>>>>>>>>>>>>> care by the Kafka broker or other mechanism, such as
>>>> "Samza
>>>>>>>>>> worker"
>>>>>>>>>>>>>>>> (just make up the name) ?
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 2. What about other features, such as auto-scaling,
>>> shared
>>>>>>>> state,
>>>>>>>>>>>>>>>> monitoring?
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> *** If we have Samza standalone, (is this what Chris
>>>>>>> suggests?)
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 1. we still need to ingest data from Kakfa and produce
>>> to
>>>> it.
>>>>>>>>>> Then it
>>>>>>>>>>>>>>>> becomes the same as what Samza looks like now, except it
>>>> does
>>>>>>>> not
>>>>>>>>>>> rely
>>>>>>>>>>>>>>>> on Yarn anymore.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 2. if it is standalone, how can it leverage Kafka's
>>>> metrics,
>>>>>>>> logs,
>>>>>>>>>>>>>>>> etc? Use Kafka code as the dependency?
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Fang, Yan
>>>>>>>>>>>>>>>> yanfang724@gmail.com
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
>>>>>>>> wangguoz@gmail.com
>>>>>>>>>>> 
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Read through the code example and it looks good to me.
>>> A
>>>> few
>>>>>>>>>>>>>>>>> thoughts regarding deployment:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Today Samza deploys as executable runnable like:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> deploy/samza/bin/run-job.sh --config-factory=...
>>>>>>>>>>>>>>> --config-path=file://...
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> And this proposal advocate for deploying Samza more as
>>>>>>> embedded
>>>>>>>>>>>>>>>>> libraries in user application code (ignoring the
>>>> terminology
>>>>>>>>>> since
>>>>>>>>>>>>>>>>> it is not the
>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>> as the prototype code):
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> StreamTask task = new MyStreamTask(configs); Thread
>>>> thread =
>>>>>>>> new
>>>>>>>>>>>>>>>>> Thread(task); thread.start();
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> I think both of these deployment modes are important
>>> for
>>>>>>>>>> different
>>>>>>>>>>>>>>>>> types
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> users. That said, I think making Samza purely
>>> standalone
>>>> is
>>>>>>>> still
>>>>>>>>>>>>>>>>> sufficient for either runnable or library modes.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
>>>>>>> jay@confluent.io>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Looks like gmail mangled the code example, it was
>>>> supposed
>>>>>>> to
>>>>>>>>>> look
>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Properties props = new Properties();
>>>>>>>>>>>>>>>>>> props.put("bootstrap.servers", "localhost:4242");
>>>>>>>>>> StreamingConfig
>>>>>>>>>>>>>>>>>> config = new StreamingConfig(props);
>>>>>>>>>>>>>>>>>> config.subscribe("test-topic-1", "test-topic-2");
>>>>>>>>>>>>>>>>>> config.processor(ExampleStreamProcessor.class);
>>>>>>>>>>>>>>>>>> config.serialization(new StringSerializer(), new
>>>>>>>>>>>>>>>>>> StringDeserializer()); KafkaStreaming container = new
>>>>>>>>>>>>>>>>>> KafkaStreaming(config); container.run();
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
>>>>>>> jay@confluent.io
>>>>>>>>> 
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Hey guys,
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> This came out of some conversations Chris and I were
>>>> having
>>>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>> whether
>>>>>>>>>>>>>>>>>>> it would make sense to use Samza as a kind of data
>>>>>>> ingestion
>>>>>>>>>>>>>>>> framework
>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat").
>>> This
>>>>>>> kind
>>>>>>>> of
>>>>>>>>>>>>>>>>> combined
>>>>>>>>>>>>>>>>>>> with complaints around config and YARN and the
>>>> discussion
>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> best do a standalone mode.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> So the thought experiment was, given that Samza was
>>>>>>> basically
>>>>>>>>>>>>>>>>>>> already totally Kafka specific, what if you just
>>>> embraced
>>>>>>>> that
>>>>>>>>>>>>>>>>>>> and turned it
>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>> something less like a heavyweight framework and more
>>>> like a
>>>>>>>>>>>>>>>>>>> third
>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>> client--a kind of "producing consumer" with state
>>>>>>> management
>>>>>>>>>>>>>>>>> facilities.
>>>>>>>>>>>>>>>>>>> Basically a library. Instead of a complex stream
>>>> processing
>>>>>>>>>>>>>>>>>>> framework
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> would actually be a very simple thing, not much more
>>>>>>>>>> complicated
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>> operate than a Kafka consumer. As Chris said we
>>> thought
>>>>>>> about
>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> lot
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> what Samza (and the other stream processing systems
>>>> were
>>>>>>>> doing)
>>>>>>>>>>>>>>>> seemed
>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>> kind of a hangover from MapReduce.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Of course you need to ingest/output data to and from
>>>> the
>>>>>>>> stream
>>>>>>>>>>>>>>>>>>> processing. But when we actually looked into how that
>>>> would
>>>>>>>>>>>>>>>>>>> work,
>>>>>>>>>>>>>>>> Samza
>>>>>>>>>>>>>>>>>>> isn't really an ideal data ingestion framework for a
>>>> bunch
>>>>>>> of
>>>>>>>>>>>>>>>> reasons.
>>>>>>>>>>>>>>>>> To
>>>>>>>>>>>>>>>>>>> really do that right you need a pretty different
>>>> internal
>>>>>>>> data
>>>>>>>>>>>>>>>>>>> model
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> set of apis. So what if you split them and had an api
>>>> for
>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate
>>> api
>>>> for
>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>> transformation (Samza).
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> This would also allow really embracing the same
>>>> terminology
>>>>>>>> and
>>>>>>>>>>>>>>>>>>> conventions. One complaint about the current state is
>>>> that
>>>>>>>> the
>>>>>>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>>> systems
>>>>>>>>>>>>>>>>>>> kind of feel bolted on. Terminology like "stream" vs
>>>>>>> "topic"
>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>> config and monitoring systems means you kind of have
>>> to
>>>>>>> learn
>>>>>>>>>>>>>>>>>>> Kafka's
>>>>>>>>>>>>>>>>>> way,
>>>>>>>>>>>>>>>>>>> then learn Samza's slightly different way, then kind
>>> of
>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>> they
>>>>>>>>>>>>>>>>>>> map to each other, which having walked a few people
>>>> through
>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> is surprisingly tricky for folks to get.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Since I have been spending a lot of time on
>>> airplanes I
>>>>>>>> hacked
>>>>>>>>>>>>>>>>>>> up an ernest but still somewhat incomplete prototype
>>> of
>>>>>>> what
>>>>>>>>>>>>>>>>>>> this would
>>>>>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>> like. This is just unceremoniously dumped into Kafka
>>>> as it
>>>>>>>>>>>>>>>>>>> required a
>>>>>>>>>>>>>>>>> few
>>>>>>>>>>>>>>>>>>> changes to the new consumer. Here is the code:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>> 
>>>> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>>>>>>>>>>>>>>>> /apache/kafka/clients/streaming
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> For the purpose of the prototype I just liberally
>>>> renamed
>>>>>>>>>>>>>>>>>>> everything
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> try to align it with Kafka with no regard for
>>>>>>> compatibility.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> To use this would be something like this:
>>>>>>>>>>>>>>>>>>> Properties props = new Properties();
>>>>>>>>>>>>>>>>>>> props.put("bootstrap.servers", "localhost:4242");
>>>>>>>>>>>>>>>>>>> StreamingConfig config = new
>>>>>>>>>>>>>>>> StreamingConfig(props);
>>>>>>>>>>>>>>>>>> config.subscribe("test-topic-1",
>>>>>>>>>>>>>>>>>>> "test-topic-2");
>>>>>>>>>> config.processor(ExampleStreamProcessor.class);
>>>>>>>>>>>>>>>>>> config.serialization(new
>>>>>>>>>>>>>>>>>>> StringSerializer(), new StringDeserializer());
>>>>>>> KafkaStreaming
>>>>>>>>>>>>>>>>> container =
>>>>>>>>>>>>>>>>>>> new KafkaStreaming(config); container.run();
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> KafkaStreaming is basically the SamzaContainer;
>>>>>>>> StreamProcessor
>>>>>>>>>>>>>>>>>>> is basically StreamTask.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> So rather than putting all the class names in a file
>>>> and
>>>>>>> then
>>>>>>>>>>>>>>>>>>> having
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> job assembled by reflection, you just instantiate the
>>>>>>>> container
>>>>>>>>>>>>>>>>>>> programmatically. Work is balanced over however many
>>>>>>>> instances
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>> alive at any time (i.e. if an instance dies, new
>>> tasks
>>>> are
>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> existing containers without shutting them down).
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> We would provide some glue for running this stuff in
>>>> YARN
>>>>>>> via
>>>>>>>>>>>>>>>>>>> Slider, Mesos via Marathon, and AWS using some of
>>> their
>>>>>>> tools
>>>>>>>>>>>>>>>>>>> but from the
>>>>>>>>>>>>>>>>> point
>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> view of these frameworks these stream processing jobs
>>>> are
>>>>>>>> just
>>>>>>>>>>>>>>>>> stateless
>>>>>>>>>>>>>>>>>>> services that can come and go and expand and contract
>>>> at
>>>>>>>> will.
>>>>>>>>>>>>>>>>>>> There
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> no
>>>>>>>>>>>>>>>>>>> more custom scheduler.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Here are some relevant details:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 1. It is only ~1300 lines of code, it would get
>>>> larger if
>>>>>>> we
>>>>>>>>>>>>>>>>>>> productionized but not vastly larger. We really do
>>>> get a
>>>>>>> ton
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> leverage
>>>>>>>>>>>>>>>>>>> out of Kafka.
>>>>>>>>>>>>>>>>>>> 2. Partition management is fully delegated to the
>>> new
>>>>>>>>>> consumer.
>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>> is nice since now any partition management strategy
>>>>>>>> available
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>> consumer is also available to Samza (and vice versa)
>>>> and
>>>>>>>> with
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> exact
>>>>>>>>>>>>>>>>>>> same configs.
>>>>>>>>>>>>>>>>>>> 3. It supports state as well as state reuse
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Anyhow take a look, hopefully it is thought
>>> provoking.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>>>>>>>>>>>>>>>>> criccomini@apache.org>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Hey all,
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> I have had some discussions with Samza engineers at
>>>>>>> LinkedIn
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> Confluent
>>>>>>>>>>>>>>>>>>>> and we came up with a few observations and would
>>> like
>>>> to
>>>>>>>>>>>>>>>>>>>> propose
>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>> changes.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> We've observed some things that I want to call out
>>>> about
>>>>>>>>>>>>>>>>>>>> Samza's
>>>>>>>>>>>>>>>>> design,
>>>>>>>>>>>>>>>>>>>> and I'd like to propose some changes.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> * Samza is dependent upon a dynamic deployment
>>> system.
>>>>>>>>>>>>>>>>>>>> * Samza is too pluggable.
>>>>>>>>>>>>>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
>>>>>>> consumer
>>>>>>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>> trying to solve a lot of the same problems.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> All three of these issues are related, but I'll
>>>> address
>>>>>>> them
>>>>>>>>>> in
>>>>>>>>>>>>>>>> order.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Deployment
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Samza strongly depends on the use of a dynamic
>>>> deployment
>>>>>>>>>>>>>>>>>>>> scheduler
>>>>>>>>>>>>>>>>> such
>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we
>>>> bet
>>>>>>> that
>>>>>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> one or two winners in this area, and we could
>>> support
>>>>>>> them,
>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> rest
>>>>>>>>>>>>>>>>>>>> would go away. In reality, there are many
>>> variations.
>>>>>>>>>>>>>>>>>>>> Furthermore,
>>>>>>>>>>>>>>>>> many
>>>>>>>>>>>>>>>>>>>> people still prefer to just start their processors
>>>> like
>>>>>>>> normal
>>>>>>>>>>>>>>>>>>>> Java processes, and use traditional deployment
>>> scripts
>>>>>>> such
>>>>>>>> as
>>>>>>>>>>>>>>>>>>>> Fabric,
>>>>>>>>>>>>>>>>> Chef,
>>>>>>>>>>>>>>>>>>>> Ansible, etc. Forcing a deployment system on users
>>>> makes
>>>>>>> the
>>>>>>>>>>>>>>>>>>>> Samza start-up process really painful for first time
>>>>>>> users.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Dynamic deployment as a requirement was also a bit
>>> of
>>>> a
>>>>>>>>>>>>>>>>>>>> mis-fire
>>>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> a fundamental misunderstanding between the nature of
>>>> batch
>>>>>>>>>> jobs
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>> processing jobs. Early on, we made conscious effort
>>> to
>>>>>>> favor
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> Hadoop
>>>>>>>>>>>>>>>>>>>> (Map/Reduce) way of doing things, since it worked
>>> and
>>>> was
>>>>>>>> well
>>>>>>>>>>>>>>>>>> understood.
>>>>>>>>>>>>>>>>>>>> One thing that we missed was that batch jobs have a
>>>>>>> definite
>>>>>>>>>>>>>>>>> beginning,
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> end, and stream processing jobs don't (usually).
>>> This
>>>>>>> leads
>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>> simpler scheduling problem for stream processors.
>>> You
>>>>>>>>>> basically
>>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>>>> to find a place to start the processor, and start
>>> it.
>>>> The
>>>>>>>> way
>>>>>>>>>>>>>>>>>>>> we run grids, at LinkedIn, there's no concept of a
>>>> cluster
>>>>>>>>>>>>>>>>>>>> being "full". We always
>>>>>>>>>>>>>>>>> add
>>>>>>>>>>>>>>>>>>>> more machines. The problem with coupling Samza with
>>> a
>>>>>>>>>> scheduler
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> Samza (as a framework) now has to handle deployment.
>>>> This
>>>>>>>>>> pulls
>>>>>>>>>>>>>>>>>>>> in a
>>>>>>>>>>>>>>>>>> bunch
>>>>>>>>>>>>>>>>>>>> of things such as configuration distribution (config
>>>>>>>> stream),
>>>>>>>>>>>>>>>>>>>> shell
>>>>>>>>>>>>>>>>>> scrips
>>>>>>>>>>>>>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
>>>>>>> stuff),
>>>>>>>>>> etc.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Another reason for requiring dynamic deployment was
>>> to
>>>>>>>> support
>>>>>>>>>>>>>>>>>>>> data locality. If you want to have locality, you
>>> need
>>>> to
>>>>>>> put
>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>> processors
>>>>>>>>>>>>>>>>>>>> close to the data they're processing. Upon further
>>>>>>>>>>>>>>>>>>>> investigation,
>>>>>>>>>>>>>>>>>> though,
>>>>>>>>>>>>>>>>>>>> this feature is not that beneficial. There is some
>>>> good
>>>>>>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>> some problems with it on SAMZA-335. Again, we took
>>> the
>>>>>>>>>>>>>>>>>>>> Map/Reduce
>>>>>>>>>>>>>>>>> path,
>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>> there are some fundamental differences between HDFS
>>>> and
>>>>>>>> Kafka.
>>>>>>>>>>>>>>>>>>>> HDFS
>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>> blocks, while Kafka has partitions. This leads to
>>> less
>>>>>>>>>>>>>>>>>>>> optimization potential with stream processors on top
>>>> of
>>>>>>>> Kafka.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> This feature is also used as a crutch. Samza doesn't
>>>> have
>>>>>>>> any
>>>>>>>>>>>>>>>>>>>> built
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> fault-tolerance logic. Instead, it depends on the
>>>> dynamic
>>>>>>>>>>>>>>>>>>>> deployment scheduling system to handle restarts
>>> when a
>>>>>>>>>>>>>>>>>>>> processor dies. This has
>>>>>>>>>>>>>>>>>> made
>>>>>>>>>>>>>>>>>>>> it very difficult to write a standalone Samza
>>>> container
>>>>>>>>>>>>>>> (SAMZA-516).
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Pluggability
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> In some cases pluggability is good, but I think that
>>>> we've
>>>>>>>>>> gone
>>>>>>>>>>>>>>>>>>>> too
>>>>>>>>>>>>>>>>> far
>>>>>>>>>>>>>>>>>>>> with it. Currently, Samza has:
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> * Pluggable config.
>>>>>>>>>>>>>>>>>>>> * Pluggable metrics.
>>>>>>>>>>>>>>>>>>>> * Pluggable deployment systems.
>>>>>>>>>>>>>>>>>>>> * Pluggable streaming systems (SystemConsumer,
>>>>>>>> SystemProducer,
>>>>>>>>>>>>>>> etc).
>>>>>>>>>>>>>>>>>>>> * Pluggable serdes.
>>>>>>>>>>>>>>>>>>>> * Pluggable storage engines.
>>>>>>>>>>>>>>>>>>>> * Pluggable strategies for just about every
>>> component
>>>>>>>>>>>>>>>> (MessageChooser,
>>>>>>>>>>>>>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> There's probably more that I've forgotten, as well.
>>>> Some
>>>>>>> of
>>>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>> useful, but some have proven not to be. This all
>>>> comes at
>>>>>>> a
>>>>>>>>>> cost:
>>>>>>>>>>>>>>>>>>>> complexity. This complexity is making it harder for
>>>> our
>>>>>>>> users
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> pick
>>>>>>>>>>>>>>>>> up
>>>>>>>>>>>>>>>>>>>> and use Samza out of the box. It also makes it
>>>> difficult
>>>>>>> for
>>>>>>>>>>>>>>>>>>>> Samza developers to reason about what the
>>>> characteristics
>>>>>>> of
>>>>>>>>>>>>>>>>>>>> the container (since the characteristics change
>>>> depending
>>>>>>> on
>>>>>>>>>>>>>>>>>>>> which plugins are use).
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> The issues with pluggability are most visible in the
>>>>>>> System
>>>>>>>>>> APIs.
>>>>>>>>>>>>>>>> What
>>>>>>>>>>>>>>>>>>>> Samza really requires to be functional is Kafka as
>>> its
>>>>>>>>>>>>>>>>>>>> transport
>>>>>>>>>>>>>>>>> layer.
>>>>>>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>>>> we've conflated two unrelated use cases into one
>>> API:
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 1. Get data into/out of Kafka.
>>>>>>>>>>>>>>>>>>>> 2. Process the data in Kafka.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> The current System API supports both of these use
>>>> cases.
>>>>>>> The
>>>>>>>>>>>>>>>>>>>> problem
>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> actually want different features for each use case.
>>> By
>>>>>>>>>> papering
>>>>>>>>>>>>>>>>>>>> over
>>>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>> two use cases, and providing a single API, we've
>>>>>>> introduced
>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> ton of
>>>>>>>>>>>>>>>>>> leaky
>>>>>>>>>>>>>>>>>>>> abstractions.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> For example, what we'd really like in (2) is to have
>>>>>>>>>>>>>>>>>>>> monotonically increasing longs for offsets (like
>>>> Kafka).
>>>>>>>> This
>>>>>>>>>>>>>>>>>>>> would be at odds
>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>> (1),
>>>>>>>>>>>>>>>>>>>> though, since different systems have different
>>>>>>>>>>>>>>>>>> SCNs/Offsets/UUIDs/vectors.
>>>>>>>>>>>>>>>>>>>> There was discussion both on the mailing list and
>>> the
>>>> SQL
>>>>>>>>>> JIRAs
>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> need for this.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> The same thing holds true for replayability. Kafka
>>>> allows
>>>>>>> us
>>>>>>>>>> to
>>>>>>>>>>>>>>>> rewind
>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>> we have a failure. Many other systems don't. In some
>>>>>>> cases,
>>>>>>>>>>>>>>>>>>>> systems
>>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>> null for their offsets (e.g.
>>> WikipediaSystemConsumer)
>>>>>>>> because
>>>>>>>>>>>>>>>>>>>> they
>>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>> no
>>>>>>>>>>>>>>>>>>>> offsets.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Partitioning is another example. Kafka supports
>>>>>>>> partitioning,
>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>> many
>>>>>>>>>>>>>>>>>>>> systems don't. We model this by having a single
>>>> partition
>>>>>>>> for
>>>>>>>>>>>>>>>>>>>> those systems. Still, other systems model
>>> partitioning
>>>>>>>>>>>>>>> differently (e.g.
>>>>>>>>>>>>>>>>>>>> Kinesis).
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> The SystemAdmin interface is also a mess. Creating
>>>> streams
>>>>>>>> in
>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> system-agnostic way is almost impossible. As is
>>>> modeling
>>>>>>>>>>>>>>>>>>>> metadata
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> system (replication factor, partitions, location,
>>>> etc).
>>>>>>> The
>>>>>>>>>>>>>>>>>>>> list
>>>>>>>>>>>>>>>> goes
>>>>>>>>>>>>>>>>>> on.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Duplicate work
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> At the time that we began writing Samza, Kafka's
>>>> consumer
>>>>>>>> and
>>>>>>>>>>>>>>>> producer
>>>>>>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>>>> had a relatively weak feature set. On the
>>>> consumer-side,
>>>>>>> you
>>>>>>>>>>>>>>>>>>>> had two
>>>>>>>>>>>>>>>>>>>> options: use the high level consumer, or the simple
>>>>>>>> consumer.
>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>> problem
>>>>>>>>>>>>>>>>>>>> with the high-level consumer was that it controlled
>>>> your
>>>>>>>>>>>>>>>>>>>> offsets, partition assignments, and the order in
>>>> which you
>>>>>>>>>>>>>>>>>>>> received messages. The
>>>>>>>>>>>>>>>> problem
>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>> the simple consumer is that it's not simple. It's
>>>> basic.
>>>>>>> You
>>>>>>>>>>>>>>>>>>>> end up
>>>>>>>>>>>>>>>>>> having
>>>>>>>>>>>>>>>>>>>> to handle a lot of really low-level stuff that you
>>>>>>>> shouldn't.
>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>> spent a
>>>>>>>>>>>>>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
>>>>>>> robust.
>>>>>>>>>> It
>>>>>>>>>>>>>>>>>>>> also allows us to support some cool features:
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> * Per-partition message ordering and prioritization.
>>>>>>>>>>>>>>>>>>>> * Tight control over partition assignment to support
>>>>>>> joins,
>>>>>>>>>>>>>>>>>>>> global
>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>> (if we want to implement it :)), etc.
>>>>>>>>>>>>>>>>>>>> * Tight control over offset checkpointing.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> What we didn't realize at the time is that these
>>>> features
>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>> be in Kafka. A lot of Kafka consumers (not just
>>> Samza
>>>>>>> stream
>>>>>>>>>>>>>>>>> processors)
>>>>>>>>>>>>>>>>>>>> end up wanting to do things like joins and partition
>>>>>>>>>>>>>>>>>>>> assignment. The
>>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>> community has come to the same conclusion. They're
>>>> adding
>>>>>>> a
>>>>>>>>>> ton
>>>>>>>>>>>>>>>>>>>> of upgrades into their new Kafka consumer
>>>> implementation.
>>>>>>>> To a
>>>>>>>>>>>>>>>>>>>> large extent,
>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>> duplicate work to what we've already done in Samza.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> On top of this, Kafka ended up taking a very similar
>>>>>>>> approach
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> Samza's
>>>>>>>>>>>>>>>>>>>> KafkaCheckpointManager implementation for handling
>>>> offset
>>>>>>>>>>>>>>>>> checkpointing.
>>>>>>>>>>>>>>>>>>>> Like Samza, Kafka's new offset management feature
>>>> stores
>>>>>>>>>> offset
>>>>>>>>>>>>>>>>>>>> checkpoints in a topic, and allows you to fetch them
>>>> from
>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> broker.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> A lot of this seems like a waste, since we could
>>> have
>>>>>>> shared
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> work
>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>> had been done in Kafka from the get-go.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Vision
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> All of this leads me to a rather radical proposal.
>>>> Samza
>>>>>>> is
>>>>>>>>>>>>>>>> relatively
>>>>>>>>>>>>>>>>>>>> stable at this point. I'd venture to say that we're
>>>> near a
>>>>>>>> 1.0
>>>>>>>>>>>>>>>>> release.
>>>>>>>>>>>>>>>>>>>> I'd
>>>>>>>>>>>>>>>>>>>> like to propose that we take what we've learned, and
>>>> begin
>>>>>>>>>>>>>>>>>>>> thinking
>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>> Samza beyond 1.0. What would we change if we were
>>>> starting
>>>>>>>>>> from
>>>>>>>>>>>>>>>>> scratch?
>>>>>>>>>>>>>>>>>>>> My
>>>>>>>>>>>>>>>>>>>> proposal is to:
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
>>>>>>>>>>>>>>>>>>>> processors, and eliminate all direct dependences on
>>>> YARN,
>>>>>>>>>> Mesos,
>>>>>>>>>>>>>>> etc.
>>>>>>>>>>>>>>>>>>>> 2. Make a definitive call to support only Kafka as
>>> the
>>>>>>>> stream
>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>> layer.
>>>>>>>>>>>>>>>>>>>> 3. Eliminate Samza's metrics, logging,
>>> serialization,
>>>> and
>>>>>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>> systems,
>>>>>>>>>>>>>>>>>>>> and simply use Kafka's instead.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> This would fix all of the issues that I outlined
>>>> above. It
>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>> shrink the Samza code base pretty dramatically.
>>>> Supporting
>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>> a standalone container will allow Samza to be
>>>> executed on
>>>>>>>> YARN
>>>>>>>>>>>>>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or
>>> most
>>>>>>> other
>>>>>>>>>>>>>>>>>>>> in-house
>>>>>>>>>>>>>>>>>> deployment
>>>>>>>>>>>>>>>>>>>> systems. This should make life a lot easier for new
>>>> users.
>>>>>>>>>>>>>>>>>>>> Imagine
>>>>>>>>>>>>>>>>>> having
>>>>>>>>>>>>>>>>>>>> the hello-samza tutorial without YARN. The drop in
>>>> mailing
>>>>>>>>>> list
>>>>>>>>>>>>>>>>> traffic
>>>>>>>>>>>>>>>>>>>> will be pretty dramatic.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Coupling with Kafka seems long overdue to me. The
>>>> reality
>>>>>>>> is,
>>>>>>>>>>>>>>>> everyone
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> I'm aware of is using Samza with Kafka. We basically
>>>>>>> require
>>>>>>>>>> it
>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> order for most features to work. Those that are
>>> using
>>>>>>> other
>>>>>>>>>>>>>>>>>>>> systems
>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>> generally using it for ingest into Kafka (1), and
>>> then
>>>>>>> they
>>>>>>>> do
>>>>>>>>>>>>>>>>>>>> the processing on top. There is already discussion (
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>> 
>>>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>>>>>>>>>>>>>>>> 767
>>>>>>>>>>>>>>>>>>>> )
>>>>>>>>>>>>>>>>>>>> in Kafka to make ingesting into Kafka extremely
>>> easy.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Once we make the call to couple with Kafka, we can
>>>>>>> leverage
>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> ton of
>>>>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>> ecosystem. We no longer have to maintain our own
>>>> config,
>>>>>>>>>>>>>>>>>>>> metrics,
>>>>>>>>>>>>>>>> etc.
>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>> can all share the same libraries, and make them
>>>> better.
>>>>>>> This
>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>>> allow us to share the consumer/producer APIs, and
>>>> will let
>>>>>>>> us
>>>>>>>>>>>>>>>> leverage
>>>>>>>>>>>>>>>>>>>> their offset management and partition management,
>>>> rather
>>>>>>>> than
>>>>>>>>>>>>>>>>>>>> having
>>>>>>>>>>>>>>>>> our
>>>>>>>>>>>>>>>>>>>> own. All of the coordinator stream code would go
>>>> away, as
>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> most
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> YARN AppMaster code. We'd probably have to push some
>>>>>>>> partition
>>>>>>>>>>>>>>>>>> management
>>>>>>>>>>>>>>>>>>>> features into the Kafka broker, but they're already
>>>> moving
>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> that direction with the new consumer API. The
>>>> features we
>>>>>>>> have
>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>> partition
>>>>>>>>>>>>>>>>>>>> assignment aren't unique to Samza, and seem like
>>> they
>>>>>>> should
>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>> anyway. There will always be some niche usages which
>>>> will
>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>> extra
>>>>>>>>>>>>>>>>>>>> care and hence full control over partition
>>> assignments
>>>>>>> much
>>>>>>>>>>>>>>>>>>>> like the
>>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>> low level consumer api. These would continue to be
>>>>>>>> supported.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> These items will be good for the Samza community.
>>>> They'll
>>>>>>>> make
>>>>>>>>>>>>>>>>>>>> Samza easier to use, and make it easier for
>>>> developers to
>>>>>>>> add
>>>>>>>>>>>>>>>>>>>> new features.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Obviously this is a fairly large (and somewhat
>>>> backwards
>>>>>>>>>>>>>>>> incompatible
>>>>>>>>>>>>>>>>>>>> change). If we choose to go this route, it's
>>> important
>>>>>>> that
>>>>>>>> we
>>>>>>>>>>>>>>>> openly
>>>>>>>>>>>>>>>>>>>> communicate how we're going to provide a migration
>>>> path
>>>>>>> from
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>> APIs to the new ones (if we make incompatible
>>>> changes). I
>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>> at a minimum, we'd probably need to provide a
>>> wrapper
>>>> to
>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>> existing StreamTask implementations to continue
>>>> running on
>>>>>>>> the
>>>>>>>>>>>>>>> new container.
>>>>>>>>>>>>>>>>>> It's
>>>>>>>>>>>>>>>>>>>> also important that we openly communicate about
>>>> timing,
>>>>>>> and
>>>>>>>>>>>>>>>>>>>> stages
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> migration.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> If you made it this far, I'm sure you have opinions.
>>>> :)
>>>>>>>> Please
>>>>>>>>>>>>>>>>>>>> send
>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>> thoughts and feedback.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>> Chris
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>> 
>>> 
>> 
>> 
>> 
>> --
>> Jordan Shaw
>> Full Stack Software Engineer
>> PubNub Inc
>> 1045 17th St
>> San Francisco, CA 94107


Re: Thoughts and obesrvations on Samza

Posted by Julian Hyde <jh...@apache.org>.
Wow, what a great discussion. A brave discussion, since no project
wants to reduce its scope. And important, because "right-sizing"
technology components can help them win in the long run.

I have a couple of let-me-play-devil's-advocate questions.

1. Community over code

Let's look at this in terms of the Apache Way. The Apache Way
advocates "community over code", and as Jakob points out, the Samza
community is distinct from the Kafka community. It seems that we are
talking here about Samza-the-code.

According to the Apache Way, what Samza-the-project should be doing is
what Samza-the-community is good at. Samza-the-code-as-it-is-today can
move to Kafka, stay in Samza, or be deleted if it has been superseded.

Architectural discussions are important to have, and the Apache Way
gets in the way of good architecture sometimes. When we're thinking
about moving code, let's also think about the community of people
working on the code.

Apache Phoenix is a good analogy. Phoenix is technically very closely
tied to HBase, but a distinct community, with different skill-sets.
(HBase, like Kafka, is "hard core", and not for everyone.) They have
also been good at re-examining their project scope and re-scoping
where necessary.

2. Architecture

This proposal retreats from the grand vision of "distributed stream
management system" where not only storage is distributed but also
processing. There is no architectural piece that says "I need 10 JVMs
to process this CPU intensive standing query and I currently only have
6." What projects, current or envisioned, would fit that gap? Is that
work a good fit for the Samza community?

Julian



On Wed, Jul 8, 2015 at 10:47 PM, Jordan Shaw <jo...@pubnub.com> wrote:
> I'm all for any optimizations that can be made to the Yarn workflow.
>
> I actually agree with Jakob in regard to the producers/consumers. I have
> spent sometime writing consumers and producers for other transport
> abstractions and overall the current api abstractions in Samza I feel are
> pretty good. There are some things that are sort of anomalous and catered
> more toward the Kafka model but easy enough to work around and I've been
> able to make other Producers and Consumers work that are no where near the
> same paradigm as Kafka.
>
> To Jay's point although Kafka is great and does the streaming data paradigm
> very well there is really no reason why a different transport application
> implemented properly wouldn't be able to stream data with the same
> effectiveness as Kafka and that transport may suite the user's use case
> better or be more cost effective than Kafka. For example we had to decide
> if Kafka was worth the extra cost of running a zookeeper cluster and if the
> scaling through partitioning was worth the operational overhead vs having a
> mesh network over ZeroMQ. After deciding that our use case would fit with
> Kafka fine there were other challenges like understanding how AWS EC2 SSD's
> behaved (AWS amortizes all disk into into Random I/O this is bad for Kafka).
>
> Thus, I would lend of the side of transport flexibility for a framework
> like Samza over binding to a transport medium like Kafka.
>
>
> On Wed, Jul 8, 2015 at 1:39 PM, Jay Kreps <ja...@gmail.com> wrote:
>
>> Good summary Jakob.
>>
>> WRT to the general purpose vs Kafka-specific, I actually see it slightly
>> differently. Consider how Storm works as an example, there is a data source
>> (spout) which could be Kafka, Database, etc, and then there is a transport
>> (a netty TCP thing iiuc). Storm allows you to process data from any source,
>> but when it comes from a source they always funnel it through their
>> transport to get to the tasks/bolts. It is natural to think of Kafka as the
>> Spout, but I think the better analogy is actually that Kafka is the
>> transport.
>>
>> It is really hard to make the transport truly pluggable because this is
>> what the tasks interact with and you need to have guarantees about delivery
>> (and reprocessing), partitioning, atomicity of output, ordering, etc so
>> your stream processing can get the right answer. From my point of view what
>> this proposal says is that Kafka would be non-pluggable as the *transport*.
>>
>> So in this proposal data would still come into and out of Kafka from a wide
>> variety of sources, but by requiring Kafka as the transport the interaction
>> with the tasks will always look the same (a persistent, partitioned, log).
>> So going back to the Storm analogy it is something like
>>   Spout interface = copy cat
>>   Bolt interface = samza
>>
>> This does obviously make Samza dependent on Kafka but it doesn't mean you
>> wouldn't be processing data from all kinds of sources--indeed that is the
>> whole purpose. It just means that each of these data streams would be
>> available as a multi-subscriber Kafka topic to other systems, applications,
>> etc, not just for your job.
>>
>> If you think about how things are now Samza already depends on a
>> partitioned, persistent, offset addressable log with log
>> compaction...which, unsurprisingly, so I don't think this is really a new
>> dependency.
>>
>> Philosophically I think this makes sense too. To make a bunch of programs
>> fit together you have to standardize something. In this proposal what you
>> are standardizing around is really Kafka's protocol for streaming data and
>> your data format. The transformations that connect these streams can be
>> done via Samza, Storm, Spark, standalone java or python programs, etc but
>> the ultimate output and contract to the rest of the organization/world will
>> be the resulting Kafka topic. Philosophically I think this kind of data and
>> protocol based contract is the right way to go rather than saying that the
>> contract is a particular java api and the stream/data is what is pluggable.
>>
>> -Jay
>>
>>
>>
>> On Wed, Jul 8, 2015 at 11:03 AM, Jakob Homan <jg...@gmail.com> wrote:
>>
>> > Rewinding back to the beginning of this topic, there are effectively
>> > three proposals on the table:
>> >
>> > 1) Chris' ideas for a direction towards a 2.0 release with an emphasis
>> > on API and configuration simplification.  This ideas are based on lots
>> > of lessons learned from the 0.x branch and are worthy of a 2.0 label
>> > and breaking backwards compability.  I'm not sure I agree with all of
>> > them, but they're definitely worth pursuing.
>> >
>> > 2) Chris' alternative proposal, which goes beyond his first and is
>> > essentially a reboot of Samza to a more limited, entirely
>> > Kafka-focused approach.  Samza would cease being a general purpose
>> > stream processing framework, akin to and an alternative to say, Apache
>> > Storm, and would instead become a standalone complement to the Kafka
>> > project.
>> >
>> > 3) Jay's proposal, which goes even further, and suggests that the
>> > Kafka community would be better served by adding stream processing as
>> > a module to Kafka.  This is a perfectly valid approach, but since it's
>> > entirely confined to the Kafka project, doesn't really involve Samza.
>> > If the Kafka team were to go this route, there would be no obligation
>> > on the Samza team to shut down, disband, etc.
>> >
>> > This last bit is important because Samza and Kafka, while closely
>> > linked, are distinct communities.  The intersection of committers on
>> > both Kafka and Samza is three people out of a combined 18 committers
>> > across both projects.   Samza is a distinct community that shares
>> > quite a few users with Kafka, but is able to chart its own course.
>> >
>> > My own view is that Samza has had an amazing year and is taking off at
>> > a rapid rate.  It was only proposed for Incubator two years ago and is
>> > still very young. The original team at LinkedIn has left that company
>> > but the project has continued to grow via contributions both from
>> > LinkedIn and from without.  We've recently seen a significant uptake
>> > in discussion and bug reports.
>> >
>> > The API, deployment and configuration changes Chris suggests are good
>> > ideas, but I think there is still serious value in having a
>> > stand-alone general stream processing framework that supports other
>> > input sources than Kafka.  We've already had contributions for adding
>> > producer support to ElasticSearch and HDFS.  As more users come on
>> > board, I would expect them to contribute more consumers and producers.
>> >
>> > It's a bit of chicken-and-the-egg problem; since the original team
>> > didn't have cycles to prioritize support for non-Kafka systems
>> > (kinesis, eventhub, twitter, flume, zeromq, etc.), Samza was less
>> > compelling than other stream processing frameworks that did have
>> > support and was therefore not used in those situations.  I'd love to
>> > see those added and the SystemConsumer/Producer APIs improved to
>> > fluently support them as well as Kafka.
>> > Martin had a question regarding the tight coupling between Hadoop HDFS
>> > and MapReduce (and YARN and Common).  This has been a problem for
>> > years and there have been several aborted attempts to split the
>> > projects out.  Each time there turned out to be a strong need for
>> > cross-cutting collaboration and so the effort was dropped.  Absent the
>> > third option above (Kafka adding stream support to itself directly), I
>> > would imagine something similar would play out here.
>> >
>> > We should get a feeling for which of the three proposals the Samza
>> > community is behind, technical details of each notwithstanding.  This
>> > would include not just the committers/PMC members, but also the users,
>> > contributors and lurkers.
>> >
>> > -Jakob
>> >
>> > On 8 July 2015 at 07:41, Ben Kirwin <be...@kirw.in> wrote:
>> > > Hi all,
>> > >
>> > > Interesting stuff! Jumping in a bit late, but here goes...
>> > >
>> > > I'd definitely be excited about a slimmed-down and more Kafka-specific
>> > > Samza -- you don't seem to lose much functionality that people
>> > > actually use, and the gains in simplicity / code sharing seem
>> > > potentially very large. (I've spent a bunch of time peeling back those
>> > > layers of abstraction to get eg. more control over message send order,
>> > > and working directly against Kafka's APIs would have been much
>> > > easier.) I also like the approach of letting Kafka code do the heavy
>> > > lifting and letting stream processing systems build on those -- good,
>> > > reusable implementations would be great for the whole
>> > > stream-processing ecosystem, and Samza in particular.
>> > >
>> > > On the other hand, I do hope that using Kafka's group membership /
>> > > partition assignment / etc. stays optional. As far as I can tell,
>> > > ~every major stream processing system that uses Kafka has chosen (or
>> > > switched to) 'static' partitioning, where each logical task consumes a
>> > > fixed set of partitions. When 'dynamic deploying' (a la Storm / Mesos
>> > > / Yarn) the underlying system is already doing failure detection and
>> > > transferring work between hosts when machines go down, so using
>> > > Kafka's implementation is redundant at best -- and at worst, the
>> > > interaction between the two systems can make outages worse.
>> > >
>> > > And thanks to Chris / Jay for getting this ball rolling. Exciting
>> > times...
>> > >
>> > > On Tue, Jul 7, 2015 at 2:35 PM, Jay Kreps <ja...@confluent.io> wrote:
>> > >> Hey Roger,
>> > >>
>> > >> I couldn't agree more. We spent a bunch of time talking to people and
>> > that
>> > >> is exactly the stuff we heard time and again. What makes it hard, of
>> > >> course, is that there is some tension between compatibility with
>> what's
>> > >> there now and making things better for new users.
>> > >>
>> > >> I also strongly agree with the importance of multi-language support.
>> We
>> > are
>> > >> talking now about Java, but for application development use cases
>> people
>> > >> want to work in whatever language they are using elsewhere. I think
>> > moving
>> > >> to a model where Kafka itself does the group membership, lifecycle
>> > control,
>> > >> and partition assignment has the advantage of putting all that complex
>> > >> stuff behind a clean api that the clients are already going to be
>> > >> implementing for their consumer, so the added functionality for stream
>> > >> processing beyond a consumer becomes very minor.
>> > >>
>> > >> -Jay
>> > >>
>> > >> On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <roger.hoover@gmail.com
>> >
>> > >> wrote:
>> > >>
>> > >>> Metamorphosis...nice. :)
>> > >>>
>> > >>> This has been a great discussion.  As a user of Samza who's recently
>> > >>> integrated it into a relatively large organization, I just want to
>> add
>> > >>> support to a few points already made.
>> > >>>
>> > >>> The biggest hurdles to adoption of Samza as it currently exists that
>> > I've
>> > >>> experienced are:
>> > >>> 1) YARN - YARN is overly complex in many environments where Puppet
>> > would do
>> > >>> just fine but it was the only mechanism to get fault tolerance.
>> > >>> 2) Configuration - I think I like the idea of configuring most of the
>> > job
>> > >>> in code rather than config files.  In general, I think the goal
>> should
>> > be
>> > >>> to make it harder to make mistakes, especially of the kind where the
>> > code
>> > >>> expects something and the config doesn't match.  The current config
>> is
>> > >>> quite intricate and error-prone.  For example, the application logic
>> > may
>> > >>> depend on bootstrapping a topic but rather than asserting that in the
>> > code,
>> > >>> you have to rely on getting the config right.  Likewise with serdes,
>> > the
>> > >>> Java representations produced by various serdes (JSON, Avro, etc.)
>> are
>> > not
>> > >>> equivalent so you cannot just reconfigure a serde without changing
>> the
>> > >>> code.   It would be nice for jobs to be able to assert what they
>> expect
>> > >>> from their input topics in terms of partitioning.  This is getting a
>> > little
>> > >>> off topic but I was even thinking about creating a "Samza config
>> > linter"
>> > >>> that would sanity check a set of configs.  Especially in
>> organizations
>> > >>> where config is managed by a different team than the application
>> > developer,
>> > >>> it's very hard to get avoid config mistakes.
>> > >>> 3) Java/Scala centric - for many teams (especially DevOps-type
>> folks),
>> > the
>> > >>> pain of the Java toolchain (maven, slow builds, weak command line
>> > support,
>> > >>> configuration over convention) really inhibits productivity.  As more
>> > and
>> > >>> more high-quality clients become available for Kafka, I hope they'll
>> > follow
>> > >>> Samza's model.  Not sure how much it affects the proposals in this
>> > thread
>> > >>> but please consider other languages in the ecosystem as well.  From
>> > what
>> > >>> I've heard, Spark has more Python users than Java/Scala.
>> > >>> (FYI, we added a Jython wrapper for the Samza API
>> > >>>
>> > >>>
>> >
>> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
>> > >>> and are working on a Yeoman generator
>> > >>> https://github.com/Quantiply/generator-rico for Jython/Samza
>> projects
>> > to
>> > >>> alleviate some of the pain)
>> > >>>
>> > >>> I also want to underscore Jay's point about improving the user
>> > experience.
>> > >>> That's a very important factor for adoption.  I think the goal should
>> > be to
>> > >>> make Samza as easy to get started with as something like Logstash.
>> > >>> Logstash is vastly inferior in terms of capabilities to Samza but
>> it's
>> > easy
>> > >>> to get started and that makes a big difference.
>> > >>>
>> > >>> Cheers,
>> > >>>
>> > >>> Roger
>> > >>>
>> > >>>
>> > >>>
>> > >>>
>> > >>>
>> > >>> On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
>> > >>> gdfm@apache.org> wrote:
>> > >>>
>> > >>> > Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
>> > >>> winner
>> > >>> > :)
>> > >>> >
>> > >>> > --
>> > >>> > Gianmarco
>> > >>> >
>> > >>> > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
>> > gdfm@apache.org
>> > >>> >
>> > >>> > wrote:
>> > >>> >
>> > >>> > > Hi,
>> > >>> > >
>> > >>> > > @Martin, thanks for you comments.
>> > >>> > > Maybe I'm missing some important point, but I think coupling the
>> > >>> releases
>> > >>> > > is actually a *good* thing.
>> > >>> > > To make an example, would it be better if the MR and HDFS
>> > components of
>> > >>> > > Hadoop had different release schedules?
>> > >>> > >
>> > >>> > > Actually, keeping the discussion in a single place would make
>> > agreeing
>> > >>> on
>> > >>> > > releases (and backwards compatibility) much easier, as everybody
>> > would
>> > >>> be
>> > >>> > > responsible for the whole codebase.
>> > >>> > >
>> > >>> > > That said, I like the idea of absorbing samza-core as a
>> > sub-project,
>> > >>> and
>> > >>> > > leave the fancy stuff separate.
>> > >>> > > It probably gives 90% of the benefits we have been discussing
>> here.
>> > >>> > >
>> > >>> > > Cheers,
>> > >>> > >
>> > >>> > > --
>> > >>> > > Gianmarco
>> > >>> > >
>> > >>> > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
>> > >>> > >
>> > >>> > >> Hey Martin,
>> > >>> > >>
>> > >>> > >> I agree coupling release schedules is a downside.
>> > >>> > >>
>> > >>> > >> Definitely we can try to solve some of the integration problems
>> in
>> > >>> > >> Confluent Platform or in other distributions. But I think this
>> > ends up
>> > >>> > >> being really shallow. I guess I feel to really get a good user
>> > >>> > experience
>> > >>> > >> the two systems have to kind of feel like part of the same thing
>> > and
>> > >>> you
>> > >>> > >> can't really add that in later--you can put both in the same
>> > >>> > downloadable
>> > >>> > >> tar file but it doesn't really give a very cohesive feeling. I
>> > agree
>> > >>> > that
>> > >>> > >> ultimately any of the project stuff is as much social and naming
>> > as
>> > >>> > >> anything else--theoretically two totally independent projects
>> > could
>> > >>> work
>> > >>> > >> to
>> > >>> > >> tightly align. In practice this seems to be quite difficult
>> > though.
>> > >>> > >>
>> > >>> > >> For the frameworks--totally agree it would be good to maintain
>> the
>> > >>> > >> framework support with the project. In some cases there may not
>> > be too
>> > >>> > >> much
>> > >>> > >> there since the integration gets lighter but I think whatever
>> > stubs
>> > >>> you
>> > >>> > >> need should be included. So no I definitely wasn't trying to
>> imply
>> > >>> > >> dropping
>> > >>> > >> support for these frameworks, just making the integration
>> lighter
>> > by
>> > >>> > >> separating process management from partition management.
>> > >>> > >>
>> > >>> > >> You raise two good points we would have to figure out if we went
>> > down
>> > >>> > the
>> > >>> > >> alignment path:
>> > >>> > >> 1. With respect to the name, yeah I think the first question is
>> > >>> whether
>> > >>> > >> some "re-branding" would be worth it. If so then I think we can
>> > have a
>> > >>> > big
>> > >>> > >> thread on the name. I'm definitely not set on Kafka Streaming or
>> > Kafka
>> > >>> > >> Streams I was just using them to be kind of illustrative. I
>> agree
>> > with
>> > >>> > >> your
>> > >>> > >> critique of these names, though I think people would get the
>> idea.
>> > >>> > >> 2. Yeah you also raise a good point about how to "factor" it.
>> > Here are
>> > >>> > the
>> > >>> > >> options I see (I could get enthusiastic about any of them):
>> > >>> > >>    a. One repo for both Kafka and Samza
>> > >>> > >>    b. Two repos, retaining the current seperation
>> > >>> > >>    c. Two repos, the equivalent of samza-api and samza-core is
>> > >>> absorbed
>> > >>> > >> almost like a third client
>> > >>> > >>
>> > >>> > >> Cheers,
>> > >>> > >>
>> > >>> > >> -Jay
>> > >>> > >>
>> > >>> > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
>> > >>> martin@kleppmann.com>
>> > >>> > >> wrote:
>> > >>> > >>
>> > >>> > >> > Ok, thanks for the clarifications. Just a few follow-up
>> > comments.
>> > >>> > >> >
>> > >>> > >> > - I see the appeal of merging with Kafka or becoming a
>> > subproject:
>> > >>> the
>> > >>> > >> > reasons you mention are good. The risk I see is that release
>> > >>> schedules
>> > >>> > >> > become coupled to each other, which can slow everyone down,
>> and
>> > >>> large
>> > >>> > >> > projects with many contributors are harder to manage. (Jakob,
>> > can
>> > >>> you
>> > >>> > >> speak
>> > >>> > >> > from experience, having seen a wider range of Hadoop ecosystem
>> > >>> > >> projects?)
>> > >>> > >> >
>> > >>> > >> > Some of the goals of a better unified developer experience
>> could
>> > >>> also
>> > >>> > be
>> > >>> > >> > solved by integrating Samza nicely into a Kafka distribution
>> > (such
>> > >>> as
>> > >>> > >> > Confluent's). I'm not against merging projects if we decide
>> > that's
>> > >>> the
>> > >>> > >> way
>> > >>> > >> > to go, just pointing out the same goals can perhaps also be
>> > achieved
>> > >>> > in
>> > >>> > >> > other ways.
>> > >>> > >> >
>> > >>> > >> > - With regard to dropping the YARN dependency: are you
>> proposing
>> > >>> that
>> > >>> > >> > Samza doesn't give any help to people wanting to run on
>> > >>> > >> YARN/Mesos/AWS/etc?
>> > >>> > >> > So the docs would basically have a link to Slider and nothing
>> > else?
>> > >>> Or
>> > >>> > >> > would we maintain integrations with a bunch of popular
>> > deployment
>> > >>> > >> methods
>> > >>> > >> > (e.g. the necessary glue and shell scripts to make Samza work
>> > with
>> > >>> > >> Slider)?
>> > >>> > >> >
>> > >>> > >> > I absolutely think it's a good idea to have the "as a library"
>> > and
>> > >>> > "as a
>> > >>> > >> > process" (using Yi's taxonomy) options for people who want
>> them,
>> > >>> but I
>> > >>> > >> > think there should also be a low-friction path for common "as
>> a
>> > >>> > service"
>> > >>> > >> > deployment methods, for which we probably need to maintain
>> > >>> > integrations.
>> > >>> > >> >
>> > >>> > >> > - Project naming: "Kafka Streams" seems odd to me, because
>> > Kafka is
>> > >>> > all
>> > >>> > >> > about streams already. Perhaps "Kafka Transformers" or "Kafka
>> > >>> Filters"
>> > >>> > >> > would be more apt?
>> > >>> > >> >
>> > >>> > >> > One suggestion: perhaps the core of Samza (stream
>> transformation
>> > >>> with
>> > >>> > >> > state management -- i.e. the "Samza as a library" bit) could
>> > become
>> > >>> > >> part of
>> > >>> > >> > Kafka, while higher-level tools such as streaming SQL and
>> > >>> integrations
>> > >>> > >> with
>> > >>> > >> > deployment frameworks remain in a separate project? In other
>> > words,
>> > >>> > >> Kafka
>> > >>> > >> > would absorb the proven, stable core of Samza, which would
>> > become
>> > >>> the
>> > >>> > >> > "third Kafka client" mentioned early in this thread. The Samza
>> > >>> project
>> > >>> > >> > would then target that third Kafka client as its base API, and
>> > the
>> > >>> > >> project
>> > >>> > >> > would be freed up to explore more experimental new horizons.
>> > >>> > >> >
>> > >>> > >> > Martin
>> > >>> > >> >
>> > >>> > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com>
>> wrote:
>> > >>> > >> >
>> > >>> > >> > > Hey Martin,
>> > >>> > >> > >
>> > >>> > >> > > For the YARN/Mesos/etc decoupling I actually don't think it
>> > ties
>> > >>> our
>> > >>> > >> > hands
>> > >>> > >> > > at all, all it does is refactor things. The division of
>> > >>> > >> responsibility is
>> > >>> > >> > > that Samza core is responsible for task lifecycle, state,
>> and
>> > >>> > >> partition
>> > >>> > >> > > management (using the Kafka co-ordinator) but it is NOT
>> > >>> responsible
>> > >>> > >> for
>> > >>> > >> > > packaging, configuration deployment or execution of
>> > processes. The
>> > >>> > >> > problem
>> > >>> > >> > > of packaging and starting these processes is
>> > >>> > >> > > framework/environment-specific. This leaves individual
>> > frameworks
>> > >>> to
>> > >>> > >> be
>> > >>> > >> > as
>> > >>> > >> > > fancy or vanilla as they like. So you can get simple
>> stateless
>> > >>> > >> support in
>> > >>> > >> > > YARN, Mesos, etc using their off-the-shelf app framework
>> > (Slider,
>> > >>> > >> > Marathon,
>> > >>> > >> > > etc). These are well known by people and have nice UIs and a
>> > lot
>> > >>> of
>> > >>> > >> > > flexibility. I don't think they have node affinity as a
>> built
>> > in
>> > >>> > >> option
>> > >>> > >> > > (though I could be wrong). So if we want that we can either
>> > wait
>> > >>> for
>> > >>> > >> them
>> > >>> > >> > > to add it or do a custom framework to add that feature (as
>> > now).
>> > >>> > >> > Obviously
>> > >>> > >> > > if you manage things with old-school ops tools
>> > (puppet/chef/etc)
>> > >>> you
>> > >>> > >> get
>> > >>> > >> > > locality easily. The nice thing, though, is that all the
>> samza
>> > >>> > >> "business
>> > >>> > >> > > logic" around partition management and fault tolerance is in
>> > Samza
>> > >>> > >> core
>> > >>> > >> > so
>> > >>> > >> > > it is shared across frameworks and the framework specific
>> bit
>> > is
>> > >>> > just
>> > >>> > >> > > whether it is smart enough to try to get the same host when
>> a
>> > job
>> > >>> is
>> > >>> > >> > > restarted.
>> > >>> > >> > >
>> > >>> > >> > > With respect to the Kafka-alignment, yeah I think the goal
>> > would
>> > >>> be
>> > >>> > >> (a)
>> > >>> > >> > > actually get better alignment in user experience, and (b)
>> > express
>> > >>> > >> this in
>> > >>> > >> > > the naming and project branding. Specifically:
>> > >>> > >> > > 1. Website/docs, it would be nice for the "transformation"
>> > api to
>> > >>> be
>> > >>> > >> > > discoverable in the main Kafka docs--i.e. be able to explain
>> > when
>> > >>> to
>> > >>> > >> use
>> > >>> > >> > > the consumer and when to use the stream processing
>> > functionality
>> > >>> and
>> > >>> > >> lead
>> > >>> > >> > > people into that experience.
>> > >>> > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever)
>> > that
>> > >>> has
>> > >>> > >> both
>> > >>> > >> > > Kafka and the stream processing part and they actually work
>> > >>> > together.
>> > >>> > >> > > 3. Unify the programming experience so the client and Samza
>> > api
>> > >>> > share
>> > >>> > >> > > config/monitoring/naming/packaging/etc.
>> > >>> > >> > >
>> > >>> > >> > > I think sub-projects keep separate committers and can have a
>> > >>> > separate
>> > >>> > >> > repo,
>> > >>> > >> > > but I'm actually not really sure (I can't find a definition
>> > of a
>> > >>> > >> > subproject
>> > >>> > >> > > in Apache).
>> > >>> > >> > >
>> > >>> > >> > > Basically at a high-level you want the experience to "feel"
>> > like a
>> > >>> > >> single
>> > >>> > >> > > system, not to relatively independent things that are kind
>> of
>> > >>> > >> awkwardly
>> > >>> > >> > > glued together.
>> > >>> > >> > >
>> > >>> > >> > > I think if we did that they having naming or branding like
>> > "kafka
>> > >>> > >> > > streaming" or "kafka streams" or something like that would
>> > >>> actually
>> > >>> > >> do a
>> > >>> > >> > > good job of conveying what it is. I do that this would help
>> > >>> adoption
>> > >>> > >> > quite
>> > >>> > >> > > a lot as it would correctly convey that using Kafka
>> Streaming
>> > with
>> > >>> > >> Kafka
>> > >>> > >> > is
>> > >>> > >> > > a fairly seamless experience and Kafka is pretty heavily
>> > adopted
>> > >>> at
>> > >>> > >> this
>> > >>> > >> > > point.
>> > >>> > >> > >
>> > >>> > >> > > Fwiw we actually considered this model originally when open
>> > >>> sourcing
>> > >>> > >> > Samza,
>> > >>> > >> > > however at that time Kafka was relatively unknown and we
>> > decided
>> > >>> not
>> > >>> > >> to
>> > >>> > >> > do
>> > >>> > >> > > it since we felt it would be limiting. From my point of view
>> > the
>> > >>> > three
>> > >>> > >> > > things have changed (1) Kafka is now really heavily used for
>> > >>> stream
>> > >>> > >> > > processing, (2) we learned that abstracting out the stream
>> > well is
>> > >>> > >> > > basically impossible, (3) we learned it is really hard to
>> > keep the
>> > >>> > two
>> > >>> > >> > > things feeling like a single product.
>> > >>> > >> > >
>> > >>> > >> > > -Jay
>> > >>> > >> > >
>> > >>> > >> > >
>> > >>> > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
>> > >>> > >> martin@kleppmann.com>
>> > >>> > >> > > wrote:
>> > >>> > >> > >
>> > >>> > >> > >> Hi all,
>> > >>> > >> > >>
>> > >>> > >> > >> Lots of good thoughts here.
>> > >>> > >> > >>
>> > >>> > >> > >> I agree with the general philosophy of tying Samza more
>> > firmly to
>> > >>> > >> Kafka.
>> > >>> > >> > >> After I spent a while looking at integrating other message
>> > >>> brokers
>> > >>> > >> (e.g.
>> > >>> > >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
>> > >>> > >> > SystemConsumer
>> > >>> > >> > >> tacitly assumes a model so much like Kafka's that pretty
>> much
>> > >>> > nobody
>> > >>> > >> but
>> > >>> > >> > >> Kafka actually implements it. (Databus is perhaps an
>> > exception,
>> > >>> but
>> > >>> > >> it
>> > >>> > >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza
>> > fully
>> > >>> > >> > dependent
>> > >>> > >> > >> on Kafka acknowledges that the system-independence was
>> never
>> > as
>> > >>> > real
>> > >>> > >> as
>> > >>> > >> > we
>> > >>> > >> > >> perhaps made it out to be. The gains of code reuse are
>> real.
>> > >>> > >> > >>
>> > >>> > >> > >> The idea of decoupling Samza from YARN has also always been
>> > >>> > >> appealing to
>> > >>> > >> > >> me, for various reasons already mentioned in this thread.
>> > >>> Although
>> > >>> > >> > making
>> > >>> > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc)
>> seems
>> > >>> > >> laudable,
>> > >>> > >> > I am
>> > >>> > >> > >> a little concerned that it will restrict us to a lowest
>> > common
>> > >>> > >> > denominator.
>> > >>> > >> > >> For example, would host affinity (SAMZA-617) still be
>> > possible?
>> > >>> For
>> > >>> > >> jobs
>> > >>> > >> > >> with large amounts of state, I think SAMZA-617 would be a
>> big
>> > >>> boon,
>> > >>> > >> > since
>> > >>> > >> > >> restoring state off the changelog on every single restart
>> is
>> > >>> > painful,
>> > >>> > >> > due
>> > >>> > >> > >> to long recovery times. It would be a shame if the
>> decoupling
>> > >>> from
>> > >>> > >> YARN
>> > >>> > >> > >> made host affinity impossible.
>> > >>> > >> > >>
>> > >>> > >> > >> Jay, a question about the proposed API for instantiating a
>> > job in
>> > >>> > >> code
>> > >>> > >> > >> (rather than a properties file): when submitting a job to a
>> > >>> > cluster,
>> > >>> > >> is
>> > >>> > >> > the
>> > >>> > >> > >> idea that the instantiation code runs on a client
>> somewhere,
>> > >>> which
>> > >>> > >> then
>> > >>> > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or
>> does
>> > that
>> > >>> > >> code
>> > >>> > >> > run
>> > >>> > >> > >> on each container that is part of the job (in which case,
>> how
>> > >>> does
>> > >>> > >> the
>> > >>> > >> > job
>> > >>> > >> > >> submission to the cluster work)?
>> > >>> > >> > >>
>> > >>> > >> > >> I agree with Garry that it doesn't feel right to make a 1.0
>> > >>> release
>> > >>> > >> > with a
>> > >>> > >> > >> plan for it to be immediately obsolete. So if this is going
>> > to
>> > >>> > >> happen, I
>> > >>> > >> > >> think it would be more honest to stick with 0.* version
>> > numbers
>> > >>> > until
>> > >>> > >> > the
>> > >>> > >> > >> library-ified Samza has been implemented, is stable and
>> > widely
>> > >>> > used.
>> > >>> > >> > >>
>> > >>> > >> > >> Should the new Samza be a subproject of Kafka? There is
>> > precedent
>> > >>> > for
>> > >>> > >> > >> tight coupling between different Apache projects (e.g.
>> > Curator
>> > >>> and
>> > >>> > >> > >> Zookeeper, or Slider and YARN), so I think remaining
>> separate
>> > >>> would
>> > >>> > >> be
>> > >>> > >> > ok.
>> > >>> > >> > >> Even if Samza is fully dependent on Kafka, there is enough
>> > >>> > substance
>> > >>> > >> in
>> > >>> > >> > >> Samza that it warrants being a separate project. An
>> argument
>> > in
>> > >>> > >> favour
>> > >>> > >> > of
>> > >>> > >> > >> merging would be if we think Kafka has a much stronger
>> "brand
>> > >>> > >> presence"
>> > >>> > >> > >> than Samza; I'm ambivalent on that one. If the Kafka
>> project
>> > is
>> > >>> > >> willing
>> > >>> > >> > to
>> > >>> > >> > >> endorse Samza as the "official" way of doing stateful
>> stream
>> > >>> > >> > >> transformations, that would probably have much the same
>> > effect as
>> > >>> > >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike.
>> > Close
>> > >>> > >> > >> collaboration between the two projects will be needed in
>> any
>> > >>> case.
>> > >>> > >> > >>
>> > >>> > >> > >> From a project management perspective, I guess the "new
>> > Samza"
>> > >>> > would
>> > >>> > >> > have
>> > >>> > >> > >> to be developed on a branch alongside ongoing maintenance
>> of
>> > the
>> > >>> > >> current
>> > >>> > >> > >> line of development? I think it would be important to
>> > continue
>> > >>> > >> > supporting
>> > >>> > >> > >> existing users, and provide a graceful migration path to
>> the
>> > new
>> > >>> > >> > version.
>> > >>> > >> > >> Leaving the current versions unsupported and forcing people
>> > to
>> > >>> > >> rewrite
>> > >>> > >> > >> their jobs would send a bad signal.
>> > >>> > >> > >>
>> > >>> > >> > >> Best,
>> > >>> > >> > >> Martin
>> > >>> > >> > >>
>> > >>> > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io>
>> wrote:
>> > >>> > >> > >>
>> > >>> > >> > >>> Hey Garry,
>> > >>> > >> > >>>
>> > >>> > >> > >>> Yeah that's super frustrating. I'd be happy to chat more
>> > about
>> > >>> > this
>> > >>> > >> if
>> > >>> > >> > >>> you'd be interested. I think Chris and I started with the
>> > idea
>> > >>> of
>> > >>> > >> "what
>> > >>> > >> > >>> would it take to make Samza a kick-ass ingestion tool" but
>> > >>> > >> ultimately
>> > >>> > >> > we
>> > >>> > >> > >>> kind of came around to the idea that ingestion and
>> > >>> transformation
>> > >>> > >> had
>> > >>> > >> > >>> pretty different needs and coupling the two made things
>> > hard.
>> > >>> > >> > >>>
>> > >>> > >> > >>> For what it's worth I think copycat (KIP-26) actually will
>> > do
>> > >>> what
>> > >>> > >> you
>> > >>> > >> > >> are
>> > >>> > >> > >>> looking for.
>> > >>> > >> > >>>
>> > >>> > >> > >>> With regard to your point about slider, I don't
>> necessarily
>> > >>> > >> disagree.
>> > >>> > >> > >> But I
>> > >>> > >> > >>> think getting good YARN support is quite doable and I
>> think
>> > we
>> > >>> can
>> > >>> > >> make
>> > >>> > >> > >>> that work well. I think the issue this proposal solves is
>> > that
>> > >>> > >> > >> technically
>> > >>> > >> > >>> it is pretty hard to support multiple cluster management
>> > systems
>> > >>> > the
>> > >>> > >> > way
>> > >>> > >> > >>> things are now, you need to write an "app master" or
>> > "framework"
>> > >>> > for
>> > >>> > >> > each
>> > >>> > >> > >>> and they are all a little different so testing is really
>> > hard.
>> > >>> In
>> > >>> > >> the
>> > >>> > >> > >>> absence of this we have been stuck with just YARN which
>> has
>> > >>> > >> fantastic
>> > >>> > >> > >>> penetration in the Hadoopy part of the org, but zero
>> > penetration
>> > >>> > >> > >> elsewhere.
>> > >>> > >> > >>> Given the huge amount of work being put in to slider,
>> > marathon,
>> > >>> > aws
>> > >>> > >> > >>> tooling, not to mention the umpteen related packaging
>> > >>> technologies
>> > >>> > >> > people
>> > >>> > >> > >>> want to use (Docker, Kubernetes, various cloud-specific
>> > deploy
>> > >>> > >> tools,
>> > >>> > >> > >> etc)
>> > >>> > >> > >>> I really think it is important to get this right.
>> > >>> > >> > >>>
>> > >>> > >> > >>> -Jay
>> > >>> > >> > >>>
>> > >>> > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>> > >>> > >> > >>> g.turkington@improvedigital.com> wrote:
>> > >>> > >> > >>>
>> > >>> > >> > >>>> Hi all,
>> > >>> > >> > >>>>
>> > >>> > >> > >>>> I think the question below re does Samza become a
>> > sub-project
>> > >>> of
>> > >>> > >> Kafka
>> > >>> > >> > >>>> highlights the broader point around migration. Chris
>> > mentions
>> > >>> > >> Samza's
>> > >>> > >> > >>>> maturity is heading towards a v1 release but I'm not sure
>> > it
>> > >>> > feels
>> > >>> > >> > >> right to
>> > >>> > >> > >>>> launch a v1 then immediately plan to deprecate most of
>> it.
>> > >>> > >> > >>>>
>> > >>> > >> > >>>> From a selfish perspective I have some guys who have
>> > started
>> > >>> > >> working
>> > >>> > >> > >> with
>> > >>> > >> > >>>> Samza and building some new consumers/producers was next
>> > up.
>> > >>> > Sounds
>> > >>> > >> > like
>> > >>> > >> > >>>> that is absolutely not the direction to go. I need to
>> look
>> > into
>> > >>> > the
>> > >>> > >> > KIP
>> > >>> > >> > >> in
>> > >>> > >> > >>>> more detail but for me the attractiveness of adding new
>> > Samza
>> > >>> > >> > >>>> consumer/producers -- even if yes all they were doing was
>> > >>> really
>> > >>> > >> > getting
>> > >>> > >> > >>>> data into and out of Kafka --  was to avoid  having to
>> > worry
>> > >>> > about
>> > >>> > >> the
>> > >>> > >> > >>>> lifecycle management of external clients. If there is a
>> > generic
>> > >>> > >> Kafka
>> > >>> > >> > >>>> ingress/egress layer that I can plug a new connector into
>> > and
>> > >>> > have
>> > >>> > >> a
>> > >>> > >> > >> lot of
>> > >>> > >> > >>>> the heavy lifting re scale and reliability done for me
>> > then it
>> > >>> > >> gives
>> > >>> > >> > me
>> > >>> > >> > >> all
>> > >>> > >> > >>>> the pushing new consumers/producers would. If not then it
>> > >>> > >> complicates
>> > >>> > >> > my
>> > >>> > >> > >>>> operational deployments.
>> > >>> > >> > >>>>
>> > >>> > >> > >>>> Which is similar to my other question with the proposal
>> --
>> > if
>> > >>> we
>> > >>> > >> > build a
>> > >>> > >> > >>>> fully available/stand-alone Samza plus the requisite
>> shims
>> > to
>> > >>> > >> > integrate
>> > >>> > >> > >>>> with Slider etc I suspect the former may be a lot more
>> work
>> > >>> than
>> > >>> > we
>> > >>> > >> > >> think.
>> > >>> > >> > >>>> We may make it much easier for a newcomer to get
>> something
>> > >>> > running
>> > >>> > >> but
>> > >>> > >> > >>>> having them step up and get a reliable production
>> > deployment
>> > >>> may
>> > >>> > >> still
>> > >>> > >> > >>>> dominate mailing list  traffic, if for different reasons
>> > than
>> > >>> > >> today.
>> > >>> > >> > >>>>
>> > >>> > >> > >>>> Don't get me wrong -- I'm comfortable with making the
>> Samza
>> > >>> > >> dependency
>> > >>> > >> > >> on
>> > >>> > >> > >>>> Kafka much more explicit and I absolutely see the
>> > benefits  in
>> > >>> > the
>> > >>> > >> > >>>> reduction of duplication and clashing
>> > >>> terminologies/abstractions
>> > >>> > >> that
>> > >>> > >> > >>>> Chris/Jay describe. Samza as a library would likely be a
>> > very
>> > >>> > nice
>> > >>> > >> > tool
>> > >>> > >> > >> to
>> > >>> > >> > >>>> add to the Kafka ecosystem. I just have the concerns
>> above
>> > re
>> > >>> the
>> > >>> > >> > >>>> operational side.
>> > >>> > >> > >>>>
>> > >>> > >> > >>>> Garry
>> > >>> > >> > >>>>
>> > >>> > >> > >>>> -----Original Message-----
>> > >>> > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
>> > gdfm@apache.org]
>> > >>> > >> > >>>> Sent: 02 July 2015 12:56
>> > >>> > >> > >>>> To: dev@samza.apache.org
>> > >>> > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
>> > >>> > >> > >>>>
>> > >>> > >> > >>>> Very interesting thoughts.
>> > >>> > >> > >>>> From outside, I have always perceived Samza as a
>> computing
>> > >>> layer
>> > >>> > >> over
>> > >>> > >> > >>>> Kafka.
>> > >>> > >> > >>>>
>> > >>> > >> > >>>> The question, maybe a bit provocative, is "should Samza
>> be
>> > a
>> > >>> > >> > sub-project
>> > >>> > >> > >>>> of Kafka then?"
>> > >>> > >> > >>>> Or does it make sense to keep it as a separate project
>> > with a
>> > >>> > >> separate
>> > >>> > >> > >>>> governance?
>> > >>> > >> > >>>>
>> > >>> > >> > >>>> Cheers,
>> > >>> > >> > >>>>
>> > >>> > >> > >>>> --
>> > >>> > >> > >>>> Gianmarco
>> > >>> > >> > >>>>
>> > >>> > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
>> > >>> wrote:
>> > >>> > >> > >>>>
>> > >>> > >> > >>>>> Overall, I agree to couple with Kafka more tightly.
>> > Because
>> > >>> > Samza
>> > >>> > >> de
>> > >>> > >> > >>>>> facto is based on Kafka, and it should leverage what
>> Kafka
>> > >>> has.
>> > >>> > At
>> > >>> > >> > the
>> > >>> > >> > >>>>> same time, Kafka does not need to reinvent what Samza
>> > already
>> > >>> > >> has. I
>> > >>> > >> > >>>>> also like the idea of separating the ingestion and
>> > >>> > transformation.
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>> But it is a little difficult for me to image how the
>> Samza
>> > >>> will
>> > >>> > >> look
>> > >>> > >> > >>>> like.
>> > >>> > >> > >>>>> And I feel Chris and Jay have a little difference in
>> > terms of
>> > >>> > how
>> > >>> > >> > >>>>> Samza should look like.
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>> *** Will it look like what Jay's code shows (A client of
>> > >>> Kakfa)
>> > >>> > ?
>> > >>> > >> And
>> > >>> > >> > >>>>> user's application code calls this client?
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>> 1. If we make Samza be a library of Kafka (like what the
>> > code
>> > >>> > >> shows),
>> > >>> > >> > >>>>> how do we implement auto-balance and fault-tolerance?
>> Are
>> > they
>> > >>> > >> taken
>> > >>> > >> > >>>>> care by the Kafka broker or other mechanism, such as
>> > "Samza
>> > >>> > >> worker"
>> > >>> > >> > >>>>> (just make up the name) ?
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>> 2. What about other features, such as auto-scaling,
>> shared
>> > >>> > state,
>> > >>> > >> > >>>>> monitoring?
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>> *** If we have Samza standalone, (is this what Chris
>> > >>> suggests?)
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>> 1. we still need to ingest data from Kakfa and produce
>> to
>> > it.
>> > >>> > >> Then it
>> > >>> > >> > >>>>> becomes the same as what Samza looks like now, except it
>> > does
>> > >>> > not
>> > >>> > >> > rely
>> > >>> > >> > >>>>> on Yarn anymore.
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's
>> > metrics,
>> > >>> > logs,
>> > >>> > >> > >>>>> etc? Use Kafka code as the dependency?
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>> Thanks,
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>> Fang, Yan
>> > >>> > >> > >>>>> yanfang724@gmail.com
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
>> > >>> > wangguoz@gmail.com
>> > >>> > >> >
>> > >>> > >> > >>>> wrote:
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>>> Read through the code example and it looks good to me.
>> A
>> > few
>> > >>> > >> > >>>>>> thoughts regarding deployment:
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>> Today Samza deploys as executable runnable like:
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
>> > >>> > >> > >>>> --config-path=file://...
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>> And this proposal advocate for deploying Samza more as
>> > >>> embedded
>> > >>> > >> > >>>>>> libraries in user application code (ignoring the
>> > terminology
>> > >>> > >> since
>> > >>> > >> > >>>>>> it is not the
>> > >>> > >> > >>>>> same
>> > >>> > >> > >>>>>> as the prototype code):
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread
>> > thread =
>> > >>> > new
>> > >>> > >> > >>>>>> Thread(task); thread.start();
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>> I think both of these deployment modes are important
>> for
>> > >>> > >> different
>> > >>> > >> > >>>>>> types
>> > >>> > >> > >>>>> of
>> > >>> > >> > >>>>>> users. That said, I think making Samza purely
>> standalone
>> > is
>> > >>> > still
>> > >>> > >> > >>>>>> sufficient for either runnable or library modes.
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>> Guozhang
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
>> > >>> jay@confluent.io>
>> > >>> > >> > wrote:
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>>> Looks like gmail mangled the code example, it was
>> > supposed
>> > >>> to
>> > >>> > >> look
>> > >>> > >> > >>>>>>> like
>> > >>> > >> > >>>>>>> this:
>> > >>> > >> > >>>>>>>
>> > >>> > >> > >>>>>>> Properties props = new Properties();
>> > >>> > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
>> > >>> > >> StreamingConfig
>> > >>> > >> > >>>>>>> config = new StreamingConfig(props);
>> > >>> > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
>> > >>> > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
>> > >>> > >> > >>>>>>> config.serialization(new StringSerializer(), new
>> > >>> > >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
>> > >>> > >> > >>>>>>> KafkaStreaming(config); container.run();
>> > >>> > >> > >>>>>>>
>> > >>> > >> > >>>>>>> -Jay
>> > >>> > >> > >>>>>>>
>> > >>> > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
>> > >>> jay@confluent.io
>> > >>> > >
>> > >>> > >> > >>>> wrote:
>> > >>> > >> > >>>>>>>
>> > >>> > >> > >>>>>>>> Hey guys,
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> This came out of some conversations Chris and I were
>> > having
>> > >>> > >> > >>>>>>>> around
>> > >>> > >> > >>>>>>> whether
>> > >>> > >> > >>>>>>>> it would make sense to use Samza as a kind of data
>> > >>> ingestion
>> > >>> > >> > >>>>> framework
>> > >>> > >> > >>>>>>> for
>> > >>> > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat").
>> This
>> > >>> kind
>> > >>> > of
>> > >>> > >> > >>>>>> combined
>> > >>> > >> > >>>>>>>> with complaints around config and YARN and the
>> > discussion
>> > >>> > >> around
>> > >>> > >> > >>>>>>>> how
>> > >>> > >> > >>>>> to
>> > >>> > >> > >>>>>>>> best do a standalone mode.
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> So the thought experiment was, given that Samza was
>> > >>> basically
>> > >>> > >> > >>>>>>>> already totally Kafka specific, what if you just
>> > embraced
>> > >>> > that
>> > >>> > >> > >>>>>>>> and turned it
>> > >>> > >> > >>>>>> into
>> > >>> > >> > >>>>>>>> something less like a heavyweight framework and more
>> > like a
>> > >>> > >> > >>>>>>>> third
>> > >>> > >> > >>>>> Kafka
>> > >>> > >> > >>>>>>>> client--a kind of "producing consumer" with state
>> > >>> management
>> > >>> > >> > >>>>>> facilities.
>> > >>> > >> > >>>>>>>> Basically a library. Instead of a complex stream
>> > processing
>> > >>> > >> > >>>>>>>> framework
>> > >>> > >> > >>>>>>> this
>> > >>> > >> > >>>>>>>> would actually be a very simple thing, not much more
>> > >>> > >> complicated
>> > >>> > >> > >>>>>>>> to
>> > >>> > >> > >>>>> use
>> > >>> > >> > >>>>>>> or
>> > >>> > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we
>> thought
>> > >>> about
>> > >>> > >> it
>> > >>> > >> > >>>>>>>> a
>> > >>> > >> > >>>>> lot
>> > >>> > >> > >>>>>> of
>> > >>> > >> > >>>>>>>> what Samza (and the other stream processing systems
>> > were
>> > >>> > doing)
>> > >>> > >> > >>>>> seemed
>> > >>> > >> > >>>>>>> like
>> > >>> > >> > >>>>>>>> kind of a hangover from MapReduce.
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> Of course you need to ingest/output data to and from
>> > the
>> > >>> > stream
>> > >>> > >> > >>>>>>>> processing. But when we actually looked into how that
>> > would
>> > >>> > >> > >>>>>>>> work,
>> > >>> > >> > >>>>> Samza
>> > >>> > >> > >>>>>>>> isn't really an ideal data ingestion framework for a
>> > bunch
>> > >>> of
>> > >>> > >> > >>>>> reasons.
>> > >>> > >> > >>>>>> To
>> > >>> > >> > >>>>>>>> really do that right you need a pretty different
>> > internal
>> > >>> > data
>> > >>> > >> > >>>>>>>> model
>> > >>> > >> > >>>>>> and
>> > >>> > >> > >>>>>>>> set of apis. So what if you split them and had an api
>> > for
>> > >>> > Kafka
>> > >>> > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate
>> api
>> > for
>> > >>> > >> Kafka
>> > >>> > >> > >>>>>>>> transformation (Samza).
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> This would also allow really embracing the same
>> > terminology
>> > >>> > and
>> > >>> > >> > >>>>>>>> conventions. One complaint about the current state is
>> > that
>> > >>> > the
>> > >>> > >> > >>>>>>>> two
>> > >>> > >> > >>>>>>> systems
>> > >>> > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs
>> > >>> "topic"
>> > >>> > >> and
>> > >>> > >> > >>>>>>> different
>> > >>> > >> > >>>>>>>> config and monitoring systems means you kind of have
>> to
>> > >>> learn
>> > >>> > >> > >>>>>>>> Kafka's
>> > >>> > >> > >>>>>>> way,
>> > >>> > >> > >>>>>>>> then learn Samza's slightly different way, then kind
>> of
>> > >>> > >> > >>>>>>>> understand
>> > >>> > >> > >>>>> how
>> > >>> > >> > >>>>>>> they
>> > >>> > >> > >>>>>>>> map to each other, which having walked a few people
>> > through
>> > >>> > >> this
>> > >>> > >> > >>>>>>>> is surprisingly tricky for folks to get.
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> Since I have been spending a lot of time on
>> airplanes I
>> > >>> > hacked
>> > >>> > >> > >>>>>>>> up an ernest but still somewhat incomplete prototype
>> of
>> > >>> what
>> > >>> > >> > >>>>>>>> this would
>> > >>> > >> > >>>>> look
>> > >>> > >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka
>> > as it
>> > >>> > >> > >>>>>>>> required a
>> > >>> > >> > >>>>>> few
>> > >>> > >> > >>>>>>>> changes to the new consumer. Here is the code:
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>
>> > >>> > >> >
>> > >>> >
>> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> > >>> > >> > >>>>> /apache/kafka/clients/streaming
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> For the purpose of the prototype I just liberally
>> > renamed
>> > >>> > >> > >>>>>>>> everything
>> > >>> > >> > >>>>> to
>> > >>> > >> > >>>>>>>> try to align it with Kafka with no regard for
>> > >>> compatibility.
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> To use this would be something like this:
>> > >>> > >> > >>>>>>>> Properties props = new Properties();
>> > >>> > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
>> > >>> > >> > >>>>>>>> StreamingConfig config = new
>> > >>> > >> > >>>>> StreamingConfig(props);
>> > >>> > >> > >>>>>>> config.subscribe("test-topic-1",
>> > >>> > >> > >>>>>>>> "test-topic-2");
>> > >>> > >> config.processor(ExampleStreamProcessor.class);
>> > >>> > >> > >>>>>>> config.serialization(new
>> > >>> > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
>> > >>> KafkaStreaming
>> > >>> > >> > >>>>>> container =
>> > >>> > >> > >>>>>>>> new KafkaStreaming(config); container.run();
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
>> > >>> > StreamProcessor
>> > >>> > >> > >>>>>>>> is basically StreamTask.
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> So rather than putting all the class names in a file
>> > and
>> > >>> then
>> > >>> > >> > >>>>>>>> having
>> > >>> > >> > >>>>>> the
>> > >>> > >> > >>>>>>>> job assembled by reflection, you just instantiate the
>> > >>> > container
>> > >>> > >> > >>>>>>>> programmatically. Work is balanced over however many
>> > >>> > instances
>> > >>> > >> > >>>>>>>> of
>> > >>> > >> > >>>>> this
>> > >>> > >> > >>>>>>> are
>> > >>> > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new
>> tasks
>> > are
>> > >>> > >> added
>> > >>> > >> > >>>>>>>> to
>> > >>> > >> > >>>>> the
>> > >>> > >> > >>>>>>>> existing containers without shutting them down).
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> We would provide some glue for running this stuff in
>> > YARN
>> > >>> via
>> > >>> > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of
>> their
>> > >>> tools
>> > >>> > >> > >>>>>>>> but from the
>> > >>> > >> > >>>>>> point
>> > >>> > >> > >>>>>>> of
>> > >>> > >> > >>>>>>>> view of these frameworks these stream processing jobs
>> > are
>> > >>> > just
>> > >>> > >> > >>>>>> stateless
>> > >>> > >> > >>>>>>>> services that can come and go and expand and contract
>> > at
>> > >>> > will.
>> > >>> > >> > >>>>>>>> There
>> > >>> > >> > >>>>> is
>> > >>> > >> > >>>>>>> no
>> > >>> > >> > >>>>>>>> more custom scheduler.
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> Here are some relevant details:
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get
>> > larger if
>> > >>> we
>> > >>> > >> > >>>>>>>>  productionized but not vastly larger. We really do
>> > get a
>> > >>> ton
>> > >>> > >> > >>>>>>>> of
>> > >>> > >> > >>>>>>> leverage
>> > >>> > >> > >>>>>>>>  out of Kafka.
>> > >>> > >> > >>>>>>>>  2. Partition management is fully delegated to the
>> new
>> > >>> > >> consumer.
>> > >>> > >> > >>>>> This
>> > >>> > >> > >>>>>>>>  is nice since now any partition management strategy
>> > >>> > available
>> > >>> > >> > >>>>>>>> to
>> > >>> > >> > >>>>>> Kafka
>> > >>> > >> > >>>>>>>>  consumer is also available to Samza (and vice versa)
>> > and
>> > >>> > with
>> > >>> > >> > >>>>>>>> the
>> > >>> > >> > >>>>>>> exact
>> > >>> > >> > >>>>>>>>  same configs.
>> > >>> > >> > >>>>>>>>  3. It supports state as well as state reuse
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> Anyhow take a look, hopefully it is thought
>> provoking.
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> -Jay
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>> > >>> > >> > >>>>>> criccomini@apache.org>
>> > >>> > >> > >>>>>>>> wrote:
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>>> Hey all,
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> I have had some discussions with Samza engineers at
>> > >>> LinkedIn
>> > >>> > >> > >>>>>>>>> and
>> > >>> > >> > >>>>>>> Confluent
>> > >>> > >> > >>>>>>>>> and we came up with a few observations and would
>> like
>> > to
>> > >>> > >> > >>>>>>>>> propose
>> > >>> > >> > >>>>> some
>> > >>> > >> > >>>>>>>>> changes.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> We've observed some things that I want to call out
>> > about
>> > >>> > >> > >>>>>>>>> Samza's
>> > >>> > >> > >>>>>> design,
>> > >>> > >> > >>>>>>>>> and I'd like to propose some changes.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment
>> system.
>> > >>> > >> > >>>>>>>>> * Samza is too pluggable.
>> > >>> > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
>> > >>> consumer
>> > >>> > >> > >>>>>>>>> APIs
>> > >>> > >> > >>>>> are
>> > >>> > >> > >>>>>>>>> trying to solve a lot of the same problems.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> All three of these issues are related, but I'll
>> > address
>> > >>> them
>> > >>> > >> in
>> > >>> > >> > >>>>> order.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Deployment
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic
>> > deployment
>> > >>> > >> > >>>>>>>>> scheduler
>> > >>> > >> > >>>>>> such
>> > >>> > >> > >>>>>>>>> as
>> > >>> > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we
>> > bet
>> > >>> that
>> > >>> > >> > >>>>>>>>> there
>> > >>> > >> > >>>>>> would
>> > >>> > >> > >>>>>>>>> be
>> > >>> > >> > >>>>>>>>> one or two winners in this area, and we could
>> support
>> > >>> them,
>> > >>> > >> and
>> > >>> > >> > >>>>>>>>> the
>> > >>> > >> > >>>>>> rest
>> > >>> > >> > >>>>>>>>> would go away. In reality, there are many
>> variations.
>> > >>> > >> > >>>>>>>>> Furthermore,
>> > >>> > >> > >>>>>> many
>> > >>> > >> > >>>>>>>>> people still prefer to just start their processors
>> > like
>> > >>> > normal
>> > >>> > >> > >>>>>>>>> Java processes, and use traditional deployment
>> scripts
>> > >>> such
>> > >>> > as
>> > >>> > >> > >>>>>>>>> Fabric,
>> > >>> > >> > >>>>>> Chef,
>> > >>> > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users
>> > makes
>> > >>> the
>> > >>> > >> > >>>>>>>>> Samza start-up process really painful for first time
>> > >>> users.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit
>> of
>> > a
>> > >>> > >> > >>>>>>>>> mis-fire
>> > >>> > >> > >>>>>> because
>> > >>> > >> > >>>>>>>>> of
>> > >>> > >> > >>>>>>>>> a fundamental misunderstanding between the nature of
>> > batch
>> > >>> > >> jobs
>> > >>> > >> > >>>>>>>>> and
>> > >>> > >> > >>>>>>> stream
>> > >>> > >> > >>>>>>>>> processing jobs. Early on, we made conscious effort
>> to
>> > >>> favor
>> > >>> > >> > >>>>>>>>> the
>> > >>> > >> > >>>>>> Hadoop
>> > >>> > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked
>> and
>> > was
>> > >>> > well
>> > >>> > >> > >>>>>>> understood.
>> > >>> > >> > >>>>>>>>> One thing that we missed was that batch jobs have a
>> > >>> definite
>> > >>> > >> > >>>>>> beginning,
>> > >>> > >> > >>>>>>>>> and
>> > >>> > >> > >>>>>>>>> end, and stream processing jobs don't (usually).
>> This
>> > >>> leads
>> > >>> > to
>> > >>> > >> > >>>>>>>>> a
>> > >>> > >> > >>>>> much
>> > >>> > >> > >>>>>>>>> simpler scheduling problem for stream processors.
>> You
>> > >>> > >> basically
>> > >>> > >> > >>>>>>>>> just
>> > >>> > >> > >>>>>>> need
>> > >>> > >> > >>>>>>>>> to find a place to start the processor, and start
>> it.
>> > The
>> > >>> > way
>> > >>> > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a
>> > cluster
>> > >>> > >> > >>>>>>>>> being "full". We always
>> > >>> > >> > >>>>>> add
>> > >>> > >> > >>>>>>>>> more machines. The problem with coupling Samza with
>> a
>> > >>> > >> scheduler
>> > >>> > >> > >>>>>>>>> is
>> > >>> > >> > >>>>>> that
>> > >>> > >> > >>>>>>>>> Samza (as a framework) now has to handle deployment.
>> > This
>> > >>> > >> pulls
>> > >>> > >> > >>>>>>>>> in a
>> > >>> > >> > >>>>>>> bunch
>> > >>> > >> > >>>>>>>>> of things such as configuration distribution (config
>> > >>> > stream),
>> > >>> > >> > >>>>>>>>> shell
>> > >>> > >> > >>>>>>> scrips
>> > >>> > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
>> > >>> stuff),
>> > >>> > >> etc.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Another reason for requiring dynamic deployment was
>> to
>> > >>> > support
>> > >>> > >> > >>>>>>>>> data locality. If you want to have locality, you
>> need
>> > to
>> > >>> put
>> > >>> > >> > >>>>>>>>> your
>> > >>> > >> > >>>>>> processors
>> > >>> > >> > >>>>>>>>> close to the data they're processing. Upon further
>> > >>> > >> > >>>>>>>>> investigation,
>> > >>> > >> > >>>>>>> though,
>> > >>> > >> > >>>>>>>>> this feature is not that beneficial. There is some
>> > good
>> > >>> > >> > >>>>>>>>> discussion
>> > >>> > >> > >>>>>> about
>> > >>> > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took
>> the
>> > >>> > >> > >>>>>>>>> Map/Reduce
>> > >>> > >> > >>>>>> path,
>> > >>> > >> > >>>>>>>>> but
>> > >>> > >> > >>>>>>>>> there are some fundamental differences between HDFS
>> > and
>> > >>> > Kafka.
>> > >>> > >> > >>>>>>>>> HDFS
>> > >>> > >> > >>>>>> has
>> > >>> > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to
>> less
>> > >>> > >> > >>>>>>>>> optimization potential with stream processors on top
>> > of
>> > >>> > Kafka.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't
>> > have
>> > >>> > any
>> > >>> > >> > >>>>>>>>> built
>> > >>> > >> > >>>>> in
>> > >>> > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the
>> > dynamic
>> > >>> > >> > >>>>>>>>> deployment scheduling system to handle restarts
>> when a
>> > >>> > >> > >>>>>>>>> processor dies. This has
>> > >>> > >> > >>>>>>> made
>> > >>> > >> > >>>>>>>>> it very difficult to write a standalone Samza
>> > container
>> > >>> > >> > >>>> (SAMZA-516).
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Pluggability
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> In some cases pluggability is good, but I think that
>> > we've
>> > >>> > >> gone
>> > >>> > >> > >>>>>>>>> too
>> > >>> > >> > >>>>>> far
>> > >>> > >> > >>>>>>>>> with it. Currently, Samza has:
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> * Pluggable config.
>> > >>> > >> > >>>>>>>>> * Pluggable metrics.
>> > >>> > >> > >>>>>>>>> * Pluggable deployment systems.
>> > >>> > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
>> > >>> > SystemProducer,
>> > >>> > >> > >>>> etc).
>> > >>> > >> > >>>>>>>>> * Pluggable serdes.
>> > >>> > >> > >>>>>>>>> * Pluggable storage engines.
>> > >>> > >> > >>>>>>>>> * Pluggable strategies for just about every
>> component
>> > >>> > >> > >>>>> (MessageChooser,
>> > >>> > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> There's probably more that I've forgotten, as well.
>> > Some
>> > >>> of
>> > >>> > >> > >>>>>>>>> these
>> > >>> > >> > >>>>> are
>> > >>> > >> > >>>>>>>>> useful, but some have proven not to be. This all
>> > comes at
>> > >>> a
>> > >>> > >> cost:
>> > >>> > >> > >>>>>>>>> complexity. This complexity is making it harder for
>> > our
>> > >>> > users
>> > >>> > >> > >>>>>>>>> to
>> > >>> > >> > >>>>> pick
>> > >>> > >> > >>>>>> up
>> > >>> > >> > >>>>>>>>> and use Samza out of the box. It also makes it
>> > difficult
>> > >>> for
>> > >>> > >> > >>>>>>>>> Samza developers to reason about what the
>> > characteristics
>> > >>> of
>> > >>> > >> > >>>>>>>>> the container (since the characteristics change
>> > depending
>> > >>> on
>> > >>> > >> > >>>>>>>>> which plugins are use).
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> The issues with pluggability are most visible in the
>> > >>> System
>> > >>> > >> APIs.
>> > >>> > >> > >>>>> What
>> > >>> > >> > >>>>>>>>> Samza really requires to be functional is Kafka as
>> its
>> > >>> > >> > >>>>>>>>> transport
>> > >>> > >> > >>>>>> layer.
>> > >>> > >> > >>>>>>>>> But
>> > >>> > >> > >>>>>>>>> we've conflated two unrelated use cases into one
>> API:
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> 1. Get data into/out of Kafka.
>> > >>> > >> > >>>>>>>>> 2. Process the data in Kafka.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> The current System API supports both of these use
>> > cases.
>> > >>> The
>> > >>> > >> > >>>>>>>>> problem
>> > >>> > >> > >>>>>> is,
>> > >>> > >> > >>>>>>>>> we
>> > >>> > >> > >>>>>>>>> actually want different features for each use case.
>> By
>> > >>> > >> papering
>> > >>> > >> > >>>>>>>>> over
>> > >>> > >> > >>>>>>> these
>> > >>> > >> > >>>>>>>>> two use cases, and providing a single API, we've
>> > >>> introduced
>> > >>> > a
>> > >>> > >> > >>>>>>>>> ton of
>> > >>> > >> > >>>>>>> leaky
>> > >>> > >> > >>>>>>>>> abstractions.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> For example, what we'd really like in (2) is to have
>> > >>> > >> > >>>>>>>>> monotonically increasing longs for offsets (like
>> > Kafka).
>> > >>> > This
>> > >>> > >> > >>>>>>>>> would be at odds
>> > >>> > >> > >>>>> with
>> > >>> > >> > >>>>>>> (1),
>> > >>> > >> > >>>>>>>>> though, since different systems have different
>> > >>> > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
>> > >>> > >> > >>>>>>>>> There was discussion both on the mailing list and
>> the
>> > SQL
>> > >>> > >> JIRAs
>> > >>> > >> > >>>>> about
>> > >>> > >> > >>>>>>> the
>> > >>> > >> > >>>>>>>>> need for this.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> The same thing holds true for replayability. Kafka
>> > allows
>> > >>> us
>> > >>> > >> to
>> > >>> > >> > >>>>> rewind
>> > >>> > >> > >>>>>>>>> when
>> > >>> > >> > >>>>>>>>> we have a failure. Many other systems don't. In some
>> > >>> cases,
>> > >>> > >> > >>>>>>>>> systems
>> > >>> > >> > >>>>>>> return
>> > >>> > >> > >>>>>>>>> null for their offsets (e.g.
>> WikipediaSystemConsumer)
>> > >>> > because
>> > >>> > >> > >>>>>>>>> they
>> > >>> > >> > >>>>>> have
>> > >>> > >> > >>>>>>> no
>> > >>> > >> > >>>>>>>>> offsets.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Partitioning is another example. Kafka supports
>> > >>> > partitioning,
>> > >>> > >> > >>>>>>>>> but
>> > >>> > >> > >>>>> many
>> > >>> > >> > >>>>>>>>> systems don't. We model this by having a single
>> > partition
>> > >>> > for
>> > >>> > >> > >>>>>>>>> those systems. Still, other systems model
>> partitioning
>> > >>> > >> > >>>> differently (e.g.
>> > >>> > >> > >>>>>>>>> Kinesis).
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating
>> > streams
>> > >>> > in
>> > >>> > >> a
>> > >>> > >> > >>>>>>>>> system-agnostic way is almost impossible. As is
>> > modeling
>> > >>> > >> > >>>>>>>>> metadata
>> > >>> > >> > >>>>> for
>> > >>> > >> > >>>>>>> the
>> > >>> > >> > >>>>>>>>> system (replication factor, partitions, location,
>> > etc).
>> > >>> The
>> > >>> > >> > >>>>>>>>> list
>> > >>> > >> > >>>>> goes
>> > >>> > >> > >>>>>>> on.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Duplicate work
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's
>> > consumer
>> > >>> > and
>> > >>> > >> > >>>>> producer
>> > >>> > >> > >>>>>>>>> APIs
>> > >>> > >> > >>>>>>>>> had a relatively weak feature set. On the
>> > consumer-side,
>> > >>> you
>> > >>> > >> > >>>>>>>>> had two
>> > >>> > >> > >>>>>>>>> options: use the high level consumer, or the simple
>> > >>> > consumer.
>> > >>> > >> > >>>>>>>>> The
>> > >>> > >> > >>>>>>> problem
>> > >>> > >> > >>>>>>>>> with the high-level consumer was that it controlled
>> > your
>> > >>> > >> > >>>>>>>>> offsets, partition assignments, and the order in
>> > which you
>> > >>> > >> > >>>>>>>>> received messages. The
>> > >>> > >> > >>>>> problem
>> > >>> > >> > >>>>>>>>> with
>> > >>> > >> > >>>>>>>>> the simple consumer is that it's not simple. It's
>> > basic.
>> > >>> You
>> > >>> > >> > >>>>>>>>> end up
>> > >>> > >> > >>>>>>> having
>> > >>> > >> > >>>>>>>>> to handle a lot of really low-level stuff that you
>> > >>> > shouldn't.
>> > >>> > >> > >>>>>>>>> We
>> > >>> > >> > >>>>>> spent a
>> > >>> > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
>> > >>> robust.
>> > >>> > >> It
>> > >>> > >> > >>>>>>>>> also allows us to support some cool features:
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> * Per-partition message ordering and prioritization.
>> > >>> > >> > >>>>>>>>> * Tight control over partition assignment to support
>> > >>> joins,
>> > >>> > >> > >>>>>>>>> global
>> > >>> > >> > >>>>>> state
>> > >>> > >> > >>>>>>>>> (if we want to implement it :)), etc.
>> > >>> > >> > >>>>>>>>> * Tight control over offset checkpointing.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> What we didn't realize at the time is that these
>> > features
>> > >>> > >> > >>>>>>>>> should
>> > >>> > >> > >>>>>>> actually
>> > >>> > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just
>> Samza
>> > >>> stream
>> > >>> > >> > >>>>>> processors)
>> > >>> > >> > >>>>>>>>> end up wanting to do things like joins and partition
>> > >>> > >> > >>>>>>>>> assignment. The
>> > >>> > >> > >>>>>>> Kafka
>> > >>> > >> > >>>>>>>>> community has come to the same conclusion. They're
>> > adding
>> > >>> a
>> > >>> > >> ton
>> > >>> > >> > >>>>>>>>> of upgrades into their new Kafka consumer
>> > implementation.
>> > >>> > To a
>> > >>> > >> > >>>>>>>>> large extent,
>> > >>> > >> > >>>>> it's
>> > >>> > >> > >>>>>>>>> duplicate work to what we've already done in Samza.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
>> > >>> > approach
>> > >>> > >> > >>>>>>>>> to
>> > >>> > >> > >>>>>> Samza's
>> > >>> > >> > >>>>>>>>> KafkaCheckpointManager implementation for handling
>> > offset
>> > >>> > >> > >>>>>> checkpointing.
>> > >>> > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature
>> > stores
>> > >>> > >> offset
>> > >>> > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them
>> > from
>> > >>> > the
>> > >>> > >> > >>>>>>>>> broker.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> A lot of this seems like a waste, since we could
>> have
>> > >>> shared
>> > >>> > >> > >>>>>>>>> the
>> > >>> > >> > >>>>> work
>> > >>> > >> > >>>>>> if
>> > >>> > >> > >>>>>>>>> it
>> > >>> > >> > >>>>>>>>> had been done in Kafka from the get-go.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Vision
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> All of this leads me to a rather radical proposal.
>> > Samza
>> > >>> is
>> > >>> > >> > >>>>> relatively
>> > >>> > >> > >>>>>>>>> stable at this point. I'd venture to say that we're
>> > near a
>> > >>> > 1.0
>> > >>> > >> > >>>>>> release.
>> > >>> > >> > >>>>>>>>> I'd
>> > >>> > >> > >>>>>>>>> like to propose that we take what we've learned, and
>> > begin
>> > >>> > >> > >>>>>>>>> thinking
>> > >>> > >> > >>>>>>> about
>> > >>> > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were
>> > starting
>> > >>> > >> from
>> > >>> > >> > >>>>>> scratch?
>> > >>> > >> > >>>>>>>>> My
>> > >>> > >> > >>>>>>>>> proposal is to:
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
>> > >>> > >> > >>>>>>>>> processors, and eliminate all direct dependences on
>> > YARN,
>> > >>> > >> Mesos,
>> > >>> > >> > >>>> etc.
>> > >>> > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as
>> the
>> > >>> > stream
>> > >>> > >> > >>>>>> processing
>> > >>> > >> > >>>>>>>>> layer.
>> > >>> > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
>> serialization,
>> > and
>> > >>> > >> > >>>>>>>>> config
>> > >>> > >> > >>>>>>> systems,
>> > >>> > >> > >>>>>>>>> and simply use Kafka's instead.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> This would fix all of the issues that I outlined
>> > above. It
>> > >>> > >> > >>>>>>>>> should
>> > >>> > >> > >>>>> also
>> > >>> > >> > >>>>>>>>> shrink the Samza code base pretty dramatically.
>> > Supporting
>> > >>> > >> only
>> > >>> > >> > >>>>>>>>> a standalone container will allow Samza to be
>> > executed on
>> > >>> > YARN
>> > >>> > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or
>> most
>> > >>> other
>> > >>> > >> > >>>>>>>>> in-house
>> > >>> > >> > >>>>>>> deployment
>> > >>> > >> > >>>>>>>>> systems. This should make life a lot easier for new
>> > users.
>> > >>> > >> > >>>>>>>>> Imagine
>> > >>> > >> > >>>>>>> having
>> > >>> > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in
>> > mailing
>> > >>> > >> list
>> > >>> > >> > >>>>>> traffic
>> > >>> > >> > >>>>>>>>> will be pretty dramatic.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The
>> > reality
>> > >>> > is,
>> > >>> > >> > >>>>> everyone
>> > >>> > >> > >>>>>>>>> that
>> > >>> > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically
>> > >>> require
>> > >>> > >> it
>> > >>> > >> > >>>>>> already
>> > >>> > >> > >>>>>>> in
>> > >>> > >> > >>>>>>>>> order for most features to work. Those that are
>> using
>> > >>> other
>> > >>> > >> > >>>>>>>>> systems
>> > >>> > >> > >>>>>> are
>> > >>> > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and
>> then
>> > >>> they
>> > >>> > do
>> > >>> > >> > >>>>>>>>> the processing on top. There is already discussion (
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>
>> > >>> > >> >
>> > >>> >
>> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> > >>> > >> > >>>>> 767
>> > >>> > >> > >>>>>>>>> )
>> > >>> > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely
>> easy.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Once we make the call to couple with Kafka, we can
>> > >>> leverage
>> > >>> > a
>> > >>> > >> > >>>>>>>>> ton of
>> > >>> > >> > >>>>>>> their
>> > >>> > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own
>> > config,
>> > >>> > >> > >>>>>>>>> metrics,
>> > >>> > >> > >>>>> etc.
>> > >>> > >> > >>>>>>> We
>> > >>> > >> > >>>>>>>>> can all share the same libraries, and make them
>> > better.
>> > >>> This
>> > >>> > >> > >>>>>>>>> will
>> > >>> > >> > >>>>> also
>> > >>> > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and
>> > will let
>> > >>> > us
>> > >>> > >> > >>>>> leverage
>> > >>> > >> > >>>>>>>>> their offset management and partition management,
>> > rather
>> > >>> > than
>> > >>> > >> > >>>>>>>>> having
>> > >>> > >> > >>>>>> our
>> > >>> > >> > >>>>>>>>> own. All of the coordinator stream code would go
>> > away, as
>> > >>> > >> would
>> > >>> > >> > >>>>>>>>> most
>> > >>> > >> > >>>>>> of
>> > >>> > >> > >>>>>>>>> the
>> > >>> > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
>> > >>> > partition
>> > >>> > >> > >>>>>>> management
>> > >>> > >> > >>>>>>>>> features into the Kafka broker, but they're already
>> > moving
>> > >>> > in
>> > >>> > >> > >>>>>>>>> that direction with the new consumer API. The
>> > features we
>> > >>> > have
>> > >>> > >> > >>>>>>>>> for
>> > >>> > >> > >>>>>> partition
>> > >>> > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like
>> they
>> > >>> should
>> > >>> > >> be
>> > >>> > >> > >>>>>>>>> in
>> > >>> > >> > >>>>>> Kafka
>> > >>> > >> > >>>>>>>>> anyway. There will always be some niche usages which
>> > will
>> > >>> > >> > >>>>>>>>> require
>> > >>> > >> > >>>>>> extra
>> > >>> > >> > >>>>>>>>> care and hence full control over partition
>> assignments
>> > >>> much
>> > >>> > >> > >>>>>>>>> like the
>> > >>> > >> > >>>>>>> Kafka
>> > >>> > >> > >>>>>>>>> low level consumer api. These would continue to be
>> > >>> > supported.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> These items will be good for the Samza community.
>> > They'll
>> > >>> > make
>> > >>> > >> > >>>>>>>>> Samza easier to use, and make it easier for
>> > developers to
>> > >>> > add
>> > >>> > >> > >>>>>>>>> new features.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat
>> > backwards
>> > >>> > >> > >>>>> incompatible
>> > >>> > >> > >>>>>>>>> change). If we choose to go this route, it's
>> important
>> > >>> that
>> > >>> > we
>> > >>> > >> > >>>>> openly
>> > >>> > >> > >>>>>>>>> communicate how we're going to provide a migration
>> > path
>> > >>> from
>> > >>> > >> > >>>>>>>>> the
>> > >>> > >> > >>>>>>> existing
>> > >>> > >> > >>>>>>>>> APIs to the new ones (if we make incompatible
>> > changes). I
>> > >>> > >> think
>> > >>> > >> > >>>>>>>>> at a minimum, we'd probably need to provide a
>> wrapper
>> > to
>> > >>> > allow
>> > >>> > >> > >>>>>>>>> existing StreamTask implementations to continue
>> > running on
>> > >>> > the
>> > >>> > >> > >>>> new container.
>> > >>> > >> > >>>>>>> It's
>> > >>> > >> > >>>>>>>>> also important that we openly communicate about
>> > timing,
>> > >>> and
>> > >>> > >> > >>>>>>>>> stages
>> > >>> > >> > >>>>> of
>> > >>> > >> > >>>>>>> the
>> > >>> > >> > >>>>>>>>> migration.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> If you made it this far, I'm sure you have opinions.
>> > :)
>> > >>> > Please
>> > >>> > >> > >>>>>>>>> send
>> > >>> > >> > >>>>>> your
>> > >>> > >> > >>>>>>>>> thoughts and feedback.
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>> Cheers,
>> > >>> > >> > >>>>>>>>> Chris
>> > >>> > >> > >>>>>>>>>
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>>
>> > >>> > >> > >>>>>>>
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>> --
>> > >>> > >> > >>>>>> -- Guozhang
>> > >>> > >> > >>>>>>
>> > >>> > >> > >>>>>
>> > >>> > >> > >>>>
>> > >>> > >> > >>
>> > >>> > >> > >>
>> > >>> > >> >
>> > >>> > >> >
>> > >>> > >> >
>> > >>> > >>
>> > >>> > >
>> > >>> > >
>> > >>> >
>> > >>>
>> >
>>
>
>
>
> --
> Jordan Shaw
> Full Stack Software Engineer
> PubNub Inc
> 1045 17th St
> San Francisco, CA 94107

Re: Thoughts and obesrvations on Samza

Posted by Jordan Shaw <jo...@pubnub.com>.
I'm all for any optimizations that can be made to the Yarn workflow.

I actually agree with Jakob in regard to the producers/consumers. I have
spent sometime writing consumers and producers for other transport
abstractions and overall the current api abstractions in Samza I feel are
pretty good. There are some things that are sort of anomalous and catered
more toward the Kafka model but easy enough to work around and I've been
able to make other Producers and Consumers work that are no where near the
same paradigm as Kafka.

To Jay's point although Kafka is great and does the streaming data paradigm
very well there is really no reason why a different transport application
implemented properly wouldn't be able to stream data with the same
effectiveness as Kafka and that transport may suite the user's use case
better or be more cost effective than Kafka. For example we had to decide
if Kafka was worth the extra cost of running a zookeeper cluster and if the
scaling through partitioning was worth the operational overhead vs having a
mesh network over ZeroMQ. After deciding that our use case would fit with
Kafka fine there were other challenges like understanding how AWS EC2 SSD's
behaved (AWS amortizes all disk into into Random I/O this is bad for Kafka).

Thus, I would lend of the side of transport flexibility for a framework
like Samza over binding to a transport medium like Kafka.


On Wed, Jul 8, 2015 at 1:39 PM, Jay Kreps <ja...@gmail.com> wrote:

> Good summary Jakob.
>
> WRT to the general purpose vs Kafka-specific, I actually see it slightly
> differently. Consider how Storm works as an example, there is a data source
> (spout) which could be Kafka, Database, etc, and then there is a transport
> (a netty TCP thing iiuc). Storm allows you to process data from any source,
> but when it comes from a source they always funnel it through their
> transport to get to the tasks/bolts. It is natural to think of Kafka as the
> Spout, but I think the better analogy is actually that Kafka is the
> transport.
>
> It is really hard to make the transport truly pluggable because this is
> what the tasks interact with and you need to have guarantees about delivery
> (and reprocessing), partitioning, atomicity of output, ordering, etc so
> your stream processing can get the right answer. From my point of view what
> this proposal says is that Kafka would be non-pluggable as the *transport*.
>
> So in this proposal data would still come into and out of Kafka from a wide
> variety of sources, but by requiring Kafka as the transport the interaction
> with the tasks will always look the same (a persistent, partitioned, log).
> So going back to the Storm analogy it is something like
>   Spout interface = copy cat
>   Bolt interface = samza
>
> This does obviously make Samza dependent on Kafka but it doesn't mean you
> wouldn't be processing data from all kinds of sources--indeed that is the
> whole purpose. It just means that each of these data streams would be
> available as a multi-subscriber Kafka topic to other systems, applications,
> etc, not just for your job.
>
> If you think about how things are now Samza already depends on a
> partitioned, persistent, offset addressable log with log
> compaction...which, unsurprisingly, so I don't think this is really a new
> dependency.
>
> Philosophically I think this makes sense too. To make a bunch of programs
> fit together you have to standardize something. In this proposal what you
> are standardizing around is really Kafka's protocol for streaming data and
> your data format. The transformations that connect these streams can be
> done via Samza, Storm, Spark, standalone java or python programs, etc but
> the ultimate output and contract to the rest of the organization/world will
> be the resulting Kafka topic. Philosophically I think this kind of data and
> protocol based contract is the right way to go rather than saying that the
> contract is a particular java api and the stream/data is what is pluggable.
>
> -Jay
>
>
>
> On Wed, Jul 8, 2015 at 11:03 AM, Jakob Homan <jg...@gmail.com> wrote:
>
> > Rewinding back to the beginning of this topic, there are effectively
> > three proposals on the table:
> >
> > 1) Chris' ideas for a direction towards a 2.0 release with an emphasis
> > on API and configuration simplification.  This ideas are based on lots
> > of lessons learned from the 0.x branch and are worthy of a 2.0 label
> > and breaking backwards compability.  I'm not sure I agree with all of
> > them, but they're definitely worth pursuing.
> >
> > 2) Chris' alternative proposal, which goes beyond his first and is
> > essentially a reboot of Samza to a more limited, entirely
> > Kafka-focused approach.  Samza would cease being a general purpose
> > stream processing framework, akin to and an alternative to say, Apache
> > Storm, and would instead become a standalone complement to the Kafka
> > project.
> >
> > 3) Jay's proposal, which goes even further, and suggests that the
> > Kafka community would be better served by adding stream processing as
> > a module to Kafka.  This is a perfectly valid approach, but since it's
> > entirely confined to the Kafka project, doesn't really involve Samza.
> > If the Kafka team were to go this route, there would be no obligation
> > on the Samza team to shut down, disband, etc.
> >
> > This last bit is important because Samza and Kafka, while closely
> > linked, are distinct communities.  The intersection of committers on
> > both Kafka and Samza is three people out of a combined 18 committers
> > across both projects.   Samza is a distinct community that shares
> > quite a few users with Kafka, but is able to chart its own course.
> >
> > My own view is that Samza has had an amazing year and is taking off at
> > a rapid rate.  It was only proposed for Incubator two years ago and is
> > still very young. The original team at LinkedIn has left that company
> > but the project has continued to grow via contributions both from
> > LinkedIn and from without.  We've recently seen a significant uptake
> > in discussion and bug reports.
> >
> > The API, deployment and configuration changes Chris suggests are good
> > ideas, but I think there is still serious value in having a
> > stand-alone general stream processing framework that supports other
> > input sources than Kafka.  We've already had contributions for adding
> > producer support to ElasticSearch and HDFS.  As more users come on
> > board, I would expect them to contribute more consumers and producers.
> >
> > It's a bit of chicken-and-the-egg problem; since the original team
> > didn't have cycles to prioritize support for non-Kafka systems
> > (kinesis, eventhub, twitter, flume, zeromq, etc.), Samza was less
> > compelling than other stream processing frameworks that did have
> > support and was therefore not used in those situations.  I'd love to
> > see those added and the SystemConsumer/Producer APIs improved to
> > fluently support them as well as Kafka.
> > Martin had a question regarding the tight coupling between Hadoop HDFS
> > and MapReduce (and YARN and Common).  This has been a problem for
> > years and there have been several aborted attempts to split the
> > projects out.  Each time there turned out to be a strong need for
> > cross-cutting collaboration and so the effort was dropped.  Absent the
> > third option above (Kafka adding stream support to itself directly), I
> > would imagine something similar would play out here.
> >
> > We should get a feeling for which of the three proposals the Samza
> > community is behind, technical details of each notwithstanding.  This
> > would include not just the committers/PMC members, but also the users,
> > contributors and lurkers.
> >
> > -Jakob
> >
> > On 8 July 2015 at 07:41, Ben Kirwin <be...@kirw.in> wrote:
> > > Hi all,
> > >
> > > Interesting stuff! Jumping in a bit late, but here goes...
> > >
> > > I'd definitely be excited about a slimmed-down and more Kafka-specific
> > > Samza -- you don't seem to lose much functionality that people
> > > actually use, and the gains in simplicity / code sharing seem
> > > potentially very large. (I've spent a bunch of time peeling back those
> > > layers of abstraction to get eg. more control over message send order,
> > > and working directly against Kafka's APIs would have been much
> > > easier.) I also like the approach of letting Kafka code do the heavy
> > > lifting and letting stream processing systems build on those -- good,
> > > reusable implementations would be great for the whole
> > > stream-processing ecosystem, and Samza in particular.
> > >
> > > On the other hand, I do hope that using Kafka's group membership /
> > > partition assignment / etc. stays optional. As far as I can tell,
> > > ~every major stream processing system that uses Kafka has chosen (or
> > > switched to) 'static' partitioning, where each logical task consumes a
> > > fixed set of partitions. When 'dynamic deploying' (a la Storm / Mesos
> > > / Yarn) the underlying system is already doing failure detection and
> > > transferring work between hosts when machines go down, so using
> > > Kafka's implementation is redundant at best -- and at worst, the
> > > interaction between the two systems can make outages worse.
> > >
> > > And thanks to Chris / Jay for getting this ball rolling. Exciting
> > times...
> > >
> > > On Tue, Jul 7, 2015 at 2:35 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >> Hey Roger,
> > >>
> > >> I couldn't agree more. We spent a bunch of time talking to people and
> > that
> > >> is exactly the stuff we heard time and again. What makes it hard, of
> > >> course, is that there is some tension between compatibility with
> what's
> > >> there now and making things better for new users.
> > >>
> > >> I also strongly agree with the importance of multi-language support.
> We
> > are
> > >> talking now about Java, but for application development use cases
> people
> > >> want to work in whatever language they are using elsewhere. I think
> > moving
> > >> to a model where Kafka itself does the group membership, lifecycle
> > control,
> > >> and partition assignment has the advantage of putting all that complex
> > >> stuff behind a clean api that the clients are already going to be
> > >> implementing for their consumer, so the added functionality for stream
> > >> processing beyond a consumer becomes very minor.
> > >>
> > >> -Jay
> > >>
> > >> On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <roger.hoover@gmail.com
> >
> > >> wrote:
> > >>
> > >>> Metamorphosis...nice. :)
> > >>>
> > >>> This has been a great discussion.  As a user of Samza who's recently
> > >>> integrated it into a relatively large organization, I just want to
> add
> > >>> support to a few points already made.
> > >>>
> > >>> The biggest hurdles to adoption of Samza as it currently exists that
> > I've
> > >>> experienced are:
> > >>> 1) YARN - YARN is overly complex in many environments where Puppet
> > would do
> > >>> just fine but it was the only mechanism to get fault tolerance.
> > >>> 2) Configuration - I think I like the idea of configuring most of the
> > job
> > >>> in code rather than config files.  In general, I think the goal
> should
> > be
> > >>> to make it harder to make mistakes, especially of the kind where the
> > code
> > >>> expects something and the config doesn't match.  The current config
> is
> > >>> quite intricate and error-prone.  For example, the application logic
> > may
> > >>> depend on bootstrapping a topic but rather than asserting that in the
> > code,
> > >>> you have to rely on getting the config right.  Likewise with serdes,
> > the
> > >>> Java representations produced by various serdes (JSON, Avro, etc.)
> are
> > not
> > >>> equivalent so you cannot just reconfigure a serde without changing
> the
> > >>> code.   It would be nice for jobs to be able to assert what they
> expect
> > >>> from their input topics in terms of partitioning.  This is getting a
> > little
> > >>> off topic but I was even thinking about creating a "Samza config
> > linter"
> > >>> that would sanity check a set of configs.  Especially in
> organizations
> > >>> where config is managed by a different team than the application
> > developer,
> > >>> it's very hard to get avoid config mistakes.
> > >>> 3) Java/Scala centric - for many teams (especially DevOps-type
> folks),
> > the
> > >>> pain of the Java toolchain (maven, slow builds, weak command line
> > support,
> > >>> configuration over convention) really inhibits productivity.  As more
> > and
> > >>> more high-quality clients become available for Kafka, I hope they'll
> > follow
> > >>> Samza's model.  Not sure how much it affects the proposals in this
> > thread
> > >>> but please consider other languages in the ecosystem as well.  From
> > what
> > >>> I've heard, Spark has more Python users than Java/Scala.
> > >>> (FYI, we added a Jython wrapper for the Samza API
> > >>>
> > >>>
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > >>> and are working on a Yeoman generator
> > >>> https://github.com/Quantiply/generator-rico for Jython/Samza
> projects
> > to
> > >>> alleviate some of the pain)
> > >>>
> > >>> I also want to underscore Jay's point about improving the user
> > experience.
> > >>> That's a very important factor for adoption.  I think the goal should
> > be to
> > >>> make Samza as easy to get started with as something like Logstash.
> > >>> Logstash is vastly inferior in terms of capabilities to Samza but
> it's
> > easy
> > >>> to get started and that makes a big difference.
> > >>>
> > >>> Cheers,
> > >>>
> > >>> Roger
> > >>>
> > >>>
> > >>>
> > >>>
> > >>>
> > >>> On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
> > >>> gdfm@apache.org> wrote:
> > >>>
> > >>> > Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
> > >>> winner
> > >>> > :)
> > >>> >
> > >>> > --
> > >>> > Gianmarco
> > >>> >
> > >>> > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
> > gdfm@apache.org
> > >>> >
> > >>> > wrote:
> > >>> >
> > >>> > > Hi,
> > >>> > >
> > >>> > > @Martin, thanks for you comments.
> > >>> > > Maybe I'm missing some important point, but I think coupling the
> > >>> releases
> > >>> > > is actually a *good* thing.
> > >>> > > To make an example, would it be better if the MR and HDFS
> > components of
> > >>> > > Hadoop had different release schedules?
> > >>> > >
> > >>> > > Actually, keeping the discussion in a single place would make
> > agreeing
> > >>> on
> > >>> > > releases (and backwards compatibility) much easier, as everybody
> > would
> > >>> be
> > >>> > > responsible for the whole codebase.
> > >>> > >
> > >>> > > That said, I like the idea of absorbing samza-core as a
> > sub-project,
> > >>> and
> > >>> > > leave the fancy stuff separate.
> > >>> > > It probably gives 90% of the benefits we have been discussing
> here.
> > >>> > >
> > >>> > > Cheers,
> > >>> > >
> > >>> > > --
> > >>> > > Gianmarco
> > >>> > >
> > >>> > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
> > >>> > >
> > >>> > >> Hey Martin,
> > >>> > >>
> > >>> > >> I agree coupling release schedules is a downside.
> > >>> > >>
> > >>> > >> Definitely we can try to solve some of the integration problems
> in
> > >>> > >> Confluent Platform or in other distributions. But I think this
> > ends up
> > >>> > >> being really shallow. I guess I feel to really get a good user
> > >>> > experience
> > >>> > >> the two systems have to kind of feel like part of the same thing
> > and
> > >>> you
> > >>> > >> can't really add that in later--you can put both in the same
> > >>> > downloadable
> > >>> > >> tar file but it doesn't really give a very cohesive feeling. I
> > agree
> > >>> > that
> > >>> > >> ultimately any of the project stuff is as much social and naming
> > as
> > >>> > >> anything else--theoretically two totally independent projects
> > could
> > >>> work
> > >>> > >> to
> > >>> > >> tightly align. In practice this seems to be quite difficult
> > though.
> > >>> > >>
> > >>> > >> For the frameworks--totally agree it would be good to maintain
> the
> > >>> > >> framework support with the project. In some cases there may not
> > be too
> > >>> > >> much
> > >>> > >> there since the integration gets lighter but I think whatever
> > stubs
> > >>> you
> > >>> > >> need should be included. So no I definitely wasn't trying to
> imply
> > >>> > >> dropping
> > >>> > >> support for these frameworks, just making the integration
> lighter
> > by
> > >>> > >> separating process management from partition management.
> > >>> > >>
> > >>> > >> You raise two good points we would have to figure out if we went
> > down
> > >>> > the
> > >>> > >> alignment path:
> > >>> > >> 1. With respect to the name, yeah I think the first question is
> > >>> whether
> > >>> > >> some "re-branding" would be worth it. If so then I think we can
> > have a
> > >>> > big
> > >>> > >> thread on the name. I'm definitely not set on Kafka Streaming or
> > Kafka
> > >>> > >> Streams I was just using them to be kind of illustrative. I
> agree
> > with
> > >>> > >> your
> > >>> > >> critique of these names, though I think people would get the
> idea.
> > >>> > >> 2. Yeah you also raise a good point about how to "factor" it.
> > Here are
> > >>> > the
> > >>> > >> options I see (I could get enthusiastic about any of them):
> > >>> > >>    a. One repo for both Kafka and Samza
> > >>> > >>    b. Two repos, retaining the current seperation
> > >>> > >>    c. Two repos, the equivalent of samza-api and samza-core is
> > >>> absorbed
> > >>> > >> almost like a third client
> > >>> > >>
> > >>> > >> Cheers,
> > >>> > >>
> > >>> > >> -Jay
> > >>> > >>
> > >>> > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> > >>> martin@kleppmann.com>
> > >>> > >> wrote:
> > >>> > >>
> > >>> > >> > Ok, thanks for the clarifications. Just a few follow-up
> > comments.
> > >>> > >> >
> > >>> > >> > - I see the appeal of merging with Kafka or becoming a
> > subproject:
> > >>> the
> > >>> > >> > reasons you mention are good. The risk I see is that release
> > >>> schedules
> > >>> > >> > become coupled to each other, which can slow everyone down,
> and
> > >>> large
> > >>> > >> > projects with many contributors are harder to manage. (Jakob,
> > can
> > >>> you
> > >>> > >> speak
> > >>> > >> > from experience, having seen a wider range of Hadoop ecosystem
> > >>> > >> projects?)
> > >>> > >> >
> > >>> > >> > Some of the goals of a better unified developer experience
> could
> > >>> also
> > >>> > be
> > >>> > >> > solved by integrating Samza nicely into a Kafka distribution
> > (such
> > >>> as
> > >>> > >> > Confluent's). I'm not against merging projects if we decide
> > that's
> > >>> the
> > >>> > >> way
> > >>> > >> > to go, just pointing out the same goals can perhaps also be
> > achieved
> > >>> > in
> > >>> > >> > other ways.
> > >>> > >> >
> > >>> > >> > - With regard to dropping the YARN dependency: are you
> proposing
> > >>> that
> > >>> > >> > Samza doesn't give any help to people wanting to run on
> > >>> > >> YARN/Mesos/AWS/etc?
> > >>> > >> > So the docs would basically have a link to Slider and nothing
> > else?
> > >>> Or
> > >>> > >> > would we maintain integrations with a bunch of popular
> > deployment
> > >>> > >> methods
> > >>> > >> > (e.g. the necessary glue and shell scripts to make Samza work
> > with
> > >>> > >> Slider)?
> > >>> > >> >
> > >>> > >> > I absolutely think it's a good idea to have the "as a library"
> > and
> > >>> > "as a
> > >>> > >> > process" (using Yi's taxonomy) options for people who want
> them,
> > >>> but I
> > >>> > >> > think there should also be a low-friction path for common "as
> a
> > >>> > service"
> > >>> > >> > deployment methods, for which we probably need to maintain
> > >>> > integrations.
> > >>> > >> >
> > >>> > >> > - Project naming: "Kafka Streams" seems odd to me, because
> > Kafka is
> > >>> > all
> > >>> > >> > about streams already. Perhaps "Kafka Transformers" or "Kafka
> > >>> Filters"
> > >>> > >> > would be more apt?
> > >>> > >> >
> > >>> > >> > One suggestion: perhaps the core of Samza (stream
> transformation
> > >>> with
> > >>> > >> > state management -- i.e. the "Samza as a library" bit) could
> > become
> > >>> > >> part of
> > >>> > >> > Kafka, while higher-level tools such as streaming SQL and
> > >>> integrations
> > >>> > >> with
> > >>> > >> > deployment frameworks remain in a separate project? In other
> > words,
> > >>> > >> Kafka
> > >>> > >> > would absorb the proven, stable core of Samza, which would
> > become
> > >>> the
> > >>> > >> > "third Kafka client" mentioned early in this thread. The Samza
> > >>> project
> > >>> > >> > would then target that third Kafka client as its base API, and
> > the
> > >>> > >> project
> > >>> > >> > would be freed up to explore more experimental new horizons.
> > >>> > >> >
> > >>> > >> > Martin
> > >>> > >> >
> > >>> > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com>
> wrote:
> > >>> > >> >
> > >>> > >> > > Hey Martin,
> > >>> > >> > >
> > >>> > >> > > For the YARN/Mesos/etc decoupling I actually don't think it
> > ties
> > >>> our
> > >>> > >> > hands
> > >>> > >> > > at all, all it does is refactor things. The division of
> > >>> > >> responsibility is
> > >>> > >> > > that Samza core is responsible for task lifecycle, state,
> and
> > >>> > >> partition
> > >>> > >> > > management (using the Kafka co-ordinator) but it is NOT
> > >>> responsible
> > >>> > >> for
> > >>> > >> > > packaging, configuration deployment or execution of
> > processes. The
> > >>> > >> > problem
> > >>> > >> > > of packaging and starting these processes is
> > >>> > >> > > framework/environment-specific. This leaves individual
> > frameworks
> > >>> to
> > >>> > >> be
> > >>> > >> > as
> > >>> > >> > > fancy or vanilla as they like. So you can get simple
> stateless
> > >>> > >> support in
> > >>> > >> > > YARN, Mesos, etc using their off-the-shelf app framework
> > (Slider,
> > >>> > >> > Marathon,
> > >>> > >> > > etc). These are well known by people and have nice UIs and a
> > lot
> > >>> of
> > >>> > >> > > flexibility. I don't think they have node affinity as a
> built
> > in
> > >>> > >> option
> > >>> > >> > > (though I could be wrong). So if we want that we can either
> > wait
> > >>> for
> > >>> > >> them
> > >>> > >> > > to add it or do a custom framework to add that feature (as
> > now).
> > >>> > >> > Obviously
> > >>> > >> > > if you manage things with old-school ops tools
> > (puppet/chef/etc)
> > >>> you
> > >>> > >> get
> > >>> > >> > > locality easily. The nice thing, though, is that all the
> samza
> > >>> > >> "business
> > >>> > >> > > logic" around partition management and fault tolerance is in
> > Samza
> > >>> > >> core
> > >>> > >> > so
> > >>> > >> > > it is shared across frameworks and the framework specific
> bit
> > is
> > >>> > just
> > >>> > >> > > whether it is smart enough to try to get the same host when
> a
> > job
> > >>> is
> > >>> > >> > > restarted.
> > >>> > >> > >
> > >>> > >> > > With respect to the Kafka-alignment, yeah I think the goal
> > would
> > >>> be
> > >>> > >> (a)
> > >>> > >> > > actually get better alignment in user experience, and (b)
> > express
> > >>> > >> this in
> > >>> > >> > > the naming and project branding. Specifically:
> > >>> > >> > > 1. Website/docs, it would be nice for the "transformation"
> > api to
> > >>> be
> > >>> > >> > > discoverable in the main Kafka docs--i.e. be able to explain
> > when
> > >>> to
> > >>> > >> use
> > >>> > >> > > the consumer and when to use the stream processing
> > functionality
> > >>> and
> > >>> > >> lead
> > >>> > >> > > people into that experience.
> > >>> > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever)
> > that
> > >>> has
> > >>> > >> both
> > >>> > >> > > Kafka and the stream processing part and they actually work
> > >>> > together.
> > >>> > >> > > 3. Unify the programming experience so the client and Samza
> > api
> > >>> > share
> > >>> > >> > > config/monitoring/naming/packaging/etc.
> > >>> > >> > >
> > >>> > >> > > I think sub-projects keep separate committers and can have a
> > >>> > separate
> > >>> > >> > repo,
> > >>> > >> > > but I'm actually not really sure (I can't find a definition
> > of a
> > >>> > >> > subproject
> > >>> > >> > > in Apache).
> > >>> > >> > >
> > >>> > >> > > Basically at a high-level you want the experience to "feel"
> > like a
> > >>> > >> single
> > >>> > >> > > system, not to relatively independent things that are kind
> of
> > >>> > >> awkwardly
> > >>> > >> > > glued together.
> > >>> > >> > >
> > >>> > >> > > I think if we did that they having naming or branding like
> > "kafka
> > >>> > >> > > streaming" or "kafka streams" or something like that would
> > >>> actually
> > >>> > >> do a
> > >>> > >> > > good job of conveying what it is. I do that this would help
> > >>> adoption
> > >>> > >> > quite
> > >>> > >> > > a lot as it would correctly convey that using Kafka
> Streaming
> > with
> > >>> > >> Kafka
> > >>> > >> > is
> > >>> > >> > > a fairly seamless experience and Kafka is pretty heavily
> > adopted
> > >>> at
> > >>> > >> this
> > >>> > >> > > point.
> > >>> > >> > >
> > >>> > >> > > Fwiw we actually considered this model originally when open
> > >>> sourcing
> > >>> > >> > Samza,
> > >>> > >> > > however at that time Kafka was relatively unknown and we
> > decided
> > >>> not
> > >>> > >> to
> > >>> > >> > do
> > >>> > >> > > it since we felt it would be limiting. From my point of view
> > the
> > >>> > three
> > >>> > >> > > things have changed (1) Kafka is now really heavily used for
> > >>> stream
> > >>> > >> > > processing, (2) we learned that abstracting out the stream
> > well is
> > >>> > >> > > basically impossible, (3) we learned it is really hard to
> > keep the
> > >>> > two
> > >>> > >> > > things feeling like a single product.
> > >>> > >> > >
> > >>> > >> > > -Jay
> > >>> > >> > >
> > >>> > >> > >
> > >>> > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> > >>> > >> martin@kleppmann.com>
> > >>> > >> > > wrote:
> > >>> > >> > >
> > >>> > >> > >> Hi all,
> > >>> > >> > >>
> > >>> > >> > >> Lots of good thoughts here.
> > >>> > >> > >>
> > >>> > >> > >> I agree with the general philosophy of tying Samza more
> > firmly to
> > >>> > >> Kafka.
> > >>> > >> > >> After I spent a while looking at integrating other message
> > >>> brokers
> > >>> > >> (e.g.
> > >>> > >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
> > >>> > >> > SystemConsumer
> > >>> > >> > >> tacitly assumes a model so much like Kafka's that pretty
> much
> > >>> > nobody
> > >>> > >> but
> > >>> > >> > >> Kafka actually implements it. (Databus is perhaps an
> > exception,
> > >>> but
> > >>> > >> it
> > >>> > >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza
> > fully
> > >>> > >> > dependent
> > >>> > >> > >> on Kafka acknowledges that the system-independence was
> never
> > as
> > >>> > real
> > >>> > >> as
> > >>> > >> > we
> > >>> > >> > >> perhaps made it out to be. The gains of code reuse are
> real.
> > >>> > >> > >>
> > >>> > >> > >> The idea of decoupling Samza from YARN has also always been
> > >>> > >> appealing to
> > >>> > >> > >> me, for various reasons already mentioned in this thread.
> > >>> Although
> > >>> > >> > making
> > >>> > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc)
> seems
> > >>> > >> laudable,
> > >>> > >> > I am
> > >>> > >> > >> a little concerned that it will restrict us to a lowest
> > common
> > >>> > >> > denominator.
> > >>> > >> > >> For example, would host affinity (SAMZA-617) still be
> > possible?
> > >>> For
> > >>> > >> jobs
> > >>> > >> > >> with large amounts of state, I think SAMZA-617 would be a
> big
> > >>> boon,
> > >>> > >> > since
> > >>> > >> > >> restoring state off the changelog on every single restart
> is
> > >>> > painful,
> > >>> > >> > due
> > >>> > >> > >> to long recovery times. It would be a shame if the
> decoupling
> > >>> from
> > >>> > >> YARN
> > >>> > >> > >> made host affinity impossible.
> > >>> > >> > >>
> > >>> > >> > >> Jay, a question about the proposed API for instantiating a
> > job in
> > >>> > >> code
> > >>> > >> > >> (rather than a properties file): when submitting a job to a
> > >>> > cluster,
> > >>> > >> is
> > >>> > >> > the
> > >>> > >> > >> idea that the instantiation code runs on a client
> somewhere,
> > >>> which
> > >>> > >> then
> > >>> > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or
> does
> > that
> > >>> > >> code
> > >>> > >> > run
> > >>> > >> > >> on each container that is part of the job (in which case,
> how
> > >>> does
> > >>> > >> the
> > >>> > >> > job
> > >>> > >> > >> submission to the cluster work)?
> > >>> > >> > >>
> > >>> > >> > >> I agree with Garry that it doesn't feel right to make a 1.0
> > >>> release
> > >>> > >> > with a
> > >>> > >> > >> plan for it to be immediately obsolete. So if this is going
> > to
> > >>> > >> happen, I
> > >>> > >> > >> think it would be more honest to stick with 0.* version
> > numbers
> > >>> > until
> > >>> > >> > the
> > >>> > >> > >> library-ified Samza has been implemented, is stable and
> > widely
> > >>> > used.
> > >>> > >> > >>
> > >>> > >> > >> Should the new Samza be a subproject of Kafka? There is
> > precedent
> > >>> > for
> > >>> > >> > >> tight coupling between different Apache projects (e.g.
> > Curator
> > >>> and
> > >>> > >> > >> Zookeeper, or Slider and YARN), so I think remaining
> separate
> > >>> would
> > >>> > >> be
> > >>> > >> > ok.
> > >>> > >> > >> Even if Samza is fully dependent on Kafka, there is enough
> > >>> > substance
> > >>> > >> in
> > >>> > >> > >> Samza that it warrants being a separate project. An
> argument
> > in
> > >>> > >> favour
> > >>> > >> > of
> > >>> > >> > >> merging would be if we think Kafka has a much stronger
> "brand
> > >>> > >> presence"
> > >>> > >> > >> than Samza; I'm ambivalent on that one. If the Kafka
> project
> > is
> > >>> > >> willing
> > >>> > >> > to
> > >>> > >> > >> endorse Samza as the "official" way of doing stateful
> stream
> > >>> > >> > >> transformations, that would probably have much the same
> > effect as
> > >>> > >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike.
> > Close
> > >>> > >> > >> collaboration between the two projects will be needed in
> any
> > >>> case.
> > >>> > >> > >>
> > >>> > >> > >> From a project management perspective, I guess the "new
> > Samza"
> > >>> > would
> > >>> > >> > have
> > >>> > >> > >> to be developed on a branch alongside ongoing maintenance
> of
> > the
> > >>> > >> current
> > >>> > >> > >> line of development? I think it would be important to
> > continue
> > >>> > >> > supporting
> > >>> > >> > >> existing users, and provide a graceful migration path to
> the
> > new
> > >>> > >> > version.
> > >>> > >> > >> Leaving the current versions unsupported and forcing people
> > to
> > >>> > >> rewrite
> > >>> > >> > >> their jobs would send a bad signal.
> > >>> > >> > >>
> > >>> > >> > >> Best,
> > >>> > >> > >> Martin
> > >>> > >> > >>
> > >>> > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io>
> wrote:
> > >>> > >> > >>
> > >>> > >> > >>> Hey Garry,
> > >>> > >> > >>>
> > >>> > >> > >>> Yeah that's super frustrating. I'd be happy to chat more
> > about
> > >>> > this
> > >>> > >> if
> > >>> > >> > >>> you'd be interested. I think Chris and I started with the
> > idea
> > >>> of
> > >>> > >> "what
> > >>> > >> > >>> would it take to make Samza a kick-ass ingestion tool" but
> > >>> > >> ultimately
> > >>> > >> > we
> > >>> > >> > >>> kind of came around to the idea that ingestion and
> > >>> transformation
> > >>> > >> had
> > >>> > >> > >>> pretty different needs and coupling the two made things
> > hard.
> > >>> > >> > >>>
> > >>> > >> > >>> For what it's worth I think copycat (KIP-26) actually will
> > do
> > >>> what
> > >>> > >> you
> > >>> > >> > >> are
> > >>> > >> > >>> looking for.
> > >>> > >> > >>>
> > >>> > >> > >>> With regard to your point about slider, I don't
> necessarily
> > >>> > >> disagree.
> > >>> > >> > >> But I
> > >>> > >> > >>> think getting good YARN support is quite doable and I
> think
> > we
> > >>> can
> > >>> > >> make
> > >>> > >> > >>> that work well. I think the issue this proposal solves is
> > that
> > >>> > >> > >> technically
> > >>> > >> > >>> it is pretty hard to support multiple cluster management
> > systems
> > >>> > the
> > >>> > >> > way
> > >>> > >> > >>> things are now, you need to write an "app master" or
> > "framework"
> > >>> > for
> > >>> > >> > each
> > >>> > >> > >>> and they are all a little different so testing is really
> > hard.
> > >>> In
> > >>> > >> the
> > >>> > >> > >>> absence of this we have been stuck with just YARN which
> has
> > >>> > >> fantastic
> > >>> > >> > >>> penetration in the Hadoopy part of the org, but zero
> > penetration
> > >>> > >> > >> elsewhere.
> > >>> > >> > >>> Given the huge amount of work being put in to slider,
> > marathon,
> > >>> > aws
> > >>> > >> > >>> tooling, not to mention the umpteen related packaging
> > >>> technologies
> > >>> > >> > people
> > >>> > >> > >>> want to use (Docker, Kubernetes, various cloud-specific
> > deploy
> > >>> > >> tools,
> > >>> > >> > >> etc)
> > >>> > >> > >>> I really think it is important to get this right.
> > >>> > >> > >>>
> > >>> > >> > >>> -Jay
> > >>> > >> > >>>
> > >>> > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > >>> > >> > >>> g.turkington@improvedigital.com> wrote:
> > >>> > >> > >>>
> > >>> > >> > >>>> Hi all,
> > >>> > >> > >>>>
> > >>> > >> > >>>> I think the question below re does Samza become a
> > sub-project
> > >>> of
> > >>> > >> Kafka
> > >>> > >> > >>>> highlights the broader point around migration. Chris
> > mentions
> > >>> > >> Samza's
> > >>> > >> > >>>> maturity is heading towards a v1 release but I'm not sure
> > it
> > >>> > feels
> > >>> > >> > >> right to
> > >>> > >> > >>>> launch a v1 then immediately plan to deprecate most of
> it.
> > >>> > >> > >>>>
> > >>> > >> > >>>> From a selfish perspective I have some guys who have
> > started
> > >>> > >> working
> > >>> > >> > >> with
> > >>> > >> > >>>> Samza and building some new consumers/producers was next
> > up.
> > >>> > Sounds
> > >>> > >> > like
> > >>> > >> > >>>> that is absolutely not the direction to go. I need to
> look
> > into
> > >>> > the
> > >>> > >> > KIP
> > >>> > >> > >> in
> > >>> > >> > >>>> more detail but for me the attractiveness of adding new
> > Samza
> > >>> > >> > >>>> consumer/producers -- even if yes all they were doing was
> > >>> really
> > >>> > >> > getting
> > >>> > >> > >>>> data into and out of Kafka --  was to avoid  having to
> > worry
> > >>> > about
> > >>> > >> the
> > >>> > >> > >>>> lifecycle management of external clients. If there is a
> > generic
> > >>> > >> Kafka
> > >>> > >> > >>>> ingress/egress layer that I can plug a new connector into
> > and
> > >>> > have
> > >>> > >> a
> > >>> > >> > >> lot of
> > >>> > >> > >>>> the heavy lifting re scale and reliability done for me
> > then it
> > >>> > >> gives
> > >>> > >> > me
> > >>> > >> > >> all
> > >>> > >> > >>>> the pushing new consumers/producers would. If not then it
> > >>> > >> complicates
> > >>> > >> > my
> > >>> > >> > >>>> operational deployments.
> > >>> > >> > >>>>
> > >>> > >> > >>>> Which is similar to my other question with the proposal
> --
> > if
> > >>> we
> > >>> > >> > build a
> > >>> > >> > >>>> fully available/stand-alone Samza plus the requisite
> shims
> > to
> > >>> > >> > integrate
> > >>> > >> > >>>> with Slider etc I suspect the former may be a lot more
> work
> > >>> than
> > >>> > we
> > >>> > >> > >> think.
> > >>> > >> > >>>> We may make it much easier for a newcomer to get
> something
> > >>> > running
> > >>> > >> but
> > >>> > >> > >>>> having them step up and get a reliable production
> > deployment
> > >>> may
> > >>> > >> still
> > >>> > >> > >>>> dominate mailing list  traffic, if for different reasons
> > than
> > >>> > >> today.
> > >>> > >> > >>>>
> > >>> > >> > >>>> Don't get me wrong -- I'm comfortable with making the
> Samza
> > >>> > >> dependency
> > >>> > >> > >> on
> > >>> > >> > >>>> Kafka much more explicit and I absolutely see the
> > benefits  in
> > >>> > the
> > >>> > >> > >>>> reduction of duplication and clashing
> > >>> terminologies/abstractions
> > >>> > >> that
> > >>> > >> > >>>> Chris/Jay describe. Samza as a library would likely be a
> > very
> > >>> > nice
> > >>> > >> > tool
> > >>> > >> > >> to
> > >>> > >> > >>>> add to the Kafka ecosystem. I just have the concerns
> above
> > re
> > >>> the
> > >>> > >> > >>>> operational side.
> > >>> > >> > >>>>
> > >>> > >> > >>>> Garry
> > >>> > >> > >>>>
> > >>> > >> > >>>> -----Original Message-----
> > >>> > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
> > gdfm@apache.org]
> > >>> > >> > >>>> Sent: 02 July 2015 12:56
> > >>> > >> > >>>> To: dev@samza.apache.org
> > >>> > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> > >>> > >> > >>>>
> > >>> > >> > >>>> Very interesting thoughts.
> > >>> > >> > >>>> From outside, I have always perceived Samza as a
> computing
> > >>> layer
> > >>> > >> over
> > >>> > >> > >>>> Kafka.
> > >>> > >> > >>>>
> > >>> > >> > >>>> The question, maybe a bit provocative, is "should Samza
> be
> > a
> > >>> > >> > sub-project
> > >>> > >> > >>>> of Kafka then?"
> > >>> > >> > >>>> Or does it make sense to keep it as a separate project
> > with a
> > >>> > >> separate
> > >>> > >> > >>>> governance?
> > >>> > >> > >>>>
> > >>> > >> > >>>> Cheers,
> > >>> > >> > >>>>
> > >>> > >> > >>>> --
> > >>> > >> > >>>> Gianmarco
> > >>> > >> > >>>>
> > >>> > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> > >>> wrote:
> > >>> > >> > >>>>
> > >>> > >> > >>>>> Overall, I agree to couple with Kafka more tightly.
> > Because
> > >>> > Samza
> > >>> > >> de
> > >>> > >> > >>>>> facto is based on Kafka, and it should leverage what
> Kafka
> > >>> has.
> > >>> > At
> > >>> > >> > the
> > >>> > >> > >>>>> same time, Kafka does not need to reinvent what Samza
> > already
> > >>> > >> has. I
> > >>> > >> > >>>>> also like the idea of separating the ingestion and
> > >>> > transformation.
> > >>> > >> > >>>>>
> > >>> > >> > >>>>> But it is a little difficult for me to image how the
> Samza
> > >>> will
> > >>> > >> look
> > >>> > >> > >>>> like.
> > >>> > >> > >>>>> And I feel Chris and Jay have a little difference in
> > terms of
> > >>> > how
> > >>> > >> > >>>>> Samza should look like.
> > >>> > >> > >>>>>
> > >>> > >> > >>>>> *** Will it look like what Jay's code shows (A client of
> > >>> Kakfa)
> > >>> > ?
> > >>> > >> And
> > >>> > >> > >>>>> user's application code calls this client?
> > >>> > >> > >>>>>
> > >>> > >> > >>>>> 1. If we make Samza be a library of Kafka (like what the
> > code
> > >>> > >> shows),
> > >>> > >> > >>>>> how do we implement auto-balance and fault-tolerance?
> Are
> > they
> > >>> > >> taken
> > >>> > >> > >>>>> care by the Kafka broker or other mechanism, such as
> > "Samza
> > >>> > >> worker"
> > >>> > >> > >>>>> (just make up the name) ?
> > >>> > >> > >>>>>
> > >>> > >> > >>>>> 2. What about other features, such as auto-scaling,
> shared
> > >>> > state,
> > >>> > >> > >>>>> monitoring?
> > >>> > >> > >>>>>
> > >>> > >> > >>>>>
> > >>> > >> > >>>>> *** If we have Samza standalone, (is this what Chris
> > >>> suggests?)
> > >>> > >> > >>>>>
> > >>> > >> > >>>>> 1. we still need to ingest data from Kakfa and produce
> to
> > it.
> > >>> > >> Then it
> > >>> > >> > >>>>> becomes the same as what Samza looks like now, except it
> > does
> > >>> > not
> > >>> > >> > rely
> > >>> > >> > >>>>> on Yarn anymore.
> > >>> > >> > >>>>>
> > >>> > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's
> > metrics,
> > >>> > logs,
> > >>> > >> > >>>>> etc? Use Kafka code as the dependency?
> > >>> > >> > >>>>>
> > >>> > >> > >>>>>
> > >>> > >> > >>>>> Thanks,
> > >>> > >> > >>>>>
> > >>> > >> > >>>>> Fang, Yan
> > >>> > >> > >>>>> yanfang724@gmail.com
> > >>> > >> > >>>>>
> > >>> > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > >>> > wangguoz@gmail.com
> > >>> > >> >
> > >>> > >> > >>>> wrote:
> > >>> > >> > >>>>>
> > >>> > >> > >>>>>> Read through the code example and it looks good to me.
> A
> > few
> > >>> > >> > >>>>>> thoughts regarding deployment:
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>> Today Samza deploys as executable runnable like:
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> > >>> > >> > >>>> --config-path=file://...
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>> And this proposal advocate for deploying Samza more as
> > >>> embedded
> > >>> > >> > >>>>>> libraries in user application code (ignoring the
> > terminology
> > >>> > >> since
> > >>> > >> > >>>>>> it is not the
> > >>> > >> > >>>>> same
> > >>> > >> > >>>>>> as the prototype code):
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread
> > thread =
> > >>> > new
> > >>> > >> > >>>>>> Thread(task); thread.start();
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>> I think both of these deployment modes are important
> for
> > >>> > >> different
> > >>> > >> > >>>>>> types
> > >>> > >> > >>>>> of
> > >>> > >> > >>>>>> users. That said, I think making Samza purely
> standalone
> > is
> > >>> > still
> > >>> > >> > >>>>>> sufficient for either runnable or library modes.
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>> Guozhang
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> > >>> jay@confluent.io>
> > >>> > >> > wrote:
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>>> Looks like gmail mangled the code example, it was
> > supposed
> > >>> to
> > >>> > >> look
> > >>> > >> > >>>>>>> like
> > >>> > >> > >>>>>>> this:
> > >>> > >> > >>>>>>>
> > >>> > >> > >>>>>>> Properties props = new Properties();
> > >>> > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > >>> > >> StreamingConfig
> > >>> > >> > >>>>>>> config = new StreamingConfig(props);
> > >>> > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> > >>> > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> > >>> > >> > >>>>>>> config.serialization(new StringSerializer(), new
> > >>> > >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
> > >>> > >> > >>>>>>> KafkaStreaming(config); container.run();
> > >>> > >> > >>>>>>>
> > >>> > >> > >>>>>>> -Jay
> > >>> > >> > >>>>>>>
> > >>> > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> > >>> jay@confluent.io
> > >>> > >
> > >>> > >> > >>>> wrote:
> > >>> > >> > >>>>>>>
> > >>> > >> > >>>>>>>> Hey guys,
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> This came out of some conversations Chris and I were
> > having
> > >>> > >> > >>>>>>>> around
> > >>> > >> > >>>>>>> whether
> > >>> > >> > >>>>>>>> it would make sense to use Samza as a kind of data
> > >>> ingestion
> > >>> > >> > >>>>> framework
> > >>> > >> > >>>>>>> for
> > >>> > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat").
> This
> > >>> kind
> > >>> > of
> > >>> > >> > >>>>>> combined
> > >>> > >> > >>>>>>>> with complaints around config and YARN and the
> > discussion
> > >>> > >> around
> > >>> > >> > >>>>>>>> how
> > >>> > >> > >>>>> to
> > >>> > >> > >>>>>>>> best do a standalone mode.
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> So the thought experiment was, given that Samza was
> > >>> basically
> > >>> > >> > >>>>>>>> already totally Kafka specific, what if you just
> > embraced
> > >>> > that
> > >>> > >> > >>>>>>>> and turned it
> > >>> > >> > >>>>>> into
> > >>> > >> > >>>>>>>> something less like a heavyweight framework and more
> > like a
> > >>> > >> > >>>>>>>> third
> > >>> > >> > >>>>> Kafka
> > >>> > >> > >>>>>>>> client--a kind of "producing consumer" with state
> > >>> management
> > >>> > >> > >>>>>> facilities.
> > >>> > >> > >>>>>>>> Basically a library. Instead of a complex stream
> > processing
> > >>> > >> > >>>>>>>> framework
> > >>> > >> > >>>>>>> this
> > >>> > >> > >>>>>>>> would actually be a very simple thing, not much more
> > >>> > >> complicated
> > >>> > >> > >>>>>>>> to
> > >>> > >> > >>>>> use
> > >>> > >> > >>>>>>> or
> > >>> > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we
> thought
> > >>> about
> > >>> > >> it
> > >>> > >> > >>>>>>>> a
> > >>> > >> > >>>>> lot
> > >>> > >> > >>>>>> of
> > >>> > >> > >>>>>>>> what Samza (and the other stream processing systems
> > were
> > >>> > doing)
> > >>> > >> > >>>>> seemed
> > >>> > >> > >>>>>>> like
> > >>> > >> > >>>>>>>> kind of a hangover from MapReduce.
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> Of course you need to ingest/output data to and from
> > the
> > >>> > stream
> > >>> > >> > >>>>>>>> processing. But when we actually looked into how that
> > would
> > >>> > >> > >>>>>>>> work,
> > >>> > >> > >>>>> Samza
> > >>> > >> > >>>>>>>> isn't really an ideal data ingestion framework for a
> > bunch
> > >>> of
> > >>> > >> > >>>>> reasons.
> > >>> > >> > >>>>>> To
> > >>> > >> > >>>>>>>> really do that right you need a pretty different
> > internal
> > >>> > data
> > >>> > >> > >>>>>>>> model
> > >>> > >> > >>>>>> and
> > >>> > >> > >>>>>>>> set of apis. So what if you split them and had an api
> > for
> > >>> > Kafka
> > >>> > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate
> api
> > for
> > >>> > >> Kafka
> > >>> > >> > >>>>>>>> transformation (Samza).
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> This would also allow really embracing the same
> > terminology
> > >>> > and
> > >>> > >> > >>>>>>>> conventions. One complaint about the current state is
> > that
> > >>> > the
> > >>> > >> > >>>>>>>> two
> > >>> > >> > >>>>>>> systems
> > >>> > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs
> > >>> "topic"
> > >>> > >> and
> > >>> > >> > >>>>>>> different
> > >>> > >> > >>>>>>>> config and monitoring systems means you kind of have
> to
> > >>> learn
> > >>> > >> > >>>>>>>> Kafka's
> > >>> > >> > >>>>>>> way,
> > >>> > >> > >>>>>>>> then learn Samza's slightly different way, then kind
> of
> > >>> > >> > >>>>>>>> understand
> > >>> > >> > >>>>> how
> > >>> > >> > >>>>>>> they
> > >>> > >> > >>>>>>>> map to each other, which having walked a few people
> > through
> > >>> > >> this
> > >>> > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> Since I have been spending a lot of time on
> airplanes I
> > >>> > hacked
> > >>> > >> > >>>>>>>> up an ernest but still somewhat incomplete prototype
> of
> > >>> what
> > >>> > >> > >>>>>>>> this would
> > >>> > >> > >>>>> look
> > >>> > >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka
> > as it
> > >>> > >> > >>>>>>>> required a
> > >>> > >> > >>>>>> few
> > >>> > >> > >>>>>>>> changes to the new consumer. Here is the code:
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>
> > >>> > >> >
> > >>> >
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >>> > >> > >>>>> /apache/kafka/clients/streaming
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> For the purpose of the prototype I just liberally
> > renamed
> > >>> > >> > >>>>>>>> everything
> > >>> > >> > >>>>> to
> > >>> > >> > >>>>>>>> try to align it with Kafka with no regard for
> > >>> compatibility.
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> To use this would be something like this:
> > >>> > >> > >>>>>>>> Properties props = new Properties();
> > >>> > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > >>> > >> > >>>>>>>> StreamingConfig config = new
> > >>> > >> > >>>>> StreamingConfig(props);
> > >>> > >> > >>>>>>> config.subscribe("test-topic-1",
> > >>> > >> > >>>>>>>> "test-topic-2");
> > >>> > >> config.processor(ExampleStreamProcessor.class);
> > >>> > >> > >>>>>>> config.serialization(new
> > >>> > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
> > >>> KafkaStreaming
> > >>> > >> > >>>>>> container =
> > >>> > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> > >>> > StreamProcessor
> > >>> > >> > >>>>>>>> is basically StreamTask.
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> So rather than putting all the class names in a file
> > and
> > >>> then
> > >>> > >> > >>>>>>>> having
> > >>> > >> > >>>>>> the
> > >>> > >> > >>>>>>>> job assembled by reflection, you just instantiate the
> > >>> > container
> > >>> > >> > >>>>>>>> programmatically. Work is balanced over however many
> > >>> > instances
> > >>> > >> > >>>>>>>> of
> > >>> > >> > >>>>> this
> > >>> > >> > >>>>>>> are
> > >>> > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new
> tasks
> > are
> > >>> > >> added
> > >>> > >> > >>>>>>>> to
> > >>> > >> > >>>>> the
> > >>> > >> > >>>>>>>> existing containers without shutting them down).
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> We would provide some glue for running this stuff in
> > YARN
> > >>> via
> > >>> > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of
> their
> > >>> tools
> > >>> > >> > >>>>>>>> but from the
> > >>> > >> > >>>>>> point
> > >>> > >> > >>>>>>> of
> > >>> > >> > >>>>>>>> view of these frameworks these stream processing jobs
> > are
> > >>> > just
> > >>> > >> > >>>>>> stateless
> > >>> > >> > >>>>>>>> services that can come and go and expand and contract
> > at
> > >>> > will.
> > >>> > >> > >>>>>>>> There
> > >>> > >> > >>>>> is
> > >>> > >> > >>>>>>> no
> > >>> > >> > >>>>>>>> more custom scheduler.
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> Here are some relevant details:
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get
> > larger if
> > >>> we
> > >>> > >> > >>>>>>>>  productionized but not vastly larger. We really do
> > get a
> > >>> ton
> > >>> > >> > >>>>>>>> of
> > >>> > >> > >>>>>>> leverage
> > >>> > >> > >>>>>>>>  out of Kafka.
> > >>> > >> > >>>>>>>>  2. Partition management is fully delegated to the
> new
> > >>> > >> consumer.
> > >>> > >> > >>>>> This
> > >>> > >> > >>>>>>>>  is nice since now any partition management strategy
> > >>> > available
> > >>> > >> > >>>>>>>> to
> > >>> > >> > >>>>>> Kafka
> > >>> > >> > >>>>>>>>  consumer is also available to Samza (and vice versa)
> > and
> > >>> > with
> > >>> > >> > >>>>>>>> the
> > >>> > >> > >>>>>>> exact
> > >>> > >> > >>>>>>>>  same configs.
> > >>> > >> > >>>>>>>>  3. It supports state as well as state reuse
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> Anyhow take a look, hopefully it is thought
> provoking.
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> -Jay
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > >>> > >> > >>>>>> criccomini@apache.org>
> > >>> > >> > >>>>>>>> wrote:
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>>> Hey all,
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> I have had some discussions with Samza engineers at
> > >>> LinkedIn
> > >>> > >> > >>>>>>>>> and
> > >>> > >> > >>>>>>> Confluent
> > >>> > >> > >>>>>>>>> and we came up with a few observations and would
> like
> > to
> > >>> > >> > >>>>>>>>> propose
> > >>> > >> > >>>>> some
> > >>> > >> > >>>>>>>>> changes.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> We've observed some things that I want to call out
> > about
> > >>> > >> > >>>>>>>>> Samza's
> > >>> > >> > >>>>>> design,
> > >>> > >> > >>>>>>>>> and I'd like to propose some changes.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment
> system.
> > >>> > >> > >>>>>>>>> * Samza is too pluggable.
> > >>> > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> > >>> consumer
> > >>> > >> > >>>>>>>>> APIs
> > >>> > >> > >>>>> are
> > >>> > >> > >>>>>>>>> trying to solve a lot of the same problems.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> All three of these issues are related, but I'll
> > address
> > >>> them
> > >>> > >> in
> > >>> > >> > >>>>> order.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Deployment
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic
> > deployment
> > >>> > >> > >>>>>>>>> scheduler
> > >>> > >> > >>>>>> such
> > >>> > >> > >>>>>>>>> as
> > >>> > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we
> > bet
> > >>> that
> > >>> > >> > >>>>>>>>> there
> > >>> > >> > >>>>>> would
> > >>> > >> > >>>>>>>>> be
> > >>> > >> > >>>>>>>>> one or two winners in this area, and we could
> support
> > >>> them,
> > >>> > >> and
> > >>> > >> > >>>>>>>>> the
> > >>> > >> > >>>>>> rest
> > >>> > >> > >>>>>>>>> would go away. In reality, there are many
> variations.
> > >>> > >> > >>>>>>>>> Furthermore,
> > >>> > >> > >>>>>> many
> > >>> > >> > >>>>>>>>> people still prefer to just start their processors
> > like
> > >>> > normal
> > >>> > >> > >>>>>>>>> Java processes, and use traditional deployment
> scripts
> > >>> such
> > >>> > as
> > >>> > >> > >>>>>>>>> Fabric,
> > >>> > >> > >>>>>> Chef,
> > >>> > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users
> > makes
> > >>> the
> > >>> > >> > >>>>>>>>> Samza start-up process really painful for first time
> > >>> users.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit
> of
> > a
> > >>> > >> > >>>>>>>>> mis-fire
> > >>> > >> > >>>>>> because
> > >>> > >> > >>>>>>>>> of
> > >>> > >> > >>>>>>>>> a fundamental misunderstanding between the nature of
> > batch
> > >>> > >> jobs
> > >>> > >> > >>>>>>>>> and
> > >>> > >> > >>>>>>> stream
> > >>> > >> > >>>>>>>>> processing jobs. Early on, we made conscious effort
> to
> > >>> favor
> > >>> > >> > >>>>>>>>> the
> > >>> > >> > >>>>>> Hadoop
> > >>> > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked
> and
> > was
> > >>> > well
> > >>> > >> > >>>>>>> understood.
> > >>> > >> > >>>>>>>>> One thing that we missed was that batch jobs have a
> > >>> definite
> > >>> > >> > >>>>>> beginning,
> > >>> > >> > >>>>>>>>> and
> > >>> > >> > >>>>>>>>> end, and stream processing jobs don't (usually).
> This
> > >>> leads
> > >>> > to
> > >>> > >> > >>>>>>>>> a
> > >>> > >> > >>>>> much
> > >>> > >> > >>>>>>>>> simpler scheduling problem for stream processors.
> You
> > >>> > >> basically
> > >>> > >> > >>>>>>>>> just
> > >>> > >> > >>>>>>> need
> > >>> > >> > >>>>>>>>> to find a place to start the processor, and start
> it.
> > The
> > >>> > way
> > >>> > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a
> > cluster
> > >>> > >> > >>>>>>>>> being "full". We always
> > >>> > >> > >>>>>> add
> > >>> > >> > >>>>>>>>> more machines. The problem with coupling Samza with
> a
> > >>> > >> scheduler
> > >>> > >> > >>>>>>>>> is
> > >>> > >> > >>>>>> that
> > >>> > >> > >>>>>>>>> Samza (as a framework) now has to handle deployment.
> > This
> > >>> > >> pulls
> > >>> > >> > >>>>>>>>> in a
> > >>> > >> > >>>>>>> bunch
> > >>> > >> > >>>>>>>>> of things such as configuration distribution (config
> > >>> > stream),
> > >>> > >> > >>>>>>>>> shell
> > >>> > >> > >>>>>>> scrips
> > >>> > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> > >>> stuff),
> > >>> > >> etc.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Another reason for requiring dynamic deployment was
> to
> > >>> > support
> > >>> > >> > >>>>>>>>> data locality. If you want to have locality, you
> need
> > to
> > >>> put
> > >>> > >> > >>>>>>>>> your
> > >>> > >> > >>>>>> processors
> > >>> > >> > >>>>>>>>> close to the data they're processing. Upon further
> > >>> > >> > >>>>>>>>> investigation,
> > >>> > >> > >>>>>>> though,
> > >>> > >> > >>>>>>>>> this feature is not that beneficial. There is some
> > good
> > >>> > >> > >>>>>>>>> discussion
> > >>> > >> > >>>>>> about
> > >>> > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took
> the
> > >>> > >> > >>>>>>>>> Map/Reduce
> > >>> > >> > >>>>>> path,
> > >>> > >> > >>>>>>>>> but
> > >>> > >> > >>>>>>>>> there are some fundamental differences between HDFS
> > and
> > >>> > Kafka.
> > >>> > >> > >>>>>>>>> HDFS
> > >>> > >> > >>>>>> has
> > >>> > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to
> less
> > >>> > >> > >>>>>>>>> optimization potential with stream processors on top
> > of
> > >>> > Kafka.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't
> > have
> > >>> > any
> > >>> > >> > >>>>>>>>> built
> > >>> > >> > >>>>> in
> > >>> > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the
> > dynamic
> > >>> > >> > >>>>>>>>> deployment scheduling system to handle restarts
> when a
> > >>> > >> > >>>>>>>>> processor dies. This has
> > >>> > >> > >>>>>>> made
> > >>> > >> > >>>>>>>>> it very difficult to write a standalone Samza
> > container
> > >>> > >> > >>>> (SAMZA-516).
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Pluggability
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> In some cases pluggability is good, but I think that
> > we've
> > >>> > >> gone
> > >>> > >> > >>>>>>>>> too
> > >>> > >> > >>>>>> far
> > >>> > >> > >>>>>>>>> with it. Currently, Samza has:
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> * Pluggable config.
> > >>> > >> > >>>>>>>>> * Pluggable metrics.
> > >>> > >> > >>>>>>>>> * Pluggable deployment systems.
> > >>> > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> > >>> > SystemProducer,
> > >>> > >> > >>>> etc).
> > >>> > >> > >>>>>>>>> * Pluggable serdes.
> > >>> > >> > >>>>>>>>> * Pluggable storage engines.
> > >>> > >> > >>>>>>>>> * Pluggable strategies for just about every
> component
> > >>> > >> > >>>>> (MessageChooser,
> > >>> > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> There's probably more that I've forgotten, as well.
> > Some
> > >>> of
> > >>> > >> > >>>>>>>>> these
> > >>> > >> > >>>>> are
> > >>> > >> > >>>>>>>>> useful, but some have proven not to be. This all
> > comes at
> > >>> a
> > >>> > >> cost:
> > >>> > >> > >>>>>>>>> complexity. This complexity is making it harder for
> > our
> > >>> > users
> > >>> > >> > >>>>>>>>> to
> > >>> > >> > >>>>> pick
> > >>> > >> > >>>>>> up
> > >>> > >> > >>>>>>>>> and use Samza out of the box. It also makes it
> > difficult
> > >>> for
> > >>> > >> > >>>>>>>>> Samza developers to reason about what the
> > characteristics
> > >>> of
> > >>> > >> > >>>>>>>>> the container (since the characteristics change
> > depending
> > >>> on
> > >>> > >> > >>>>>>>>> which plugins are use).
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> The issues with pluggability are most visible in the
> > >>> System
> > >>> > >> APIs.
> > >>> > >> > >>>>> What
> > >>> > >> > >>>>>>>>> Samza really requires to be functional is Kafka as
> its
> > >>> > >> > >>>>>>>>> transport
> > >>> > >> > >>>>>> layer.
> > >>> > >> > >>>>>>>>> But
> > >>> > >> > >>>>>>>>> we've conflated two unrelated use cases into one
> API:
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > >>> > >> > >>>>>>>>> 2. Process the data in Kafka.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> The current System API supports both of these use
> > cases.
> > >>> The
> > >>> > >> > >>>>>>>>> problem
> > >>> > >> > >>>>>> is,
> > >>> > >> > >>>>>>>>> we
> > >>> > >> > >>>>>>>>> actually want different features for each use case.
> By
> > >>> > >> papering
> > >>> > >> > >>>>>>>>> over
> > >>> > >> > >>>>>>> these
> > >>> > >> > >>>>>>>>> two use cases, and providing a single API, we've
> > >>> introduced
> > >>> > a
> > >>> > >> > >>>>>>>>> ton of
> > >>> > >> > >>>>>>> leaky
> > >>> > >> > >>>>>>>>> abstractions.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> For example, what we'd really like in (2) is to have
> > >>> > >> > >>>>>>>>> monotonically increasing longs for offsets (like
> > Kafka).
> > >>> > This
> > >>> > >> > >>>>>>>>> would be at odds
> > >>> > >> > >>>>> with
> > >>> > >> > >>>>>>> (1),
> > >>> > >> > >>>>>>>>> though, since different systems have different
> > >>> > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > >>> > >> > >>>>>>>>> There was discussion both on the mailing list and
> the
> > SQL
> > >>> > >> JIRAs
> > >>> > >> > >>>>> about
> > >>> > >> > >>>>>>> the
> > >>> > >> > >>>>>>>>> need for this.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> The same thing holds true for replayability. Kafka
> > allows
> > >>> us
> > >>> > >> to
> > >>> > >> > >>>>> rewind
> > >>> > >> > >>>>>>>>> when
> > >>> > >> > >>>>>>>>> we have a failure. Many other systems don't. In some
> > >>> cases,
> > >>> > >> > >>>>>>>>> systems
> > >>> > >> > >>>>>>> return
> > >>> > >> > >>>>>>>>> null for their offsets (e.g.
> WikipediaSystemConsumer)
> > >>> > because
> > >>> > >> > >>>>>>>>> they
> > >>> > >> > >>>>>> have
> > >>> > >> > >>>>>>> no
> > >>> > >> > >>>>>>>>> offsets.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Partitioning is another example. Kafka supports
> > >>> > partitioning,
> > >>> > >> > >>>>>>>>> but
> > >>> > >> > >>>>> many
> > >>> > >> > >>>>>>>>> systems don't. We model this by having a single
> > partition
> > >>> > for
> > >>> > >> > >>>>>>>>> those systems. Still, other systems model
> partitioning
> > >>> > >> > >>>> differently (e.g.
> > >>> > >> > >>>>>>>>> Kinesis).
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating
> > streams
> > >>> > in
> > >>> > >> a
> > >>> > >> > >>>>>>>>> system-agnostic way is almost impossible. As is
> > modeling
> > >>> > >> > >>>>>>>>> metadata
> > >>> > >> > >>>>> for
> > >>> > >> > >>>>>>> the
> > >>> > >> > >>>>>>>>> system (replication factor, partitions, location,
> > etc).
> > >>> The
> > >>> > >> > >>>>>>>>> list
> > >>> > >> > >>>>> goes
> > >>> > >> > >>>>>>> on.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Duplicate work
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's
> > consumer
> > >>> > and
> > >>> > >> > >>>>> producer
> > >>> > >> > >>>>>>>>> APIs
> > >>> > >> > >>>>>>>>> had a relatively weak feature set. On the
> > consumer-side,
> > >>> you
> > >>> > >> > >>>>>>>>> had two
> > >>> > >> > >>>>>>>>> options: use the high level consumer, or the simple
> > >>> > consumer.
> > >>> > >> > >>>>>>>>> The
> > >>> > >> > >>>>>>> problem
> > >>> > >> > >>>>>>>>> with the high-level consumer was that it controlled
> > your
> > >>> > >> > >>>>>>>>> offsets, partition assignments, and the order in
> > which you
> > >>> > >> > >>>>>>>>> received messages. The
> > >>> > >> > >>>>> problem
> > >>> > >> > >>>>>>>>> with
> > >>> > >> > >>>>>>>>> the simple consumer is that it's not simple. It's
> > basic.
> > >>> You
> > >>> > >> > >>>>>>>>> end up
> > >>> > >> > >>>>>>> having
> > >>> > >> > >>>>>>>>> to handle a lot of really low-level stuff that you
> > >>> > shouldn't.
> > >>> > >> > >>>>>>>>> We
> > >>> > >> > >>>>>> spent a
> > >>> > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> > >>> robust.
> > >>> > >> It
> > >>> > >> > >>>>>>>>> also allows us to support some cool features:
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> * Per-partition message ordering and prioritization.
> > >>> > >> > >>>>>>>>> * Tight control over partition assignment to support
> > >>> joins,
> > >>> > >> > >>>>>>>>> global
> > >>> > >> > >>>>>> state
> > >>> > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > >>> > >> > >>>>>>>>> * Tight control over offset checkpointing.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> What we didn't realize at the time is that these
> > features
> > >>> > >> > >>>>>>>>> should
> > >>> > >> > >>>>>>> actually
> > >>> > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just
> Samza
> > >>> stream
> > >>> > >> > >>>>>> processors)
> > >>> > >> > >>>>>>>>> end up wanting to do things like joins and partition
> > >>> > >> > >>>>>>>>> assignment. The
> > >>> > >> > >>>>>>> Kafka
> > >>> > >> > >>>>>>>>> community has come to the same conclusion. They're
> > adding
> > >>> a
> > >>> > >> ton
> > >>> > >> > >>>>>>>>> of upgrades into their new Kafka consumer
> > implementation.
> > >>> > To a
> > >>> > >> > >>>>>>>>> large extent,
> > >>> > >> > >>>>> it's
> > >>> > >> > >>>>>>>>> duplicate work to what we've already done in Samza.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
> > >>> > approach
> > >>> > >> > >>>>>>>>> to
> > >>> > >> > >>>>>> Samza's
> > >>> > >> > >>>>>>>>> KafkaCheckpointManager implementation for handling
> > offset
> > >>> > >> > >>>>>> checkpointing.
> > >>> > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature
> > stores
> > >>> > >> offset
> > >>> > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them
> > from
> > >>> > the
> > >>> > >> > >>>>>>>>> broker.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> A lot of this seems like a waste, since we could
> have
> > >>> shared
> > >>> > >> > >>>>>>>>> the
> > >>> > >> > >>>>> work
> > >>> > >> > >>>>>> if
> > >>> > >> > >>>>>>>>> it
> > >>> > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Vision
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> All of this leads me to a rather radical proposal.
> > Samza
> > >>> is
> > >>> > >> > >>>>> relatively
> > >>> > >> > >>>>>>>>> stable at this point. I'd venture to say that we're
> > near a
> > >>> > 1.0
> > >>> > >> > >>>>>> release.
> > >>> > >> > >>>>>>>>> I'd
> > >>> > >> > >>>>>>>>> like to propose that we take what we've learned, and
> > begin
> > >>> > >> > >>>>>>>>> thinking
> > >>> > >> > >>>>>>> about
> > >>> > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were
> > starting
> > >>> > >> from
> > >>> > >> > >>>>>> scratch?
> > >>> > >> > >>>>>>>>> My
> > >>> > >> > >>>>>>>>> proposal is to:
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > >>> > >> > >>>>>>>>> processors, and eliminate all direct dependences on
> > YARN,
> > >>> > >> Mesos,
> > >>> > >> > >>>> etc.
> > >>> > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as
> the
> > >>> > stream
> > >>> > >> > >>>>>> processing
> > >>> > >> > >>>>>>>>> layer.
> > >>> > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
> serialization,
> > and
> > >>> > >> > >>>>>>>>> config
> > >>> > >> > >>>>>>> systems,
> > >>> > >> > >>>>>>>>> and simply use Kafka's instead.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> This would fix all of the issues that I outlined
> > above. It
> > >>> > >> > >>>>>>>>> should
> > >>> > >> > >>>>> also
> > >>> > >> > >>>>>>>>> shrink the Samza code base pretty dramatically.
> > Supporting
> > >>> > >> only
> > >>> > >> > >>>>>>>>> a standalone container will allow Samza to be
> > executed on
> > >>> > YARN
> > >>> > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or
> most
> > >>> other
> > >>> > >> > >>>>>>>>> in-house
> > >>> > >> > >>>>>>> deployment
> > >>> > >> > >>>>>>>>> systems. This should make life a lot easier for new
> > users.
> > >>> > >> > >>>>>>>>> Imagine
> > >>> > >> > >>>>>>> having
> > >>> > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in
> > mailing
> > >>> > >> list
> > >>> > >> > >>>>>> traffic
> > >>> > >> > >>>>>>>>> will be pretty dramatic.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The
> > reality
> > >>> > is,
> > >>> > >> > >>>>> everyone
> > >>> > >> > >>>>>>>>> that
> > >>> > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically
> > >>> require
> > >>> > >> it
> > >>> > >> > >>>>>> already
> > >>> > >> > >>>>>>> in
> > >>> > >> > >>>>>>>>> order for most features to work. Those that are
> using
> > >>> other
> > >>> > >> > >>>>>>>>> systems
> > >>> > >> > >>>>>> are
> > >>> > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and
> then
> > >>> they
> > >>> > do
> > >>> > >> > >>>>>>>>> the processing on top. There is already discussion (
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>
> > >>> > >> >
> > >>> >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >>> > >> > >>>>> 767
> > >>> > >> > >>>>>>>>> )
> > >>> > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely
> easy.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Once we make the call to couple with Kafka, we can
> > >>> leverage
> > >>> > a
> > >>> > >> > >>>>>>>>> ton of
> > >>> > >> > >>>>>>> their
> > >>> > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own
> > config,
> > >>> > >> > >>>>>>>>> metrics,
> > >>> > >> > >>>>> etc.
> > >>> > >> > >>>>>>> We
> > >>> > >> > >>>>>>>>> can all share the same libraries, and make them
> > better.
> > >>> This
> > >>> > >> > >>>>>>>>> will
> > >>> > >> > >>>>> also
> > >>> > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and
> > will let
> > >>> > us
> > >>> > >> > >>>>> leverage
> > >>> > >> > >>>>>>>>> their offset management and partition management,
> > rather
> > >>> > than
> > >>> > >> > >>>>>>>>> having
> > >>> > >> > >>>>>> our
> > >>> > >> > >>>>>>>>> own. All of the coordinator stream code would go
> > away, as
> > >>> > >> would
> > >>> > >> > >>>>>>>>> most
> > >>> > >> > >>>>>> of
> > >>> > >> > >>>>>>>>> the
> > >>> > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
> > >>> > partition
> > >>> > >> > >>>>>>> management
> > >>> > >> > >>>>>>>>> features into the Kafka broker, but they're already
> > moving
> > >>> > in
> > >>> > >> > >>>>>>>>> that direction with the new consumer API. The
> > features we
> > >>> > have
> > >>> > >> > >>>>>>>>> for
> > >>> > >> > >>>>>> partition
> > >>> > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like
> they
> > >>> should
> > >>> > >> be
> > >>> > >> > >>>>>>>>> in
> > >>> > >> > >>>>>> Kafka
> > >>> > >> > >>>>>>>>> anyway. There will always be some niche usages which
> > will
> > >>> > >> > >>>>>>>>> require
> > >>> > >> > >>>>>> extra
> > >>> > >> > >>>>>>>>> care and hence full control over partition
> assignments
> > >>> much
> > >>> > >> > >>>>>>>>> like the
> > >>> > >> > >>>>>>> Kafka
> > >>> > >> > >>>>>>>>> low level consumer api. These would continue to be
> > >>> > supported.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> These items will be good for the Samza community.
> > They'll
> > >>> > make
> > >>> > >> > >>>>>>>>> Samza easier to use, and make it easier for
> > developers to
> > >>> > add
> > >>> > >> > >>>>>>>>> new features.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat
> > backwards
> > >>> > >> > >>>>> incompatible
> > >>> > >> > >>>>>>>>> change). If we choose to go this route, it's
> important
> > >>> that
> > >>> > we
> > >>> > >> > >>>>> openly
> > >>> > >> > >>>>>>>>> communicate how we're going to provide a migration
> > path
> > >>> from
> > >>> > >> > >>>>>>>>> the
> > >>> > >> > >>>>>>> existing
> > >>> > >> > >>>>>>>>> APIs to the new ones (if we make incompatible
> > changes). I
> > >>> > >> think
> > >>> > >> > >>>>>>>>> at a minimum, we'd probably need to provide a
> wrapper
> > to
> > >>> > allow
> > >>> > >> > >>>>>>>>> existing StreamTask implementations to continue
> > running on
> > >>> > the
> > >>> > >> > >>>> new container.
> > >>> > >> > >>>>>>> It's
> > >>> > >> > >>>>>>>>> also important that we openly communicate about
> > timing,
> > >>> and
> > >>> > >> > >>>>>>>>> stages
> > >>> > >> > >>>>> of
> > >>> > >> > >>>>>>> the
> > >>> > >> > >>>>>>>>> migration.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> If you made it this far, I'm sure you have opinions.
> > :)
> > >>> > Please
> > >>> > >> > >>>>>>>>> send
> > >>> > >> > >>>>>> your
> > >>> > >> > >>>>>>>>> thoughts and feedback.
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>> Cheers,
> > >>> > >> > >>>>>>>>> Chris
> > >>> > >> > >>>>>>>>>
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>>
> > >>> > >> > >>>>>>>
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>> --
> > >>> > >> > >>>>>> -- Guozhang
> > >>> > >> > >>>>>>
> > >>> > >> > >>>>>
> > >>> > >> > >>>>
> > >>> > >> > >>
> > >>> > >> > >>
> > >>> > >> >
> > >>> > >> >
> > >>> > >> >
> > >>> > >>
> > >>> > >
> > >>> > >
> > >>> >
> > >>>
> >
>



-- 
Jordan Shaw
Full Stack Software Engineer
PubNub Inc
1045 17th St
San Francisco, CA 94107

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@gmail.com>.
Good summary Jakob.

WRT to the general purpose vs Kafka-specific, I actually see it slightly
differently. Consider how Storm works as an example, there is a data source
(spout) which could be Kafka, Database, etc, and then there is a transport
(a netty TCP thing iiuc). Storm allows you to process data from any source,
but when it comes from a source they always funnel it through their
transport to get to the tasks/bolts. It is natural to think of Kafka as the
Spout, but I think the better analogy is actually that Kafka is the
transport.

It is really hard to make the transport truly pluggable because this is
what the tasks interact with and you need to have guarantees about delivery
(and reprocessing), partitioning, atomicity of output, ordering, etc so
your stream processing can get the right answer. From my point of view what
this proposal says is that Kafka would be non-pluggable as the *transport*.

So in this proposal data would still come into and out of Kafka from a wide
variety of sources, but by requiring Kafka as the transport the interaction
with the tasks will always look the same (a persistent, partitioned, log).
So going back to the Storm analogy it is something like
  Spout interface = copy cat
  Bolt interface = samza

This does obviously make Samza dependent on Kafka but it doesn't mean you
wouldn't be processing data from all kinds of sources--indeed that is the
whole purpose. It just means that each of these data streams would be
available as a multi-subscriber Kafka topic to other systems, applications,
etc, not just for your job.

If you think about how things are now Samza already depends on a
partitioned, persistent, offset addressable log with log
compaction...which, unsurprisingly, so I don't think this is really a new
dependency.

Philosophically I think this makes sense too. To make a bunch of programs
fit together you have to standardize something. In this proposal what you
are standardizing around is really Kafka's protocol for streaming data and
your data format. The transformations that connect these streams can be
done via Samza, Storm, Spark, standalone java or python programs, etc but
the ultimate output and contract to the rest of the organization/world will
be the resulting Kafka topic. Philosophically I think this kind of data and
protocol based contract is the right way to go rather than saying that the
contract is a particular java api and the stream/data is what is pluggable.

-Jay



On Wed, Jul 8, 2015 at 11:03 AM, Jakob Homan <jg...@gmail.com> wrote:

> Rewinding back to the beginning of this topic, there are effectively
> three proposals on the table:
>
> 1) Chris' ideas for a direction towards a 2.0 release with an emphasis
> on API and configuration simplification.  This ideas are based on lots
> of lessons learned from the 0.x branch and are worthy of a 2.0 label
> and breaking backwards compability.  I'm not sure I agree with all of
> them, but they're definitely worth pursuing.
>
> 2) Chris' alternative proposal, which goes beyond his first and is
> essentially a reboot of Samza to a more limited, entirely
> Kafka-focused approach.  Samza would cease being a general purpose
> stream processing framework, akin to and an alternative to say, Apache
> Storm, and would instead become a standalone complement to the Kafka
> project.
>
> 3) Jay's proposal, which goes even further, and suggests that the
> Kafka community would be better served by adding stream processing as
> a module to Kafka.  This is a perfectly valid approach, but since it's
> entirely confined to the Kafka project, doesn't really involve Samza.
> If the Kafka team were to go this route, there would be no obligation
> on the Samza team to shut down, disband, etc.
>
> This last bit is important because Samza and Kafka, while closely
> linked, are distinct communities.  The intersection of committers on
> both Kafka and Samza is three people out of a combined 18 committers
> across both projects.   Samza is a distinct community that shares
> quite a few users with Kafka, but is able to chart its own course.
>
> My own view is that Samza has had an amazing year and is taking off at
> a rapid rate.  It was only proposed for Incubator two years ago and is
> still very young. The original team at LinkedIn has left that company
> but the project has continued to grow via contributions both from
> LinkedIn and from without.  We've recently seen a significant uptake
> in discussion and bug reports.
>
> The API, deployment and configuration changes Chris suggests are good
> ideas, but I think there is still serious value in having a
> stand-alone general stream processing framework that supports other
> input sources than Kafka.  We've already had contributions for adding
> producer support to ElasticSearch and HDFS.  As more users come on
> board, I would expect them to contribute more consumers and producers.
>
> It's a bit of chicken-and-the-egg problem; since the original team
> didn't have cycles to prioritize support for non-Kafka systems
> (kinesis, eventhub, twitter, flume, zeromq, etc.), Samza was less
> compelling than other stream processing frameworks that did have
> support and was therefore not used in those situations.  I'd love to
> see those added and the SystemConsumer/Producer APIs improved to
> fluently support them as well as Kafka.
> Martin had a question regarding the tight coupling between Hadoop HDFS
> and MapReduce (and YARN and Common).  This has been a problem for
> years and there have been several aborted attempts to split the
> projects out.  Each time there turned out to be a strong need for
> cross-cutting collaboration and so the effort was dropped.  Absent the
> third option above (Kafka adding stream support to itself directly), I
> would imagine something similar would play out here.
>
> We should get a feeling for which of the three proposals the Samza
> community is behind, technical details of each notwithstanding.  This
> would include not just the committers/PMC members, but also the users,
> contributors and lurkers.
>
> -Jakob
>
> On 8 July 2015 at 07:41, Ben Kirwin <be...@kirw.in> wrote:
> > Hi all,
> >
> > Interesting stuff! Jumping in a bit late, but here goes...
> >
> > I'd definitely be excited about a slimmed-down and more Kafka-specific
> > Samza -- you don't seem to lose much functionality that people
> > actually use, and the gains in simplicity / code sharing seem
> > potentially very large. (I've spent a bunch of time peeling back those
> > layers of abstraction to get eg. more control over message send order,
> > and working directly against Kafka's APIs would have been much
> > easier.) I also like the approach of letting Kafka code do the heavy
> > lifting and letting stream processing systems build on those -- good,
> > reusable implementations would be great for the whole
> > stream-processing ecosystem, and Samza in particular.
> >
> > On the other hand, I do hope that using Kafka's group membership /
> > partition assignment / etc. stays optional. As far as I can tell,
> > ~every major stream processing system that uses Kafka has chosen (or
> > switched to) 'static' partitioning, where each logical task consumes a
> > fixed set of partitions. When 'dynamic deploying' (a la Storm / Mesos
> > / Yarn) the underlying system is already doing failure detection and
> > transferring work between hosts when machines go down, so using
> > Kafka's implementation is redundant at best -- and at worst, the
> > interaction between the two systems can make outages worse.
> >
> > And thanks to Chris / Jay for getting this ball rolling. Exciting
> times...
> >
> > On Tue, Jul 7, 2015 at 2:35 PM, Jay Kreps <ja...@confluent.io> wrote:
> >> Hey Roger,
> >>
> >> I couldn't agree more. We spent a bunch of time talking to people and
> that
> >> is exactly the stuff we heard time and again. What makes it hard, of
> >> course, is that there is some tension between compatibility with what's
> >> there now and making things better for new users.
> >>
> >> I also strongly agree with the importance of multi-language support. We
> are
> >> talking now about Java, but for application development use cases people
> >> want to work in whatever language they are using elsewhere. I think
> moving
> >> to a model where Kafka itself does the group membership, lifecycle
> control,
> >> and partition assignment has the advantage of putting all that complex
> >> stuff behind a clean api that the clients are already going to be
> >> implementing for their consumer, so the added functionality for stream
> >> processing beyond a consumer becomes very minor.
> >>
> >> -Jay
> >>
> >> On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <ro...@gmail.com>
> >> wrote:
> >>
> >>> Metamorphosis...nice. :)
> >>>
> >>> This has been a great discussion.  As a user of Samza who's recently
> >>> integrated it into a relatively large organization, I just want to add
> >>> support to a few points already made.
> >>>
> >>> The biggest hurdles to adoption of Samza as it currently exists that
> I've
> >>> experienced are:
> >>> 1) YARN - YARN is overly complex in many environments where Puppet
> would do
> >>> just fine but it was the only mechanism to get fault tolerance.
> >>> 2) Configuration - I think I like the idea of configuring most of the
> job
> >>> in code rather than config files.  In general, I think the goal should
> be
> >>> to make it harder to make mistakes, especially of the kind where the
> code
> >>> expects something and the config doesn't match.  The current config is
> >>> quite intricate and error-prone.  For example, the application logic
> may
> >>> depend on bootstrapping a topic but rather than asserting that in the
> code,
> >>> you have to rely on getting the config right.  Likewise with serdes,
> the
> >>> Java representations produced by various serdes (JSON, Avro, etc.) are
> not
> >>> equivalent so you cannot just reconfigure a serde without changing the
> >>> code.   It would be nice for jobs to be able to assert what they expect
> >>> from their input topics in terms of partitioning.  This is getting a
> little
> >>> off topic but I was even thinking about creating a "Samza config
> linter"
> >>> that would sanity check a set of configs.  Especially in organizations
> >>> where config is managed by a different team than the application
> developer,
> >>> it's very hard to get avoid config mistakes.
> >>> 3) Java/Scala centric - for many teams (especially DevOps-type folks),
> the
> >>> pain of the Java toolchain (maven, slow builds, weak command line
> support,
> >>> configuration over convention) really inhibits productivity.  As more
> and
> >>> more high-quality clients become available for Kafka, I hope they'll
> follow
> >>> Samza's model.  Not sure how much it affects the proposals in this
> thread
> >>> but please consider other languages in the ecosystem as well.  From
> what
> >>> I've heard, Spark has more Python users than Java/Scala.
> >>> (FYI, we added a Jython wrapper for the Samza API
> >>>
> >>>
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> >>> and are working on a Yeoman generator
> >>> https://github.com/Quantiply/generator-rico for Jython/Samza projects
> to
> >>> alleviate some of the pain)
> >>>
> >>> I also want to underscore Jay's point about improving the user
> experience.
> >>> That's a very important factor for adoption.  I think the goal should
> be to
> >>> make Samza as easy to get started with as something like Logstash.
> >>> Logstash is vastly inferior in terms of capabilities to Samza but it's
> easy
> >>> to get started and that makes a big difference.
> >>>
> >>> Cheers,
> >>>
> >>> Roger
> >>>
> >>>
> >>>
> >>>
> >>>
> >>> On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
> >>> gdfm@apache.org> wrote:
> >>>
> >>> > Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
> >>> winner
> >>> > :)
> >>> >
> >>> > --
> >>> > Gianmarco
> >>> >
> >>> > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
> gdfm@apache.org
> >>> >
> >>> > wrote:
> >>> >
> >>> > > Hi,
> >>> > >
> >>> > > @Martin, thanks for you comments.
> >>> > > Maybe I'm missing some important point, but I think coupling the
> >>> releases
> >>> > > is actually a *good* thing.
> >>> > > To make an example, would it be better if the MR and HDFS
> components of
> >>> > > Hadoop had different release schedules?
> >>> > >
> >>> > > Actually, keeping the discussion in a single place would make
> agreeing
> >>> on
> >>> > > releases (and backwards compatibility) much easier, as everybody
> would
> >>> be
> >>> > > responsible for the whole codebase.
> >>> > >
> >>> > > That said, I like the idea of absorbing samza-core as a
> sub-project,
> >>> and
> >>> > > leave the fancy stuff separate.
> >>> > > It probably gives 90% of the benefits we have been discussing here.
> >>> > >
> >>> > > Cheers,
> >>> > >
> >>> > > --
> >>> > > Gianmarco
> >>> > >
> >>> > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
> >>> > >
> >>> > >> Hey Martin,
> >>> > >>
> >>> > >> I agree coupling release schedules is a downside.
> >>> > >>
> >>> > >> Definitely we can try to solve some of the integration problems in
> >>> > >> Confluent Platform or in other distributions. But I think this
> ends up
> >>> > >> being really shallow. I guess I feel to really get a good user
> >>> > experience
> >>> > >> the two systems have to kind of feel like part of the same thing
> and
> >>> you
> >>> > >> can't really add that in later--you can put both in the same
> >>> > downloadable
> >>> > >> tar file but it doesn't really give a very cohesive feeling. I
> agree
> >>> > that
> >>> > >> ultimately any of the project stuff is as much social and naming
> as
> >>> > >> anything else--theoretically two totally independent projects
> could
> >>> work
> >>> > >> to
> >>> > >> tightly align. In practice this seems to be quite difficult
> though.
> >>> > >>
> >>> > >> For the frameworks--totally agree it would be good to maintain the
> >>> > >> framework support with the project. In some cases there may not
> be too
> >>> > >> much
> >>> > >> there since the integration gets lighter but I think whatever
> stubs
> >>> you
> >>> > >> need should be included. So no I definitely wasn't trying to imply
> >>> > >> dropping
> >>> > >> support for these frameworks, just making the integration lighter
> by
> >>> > >> separating process management from partition management.
> >>> > >>
> >>> > >> You raise two good points we would have to figure out if we went
> down
> >>> > the
> >>> > >> alignment path:
> >>> > >> 1. With respect to the name, yeah I think the first question is
> >>> whether
> >>> > >> some "re-branding" would be worth it. If so then I think we can
> have a
> >>> > big
> >>> > >> thread on the name. I'm definitely not set on Kafka Streaming or
> Kafka
> >>> > >> Streams I was just using them to be kind of illustrative. I agree
> with
> >>> > >> your
> >>> > >> critique of these names, though I think people would get the idea.
> >>> > >> 2. Yeah you also raise a good point about how to "factor" it.
> Here are
> >>> > the
> >>> > >> options I see (I could get enthusiastic about any of them):
> >>> > >>    a. One repo for both Kafka and Samza
> >>> > >>    b. Two repos, retaining the current seperation
> >>> > >>    c. Two repos, the equivalent of samza-api and samza-core is
> >>> absorbed
> >>> > >> almost like a third client
> >>> > >>
> >>> > >> Cheers,
> >>> > >>
> >>> > >> -Jay
> >>> > >>
> >>> > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> >>> martin@kleppmann.com>
> >>> > >> wrote:
> >>> > >>
> >>> > >> > Ok, thanks for the clarifications. Just a few follow-up
> comments.
> >>> > >> >
> >>> > >> > - I see the appeal of merging with Kafka or becoming a
> subproject:
> >>> the
> >>> > >> > reasons you mention are good. The risk I see is that release
> >>> schedules
> >>> > >> > become coupled to each other, which can slow everyone down, and
> >>> large
> >>> > >> > projects with many contributors are harder to manage. (Jakob,
> can
> >>> you
> >>> > >> speak
> >>> > >> > from experience, having seen a wider range of Hadoop ecosystem
> >>> > >> projects?)
> >>> > >> >
> >>> > >> > Some of the goals of a better unified developer experience could
> >>> also
> >>> > be
> >>> > >> > solved by integrating Samza nicely into a Kafka distribution
> (such
> >>> as
> >>> > >> > Confluent's). I'm not against merging projects if we decide
> that's
> >>> the
> >>> > >> way
> >>> > >> > to go, just pointing out the same goals can perhaps also be
> achieved
> >>> > in
> >>> > >> > other ways.
> >>> > >> >
> >>> > >> > - With regard to dropping the YARN dependency: are you proposing
> >>> that
> >>> > >> > Samza doesn't give any help to people wanting to run on
> >>> > >> YARN/Mesos/AWS/etc?
> >>> > >> > So the docs would basically have a link to Slider and nothing
> else?
> >>> Or
> >>> > >> > would we maintain integrations with a bunch of popular
> deployment
> >>> > >> methods
> >>> > >> > (e.g. the necessary glue and shell scripts to make Samza work
> with
> >>> > >> Slider)?
> >>> > >> >
> >>> > >> > I absolutely think it's a good idea to have the "as a library"
> and
> >>> > "as a
> >>> > >> > process" (using Yi's taxonomy) options for people who want them,
> >>> but I
> >>> > >> > think there should also be a low-friction path for common "as a
> >>> > service"
> >>> > >> > deployment methods, for which we probably need to maintain
> >>> > integrations.
> >>> > >> >
> >>> > >> > - Project naming: "Kafka Streams" seems odd to me, because
> Kafka is
> >>> > all
> >>> > >> > about streams already. Perhaps "Kafka Transformers" or "Kafka
> >>> Filters"
> >>> > >> > would be more apt?
> >>> > >> >
> >>> > >> > One suggestion: perhaps the core of Samza (stream transformation
> >>> with
> >>> > >> > state management -- i.e. the "Samza as a library" bit) could
> become
> >>> > >> part of
> >>> > >> > Kafka, while higher-level tools such as streaming SQL and
> >>> integrations
> >>> > >> with
> >>> > >> > deployment frameworks remain in a separate project? In other
> words,
> >>> > >> Kafka
> >>> > >> > would absorb the proven, stable core of Samza, which would
> become
> >>> the
> >>> > >> > "third Kafka client" mentioned early in this thread. The Samza
> >>> project
> >>> > >> > would then target that third Kafka client as its base API, and
> the
> >>> > >> project
> >>> > >> > would be freed up to explore more experimental new horizons.
> >>> > >> >
> >>> > >> > Martin
> >>> > >> >
> >>> > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
> >>> > >> >
> >>> > >> > > Hey Martin,
> >>> > >> > >
> >>> > >> > > For the YARN/Mesos/etc decoupling I actually don't think it
> ties
> >>> our
> >>> > >> > hands
> >>> > >> > > at all, all it does is refactor things. The division of
> >>> > >> responsibility is
> >>> > >> > > that Samza core is responsible for task lifecycle, state, and
> >>> > >> partition
> >>> > >> > > management (using the Kafka co-ordinator) but it is NOT
> >>> responsible
> >>> > >> for
> >>> > >> > > packaging, configuration deployment or execution of
> processes. The
> >>> > >> > problem
> >>> > >> > > of packaging and starting these processes is
> >>> > >> > > framework/environment-specific. This leaves individual
> frameworks
> >>> to
> >>> > >> be
> >>> > >> > as
> >>> > >> > > fancy or vanilla as they like. So you can get simple stateless
> >>> > >> support in
> >>> > >> > > YARN, Mesos, etc using their off-the-shelf app framework
> (Slider,
> >>> > >> > Marathon,
> >>> > >> > > etc). These are well known by people and have nice UIs and a
> lot
> >>> of
> >>> > >> > > flexibility. I don't think they have node affinity as a built
> in
> >>> > >> option
> >>> > >> > > (though I could be wrong). So if we want that we can either
> wait
> >>> for
> >>> > >> them
> >>> > >> > > to add it or do a custom framework to add that feature (as
> now).
> >>> > >> > Obviously
> >>> > >> > > if you manage things with old-school ops tools
> (puppet/chef/etc)
> >>> you
> >>> > >> get
> >>> > >> > > locality easily. The nice thing, though, is that all the samza
> >>> > >> "business
> >>> > >> > > logic" around partition management and fault tolerance is in
> Samza
> >>> > >> core
> >>> > >> > so
> >>> > >> > > it is shared across frameworks and the framework specific bit
> is
> >>> > just
> >>> > >> > > whether it is smart enough to try to get the same host when a
> job
> >>> is
> >>> > >> > > restarted.
> >>> > >> > >
> >>> > >> > > With respect to the Kafka-alignment, yeah I think the goal
> would
> >>> be
> >>> > >> (a)
> >>> > >> > > actually get better alignment in user experience, and (b)
> express
> >>> > >> this in
> >>> > >> > > the naming and project branding. Specifically:
> >>> > >> > > 1. Website/docs, it would be nice for the "transformation"
> api to
> >>> be
> >>> > >> > > discoverable in the main Kafka docs--i.e. be able to explain
> when
> >>> to
> >>> > >> use
> >>> > >> > > the consumer and when to use the stream processing
> functionality
> >>> and
> >>> > >> lead
> >>> > >> > > people into that experience.
> >>> > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever)
> that
> >>> has
> >>> > >> both
> >>> > >> > > Kafka and the stream processing part and they actually work
> >>> > together.
> >>> > >> > > 3. Unify the programming experience so the client and Samza
> api
> >>> > share
> >>> > >> > > config/monitoring/naming/packaging/etc.
> >>> > >> > >
> >>> > >> > > I think sub-projects keep separate committers and can have a
> >>> > separate
> >>> > >> > repo,
> >>> > >> > > but I'm actually not really sure (I can't find a definition
> of a
> >>> > >> > subproject
> >>> > >> > > in Apache).
> >>> > >> > >
> >>> > >> > > Basically at a high-level you want the experience to "feel"
> like a
> >>> > >> single
> >>> > >> > > system, not to relatively independent things that are kind of
> >>> > >> awkwardly
> >>> > >> > > glued together.
> >>> > >> > >
> >>> > >> > > I think if we did that they having naming or branding like
> "kafka
> >>> > >> > > streaming" or "kafka streams" or something like that would
> >>> actually
> >>> > >> do a
> >>> > >> > > good job of conveying what it is. I do that this would help
> >>> adoption
> >>> > >> > quite
> >>> > >> > > a lot as it would correctly convey that using Kafka Streaming
> with
> >>> > >> Kafka
> >>> > >> > is
> >>> > >> > > a fairly seamless experience and Kafka is pretty heavily
> adopted
> >>> at
> >>> > >> this
> >>> > >> > > point.
> >>> > >> > >
> >>> > >> > > Fwiw we actually considered this model originally when open
> >>> sourcing
> >>> > >> > Samza,
> >>> > >> > > however at that time Kafka was relatively unknown and we
> decided
> >>> not
> >>> > >> to
> >>> > >> > do
> >>> > >> > > it since we felt it would be limiting. From my point of view
> the
> >>> > three
> >>> > >> > > things have changed (1) Kafka is now really heavily used for
> >>> stream
> >>> > >> > > processing, (2) we learned that abstracting out the stream
> well is
> >>> > >> > > basically impossible, (3) we learned it is really hard to
> keep the
> >>> > two
> >>> > >> > > things feeling like a single product.
> >>> > >> > >
> >>> > >> > > -Jay
> >>> > >> > >
> >>> > >> > >
> >>> > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> >>> > >> martin@kleppmann.com>
> >>> > >> > > wrote:
> >>> > >> > >
> >>> > >> > >> Hi all,
> >>> > >> > >>
> >>> > >> > >> Lots of good thoughts here.
> >>> > >> > >>
> >>> > >> > >> I agree with the general philosophy of tying Samza more
> firmly to
> >>> > >> Kafka.
> >>> > >> > >> After I spent a while looking at integrating other message
> >>> brokers
> >>> > >> (e.g.
> >>> > >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
> >>> > >> > SystemConsumer
> >>> > >> > >> tacitly assumes a model so much like Kafka's that pretty much
> >>> > nobody
> >>> > >> but
> >>> > >> > >> Kafka actually implements it. (Databus is perhaps an
> exception,
> >>> but
> >>> > >> it
> >>> > >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza
> fully
> >>> > >> > dependent
> >>> > >> > >> on Kafka acknowledges that the system-independence was never
> as
> >>> > real
> >>> > >> as
> >>> > >> > we
> >>> > >> > >> perhaps made it out to be. The gains of code reuse are real.
> >>> > >> > >>
> >>> > >> > >> The idea of decoupling Samza from YARN has also always been
> >>> > >> appealing to
> >>> > >> > >> me, for various reasons already mentioned in this thread.
> >>> Although
> >>> > >> > making
> >>> > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems
> >>> > >> laudable,
> >>> > >> > I am
> >>> > >> > >> a little concerned that it will restrict us to a lowest
> common
> >>> > >> > denominator.
> >>> > >> > >> For example, would host affinity (SAMZA-617) still be
> possible?
> >>> For
> >>> > >> jobs
> >>> > >> > >> with large amounts of state, I think SAMZA-617 would be a big
> >>> boon,
> >>> > >> > since
> >>> > >> > >> restoring state off the changelog on every single restart is
> >>> > painful,
> >>> > >> > due
> >>> > >> > >> to long recovery times. It would be a shame if the decoupling
> >>> from
> >>> > >> YARN
> >>> > >> > >> made host affinity impossible.
> >>> > >> > >>
> >>> > >> > >> Jay, a question about the proposed API for instantiating a
> job in
> >>> > >> code
> >>> > >> > >> (rather than a properties file): when submitting a job to a
> >>> > cluster,
> >>> > >> is
> >>> > >> > the
> >>> > >> > >> idea that the instantiation code runs on a client somewhere,
> >>> which
> >>> > >> then
> >>> > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does
> that
> >>> > >> code
> >>> > >> > run
> >>> > >> > >> on each container that is part of the job (in which case, how
> >>> does
> >>> > >> the
> >>> > >> > job
> >>> > >> > >> submission to the cluster work)?
> >>> > >> > >>
> >>> > >> > >> I agree with Garry that it doesn't feel right to make a 1.0
> >>> release
> >>> > >> > with a
> >>> > >> > >> plan for it to be immediately obsolete. So if this is going
> to
> >>> > >> happen, I
> >>> > >> > >> think it would be more honest to stick with 0.* version
> numbers
> >>> > until
> >>> > >> > the
> >>> > >> > >> library-ified Samza has been implemented, is stable and
> widely
> >>> > used.
> >>> > >> > >>
> >>> > >> > >> Should the new Samza be a subproject of Kafka? There is
> precedent
> >>> > for
> >>> > >> > >> tight coupling between different Apache projects (e.g.
> Curator
> >>> and
> >>> > >> > >> Zookeeper, or Slider and YARN), so I think remaining separate
> >>> would
> >>> > >> be
> >>> > >> > ok.
> >>> > >> > >> Even if Samza is fully dependent on Kafka, there is enough
> >>> > substance
> >>> > >> in
> >>> > >> > >> Samza that it warrants being a separate project. An argument
> in
> >>> > >> favour
> >>> > >> > of
> >>> > >> > >> merging would be if we think Kafka has a much stronger "brand
> >>> > >> presence"
> >>> > >> > >> than Samza; I'm ambivalent on that one. If the Kafka project
> is
> >>> > >> willing
> >>> > >> > to
> >>> > >> > >> endorse Samza as the "official" way of doing stateful stream
> >>> > >> > >> transformations, that would probably have much the same
> effect as
> >>> > >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike.
> Close
> >>> > >> > >> collaboration between the two projects will be needed in any
> >>> case.
> >>> > >> > >>
> >>> > >> > >> From a project management perspective, I guess the "new
> Samza"
> >>> > would
> >>> > >> > have
> >>> > >> > >> to be developed on a branch alongside ongoing maintenance of
> the
> >>> > >> current
> >>> > >> > >> line of development? I think it would be important to
> continue
> >>> > >> > supporting
> >>> > >> > >> existing users, and provide a graceful migration path to the
> new
> >>> > >> > version.
> >>> > >> > >> Leaving the current versions unsupported and forcing people
> to
> >>> > >> rewrite
> >>> > >> > >> their jobs would send a bad signal.
> >>> > >> > >>
> >>> > >> > >> Best,
> >>> > >> > >> Martin
> >>> > >> > >>
> >>> > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
> >>> > >> > >>
> >>> > >> > >>> Hey Garry,
> >>> > >> > >>>
> >>> > >> > >>> Yeah that's super frustrating. I'd be happy to chat more
> about
> >>> > this
> >>> > >> if
> >>> > >> > >>> you'd be interested. I think Chris and I started with the
> idea
> >>> of
> >>> > >> "what
> >>> > >> > >>> would it take to make Samza a kick-ass ingestion tool" but
> >>> > >> ultimately
> >>> > >> > we
> >>> > >> > >>> kind of came around to the idea that ingestion and
> >>> transformation
> >>> > >> had
> >>> > >> > >>> pretty different needs and coupling the two made things
> hard.
> >>> > >> > >>>
> >>> > >> > >>> For what it's worth I think copycat (KIP-26) actually will
> do
> >>> what
> >>> > >> you
> >>> > >> > >> are
> >>> > >> > >>> looking for.
> >>> > >> > >>>
> >>> > >> > >>> With regard to your point about slider, I don't necessarily
> >>> > >> disagree.
> >>> > >> > >> But I
> >>> > >> > >>> think getting good YARN support is quite doable and I think
> we
> >>> can
> >>> > >> make
> >>> > >> > >>> that work well. I think the issue this proposal solves is
> that
> >>> > >> > >> technically
> >>> > >> > >>> it is pretty hard to support multiple cluster management
> systems
> >>> > the
> >>> > >> > way
> >>> > >> > >>> things are now, you need to write an "app master" or
> "framework"
> >>> > for
> >>> > >> > each
> >>> > >> > >>> and they are all a little different so testing is really
> hard.
> >>> In
> >>> > >> the
> >>> > >> > >>> absence of this we have been stuck with just YARN which has
> >>> > >> fantastic
> >>> > >> > >>> penetration in the Hadoopy part of the org, but zero
> penetration
> >>> > >> > >> elsewhere.
> >>> > >> > >>> Given the huge amount of work being put in to slider,
> marathon,
> >>> > aws
> >>> > >> > >>> tooling, not to mention the umpteen related packaging
> >>> technologies
> >>> > >> > people
> >>> > >> > >>> want to use (Docker, Kubernetes, various cloud-specific
> deploy
> >>> > >> tools,
> >>> > >> > >> etc)
> >>> > >> > >>> I really think it is important to get this right.
> >>> > >> > >>>
> >>> > >> > >>> -Jay
> >>> > >> > >>>
> >>> > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> >>> > >> > >>> g.turkington@improvedigital.com> wrote:
> >>> > >> > >>>
> >>> > >> > >>>> Hi all,
> >>> > >> > >>>>
> >>> > >> > >>>> I think the question below re does Samza become a
> sub-project
> >>> of
> >>> > >> Kafka
> >>> > >> > >>>> highlights the broader point around migration. Chris
> mentions
> >>> > >> Samza's
> >>> > >> > >>>> maturity is heading towards a v1 release but I'm not sure
> it
> >>> > feels
> >>> > >> > >> right to
> >>> > >> > >>>> launch a v1 then immediately plan to deprecate most of it.
> >>> > >> > >>>>
> >>> > >> > >>>> From a selfish perspective I have some guys who have
> started
> >>> > >> working
> >>> > >> > >> with
> >>> > >> > >>>> Samza and building some new consumers/producers was next
> up.
> >>> > Sounds
> >>> > >> > like
> >>> > >> > >>>> that is absolutely not the direction to go. I need to look
> into
> >>> > the
> >>> > >> > KIP
> >>> > >> > >> in
> >>> > >> > >>>> more detail but for me the attractiveness of adding new
> Samza
> >>> > >> > >>>> consumer/producers -- even if yes all they were doing was
> >>> really
> >>> > >> > getting
> >>> > >> > >>>> data into and out of Kafka --  was to avoid  having to
> worry
> >>> > about
> >>> > >> the
> >>> > >> > >>>> lifecycle management of external clients. If there is a
> generic
> >>> > >> Kafka
> >>> > >> > >>>> ingress/egress layer that I can plug a new connector into
> and
> >>> > have
> >>> > >> a
> >>> > >> > >> lot of
> >>> > >> > >>>> the heavy lifting re scale and reliability done for me
> then it
> >>> > >> gives
> >>> > >> > me
> >>> > >> > >> all
> >>> > >> > >>>> the pushing new consumers/producers would. If not then it
> >>> > >> complicates
> >>> > >> > my
> >>> > >> > >>>> operational deployments.
> >>> > >> > >>>>
> >>> > >> > >>>> Which is similar to my other question with the proposal --
> if
> >>> we
> >>> > >> > build a
> >>> > >> > >>>> fully available/stand-alone Samza plus the requisite shims
> to
> >>> > >> > integrate
> >>> > >> > >>>> with Slider etc I suspect the former may be a lot more work
> >>> than
> >>> > we
> >>> > >> > >> think.
> >>> > >> > >>>> We may make it much easier for a newcomer to get something
> >>> > running
> >>> > >> but
> >>> > >> > >>>> having them step up and get a reliable production
> deployment
> >>> may
> >>> > >> still
> >>> > >> > >>>> dominate mailing list  traffic, if for different reasons
> than
> >>> > >> today.
> >>> > >> > >>>>
> >>> > >> > >>>> Don't get me wrong -- I'm comfortable with making the Samza
> >>> > >> dependency
> >>> > >> > >> on
> >>> > >> > >>>> Kafka much more explicit and I absolutely see the
> benefits  in
> >>> > the
> >>> > >> > >>>> reduction of duplication and clashing
> >>> terminologies/abstractions
> >>> > >> that
> >>> > >> > >>>> Chris/Jay describe. Samza as a library would likely be a
> very
> >>> > nice
> >>> > >> > tool
> >>> > >> > >> to
> >>> > >> > >>>> add to the Kafka ecosystem. I just have the concerns above
> re
> >>> the
> >>> > >> > >>>> operational side.
> >>> > >> > >>>>
> >>> > >> > >>>> Garry
> >>> > >> > >>>>
> >>> > >> > >>>> -----Original Message-----
> >>> > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
> gdfm@apache.org]
> >>> > >> > >>>> Sent: 02 July 2015 12:56
> >>> > >> > >>>> To: dev@samza.apache.org
> >>> > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> >>> > >> > >>>>
> >>> > >> > >>>> Very interesting thoughts.
> >>> > >> > >>>> From outside, I have always perceived Samza as a computing
> >>> layer
> >>> > >> over
> >>> > >> > >>>> Kafka.
> >>> > >> > >>>>
> >>> > >> > >>>> The question, maybe a bit provocative, is "should Samza be
> a
> >>> > >> > sub-project
> >>> > >> > >>>> of Kafka then?"
> >>> > >> > >>>> Or does it make sense to keep it as a separate project
> with a
> >>> > >> separate
> >>> > >> > >>>> governance?
> >>> > >> > >>>>
> >>> > >> > >>>> Cheers,
> >>> > >> > >>>>
> >>> > >> > >>>> --
> >>> > >> > >>>> Gianmarco
> >>> > >> > >>>>
> >>> > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> >>> wrote:
> >>> > >> > >>>>
> >>> > >> > >>>>> Overall, I agree to couple with Kafka more tightly.
> Because
> >>> > Samza
> >>> > >> de
> >>> > >> > >>>>> facto is based on Kafka, and it should leverage what Kafka
> >>> has.
> >>> > At
> >>> > >> > the
> >>> > >> > >>>>> same time, Kafka does not need to reinvent what Samza
> already
> >>> > >> has. I
> >>> > >> > >>>>> also like the idea of separating the ingestion and
> >>> > transformation.
> >>> > >> > >>>>>
> >>> > >> > >>>>> But it is a little difficult for me to image how the Samza
> >>> will
> >>> > >> look
> >>> > >> > >>>> like.
> >>> > >> > >>>>> And I feel Chris and Jay have a little difference in
> terms of
> >>> > how
> >>> > >> > >>>>> Samza should look like.
> >>> > >> > >>>>>
> >>> > >> > >>>>> *** Will it look like what Jay's code shows (A client of
> >>> Kakfa)
> >>> > ?
> >>> > >> And
> >>> > >> > >>>>> user's application code calls this client?
> >>> > >> > >>>>>
> >>> > >> > >>>>> 1. If we make Samza be a library of Kafka (like what the
> code
> >>> > >> shows),
> >>> > >> > >>>>> how do we implement auto-balance and fault-tolerance? Are
> they
> >>> > >> taken
> >>> > >> > >>>>> care by the Kafka broker or other mechanism, such as
> "Samza
> >>> > >> worker"
> >>> > >> > >>>>> (just make up the name) ?
> >>> > >> > >>>>>
> >>> > >> > >>>>> 2. What about other features, such as auto-scaling, shared
> >>> > state,
> >>> > >> > >>>>> monitoring?
> >>> > >> > >>>>>
> >>> > >> > >>>>>
> >>> > >> > >>>>> *** If we have Samza standalone, (is this what Chris
> >>> suggests?)
> >>> > >> > >>>>>
> >>> > >> > >>>>> 1. we still need to ingest data from Kakfa and produce to
> it.
> >>> > >> Then it
> >>> > >> > >>>>> becomes the same as what Samza looks like now, except it
> does
> >>> > not
> >>> > >> > rely
> >>> > >> > >>>>> on Yarn anymore.
> >>> > >> > >>>>>
> >>> > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's
> metrics,
> >>> > logs,
> >>> > >> > >>>>> etc? Use Kafka code as the dependency?
> >>> > >> > >>>>>
> >>> > >> > >>>>>
> >>> > >> > >>>>> Thanks,
> >>> > >> > >>>>>
> >>> > >> > >>>>> Fang, Yan
> >>> > >> > >>>>> yanfang724@gmail.com
> >>> > >> > >>>>>
> >>> > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> >>> > wangguoz@gmail.com
> >>> > >> >
> >>> > >> > >>>> wrote:
> >>> > >> > >>>>>
> >>> > >> > >>>>>> Read through the code example and it looks good to me. A
> few
> >>> > >> > >>>>>> thoughts regarding deployment:
> >>> > >> > >>>>>>
> >>> > >> > >>>>>> Today Samza deploys as executable runnable like:
> >>> > >> > >>>>>>
> >>> > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> >>> > >> > >>>> --config-path=file://...
> >>> > >> > >>>>>>
> >>> > >> > >>>>>> And this proposal advocate for deploying Samza more as
> >>> embedded
> >>> > >> > >>>>>> libraries in user application code (ignoring the
> terminology
> >>> > >> since
> >>> > >> > >>>>>> it is not the
> >>> > >> > >>>>> same
> >>> > >> > >>>>>> as the prototype code):
> >>> > >> > >>>>>>
> >>> > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread
> thread =
> >>> > new
> >>> > >> > >>>>>> Thread(task); thread.start();
> >>> > >> > >>>>>>
> >>> > >> > >>>>>> I think both of these deployment modes are important for
> >>> > >> different
> >>> > >> > >>>>>> types
> >>> > >> > >>>>> of
> >>> > >> > >>>>>> users. That said, I think making Samza purely standalone
> is
> >>> > still
> >>> > >> > >>>>>> sufficient for either runnable or library modes.
> >>> > >> > >>>>>>
> >>> > >> > >>>>>> Guozhang
> >>> > >> > >>>>>>
> >>> > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> >>> jay@confluent.io>
> >>> > >> > wrote:
> >>> > >> > >>>>>>
> >>> > >> > >>>>>>> Looks like gmail mangled the code example, it was
> supposed
> >>> to
> >>> > >> look
> >>> > >> > >>>>>>> like
> >>> > >> > >>>>>>> this:
> >>> > >> > >>>>>>>
> >>> > >> > >>>>>>> Properties props = new Properties();
> >>> > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
> >>> > >> StreamingConfig
> >>> > >> > >>>>>>> config = new StreamingConfig(props);
> >>> > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> >>> > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> >>> > >> > >>>>>>> config.serialization(new StringSerializer(), new
> >>> > >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
> >>> > >> > >>>>>>> KafkaStreaming(config); container.run();
> >>> > >> > >>>>>>>
> >>> > >> > >>>>>>> -Jay
> >>> > >> > >>>>>>>
> >>> > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> >>> jay@confluent.io
> >>> > >
> >>> > >> > >>>> wrote:
> >>> > >> > >>>>>>>
> >>> > >> > >>>>>>>> Hey guys,
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> This came out of some conversations Chris and I were
> having
> >>> > >> > >>>>>>>> around
> >>> > >> > >>>>>>> whether
> >>> > >> > >>>>>>>> it would make sense to use Samza as a kind of data
> >>> ingestion
> >>> > >> > >>>>> framework
> >>> > >> > >>>>>>> for
> >>> > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This
> >>> kind
> >>> > of
> >>> > >> > >>>>>> combined
> >>> > >> > >>>>>>>> with complaints around config and YARN and the
> discussion
> >>> > >> around
> >>> > >> > >>>>>>>> how
> >>> > >> > >>>>> to
> >>> > >> > >>>>>>>> best do a standalone mode.
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> So the thought experiment was, given that Samza was
> >>> basically
> >>> > >> > >>>>>>>> already totally Kafka specific, what if you just
> embraced
> >>> > that
> >>> > >> > >>>>>>>> and turned it
> >>> > >> > >>>>>> into
> >>> > >> > >>>>>>>> something less like a heavyweight framework and more
> like a
> >>> > >> > >>>>>>>> third
> >>> > >> > >>>>> Kafka
> >>> > >> > >>>>>>>> client--a kind of "producing consumer" with state
> >>> management
> >>> > >> > >>>>>> facilities.
> >>> > >> > >>>>>>>> Basically a library. Instead of a complex stream
> processing
> >>> > >> > >>>>>>>> framework
> >>> > >> > >>>>>>> this
> >>> > >> > >>>>>>>> would actually be a very simple thing, not much more
> >>> > >> complicated
> >>> > >> > >>>>>>>> to
> >>> > >> > >>>>> use
> >>> > >> > >>>>>>> or
> >>> > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we thought
> >>> about
> >>> > >> it
> >>> > >> > >>>>>>>> a
> >>> > >> > >>>>> lot
> >>> > >> > >>>>>> of
> >>> > >> > >>>>>>>> what Samza (and the other stream processing systems
> were
> >>> > doing)
> >>> > >> > >>>>> seemed
> >>> > >> > >>>>>>> like
> >>> > >> > >>>>>>>> kind of a hangover from MapReduce.
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> Of course you need to ingest/output data to and from
> the
> >>> > stream
> >>> > >> > >>>>>>>> processing. But when we actually looked into how that
> would
> >>> > >> > >>>>>>>> work,
> >>> > >> > >>>>> Samza
> >>> > >> > >>>>>>>> isn't really an ideal data ingestion framework for a
> bunch
> >>> of
> >>> > >> > >>>>> reasons.
> >>> > >> > >>>>>> To
> >>> > >> > >>>>>>>> really do that right you need a pretty different
> internal
> >>> > data
> >>> > >> > >>>>>>>> model
> >>> > >> > >>>>>> and
> >>> > >> > >>>>>>>> set of apis. So what if you split them and had an api
> for
> >>> > Kafka
> >>> > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api
> for
> >>> > >> Kafka
> >>> > >> > >>>>>>>> transformation (Samza).
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> This would also allow really embracing the same
> terminology
> >>> > and
> >>> > >> > >>>>>>>> conventions. One complaint about the current state is
> that
> >>> > the
> >>> > >> > >>>>>>>> two
> >>> > >> > >>>>>>> systems
> >>> > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs
> >>> "topic"
> >>> > >> and
> >>> > >> > >>>>>>> different
> >>> > >> > >>>>>>>> config and monitoring systems means you kind of have to
> >>> learn
> >>> > >> > >>>>>>>> Kafka's
> >>> > >> > >>>>>>> way,
> >>> > >> > >>>>>>>> then learn Samza's slightly different way, then kind of
> >>> > >> > >>>>>>>> understand
> >>> > >> > >>>>> how
> >>> > >> > >>>>>>> they
> >>> > >> > >>>>>>>> map to each other, which having walked a few people
> through
> >>> > >> this
> >>> > >> > >>>>>>>> is surprisingly tricky for folks to get.
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> Since I have been spending a lot of time on airplanes I
> >>> > hacked
> >>> > >> > >>>>>>>> up an ernest but still somewhat incomplete prototype of
> >>> what
> >>> > >> > >>>>>>>> this would
> >>> > >> > >>>>> look
> >>> > >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka
> as it
> >>> > >> > >>>>>>>> required a
> >>> > >> > >>>>>> few
> >>> > >> > >>>>>>>> changes to the new consumer. Here is the code:
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>
> >>> > >> > >>>>>>
> >>> > >> > >>>>>
> >>> > >> >
> >>> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >>> > >> > >>>>> /apache/kafka/clients/streaming
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> For the purpose of the prototype I just liberally
> renamed
> >>> > >> > >>>>>>>> everything
> >>> > >> > >>>>> to
> >>> > >> > >>>>>>>> try to align it with Kafka with no regard for
> >>> compatibility.
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> To use this would be something like this:
> >>> > >> > >>>>>>>> Properties props = new Properties();
> >>> > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> >>> > >> > >>>>>>>> StreamingConfig config = new
> >>> > >> > >>>>> StreamingConfig(props);
> >>> > >> > >>>>>>> config.subscribe("test-topic-1",
> >>> > >> > >>>>>>>> "test-topic-2");
> >>> > >> config.processor(ExampleStreamProcessor.class);
> >>> > >> > >>>>>>> config.serialization(new
> >>> > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
> >>> KafkaStreaming
> >>> > >> > >>>>>> container =
> >>> > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> >>> > StreamProcessor
> >>> > >> > >>>>>>>> is basically StreamTask.
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> So rather than putting all the class names in a file
> and
> >>> then
> >>> > >> > >>>>>>>> having
> >>> > >> > >>>>>> the
> >>> > >> > >>>>>>>> job assembled by reflection, you just instantiate the
> >>> > container
> >>> > >> > >>>>>>>> programmatically. Work is balanced over however many
> >>> > instances
> >>> > >> > >>>>>>>> of
> >>> > >> > >>>>> this
> >>> > >> > >>>>>>> are
> >>> > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new tasks
> are
> >>> > >> added
> >>> > >> > >>>>>>>> to
> >>> > >> > >>>>> the
> >>> > >> > >>>>>>>> existing containers without shutting them down).
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> We would provide some glue for running this stuff in
> YARN
> >>> via
> >>> > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their
> >>> tools
> >>> > >> > >>>>>>>> but from the
> >>> > >> > >>>>>> point
> >>> > >> > >>>>>>> of
> >>> > >> > >>>>>>>> view of these frameworks these stream processing jobs
> are
> >>> > just
> >>> > >> > >>>>>> stateless
> >>> > >> > >>>>>>>> services that can come and go and expand and contract
> at
> >>> > will.
> >>> > >> > >>>>>>>> There
> >>> > >> > >>>>> is
> >>> > >> > >>>>>>> no
> >>> > >> > >>>>>>>> more custom scheduler.
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> Here are some relevant details:
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get
> larger if
> >>> we
> >>> > >> > >>>>>>>>  productionized but not vastly larger. We really do
> get a
> >>> ton
> >>> > >> > >>>>>>>> of
> >>> > >> > >>>>>>> leverage
> >>> > >> > >>>>>>>>  out of Kafka.
> >>> > >> > >>>>>>>>  2. Partition management is fully delegated to the new
> >>> > >> consumer.
> >>> > >> > >>>>> This
> >>> > >> > >>>>>>>>  is nice since now any partition management strategy
> >>> > available
> >>> > >> > >>>>>>>> to
> >>> > >> > >>>>>> Kafka
> >>> > >> > >>>>>>>>  consumer is also available to Samza (and vice versa)
> and
> >>> > with
> >>> > >> > >>>>>>>> the
> >>> > >> > >>>>>>> exact
> >>> > >> > >>>>>>>>  same configs.
> >>> > >> > >>>>>>>>  3. It supports state as well as state reuse
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> -Jay
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> >>> > >> > >>>>>> criccomini@apache.org>
> >>> > >> > >>>>>>>> wrote:
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>>> Hey all,
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> I have had some discussions with Samza engineers at
> >>> LinkedIn
> >>> > >> > >>>>>>>>> and
> >>> > >> > >>>>>>> Confluent
> >>> > >> > >>>>>>>>> and we came up with a few observations and would like
> to
> >>> > >> > >>>>>>>>> propose
> >>> > >> > >>>>> some
> >>> > >> > >>>>>>>>> changes.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> We've observed some things that I want to call out
> about
> >>> > >> > >>>>>>>>> Samza's
> >>> > >> > >>>>>> design,
> >>> > >> > >>>>>>>>> and I'd like to propose some changes.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
> >>> > >> > >>>>>>>>> * Samza is too pluggable.
> >>> > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> >>> consumer
> >>> > >> > >>>>>>>>> APIs
> >>> > >> > >>>>> are
> >>> > >> > >>>>>>>>> trying to solve a lot of the same problems.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> All three of these issues are related, but I'll
> address
> >>> them
> >>> > >> in
> >>> > >> > >>>>> order.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Deployment
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic
> deployment
> >>> > >> > >>>>>>>>> scheduler
> >>> > >> > >>>>>> such
> >>> > >> > >>>>>>>>> as
> >>> > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we
> bet
> >>> that
> >>> > >> > >>>>>>>>> there
> >>> > >> > >>>>>> would
> >>> > >> > >>>>>>>>> be
> >>> > >> > >>>>>>>>> one or two winners in this area, and we could support
> >>> them,
> >>> > >> and
> >>> > >> > >>>>>>>>> the
> >>> > >> > >>>>>> rest
> >>> > >> > >>>>>>>>> would go away. In reality, there are many variations.
> >>> > >> > >>>>>>>>> Furthermore,
> >>> > >> > >>>>>> many
> >>> > >> > >>>>>>>>> people still prefer to just start their processors
> like
> >>> > normal
> >>> > >> > >>>>>>>>> Java processes, and use traditional deployment scripts
> >>> such
> >>> > as
> >>> > >> > >>>>>>>>> Fabric,
> >>> > >> > >>>>>> Chef,
> >>> > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users
> makes
> >>> the
> >>> > >> > >>>>>>>>> Samza start-up process really painful for first time
> >>> users.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit of
> a
> >>> > >> > >>>>>>>>> mis-fire
> >>> > >> > >>>>>> because
> >>> > >> > >>>>>>>>> of
> >>> > >> > >>>>>>>>> a fundamental misunderstanding between the nature of
> batch
> >>> > >> jobs
> >>> > >> > >>>>>>>>> and
> >>> > >> > >>>>>>> stream
> >>> > >> > >>>>>>>>> processing jobs. Early on, we made conscious effort to
> >>> favor
> >>> > >> > >>>>>>>>> the
> >>> > >> > >>>>>> Hadoop
> >>> > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked and
> was
> >>> > well
> >>> > >> > >>>>>>> understood.
> >>> > >> > >>>>>>>>> One thing that we missed was that batch jobs have a
> >>> definite
> >>> > >> > >>>>>> beginning,
> >>> > >> > >>>>>>>>> and
> >>> > >> > >>>>>>>>> end, and stream processing jobs don't (usually). This
> >>> leads
> >>> > to
> >>> > >> > >>>>>>>>> a
> >>> > >> > >>>>> much
> >>> > >> > >>>>>>>>> simpler scheduling problem for stream processors. You
> >>> > >> basically
> >>> > >> > >>>>>>>>> just
> >>> > >> > >>>>>>> need
> >>> > >> > >>>>>>>>> to find a place to start the processor, and start it.
> The
> >>> > way
> >>> > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a
> cluster
> >>> > >> > >>>>>>>>> being "full". We always
> >>> > >> > >>>>>> add
> >>> > >> > >>>>>>>>> more machines. The problem with coupling Samza with a
> >>> > >> scheduler
> >>> > >> > >>>>>>>>> is
> >>> > >> > >>>>>> that
> >>> > >> > >>>>>>>>> Samza (as a framework) now has to handle deployment.
> This
> >>> > >> pulls
> >>> > >> > >>>>>>>>> in a
> >>> > >> > >>>>>>> bunch
> >>> > >> > >>>>>>>>> of things such as configuration distribution (config
> >>> > stream),
> >>> > >> > >>>>>>>>> shell
> >>> > >> > >>>>>>> scrips
> >>> > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> >>> stuff),
> >>> > >> etc.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Another reason for requiring dynamic deployment was to
> >>> > support
> >>> > >> > >>>>>>>>> data locality. If you want to have locality, you need
> to
> >>> put
> >>> > >> > >>>>>>>>> your
> >>> > >> > >>>>>> processors
> >>> > >> > >>>>>>>>> close to the data they're processing. Upon further
> >>> > >> > >>>>>>>>> investigation,
> >>> > >> > >>>>>>> though,
> >>> > >> > >>>>>>>>> this feature is not that beneficial. There is some
> good
> >>> > >> > >>>>>>>>> discussion
> >>> > >> > >>>>>> about
> >>> > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
> >>> > >> > >>>>>>>>> Map/Reduce
> >>> > >> > >>>>>> path,
> >>> > >> > >>>>>>>>> but
> >>> > >> > >>>>>>>>> there are some fundamental differences between HDFS
> and
> >>> > Kafka.
> >>> > >> > >>>>>>>>> HDFS
> >>> > >> > >>>>>> has
> >>> > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to less
> >>> > >> > >>>>>>>>> optimization potential with stream processors on top
> of
> >>> > Kafka.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't
> have
> >>> > any
> >>> > >> > >>>>>>>>> built
> >>> > >> > >>>>> in
> >>> > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the
> dynamic
> >>> > >> > >>>>>>>>> deployment scheduling system to handle restarts when a
> >>> > >> > >>>>>>>>> processor dies. This has
> >>> > >> > >>>>>>> made
> >>> > >> > >>>>>>>>> it very difficult to write a standalone Samza
> container
> >>> > >> > >>>> (SAMZA-516).
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Pluggability
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> In some cases pluggability is good, but I think that
> we've
> >>> > >> gone
> >>> > >> > >>>>>>>>> too
> >>> > >> > >>>>>> far
> >>> > >> > >>>>>>>>> with it. Currently, Samza has:
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> * Pluggable config.
> >>> > >> > >>>>>>>>> * Pluggable metrics.
> >>> > >> > >>>>>>>>> * Pluggable deployment systems.
> >>> > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> >>> > SystemProducer,
> >>> > >> > >>>> etc).
> >>> > >> > >>>>>>>>> * Pluggable serdes.
> >>> > >> > >>>>>>>>> * Pluggable storage engines.
> >>> > >> > >>>>>>>>> * Pluggable strategies for just about every component
> >>> > >> > >>>>> (MessageChooser,
> >>> > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> There's probably more that I've forgotten, as well.
> Some
> >>> of
> >>> > >> > >>>>>>>>> these
> >>> > >> > >>>>> are
> >>> > >> > >>>>>>>>> useful, but some have proven not to be. This all
> comes at
> >>> a
> >>> > >> cost:
> >>> > >> > >>>>>>>>> complexity. This complexity is making it harder for
> our
> >>> > users
> >>> > >> > >>>>>>>>> to
> >>> > >> > >>>>> pick
> >>> > >> > >>>>>> up
> >>> > >> > >>>>>>>>> and use Samza out of the box. It also makes it
> difficult
> >>> for
> >>> > >> > >>>>>>>>> Samza developers to reason about what the
> characteristics
> >>> of
> >>> > >> > >>>>>>>>> the container (since the characteristics change
> depending
> >>> on
> >>> > >> > >>>>>>>>> which plugins are use).
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> The issues with pluggability are most visible in the
> >>> System
> >>> > >> APIs.
> >>> > >> > >>>>> What
> >>> > >> > >>>>>>>>> Samza really requires to be functional is Kafka as its
> >>> > >> > >>>>>>>>> transport
> >>> > >> > >>>>>> layer.
> >>> > >> > >>>>>>>>> But
> >>> > >> > >>>>>>>>> we've conflated two unrelated use cases into one API:
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> >>> > >> > >>>>>>>>> 2. Process the data in Kafka.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> The current System API supports both of these use
> cases.
> >>> The
> >>> > >> > >>>>>>>>> problem
> >>> > >> > >>>>>> is,
> >>> > >> > >>>>>>>>> we
> >>> > >> > >>>>>>>>> actually want different features for each use case. By
> >>> > >> papering
> >>> > >> > >>>>>>>>> over
> >>> > >> > >>>>>>> these
> >>> > >> > >>>>>>>>> two use cases, and providing a single API, we've
> >>> introduced
> >>> > a
> >>> > >> > >>>>>>>>> ton of
> >>> > >> > >>>>>>> leaky
> >>> > >> > >>>>>>>>> abstractions.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> For example, what we'd really like in (2) is to have
> >>> > >> > >>>>>>>>> monotonically increasing longs for offsets (like
> Kafka).
> >>> > This
> >>> > >> > >>>>>>>>> would be at odds
> >>> > >> > >>>>> with
> >>> > >> > >>>>>>> (1),
> >>> > >> > >>>>>>>>> though, since different systems have different
> >>> > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> >>> > >> > >>>>>>>>> There was discussion both on the mailing list and the
> SQL
> >>> > >> JIRAs
> >>> > >> > >>>>> about
> >>> > >> > >>>>>>> the
> >>> > >> > >>>>>>>>> need for this.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> The same thing holds true for replayability. Kafka
> allows
> >>> us
> >>> > >> to
> >>> > >> > >>>>> rewind
> >>> > >> > >>>>>>>>> when
> >>> > >> > >>>>>>>>> we have a failure. Many other systems don't. In some
> >>> cases,
> >>> > >> > >>>>>>>>> systems
> >>> > >> > >>>>>>> return
> >>> > >> > >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> >>> > because
> >>> > >> > >>>>>>>>> they
> >>> > >> > >>>>>> have
> >>> > >> > >>>>>>> no
> >>> > >> > >>>>>>>>> offsets.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Partitioning is another example. Kafka supports
> >>> > partitioning,
> >>> > >> > >>>>>>>>> but
> >>> > >> > >>>>> many
> >>> > >> > >>>>>>>>> systems don't. We model this by having a single
> partition
> >>> > for
> >>> > >> > >>>>>>>>> those systems. Still, other systems model partitioning
> >>> > >> > >>>> differently (e.g.
> >>> > >> > >>>>>>>>> Kinesis).
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating
> streams
> >>> > in
> >>> > >> a
> >>> > >> > >>>>>>>>> system-agnostic way is almost impossible. As is
> modeling
> >>> > >> > >>>>>>>>> metadata
> >>> > >> > >>>>> for
> >>> > >> > >>>>>>> the
> >>> > >> > >>>>>>>>> system (replication factor, partitions, location,
> etc).
> >>> The
> >>> > >> > >>>>>>>>> list
> >>> > >> > >>>>> goes
> >>> > >> > >>>>>>> on.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Duplicate work
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's
> consumer
> >>> > and
> >>> > >> > >>>>> producer
> >>> > >> > >>>>>>>>> APIs
> >>> > >> > >>>>>>>>> had a relatively weak feature set. On the
> consumer-side,
> >>> you
> >>> > >> > >>>>>>>>> had two
> >>> > >> > >>>>>>>>> options: use the high level consumer, or the simple
> >>> > consumer.
> >>> > >> > >>>>>>>>> The
> >>> > >> > >>>>>>> problem
> >>> > >> > >>>>>>>>> with the high-level consumer was that it controlled
> your
> >>> > >> > >>>>>>>>> offsets, partition assignments, and the order in
> which you
> >>> > >> > >>>>>>>>> received messages. The
> >>> > >> > >>>>> problem
> >>> > >> > >>>>>>>>> with
> >>> > >> > >>>>>>>>> the simple consumer is that it's not simple. It's
> basic.
> >>> You
> >>> > >> > >>>>>>>>> end up
> >>> > >> > >>>>>>> having
> >>> > >> > >>>>>>>>> to handle a lot of really low-level stuff that you
> >>> > shouldn't.
> >>> > >> > >>>>>>>>> We
> >>> > >> > >>>>>> spent a
> >>> > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> >>> robust.
> >>> > >> It
> >>> > >> > >>>>>>>>> also allows us to support some cool features:
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> * Per-partition message ordering and prioritization.
> >>> > >> > >>>>>>>>> * Tight control over partition assignment to support
> >>> joins,
> >>> > >> > >>>>>>>>> global
> >>> > >> > >>>>>> state
> >>> > >> > >>>>>>>>> (if we want to implement it :)), etc.
> >>> > >> > >>>>>>>>> * Tight control over offset checkpointing.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> What we didn't realize at the time is that these
> features
> >>> > >> > >>>>>>>>> should
> >>> > >> > >>>>>>> actually
> >>> > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
> >>> stream
> >>> > >> > >>>>>> processors)
> >>> > >> > >>>>>>>>> end up wanting to do things like joins and partition
> >>> > >> > >>>>>>>>> assignment. The
> >>> > >> > >>>>>>> Kafka
> >>> > >> > >>>>>>>>> community has come to the same conclusion. They're
> adding
> >>> a
> >>> > >> ton
> >>> > >> > >>>>>>>>> of upgrades into their new Kafka consumer
> implementation.
> >>> > To a
> >>> > >> > >>>>>>>>> large extent,
> >>> > >> > >>>>> it's
> >>> > >> > >>>>>>>>> duplicate work to what we've already done in Samza.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
> >>> > approach
> >>> > >> > >>>>>>>>> to
> >>> > >> > >>>>>> Samza's
> >>> > >> > >>>>>>>>> KafkaCheckpointManager implementation for handling
> offset
> >>> > >> > >>>>>> checkpointing.
> >>> > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature
> stores
> >>> > >> offset
> >>> > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them
> from
> >>> > the
> >>> > >> > >>>>>>>>> broker.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> A lot of this seems like a waste, since we could have
> >>> shared
> >>> > >> > >>>>>>>>> the
> >>> > >> > >>>>> work
> >>> > >> > >>>>>> if
> >>> > >> > >>>>>>>>> it
> >>> > >> > >>>>>>>>> had been done in Kafka from the get-go.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Vision
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> All of this leads me to a rather radical proposal.
> Samza
> >>> is
> >>> > >> > >>>>> relatively
> >>> > >> > >>>>>>>>> stable at this point. I'd venture to say that we're
> near a
> >>> > 1.0
> >>> > >> > >>>>>> release.
> >>> > >> > >>>>>>>>> I'd
> >>> > >> > >>>>>>>>> like to propose that we take what we've learned, and
> begin
> >>> > >> > >>>>>>>>> thinking
> >>> > >> > >>>>>>> about
> >>> > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were
> starting
> >>> > >> from
> >>> > >> > >>>>>> scratch?
> >>> > >> > >>>>>>>>> My
> >>> > >> > >>>>>>>>> proposal is to:
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> >>> > >> > >>>>>>>>> processors, and eliminate all direct dependences on
> YARN,
> >>> > >> Mesos,
> >>> > >> > >>>> etc.
> >>> > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as the
> >>> > stream
> >>> > >> > >>>>>> processing
> >>> > >> > >>>>>>>>> layer.
> >>> > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization,
> and
> >>> > >> > >>>>>>>>> config
> >>> > >> > >>>>>>> systems,
> >>> > >> > >>>>>>>>> and simply use Kafka's instead.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> This would fix all of the issues that I outlined
> above. It
> >>> > >> > >>>>>>>>> should
> >>> > >> > >>>>> also
> >>> > >> > >>>>>>>>> shrink the Samza code base pretty dramatically.
> Supporting
> >>> > >> only
> >>> > >> > >>>>>>>>> a standalone container will allow Samza to be
> executed on
> >>> > YARN
> >>> > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most
> >>> other
> >>> > >> > >>>>>>>>> in-house
> >>> > >> > >>>>>>> deployment
> >>> > >> > >>>>>>>>> systems. This should make life a lot easier for new
> users.
> >>> > >> > >>>>>>>>> Imagine
> >>> > >> > >>>>>>> having
> >>> > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in
> mailing
> >>> > >> list
> >>> > >> > >>>>>> traffic
> >>> > >> > >>>>>>>>> will be pretty dramatic.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The
> reality
> >>> > is,
> >>> > >> > >>>>> everyone
> >>> > >> > >>>>>>>>> that
> >>> > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically
> >>> require
> >>> > >> it
> >>> > >> > >>>>>> already
> >>> > >> > >>>>>>> in
> >>> > >> > >>>>>>>>> order for most features to work. Those that are using
> >>> other
> >>> > >> > >>>>>>>>> systems
> >>> > >> > >>>>>> are
> >>> > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and then
> >>> they
> >>> > do
> >>> > >> > >>>>>>>>> the processing on top. There is already discussion (
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>
> >>> > >> > >>>>>>
> >>> > >> > >>>>>
> >>> > >> >
> >>> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >>> > >> > >>>>> 767
> >>> > >> > >>>>>>>>> )
> >>> > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Once we make the call to couple with Kafka, we can
> >>> leverage
> >>> > a
> >>> > >> > >>>>>>>>> ton of
> >>> > >> > >>>>>>> their
> >>> > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own
> config,
> >>> > >> > >>>>>>>>> metrics,
> >>> > >> > >>>>> etc.
> >>> > >> > >>>>>>> We
> >>> > >> > >>>>>>>>> can all share the same libraries, and make them
> better.
> >>> This
> >>> > >> > >>>>>>>>> will
> >>> > >> > >>>>> also
> >>> > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and
> will let
> >>> > us
> >>> > >> > >>>>> leverage
> >>> > >> > >>>>>>>>> their offset management and partition management,
> rather
> >>> > than
> >>> > >> > >>>>>>>>> having
> >>> > >> > >>>>>> our
> >>> > >> > >>>>>>>>> own. All of the coordinator stream code would go
> away, as
> >>> > >> would
> >>> > >> > >>>>>>>>> most
> >>> > >> > >>>>>> of
> >>> > >> > >>>>>>>>> the
> >>> > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
> >>> > partition
> >>> > >> > >>>>>>> management
> >>> > >> > >>>>>>>>> features into the Kafka broker, but they're already
> moving
> >>> > in
> >>> > >> > >>>>>>>>> that direction with the new consumer API. The
> features we
> >>> > have
> >>> > >> > >>>>>>>>> for
> >>> > >> > >>>>>> partition
> >>> > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like they
> >>> should
> >>> > >> be
> >>> > >> > >>>>>>>>> in
> >>> > >> > >>>>>> Kafka
> >>> > >> > >>>>>>>>> anyway. There will always be some niche usages which
> will
> >>> > >> > >>>>>>>>> require
> >>> > >> > >>>>>> extra
> >>> > >> > >>>>>>>>> care and hence full control over partition assignments
> >>> much
> >>> > >> > >>>>>>>>> like the
> >>> > >> > >>>>>>> Kafka
> >>> > >> > >>>>>>>>> low level consumer api. These would continue to be
> >>> > supported.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> These items will be good for the Samza community.
> They'll
> >>> > make
> >>> > >> > >>>>>>>>> Samza easier to use, and make it easier for
> developers to
> >>> > add
> >>> > >> > >>>>>>>>> new features.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat
> backwards
> >>> > >> > >>>>> incompatible
> >>> > >> > >>>>>>>>> change). If we choose to go this route, it's important
> >>> that
> >>> > we
> >>> > >> > >>>>> openly
> >>> > >> > >>>>>>>>> communicate how we're going to provide a migration
> path
> >>> from
> >>> > >> > >>>>>>>>> the
> >>> > >> > >>>>>>> existing
> >>> > >> > >>>>>>>>> APIs to the new ones (if we make incompatible
> changes). I
> >>> > >> think
> >>> > >> > >>>>>>>>> at a minimum, we'd probably need to provide a wrapper
> to
> >>> > allow
> >>> > >> > >>>>>>>>> existing StreamTask implementations to continue
> running on
> >>> > the
> >>> > >> > >>>> new container.
> >>> > >> > >>>>>>> It's
> >>> > >> > >>>>>>>>> also important that we openly communicate about
> timing,
> >>> and
> >>> > >> > >>>>>>>>> stages
> >>> > >> > >>>>> of
> >>> > >> > >>>>>>> the
> >>> > >> > >>>>>>>>> migration.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> If you made it this far, I'm sure you have opinions.
> :)
> >>> > Please
> >>> > >> > >>>>>>>>> send
> >>> > >> > >>>>>> your
> >>> > >> > >>>>>>>>> thoughts and feedback.
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>> Cheers,
> >>> > >> > >>>>>>>>> Chris
> >>> > >> > >>>>>>>>>
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>>
> >>> > >> > >>>>>>>
> >>> > >> > >>>>>>
> >>> > >> > >>>>>>
> >>> > >> > >>>>>>
> >>> > >> > >>>>>> --
> >>> > >> > >>>>>> -- Guozhang
> >>> > >> > >>>>>>
> >>> > >> > >>>>>
> >>> > >> > >>>>
> >>> > >> > >>
> >>> > >> > >>
> >>> > >> >
> >>> > >> >
> >>> > >> >
> >>> > >>
> >>> > >
> >>> > >
> >>> >
> >>>
>

Re: Thoughts and obesrvations on Samza

Posted by Jakob Homan <jg...@gmail.com>.
Rewinding back to the beginning of this topic, there are effectively
three proposals on the table:

1) Chris' ideas for a direction towards a 2.0 release with an emphasis
on API and configuration simplification.  This ideas are based on lots
of lessons learned from the 0.x branch and are worthy of a 2.0 label
and breaking backwards compability.  I'm not sure I agree with all of
them, but they're definitely worth pursuing.

2) Chris' alternative proposal, which goes beyond his first and is
essentially a reboot of Samza to a more limited, entirely
Kafka-focused approach.  Samza would cease being a general purpose
stream processing framework, akin to and an alternative to say, Apache
Storm, and would instead become a standalone complement to the Kafka
project.

3) Jay's proposal, which goes even further, and suggests that the
Kafka community would be better served by adding stream processing as
a module to Kafka.  This is a perfectly valid approach, but since it's
entirely confined to the Kafka project, doesn't really involve Samza.
If the Kafka team were to go this route, there would be no obligation
on the Samza team to shut down, disband, etc.

This last bit is important because Samza and Kafka, while closely
linked, are distinct communities.  The intersection of committers on
both Kafka and Samza is three people out of a combined 18 committers
across both projects.   Samza is a distinct community that shares
quite a few users with Kafka, but is able to chart its own course.

My own view is that Samza has had an amazing year and is taking off at
a rapid rate.  It was only proposed for Incubator two years ago and is
still very young. The original team at LinkedIn has left that company
but the project has continued to grow via contributions both from
LinkedIn and from without.  We've recently seen a significant uptake
in discussion and bug reports.

The API, deployment and configuration changes Chris suggests are good
ideas, but I think there is still serious value in having a
stand-alone general stream processing framework that supports other
input sources than Kafka.  We've already had contributions for adding
producer support to ElasticSearch and HDFS.  As more users come on
board, I would expect them to contribute more consumers and producers.

It's a bit of chicken-and-the-egg problem; since the original team
didn't have cycles to prioritize support for non-Kafka systems
(kinesis, eventhub, twitter, flume, zeromq, etc.), Samza was less
compelling than other stream processing frameworks that did have
support and was therefore not used in those situations.  I'd love to
see those added and the SystemConsumer/Producer APIs improved to
fluently support them as well as Kafka.
Martin had a question regarding the tight coupling between Hadoop HDFS
and MapReduce (and YARN and Common).  This has been a problem for
years and there have been several aborted attempts to split the
projects out.  Each time there turned out to be a strong need for
cross-cutting collaboration and so the effort was dropped.  Absent the
third option above (Kafka adding stream support to itself directly), I
would imagine something similar would play out here.

We should get a feeling for which of the three proposals the Samza
community is behind, technical details of each notwithstanding.  This
would include not just the committers/PMC members, but also the users,
contributors and lurkers.

-Jakob

On 8 July 2015 at 07:41, Ben Kirwin <be...@kirw.in> wrote:
> Hi all,
>
> Interesting stuff! Jumping in a bit late, but here goes...
>
> I'd definitely be excited about a slimmed-down and more Kafka-specific
> Samza -- you don't seem to lose much functionality that people
> actually use, and the gains in simplicity / code sharing seem
> potentially very large. (I've spent a bunch of time peeling back those
> layers of abstraction to get eg. more control over message send order,
> and working directly against Kafka's APIs would have been much
> easier.) I also like the approach of letting Kafka code do the heavy
> lifting and letting stream processing systems build on those -- good,
> reusable implementations would be great for the whole
> stream-processing ecosystem, and Samza in particular.
>
> On the other hand, I do hope that using Kafka's group membership /
> partition assignment / etc. stays optional. As far as I can tell,
> ~every major stream processing system that uses Kafka has chosen (or
> switched to) 'static' partitioning, where each logical task consumes a
> fixed set of partitions. When 'dynamic deploying' (a la Storm / Mesos
> / Yarn) the underlying system is already doing failure detection and
> transferring work between hosts when machines go down, so using
> Kafka's implementation is redundant at best -- and at worst, the
> interaction between the two systems can make outages worse.
>
> And thanks to Chris / Jay for getting this ball rolling. Exciting times...
>
> On Tue, Jul 7, 2015 at 2:35 PM, Jay Kreps <ja...@confluent.io> wrote:
>> Hey Roger,
>>
>> I couldn't agree more. We spent a bunch of time talking to people and that
>> is exactly the stuff we heard time and again. What makes it hard, of
>> course, is that there is some tension between compatibility with what's
>> there now and making things better for new users.
>>
>> I also strongly agree with the importance of multi-language support. We are
>> talking now about Java, but for application development use cases people
>> want to work in whatever language they are using elsewhere. I think moving
>> to a model where Kafka itself does the group membership, lifecycle control,
>> and partition assignment has the advantage of putting all that complex
>> stuff behind a clean api that the clients are already going to be
>> implementing for their consumer, so the added functionality for stream
>> processing beyond a consumer becomes very minor.
>>
>> -Jay
>>
>> On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <ro...@gmail.com>
>> wrote:
>>
>>> Metamorphosis...nice. :)
>>>
>>> This has been a great discussion.  As a user of Samza who's recently
>>> integrated it into a relatively large organization, I just want to add
>>> support to a few points already made.
>>>
>>> The biggest hurdles to adoption of Samza as it currently exists that I've
>>> experienced are:
>>> 1) YARN - YARN is overly complex in many environments where Puppet would do
>>> just fine but it was the only mechanism to get fault tolerance.
>>> 2) Configuration - I think I like the idea of configuring most of the job
>>> in code rather than config files.  In general, I think the goal should be
>>> to make it harder to make mistakes, especially of the kind where the code
>>> expects something and the config doesn't match.  The current config is
>>> quite intricate and error-prone.  For example, the application logic may
>>> depend on bootstrapping a topic but rather than asserting that in the code,
>>> you have to rely on getting the config right.  Likewise with serdes, the
>>> Java representations produced by various serdes (JSON, Avro, etc.) are not
>>> equivalent so you cannot just reconfigure a serde without changing the
>>> code.   It would be nice for jobs to be able to assert what they expect
>>> from their input topics in terms of partitioning.  This is getting a little
>>> off topic but I was even thinking about creating a "Samza config linter"
>>> that would sanity check a set of configs.  Especially in organizations
>>> where config is managed by a different team than the application developer,
>>> it's very hard to get avoid config mistakes.
>>> 3) Java/Scala centric - for many teams (especially DevOps-type folks), the
>>> pain of the Java toolchain (maven, slow builds, weak command line support,
>>> configuration over convention) really inhibits productivity.  As more and
>>> more high-quality clients become available for Kafka, I hope they'll follow
>>> Samza's model.  Not sure how much it affects the proposals in this thread
>>> but please consider other languages in the ecosystem as well.  From what
>>> I've heard, Spark has more Python users than Java/Scala.
>>> (FYI, we added a Jython wrapper for the Samza API
>>>
>>> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
>>> and are working on a Yeoman generator
>>> https://github.com/Quantiply/generator-rico for Jython/Samza projects to
>>> alleviate some of the pain)
>>>
>>> I also want to underscore Jay's point about improving the user experience.
>>> That's a very important factor for adoption.  I think the goal should be to
>>> make Samza as easy to get started with as something like Logstash.
>>> Logstash is vastly inferior in terms of capabilities to Samza but it's easy
>>> to get started and that makes a big difference.
>>>
>>> Cheers,
>>>
>>> Roger
>>>
>>>
>>>
>>>
>>>
>>> On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
>>> gdfm@apache.org> wrote:
>>>
>>> > Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
>>> winner
>>> > :)
>>> >
>>> > --
>>> > Gianmarco
>>> >
>>> > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <gdfm@apache.org
>>> >
>>> > wrote:
>>> >
>>> > > Hi,
>>> > >
>>> > > @Martin, thanks for you comments.
>>> > > Maybe I'm missing some important point, but I think coupling the
>>> releases
>>> > > is actually a *good* thing.
>>> > > To make an example, would it be better if the MR and HDFS components of
>>> > > Hadoop had different release schedules?
>>> > >
>>> > > Actually, keeping the discussion in a single place would make agreeing
>>> on
>>> > > releases (and backwards compatibility) much easier, as everybody would
>>> be
>>> > > responsible for the whole codebase.
>>> > >
>>> > > That said, I like the idea of absorbing samza-core as a sub-project,
>>> and
>>> > > leave the fancy stuff separate.
>>> > > It probably gives 90% of the benefits we have been discussing here.
>>> > >
>>> > > Cheers,
>>> > >
>>> > > --
>>> > > Gianmarco
>>> > >
>>> > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
>>> > >
>>> > >> Hey Martin,
>>> > >>
>>> > >> I agree coupling release schedules is a downside.
>>> > >>
>>> > >> Definitely we can try to solve some of the integration problems in
>>> > >> Confluent Platform or in other distributions. But I think this ends up
>>> > >> being really shallow. I guess I feel to really get a good user
>>> > experience
>>> > >> the two systems have to kind of feel like part of the same thing and
>>> you
>>> > >> can't really add that in later--you can put both in the same
>>> > downloadable
>>> > >> tar file but it doesn't really give a very cohesive feeling. I agree
>>> > that
>>> > >> ultimately any of the project stuff is as much social and naming as
>>> > >> anything else--theoretically two totally independent projects could
>>> work
>>> > >> to
>>> > >> tightly align. In practice this seems to be quite difficult though.
>>> > >>
>>> > >> For the frameworks--totally agree it would be good to maintain the
>>> > >> framework support with the project. In some cases there may not be too
>>> > >> much
>>> > >> there since the integration gets lighter but I think whatever stubs
>>> you
>>> > >> need should be included. So no I definitely wasn't trying to imply
>>> > >> dropping
>>> > >> support for these frameworks, just making the integration lighter by
>>> > >> separating process management from partition management.
>>> > >>
>>> > >> You raise two good points we would have to figure out if we went down
>>> > the
>>> > >> alignment path:
>>> > >> 1. With respect to the name, yeah I think the first question is
>>> whether
>>> > >> some "re-branding" would be worth it. If so then I think we can have a
>>> > big
>>> > >> thread on the name. I'm definitely not set on Kafka Streaming or Kafka
>>> > >> Streams I was just using them to be kind of illustrative. I agree with
>>> > >> your
>>> > >> critique of these names, though I think people would get the idea.
>>> > >> 2. Yeah you also raise a good point about how to "factor" it. Here are
>>> > the
>>> > >> options I see (I could get enthusiastic about any of them):
>>> > >>    a. One repo for both Kafka and Samza
>>> > >>    b. Two repos, retaining the current seperation
>>> > >>    c. Two repos, the equivalent of samza-api and samza-core is
>>> absorbed
>>> > >> almost like a third client
>>> > >>
>>> > >> Cheers,
>>> > >>
>>> > >> -Jay
>>> > >>
>>> > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
>>> martin@kleppmann.com>
>>> > >> wrote:
>>> > >>
>>> > >> > Ok, thanks for the clarifications. Just a few follow-up comments.
>>> > >> >
>>> > >> > - I see the appeal of merging with Kafka or becoming a subproject:
>>> the
>>> > >> > reasons you mention are good. The risk I see is that release
>>> schedules
>>> > >> > become coupled to each other, which can slow everyone down, and
>>> large
>>> > >> > projects with many contributors are harder to manage. (Jakob, can
>>> you
>>> > >> speak
>>> > >> > from experience, having seen a wider range of Hadoop ecosystem
>>> > >> projects?)
>>> > >> >
>>> > >> > Some of the goals of a better unified developer experience could
>>> also
>>> > be
>>> > >> > solved by integrating Samza nicely into a Kafka distribution (such
>>> as
>>> > >> > Confluent's). I'm not against merging projects if we decide that's
>>> the
>>> > >> way
>>> > >> > to go, just pointing out the same goals can perhaps also be achieved
>>> > in
>>> > >> > other ways.
>>> > >> >
>>> > >> > - With regard to dropping the YARN dependency: are you proposing
>>> that
>>> > >> > Samza doesn't give any help to people wanting to run on
>>> > >> YARN/Mesos/AWS/etc?
>>> > >> > So the docs would basically have a link to Slider and nothing else?
>>> Or
>>> > >> > would we maintain integrations with a bunch of popular deployment
>>> > >> methods
>>> > >> > (e.g. the necessary glue and shell scripts to make Samza work with
>>> > >> Slider)?
>>> > >> >
>>> > >> > I absolutely think it's a good idea to have the "as a library" and
>>> > "as a
>>> > >> > process" (using Yi's taxonomy) options for people who want them,
>>> but I
>>> > >> > think there should also be a low-friction path for common "as a
>>> > service"
>>> > >> > deployment methods, for which we probably need to maintain
>>> > integrations.
>>> > >> >
>>> > >> > - Project naming: "Kafka Streams" seems odd to me, because Kafka is
>>> > all
>>> > >> > about streams already. Perhaps "Kafka Transformers" or "Kafka
>>> Filters"
>>> > >> > would be more apt?
>>> > >> >
>>> > >> > One suggestion: perhaps the core of Samza (stream transformation
>>> with
>>> > >> > state management -- i.e. the "Samza as a library" bit) could become
>>> > >> part of
>>> > >> > Kafka, while higher-level tools such as streaming SQL and
>>> integrations
>>> > >> with
>>> > >> > deployment frameworks remain in a separate project? In other words,
>>> > >> Kafka
>>> > >> > would absorb the proven, stable core of Samza, which would become
>>> the
>>> > >> > "third Kafka client" mentioned early in this thread. The Samza
>>> project
>>> > >> > would then target that third Kafka client as its base API, and the
>>> > >> project
>>> > >> > would be freed up to explore more experimental new horizons.
>>> > >> >
>>> > >> > Martin
>>> > >> >
>>> > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
>>> > >> >
>>> > >> > > Hey Martin,
>>> > >> > >
>>> > >> > > For the YARN/Mesos/etc decoupling I actually don't think it ties
>>> our
>>> > >> > hands
>>> > >> > > at all, all it does is refactor things. The division of
>>> > >> responsibility is
>>> > >> > > that Samza core is responsible for task lifecycle, state, and
>>> > >> partition
>>> > >> > > management (using the Kafka co-ordinator) but it is NOT
>>> responsible
>>> > >> for
>>> > >> > > packaging, configuration deployment or execution of processes. The
>>> > >> > problem
>>> > >> > > of packaging and starting these processes is
>>> > >> > > framework/environment-specific. This leaves individual frameworks
>>> to
>>> > >> be
>>> > >> > as
>>> > >> > > fancy or vanilla as they like. So you can get simple stateless
>>> > >> support in
>>> > >> > > YARN, Mesos, etc using their off-the-shelf app framework (Slider,
>>> > >> > Marathon,
>>> > >> > > etc). These are well known by people and have nice UIs and a lot
>>> of
>>> > >> > > flexibility. I don't think they have node affinity as a built in
>>> > >> option
>>> > >> > > (though I could be wrong). So if we want that we can either wait
>>> for
>>> > >> them
>>> > >> > > to add it or do a custom framework to add that feature (as now).
>>> > >> > Obviously
>>> > >> > > if you manage things with old-school ops tools (puppet/chef/etc)
>>> you
>>> > >> get
>>> > >> > > locality easily. The nice thing, though, is that all the samza
>>> > >> "business
>>> > >> > > logic" around partition management and fault tolerance is in Samza
>>> > >> core
>>> > >> > so
>>> > >> > > it is shared across frameworks and the framework specific bit is
>>> > just
>>> > >> > > whether it is smart enough to try to get the same host when a job
>>> is
>>> > >> > > restarted.
>>> > >> > >
>>> > >> > > With respect to the Kafka-alignment, yeah I think the goal would
>>> be
>>> > >> (a)
>>> > >> > > actually get better alignment in user experience, and (b) express
>>> > >> this in
>>> > >> > > the naming and project branding. Specifically:
>>> > >> > > 1. Website/docs, it would be nice for the "transformation" api to
>>> be
>>> > >> > > discoverable in the main Kafka docs--i.e. be able to explain when
>>> to
>>> > >> use
>>> > >> > > the consumer and when to use the stream processing functionality
>>> and
>>> > >> lead
>>> > >> > > people into that experience.
>>> > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that
>>> has
>>> > >> both
>>> > >> > > Kafka and the stream processing part and they actually work
>>> > together.
>>> > >> > > 3. Unify the programming experience so the client and Samza api
>>> > share
>>> > >> > > config/monitoring/naming/packaging/etc.
>>> > >> > >
>>> > >> > > I think sub-projects keep separate committers and can have a
>>> > separate
>>> > >> > repo,
>>> > >> > > but I'm actually not really sure (I can't find a definition of a
>>> > >> > subproject
>>> > >> > > in Apache).
>>> > >> > >
>>> > >> > > Basically at a high-level you want the experience to "feel" like a
>>> > >> single
>>> > >> > > system, not to relatively independent things that are kind of
>>> > >> awkwardly
>>> > >> > > glued together.
>>> > >> > >
>>> > >> > > I think if we did that they having naming or branding like "kafka
>>> > >> > > streaming" or "kafka streams" or something like that would
>>> actually
>>> > >> do a
>>> > >> > > good job of conveying what it is. I do that this would help
>>> adoption
>>> > >> > quite
>>> > >> > > a lot as it would correctly convey that using Kafka Streaming with
>>> > >> Kafka
>>> > >> > is
>>> > >> > > a fairly seamless experience and Kafka is pretty heavily adopted
>>> at
>>> > >> this
>>> > >> > > point.
>>> > >> > >
>>> > >> > > Fwiw we actually considered this model originally when open
>>> sourcing
>>> > >> > Samza,
>>> > >> > > however at that time Kafka was relatively unknown and we decided
>>> not
>>> > >> to
>>> > >> > do
>>> > >> > > it since we felt it would be limiting. From my point of view the
>>> > three
>>> > >> > > things have changed (1) Kafka is now really heavily used for
>>> stream
>>> > >> > > processing, (2) we learned that abstracting out the stream well is
>>> > >> > > basically impossible, (3) we learned it is really hard to keep the
>>> > two
>>> > >> > > things feeling like a single product.
>>> > >> > >
>>> > >> > > -Jay
>>> > >> > >
>>> > >> > >
>>> > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
>>> > >> martin@kleppmann.com>
>>> > >> > > wrote:
>>> > >> > >
>>> > >> > >> Hi all,
>>> > >> > >>
>>> > >> > >> Lots of good thoughts here.
>>> > >> > >>
>>> > >> > >> I agree with the general philosophy of tying Samza more firmly to
>>> > >> Kafka.
>>> > >> > >> After I spent a while looking at integrating other message
>>> brokers
>>> > >> (e.g.
>>> > >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
>>> > >> > SystemConsumer
>>> > >> > >> tacitly assumes a model so much like Kafka's that pretty much
>>> > nobody
>>> > >> but
>>> > >> > >> Kafka actually implements it. (Databus is perhaps an exception,
>>> but
>>> > >> it
>>> > >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza fully
>>> > >> > dependent
>>> > >> > >> on Kafka acknowledges that the system-independence was never as
>>> > real
>>> > >> as
>>> > >> > we
>>> > >> > >> perhaps made it out to be. The gains of code reuse are real.
>>> > >> > >>
>>> > >> > >> The idea of decoupling Samza from YARN has also always been
>>> > >> appealing to
>>> > >> > >> me, for various reasons already mentioned in this thread.
>>> Although
>>> > >> > making
>>> > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems
>>> > >> laudable,
>>> > >> > I am
>>> > >> > >> a little concerned that it will restrict us to a lowest common
>>> > >> > denominator.
>>> > >> > >> For example, would host affinity (SAMZA-617) still be possible?
>>> For
>>> > >> jobs
>>> > >> > >> with large amounts of state, I think SAMZA-617 would be a big
>>> boon,
>>> > >> > since
>>> > >> > >> restoring state off the changelog on every single restart is
>>> > painful,
>>> > >> > due
>>> > >> > >> to long recovery times. It would be a shame if the decoupling
>>> from
>>> > >> YARN
>>> > >> > >> made host affinity impossible.
>>> > >> > >>
>>> > >> > >> Jay, a question about the proposed API for instantiating a job in
>>> > >> code
>>> > >> > >> (rather than a properties file): when submitting a job to a
>>> > cluster,
>>> > >> is
>>> > >> > the
>>> > >> > >> idea that the instantiation code runs on a client somewhere,
>>> which
>>> > >> then
>>> > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that
>>> > >> code
>>> > >> > run
>>> > >> > >> on each container that is part of the job (in which case, how
>>> does
>>> > >> the
>>> > >> > job
>>> > >> > >> submission to the cluster work)?
>>> > >> > >>
>>> > >> > >> I agree with Garry that it doesn't feel right to make a 1.0
>>> release
>>> > >> > with a
>>> > >> > >> plan for it to be immediately obsolete. So if this is going to
>>> > >> happen, I
>>> > >> > >> think it would be more honest to stick with 0.* version numbers
>>> > until
>>> > >> > the
>>> > >> > >> library-ified Samza has been implemented, is stable and widely
>>> > used.
>>> > >> > >>
>>> > >> > >> Should the new Samza be a subproject of Kafka? There is precedent
>>> > for
>>> > >> > >> tight coupling between different Apache projects (e.g. Curator
>>> and
>>> > >> > >> Zookeeper, or Slider and YARN), so I think remaining separate
>>> would
>>> > >> be
>>> > >> > ok.
>>> > >> > >> Even if Samza is fully dependent on Kafka, there is enough
>>> > substance
>>> > >> in
>>> > >> > >> Samza that it warrants being a separate project. An argument in
>>> > >> favour
>>> > >> > of
>>> > >> > >> merging would be if we think Kafka has a much stronger "brand
>>> > >> presence"
>>> > >> > >> than Samza; I'm ambivalent on that one. If the Kafka project is
>>> > >> willing
>>> > >> > to
>>> > >> > >> endorse Samza as the "official" way of doing stateful stream
>>> > >> > >> transformations, that would probably have much the same effect as
>>> > >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
>>> > >> > >> collaboration between the two projects will be needed in any
>>> case.
>>> > >> > >>
>>> > >> > >> From a project management perspective, I guess the "new Samza"
>>> > would
>>> > >> > have
>>> > >> > >> to be developed on a branch alongside ongoing maintenance of the
>>> > >> current
>>> > >> > >> line of development? I think it would be important to continue
>>> > >> > supporting
>>> > >> > >> existing users, and provide a graceful migration path to the new
>>> > >> > version.
>>> > >> > >> Leaving the current versions unsupported and forcing people to
>>> > >> rewrite
>>> > >> > >> their jobs would send a bad signal.
>>> > >> > >>
>>> > >> > >> Best,
>>> > >> > >> Martin
>>> > >> > >>
>>> > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
>>> > >> > >>
>>> > >> > >>> Hey Garry,
>>> > >> > >>>
>>> > >> > >>> Yeah that's super frustrating. I'd be happy to chat more about
>>> > this
>>> > >> if
>>> > >> > >>> you'd be interested. I think Chris and I started with the idea
>>> of
>>> > >> "what
>>> > >> > >>> would it take to make Samza a kick-ass ingestion tool" but
>>> > >> ultimately
>>> > >> > we
>>> > >> > >>> kind of came around to the idea that ingestion and
>>> transformation
>>> > >> had
>>> > >> > >>> pretty different needs and coupling the two made things hard.
>>> > >> > >>>
>>> > >> > >>> For what it's worth I think copycat (KIP-26) actually will do
>>> what
>>> > >> you
>>> > >> > >> are
>>> > >> > >>> looking for.
>>> > >> > >>>
>>> > >> > >>> With regard to your point about slider, I don't necessarily
>>> > >> disagree.
>>> > >> > >> But I
>>> > >> > >>> think getting good YARN support is quite doable and I think we
>>> can
>>> > >> make
>>> > >> > >>> that work well. I think the issue this proposal solves is that
>>> > >> > >> technically
>>> > >> > >>> it is pretty hard to support multiple cluster management systems
>>> > the
>>> > >> > way
>>> > >> > >>> things are now, you need to write an "app master" or "framework"
>>> > for
>>> > >> > each
>>> > >> > >>> and they are all a little different so testing is really hard.
>>> In
>>> > >> the
>>> > >> > >>> absence of this we have been stuck with just YARN which has
>>> > >> fantastic
>>> > >> > >>> penetration in the Hadoopy part of the org, but zero penetration
>>> > >> > >> elsewhere.
>>> > >> > >>> Given the huge amount of work being put in to slider, marathon,
>>> > aws
>>> > >> > >>> tooling, not to mention the umpteen related packaging
>>> technologies
>>> > >> > people
>>> > >> > >>> want to use (Docker, Kubernetes, various cloud-specific deploy
>>> > >> tools,
>>> > >> > >> etc)
>>> > >> > >>> I really think it is important to get this right.
>>> > >> > >>>
>>> > >> > >>> -Jay
>>> > >> > >>>
>>> > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>>> > >> > >>> g.turkington@improvedigital.com> wrote:
>>> > >> > >>>
>>> > >> > >>>> Hi all,
>>> > >> > >>>>
>>> > >> > >>>> I think the question below re does Samza become a sub-project
>>> of
>>> > >> Kafka
>>> > >> > >>>> highlights the broader point around migration. Chris mentions
>>> > >> Samza's
>>> > >> > >>>> maturity is heading towards a v1 release but I'm not sure it
>>> > feels
>>> > >> > >> right to
>>> > >> > >>>> launch a v1 then immediately plan to deprecate most of it.
>>> > >> > >>>>
>>> > >> > >>>> From a selfish perspective I have some guys who have started
>>> > >> working
>>> > >> > >> with
>>> > >> > >>>> Samza and building some new consumers/producers was next up.
>>> > Sounds
>>> > >> > like
>>> > >> > >>>> that is absolutely not the direction to go. I need to look into
>>> > the
>>> > >> > KIP
>>> > >> > >> in
>>> > >> > >>>> more detail but for me the attractiveness of adding new Samza
>>> > >> > >>>> consumer/producers -- even if yes all they were doing was
>>> really
>>> > >> > getting
>>> > >> > >>>> data into and out of Kafka --  was to avoid  having to worry
>>> > about
>>> > >> the
>>> > >> > >>>> lifecycle management of external clients. If there is a generic
>>> > >> Kafka
>>> > >> > >>>> ingress/egress layer that I can plug a new connector into and
>>> > have
>>> > >> a
>>> > >> > >> lot of
>>> > >> > >>>> the heavy lifting re scale and reliability done for me then it
>>> > >> gives
>>> > >> > me
>>> > >> > >> all
>>> > >> > >>>> the pushing new consumers/producers would. If not then it
>>> > >> complicates
>>> > >> > my
>>> > >> > >>>> operational deployments.
>>> > >> > >>>>
>>> > >> > >>>> Which is similar to my other question with the proposal -- if
>>> we
>>> > >> > build a
>>> > >> > >>>> fully available/stand-alone Samza plus the requisite shims to
>>> > >> > integrate
>>> > >> > >>>> with Slider etc I suspect the former may be a lot more work
>>> than
>>> > we
>>> > >> > >> think.
>>> > >> > >>>> We may make it much easier for a newcomer to get something
>>> > running
>>> > >> but
>>> > >> > >>>> having them step up and get a reliable production deployment
>>> may
>>> > >> still
>>> > >> > >>>> dominate mailing list  traffic, if for different reasons than
>>> > >> today.
>>> > >> > >>>>
>>> > >> > >>>> Don't get me wrong -- I'm comfortable with making the Samza
>>> > >> dependency
>>> > >> > >> on
>>> > >> > >>>> Kafka much more explicit and I absolutely see the benefits  in
>>> > the
>>> > >> > >>>> reduction of duplication and clashing
>>> terminologies/abstractions
>>> > >> that
>>> > >> > >>>> Chris/Jay describe. Samza as a library would likely be a very
>>> > nice
>>> > >> > tool
>>> > >> > >> to
>>> > >> > >>>> add to the Kafka ecosystem. I just have the concerns above re
>>> the
>>> > >> > >>>> operational side.
>>> > >> > >>>>
>>> > >> > >>>> Garry
>>> > >> > >>>>
>>> > >> > >>>> -----Original Message-----
>>> > >> > >>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
>>> > >> > >>>> Sent: 02 July 2015 12:56
>>> > >> > >>>> To: dev@samza.apache.org
>>> > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
>>> > >> > >>>>
>>> > >> > >>>> Very interesting thoughts.
>>> > >> > >>>> From outside, I have always perceived Samza as a computing
>>> layer
>>> > >> over
>>> > >> > >>>> Kafka.
>>> > >> > >>>>
>>> > >> > >>>> The question, maybe a bit provocative, is "should Samza be a
>>> > >> > sub-project
>>> > >> > >>>> of Kafka then?"
>>> > >> > >>>> Or does it make sense to keep it as a separate project with a
>>> > >> separate
>>> > >> > >>>> governance?
>>> > >> > >>>>
>>> > >> > >>>> Cheers,
>>> > >> > >>>>
>>> > >> > >>>> --
>>> > >> > >>>> Gianmarco
>>> > >> > >>>>
>>> > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
>>> wrote:
>>> > >> > >>>>
>>> > >> > >>>>> Overall, I agree to couple with Kafka more tightly. Because
>>> > Samza
>>> > >> de
>>> > >> > >>>>> facto is based on Kafka, and it should leverage what Kafka
>>> has.
>>> > At
>>> > >> > the
>>> > >> > >>>>> same time, Kafka does not need to reinvent what Samza already
>>> > >> has. I
>>> > >> > >>>>> also like the idea of separating the ingestion and
>>> > transformation.
>>> > >> > >>>>>
>>> > >> > >>>>> But it is a little difficult for me to image how the Samza
>>> will
>>> > >> look
>>> > >> > >>>> like.
>>> > >> > >>>>> And I feel Chris and Jay have a little difference in terms of
>>> > how
>>> > >> > >>>>> Samza should look like.
>>> > >> > >>>>>
>>> > >> > >>>>> *** Will it look like what Jay's code shows (A client of
>>> Kakfa)
>>> > ?
>>> > >> And
>>> > >> > >>>>> user's application code calls this client?
>>> > >> > >>>>>
>>> > >> > >>>>> 1. If we make Samza be a library of Kafka (like what the code
>>> > >> shows),
>>> > >> > >>>>> how do we implement auto-balance and fault-tolerance? Are they
>>> > >> taken
>>> > >> > >>>>> care by the Kafka broker or other mechanism, such as "Samza
>>> > >> worker"
>>> > >> > >>>>> (just make up the name) ?
>>> > >> > >>>>>
>>> > >> > >>>>> 2. What about other features, such as auto-scaling, shared
>>> > state,
>>> > >> > >>>>> monitoring?
>>> > >> > >>>>>
>>> > >> > >>>>>
>>> > >> > >>>>> *** If we have Samza standalone, (is this what Chris
>>> suggests?)
>>> > >> > >>>>>
>>> > >> > >>>>> 1. we still need to ingest data from Kakfa and produce to it.
>>> > >> Then it
>>> > >> > >>>>> becomes the same as what Samza looks like now, except it does
>>> > not
>>> > >> > rely
>>> > >> > >>>>> on Yarn anymore.
>>> > >> > >>>>>
>>> > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's metrics,
>>> > logs,
>>> > >> > >>>>> etc? Use Kafka code as the dependency?
>>> > >> > >>>>>
>>> > >> > >>>>>
>>> > >> > >>>>> Thanks,
>>> > >> > >>>>>
>>> > >> > >>>>> Fang, Yan
>>> > >> > >>>>> yanfang724@gmail.com
>>> > >> > >>>>>
>>> > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
>>> > wangguoz@gmail.com
>>> > >> >
>>> > >> > >>>> wrote:
>>> > >> > >>>>>
>>> > >> > >>>>>> Read through the code example and it looks good to me. A few
>>> > >> > >>>>>> thoughts regarding deployment:
>>> > >> > >>>>>>
>>> > >> > >>>>>> Today Samza deploys as executable runnable like:
>>> > >> > >>>>>>
>>> > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
>>> > >> > >>>> --config-path=file://...
>>> > >> > >>>>>>
>>> > >> > >>>>>> And this proposal advocate for deploying Samza more as
>>> embedded
>>> > >> > >>>>>> libraries in user application code (ignoring the terminology
>>> > >> since
>>> > >> > >>>>>> it is not the
>>> > >> > >>>>> same
>>> > >> > >>>>>> as the prototype code):
>>> > >> > >>>>>>
>>> > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread thread =
>>> > new
>>> > >> > >>>>>> Thread(task); thread.start();
>>> > >> > >>>>>>
>>> > >> > >>>>>> I think both of these deployment modes are important for
>>> > >> different
>>> > >> > >>>>>> types
>>> > >> > >>>>> of
>>> > >> > >>>>>> users. That said, I think making Samza purely standalone is
>>> > still
>>> > >> > >>>>>> sufficient for either runnable or library modes.
>>> > >> > >>>>>>
>>> > >> > >>>>>> Guozhang
>>> > >> > >>>>>>
>>> > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
>>> jay@confluent.io>
>>> > >> > wrote:
>>> > >> > >>>>>>
>>> > >> > >>>>>>> Looks like gmail mangled the code example, it was supposed
>>> to
>>> > >> look
>>> > >> > >>>>>>> like
>>> > >> > >>>>>>> this:
>>> > >> > >>>>>>>
>>> > >> > >>>>>>> Properties props = new Properties();
>>> > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
>>> > >> StreamingConfig
>>> > >> > >>>>>>> config = new StreamingConfig(props);
>>> > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
>>> > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
>>> > >> > >>>>>>> config.serialization(new StringSerializer(), new
>>> > >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
>>> > >> > >>>>>>> KafkaStreaming(config); container.run();
>>> > >> > >>>>>>>
>>> > >> > >>>>>>> -Jay
>>> > >> > >>>>>>>
>>> > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
>>> jay@confluent.io
>>> > >
>>> > >> > >>>> wrote:
>>> > >> > >>>>>>>
>>> > >> > >>>>>>>> Hey guys,
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> This came out of some conversations Chris and I were having
>>> > >> > >>>>>>>> around
>>> > >> > >>>>>>> whether
>>> > >> > >>>>>>>> it would make sense to use Samza as a kind of data
>>> ingestion
>>> > >> > >>>>> framework
>>> > >> > >>>>>>> for
>>> > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This
>>> kind
>>> > of
>>> > >> > >>>>>> combined
>>> > >> > >>>>>>>> with complaints around config and YARN and the discussion
>>> > >> around
>>> > >> > >>>>>>>> how
>>> > >> > >>>>> to
>>> > >> > >>>>>>>> best do a standalone mode.
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> So the thought experiment was, given that Samza was
>>> basically
>>> > >> > >>>>>>>> already totally Kafka specific, what if you just embraced
>>> > that
>>> > >> > >>>>>>>> and turned it
>>> > >> > >>>>>> into
>>> > >> > >>>>>>>> something less like a heavyweight framework and more like a
>>> > >> > >>>>>>>> third
>>> > >> > >>>>> Kafka
>>> > >> > >>>>>>>> client--a kind of "producing consumer" with state
>>> management
>>> > >> > >>>>>> facilities.
>>> > >> > >>>>>>>> Basically a library. Instead of a complex stream processing
>>> > >> > >>>>>>>> framework
>>> > >> > >>>>>>> this
>>> > >> > >>>>>>>> would actually be a very simple thing, not much more
>>> > >> complicated
>>> > >> > >>>>>>>> to
>>> > >> > >>>>> use
>>> > >> > >>>>>>> or
>>> > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we thought
>>> about
>>> > >> it
>>> > >> > >>>>>>>> a
>>> > >> > >>>>> lot
>>> > >> > >>>>>> of
>>> > >> > >>>>>>>> what Samza (and the other stream processing systems were
>>> > doing)
>>> > >> > >>>>> seemed
>>> > >> > >>>>>>> like
>>> > >> > >>>>>>>> kind of a hangover from MapReduce.
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> Of course you need to ingest/output data to and from the
>>> > stream
>>> > >> > >>>>>>>> processing. But when we actually looked into how that would
>>> > >> > >>>>>>>> work,
>>> > >> > >>>>> Samza
>>> > >> > >>>>>>>> isn't really an ideal data ingestion framework for a bunch
>>> of
>>> > >> > >>>>> reasons.
>>> > >> > >>>>>> To
>>> > >> > >>>>>>>> really do that right you need a pretty different internal
>>> > data
>>> > >> > >>>>>>>> model
>>> > >> > >>>>>> and
>>> > >> > >>>>>>>> set of apis. So what if you split them and had an api for
>>> > Kafka
>>> > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
>>> > >> Kafka
>>> > >> > >>>>>>>> transformation (Samza).
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> This would also allow really embracing the same terminology
>>> > and
>>> > >> > >>>>>>>> conventions. One complaint about the current state is that
>>> > the
>>> > >> > >>>>>>>> two
>>> > >> > >>>>>>> systems
>>> > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs
>>> "topic"
>>> > >> and
>>> > >> > >>>>>>> different
>>> > >> > >>>>>>>> config and monitoring systems means you kind of have to
>>> learn
>>> > >> > >>>>>>>> Kafka's
>>> > >> > >>>>>>> way,
>>> > >> > >>>>>>>> then learn Samza's slightly different way, then kind of
>>> > >> > >>>>>>>> understand
>>> > >> > >>>>> how
>>> > >> > >>>>>>> they
>>> > >> > >>>>>>>> map to each other, which having walked a few people through
>>> > >> this
>>> > >> > >>>>>>>> is surprisingly tricky for folks to get.
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> Since I have been spending a lot of time on airplanes I
>>> > hacked
>>> > >> > >>>>>>>> up an ernest but still somewhat incomplete prototype of
>>> what
>>> > >> > >>>>>>>> this would
>>> > >> > >>>>> look
>>> > >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka as it
>>> > >> > >>>>>>>> required a
>>> > >> > >>>>>> few
>>> > >> > >>>>>>>> changes to the new consumer. Here is the code:
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>
>>> > >> > >>>>>>
>>> > >> > >>>>>
>>> > >> >
>>> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>>> > >> > >>>>> /apache/kafka/clients/streaming
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> For the purpose of the prototype I just liberally renamed
>>> > >> > >>>>>>>> everything
>>> > >> > >>>>> to
>>> > >> > >>>>>>>> try to align it with Kafka with no regard for
>>> compatibility.
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> To use this would be something like this:
>>> > >> > >>>>>>>> Properties props = new Properties();
>>> > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
>>> > >> > >>>>>>>> StreamingConfig config = new
>>> > >> > >>>>> StreamingConfig(props);
>>> > >> > >>>>>>> config.subscribe("test-topic-1",
>>> > >> > >>>>>>>> "test-topic-2");
>>> > >> config.processor(ExampleStreamProcessor.class);
>>> > >> > >>>>>>> config.serialization(new
>>> > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
>>> KafkaStreaming
>>> > >> > >>>>>> container =
>>> > >> > >>>>>>>> new KafkaStreaming(config); container.run();
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
>>> > StreamProcessor
>>> > >> > >>>>>>>> is basically StreamTask.
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> So rather than putting all the class names in a file and
>>> then
>>> > >> > >>>>>>>> having
>>> > >> > >>>>>> the
>>> > >> > >>>>>>>> job assembled by reflection, you just instantiate the
>>> > container
>>> > >> > >>>>>>>> programmatically. Work is balanced over however many
>>> > instances
>>> > >> > >>>>>>>> of
>>> > >> > >>>>> this
>>> > >> > >>>>>>> are
>>> > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new tasks are
>>> > >> added
>>> > >> > >>>>>>>> to
>>> > >> > >>>>> the
>>> > >> > >>>>>>>> existing containers without shutting them down).
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> We would provide some glue for running this stuff in YARN
>>> via
>>> > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their
>>> tools
>>> > >> > >>>>>>>> but from the
>>> > >> > >>>>>> point
>>> > >> > >>>>>>> of
>>> > >> > >>>>>>>> view of these frameworks these stream processing jobs are
>>> > just
>>> > >> > >>>>>> stateless
>>> > >> > >>>>>>>> services that can come and go and expand and contract at
>>> > will.
>>> > >> > >>>>>>>> There
>>> > >> > >>>>> is
>>> > >> > >>>>>>> no
>>> > >> > >>>>>>>> more custom scheduler.
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> Here are some relevant details:
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get larger if
>>> we
>>> > >> > >>>>>>>>  productionized but not vastly larger. We really do get a
>>> ton
>>> > >> > >>>>>>>> of
>>> > >> > >>>>>>> leverage
>>> > >> > >>>>>>>>  out of Kafka.
>>> > >> > >>>>>>>>  2. Partition management is fully delegated to the new
>>> > >> consumer.
>>> > >> > >>>>> This
>>> > >> > >>>>>>>>  is nice since now any partition management strategy
>>> > available
>>> > >> > >>>>>>>> to
>>> > >> > >>>>>> Kafka
>>> > >> > >>>>>>>>  consumer is also available to Samza (and vice versa) and
>>> > with
>>> > >> > >>>>>>>> the
>>> > >> > >>>>>>> exact
>>> > >> > >>>>>>>>  same configs.
>>> > >> > >>>>>>>>  3. It supports state as well as state reuse
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> -Jay
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>>> > >> > >>>>>> criccomini@apache.org>
>>> > >> > >>>>>>>> wrote:
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>>> Hey all,
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> I have had some discussions with Samza engineers at
>>> LinkedIn
>>> > >> > >>>>>>>>> and
>>> > >> > >>>>>>> Confluent
>>> > >> > >>>>>>>>> and we came up with a few observations and would like to
>>> > >> > >>>>>>>>> propose
>>> > >> > >>>>> some
>>> > >> > >>>>>>>>> changes.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> We've observed some things that I want to call out about
>>> > >> > >>>>>>>>> Samza's
>>> > >> > >>>>>> design,
>>> > >> > >>>>>>>>> and I'd like to propose some changes.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
>>> > >> > >>>>>>>>> * Samza is too pluggable.
>>> > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
>>> consumer
>>> > >> > >>>>>>>>> APIs
>>> > >> > >>>>> are
>>> > >> > >>>>>>>>> trying to solve a lot of the same problems.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> All three of these issues are related, but I'll address
>>> them
>>> > >> in
>>> > >> > >>>>> order.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Deployment
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic deployment
>>> > >> > >>>>>>>>> scheduler
>>> > >> > >>>>>> such
>>> > >> > >>>>>>>>> as
>>> > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet
>>> that
>>> > >> > >>>>>>>>> there
>>> > >> > >>>>>> would
>>> > >> > >>>>>>>>> be
>>> > >> > >>>>>>>>> one or two winners in this area, and we could support
>>> them,
>>> > >> and
>>> > >> > >>>>>>>>> the
>>> > >> > >>>>>> rest
>>> > >> > >>>>>>>>> would go away. In reality, there are many variations.
>>> > >> > >>>>>>>>> Furthermore,
>>> > >> > >>>>>> many
>>> > >> > >>>>>>>>> people still prefer to just start their processors like
>>> > normal
>>> > >> > >>>>>>>>> Java processes, and use traditional deployment scripts
>>> such
>>> > as
>>> > >> > >>>>>>>>> Fabric,
>>> > >> > >>>>>> Chef,
>>> > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users makes
>>> the
>>> > >> > >>>>>>>>> Samza start-up process really painful for first time
>>> users.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit of a
>>> > >> > >>>>>>>>> mis-fire
>>> > >> > >>>>>> because
>>> > >> > >>>>>>>>> of
>>> > >> > >>>>>>>>> a fundamental misunderstanding between the nature of batch
>>> > >> jobs
>>> > >> > >>>>>>>>> and
>>> > >> > >>>>>>> stream
>>> > >> > >>>>>>>>> processing jobs. Early on, we made conscious effort to
>>> favor
>>> > >> > >>>>>>>>> the
>>> > >> > >>>>>> Hadoop
>>> > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked and was
>>> > well
>>> > >> > >>>>>>> understood.
>>> > >> > >>>>>>>>> One thing that we missed was that batch jobs have a
>>> definite
>>> > >> > >>>>>> beginning,
>>> > >> > >>>>>>>>> and
>>> > >> > >>>>>>>>> end, and stream processing jobs don't (usually). This
>>> leads
>>> > to
>>> > >> > >>>>>>>>> a
>>> > >> > >>>>> much
>>> > >> > >>>>>>>>> simpler scheduling problem for stream processors. You
>>> > >> basically
>>> > >> > >>>>>>>>> just
>>> > >> > >>>>>>> need
>>> > >> > >>>>>>>>> to find a place to start the processor, and start it. The
>>> > way
>>> > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
>>> > >> > >>>>>>>>> being "full". We always
>>> > >> > >>>>>> add
>>> > >> > >>>>>>>>> more machines. The problem with coupling Samza with a
>>> > >> scheduler
>>> > >> > >>>>>>>>> is
>>> > >> > >>>>>> that
>>> > >> > >>>>>>>>> Samza (as a framework) now has to handle deployment. This
>>> > >> pulls
>>> > >> > >>>>>>>>> in a
>>> > >> > >>>>>>> bunch
>>> > >> > >>>>>>>>> of things such as configuration distribution (config
>>> > stream),
>>> > >> > >>>>>>>>> shell
>>> > >> > >>>>>>> scrips
>>> > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
>>> stuff),
>>> > >> etc.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Another reason for requiring dynamic deployment was to
>>> > support
>>> > >> > >>>>>>>>> data locality. If you want to have locality, you need to
>>> put
>>> > >> > >>>>>>>>> your
>>> > >> > >>>>>> processors
>>> > >> > >>>>>>>>> close to the data they're processing. Upon further
>>> > >> > >>>>>>>>> investigation,
>>> > >> > >>>>>>> though,
>>> > >> > >>>>>>>>> this feature is not that beneficial. There is some good
>>> > >> > >>>>>>>>> discussion
>>> > >> > >>>>>> about
>>> > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
>>> > >> > >>>>>>>>> Map/Reduce
>>> > >> > >>>>>> path,
>>> > >> > >>>>>>>>> but
>>> > >> > >>>>>>>>> there are some fundamental differences between HDFS and
>>> > Kafka.
>>> > >> > >>>>>>>>> HDFS
>>> > >> > >>>>>> has
>>> > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to less
>>> > >> > >>>>>>>>> optimization potential with stream processors on top of
>>> > Kafka.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't have
>>> > any
>>> > >> > >>>>>>>>> built
>>> > >> > >>>>> in
>>> > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
>>> > >> > >>>>>>>>> deployment scheduling system to handle restarts when a
>>> > >> > >>>>>>>>> processor dies. This has
>>> > >> > >>>>>>> made
>>> > >> > >>>>>>>>> it very difficult to write a standalone Samza container
>>> > >> > >>>> (SAMZA-516).
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Pluggability
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> In some cases pluggability is good, but I think that we've
>>> > >> gone
>>> > >> > >>>>>>>>> too
>>> > >> > >>>>>> far
>>> > >> > >>>>>>>>> with it. Currently, Samza has:
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> * Pluggable config.
>>> > >> > >>>>>>>>> * Pluggable metrics.
>>> > >> > >>>>>>>>> * Pluggable deployment systems.
>>> > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
>>> > SystemProducer,
>>> > >> > >>>> etc).
>>> > >> > >>>>>>>>> * Pluggable serdes.
>>> > >> > >>>>>>>>> * Pluggable storage engines.
>>> > >> > >>>>>>>>> * Pluggable strategies for just about every component
>>> > >> > >>>>> (MessageChooser,
>>> > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> There's probably more that I've forgotten, as well. Some
>>> of
>>> > >> > >>>>>>>>> these
>>> > >> > >>>>> are
>>> > >> > >>>>>>>>> useful, but some have proven not to be. This all comes at
>>> a
>>> > >> cost:
>>> > >> > >>>>>>>>> complexity. This complexity is making it harder for our
>>> > users
>>> > >> > >>>>>>>>> to
>>> > >> > >>>>> pick
>>> > >> > >>>>>> up
>>> > >> > >>>>>>>>> and use Samza out of the box. It also makes it difficult
>>> for
>>> > >> > >>>>>>>>> Samza developers to reason about what the characteristics
>>> of
>>> > >> > >>>>>>>>> the container (since the characteristics change depending
>>> on
>>> > >> > >>>>>>>>> which plugins are use).
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> The issues with pluggability are most visible in the
>>> System
>>> > >> APIs.
>>> > >> > >>>>> What
>>> > >> > >>>>>>>>> Samza really requires to be functional is Kafka as its
>>> > >> > >>>>>>>>> transport
>>> > >> > >>>>>> layer.
>>> > >> > >>>>>>>>> But
>>> > >> > >>>>>>>>> we've conflated two unrelated use cases into one API:
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> 1. Get data into/out of Kafka.
>>> > >> > >>>>>>>>> 2. Process the data in Kafka.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> The current System API supports both of these use cases.
>>> The
>>> > >> > >>>>>>>>> problem
>>> > >> > >>>>>> is,
>>> > >> > >>>>>>>>> we
>>> > >> > >>>>>>>>> actually want different features for each use case. By
>>> > >> papering
>>> > >> > >>>>>>>>> over
>>> > >> > >>>>>>> these
>>> > >> > >>>>>>>>> two use cases, and providing a single API, we've
>>> introduced
>>> > a
>>> > >> > >>>>>>>>> ton of
>>> > >> > >>>>>>> leaky
>>> > >> > >>>>>>>>> abstractions.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> For example, what we'd really like in (2) is to have
>>> > >> > >>>>>>>>> monotonically increasing longs for offsets (like Kafka).
>>> > This
>>> > >> > >>>>>>>>> would be at odds
>>> > >> > >>>>> with
>>> > >> > >>>>>>> (1),
>>> > >> > >>>>>>>>> though, since different systems have different
>>> > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
>>> > >> > >>>>>>>>> There was discussion both on the mailing list and the SQL
>>> > >> JIRAs
>>> > >> > >>>>> about
>>> > >> > >>>>>>> the
>>> > >> > >>>>>>>>> need for this.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> The same thing holds true for replayability. Kafka allows
>>> us
>>> > >> to
>>> > >> > >>>>> rewind
>>> > >> > >>>>>>>>> when
>>> > >> > >>>>>>>>> we have a failure. Many other systems don't. In some
>>> cases,
>>> > >> > >>>>>>>>> systems
>>> > >> > >>>>>>> return
>>> > >> > >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
>>> > because
>>> > >> > >>>>>>>>> they
>>> > >> > >>>>>> have
>>> > >> > >>>>>>> no
>>> > >> > >>>>>>>>> offsets.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Partitioning is another example. Kafka supports
>>> > partitioning,
>>> > >> > >>>>>>>>> but
>>> > >> > >>>>> many
>>> > >> > >>>>>>>>> systems don't. We model this by having a single partition
>>> > for
>>> > >> > >>>>>>>>> those systems. Still, other systems model partitioning
>>> > >> > >>>> differently (e.g.
>>> > >> > >>>>>>>>> Kinesis).
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating streams
>>> > in
>>> > >> a
>>> > >> > >>>>>>>>> system-agnostic way is almost impossible. As is modeling
>>> > >> > >>>>>>>>> metadata
>>> > >> > >>>>> for
>>> > >> > >>>>>>> the
>>> > >> > >>>>>>>>> system (replication factor, partitions, location, etc).
>>> The
>>> > >> > >>>>>>>>> list
>>> > >> > >>>>> goes
>>> > >> > >>>>>>> on.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Duplicate work
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's consumer
>>> > and
>>> > >> > >>>>> producer
>>> > >> > >>>>>>>>> APIs
>>> > >> > >>>>>>>>> had a relatively weak feature set. On the consumer-side,
>>> you
>>> > >> > >>>>>>>>> had two
>>> > >> > >>>>>>>>> options: use the high level consumer, or the simple
>>> > consumer.
>>> > >> > >>>>>>>>> The
>>> > >> > >>>>>>> problem
>>> > >> > >>>>>>>>> with the high-level consumer was that it controlled your
>>> > >> > >>>>>>>>> offsets, partition assignments, and the order in which you
>>> > >> > >>>>>>>>> received messages. The
>>> > >> > >>>>> problem
>>> > >> > >>>>>>>>> with
>>> > >> > >>>>>>>>> the simple consumer is that it's not simple. It's basic.
>>> You
>>> > >> > >>>>>>>>> end up
>>> > >> > >>>>>>> having
>>> > >> > >>>>>>>>> to handle a lot of really low-level stuff that you
>>> > shouldn't.
>>> > >> > >>>>>>>>> We
>>> > >> > >>>>>> spent a
>>> > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
>>> robust.
>>> > >> It
>>> > >> > >>>>>>>>> also allows us to support some cool features:
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> * Per-partition message ordering and prioritization.
>>> > >> > >>>>>>>>> * Tight control over partition assignment to support
>>> joins,
>>> > >> > >>>>>>>>> global
>>> > >> > >>>>>> state
>>> > >> > >>>>>>>>> (if we want to implement it :)), etc.
>>> > >> > >>>>>>>>> * Tight control over offset checkpointing.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> What we didn't realize at the time is that these features
>>> > >> > >>>>>>>>> should
>>> > >> > >>>>>>> actually
>>> > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
>>> stream
>>> > >> > >>>>>> processors)
>>> > >> > >>>>>>>>> end up wanting to do things like joins and partition
>>> > >> > >>>>>>>>> assignment. The
>>> > >> > >>>>>>> Kafka
>>> > >> > >>>>>>>>> community has come to the same conclusion. They're adding
>>> a
>>> > >> ton
>>> > >> > >>>>>>>>> of upgrades into their new Kafka consumer implementation.
>>> > To a
>>> > >> > >>>>>>>>> large extent,
>>> > >> > >>>>> it's
>>> > >> > >>>>>>>>> duplicate work to what we've already done in Samza.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
>>> > approach
>>> > >> > >>>>>>>>> to
>>> > >> > >>>>>> Samza's
>>> > >> > >>>>>>>>> KafkaCheckpointManager implementation for handling offset
>>> > >> > >>>>>> checkpointing.
>>> > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature stores
>>> > >> offset
>>> > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them from
>>> > the
>>> > >> > >>>>>>>>> broker.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> A lot of this seems like a waste, since we could have
>>> shared
>>> > >> > >>>>>>>>> the
>>> > >> > >>>>> work
>>> > >> > >>>>>> if
>>> > >> > >>>>>>>>> it
>>> > >> > >>>>>>>>> had been done in Kafka from the get-go.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Vision
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> All of this leads me to a rather radical proposal. Samza
>>> is
>>> > >> > >>>>> relatively
>>> > >> > >>>>>>>>> stable at this point. I'd venture to say that we're near a
>>> > 1.0
>>> > >> > >>>>>> release.
>>> > >> > >>>>>>>>> I'd
>>> > >> > >>>>>>>>> like to propose that we take what we've learned, and begin
>>> > >> > >>>>>>>>> thinking
>>> > >> > >>>>>>> about
>>> > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were starting
>>> > >> from
>>> > >> > >>>>>> scratch?
>>> > >> > >>>>>>>>> My
>>> > >> > >>>>>>>>> proposal is to:
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
>>> > >> > >>>>>>>>> processors, and eliminate all direct dependences on YARN,
>>> > >> Mesos,
>>> > >> > >>>> etc.
>>> > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as the
>>> > stream
>>> > >> > >>>>>> processing
>>> > >> > >>>>>>>>> layer.
>>> > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
>>> > >> > >>>>>>>>> config
>>> > >> > >>>>>>> systems,
>>> > >> > >>>>>>>>> and simply use Kafka's instead.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> This would fix all of the issues that I outlined above. It
>>> > >> > >>>>>>>>> should
>>> > >> > >>>>> also
>>> > >> > >>>>>>>>> shrink the Samza code base pretty dramatically. Supporting
>>> > >> only
>>> > >> > >>>>>>>>> a standalone container will allow Samza to be executed on
>>> > YARN
>>> > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most
>>> other
>>> > >> > >>>>>>>>> in-house
>>> > >> > >>>>>>> deployment
>>> > >> > >>>>>>>>> systems. This should make life a lot easier for new users.
>>> > >> > >>>>>>>>> Imagine
>>> > >> > >>>>>>> having
>>> > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in mailing
>>> > >> list
>>> > >> > >>>>>> traffic
>>> > >> > >>>>>>>>> will be pretty dramatic.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The reality
>>> > is,
>>> > >> > >>>>> everyone
>>> > >> > >>>>>>>>> that
>>> > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically
>>> require
>>> > >> it
>>> > >> > >>>>>> already
>>> > >> > >>>>>>> in
>>> > >> > >>>>>>>>> order for most features to work. Those that are using
>>> other
>>> > >> > >>>>>>>>> systems
>>> > >> > >>>>>> are
>>> > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and then
>>> they
>>> > do
>>> > >> > >>>>>>>>> the processing on top. There is already discussion (
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>
>>> > >> > >>>>>>
>>> > >> > >>>>>
>>> > >> >
>>> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>>> > >> > >>>>> 767
>>> > >> > >>>>>>>>> )
>>> > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Once we make the call to couple with Kafka, we can
>>> leverage
>>> > a
>>> > >> > >>>>>>>>> ton of
>>> > >> > >>>>>>> their
>>> > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own config,
>>> > >> > >>>>>>>>> metrics,
>>> > >> > >>>>> etc.
>>> > >> > >>>>>>> We
>>> > >> > >>>>>>>>> can all share the same libraries, and make them better.
>>> This
>>> > >> > >>>>>>>>> will
>>> > >> > >>>>> also
>>> > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and will let
>>> > us
>>> > >> > >>>>> leverage
>>> > >> > >>>>>>>>> their offset management and partition management, rather
>>> > than
>>> > >> > >>>>>>>>> having
>>> > >> > >>>>>> our
>>> > >> > >>>>>>>>> own. All of the coordinator stream code would go away, as
>>> > >> would
>>> > >> > >>>>>>>>> most
>>> > >> > >>>>>> of
>>> > >> > >>>>>>>>> the
>>> > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
>>> > partition
>>> > >> > >>>>>>> management
>>> > >> > >>>>>>>>> features into the Kafka broker, but they're already moving
>>> > in
>>> > >> > >>>>>>>>> that direction with the new consumer API. The features we
>>> > have
>>> > >> > >>>>>>>>> for
>>> > >> > >>>>>> partition
>>> > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like they
>>> should
>>> > >> be
>>> > >> > >>>>>>>>> in
>>> > >> > >>>>>> Kafka
>>> > >> > >>>>>>>>> anyway. There will always be some niche usages which will
>>> > >> > >>>>>>>>> require
>>> > >> > >>>>>> extra
>>> > >> > >>>>>>>>> care and hence full control over partition assignments
>>> much
>>> > >> > >>>>>>>>> like the
>>> > >> > >>>>>>> Kafka
>>> > >> > >>>>>>>>> low level consumer api. These would continue to be
>>> > supported.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> These items will be good for the Samza community. They'll
>>> > make
>>> > >> > >>>>>>>>> Samza easier to use, and make it easier for developers to
>>> > add
>>> > >> > >>>>>>>>> new features.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat backwards
>>> > >> > >>>>> incompatible
>>> > >> > >>>>>>>>> change). If we choose to go this route, it's important
>>> that
>>> > we
>>> > >> > >>>>> openly
>>> > >> > >>>>>>>>> communicate how we're going to provide a migration path
>>> from
>>> > >> > >>>>>>>>> the
>>> > >> > >>>>>>> existing
>>> > >> > >>>>>>>>> APIs to the new ones (if we make incompatible changes). I
>>> > >> think
>>> > >> > >>>>>>>>> at a minimum, we'd probably need to provide a wrapper to
>>> > allow
>>> > >> > >>>>>>>>> existing StreamTask implementations to continue running on
>>> > the
>>> > >> > >>>> new container.
>>> > >> > >>>>>>> It's
>>> > >> > >>>>>>>>> also important that we openly communicate about timing,
>>> and
>>> > >> > >>>>>>>>> stages
>>> > >> > >>>>> of
>>> > >> > >>>>>>> the
>>> > >> > >>>>>>>>> migration.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> If you made it this far, I'm sure you have opinions. :)
>>> > Please
>>> > >> > >>>>>>>>> send
>>> > >> > >>>>>> your
>>> > >> > >>>>>>>>> thoughts and feedback.
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>> Cheers,
>>> > >> > >>>>>>>>> Chris
>>> > >> > >>>>>>>>>
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>>
>>> > >> > >>>>>>>
>>> > >> > >>>>>>
>>> > >> > >>>>>>
>>> > >> > >>>>>>
>>> > >> > >>>>>> --
>>> > >> > >>>>>> -- Guozhang
>>> > >> > >>>>>>
>>> > >> > >>>>>
>>> > >> > >>>>
>>> > >> > >>
>>> > >> > >>
>>> > >> >
>>> > >> >
>>> > >> >
>>> > >>
>>> > >
>>> > >
>>> >
>>>

Re: Thoughts and obesrvations on Samza

Posted by Ben Kirwin <be...@kirw.in>.
Hi all,

Interesting stuff! Jumping in a bit late, but here goes...

I'd definitely be excited about a slimmed-down and more Kafka-specific
Samza -- you don't seem to lose much functionality that people
actually use, and the gains in simplicity / code sharing seem
potentially very large. (I've spent a bunch of time peeling back those
layers of abstraction to get eg. more control over message send order,
and working directly against Kafka's APIs would have been much
easier.) I also like the approach of letting Kafka code do the heavy
lifting and letting stream processing systems build on those -- good,
reusable implementations would be great for the whole
stream-processing ecosystem, and Samza in particular.

On the other hand, I do hope that using Kafka's group membership /
partition assignment / etc. stays optional. As far as I can tell,
~every major stream processing system that uses Kafka has chosen (or
switched to) 'static' partitioning, where each logical task consumes a
fixed set of partitions. When 'dynamic deploying' (a la Storm / Mesos
/ Yarn) the underlying system is already doing failure detection and
transferring work between hosts when machines go down, so using
Kafka's implementation is redundant at best -- and at worst, the
interaction between the two systems can make outages worse.

And thanks to Chris / Jay for getting this ball rolling. Exciting times...

On Tue, Jul 7, 2015 at 2:35 PM, Jay Kreps <ja...@confluent.io> wrote:
> Hey Roger,
>
> I couldn't agree more. We spent a bunch of time talking to people and that
> is exactly the stuff we heard time and again. What makes it hard, of
> course, is that there is some tension between compatibility with what's
> there now and making things better for new users.
>
> I also strongly agree with the importance of multi-language support. We are
> talking now about Java, but for application development use cases people
> want to work in whatever language they are using elsewhere. I think moving
> to a model where Kafka itself does the group membership, lifecycle control,
> and partition assignment has the advantage of putting all that complex
> stuff behind a clean api that the clients are already going to be
> implementing for their consumer, so the added functionality for stream
> processing beyond a consumer becomes very minor.
>
> -Jay
>
> On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <ro...@gmail.com>
> wrote:
>
>> Metamorphosis...nice. :)
>>
>> This has been a great discussion.  As a user of Samza who's recently
>> integrated it into a relatively large organization, I just want to add
>> support to a few points already made.
>>
>> The biggest hurdles to adoption of Samza as it currently exists that I've
>> experienced are:
>> 1) YARN - YARN is overly complex in many environments where Puppet would do
>> just fine but it was the only mechanism to get fault tolerance.
>> 2) Configuration - I think I like the idea of configuring most of the job
>> in code rather than config files.  In general, I think the goal should be
>> to make it harder to make mistakes, especially of the kind where the code
>> expects something and the config doesn't match.  The current config is
>> quite intricate and error-prone.  For example, the application logic may
>> depend on bootstrapping a topic but rather than asserting that in the code,
>> you have to rely on getting the config right.  Likewise with serdes, the
>> Java representations produced by various serdes (JSON, Avro, etc.) are not
>> equivalent so you cannot just reconfigure a serde without changing the
>> code.   It would be nice for jobs to be able to assert what they expect
>> from their input topics in terms of partitioning.  This is getting a little
>> off topic but I was even thinking about creating a "Samza config linter"
>> that would sanity check a set of configs.  Especially in organizations
>> where config is managed by a different team than the application developer,
>> it's very hard to get avoid config mistakes.
>> 3) Java/Scala centric - for many teams (especially DevOps-type folks), the
>> pain of the Java toolchain (maven, slow builds, weak command line support,
>> configuration over convention) really inhibits productivity.  As more and
>> more high-quality clients become available for Kafka, I hope they'll follow
>> Samza's model.  Not sure how much it affects the proposals in this thread
>> but please consider other languages in the ecosystem as well.  From what
>> I've heard, Spark has more Python users than Java/Scala.
>> (FYI, we added a Jython wrapper for the Samza API
>>
>> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
>> and are working on a Yeoman generator
>> https://github.com/Quantiply/generator-rico for Jython/Samza projects to
>> alleviate some of the pain)
>>
>> I also want to underscore Jay's point about improving the user experience.
>> That's a very important factor for adoption.  I think the goal should be to
>> make Samza as easy to get started with as something like Logstash.
>> Logstash is vastly inferior in terms of capabilities to Samza but it's easy
>> to get started and that makes a big difference.
>>
>> Cheers,
>>
>> Roger
>>
>>
>>
>>
>>
>> On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
>> gdfm@apache.org> wrote:
>>
>> > Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
>> winner
>> > :)
>> >
>> > --
>> > Gianmarco
>> >
>> > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <gdfm@apache.org
>> >
>> > wrote:
>> >
>> > > Hi,
>> > >
>> > > @Martin, thanks for you comments.
>> > > Maybe I'm missing some important point, but I think coupling the
>> releases
>> > > is actually a *good* thing.
>> > > To make an example, would it be better if the MR and HDFS components of
>> > > Hadoop had different release schedules?
>> > >
>> > > Actually, keeping the discussion in a single place would make agreeing
>> on
>> > > releases (and backwards compatibility) much easier, as everybody would
>> be
>> > > responsible for the whole codebase.
>> > >
>> > > That said, I like the idea of absorbing samza-core as a sub-project,
>> and
>> > > leave the fancy stuff separate.
>> > > It probably gives 90% of the benefits we have been discussing here.
>> > >
>> > > Cheers,
>> > >
>> > > --
>> > > Gianmarco
>> > >
>> > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
>> > >
>> > >> Hey Martin,
>> > >>
>> > >> I agree coupling release schedules is a downside.
>> > >>
>> > >> Definitely we can try to solve some of the integration problems in
>> > >> Confluent Platform or in other distributions. But I think this ends up
>> > >> being really shallow. I guess I feel to really get a good user
>> > experience
>> > >> the two systems have to kind of feel like part of the same thing and
>> you
>> > >> can't really add that in later--you can put both in the same
>> > downloadable
>> > >> tar file but it doesn't really give a very cohesive feeling. I agree
>> > that
>> > >> ultimately any of the project stuff is as much social and naming as
>> > >> anything else--theoretically two totally independent projects could
>> work
>> > >> to
>> > >> tightly align. In practice this seems to be quite difficult though.
>> > >>
>> > >> For the frameworks--totally agree it would be good to maintain the
>> > >> framework support with the project. In some cases there may not be too
>> > >> much
>> > >> there since the integration gets lighter but I think whatever stubs
>> you
>> > >> need should be included. So no I definitely wasn't trying to imply
>> > >> dropping
>> > >> support for these frameworks, just making the integration lighter by
>> > >> separating process management from partition management.
>> > >>
>> > >> You raise two good points we would have to figure out if we went down
>> > the
>> > >> alignment path:
>> > >> 1. With respect to the name, yeah I think the first question is
>> whether
>> > >> some "re-branding" would be worth it. If so then I think we can have a
>> > big
>> > >> thread on the name. I'm definitely not set on Kafka Streaming or Kafka
>> > >> Streams I was just using them to be kind of illustrative. I agree with
>> > >> your
>> > >> critique of these names, though I think people would get the idea.
>> > >> 2. Yeah you also raise a good point about how to "factor" it. Here are
>> > the
>> > >> options I see (I could get enthusiastic about any of them):
>> > >>    a. One repo for both Kafka and Samza
>> > >>    b. Two repos, retaining the current seperation
>> > >>    c. Two repos, the equivalent of samza-api and samza-core is
>> absorbed
>> > >> almost like a third client
>> > >>
>> > >> Cheers,
>> > >>
>> > >> -Jay
>> > >>
>> > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
>> martin@kleppmann.com>
>> > >> wrote:
>> > >>
>> > >> > Ok, thanks for the clarifications. Just a few follow-up comments.
>> > >> >
>> > >> > - I see the appeal of merging with Kafka or becoming a subproject:
>> the
>> > >> > reasons you mention are good. The risk I see is that release
>> schedules
>> > >> > become coupled to each other, which can slow everyone down, and
>> large
>> > >> > projects with many contributors are harder to manage. (Jakob, can
>> you
>> > >> speak
>> > >> > from experience, having seen a wider range of Hadoop ecosystem
>> > >> projects?)
>> > >> >
>> > >> > Some of the goals of a better unified developer experience could
>> also
>> > be
>> > >> > solved by integrating Samza nicely into a Kafka distribution (such
>> as
>> > >> > Confluent's). I'm not against merging projects if we decide that's
>> the
>> > >> way
>> > >> > to go, just pointing out the same goals can perhaps also be achieved
>> > in
>> > >> > other ways.
>> > >> >
>> > >> > - With regard to dropping the YARN dependency: are you proposing
>> that
>> > >> > Samza doesn't give any help to people wanting to run on
>> > >> YARN/Mesos/AWS/etc?
>> > >> > So the docs would basically have a link to Slider and nothing else?
>> Or
>> > >> > would we maintain integrations with a bunch of popular deployment
>> > >> methods
>> > >> > (e.g. the necessary glue and shell scripts to make Samza work with
>> > >> Slider)?
>> > >> >
>> > >> > I absolutely think it's a good idea to have the "as a library" and
>> > "as a
>> > >> > process" (using Yi's taxonomy) options for people who want them,
>> but I
>> > >> > think there should also be a low-friction path for common "as a
>> > service"
>> > >> > deployment methods, for which we probably need to maintain
>> > integrations.
>> > >> >
>> > >> > - Project naming: "Kafka Streams" seems odd to me, because Kafka is
>> > all
>> > >> > about streams already. Perhaps "Kafka Transformers" or "Kafka
>> Filters"
>> > >> > would be more apt?
>> > >> >
>> > >> > One suggestion: perhaps the core of Samza (stream transformation
>> with
>> > >> > state management -- i.e. the "Samza as a library" bit) could become
>> > >> part of
>> > >> > Kafka, while higher-level tools such as streaming SQL and
>> integrations
>> > >> with
>> > >> > deployment frameworks remain in a separate project? In other words,
>> > >> Kafka
>> > >> > would absorb the proven, stable core of Samza, which would become
>> the
>> > >> > "third Kafka client" mentioned early in this thread. The Samza
>> project
>> > >> > would then target that third Kafka client as its base API, and the
>> > >> project
>> > >> > would be freed up to explore more experimental new horizons.
>> > >> >
>> > >> > Martin
>> > >> >
>> > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
>> > >> >
>> > >> > > Hey Martin,
>> > >> > >
>> > >> > > For the YARN/Mesos/etc decoupling I actually don't think it ties
>> our
>> > >> > hands
>> > >> > > at all, all it does is refactor things. The division of
>> > >> responsibility is
>> > >> > > that Samza core is responsible for task lifecycle, state, and
>> > >> partition
>> > >> > > management (using the Kafka co-ordinator) but it is NOT
>> responsible
>> > >> for
>> > >> > > packaging, configuration deployment or execution of processes. The
>> > >> > problem
>> > >> > > of packaging and starting these processes is
>> > >> > > framework/environment-specific. This leaves individual frameworks
>> to
>> > >> be
>> > >> > as
>> > >> > > fancy or vanilla as they like. So you can get simple stateless
>> > >> support in
>> > >> > > YARN, Mesos, etc using their off-the-shelf app framework (Slider,
>> > >> > Marathon,
>> > >> > > etc). These are well known by people and have nice UIs and a lot
>> of
>> > >> > > flexibility. I don't think they have node affinity as a built in
>> > >> option
>> > >> > > (though I could be wrong). So if we want that we can either wait
>> for
>> > >> them
>> > >> > > to add it or do a custom framework to add that feature (as now).
>> > >> > Obviously
>> > >> > > if you manage things with old-school ops tools (puppet/chef/etc)
>> you
>> > >> get
>> > >> > > locality easily. The nice thing, though, is that all the samza
>> > >> "business
>> > >> > > logic" around partition management and fault tolerance is in Samza
>> > >> core
>> > >> > so
>> > >> > > it is shared across frameworks and the framework specific bit is
>> > just
>> > >> > > whether it is smart enough to try to get the same host when a job
>> is
>> > >> > > restarted.
>> > >> > >
>> > >> > > With respect to the Kafka-alignment, yeah I think the goal would
>> be
>> > >> (a)
>> > >> > > actually get better alignment in user experience, and (b) express
>> > >> this in
>> > >> > > the naming and project branding. Specifically:
>> > >> > > 1. Website/docs, it would be nice for the "transformation" api to
>> be
>> > >> > > discoverable in the main Kafka docs--i.e. be able to explain when
>> to
>> > >> use
>> > >> > > the consumer and when to use the stream processing functionality
>> and
>> > >> lead
>> > >> > > people into that experience.
>> > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that
>> has
>> > >> both
>> > >> > > Kafka and the stream processing part and they actually work
>> > together.
>> > >> > > 3. Unify the programming experience so the client and Samza api
>> > share
>> > >> > > config/monitoring/naming/packaging/etc.
>> > >> > >
>> > >> > > I think sub-projects keep separate committers and can have a
>> > separate
>> > >> > repo,
>> > >> > > but I'm actually not really sure (I can't find a definition of a
>> > >> > subproject
>> > >> > > in Apache).
>> > >> > >
>> > >> > > Basically at a high-level you want the experience to "feel" like a
>> > >> single
>> > >> > > system, not to relatively independent things that are kind of
>> > >> awkwardly
>> > >> > > glued together.
>> > >> > >
>> > >> > > I think if we did that they having naming or branding like "kafka
>> > >> > > streaming" or "kafka streams" or something like that would
>> actually
>> > >> do a
>> > >> > > good job of conveying what it is. I do that this would help
>> adoption
>> > >> > quite
>> > >> > > a lot as it would correctly convey that using Kafka Streaming with
>> > >> Kafka
>> > >> > is
>> > >> > > a fairly seamless experience and Kafka is pretty heavily adopted
>> at
>> > >> this
>> > >> > > point.
>> > >> > >
>> > >> > > Fwiw we actually considered this model originally when open
>> sourcing
>> > >> > Samza,
>> > >> > > however at that time Kafka was relatively unknown and we decided
>> not
>> > >> to
>> > >> > do
>> > >> > > it since we felt it would be limiting. From my point of view the
>> > three
>> > >> > > things have changed (1) Kafka is now really heavily used for
>> stream
>> > >> > > processing, (2) we learned that abstracting out the stream well is
>> > >> > > basically impossible, (3) we learned it is really hard to keep the
>> > two
>> > >> > > things feeling like a single product.
>> > >> > >
>> > >> > > -Jay
>> > >> > >
>> > >> > >
>> > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
>> > >> martin@kleppmann.com>
>> > >> > > wrote:
>> > >> > >
>> > >> > >> Hi all,
>> > >> > >>
>> > >> > >> Lots of good thoughts here.
>> > >> > >>
>> > >> > >> I agree with the general philosophy of tying Samza more firmly to
>> > >> Kafka.
>> > >> > >> After I spent a while looking at integrating other message
>> brokers
>> > >> (e.g.
>> > >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
>> > >> > SystemConsumer
>> > >> > >> tacitly assumes a model so much like Kafka's that pretty much
>> > nobody
>> > >> but
>> > >> > >> Kafka actually implements it. (Databus is perhaps an exception,
>> but
>> > >> it
>> > >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza fully
>> > >> > dependent
>> > >> > >> on Kafka acknowledges that the system-independence was never as
>> > real
>> > >> as
>> > >> > we
>> > >> > >> perhaps made it out to be. The gains of code reuse are real.
>> > >> > >>
>> > >> > >> The idea of decoupling Samza from YARN has also always been
>> > >> appealing to
>> > >> > >> me, for various reasons already mentioned in this thread.
>> Although
>> > >> > making
>> > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems
>> > >> laudable,
>> > >> > I am
>> > >> > >> a little concerned that it will restrict us to a lowest common
>> > >> > denominator.
>> > >> > >> For example, would host affinity (SAMZA-617) still be possible?
>> For
>> > >> jobs
>> > >> > >> with large amounts of state, I think SAMZA-617 would be a big
>> boon,
>> > >> > since
>> > >> > >> restoring state off the changelog on every single restart is
>> > painful,
>> > >> > due
>> > >> > >> to long recovery times. It would be a shame if the decoupling
>> from
>> > >> YARN
>> > >> > >> made host affinity impossible.
>> > >> > >>
>> > >> > >> Jay, a question about the proposed API for instantiating a job in
>> > >> code
>> > >> > >> (rather than a properties file): when submitting a job to a
>> > cluster,
>> > >> is
>> > >> > the
>> > >> > >> idea that the instantiation code runs on a client somewhere,
>> which
>> > >> then
>> > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that
>> > >> code
>> > >> > run
>> > >> > >> on each container that is part of the job (in which case, how
>> does
>> > >> the
>> > >> > job
>> > >> > >> submission to the cluster work)?
>> > >> > >>
>> > >> > >> I agree with Garry that it doesn't feel right to make a 1.0
>> release
>> > >> > with a
>> > >> > >> plan for it to be immediately obsolete. So if this is going to
>> > >> happen, I
>> > >> > >> think it would be more honest to stick with 0.* version numbers
>> > until
>> > >> > the
>> > >> > >> library-ified Samza has been implemented, is stable and widely
>> > used.
>> > >> > >>
>> > >> > >> Should the new Samza be a subproject of Kafka? There is precedent
>> > for
>> > >> > >> tight coupling between different Apache projects (e.g. Curator
>> and
>> > >> > >> Zookeeper, or Slider and YARN), so I think remaining separate
>> would
>> > >> be
>> > >> > ok.
>> > >> > >> Even if Samza is fully dependent on Kafka, there is enough
>> > substance
>> > >> in
>> > >> > >> Samza that it warrants being a separate project. An argument in
>> > >> favour
>> > >> > of
>> > >> > >> merging would be if we think Kafka has a much stronger "brand
>> > >> presence"
>> > >> > >> than Samza; I'm ambivalent on that one. If the Kafka project is
>> > >> willing
>> > >> > to
>> > >> > >> endorse Samza as the "official" way of doing stateful stream
>> > >> > >> transformations, that would probably have much the same effect as
>> > >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
>> > >> > >> collaboration between the two projects will be needed in any
>> case.
>> > >> > >>
>> > >> > >> From a project management perspective, I guess the "new Samza"
>> > would
>> > >> > have
>> > >> > >> to be developed on a branch alongside ongoing maintenance of the
>> > >> current
>> > >> > >> line of development? I think it would be important to continue
>> > >> > supporting
>> > >> > >> existing users, and provide a graceful migration path to the new
>> > >> > version.
>> > >> > >> Leaving the current versions unsupported and forcing people to
>> > >> rewrite
>> > >> > >> their jobs would send a bad signal.
>> > >> > >>
>> > >> > >> Best,
>> > >> > >> Martin
>> > >> > >>
>> > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
>> > >> > >>
>> > >> > >>> Hey Garry,
>> > >> > >>>
>> > >> > >>> Yeah that's super frustrating. I'd be happy to chat more about
>> > this
>> > >> if
>> > >> > >>> you'd be interested. I think Chris and I started with the idea
>> of
>> > >> "what
>> > >> > >>> would it take to make Samza a kick-ass ingestion tool" but
>> > >> ultimately
>> > >> > we
>> > >> > >>> kind of came around to the idea that ingestion and
>> transformation
>> > >> had
>> > >> > >>> pretty different needs and coupling the two made things hard.
>> > >> > >>>
>> > >> > >>> For what it's worth I think copycat (KIP-26) actually will do
>> what
>> > >> you
>> > >> > >> are
>> > >> > >>> looking for.
>> > >> > >>>
>> > >> > >>> With regard to your point about slider, I don't necessarily
>> > >> disagree.
>> > >> > >> But I
>> > >> > >>> think getting good YARN support is quite doable and I think we
>> can
>> > >> make
>> > >> > >>> that work well. I think the issue this proposal solves is that
>> > >> > >> technically
>> > >> > >>> it is pretty hard to support multiple cluster management systems
>> > the
>> > >> > way
>> > >> > >>> things are now, you need to write an "app master" or "framework"
>> > for
>> > >> > each
>> > >> > >>> and they are all a little different so testing is really hard.
>> In
>> > >> the
>> > >> > >>> absence of this we have been stuck with just YARN which has
>> > >> fantastic
>> > >> > >>> penetration in the Hadoopy part of the org, but zero penetration
>> > >> > >> elsewhere.
>> > >> > >>> Given the huge amount of work being put in to slider, marathon,
>> > aws
>> > >> > >>> tooling, not to mention the umpteen related packaging
>> technologies
>> > >> > people
>> > >> > >>> want to use (Docker, Kubernetes, various cloud-specific deploy
>> > >> tools,
>> > >> > >> etc)
>> > >> > >>> I really think it is important to get this right.
>> > >> > >>>
>> > >> > >>> -Jay
>> > >> > >>>
>> > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>> > >> > >>> g.turkington@improvedigital.com> wrote:
>> > >> > >>>
>> > >> > >>>> Hi all,
>> > >> > >>>>
>> > >> > >>>> I think the question below re does Samza become a sub-project
>> of
>> > >> Kafka
>> > >> > >>>> highlights the broader point around migration. Chris mentions
>> > >> Samza's
>> > >> > >>>> maturity is heading towards a v1 release but I'm not sure it
>> > feels
>> > >> > >> right to
>> > >> > >>>> launch a v1 then immediately plan to deprecate most of it.
>> > >> > >>>>
>> > >> > >>>> From a selfish perspective I have some guys who have started
>> > >> working
>> > >> > >> with
>> > >> > >>>> Samza and building some new consumers/producers was next up.
>> > Sounds
>> > >> > like
>> > >> > >>>> that is absolutely not the direction to go. I need to look into
>> > the
>> > >> > KIP
>> > >> > >> in
>> > >> > >>>> more detail but for me the attractiveness of adding new Samza
>> > >> > >>>> consumer/producers -- even if yes all they were doing was
>> really
>> > >> > getting
>> > >> > >>>> data into and out of Kafka --  was to avoid  having to worry
>> > about
>> > >> the
>> > >> > >>>> lifecycle management of external clients. If there is a generic
>> > >> Kafka
>> > >> > >>>> ingress/egress layer that I can plug a new connector into and
>> > have
>> > >> a
>> > >> > >> lot of
>> > >> > >>>> the heavy lifting re scale and reliability done for me then it
>> > >> gives
>> > >> > me
>> > >> > >> all
>> > >> > >>>> the pushing new consumers/producers would. If not then it
>> > >> complicates
>> > >> > my
>> > >> > >>>> operational deployments.
>> > >> > >>>>
>> > >> > >>>> Which is similar to my other question with the proposal -- if
>> we
>> > >> > build a
>> > >> > >>>> fully available/stand-alone Samza plus the requisite shims to
>> > >> > integrate
>> > >> > >>>> with Slider etc I suspect the former may be a lot more work
>> than
>> > we
>> > >> > >> think.
>> > >> > >>>> We may make it much easier for a newcomer to get something
>> > running
>> > >> but
>> > >> > >>>> having them step up and get a reliable production deployment
>> may
>> > >> still
>> > >> > >>>> dominate mailing list  traffic, if for different reasons than
>> > >> today.
>> > >> > >>>>
>> > >> > >>>> Don't get me wrong -- I'm comfortable with making the Samza
>> > >> dependency
>> > >> > >> on
>> > >> > >>>> Kafka much more explicit and I absolutely see the benefits  in
>> > the
>> > >> > >>>> reduction of duplication and clashing
>> terminologies/abstractions
>> > >> that
>> > >> > >>>> Chris/Jay describe. Samza as a library would likely be a very
>> > nice
>> > >> > tool
>> > >> > >> to
>> > >> > >>>> add to the Kafka ecosystem. I just have the concerns above re
>> the
>> > >> > >>>> operational side.
>> > >> > >>>>
>> > >> > >>>> Garry
>> > >> > >>>>
>> > >> > >>>> -----Original Message-----
>> > >> > >>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
>> > >> > >>>> Sent: 02 July 2015 12:56
>> > >> > >>>> To: dev@samza.apache.org
>> > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
>> > >> > >>>>
>> > >> > >>>> Very interesting thoughts.
>> > >> > >>>> From outside, I have always perceived Samza as a computing
>> layer
>> > >> over
>> > >> > >>>> Kafka.
>> > >> > >>>>
>> > >> > >>>> The question, maybe a bit provocative, is "should Samza be a
>> > >> > sub-project
>> > >> > >>>> of Kafka then?"
>> > >> > >>>> Or does it make sense to keep it as a separate project with a
>> > >> separate
>> > >> > >>>> governance?
>> > >> > >>>>
>> > >> > >>>> Cheers,
>> > >> > >>>>
>> > >> > >>>> --
>> > >> > >>>> Gianmarco
>> > >> > >>>>
>> > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
>> wrote:
>> > >> > >>>>
>> > >> > >>>>> Overall, I agree to couple with Kafka more tightly. Because
>> > Samza
>> > >> de
>> > >> > >>>>> facto is based on Kafka, and it should leverage what Kafka
>> has.
>> > At
>> > >> > the
>> > >> > >>>>> same time, Kafka does not need to reinvent what Samza already
>> > >> has. I
>> > >> > >>>>> also like the idea of separating the ingestion and
>> > transformation.
>> > >> > >>>>>
>> > >> > >>>>> But it is a little difficult for me to image how the Samza
>> will
>> > >> look
>> > >> > >>>> like.
>> > >> > >>>>> And I feel Chris and Jay have a little difference in terms of
>> > how
>> > >> > >>>>> Samza should look like.
>> > >> > >>>>>
>> > >> > >>>>> *** Will it look like what Jay's code shows (A client of
>> Kakfa)
>> > ?
>> > >> And
>> > >> > >>>>> user's application code calls this client?
>> > >> > >>>>>
>> > >> > >>>>> 1. If we make Samza be a library of Kafka (like what the code
>> > >> shows),
>> > >> > >>>>> how do we implement auto-balance and fault-tolerance? Are they
>> > >> taken
>> > >> > >>>>> care by the Kafka broker or other mechanism, such as "Samza
>> > >> worker"
>> > >> > >>>>> (just make up the name) ?
>> > >> > >>>>>
>> > >> > >>>>> 2. What about other features, such as auto-scaling, shared
>> > state,
>> > >> > >>>>> monitoring?
>> > >> > >>>>>
>> > >> > >>>>>
>> > >> > >>>>> *** If we have Samza standalone, (is this what Chris
>> suggests?)
>> > >> > >>>>>
>> > >> > >>>>> 1. we still need to ingest data from Kakfa and produce to it.
>> > >> Then it
>> > >> > >>>>> becomes the same as what Samza looks like now, except it does
>> > not
>> > >> > rely
>> > >> > >>>>> on Yarn anymore.
>> > >> > >>>>>
>> > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's metrics,
>> > logs,
>> > >> > >>>>> etc? Use Kafka code as the dependency?
>> > >> > >>>>>
>> > >> > >>>>>
>> > >> > >>>>> Thanks,
>> > >> > >>>>>
>> > >> > >>>>> Fang, Yan
>> > >> > >>>>> yanfang724@gmail.com
>> > >> > >>>>>
>> > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
>> > wangguoz@gmail.com
>> > >> >
>> > >> > >>>> wrote:
>> > >> > >>>>>
>> > >> > >>>>>> Read through the code example and it looks good to me. A few
>> > >> > >>>>>> thoughts regarding deployment:
>> > >> > >>>>>>
>> > >> > >>>>>> Today Samza deploys as executable runnable like:
>> > >> > >>>>>>
>> > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
>> > >> > >>>> --config-path=file://...
>> > >> > >>>>>>
>> > >> > >>>>>> And this proposal advocate for deploying Samza more as
>> embedded
>> > >> > >>>>>> libraries in user application code (ignoring the terminology
>> > >> since
>> > >> > >>>>>> it is not the
>> > >> > >>>>> same
>> > >> > >>>>>> as the prototype code):
>> > >> > >>>>>>
>> > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread thread =
>> > new
>> > >> > >>>>>> Thread(task); thread.start();
>> > >> > >>>>>>
>> > >> > >>>>>> I think both of these deployment modes are important for
>> > >> different
>> > >> > >>>>>> types
>> > >> > >>>>> of
>> > >> > >>>>>> users. That said, I think making Samza purely standalone is
>> > still
>> > >> > >>>>>> sufficient for either runnable or library modes.
>> > >> > >>>>>>
>> > >> > >>>>>> Guozhang
>> > >> > >>>>>>
>> > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
>> jay@confluent.io>
>> > >> > wrote:
>> > >> > >>>>>>
>> > >> > >>>>>>> Looks like gmail mangled the code example, it was supposed
>> to
>> > >> look
>> > >> > >>>>>>> like
>> > >> > >>>>>>> this:
>> > >> > >>>>>>>
>> > >> > >>>>>>> Properties props = new Properties();
>> > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
>> > >> StreamingConfig
>> > >> > >>>>>>> config = new StreamingConfig(props);
>> > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
>> > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
>> > >> > >>>>>>> config.serialization(new StringSerializer(), new
>> > >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
>> > >> > >>>>>>> KafkaStreaming(config); container.run();
>> > >> > >>>>>>>
>> > >> > >>>>>>> -Jay
>> > >> > >>>>>>>
>> > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
>> jay@confluent.io
>> > >
>> > >> > >>>> wrote:
>> > >> > >>>>>>>
>> > >> > >>>>>>>> Hey guys,
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> This came out of some conversations Chris and I were having
>> > >> > >>>>>>>> around
>> > >> > >>>>>>> whether
>> > >> > >>>>>>>> it would make sense to use Samza as a kind of data
>> ingestion
>> > >> > >>>>> framework
>> > >> > >>>>>>> for
>> > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This
>> kind
>> > of
>> > >> > >>>>>> combined
>> > >> > >>>>>>>> with complaints around config and YARN and the discussion
>> > >> around
>> > >> > >>>>>>>> how
>> > >> > >>>>> to
>> > >> > >>>>>>>> best do a standalone mode.
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> So the thought experiment was, given that Samza was
>> basically
>> > >> > >>>>>>>> already totally Kafka specific, what if you just embraced
>> > that
>> > >> > >>>>>>>> and turned it
>> > >> > >>>>>> into
>> > >> > >>>>>>>> something less like a heavyweight framework and more like a
>> > >> > >>>>>>>> third
>> > >> > >>>>> Kafka
>> > >> > >>>>>>>> client--a kind of "producing consumer" with state
>> management
>> > >> > >>>>>> facilities.
>> > >> > >>>>>>>> Basically a library. Instead of a complex stream processing
>> > >> > >>>>>>>> framework
>> > >> > >>>>>>> this
>> > >> > >>>>>>>> would actually be a very simple thing, not much more
>> > >> complicated
>> > >> > >>>>>>>> to
>> > >> > >>>>> use
>> > >> > >>>>>>> or
>> > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we thought
>> about
>> > >> it
>> > >> > >>>>>>>> a
>> > >> > >>>>> lot
>> > >> > >>>>>> of
>> > >> > >>>>>>>> what Samza (and the other stream processing systems were
>> > doing)
>> > >> > >>>>> seemed
>> > >> > >>>>>>> like
>> > >> > >>>>>>>> kind of a hangover from MapReduce.
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> Of course you need to ingest/output data to and from the
>> > stream
>> > >> > >>>>>>>> processing. But when we actually looked into how that would
>> > >> > >>>>>>>> work,
>> > >> > >>>>> Samza
>> > >> > >>>>>>>> isn't really an ideal data ingestion framework for a bunch
>> of
>> > >> > >>>>> reasons.
>> > >> > >>>>>> To
>> > >> > >>>>>>>> really do that right you need a pretty different internal
>> > data
>> > >> > >>>>>>>> model
>> > >> > >>>>>> and
>> > >> > >>>>>>>> set of apis. So what if you split them and had an api for
>> > Kafka
>> > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
>> > >> Kafka
>> > >> > >>>>>>>> transformation (Samza).
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> This would also allow really embracing the same terminology
>> > and
>> > >> > >>>>>>>> conventions. One complaint about the current state is that
>> > the
>> > >> > >>>>>>>> two
>> > >> > >>>>>>> systems
>> > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs
>> "topic"
>> > >> and
>> > >> > >>>>>>> different
>> > >> > >>>>>>>> config and monitoring systems means you kind of have to
>> learn
>> > >> > >>>>>>>> Kafka's
>> > >> > >>>>>>> way,
>> > >> > >>>>>>>> then learn Samza's slightly different way, then kind of
>> > >> > >>>>>>>> understand
>> > >> > >>>>> how
>> > >> > >>>>>>> they
>> > >> > >>>>>>>> map to each other, which having walked a few people through
>> > >> this
>> > >> > >>>>>>>> is surprisingly tricky for folks to get.
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> Since I have been spending a lot of time on airplanes I
>> > hacked
>> > >> > >>>>>>>> up an ernest but still somewhat incomplete prototype of
>> what
>> > >> > >>>>>>>> this would
>> > >> > >>>>> look
>> > >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka as it
>> > >> > >>>>>>>> required a
>> > >> > >>>>>> few
>> > >> > >>>>>>>> changes to the new consumer. Here is the code:
>> > >> > >>>>>>>>
>> > >> > >>>>>>>>
>> > >> > >>>>>>>
>> > >> > >>>>>>
>> > >> > >>>>>
>> > >> >
>> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> > >> > >>>>> /apache/kafka/clients/streaming
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> For the purpose of the prototype I just liberally renamed
>> > >> > >>>>>>>> everything
>> > >> > >>>>> to
>> > >> > >>>>>>>> try to align it with Kafka with no regard for
>> compatibility.
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> To use this would be something like this:
>> > >> > >>>>>>>> Properties props = new Properties();
>> > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
>> > >> > >>>>>>>> StreamingConfig config = new
>> > >> > >>>>> StreamingConfig(props);
>> > >> > >>>>>>> config.subscribe("test-topic-1",
>> > >> > >>>>>>>> "test-topic-2");
>> > >> config.processor(ExampleStreamProcessor.class);
>> > >> > >>>>>>> config.serialization(new
>> > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
>> KafkaStreaming
>> > >> > >>>>>> container =
>> > >> > >>>>>>>> new KafkaStreaming(config); container.run();
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
>> > StreamProcessor
>> > >> > >>>>>>>> is basically StreamTask.
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> So rather than putting all the class names in a file and
>> then
>> > >> > >>>>>>>> having
>> > >> > >>>>>> the
>> > >> > >>>>>>>> job assembled by reflection, you just instantiate the
>> > container
>> > >> > >>>>>>>> programmatically. Work is balanced over however many
>> > instances
>> > >> > >>>>>>>> of
>> > >> > >>>>> this
>> > >> > >>>>>>> are
>> > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new tasks are
>> > >> added
>> > >> > >>>>>>>> to
>> > >> > >>>>> the
>> > >> > >>>>>>>> existing containers without shutting them down).
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> We would provide some glue for running this stuff in YARN
>> via
>> > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their
>> tools
>> > >> > >>>>>>>> but from the
>> > >> > >>>>>> point
>> > >> > >>>>>>> of
>> > >> > >>>>>>>> view of these frameworks these stream processing jobs are
>> > just
>> > >> > >>>>>> stateless
>> > >> > >>>>>>>> services that can come and go and expand and contract at
>> > will.
>> > >> > >>>>>>>> There
>> > >> > >>>>> is
>> > >> > >>>>>>> no
>> > >> > >>>>>>>> more custom scheduler.
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> Here are some relevant details:
>> > >> > >>>>>>>>
>> > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get larger if
>> we
>> > >> > >>>>>>>>  productionized but not vastly larger. We really do get a
>> ton
>> > >> > >>>>>>>> of
>> > >> > >>>>>>> leverage
>> > >> > >>>>>>>>  out of Kafka.
>> > >> > >>>>>>>>  2. Partition management is fully delegated to the new
>> > >> consumer.
>> > >> > >>>>> This
>> > >> > >>>>>>>>  is nice since now any partition management strategy
>> > available
>> > >> > >>>>>>>> to
>> > >> > >>>>>> Kafka
>> > >> > >>>>>>>>  consumer is also available to Samza (and vice versa) and
>> > with
>> > >> > >>>>>>>> the
>> > >> > >>>>>>> exact
>> > >> > >>>>>>>>  same configs.
>> > >> > >>>>>>>>  3. It supports state as well as state reuse
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> -Jay
>> > >> > >>>>>>>>
>> > >> > >>>>>>>>
>> > >> > >>>>>>>>
>> > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>> > >> > >>>>>> criccomini@apache.org>
>> > >> > >>>>>>>> wrote:
>> > >> > >>>>>>>>
>> > >> > >>>>>>>>> Hey all,
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> I have had some discussions with Samza engineers at
>> LinkedIn
>> > >> > >>>>>>>>> and
>> > >> > >>>>>>> Confluent
>> > >> > >>>>>>>>> and we came up with a few observations and would like to
>> > >> > >>>>>>>>> propose
>> > >> > >>>>> some
>> > >> > >>>>>>>>> changes.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> We've observed some things that I want to call out about
>> > >> > >>>>>>>>> Samza's
>> > >> > >>>>>> design,
>> > >> > >>>>>>>>> and I'd like to propose some changes.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
>> > >> > >>>>>>>>> * Samza is too pluggable.
>> > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
>> consumer
>> > >> > >>>>>>>>> APIs
>> > >> > >>>>> are
>> > >> > >>>>>>>>> trying to solve a lot of the same problems.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> All three of these issues are related, but I'll address
>> them
>> > >> in
>> > >> > >>>>> order.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Deployment
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic deployment
>> > >> > >>>>>>>>> scheduler
>> > >> > >>>>>> such
>> > >> > >>>>>>>>> as
>> > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet
>> that
>> > >> > >>>>>>>>> there
>> > >> > >>>>>> would
>> > >> > >>>>>>>>> be
>> > >> > >>>>>>>>> one or two winners in this area, and we could support
>> them,
>> > >> and
>> > >> > >>>>>>>>> the
>> > >> > >>>>>> rest
>> > >> > >>>>>>>>> would go away. In reality, there are many variations.
>> > >> > >>>>>>>>> Furthermore,
>> > >> > >>>>>> many
>> > >> > >>>>>>>>> people still prefer to just start their processors like
>> > normal
>> > >> > >>>>>>>>> Java processes, and use traditional deployment scripts
>> such
>> > as
>> > >> > >>>>>>>>> Fabric,
>> > >> > >>>>>> Chef,
>> > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users makes
>> the
>> > >> > >>>>>>>>> Samza start-up process really painful for first time
>> users.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit of a
>> > >> > >>>>>>>>> mis-fire
>> > >> > >>>>>> because
>> > >> > >>>>>>>>> of
>> > >> > >>>>>>>>> a fundamental misunderstanding between the nature of batch
>> > >> jobs
>> > >> > >>>>>>>>> and
>> > >> > >>>>>>> stream
>> > >> > >>>>>>>>> processing jobs. Early on, we made conscious effort to
>> favor
>> > >> > >>>>>>>>> the
>> > >> > >>>>>> Hadoop
>> > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked and was
>> > well
>> > >> > >>>>>>> understood.
>> > >> > >>>>>>>>> One thing that we missed was that batch jobs have a
>> definite
>> > >> > >>>>>> beginning,
>> > >> > >>>>>>>>> and
>> > >> > >>>>>>>>> end, and stream processing jobs don't (usually). This
>> leads
>> > to
>> > >> > >>>>>>>>> a
>> > >> > >>>>> much
>> > >> > >>>>>>>>> simpler scheduling problem for stream processors. You
>> > >> basically
>> > >> > >>>>>>>>> just
>> > >> > >>>>>>> need
>> > >> > >>>>>>>>> to find a place to start the processor, and start it. The
>> > way
>> > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
>> > >> > >>>>>>>>> being "full". We always
>> > >> > >>>>>> add
>> > >> > >>>>>>>>> more machines. The problem with coupling Samza with a
>> > >> scheduler
>> > >> > >>>>>>>>> is
>> > >> > >>>>>> that
>> > >> > >>>>>>>>> Samza (as a framework) now has to handle deployment. This
>> > >> pulls
>> > >> > >>>>>>>>> in a
>> > >> > >>>>>>> bunch
>> > >> > >>>>>>>>> of things such as configuration distribution (config
>> > stream),
>> > >> > >>>>>>>>> shell
>> > >> > >>>>>>> scrips
>> > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
>> stuff),
>> > >> etc.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Another reason for requiring dynamic deployment was to
>> > support
>> > >> > >>>>>>>>> data locality. If you want to have locality, you need to
>> put
>> > >> > >>>>>>>>> your
>> > >> > >>>>>> processors
>> > >> > >>>>>>>>> close to the data they're processing. Upon further
>> > >> > >>>>>>>>> investigation,
>> > >> > >>>>>>> though,
>> > >> > >>>>>>>>> this feature is not that beneficial. There is some good
>> > >> > >>>>>>>>> discussion
>> > >> > >>>>>> about
>> > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
>> > >> > >>>>>>>>> Map/Reduce
>> > >> > >>>>>> path,
>> > >> > >>>>>>>>> but
>> > >> > >>>>>>>>> there are some fundamental differences between HDFS and
>> > Kafka.
>> > >> > >>>>>>>>> HDFS
>> > >> > >>>>>> has
>> > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to less
>> > >> > >>>>>>>>> optimization potential with stream processors on top of
>> > Kafka.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't have
>> > any
>> > >> > >>>>>>>>> built
>> > >> > >>>>> in
>> > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
>> > >> > >>>>>>>>> deployment scheduling system to handle restarts when a
>> > >> > >>>>>>>>> processor dies. This has
>> > >> > >>>>>>> made
>> > >> > >>>>>>>>> it very difficult to write a standalone Samza container
>> > >> > >>>> (SAMZA-516).
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Pluggability
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> In some cases pluggability is good, but I think that we've
>> > >> gone
>> > >> > >>>>>>>>> too
>> > >> > >>>>>> far
>> > >> > >>>>>>>>> with it. Currently, Samza has:
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> * Pluggable config.
>> > >> > >>>>>>>>> * Pluggable metrics.
>> > >> > >>>>>>>>> * Pluggable deployment systems.
>> > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
>> > SystemProducer,
>> > >> > >>>> etc).
>> > >> > >>>>>>>>> * Pluggable serdes.
>> > >> > >>>>>>>>> * Pluggable storage engines.
>> > >> > >>>>>>>>> * Pluggable strategies for just about every component
>> > >> > >>>>> (MessageChooser,
>> > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> There's probably more that I've forgotten, as well. Some
>> of
>> > >> > >>>>>>>>> these
>> > >> > >>>>> are
>> > >> > >>>>>>>>> useful, but some have proven not to be. This all comes at
>> a
>> > >> cost:
>> > >> > >>>>>>>>> complexity. This complexity is making it harder for our
>> > users
>> > >> > >>>>>>>>> to
>> > >> > >>>>> pick
>> > >> > >>>>>> up
>> > >> > >>>>>>>>> and use Samza out of the box. It also makes it difficult
>> for
>> > >> > >>>>>>>>> Samza developers to reason about what the characteristics
>> of
>> > >> > >>>>>>>>> the container (since the characteristics change depending
>> on
>> > >> > >>>>>>>>> which plugins are use).
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> The issues with pluggability are most visible in the
>> System
>> > >> APIs.
>> > >> > >>>>> What
>> > >> > >>>>>>>>> Samza really requires to be functional is Kafka as its
>> > >> > >>>>>>>>> transport
>> > >> > >>>>>> layer.
>> > >> > >>>>>>>>> But
>> > >> > >>>>>>>>> we've conflated two unrelated use cases into one API:
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> 1. Get data into/out of Kafka.
>> > >> > >>>>>>>>> 2. Process the data in Kafka.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> The current System API supports both of these use cases.
>> The
>> > >> > >>>>>>>>> problem
>> > >> > >>>>>> is,
>> > >> > >>>>>>>>> we
>> > >> > >>>>>>>>> actually want different features for each use case. By
>> > >> papering
>> > >> > >>>>>>>>> over
>> > >> > >>>>>>> these
>> > >> > >>>>>>>>> two use cases, and providing a single API, we've
>> introduced
>> > a
>> > >> > >>>>>>>>> ton of
>> > >> > >>>>>>> leaky
>> > >> > >>>>>>>>> abstractions.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> For example, what we'd really like in (2) is to have
>> > >> > >>>>>>>>> monotonically increasing longs for offsets (like Kafka).
>> > This
>> > >> > >>>>>>>>> would be at odds
>> > >> > >>>>> with
>> > >> > >>>>>>> (1),
>> > >> > >>>>>>>>> though, since different systems have different
>> > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
>> > >> > >>>>>>>>> There was discussion both on the mailing list and the SQL
>> > >> JIRAs
>> > >> > >>>>> about
>> > >> > >>>>>>> the
>> > >> > >>>>>>>>> need for this.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> The same thing holds true for replayability. Kafka allows
>> us
>> > >> to
>> > >> > >>>>> rewind
>> > >> > >>>>>>>>> when
>> > >> > >>>>>>>>> we have a failure. Many other systems don't. In some
>> cases,
>> > >> > >>>>>>>>> systems
>> > >> > >>>>>>> return
>> > >> > >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
>> > because
>> > >> > >>>>>>>>> they
>> > >> > >>>>>> have
>> > >> > >>>>>>> no
>> > >> > >>>>>>>>> offsets.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Partitioning is another example. Kafka supports
>> > partitioning,
>> > >> > >>>>>>>>> but
>> > >> > >>>>> many
>> > >> > >>>>>>>>> systems don't. We model this by having a single partition
>> > for
>> > >> > >>>>>>>>> those systems. Still, other systems model partitioning
>> > >> > >>>> differently (e.g.
>> > >> > >>>>>>>>> Kinesis).
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating streams
>> > in
>> > >> a
>> > >> > >>>>>>>>> system-agnostic way is almost impossible. As is modeling
>> > >> > >>>>>>>>> metadata
>> > >> > >>>>> for
>> > >> > >>>>>>> the
>> > >> > >>>>>>>>> system (replication factor, partitions, location, etc).
>> The
>> > >> > >>>>>>>>> list
>> > >> > >>>>> goes
>> > >> > >>>>>>> on.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Duplicate work
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's consumer
>> > and
>> > >> > >>>>> producer
>> > >> > >>>>>>>>> APIs
>> > >> > >>>>>>>>> had a relatively weak feature set. On the consumer-side,
>> you
>> > >> > >>>>>>>>> had two
>> > >> > >>>>>>>>> options: use the high level consumer, or the simple
>> > consumer.
>> > >> > >>>>>>>>> The
>> > >> > >>>>>>> problem
>> > >> > >>>>>>>>> with the high-level consumer was that it controlled your
>> > >> > >>>>>>>>> offsets, partition assignments, and the order in which you
>> > >> > >>>>>>>>> received messages. The
>> > >> > >>>>> problem
>> > >> > >>>>>>>>> with
>> > >> > >>>>>>>>> the simple consumer is that it's not simple. It's basic.
>> You
>> > >> > >>>>>>>>> end up
>> > >> > >>>>>>> having
>> > >> > >>>>>>>>> to handle a lot of really low-level stuff that you
>> > shouldn't.
>> > >> > >>>>>>>>> We
>> > >> > >>>>>> spent a
>> > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
>> robust.
>> > >> It
>> > >> > >>>>>>>>> also allows us to support some cool features:
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> * Per-partition message ordering and prioritization.
>> > >> > >>>>>>>>> * Tight control over partition assignment to support
>> joins,
>> > >> > >>>>>>>>> global
>> > >> > >>>>>> state
>> > >> > >>>>>>>>> (if we want to implement it :)), etc.
>> > >> > >>>>>>>>> * Tight control over offset checkpointing.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> What we didn't realize at the time is that these features
>> > >> > >>>>>>>>> should
>> > >> > >>>>>>> actually
>> > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
>> stream
>> > >> > >>>>>> processors)
>> > >> > >>>>>>>>> end up wanting to do things like joins and partition
>> > >> > >>>>>>>>> assignment. The
>> > >> > >>>>>>> Kafka
>> > >> > >>>>>>>>> community has come to the same conclusion. They're adding
>> a
>> > >> ton
>> > >> > >>>>>>>>> of upgrades into their new Kafka consumer implementation.
>> > To a
>> > >> > >>>>>>>>> large extent,
>> > >> > >>>>> it's
>> > >> > >>>>>>>>> duplicate work to what we've already done in Samza.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
>> > approach
>> > >> > >>>>>>>>> to
>> > >> > >>>>>> Samza's
>> > >> > >>>>>>>>> KafkaCheckpointManager implementation for handling offset
>> > >> > >>>>>> checkpointing.
>> > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature stores
>> > >> offset
>> > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them from
>> > the
>> > >> > >>>>>>>>> broker.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> A lot of this seems like a waste, since we could have
>> shared
>> > >> > >>>>>>>>> the
>> > >> > >>>>> work
>> > >> > >>>>>> if
>> > >> > >>>>>>>>> it
>> > >> > >>>>>>>>> had been done in Kafka from the get-go.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Vision
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> All of this leads me to a rather radical proposal. Samza
>> is
>> > >> > >>>>> relatively
>> > >> > >>>>>>>>> stable at this point. I'd venture to say that we're near a
>> > 1.0
>> > >> > >>>>>> release.
>> > >> > >>>>>>>>> I'd
>> > >> > >>>>>>>>> like to propose that we take what we've learned, and begin
>> > >> > >>>>>>>>> thinking
>> > >> > >>>>>>> about
>> > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were starting
>> > >> from
>> > >> > >>>>>> scratch?
>> > >> > >>>>>>>>> My
>> > >> > >>>>>>>>> proposal is to:
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
>> > >> > >>>>>>>>> processors, and eliminate all direct dependences on YARN,
>> > >> Mesos,
>> > >> > >>>> etc.
>> > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as the
>> > stream
>> > >> > >>>>>> processing
>> > >> > >>>>>>>>> layer.
>> > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
>> > >> > >>>>>>>>> config
>> > >> > >>>>>>> systems,
>> > >> > >>>>>>>>> and simply use Kafka's instead.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> This would fix all of the issues that I outlined above. It
>> > >> > >>>>>>>>> should
>> > >> > >>>>> also
>> > >> > >>>>>>>>> shrink the Samza code base pretty dramatically. Supporting
>> > >> only
>> > >> > >>>>>>>>> a standalone container will allow Samza to be executed on
>> > YARN
>> > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most
>> other
>> > >> > >>>>>>>>> in-house
>> > >> > >>>>>>> deployment
>> > >> > >>>>>>>>> systems. This should make life a lot easier for new users.
>> > >> > >>>>>>>>> Imagine
>> > >> > >>>>>>> having
>> > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in mailing
>> > >> list
>> > >> > >>>>>> traffic
>> > >> > >>>>>>>>> will be pretty dramatic.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The reality
>> > is,
>> > >> > >>>>> everyone
>> > >> > >>>>>>>>> that
>> > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically
>> require
>> > >> it
>> > >> > >>>>>> already
>> > >> > >>>>>>> in
>> > >> > >>>>>>>>> order for most features to work. Those that are using
>> other
>> > >> > >>>>>>>>> systems
>> > >> > >>>>>> are
>> > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and then
>> they
>> > do
>> > >> > >>>>>>>>> the processing on top. There is already discussion (
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>
>> > >> > >>>>>>
>> > >> > >>>>>
>> > >> >
>> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> > >> > >>>>> 767
>> > >> > >>>>>>>>> )
>> > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Once we make the call to couple with Kafka, we can
>> leverage
>> > a
>> > >> > >>>>>>>>> ton of
>> > >> > >>>>>>> their
>> > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own config,
>> > >> > >>>>>>>>> metrics,
>> > >> > >>>>> etc.
>> > >> > >>>>>>> We
>> > >> > >>>>>>>>> can all share the same libraries, and make them better.
>> This
>> > >> > >>>>>>>>> will
>> > >> > >>>>> also
>> > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and will let
>> > us
>> > >> > >>>>> leverage
>> > >> > >>>>>>>>> their offset management and partition management, rather
>> > than
>> > >> > >>>>>>>>> having
>> > >> > >>>>>> our
>> > >> > >>>>>>>>> own. All of the coordinator stream code would go away, as
>> > >> would
>> > >> > >>>>>>>>> most
>> > >> > >>>>>> of
>> > >> > >>>>>>>>> the
>> > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
>> > partition
>> > >> > >>>>>>> management
>> > >> > >>>>>>>>> features into the Kafka broker, but they're already moving
>> > in
>> > >> > >>>>>>>>> that direction with the new consumer API. The features we
>> > have
>> > >> > >>>>>>>>> for
>> > >> > >>>>>> partition
>> > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like they
>> should
>> > >> be
>> > >> > >>>>>>>>> in
>> > >> > >>>>>> Kafka
>> > >> > >>>>>>>>> anyway. There will always be some niche usages which will
>> > >> > >>>>>>>>> require
>> > >> > >>>>>> extra
>> > >> > >>>>>>>>> care and hence full control over partition assignments
>> much
>> > >> > >>>>>>>>> like the
>> > >> > >>>>>>> Kafka
>> > >> > >>>>>>>>> low level consumer api. These would continue to be
>> > supported.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> These items will be good for the Samza community. They'll
>> > make
>> > >> > >>>>>>>>> Samza easier to use, and make it easier for developers to
>> > add
>> > >> > >>>>>>>>> new features.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat backwards
>> > >> > >>>>> incompatible
>> > >> > >>>>>>>>> change). If we choose to go this route, it's important
>> that
>> > we
>> > >> > >>>>> openly
>> > >> > >>>>>>>>> communicate how we're going to provide a migration path
>> from
>> > >> > >>>>>>>>> the
>> > >> > >>>>>>> existing
>> > >> > >>>>>>>>> APIs to the new ones (if we make incompatible changes). I
>> > >> think
>> > >> > >>>>>>>>> at a minimum, we'd probably need to provide a wrapper to
>> > allow
>> > >> > >>>>>>>>> existing StreamTask implementations to continue running on
>> > the
>> > >> > >>>> new container.
>> > >> > >>>>>>> It's
>> > >> > >>>>>>>>> also important that we openly communicate about timing,
>> and
>> > >> > >>>>>>>>> stages
>> > >> > >>>>> of
>> > >> > >>>>>>> the
>> > >> > >>>>>>>>> migration.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> If you made it this far, I'm sure you have opinions. :)
>> > Please
>> > >> > >>>>>>>>> send
>> > >> > >>>>>> your
>> > >> > >>>>>>>>> thoughts and feedback.
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>> Cheers,
>> > >> > >>>>>>>>> Chris
>> > >> > >>>>>>>>>
>> > >> > >>>>>>>>
>> > >> > >>>>>>>>
>> > >> > >>>>>>>
>> > >> > >>>>>>
>> > >> > >>>>>>
>> > >> > >>>>>>
>> > >> > >>>>>> --
>> > >> > >>>>>> -- Guozhang
>> > >> > >>>>>>
>> > >> > >>>>>
>> > >> > >>>>
>> > >> > >>
>> > >> > >>
>> > >> >
>> > >> >
>> > >> >
>> > >>
>> > >
>> > >
>> >
>>

RE: Thoughts and obesrvations on Samza

Posted by Thomas Becker <to...@Tivo.com>.
>From my perspective as a user I like the direction that's being proposed.  Like apparently many others, we've found YARN to be the biggest hurdle to operationalizing Samza, and it's a questionable fit for our deployment model (AWS).  A standalone mode that provides the ability to dynamically start and stop additional stream job instances and have the partitioning automagically rebalance (which as I understand it is part of what is being proposed) seems like a clear win in terms of both dependency reduction and functionality as well.

Looking at Jay's POC code also excites me about potentially being able to utilize Samza as a library.  For all its configurability, one thing Samza does not allow is customization of how it's various components are instantiated and wired together. This inflexibility has required us to make a few unfortunate design decisions for the sake of efficiency in our stream jobs.

Finally, after reading through the "CopyCat" framework design, I understand how that could take the place of pluggable consumers and producers in Samza.  Shedding that baggage that probably 95% of users won't use anyway feels like it could be a win.

-Tommy

________________________________________
From: Jay Kreps [jay@confluent.io]
Sent: Tuesday, July 07, 2015 2:35 PM
To: dev@samza.apache.org
Subject: Re: Thoughts and obesrvations on Samza

Hey Roger,

I couldn't agree more. We spent a bunch of time talking to people and that
is exactly the stuff we heard time and again. What makes it hard, of
course, is that there is some tension between compatibility with what's
there now and making things better for new users.

I also strongly agree with the importance of multi-language support. We are
talking now about Java, but for application development use cases people
want to work in whatever language they are using elsewhere. I think moving
to a model where Kafka itself does the group membership, lifecycle control,
and partition assignment has the advantage of putting all that complex
stuff behind a clean api that the clients are already going to be
implementing for their consumer, so the added functionality for stream
processing beyond a consumer becomes very minor.

-Jay

On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <ro...@gmail.com>
wrote:

> Metamorphosis...nice. :)
>
> This has been a great discussion.  As a user of Samza who's recently
> integrated it into a relatively large organization, I just want to add
> support to a few points already made.
>
> The biggest hurdles to adoption of Samza as it currently exists that I've
> experienced are:
> 1) YARN - YARN is overly complex in many environments where Puppet would do
> just fine but it was the only mechanism to get fault tolerance.
> 2) Configuration - I think I like the idea of configuring most of the job
> in code rather than config files.  In general, I think the goal should be
> to make it harder to make mistakes, especially of the kind where the code
> expects something and the config doesn't match.  The current config is
> quite intricate and error-prone.  For example, the application logic may
> depend on bootstrapping a topic but rather than asserting that in the code,
> you have to rely on getting the config right.  Likewise with serdes, the
> Java representations produced by various serdes (JSON, Avro, etc.) are not
> equivalent so you cannot just reconfigure a serde without changing the
> code.   It would be nice for jobs to be able to assert what they expect
> from their input topics in terms of partitioning.  This is getting a little
> off topic but I was even thinking about creating a "Samza config linter"
> that would sanity check a set of configs.  Especially in organizations
> where config is managed by a different team than the application developer,
> it's very hard to get avoid config mistakes.
> 3) Java/Scala centric - for many teams (especially DevOps-type folks), the
> pain of the Java toolchain (maven, slow builds, weak command line support,
> configuration over convention) really inhibits productivity.  As more and
> more high-quality clients become available for Kafka, I hope they'll follow
> Samza's model.  Not sure how much it affects the proposals in this thread
> but please consider other languages in the ecosystem as well.  From what
> I've heard, Spark has more Python users than Java/Scala.
> (FYI, we added a Jython wrapper for the Samza API
>
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> and are working on a Yeoman generator
> https://github.com/Quantiply/generator-rico for Jython/Samza projects to
> alleviate some of the pain)
>
> I also want to underscore Jay's point about improving the user experience.
> That's a very important factor for adoption.  I think the goal should be to
> make Samza as easy to get started with as something like Logstash.
> Logstash is vastly inferior in terms of capabilities to Samza but it's easy
> to get started and that makes a big difference.
>
> Cheers,
>
> Roger
>
>
>
>
>
> On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
> gdfm@apache.org> wrote:
>
> > Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
> winner
> > :)
> >
> > --
> > Gianmarco
> >
> > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <gdfm@apache.org
> >
> > wrote:
> >
> > > Hi,
> > >
> > > @Martin, thanks for you comments.
> > > Maybe I'm missing some important point, but I think coupling the
> releases
> > > is actually a *good* thing.
> > > To make an example, would it be better if the MR and HDFS components of
> > > Hadoop had different release schedules?
> > >
> > > Actually, keeping the discussion in a single place would make agreeing
> on
> > > releases (and backwards compatibility) much easier, as everybody would
> be
> > > responsible for the whole codebase.
> > >
> > > That said, I like the idea of absorbing samza-core as a sub-project,
> and
> > > leave the fancy stuff separate.
> > > It probably gives 90% of the benefits we have been discussing here.
> > >
> > > Cheers,
> > >
> > > --
> > > Gianmarco
> > >
> > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
> > >
> > >> Hey Martin,
> > >>
> > >> I agree coupling release schedules is a downside.
> > >>
> > >> Definitely we can try to solve some of the integration problems in
> > >> Confluent Platform or in other distributions. But I think this ends up
> > >> being really shallow. I guess I feel to really get a good user
> > experience
> > >> the two systems have to kind of feel like part of the same thing and
> you
> > >> can't really add that in later--you can put both in the same
> > downloadable
> > >> tar file but it doesn't really give a very cohesive feeling. I agree
> > that
> > >> ultimately any of the project stuff is as much social and naming as
> > >> anything else--theoretically two totally independent projects could
> work
> > >> to
> > >> tightly align. In practice this seems to be quite difficult though.
> > >>
> > >> For the frameworks--totally agree it would be good to maintain the
> > >> framework support with the project. In some cases there may not be too
> > >> much
> > >> there since the integration gets lighter but I think whatever stubs
> you
> > >> need should be included. So no I definitely wasn't trying to imply
> > >> dropping
> > >> support for these frameworks, just making the integration lighter by
> > >> separating process management from partition management.
> > >>
> > >> You raise two good points we would have to figure out if we went down
> > the
> > >> alignment path:
> > >> 1. With respect to the name, yeah I think the first question is
> whether
> > >> some "re-branding" would be worth it. If so then I think we can have a
> > big
> > >> thread on the name. I'm definitely not set on Kafka Streaming or Kafka
> > >> Streams I was just using them to be kind of illustrative. I agree with
> > >> your
> > >> critique of these names, though I think people would get the idea.
> > >> 2. Yeah you also raise a good point about how to "factor" it. Here are
> > the
> > >> options I see (I could get enthusiastic about any of them):
> > >>    a. One repo for both Kafka and Samza
> > >>    b. Two repos, retaining the current seperation
> > >>    c. Two repos, the equivalent of samza-api and samza-core is
> absorbed
> > >> almost like a third client
> > >>
> > >> Cheers,
> > >>
> > >> -Jay
> > >>
> > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> martin@kleppmann.com>
> > >> wrote:
> > >>
> > >> > Ok, thanks for the clarifications. Just a few follow-up comments.
> > >> >
> > >> > - I see the appeal of merging with Kafka or becoming a subproject:
> the
> > >> > reasons you mention are good. The risk I see is that release
> schedules
> > >> > become coupled to each other, which can slow everyone down, and
> large
> > >> > projects with many contributors are harder to manage. (Jakob, can
> you
> > >> speak
> > >> > from experience, having seen a wider range of Hadoop ecosystem
> > >> projects?)
> > >> >
> > >> > Some of the goals of a better unified developer experience could
> also
> > be
> > >> > solved by integrating Samza nicely into a Kafka distribution (such
> as
> > >> > Confluent's). I'm not against merging projects if we decide that's
> the
> > >> way
> > >> > to go, just pointing out the same goals can perhaps also be achieved
> > in
> > >> > other ways.
> > >> >
> > >> > - With regard to dropping the YARN dependency: are you proposing
> that
> > >> > Samza doesn't give any help to people wanting to run on
> > >> YARN/Mesos/AWS/etc?
> > >> > So the docs would basically have a link to Slider and nothing else?
> Or
> > >> > would we maintain integrations with a bunch of popular deployment
> > >> methods
> > >> > (e.g. the necessary glue and shell scripts to make Samza work with
> > >> Slider)?
> > >> >
> > >> > I absolutely think it's a good idea to have the "as a library" and
> > "as a
> > >> > process" (using Yi's taxonomy) options for people who want them,
> but I
> > >> > think there should also be a low-friction path for common "as a
> > service"
> > >> > deployment methods, for which we probably need to maintain
> > integrations.
> > >> >
> > >> > - Project naming: "Kafka Streams" seems odd to me, because Kafka is
> > all
> > >> > about streams already. Perhaps "Kafka Transformers" or "Kafka
> Filters"
> > >> > would be more apt?
> > >> >
> > >> > One suggestion: perhaps the core of Samza (stream transformation
> with
> > >> > state management -- i.e. the "Samza as a library" bit) could become
> > >> part of
> > >> > Kafka, while higher-level tools such as streaming SQL and
> integrations
> > >> with
> > >> > deployment frameworks remain in a separate project? In other words,
> > >> Kafka
> > >> > would absorb the proven, stable core of Samza, which would become
> the
> > >> > "third Kafka client" mentioned early in this thread. The Samza
> project
> > >> > would then target that third Kafka client as its base API, and the
> > >> project
> > >> > would be freed up to explore more experimental new horizons.
> > >> >
> > >> > Martin
> > >> >
> > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
> > >> >
> > >> > > Hey Martin,
> > >> > >
> > >> > > For the YARN/Mesos/etc decoupling I actually don't think it ties
> our
> > >> > hands
> > >> > > at all, all it does is refactor things. The division of
> > >> responsibility is
> > >> > > that Samza core is responsible for task lifecycle, state, and
> > >> partition
> > >> > > management (using the Kafka co-ordinator) but it is NOT
> responsible
> > >> for
> > >> > > packaging, configuration deployment or execution of processes. The
> > >> > problem
> > >> > > of packaging and starting these processes is
> > >> > > framework/environment-specific. This leaves individual frameworks
> to
> > >> be
> > >> > as
> > >> > > fancy or vanilla as they like. So you can get simple stateless
> > >> support in
> > >> > > YARN, Mesos, etc using their off-the-shelf app framework (Slider,
> > >> > Marathon,
> > >> > > etc). These are well known by people and have nice UIs and a lot
> of
> > >> > > flexibility. I don't think they have node affinity as a built in
> > >> option
> > >> > > (though I could be wrong). So if we want that we can either wait
> for
> > >> them
> > >> > > to add it or do a custom framework to add that feature (as now).
> > >> > Obviously
> > >> > > if you manage things with old-school ops tools (puppet/chef/etc)
> you
> > >> get
> > >> > > locality easily. The nice thing, though, is that all the samza
> > >> "business
> > >> > > logic" around partition management and fault tolerance is in Samza
> > >> core
> > >> > so
> > >> > > it is shared across frameworks and the framework specific bit is
> > just
> > >> > > whether it is smart enough to try to get the same host when a job
> is
> > >> > > restarted.
> > >> > >
> > >> > > With respect to the Kafka-alignment, yeah I think the goal would
> be
> > >> (a)
> > >> > > actually get better alignment in user experience, and (b) express
> > >> this in
> > >> > > the naming and project branding. Specifically:
> > >> > > 1. Website/docs, it would be nice for the "transformation" api to
> be
> > >> > > discoverable in the main Kafka docs--i.e. be able to explain when
> to
> > >> use
> > >> > > the consumer and when to use the stream processing functionality
> and
> > >> lead
> > >> > > people into that experience.
> > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that
> has
> > >> both
> > >> > > Kafka and the stream processing part and they actually work
> > together.
> > >> > > 3. Unify the programming experience so the client and Samza api
> > share
> > >> > > config/monitoring/naming/packaging/etc.
> > >> > >
> > >> > > I think sub-projects keep separate committers and can have a
> > separate
> > >> > repo,
> > >> > > but I'm actually not really sure (I can't find a definition of a
> > >> > subproject
> > >> > > in Apache).
> > >> > >
> > >> > > Basically at a high-level you want the experience to "feel" like a
> > >> single
> > >> > > system, not to relatively independent things that are kind of
> > >> awkwardly
> > >> > > glued together.
> > >> > >
> > >> > > I think if we did that they having naming or branding like "kafka
> > >> > > streaming" or "kafka streams" or something like that would
> actually
> > >> do a
> > >> > > good job of conveying what it is. I do that this would help
> adoption
> > >> > quite
> > >> > > a lot as it would correctly convey that using Kafka Streaming with
> > >> Kafka
> > >> > is
> > >> > > a fairly seamless experience and Kafka is pretty heavily adopted
> at
> > >> this
> > >> > > point.
> > >> > >
> > >> > > Fwiw we actually considered this model originally when open
> sourcing
> > >> > Samza,
> > >> > > however at that time Kafka was relatively unknown and we decided
> not
> > >> to
> > >> > do
> > >> > > it since we felt it would be limiting. From my point of view the
> > three
> > >> > > things have changed (1) Kafka is now really heavily used for
> stream
> > >> > > processing, (2) we learned that abstracting out the stream well is
> > >> > > basically impossible, (3) we learned it is really hard to keep the
> > two
> > >> > > things feeling like a single product.
> > >> > >
> > >> > > -Jay
> > >> > >
> > >> > >
> > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> > >> martin@kleppmann.com>
> > >> > > wrote:
> > >> > >
> > >> > >> Hi all,
> > >> > >>
> > >> > >> Lots of good thoughts here.
> > >> > >>
> > >> > >> I agree with the general philosophy of tying Samza more firmly to
> > >> Kafka.
> > >> > >> After I spent a while looking at integrating other message
> brokers
> > >> (e.g.
> > >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
> > >> > SystemConsumer
> > >> > >> tacitly assumes a model so much like Kafka's that pretty much
> > nobody
> > >> but
> > >> > >> Kafka actually implements it. (Databus is perhaps an exception,
> but
> > >> it
> > >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza fully
> > >> > dependent
> > >> > >> on Kafka acknowledges that the system-independence was never as
> > real
> > >> as
> > >> > we
> > >> > >> perhaps made it out to be. The gains of code reuse are real.
> > >> > >>
> > >> > >> The idea of decoupling Samza from YARN has also always been
> > >> appealing to
> > >> > >> me, for various reasons already mentioned in this thread.
> Although
> > >> > making
> > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems
> > >> laudable,
> > >> > I am
> > >> > >> a little concerned that it will restrict us to a lowest common
> > >> > denominator.
> > >> > >> For example, would host affinity (SAMZA-617) still be possible?
> For
> > >> jobs
> > >> > >> with large amounts of state, I think SAMZA-617 would be a big
> boon,
> > >> > since
> > >> > >> restoring state off the changelog on every single restart is
> > painful,
> > >> > due
> > >> > >> to long recovery times. It would be a shame if the decoupling
> from
> > >> YARN
> > >> > >> made host affinity impossible.
> > >> > >>
> > >> > >> Jay, a question about the proposed API for instantiating a job in
> > >> code
> > >> > >> (rather than a properties file): when submitting a job to a
> > cluster,
> > >> is
> > >> > the
> > >> > >> idea that the instantiation code runs on a client somewhere,
> which
> > >> then
> > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that
> > >> code
> > >> > run
> > >> > >> on each container that is part of the job (in which case, how
> does
> > >> the
> > >> > job
> > >> > >> submission to the cluster work)?
> > >> > >>
> > >> > >> I agree with Garry that it doesn't feel right to make a 1.0
> release
> > >> > with a
> > >> > >> plan for it to be immediately obsolete. So if this is going to
> > >> happen, I
> > >> > >> think it would be more honest to stick with 0.* version numbers
> > until
> > >> > the
> > >> > >> library-ified Samza has been implemented, is stable and widely
> > used.
> > >> > >>
> > >> > >> Should the new Samza be a subproject of Kafka? There is precedent
> > for
> > >> > >> tight coupling between different Apache projects (e.g. Curator
> and
> > >> > >> Zookeeper, or Slider and YARN), so I think remaining separate
> would
> > >> be
> > >> > ok.
> > >> > >> Even if Samza is fully dependent on Kafka, there is enough
> > substance
> > >> in
> > >> > >> Samza that it warrants being a separate project. An argument in
> > >> favour
> > >> > of
> > >> > >> merging would be if we think Kafka has a much stronger "brand
> > >> presence"
> > >> > >> than Samza; I'm ambivalent on that one. If the Kafka project is
> > >> willing
> > >> > to
> > >> > >> endorse Samza as the "official" way of doing stateful stream
> > >> > >> transformations, that would probably have much the same effect as
> > >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
> > >> > >> collaboration between the two projects will be needed in any
> case.
> > >> > >>
> > >> > >> From a project management perspective, I guess the "new Samza"
> > would
> > >> > have
> > >> > >> to be developed on a branch alongside ongoing maintenance of the
> > >> current
> > >> > >> line of development? I think it would be important to continue
> > >> > supporting
> > >> > >> existing users, and provide a graceful migration path to the new
> > >> > version.
> > >> > >> Leaving the current versions unsupported and forcing people to
> > >> rewrite
> > >> > >> their jobs would send a bad signal.
> > >> > >>
> > >> > >> Best,
> > >> > >> Martin
> > >> > >>
> > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
> > >> > >>
> > >> > >>> Hey Garry,
> > >> > >>>
> > >> > >>> Yeah that's super frustrating. I'd be happy to chat more about
> > this
> > >> if
> > >> > >>> you'd be interested. I think Chris and I started with the idea
> of
> > >> "what
> > >> > >>> would it take to make Samza a kick-ass ingestion tool" but
> > >> ultimately
> > >> > we
> > >> > >>> kind of came around to the idea that ingestion and
> transformation
> > >> had
> > >> > >>> pretty different needs and coupling the two made things hard.
> > >> > >>>
> > >> > >>> For what it's worth I think copycat (KIP-26) actually will do
> what
> > >> you
> > >> > >> are
> > >> > >>> looking for.
> > >> > >>>
> > >> > >>> With regard to your point about slider, I don't necessarily
> > >> disagree.
> > >> > >> But I
> > >> > >>> think getting good YARN support is quite doable and I think we
> can
> > >> make
> > >> > >>> that work well. I think the issue this proposal solves is that
> > >> > >> technically
> > >> > >>> it is pretty hard to support multiple cluster management systems
> > the
> > >> > way
> > >> > >>> things are now, you need to write an "app master" or "framework"
> > for
> > >> > each
> > >> > >>> and they are all a little different so testing is really hard.
> In
> > >> the
> > >> > >>> absence of this we have been stuck with just YARN which has
> > >> fantastic
> > >> > >>> penetration in the Hadoopy part of the org, but zero penetration
> > >> > >> elsewhere.
> > >> > >>> Given the huge amount of work being put in to slider, marathon,
> > aws
> > >> > >>> tooling, not to mention the umpteen related packaging
> technologies
> > >> > people
> > >> > >>> want to use (Docker, Kubernetes, various cloud-specific deploy
> > >> tools,
> > >> > >> etc)
> > >> > >>> I really think it is important to get this right.
> > >> > >>>
> > >> > >>> -Jay
> > >> > >>>
> > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > >> > >>> g.turkington@improvedigital.com> wrote:
> > >> > >>>
> > >> > >>>> Hi all,
> > >> > >>>>
> > >> > >>>> I think the question below re does Samza become a sub-project
> of
> > >> Kafka
> > >> > >>>> highlights the broader point around migration. Chris mentions
> > >> Samza's
> > >> > >>>> maturity is heading towards a v1 release but I'm not sure it
> > feels
> > >> > >> right to
> > >> > >>>> launch a v1 then immediately plan to deprecate most of it.
> > >> > >>>>
> > >> > >>>> From a selfish perspective I have some guys who have started
> > >> working
> > >> > >> with
> > >> > >>>> Samza and building some new consumers/producers was next up.
> > Sounds
> > >> > like
> > >> > >>>> that is absolutely not the direction to go. I need to look into
> > the
> > >> > KIP
> > >> > >> in
> > >> > >>>> more detail but for me the attractiveness of adding new Samza
> > >> > >>>> consumer/producers -- even if yes all they were doing was
> really
> > >> > getting
> > >> > >>>> data into and out of Kafka --  was to avoid  having to worry
> > about
> > >> the
> > >> > >>>> lifecycle management of external clients. If there is a generic
> > >> Kafka
> > >> > >>>> ingress/egress layer that I can plug a new connector into and
> > have
> > >> a
> > >> > >> lot of
> > >> > >>>> the heavy lifting re scale and reliability done for me then it
> > >> gives
> > >> > me
> > >> > >> all
> > >> > >>>> the pushing new consumers/producers would. If not then it
> > >> complicates
> > >> > my
> > >> > >>>> operational deployments.
> > >> > >>>>
> > >> > >>>> Which is similar to my other question with the proposal -- if
> we
> > >> > build a
> > >> > >>>> fully available/stand-alone Samza plus the requisite shims to
> > >> > integrate
> > >> > >>>> with Slider etc I suspect the former may be a lot more work
> than
> > we
> > >> > >> think.
> > >> > >>>> We may make it much easier for a newcomer to get something
> > running
> > >> but
> > >> > >>>> having them step up and get a reliable production deployment
> may
> > >> still
> > >> > >>>> dominate mailing list  traffic, if for different reasons than
> > >> today.
> > >> > >>>>
> > >> > >>>> Don't get me wrong -- I'm comfortable with making the Samza
> > >> dependency
> > >> > >> on
> > >> > >>>> Kafka much more explicit and I absolutely see the benefits  in
> > the
> > >> > >>>> reduction of duplication and clashing
> terminologies/abstractions
> > >> that
> > >> > >>>> Chris/Jay describe. Samza as a library would likely be a very
> > nice
> > >> > tool
> > >> > >> to
> > >> > >>>> add to the Kafka ecosystem. I just have the concerns above re
> the
> > >> > >>>> operational side.
> > >> > >>>>
> > >> > >>>> Garry
> > >> > >>>>
> > >> > >>>> -----Original Message-----
> > >> > >>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> > >> > >>>> Sent: 02 July 2015 12:56
> > >> > >>>> To: dev@samza.apache.org
> > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> > >> > >>>>
> > >> > >>>> Very interesting thoughts.
> > >> > >>>> From outside, I have always perceived Samza as a computing
> layer
> > >> over
> > >> > >>>> Kafka.
> > >> > >>>>
> > >> > >>>> The question, maybe a bit provocative, is "should Samza be a
> > >> > sub-project
> > >> > >>>> of Kafka then?"
> > >> > >>>> Or does it make sense to keep it as a separate project with a
> > >> separate
> > >> > >>>> governance?
> > >> > >>>>
> > >> > >>>> Cheers,
> > >> > >>>>
> > >> > >>>> --
> > >> > >>>> Gianmarco
> > >> > >>>>
> > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> wrote:
> > >> > >>>>
> > >> > >>>>> Overall, I agree to couple with Kafka more tightly. Because
> > Samza
> > >> de
> > >> > >>>>> facto is based on Kafka, and it should leverage what Kafka
> has.
> > At
> > >> > the
> > >> > >>>>> same time, Kafka does not need to reinvent what Samza already
> > >> has. I
> > >> > >>>>> also like the idea of separating the ingestion and
> > transformation.
> > >> > >>>>>
> > >> > >>>>> But it is a little difficult for me to image how the Samza
> will
> > >> look
> > >> > >>>> like.
> > >> > >>>>> And I feel Chris and Jay have a little difference in terms of
> > how
> > >> > >>>>> Samza should look like.
> > >> > >>>>>
> > >> > >>>>> *** Will it look like what Jay's code shows (A client of
> Kakfa)
> > ?
> > >> And
> > >> > >>>>> user's application code calls this client?
> > >> > >>>>>
> > >> > >>>>> 1. If we make Samza be a library of Kafka (like what the code
> > >> shows),
> > >> > >>>>> how do we implement auto-balance and fault-tolerance? Are they
> > >> taken
> > >> > >>>>> care by the Kafka broker or other mechanism, such as "Samza
> > >> worker"
> > >> > >>>>> (just make up the name) ?
> > >> > >>>>>
> > >> > >>>>> 2. What about other features, such as auto-scaling, shared
> > state,
> > >> > >>>>> monitoring?
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> *** If we have Samza standalone, (is this what Chris
> suggests?)
> > >> > >>>>>
> > >> > >>>>> 1. we still need to ingest data from Kakfa and produce to it.
> > >> Then it
> > >> > >>>>> becomes the same as what Samza looks like now, except it does
> > not
> > >> > rely
> > >> > >>>>> on Yarn anymore.
> > >> > >>>>>
> > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's metrics,
> > logs,
> > >> > >>>>> etc? Use Kafka code as the dependency?
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> Thanks,
> > >> > >>>>>
> > >> > >>>>> Fang, Yan
> > >> > >>>>> yanfang724@gmail.com
> > >> > >>>>>
> > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > wangguoz@gmail.com
> > >> >
> > >> > >>>> wrote:
> > >> > >>>>>
> > >> > >>>>>> Read through the code example and it looks good to me. A few
> > >> > >>>>>> thoughts regarding deployment:
> > >> > >>>>>>
> > >> > >>>>>> Today Samza deploys as executable runnable like:
> > >> > >>>>>>
> > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> > >> > >>>> --config-path=file://...
> > >> > >>>>>>
> > >> > >>>>>> And this proposal advocate for deploying Samza more as
> embedded
> > >> > >>>>>> libraries in user application code (ignoring the terminology
> > >> since
> > >> > >>>>>> it is not the
> > >> > >>>>> same
> > >> > >>>>>> as the prototype code):
> > >> > >>>>>>
> > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread thread =
> > new
> > >> > >>>>>> Thread(task); thread.start();
> > >> > >>>>>>
> > >> > >>>>>> I think both of these deployment modes are important for
> > >> different
> > >> > >>>>>> types
> > >> > >>>>> of
> > >> > >>>>>> users. That said, I think making Samza purely standalone is
> > still
> > >> > >>>>>> sufficient for either runnable or library modes.
> > >> > >>>>>>
> > >> > >>>>>> Guozhang
> > >> > >>>>>>
> > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> jay@confluent.io>
> > >> > wrote:
> > >> > >>>>>>
> > >> > >>>>>>> Looks like gmail mangled the code example, it was supposed
> to
> > >> look
> > >> > >>>>>>> like
> > >> > >>>>>>> this:
> > >> > >>>>>>>
> > >> > >>>>>>> Properties props = new Properties();
> > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > >> StreamingConfig
> > >> > >>>>>>> config = new StreamingConfig(props);
> > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> > >> > >>>>>>> config.serialization(new StringSerializer(), new
> > >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
> > >> > >>>>>>> KafkaStreaming(config); container.run();
> > >> > >>>>>>>
> > >> > >>>>>>> -Jay
> > >> > >>>>>>>
> > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> jay@confluent.io
> > >
> > >> > >>>> wrote:
> > >> > >>>>>>>
> > >> > >>>>>>>> Hey guys,
> > >> > >>>>>>>>
> > >> > >>>>>>>> This came out of some conversations Chris and I were having
> > >> > >>>>>>>> around
> > >> > >>>>>>> whether
> > >> > >>>>>>>> it would make sense to use Samza as a kind of data
> ingestion
> > >> > >>>>> framework
> > >> > >>>>>>> for
> > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This
> kind
> > of
> > >> > >>>>>> combined
> > >> > >>>>>>>> with complaints around config and YARN and the discussion
> > >> around
> > >> > >>>>>>>> how
> > >> > >>>>> to
> > >> > >>>>>>>> best do a standalone mode.
> > >> > >>>>>>>>
> > >> > >>>>>>>> So the thought experiment was, given that Samza was
> basically
> > >> > >>>>>>>> already totally Kafka specific, what if you just embraced
> > that
> > >> > >>>>>>>> and turned it
> > >> > >>>>>> into
> > >> > >>>>>>>> something less like a heavyweight framework and more like a
> > >> > >>>>>>>> third
> > >> > >>>>> Kafka
> > >> > >>>>>>>> client--a kind of "producing consumer" with state
> management
> > >> > >>>>>> facilities.
> > >> > >>>>>>>> Basically a library. Instead of a complex stream processing
> > >> > >>>>>>>> framework
> > >> > >>>>>>> this
> > >> > >>>>>>>> would actually be a very simple thing, not much more
> > >> complicated
> > >> > >>>>>>>> to
> > >> > >>>>> use
> > >> > >>>>>>> or
> > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we thought
> about
> > >> it
> > >> > >>>>>>>> a
> > >> > >>>>> lot
> > >> > >>>>>> of
> > >> > >>>>>>>> what Samza (and the other stream processing systems were
> > doing)
> > >> > >>>>> seemed
> > >> > >>>>>>> like
> > >> > >>>>>>>> kind of a hangover from MapReduce.
> > >> > >>>>>>>>
> > >> > >>>>>>>> Of course you need to ingest/output data to and from the
> > stream
> > >> > >>>>>>>> processing. But when we actually looked into how that would
> > >> > >>>>>>>> work,
> > >> > >>>>> Samza
> > >> > >>>>>>>> isn't really an ideal data ingestion framework for a bunch
> of
> > >> > >>>>> reasons.
> > >> > >>>>>> To
> > >> > >>>>>>>> really do that right you need a pretty different internal
> > data
> > >> > >>>>>>>> model
> > >> > >>>>>> and
> > >> > >>>>>>>> set of apis. So what if you split them and had an api for
> > Kafka
> > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
> > >> Kafka
> > >> > >>>>>>>> transformation (Samza).
> > >> > >>>>>>>>
> > >> > >>>>>>>> This would also allow really embracing the same terminology
> > and
> > >> > >>>>>>>> conventions. One complaint about the current state is that
> > the
> > >> > >>>>>>>> two
> > >> > >>>>>>> systems
> > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs
> "topic"
> > >> and
> > >> > >>>>>>> different
> > >> > >>>>>>>> config and monitoring systems means you kind of have to
> learn
> > >> > >>>>>>>> Kafka's
> > >> > >>>>>>> way,
> > >> > >>>>>>>> then learn Samza's slightly different way, then kind of
> > >> > >>>>>>>> understand
> > >> > >>>>> how
> > >> > >>>>>>> they
> > >> > >>>>>>>> map to each other, which having walked a few people through
> > >> this
> > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > >> > >>>>>>>>
> > >> > >>>>>>>> Since I have been spending a lot of time on airplanes I
> > hacked
> > >> > >>>>>>>> up an ernest but still somewhat incomplete prototype of
> what
> > >> > >>>>>>>> this would
> > >> > >>>>> look
> > >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka as it
> > >> > >>>>>>>> required a
> > >> > >>>>>> few
> > >> > >>>>>>>> changes to the new consumer. Here is the code:
> > >> > >>>>>>>>
> > >> > >>>>>>>>
> > >> > >>>>>>>
> > >> > >>>>>>
> > >> > >>>>>
> > >> >
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >> > >>>>> /apache/kafka/clients/streaming
> > >> > >>>>>>>>
> > >> > >>>>>>>> For the purpose of the prototype I just liberally renamed
> > >> > >>>>>>>> everything
> > >> > >>>>> to
> > >> > >>>>>>>> try to align it with Kafka with no regard for
> compatibility.
> > >> > >>>>>>>>
> > >> > >>>>>>>> To use this would be something like this:
> > >> > >>>>>>>> Properties props = new Properties();
> > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > >> > >>>>>>>> StreamingConfig config = new
> > >> > >>>>> StreamingConfig(props);
> > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > >>>>>>>> "test-topic-2");
> > >> config.processor(ExampleStreamProcessor.class);
> > >> > >>>>>>> config.serialization(new
> > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
> KafkaStreaming
> > >> > >>>>>> container =
> > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> > >> > >>>>>>>>
> > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> > StreamProcessor
> > >> > >>>>>>>> is basically StreamTask.
> > >> > >>>>>>>>
> > >> > >>>>>>>> So rather than putting all the class names in a file and
> then
> > >> > >>>>>>>> having
> > >> > >>>>>> the
> > >> > >>>>>>>> job assembled by reflection, you just instantiate the
> > container
> > >> > >>>>>>>> programmatically. Work is balanced over however many
> > instances
> > >> > >>>>>>>> of
> > >> > >>>>> this
> > >> > >>>>>>> are
> > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new tasks are
> > >> added
> > >> > >>>>>>>> to
> > >> > >>>>> the
> > >> > >>>>>>>> existing containers without shutting them down).
> > >> > >>>>>>>>
> > >> > >>>>>>>> We would provide some glue for running this stuff in YARN
> via
> > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their
> tools
> > >> > >>>>>>>> but from the
> > >> > >>>>>> point
> > >> > >>>>>>> of
> > >> > >>>>>>>> view of these frameworks these stream processing jobs are
> > just
> > >> > >>>>>> stateless
> > >> > >>>>>>>> services that can come and go and expand and contract at
> > will.
> > >> > >>>>>>>> There
> > >> > >>>>> is
> > >> > >>>>>>> no
> > >> > >>>>>>>> more custom scheduler.
> > >> > >>>>>>>>
> > >> > >>>>>>>> Here are some relevant details:
> > >> > >>>>>>>>
> > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get larger if
> we
> > >> > >>>>>>>>  productionized but not vastly larger. We really do get a
> ton
> > >> > >>>>>>>> of
> > >> > >>>>>>> leverage
> > >> > >>>>>>>>  out of Kafka.
> > >> > >>>>>>>>  2. Partition management is fully delegated to the new
> > >> consumer.
> > >> > >>>>> This
> > >> > >>>>>>>>  is nice since now any partition management strategy
> > available
> > >> > >>>>>>>> to
> > >> > >>>>>> Kafka
> > >> > >>>>>>>>  consumer is also available to Samza (and vice versa) and
> > with
> > >> > >>>>>>>> the
> > >> > >>>>>>> exact
> > >> > >>>>>>>>  same configs.
> > >> > >>>>>>>>  3. It supports state as well as state reuse
> > >> > >>>>>>>>
> > >> > >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
> > >> > >>>>>>>>
> > >> > >>>>>>>> -Jay
> > >> > >>>>>>>>
> > >> > >>>>>>>>
> > >> > >>>>>>>>
> > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > >> > >>>>>> criccomini@apache.org>
> > >> > >>>>>>>> wrote:
> > >> > >>>>>>>>
> > >> > >>>>>>>>> Hey all,
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> I have had some discussions with Samza engineers at
> LinkedIn
> > >> > >>>>>>>>> and
> > >> > >>>>>>> Confluent
> > >> > >>>>>>>>> and we came up with a few observations and would like to
> > >> > >>>>>>>>> propose
> > >> > >>>>> some
> > >> > >>>>>>>>> changes.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> We've observed some things that I want to call out about
> > >> > >>>>>>>>> Samza's
> > >> > >>>>>> design,
> > >> > >>>>>>>>> and I'd like to propose some changes.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
> > >> > >>>>>>>>> * Samza is too pluggable.
> > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> consumer
> > >> > >>>>>>>>> APIs
> > >> > >>>>> are
> > >> > >>>>>>>>> trying to solve a lot of the same problems.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> All three of these issues are related, but I'll address
> them
> > >> in
> > >> > >>>>> order.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Deployment
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic deployment
> > >> > >>>>>>>>> scheduler
> > >> > >>>>>> such
> > >> > >>>>>>>>> as
> > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet
> that
> > >> > >>>>>>>>> there
> > >> > >>>>>> would
> > >> > >>>>>>>>> be
> > >> > >>>>>>>>> one or two winners in this area, and we could support
> them,
> > >> and
> > >> > >>>>>>>>> the
> > >> > >>>>>> rest
> > >> > >>>>>>>>> would go away. In reality, there are many variations.
> > >> > >>>>>>>>> Furthermore,
> > >> > >>>>>> many
> > >> > >>>>>>>>> people still prefer to just start their processors like
> > normal
> > >> > >>>>>>>>> Java processes, and use traditional deployment scripts
> such
> > as
> > >> > >>>>>>>>> Fabric,
> > >> > >>>>>> Chef,
> > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users makes
> the
> > >> > >>>>>>>>> Samza start-up process really painful for first time
> users.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit of a
> > >> > >>>>>>>>> mis-fire
> > >> > >>>>>> because
> > >> > >>>>>>>>> of
> > >> > >>>>>>>>> a fundamental misunderstanding between the nature of batch
> > >> jobs
> > >> > >>>>>>>>> and
> > >> > >>>>>>> stream
> > >> > >>>>>>>>> processing jobs. Early on, we made conscious effort to
> favor
> > >> > >>>>>>>>> the
> > >> > >>>>>> Hadoop
> > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked and was
> > well
> > >> > >>>>>>> understood.
> > >> > >>>>>>>>> One thing that we missed was that batch jobs have a
> definite
> > >> > >>>>>> beginning,
> > >> > >>>>>>>>> and
> > >> > >>>>>>>>> end, and stream processing jobs don't (usually). This
> leads
> > to
> > >> > >>>>>>>>> a
> > >> > >>>>> much
> > >> > >>>>>>>>> simpler scheduling problem for stream processors. You
> > >> basically
> > >> > >>>>>>>>> just
> > >> > >>>>>>> need
> > >> > >>>>>>>>> to find a place to start the processor, and start it. The
> > way
> > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> > >> > >>>>>>>>> being "full". We always
> > >> > >>>>>> add
> > >> > >>>>>>>>> more machines. The problem with coupling Samza with a
> > >> scheduler
> > >> > >>>>>>>>> is
> > >> > >>>>>> that
> > >> > >>>>>>>>> Samza (as a framework) now has to handle deployment. This
> > >> pulls
> > >> > >>>>>>>>> in a
> > >> > >>>>>>> bunch
> > >> > >>>>>>>>> of things such as configuration distribution (config
> > stream),
> > >> > >>>>>>>>> shell
> > >> > >>>>>>> scrips
> > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> stuff),
> > >> etc.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Another reason for requiring dynamic deployment was to
> > support
> > >> > >>>>>>>>> data locality. If you want to have locality, you need to
> put
> > >> > >>>>>>>>> your
> > >> > >>>>>> processors
> > >> > >>>>>>>>> close to the data they're processing. Upon further
> > >> > >>>>>>>>> investigation,
> > >> > >>>>>>> though,
> > >> > >>>>>>>>> this feature is not that beneficial. There is some good
> > >> > >>>>>>>>> discussion
> > >> > >>>>>> about
> > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
> > >> > >>>>>>>>> Map/Reduce
> > >> > >>>>>> path,
> > >> > >>>>>>>>> but
> > >> > >>>>>>>>> there are some fundamental differences between HDFS and
> > Kafka.
> > >> > >>>>>>>>> HDFS
> > >> > >>>>>> has
> > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to less
> > >> > >>>>>>>>> optimization potential with stream processors on top of
> > Kafka.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't have
> > any
> > >> > >>>>>>>>> built
> > >> > >>>>> in
> > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> > >> > >>>>>>>>> deployment scheduling system to handle restarts when a
> > >> > >>>>>>>>> processor dies. This has
> > >> > >>>>>>> made
> > >> > >>>>>>>>> it very difficult to write a standalone Samza container
> > >> > >>>> (SAMZA-516).
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Pluggability
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> In some cases pluggability is good, but I think that we've
> > >> gone
> > >> > >>>>>>>>> too
> > >> > >>>>>> far
> > >> > >>>>>>>>> with it. Currently, Samza has:
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> * Pluggable config.
> > >> > >>>>>>>>> * Pluggable metrics.
> > >> > >>>>>>>>> * Pluggable deployment systems.
> > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> > SystemProducer,
> > >> > >>>> etc).
> > >> > >>>>>>>>> * Pluggable serdes.
> > >> > >>>>>>>>> * Pluggable storage engines.
> > >> > >>>>>>>>> * Pluggable strategies for just about every component
> > >> > >>>>> (MessageChooser,
> > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> There's probably more that I've forgotten, as well. Some
> of
> > >> > >>>>>>>>> these
> > >> > >>>>> are
> > >> > >>>>>>>>> useful, but some have proven not to be. This all comes at
> a
> > >> cost:
> > >> > >>>>>>>>> complexity. This complexity is making it harder for our
> > users
> > >> > >>>>>>>>> to
> > >> > >>>>> pick
> > >> > >>>>>> up
> > >> > >>>>>>>>> and use Samza out of the box. It also makes it difficult
> for
> > >> > >>>>>>>>> Samza developers to reason about what the characteristics
> of
> > >> > >>>>>>>>> the container (since the characteristics change depending
> on
> > >> > >>>>>>>>> which plugins are use).
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> The issues with pluggability are most visible in the
> System
> > >> APIs.
> > >> > >>>>> What
> > >> > >>>>>>>>> Samza really requires to be functional is Kafka as its
> > >> > >>>>>>>>> transport
> > >> > >>>>>> layer.
> > >> > >>>>>>>>> But
> > >> > >>>>>>>>> we've conflated two unrelated use cases into one API:
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > >> > >>>>>>>>> 2. Process the data in Kafka.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> The current System API supports both of these use cases.
> The
> > >> > >>>>>>>>> problem
> > >> > >>>>>> is,
> > >> > >>>>>>>>> we
> > >> > >>>>>>>>> actually want different features for each use case. By
> > >> papering
> > >> > >>>>>>>>> over
> > >> > >>>>>>> these
> > >> > >>>>>>>>> two use cases, and providing a single API, we've
> introduced
> > a
> > >> > >>>>>>>>> ton of
> > >> > >>>>>>> leaky
> > >> > >>>>>>>>> abstractions.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> For example, what we'd really like in (2) is to have
> > >> > >>>>>>>>> monotonically increasing longs for offsets (like Kafka).
> > This
> > >> > >>>>>>>>> would be at odds
> > >> > >>>>> with
> > >> > >>>>>>> (1),
> > >> > >>>>>>>>> though, since different systems have different
> > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > >> > >>>>>>>>> There was discussion both on the mailing list and the SQL
> > >> JIRAs
> > >> > >>>>> about
> > >> > >>>>>>> the
> > >> > >>>>>>>>> need for this.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> The same thing holds true for replayability. Kafka allows
> us
> > >> to
> > >> > >>>>> rewind
> > >> > >>>>>>>>> when
> > >> > >>>>>>>>> we have a failure. Many other systems don't. In some
> cases,
> > >> > >>>>>>>>> systems
> > >> > >>>>>>> return
> > >> > >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> > because
> > >> > >>>>>>>>> they
> > >> > >>>>>> have
> > >> > >>>>>>> no
> > >> > >>>>>>>>> offsets.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Partitioning is another example. Kafka supports
> > partitioning,
> > >> > >>>>>>>>> but
> > >> > >>>>> many
> > >> > >>>>>>>>> systems don't. We model this by having a single partition
> > for
> > >> > >>>>>>>>> those systems. Still, other systems model partitioning
> > >> > >>>> differently (e.g.
> > >> > >>>>>>>>> Kinesis).
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating streams
> > in
> > >> a
> > >> > >>>>>>>>> system-agnostic way is almost impossible. As is modeling
> > >> > >>>>>>>>> metadata
> > >> > >>>>> for
> > >> > >>>>>>> the
> > >> > >>>>>>>>> system (replication factor, partitions, location, etc).
> The
> > >> > >>>>>>>>> list
> > >> > >>>>> goes
> > >> > >>>>>>> on.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Duplicate work
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's consumer
> > and
> > >> > >>>>> producer
> > >> > >>>>>>>>> APIs
> > >> > >>>>>>>>> had a relatively weak feature set. On the consumer-side,
> you
> > >> > >>>>>>>>> had two
> > >> > >>>>>>>>> options: use the high level consumer, or the simple
> > consumer.
> > >> > >>>>>>>>> The
> > >> > >>>>>>> problem
> > >> > >>>>>>>>> with the high-level consumer was that it controlled your
> > >> > >>>>>>>>> offsets, partition assignments, and the order in which you
> > >> > >>>>>>>>> received messages. The
> > >> > >>>>> problem
> > >> > >>>>>>>>> with
> > >> > >>>>>>>>> the simple consumer is that it's not simple. It's basic.
> You
> > >> > >>>>>>>>> end up
> > >> > >>>>>>> having
> > >> > >>>>>>>>> to handle a lot of really low-level stuff that you
> > shouldn't.
> > >> > >>>>>>>>> We
> > >> > >>>>>> spent a
> > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> robust.
> > >> It
> > >> > >>>>>>>>> also allows us to support some cool features:
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> * Per-partition message ordering and prioritization.
> > >> > >>>>>>>>> * Tight control over partition assignment to support
> joins,
> > >> > >>>>>>>>> global
> > >> > >>>>>> state
> > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > >> > >>>>>>>>> * Tight control over offset checkpointing.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> What we didn't realize at the time is that these features
> > >> > >>>>>>>>> should
> > >> > >>>>>>> actually
> > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
> stream
> > >> > >>>>>> processors)
> > >> > >>>>>>>>> end up wanting to do things like joins and partition
> > >> > >>>>>>>>> assignment. The
> > >> > >>>>>>> Kafka
> > >> > >>>>>>>>> community has come to the same conclusion. They're adding
> a
> > >> ton
> > >> > >>>>>>>>> of upgrades into their new Kafka consumer implementation.
> > To a
> > >> > >>>>>>>>> large extent,
> > >> > >>>>> it's
> > >> > >>>>>>>>> duplicate work to what we've already done in Samza.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
> > approach
> > >> > >>>>>>>>> to
> > >> > >>>>>> Samza's
> > >> > >>>>>>>>> KafkaCheckpointManager implementation for handling offset
> > >> > >>>>>> checkpointing.
> > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature stores
> > >> offset
> > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them from
> > the
> > >> > >>>>>>>>> broker.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> A lot of this seems like a waste, since we could have
> shared
> > >> > >>>>>>>>> the
> > >> > >>>>> work
> > >> > >>>>>> if
> > >> > >>>>>>>>> it
> > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Vision
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> All of this leads me to a rather radical proposal. Samza
> is
> > >> > >>>>> relatively
> > >> > >>>>>>>>> stable at this point. I'd venture to say that we're near a
> > 1.0
> > >> > >>>>>> release.
> > >> > >>>>>>>>> I'd
> > >> > >>>>>>>>> like to propose that we take what we've learned, and begin
> > >> > >>>>>>>>> thinking
> > >> > >>>>>>> about
> > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were starting
> > >> from
> > >> > >>>>>> scratch?
> > >> > >>>>>>>>> My
> > >> > >>>>>>>>> proposal is to:
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > >> > >>>>>>>>> processors, and eliminate all direct dependences on YARN,
> > >> Mesos,
> > >> > >>>> etc.
> > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as the
> > stream
> > >> > >>>>>> processing
> > >> > >>>>>>>>> layer.
> > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> > >> > >>>>>>>>> config
> > >> > >>>>>>> systems,
> > >> > >>>>>>>>> and simply use Kafka's instead.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> This would fix all of the issues that I outlined above. It
> > >> > >>>>>>>>> should
> > >> > >>>>> also
> > >> > >>>>>>>>> shrink the Samza code base pretty dramatically. Supporting
> > >> only
> > >> > >>>>>>>>> a standalone container will allow Samza to be executed on
> > YARN
> > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most
> other
> > >> > >>>>>>>>> in-house
> > >> > >>>>>>> deployment
> > >> > >>>>>>>>> systems. This should make life a lot easier for new users.
> > >> > >>>>>>>>> Imagine
> > >> > >>>>>>> having
> > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in mailing
> > >> list
> > >> > >>>>>> traffic
> > >> > >>>>>>>>> will be pretty dramatic.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The reality
> > is,
> > >> > >>>>> everyone
> > >> > >>>>>>>>> that
> > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically
> require
> > >> it
> > >> > >>>>>> already
> > >> > >>>>>>> in
> > >> > >>>>>>>>> order for most features to work. Those that are using
> other
> > >> > >>>>>>>>> systems
> > >> > >>>>>> are
> > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and then
> they
> > do
> > >> > >>>>>>>>> the processing on top. There is already discussion (
> > >> > >>>>>>>>>
> > >> > >>>>>>>
> > >> > >>>>>>
> > >> > >>>>>
> > >> >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >> > >>>>> 767
> > >> > >>>>>>>>> )
> > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Once we make the call to couple with Kafka, we can
> leverage
> > a
> > >> > >>>>>>>>> ton of
> > >> > >>>>>>> their
> > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own config,
> > >> > >>>>>>>>> metrics,
> > >> > >>>>> etc.
> > >> > >>>>>>> We
> > >> > >>>>>>>>> can all share the same libraries, and make them better.
> This
> > >> > >>>>>>>>> will
> > >> > >>>>> also
> > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and will let
> > us
> > >> > >>>>> leverage
> > >> > >>>>>>>>> their offset management and partition management, rather
> > than
> > >> > >>>>>>>>> having
> > >> > >>>>>> our
> > >> > >>>>>>>>> own. All of the coordinator stream code would go away, as
> > >> would
> > >> > >>>>>>>>> most
> > >> > >>>>>> of
> > >> > >>>>>>>>> the
> > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
> > partition
> > >> > >>>>>>> management
> > >> > >>>>>>>>> features into the Kafka broker, but they're already moving
> > in
> > >> > >>>>>>>>> that direction with the new consumer API. The features we
> > have
> > >> > >>>>>>>>> for
> > >> > >>>>>> partition
> > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like they
> should
> > >> be
> > >> > >>>>>>>>> in
> > >> > >>>>>> Kafka
> > >> > >>>>>>>>> anyway. There will always be some niche usages which will
> > >> > >>>>>>>>> require
> > >> > >>>>>> extra
> > >> > >>>>>>>>> care and hence full control over partition assignments
> much
> > >> > >>>>>>>>> like the
> > >> > >>>>>>> Kafka
> > >> > >>>>>>>>> low level consumer api. These would continue to be
> > supported.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> These items will be good for the Samza community. They'll
> > make
> > >> > >>>>>>>>> Samza easier to use, and make it easier for developers to
> > add
> > >> > >>>>>>>>> new features.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat backwards
> > >> > >>>>> incompatible
> > >> > >>>>>>>>> change). If we choose to go this route, it's important
> that
> > we
> > >> > >>>>> openly
> > >> > >>>>>>>>> communicate how we're going to provide a migration path
> from
> > >> > >>>>>>>>> the
> > >> > >>>>>>> existing
> > >> > >>>>>>>>> APIs to the new ones (if we make incompatible changes). I
> > >> think
> > >> > >>>>>>>>> at a minimum, we'd probably need to provide a wrapper to
> > allow
> > >> > >>>>>>>>> existing StreamTask implementations to continue running on
> > the
> > >> > >>>> new container.
> > >> > >>>>>>> It's
> > >> > >>>>>>>>> also important that we openly communicate about timing,
> and
> > >> > >>>>>>>>> stages
> > >> > >>>>> of
> > >> > >>>>>>> the
> > >> > >>>>>>>>> migration.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> If you made it this far, I'm sure you have opinions. :)
> > Please
> > >> > >>>>>>>>> send
> > >> > >>>>>> your
> > >> > >>>>>>>>> thoughts and feedback.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Cheers,
> > >> > >>>>>>>>> Chris
> > >> > >>>>>>>>>
> > >> > >>>>>>>>
> > >> > >>>>>>>>
> > >> > >>>>>>>
> > >> > >>>>>>
> > >> > >>>>>>
> > >> > >>>>>>
> > >> > >>>>>> --
> > >> > >>>>>> -- Guozhang
> > >> > >>>>>>
> > >> > >>>>>
> > >> > >>>>
> > >> > >>
> > >> > >>
> > >> >
> > >> >
> > >> >
> > >>
> > >
> > >
> >
>

________________________________

This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@confluent.io>.
Julian, does this mean you AREN'T in favor of our new Kafka-based email
reader? How else will Kafka users read email :-)

Seriously, could not agree more.

-Jay

On Fri, Jul 10, 2015 at 2:03 PM, Julian Hyde <jh...@apache.org> wrote:

> I broadly support it, with one big proviso.
>
> One of the attractive things about Kafka has been its minimalism --
> the fact that it solves one part of the problem, simply, and very
> well. It is very important that it continues to do that, and that
> people continue to perceive it that way. Make Kafka into a stack, if
> you must, but make sure that the architectural layers remain clear.
>
> Making perfectly good software components into a muddled stack is one
> of the pitfalls of success, and in particular a pitfall of being a
> software company with a sales team looking for a bigger pieces to sell
> and customers looking for "simpler" large components to install. Avoid
> at all costs!
>
>
> On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io> wrote:
> > Yeah I agree with this summary. I think there are kind of two questions
> > here:
> > 1. Technically does alignment/reliance on Kafka make sense
> > 2. Branding wise (naming, website, concepts, etc) does alignment with
> Kafka
> > make sense
> >
> > Personally I do think both of these things would be really valuable, and
> > would dramatically alter the trajectory of the project.
> >
> > My preference would be to see if people can mostly agree on a direction
> > rather than splintering things off. From my point of view the ideal
> outcome
> > of all the options discussed would be to make Samza a closely aligned
> > subproject, maintained in a separate repository and retaining the
> existing
> > committership but sharing as much else as possible (website, etc). No
> idea
> > about how these things work, Jacob, you probably know more.
> >
> > No discussion amongst the Kafka folks has happened on this, but likely we
> > should figure out what the Samza community actually wants first.
> >
> > I admit that this is a fairly radical departure from how things are.
> >
> > If that doesn't fly, I think, yeah we could leave Samza as it is and do
> the
> > more radical reboot inside Kafka. From my point of view that does leave
> > things in a somewhat confusing state since now there are two stream
> > processing systems more or less coupled to Kafka in large part made by
> the
> > same people. But, arguably that might be a cleaner way to make the
> cut-over
> > and perhaps less risky for Samza community since if it works people can
> > switch and if it doesn't nothing will have changed. Dunno, how do people
> > feel about this?
> >
> > -Jay
> >
> > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com> wrote:
> >
> >> >  This leads me to thinking that merging projects and communities might
> >> be a good idea: with the union of experience from both communities, we
> will
> >> probably build a better system that is better for users.
> >> Is this what's being proposed though? Merging the projects seems like
> >> a consequence of at most one of the three directions under discussion:
> >> 1) Samza 2.0: The Samza community relies more heavily on Kafka for
> >> configuration, etc. (to a greater or lesser extent to be determined)
> >> but the Samza community would not automatically merge withe Kafka
> >> community (the Phoenix/HBase example is a good one here).
> >> 2) Samza Reboot: The Samza community continues to exist with a limited
> >> project scope, but similarly would not need to be part of the Kafka
> >> community (ie given committership) to progress.  Here, maybe the Samza
> >> team would become a subproject of Kafka (the Board frowns on
> >> subprojects at the moment, so I'm not sure if that's even feasible),
> >> but that would not be required.
> >> 3) Hey Samza! FYI, Kafka does streaming now: In this option the Kafka
> >> team builds its own streaming library, possibly off of Jay's
> >> prototype, which has not direct lineage to the Samza team.  There's no
> >> reason for the Kafka team to bring in the Samza team.
> >>
> >> Is the Kafka community on board with this?
> >>
> >> To be clear, all three options under discussion are interesting,
> >> technically valid and likely healthy directions for the project.
> >> Also, they are not mutually exclusive.  The Samza community could
> >> decide to pursue, say, 'Samza 2.0', while the Kafka community went
> >> forward with 'Hey Samza!'  My points above are directed entirely at
> >> the community aspect of these choices.
> >> -Jakob
> >>
> >> On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com> wrote:
> >> > That's great.  Thanks, Jay.
> >> >
> >> > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io> wrote:
> >> >
> >> >> Yeah totally agree. I think you have this issue even today, right?
> I.e.
> >> if
> >> >> you need to make a simple config change and you're running in YARN
> today
> >> >> you end up bouncing the job which then rebuilds state. I think the
> fix
> >> is
> >> >> exactly what you described which is to have a long timeout on
> partition
> >> >> movement for stateful jobs so that if a job is just getting bounced,
> and
> >> >> the cluster manager (or admin) is smart enough to restart it on the
> same
> >> >> host when possible, it can optimistically reuse any existing state it
> >> finds
> >> >> on disk (if it is valid).
> >> >>
> >> >> So in this model the charter of the CM is to place processes as
> >> stickily as
> >> >> possible and to restart or re-place failed processes. The charter of
> the
> >> >> partition management system is to control the assignment of work to
> >> these
> >> >> processes. The nice thing about this is that the work assignment,
> >> timeouts,
> >> >> behavior, configs, and code will all be the same across all cluster
> >> >> managers.
> >> >>
> >> >> So I think that prototype would actually give you exactly what you
> want
> >> >> today for any cluster manager (or manual placement + restart script)
> >> that
> >> >> was sticky in terms of host placement since there is already a
> >> configurable
> >> >> partition movement timeout and task-by-task state reuse with a check
> on
> >> >> state validity.
> >> >>
> >> >> -Jay
> >> >>
> >> >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> roger.hoover@gmail.com>
> >> >> wrote:
> >> >>
> >> >> > That would be great to let Kafka do as much heavy lifting as
> possible
> >> and
> >> >> > make it easier for other languages to implement Samza apis.
> >> >> >
> >> >> > One thing to watch out for is the interplay between Kafka's group
> >> >> > management and the external scheduler/process manager's fault
> >> tolerance.
> >> >> > If a container dies, the Kafka group membership protocol will try
> to
> >> >> assign
> >> >> > it's tasks to other containers while at the same time the process
> >> manager
> >> >> > is trying to relaunch the container.  Without some consideration
> for
> >> this
> >> >> > (like a configurable amount of time to wait before Kafka alters the
> >> group
> >> >> > membership), there may be thrashing going on which is especially
> bad
> >> for
> >> >> > containers with large amounts of local state.
> >> >> >
> >> >> > Someone else pointed this out already but I thought it might be
> worth
> >> >> > calling out again.
> >> >> >
> >> >> > Cheers,
> >> >> >
> >> >> > Roger
> >> >> >
> >> >> >
> >> >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <ja...@confluent.io>
> wrote:
> >> >> >
> >> >> > > Hey Roger,
> >> >> > >
> >> >> > > I couldn't agree more. We spent a bunch of time talking to people
> >> and
> >> >> > that
> >> >> > > is exactly the stuff we heard time and again. What makes it
> hard, of
> >> >> > > course, is that there is some tension between compatibility with
> >> what's
> >> >> > > there now and making things better for new users.
> >> >> > >
> >> >> > > I also strongly agree with the importance of multi-language
> >> support. We
> >> >> > are
> >> >> > > talking now about Java, but for application development use cases
> >> >> people
> >> >> > > want to work in whatever language they are using elsewhere. I
> think
> >> >> > moving
> >> >> > > to a model where Kafka itself does the group membership,
> lifecycle
> >> >> > control,
> >> >> > > and partition assignment has the advantage of putting all that
> >> complex
> >> >> > > stuff behind a clean api that the clients are already going to be
> >> >> > > implementing for their consumer, so the added functionality for
> >> stream
> >> >> > > processing beyond a consumer becomes very minor.
> >> >> > >
> >> >> > > -Jay
> >> >> > >
> >> >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> >> roger.hoover@gmail.com>
> >> >> > > wrote:
> >> >> > >
> >> >> > > > Metamorphosis...nice. :)
> >> >> > > >
> >> >> > > > This has been a great discussion.  As a user of Samza who's
> >> recently
> >> >> > > > integrated it into a relatively large organization, I just
> want to
> >> >> add
> >> >> > > > support to a few points already made.
> >> >> > > >
> >> >> > > > The biggest hurdles to adoption of Samza as it currently exists
> >> that
> >> >> > I've
> >> >> > > > experienced are:
> >> >> > > > 1) YARN - YARN is overly complex in many environments where
> Puppet
> >> >> > would
> >> >> > > do
> >> >> > > > just fine but it was the only mechanism to get fault tolerance.
> >> >> > > > 2) Configuration - I think I like the idea of configuring most
> of
> >> the
> >> >> > job
> >> >> > > > in code rather than config files.  In general, I think the goal
> >> >> should
> >> >> > be
> >> >> > > > to make it harder to make mistakes, especially of the kind
> where
> >> the
> >> >> > code
> >> >> > > > expects something and the config doesn't match.  The current
> >> config
> >> >> is
> >> >> > > > quite intricate and error-prone.  For example, the application
> >> logic
> >> >> > may
> >> >> > > > depend on bootstrapping a topic but rather than asserting that
> in
> >> the
> >> >> > > code,
> >> >> > > > you have to rely on getting the config right.  Likewise with
> >> serdes,
> >> >> > the
> >> >> > > > Java representations produced by various serdes (JSON, Avro,
> etc.)
> >> >> are
> >> >> > > not
> >> >> > > > equivalent so you cannot just reconfigure a serde without
> changing
> >> >> the
> >> >> > > > code.   It would be nice for jobs to be able to assert what
> they
> >> >> expect
> >> >> > > > from their input topics in terms of partitioning.  This is
> >> getting a
> >> >> > > little
> >> >> > > > off topic but I was even thinking about creating a "Samza
> config
> >> >> > linter"
> >> >> > > > that would sanity check a set of configs.  Especially in
> >> >> organizations
> >> >> > > > where config is managed by a different team than the
> application
> >> >> > > developer,
> >> >> > > > it's very hard to get avoid config mistakes.
> >> >> > > > 3) Java/Scala centric - for many teams (especially DevOps-type
> >> >> folks),
> >> >> > > the
> >> >> > > > pain of the Java toolchain (maven, slow builds, weak command
> line
> >> >> > > support,
> >> >> > > > configuration over convention) really inhibits productivity.
> As
> >> more
> >> >> > and
> >> >> > > > more high-quality clients become available for Kafka, I hope
> >> they'll
> >> >> > > follow
> >> >> > > > Samza's model.  Not sure how much it affects the proposals in
> this
> >> >> > thread
> >> >> > > > but please consider other languages in the ecosystem as well.
> >> From
> >> >> > what
> >> >> > > > I've heard, Spark has more Python users than Java/Scala.
> >> >> > > > (FYI, we added a Jython wrapper for the Samza API
> >> >> > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> >> >> > > > and are working on a Yeoman generator
> >> >> > > > https://github.com/Quantiply/generator-rico for Jython/Samza
> >> >> projects
> >> >> > to
> >> >> > > > alleviate some of the pain)
> >> >> > > >
> >> >> > > > I also want to underscore Jay's point about improving the user
> >> >> > > experience.
> >> >> > > > That's a very important factor for adoption.  I think the goal
> >> should
> >> >> > be
> >> >> > > to
> >> >> > > > make Samza as easy to get started with as something like
> Logstash.
> >> >> > > > Logstash is vastly inferior in terms of capabilities to Samza
> but
> >> >> it's
> >> >> > > easy
> >> >> > > > to get started and that makes a big difference.
> >> >> > > >
> >> >> > > > Cheers,
> >> >> > > >
> >> >> > > > Roger
> >> >> > > >
> >> >> > > >
> >> >> > > >
> >> >> > > >
> >> >> > > >
> >> >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales
> <
> >> >> > > > gdfm@apache.org> wrote:
> >> >> > > >
> >> >> > > > > Forgot to add. On the naming issues, Kafka Metamorphosis is a
> >> clear
> >> >> > > > winner
> >> >> > > > > :)
> >> >> > > > >
> >> >> > > > > --
> >> >> > > > > Gianmarco
> >> >> > > > >
> >> >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
> >> >> > > gdfm@apache.org
> >> >> > > > >
> >> >> > > > > wrote:
> >> >> > > > >
> >> >> > > > > > Hi,
> >> >> > > > > >
> >> >> > > > > > @Martin, thanks for you comments.
> >> >> > > > > > Maybe I'm missing some important point, but I think
> coupling
> >> the
> >> >> > > > releases
> >> >> > > > > > is actually a *good* thing.
> >> >> > > > > > To make an example, would it be better if the MR and HDFS
> >> >> > components
> >> >> > > of
> >> >> > > > > > Hadoop had different release schedules?
> >> >> > > > > >
> >> >> > > > > > Actually, keeping the discussion in a single place would
> make
> >> >> > > agreeing
> >> >> > > > on
> >> >> > > > > > releases (and backwards compatibility) much easier, as
> >> everybody
> >> >> > > would
> >> >> > > > be
> >> >> > > > > > responsible for the whole codebase.
> >> >> > > > > >
> >> >> > > > > > That said, I like the idea of absorbing samza-core as a
> >> >> > sub-project,
> >> >> > > > and
> >> >> > > > > > leave the fancy stuff separate.
> >> >> > > > > > It probably gives 90% of the benefits we have been
> discussing
> >> >> here.
> >> >> > > > > >
> >> >> > > > > > Cheers,
> >> >> > > > > >
> >> >> > > > > > --
> >> >> > > > > > Gianmarco
> >> >> > > > > >
> >> >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com>
> >> wrote:
> >> >> > > > > >
> >> >> > > > > >> Hey Martin,
> >> >> > > > > >>
> >> >> > > > > >> I agree coupling release schedules is a downside.
> >> >> > > > > >>
> >> >> > > > > >> Definitely we can try to solve some of the integration
> >> problems
> >> >> in
> >> >> > > > > >> Confluent Platform or in other distributions. But I think
> >> this
> >> >> > ends
> >> >> > > up
> >> >> > > > > >> being really shallow. I guess I feel to really get a good
> >> user
> >> >> > > > > experience
> >> >> > > > > >> the two systems have to kind of feel like part of the same
> >> thing
> >> >> > and
> >> >> > > > you
> >> >> > > > > >> can't really add that in later--you can put both in the
> same
> >> >> > > > > downloadable
> >> >> > > > > >> tar file but it doesn't really give a very cohesive
> feeling.
> >> I
> >> >> > agree
> >> >> > > > > that
> >> >> > > > > >> ultimately any of the project stuff is as much social and
> >> naming
> >> >> > as
> >> >> > > > > >> anything else--theoretically two totally independent
> projects
> >> >> > could
> >> >> > > > work
> >> >> > > > > >> to
> >> >> > > > > >> tightly align. In practice this seems to be quite
> difficult
> >> >> > though.
> >> >> > > > > >>
> >> >> > > > > >> For the frameworks--totally agree it would be good to
> >> maintain
> >> >> the
> >> >> > > > > >> framework support with the project. In some cases there
> may
> >> not
> >> >> be
> >> >> > > too
> >> >> > > > > >> much
> >> >> > > > > >> there since the integration gets lighter but I think
> whatever
> >> >> > stubs
> >> >> > > > you
> >> >> > > > > >> need should be included. So no I definitely wasn't trying
> to
> >> >> imply
> >> >> > > > > >> dropping
> >> >> > > > > >> support for these frameworks, just making the integration
> >> >> lighter
> >> >> > by
> >> >> > > > > >> separating process management from partition management.
> >> >> > > > > >>
> >> >> > > > > >> You raise two good points we would have to figure out if
> we
> >> went
> >> >> > > down
> >> >> > > > > the
> >> >> > > > > >> alignment path:
> >> >> > > > > >> 1. With respect to the name, yeah I think the first
> question
> >> is
> >> >> > > > whether
> >> >> > > > > >> some "re-branding" would be worth it. If so then I think
> we
> >> can
> >> >> > > have a
> >> >> > > > > big
> >> >> > > > > >> thread on the name. I'm definitely not set on Kafka
> >> Streaming or
> >> >> > > Kafka
> >> >> > > > > >> Streams I was just using them to be kind of illustrative.
> I
> >> >> agree
> >> >> > > with
> >> >> > > > > >> your
> >> >> > > > > >> critique of these names, though I think people would get
> the
> >> >> idea.
> >> >> > > > > >> 2. Yeah you also raise a good point about how to "factor"
> it.
> >> >> Here
> >> >> > > are
> >> >> > > > > the
> >> >> > > > > >> options I see (I could get enthusiastic about any of
> them):
> >> >> > > > > >>    a. One repo for both Kafka and Samza
> >> >> > > > > >>    b. Two repos, retaining the current seperation
> >> >> > > > > >>    c. Two repos, the equivalent of samza-api and
> samza-core
> >> is
> >> >> > > > absorbed
> >> >> > > > > >> almost like a third client
> >> >> > > > > >>
> >> >> > > > > >> Cheers,
> >> >> > > > > >>
> >> >> > > > > >> -Jay
> >> >> > > > > >>
> >> >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> >> >> > > > martin@kleppmann.com>
> >> >> > > > > >> wrote:
> >> >> > > > > >>
> >> >> > > > > >> > Ok, thanks for the clarifications. Just a few follow-up
> >> >> > comments.
> >> >> > > > > >> >
> >> >> > > > > >> > - I see the appeal of merging with Kafka or becoming a
> >> >> > subproject:
> >> >> > > > the
> >> >> > > > > >> > reasons you mention are good. The risk I see is that
> >> release
> >> >> > > > schedules
> >> >> > > > > >> > become coupled to each other, which can slow everyone
> down,
> >> >> and
> >> >> > > > large
> >> >> > > > > >> > projects with many contributors are harder to manage.
> >> (Jakob,
> >> >> > can
> >> >> > > > you
> >> >> > > > > >> speak
> >> >> > > > > >> > from experience, having seen a wider range of Hadoop
> >> ecosystem
> >> >> > > > > >> projects?)
> >> >> > > > > >> >
> >> >> > > > > >> > Some of the goals of a better unified developer
> experience
> >> >> could
> >> >> > > > also
> >> >> > > > > be
> >> >> > > > > >> > solved by integrating Samza nicely into a Kafka
> >> distribution
> >> >> > (such
> >> >> > > > as
> >> >> > > > > >> > Confluent's). I'm not against merging projects if we
> decide
> >> >> > that's
> >> >> > > > the
> >> >> > > > > >> way
> >> >> > > > > >> > to go, just pointing out the same goals can perhaps
> also be
> >> >> > > achieved
> >> >> > > > > in
> >> >> > > > > >> > other ways.
> >> >> > > > > >> >
> >> >> > > > > >> > - With regard to dropping the YARN dependency: are you
> >> >> proposing
> >> >> > > > that
> >> >> > > > > >> > Samza doesn't give any help to people wanting to run on
> >> >> > > > > >> YARN/Mesos/AWS/etc?
> >> >> > > > > >> > So the docs would basically have a link to Slider and
> >> nothing
> >> >> > > else?
> >> >> > > > Or
> >> >> > > > > >> > would we maintain integrations with a bunch of popular
> >> >> > deployment
> >> >> > > > > >> methods
> >> >> > > > > >> > (e.g. the necessary glue and shell scripts to make Samza
> >> work
> >> >> > with
> >> >> > > > > >> Slider)?
> >> >> > > > > >> >
> >> >> > > > > >> > I absolutely think it's a good idea to have the "as a
> >> library"
> >> >> > and
> >> >> > > > > "as a
> >> >> > > > > >> > process" (using Yi's taxonomy) options for people who
> want
> >> >> them,
> >> >> > > > but I
> >> >> > > > > >> > think there should also be a low-friction path for
> common
> >> "as
> >> >> a
> >> >> > > > > service"
> >> >> > > > > >> > deployment methods, for which we probably need to
> maintain
> >> >> > > > > integrations.
> >> >> > > > > >> >
> >> >> > > > > >> > - Project naming: "Kafka Streams" seems odd to me,
> because
> >> >> Kafka
> >> >> > > is
> >> >> > > > > all
> >> >> > > > > >> > about streams already. Perhaps "Kafka Transformers" or
> >> "Kafka
> >> >> > > > Filters"
> >> >> > > > > >> > would be more apt?
> >> >> > > > > >> >
> >> >> > > > > >> > One suggestion: perhaps the core of Samza (stream
> >> >> transformation
> >> >> > > > with
> >> >> > > > > >> > state management -- i.e. the "Samza as a library" bit)
> >> could
> >> >> > > become
> >> >> > > > > >> part of
> >> >> > > > > >> > Kafka, while higher-level tools such as streaming SQL
> and
> >> >> > > > integrations
> >> >> > > > > >> with
> >> >> > > > > >> > deployment frameworks remain in a separate project? In
> >> other
> >> >> > > words,
> >> >> > > > > >> Kafka
> >> >> > > > > >> > would absorb the proven, stable core of Samza, which
> would
> >> >> > become
> >> >> > > > the
> >> >> > > > > >> > "third Kafka client" mentioned early in this thread. The
> >> Samza
> >> >> > > > project
> >> >> > > > > >> > would then target that third Kafka client as its base
> API,
> >> and
> >> >> > the
> >> >> > > > > >> project
> >> >> > > > > >> > would be freed up to explore more experimental new
> >> horizons.
> >> >> > > > > >> >
> >> >> > > > > >> > Martin
> >> >> > > > > >> >
> >> >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <jay.kreps@gmail.com
> >
> >> >> wrote:
> >> >> > > > > >> >
> >> >> > > > > >> > > Hey Martin,
> >> >> > > > > >> > >
> >> >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually don't
> think
> >> it
> >> >> > ties
> >> >> > > > our
> >> >> > > > > >> > hands
> >> >> > > > > >> > > at all, all it does is refactor things. The division
> of
> >> >> > > > > >> responsibility is
> >> >> > > > > >> > > that Samza core is responsible for task lifecycle,
> state,
> >> >> and
> >> >> > > > > >> partition
> >> >> > > > > >> > > management (using the Kafka co-ordinator) but it is
> NOT
> >> >> > > > responsible
> >> >> > > > > >> for
> >> >> > > > > >> > > packaging, configuration deployment or execution of
> >> >> processes.
> >> >> > > The
> >> >> > > > > >> > problem
> >> >> > > > > >> > > of packaging and starting these processes is
> >> >> > > > > >> > > framework/environment-specific. This leaves individual
> >> >> > > frameworks
> >> >> > > > to
> >> >> > > > > >> be
> >> >> > > > > >> > as
> >> >> > > > > >> > > fancy or vanilla as they like. So you can get simple
> >> >> stateless
> >> >> > > > > >> support in
> >> >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app
> framework
> >> >> > > (Slider,
> >> >> > > > > >> > Marathon,
> >> >> > > > > >> > > etc). These are well known by people and have nice UIs
> >> and a
> >> >> > lot
> >> >> > > > of
> >> >> > > > > >> > > flexibility. I don't think they have node affinity as
> a
> >> >> built
> >> >> > in
> >> >> > > > > >> option
> >> >> > > > > >> > > (though I could be wrong). So if we want that we can
> >> either
> >> >> > wait
> >> >> > > > for
> >> >> > > > > >> them
> >> >> > > > > >> > > to add it or do a custom framework to add that feature
> >> (as
> >> >> > now).
> >> >> > > > > >> > Obviously
> >> >> > > > > >> > > if you manage things with old-school ops tools
> >> >> > (puppet/chef/etc)
> >> >> > > > you
> >> >> > > > > >> get
> >> >> > > > > >> > > locality easily. The nice thing, though, is that all
> the
> >> >> samza
> >> >> > > > > >> "business
> >> >> > > > > >> > > logic" around partition management and fault tolerance
> >> is in
> >> >> > > Samza
> >> >> > > > > >> core
> >> >> > > > > >> > so
> >> >> > > > > >> > > it is shared across frameworks and the framework
> specific
> >> >> bit
> >> >> > is
> >> >> > > > > just
> >> >> > > > > >> > > whether it is smart enough to try to get the same host
> >> when
> >> >> a
> >> >> > > job
> >> >> > > > is
> >> >> > > > > >> > > restarted.
> >> >> > > > > >> > >
> >> >> > > > > >> > > With respect to the Kafka-alignment, yeah I think the
> >> goal
> >> >> > would
> >> >> > > > be
> >> >> > > > > >> (a)
> >> >> > > > > >> > > actually get better alignment in user experience, and
> (b)
> >> >> > > express
> >> >> > > > > >> this in
> >> >> > > > > >> > > the naming and project branding. Specifically:
> >> >> > > > > >> > > 1. Website/docs, it would be nice for the
> >> "transformation"
> >> >> api
> >> >> > > to
> >> >> > > > be
> >> >> > > > > >> > > discoverable in the main Kafka docs--i.e. be able to
> >> explain
> >> >> > > when
> >> >> > > > to
> >> >> > > > > >> use
> >> >> > > > > >> > > the consumer and when to use the stream processing
> >> >> > functionality
> >> >> > > > and
> >> >> > > > > >> lead
> >> >> > > > > >> > > people into that experience.
> >> >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or
> >> whatever)
> >> >> > that
> >> >> > > > has
> >> >> > > > > >> both
> >> >> > > > > >> > > Kafka and the stream processing part and they actually
> >> work
> >> >> > > > > together.
> >> >> > > > > >> > > 3. Unify the programming experience so the client and
> >> Samza
> >> >> > api
> >> >> > > > > share
> >> >> > > > > >> > > config/monitoring/naming/packaging/etc.
> >> >> > > > > >> > >
> >> >> > > > > >> > > I think sub-projects keep separate committers and can
> >> have a
> >> >> > > > > separate
> >> >> > > > > >> > repo,
> >> >> > > > > >> > > but I'm actually not really sure (I can't find a
> >> definition
> >> >> > of a
> >> >> > > > > >> > subproject
> >> >> > > > > >> > > in Apache).
> >> >> > > > > >> > >
> >> >> > > > > >> > > Basically at a high-level you want the experience to
> >> "feel"
> >> >> > > like a
> >> >> > > > > >> single
> >> >> > > > > >> > > system, not to relatively independent things that are
> >> kind
> >> >> of
> >> >> > > > > >> awkwardly
> >> >> > > > > >> > > glued together.
> >> >> > > > > >> > >
> >> >> > > > > >> > > I think if we did that they having naming or branding
> >> like
> >> >> > > "kafka
> >> >> > > > > >> > > streaming" or "kafka streams" or something like that
> >> would
> >> >> > > > actually
> >> >> > > > > >> do a
> >> >> > > > > >> > > good job of conveying what it is. I do that this would
> >> help
> >> >> > > > adoption
> >> >> > > > > >> > quite
> >> >> > > > > >> > > a lot as it would correctly convey that using Kafka
> >> >> Streaming
> >> >> > > with
> >> >> > > > > >> Kafka
> >> >> > > > > >> > is
> >> >> > > > > >> > > a fairly seamless experience and Kafka is pretty
> heavily
> >> >> > adopted
> >> >> > > > at
> >> >> > > > > >> this
> >> >> > > > > >> > > point.
> >> >> > > > > >> > >
> >> >> > > > > >> > > Fwiw we actually considered this model originally when
> >> open
> >> >> > > > sourcing
> >> >> > > > > >> > Samza,
> >> >> > > > > >> > > however at that time Kafka was relatively unknown and
> we
> >> >> > decided
> >> >> > > > not
> >> >> > > > > >> to
> >> >> > > > > >> > do
> >> >> > > > > >> > > it since we felt it would be limiting. From my point
> of
> >> view
> >> >> > the
> >> >> > > > > three
> >> >> > > > > >> > > things have changed (1) Kafka is now really heavily
> used
> >> for
> >> >> > > > stream
> >> >> > > > > >> > > processing, (2) we learned that abstracting out the
> >> stream
> >> >> > well
> >> >> > > is
> >> >> > > > > >> > > basically impossible, (3) we learned it is really
> hard to
> >> >> keep
> >> >> > > the
> >> >> > > > > two
> >> >> > > > > >> > > things feeling like a single product.
> >> >> > > > > >> > >
> >> >> > > > > >> > > -Jay
> >> >> > > > > >> > >
> >> >> > > > > >> > >
> >> >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> >> >> > > > > >> martin@kleppmann.com>
> >> >> > > > > >> > > wrote:
> >> >> > > > > >> > >
> >> >> > > > > >> > >> Hi all,
> >> >> > > > > >> > >>
> >> >> > > > > >> > >> Lots of good thoughts here.
> >> >> > > > > >> > >>
> >> >> > > > > >> > >> I agree with the general philosophy of tying Samza
> more
> >> >> > firmly
> >> >> > > to
> >> >> > > > > >> Kafka.
> >> >> > > > > >> > >> After I spent a while looking at integrating other
> >> message
> >> >> > > > brokers
> >> >> > > > > >> (e.g.
> >> >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
> conclusion
> >> that
> >> >> > > > > >> > SystemConsumer
> >> >> > > > > >> > >> tacitly assumes a model so much like Kafka's that
> pretty
> >> >> much
> >> >> > > > > nobody
> >> >> > > > > >> but
> >> >> > > > > >> > >> Kafka actually implements it. (Databus is perhaps an
> >> >> > exception,
> >> >> > > > but
> >> >> > > > > >> it
> >> >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus, making
> >> Samza
> >> >> > > fully
> >> >> > > > > >> > dependent
> >> >> > > > > >> > >> on Kafka acknowledges that the system-independence
> was
> >> >> never
> >> >> > as
> >> >> > > > > real
> >> >> > > > > >> as
> >> >> > > > > >> > we
> >> >> > > > > >> > >> perhaps made it out to be. The gains of code reuse
> are
> >> >> real.
> >> >> > > > > >> > >>
> >> >> > > > > >> > >> The idea of decoupling Samza from YARN has also
> always
> >> been
> >> >> > > > > >> appealing to
> >> >> > > > > >> > >> me, for various reasons already mentioned in this
> >> thread.
> >> >> > > > Although
> >> >> > > > > >> > making
> >> >> > > > > >> > >> Samza jobs deployable on anything
> (YARN/Mesos/AWS/etc)
> >> >> seems
> >> >> > > > > >> laudable,
> >> >> > > > > >> > I am
> >> >> > > > > >> > >> a little concerned that it will restrict us to a
> lowest
> >> >> > common
> >> >> > > > > >> > denominator.
> >> >> > > > > >> > >> For example, would host affinity (SAMZA-617) still be
> >> >> > possible?
> >> >> > > > For
> >> >> > > > > >> jobs
> >> >> > > > > >> > >> with large amounts of state, I think SAMZA-617 would
> be
> >> a
> >> >> big
> >> >> > > > boon,
> >> >> > > > > >> > since
> >> >> > > > > >> > >> restoring state off the changelog on every single
> >> restart
> >> >> is
> >> >> > > > > painful,
> >> >> > > > > >> > due
> >> >> > > > > >> > >> to long recovery times. It would be a shame if the
> >> >> decoupling
> >> >> > > > from
> >> >> > > > > >> YARN
> >> >> > > > > >> > >> made host affinity impossible.
> >> >> > > > > >> > >>
> >> >> > > > > >> > >> Jay, a question about the proposed API for
> >> instantiating a
> >> >> > job
> >> >> > > in
> >> >> > > > > >> code
> >> >> > > > > >> > >> (rather than a properties file): when submitting a
> job
> >> to a
> >> >> > > > > cluster,
> >> >> > > > > >> is
> >> >> > > > > >> > the
> >> >> > > > > >> > >> idea that the instantiation code runs on a client
> >> >> somewhere,
> >> >> > > > which
> >> >> > > > > >> then
> >> >> > > > > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc?
> Or
> >> >> does
> >> >> > > that
> >> >> > > > > >> code
> >> >> > > > > >> > run
> >> >> > > > > >> > >> on each container that is part of the job (in which
> >> case,
> >> >> how
> >> >> > > > does
> >> >> > > > > >> the
> >> >> > > > > >> > job
> >> >> > > > > >> > >> submission to the cluster work)?
> >> >> > > > > >> > >>
> >> >> > > > > >> > >> I agree with Garry that it doesn't feel right to
> make a
> >> 1.0
> >> >> > > > release
> >> >> > > > > >> > with a
> >> >> > > > > >> > >> plan for it to be immediately obsolete. So if this is
> >> going
> >> >> > to
> >> >> > > > > >> happen, I
> >> >> > > > > >> > >> think it would be more honest to stick with 0.*
> version
> >> >> > numbers
> >> >> > > > > until
> >> >> > > > > >> > the
> >> >> > > > > >> > >> library-ified Samza has been implemented, is stable
> and
> >> >> > widely
> >> >> > > > > used.
> >> >> > > > > >> > >>
> >> >> > > > > >> > >> Should the new Samza be a subproject of Kafka? There
> is
> >> >> > > precedent
> >> >> > > > > for
> >> >> > > > > >> > >> tight coupling between different Apache projects
> (e.g.
> >> >> > Curator
> >> >> > > > and
> >> >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think remaining
> >> >> separate
> >> >> > > > would
> >> >> > > > > >> be
> >> >> > > > > >> > ok.
> >> >> > > > > >> > >> Even if Samza is fully dependent on Kafka, there is
> >> enough
> >> >> > > > > substance
> >> >> > > > > >> in
> >> >> > > > > >> > >> Samza that it warrants being a separate project. An
> >> >> argument
> >> >> > in
> >> >> > > > > >> favour
> >> >> > > > > >> > of
> >> >> > > > > >> > >> merging would be if we think Kafka has a much
> stronger
> >> >> "brand
> >> >> > > > > >> presence"
> >> >> > > > > >> > >> than Samza; I'm ambivalent on that one. If the Kafka
> >> >> project
> >> >> > is
> >> >> > > > > >> willing
> >> >> > > > > >> > to
> >> >> > > > > >> > >> endorse Samza as the "official" way of doing stateful
> >> >> stream
> >> >> > > > > >> > >> transformations, that would probably have much the
> same
> >> >> > effect
> >> >> > > as
> >> >> > > > > >> > >> re-branding Samza as "Kafka Stream Processors" or
> >> suchlike.
> >> >> > > Close
> >> >> > > > > >> > >> collaboration between the two projects will be
> needed in
> >> >> any
> >> >> > > > case.
> >> >> > > > > >> > >>
> >> >> > > > > >> > >> From a project management perspective, I guess the
> "new
> >> >> > Samza"
> >> >> > > > > would
> >> >> > > > > >> > have
> >> >> > > > > >> > >> to be developed on a branch alongside ongoing
> >> maintenance
> >> >> of
> >> >> > > the
> >> >> > > > > >> current
> >> >> > > > > >> > >> line of development? I think it would be important to
> >> >> > continue
> >> >> > > > > >> > supporting
> >> >> > > > > >> > >> existing users, and provide a graceful migration
> path to
> >> >> the
> >> >> > > new
> >> >> > > > > >> > version.
> >> >> > > > > >> > >> Leaving the current versions unsupported and forcing
> >> people
> >> >> > to
> >> >> > > > > >> rewrite
> >> >> > > > > >> > >> their jobs would send a bad signal.
> >> >> > > > > >> > >>
> >> >> > > > > >> > >> Best,
> >> >> > > > > >> > >> Martin
> >> >> > > > > >> > >>
> >> >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <jay@confluent.io
> >
> >> >> wrote:
> >> >> > > > > >> > >>
> >> >> > > > > >> > >>> Hey Garry,
> >> >> > > > > >> > >>>
> >> >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy to chat
> >> more
> >> >> > about
> >> >> > > > > this
> >> >> > > > > >> if
> >> >> > > > > >> > >>> you'd be interested. I think Chris and I started
> with
> >> the
> >> >> > idea
> >> >> > > > of
> >> >> > > > > >> "what
> >> >> > > > > >> > >>> would it take to make Samza a kick-ass ingestion
> tool"
> >> but
> >> >> > > > > >> ultimately
> >> >> > > > > >> > we
> >> >> > > > > >> > >>> kind of came around to the idea that ingestion and
> >> >> > > > transformation
> >> >> > > > > >> had
> >> >> > > > > >> > >>> pretty different needs and coupling the two made
> things
> >> >> > hard.
> >> >> > > > > >> > >>>
> >> >> > > > > >> > >>> For what it's worth I think copycat (KIP-26)
> actually
> >> will
> >> >> > do
> >> >> > > > what
> >> >> > > > > >> you
> >> >> > > > > >> > >> are
> >> >> > > > > >> > >>> looking for.
> >> >> > > > > >> > >>>
> >> >> > > > > >> > >>> With regard to your point about slider, I don't
> >> >> necessarily
> >> >> > > > > >> disagree.
> >> >> > > > > >> > >> But I
> >> >> > > > > >> > >>> think getting good YARN support is quite doable and
> I
> >> >> think
> >> >> > we
> >> >> > > > can
> >> >> > > > > >> make
> >> >> > > > > >> > >>> that work well. I think the issue this proposal
> solves
> >> is
> >> >> > that
> >> >> > > > > >> > >> technically
> >> >> > > > > >> > >>> it is pretty hard to support multiple cluster
> >> management
> >> >> > > systems
> >> >> > > > > the
> >> >> > > > > >> > way
> >> >> > > > > >> > >>> things are now, you need to write an "app master" or
> >> >> > > "framework"
> >> >> > > > > for
> >> >> > > > > >> > each
> >> >> > > > > >> > >>> and they are all a little different so testing is
> >> really
> >> >> > hard.
> >> >> > > > In
> >> >> > > > > >> the
> >> >> > > > > >> > >>> absence of this we have been stuck with just YARN
> which
> >> >> has
> >> >> > > > > >> fantastic
> >> >> > > > > >> > >>> penetration in the Hadoopy part of the org, but zero
> >> >> > > penetration
> >> >> > > > > >> > >> elsewhere.
> >> >> > > > > >> > >>> Given the huge amount of work being put in to
> slider,
> >> >> > > marathon,
> >> >> > > > > aws
> >> >> > > > > >> > >>> tooling, not to mention the umpteen related
> packaging
> >> >> > > > technologies
> >> >> > > > > >> > people
> >> >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> cloud-specific
> >> >> > deploy
> >> >> > > > > >> tools,
> >> >> > > > > >> > >> etc)
> >> >> > > > > >> > >>> I really think it is important to get this right.
> >> >> > > > > >> > >>>
> >> >> > > > > >> > >>> -Jay
> >> >> > > > > >> > >>>
> >> >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> >> >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> >> >> > > > > >> > >>>
> >> >> > > > > >> > >>>> Hi all,
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>> I think the question below re does Samza become a
> >> >> > sub-project
> >> >> > > > of
> >> >> > > > > >> Kafka
> >> >> > > > > >> > >>>> highlights the broader point around migration.
> Chris
> >> >> > mentions
> >> >> > > > > >> Samza's
> >> >> > > > > >> > >>>> maturity is heading towards a v1 release but I'm
> not
> >> sure
> >> >> > it
> >> >> > > > > feels
> >> >> > > > > >> > >> right to
> >> >> > > > > >> > >>>> launch a v1 then immediately plan to deprecate
> most of
> >> >> it.
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>> From a selfish perspective I have some guys who
> have
> >> >> > started
> >> >> > > > > >> working
> >> >> > > > > >> > >> with
> >> >> > > > > >> > >>>> Samza and building some new consumers/producers was
> >> next
> >> >> > up.
> >> >> > > > > Sounds
> >> >> > > > > >> > like
> >> >> > > > > >> > >>>> that is absolutely not the direction to go. I need
> to
> >> >> look
> >> >> > > into
> >> >> > > > > the
> >> >> > > > > >> > KIP
> >> >> > > > > >> > >> in
> >> >> > > > > >> > >>>> more detail but for me the attractiveness of adding
> >> new
> >> >> > Samza
> >> >> > > > > >> > >>>> consumer/producers -- even if yes all they were
> doing
> >> was
> >> >> > > > really
> >> >> > > > > >> > getting
> >> >> > > > > >> > >>>> data into and out of Kafka --  was to avoid
> having to
> >> >> > worry
> >> >> > > > > about
> >> >> > > > > >> the
> >> >> > > > > >> > >>>> lifecycle management of external clients. If there
> is
> >> a
> >> >> > > generic
> >> >> > > > > >> Kafka
> >> >> > > > > >> > >>>> ingress/egress layer that I can plug a new
> connector
> >> into
> >> >> > and
> >> >> > > > > have
> >> >> > > > > >> a
> >> >> > > > > >> > >> lot of
> >> >> > > > > >> > >>>> the heavy lifting re scale and reliability done
> for me
> >> >> then
> >> >> > > it
> >> >> > > > > >> gives
> >> >> > > > > >> > me
> >> >> > > > > >> > >> all
> >> >> > > > > >> > >>>> the pushing new consumers/producers would. If not
> >> then it
> >> >> > > > > >> complicates
> >> >> > > > > >> > my
> >> >> > > > > >> > >>>> operational deployments.
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>> Which is similar to my other question with the
> >> proposal
> >> >> --
> >> >> > if
> >> >> > > > we
> >> >> > > > > >> > build a
> >> >> > > > > >> > >>>> fully available/stand-alone Samza plus the
> requisite
> >> >> shims
> >> >> > to
> >> >> > > > > >> > integrate
> >> >> > > > > >> > >>>> with Slider etc I suspect the former may be a lot
> more
> >> >> work
> >> >> > > > than
> >> >> > > > > we
> >> >> > > > > >> > >> think.
> >> >> > > > > >> > >>>> We may make it much easier for a newcomer to get
> >> >> something
> >> >> > > > > running
> >> >> > > > > >> but
> >> >> > > > > >> > >>>> having them step up and get a reliable production
> >> >> > deployment
> >> >> > > > may
> >> >> > > > > >> still
> >> >> > > > > >> > >>>> dominate mailing list  traffic, if for different
> >> reasons
> >> >> > than
> >> >> > > > > >> today.
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with making
> the
> >> >> Samza
> >> >> > > > > >> dependency
> >> >> > > > > >> > >> on
> >> >> > > > > >> > >>>> Kafka much more explicit and I absolutely see the
> >> >> benefits
> >> >> > > in
> >> >> > > > > the
> >> >> > > > > >> > >>>> reduction of duplication and clashing
> >> >> > > > terminologies/abstractions
> >> >> > > > > >> that
> >> >> > > > > >> > >>>> Chris/Jay describe. Samza as a library would likely
> >> be a
> >> >> > very
> >> >> > > > > nice
> >> >> > > > > >> > tool
> >> >> > > > > >> > >> to
> >> >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the
> concerns
> >> >> above
> >> >> > re
> >> >> > > > the
> >> >> > > > > >> > >>>> operational side.
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>> Garry
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>> -----Original Message-----
> >> >> > > > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
> >> >> > gdfm@apache.org
> >> >> > > ]
> >> >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> >> >> > > > > >> > >>>> To: dev@samza.apache.org
> >> >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>> Very interesting thoughts.
> >> >> > > > > >> > >>>> From outside, I have always perceived Samza as a
> >> >> computing
> >> >> > > > layer
> >> >> > > > > >> over
> >> >> > > > > >> > >>>> Kafka.
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>> The question, maybe a bit provocative, is "should
> >> Samza
> >> >> be
> >> >> > a
> >> >> > > > > >> > sub-project
> >> >> > > > > >> > >>>> of Kafka then?"
> >> >> > > > > >> > >>>> Or does it make sense to keep it as a separate
> project
> >> >> > with a
> >> >> > > > > >> separate
> >> >> > > > > >> > >>>> governance?
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>> Cheers,
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>> --
> >> >> > > > > >> > >>>> Gianmarco
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> >> yanfang724@gmail.com>
> >> >> > > > wrote:
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more
> tightly.
> >> >> > Because
> >> >> > > > > Samza
> >> >> > > > > >> de
> >> >> > > > > >> > >>>>> facto is based on Kafka, and it should leverage
> what
> >> >> Kafka
> >> >> > > > has.
> >> >> > > > > At
> >> >> > > > > >> > the
> >> >> > > > > >> > >>>>> same time, Kafka does not need to reinvent what
> Samza
> >> >> > > already
> >> >> > > > > >> has. I
> >> >> > > > > >> > >>>>> also like the idea of separating the ingestion and
> >> >> > > > > transformation.
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>> But it is a little difficult for me to image how
> the
> >> >> Samza
> >> >> > > > will
> >> >> > > > > >> look
> >> >> > > > > >> > >>>> like.
> >> >> > > > > >> > >>>>> And I feel Chris and Jay have a little difference
> in
> >> >> terms
> >> >> > > of
> >> >> > > > > how
> >> >> > > > > >> > >>>>> Samza should look like.
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>> *** Will it look like what Jay's code shows (A
> >> client of
> >> >> > > > Kakfa)
> >> >> > > > > ?
> >> >> > > > > >> And
> >> >> > > > > >> > >>>>> user's application code calls this client?
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka (like
> what
> >> the
> >> >> > > code
> >> >> > > > > >> shows),
> >> >> > > > > >> > >>>>> how do we implement auto-balance and
> fault-tolerance?
> >> >> Are
> >> >> > > they
> >> >> > > > > >> taken
> >> >> > > > > >> > >>>>> care by the Kafka broker or other mechanism, such
> as
> >> >> > "Samza
> >> >> > > > > >> worker"
> >> >> > > > > >> > >>>>> (just make up the name) ?
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>> 2. What about other features, such as
> auto-scaling,
> >> >> shared
> >> >> > > > > state,
> >> >> > > > > >> > >>>>> monitoring?
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>> *** If we have Samza standalone, (is this what
> Chris
> >> >> > > > suggests?)
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa and
> >> produce
> >> >> to
> >> >> > > it.
> >> >> > > > > >> Then it
> >> >> > > > > >> > >>>>> becomes the same as what Samza looks like now,
> >> except it
> >> >> > > does
> >> >> > > > > not
> >> >> > > > > >> > rely
> >> >> > > > > >> > >>>>> on Yarn anymore.
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>> 2. if it is standalone, how can it leverage
> Kafka's
> >> >> > metrics,
> >> >> > > > > logs,
> >> >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>> Thanks,
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>> Fang, Yan
> >> >> > > > > >> > >>>>> yanfang724@gmail.com
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> >> >> > > > > wangguoz@gmail.com
> >> >> > > > > >> >
> >> >> > > > > >> > >>>> wrote:
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>>> Read through the code example and it looks good
> to
> >> me.
> >> >> A
> >> >> > > few
> >> >> > > > > >> > >>>>>> thoughts regarding deployment:
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>> Today Samza deploys as executable runnable like:
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> >> >> > > > > >> > >>>> --config-path=file://...
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>> And this proposal advocate for deploying Samza
> more
> >> as
> >> >> > > > embedded
> >> >> > > > > >> > >>>>>> libraries in user application code (ignoring the
> >> >> > > terminology
> >> >> > > > > >> since
> >> >> > > > > >> > >>>>>> it is not the
> >> >> > > > > >> > >>>>> same
> >> >> > > > > >> > >>>>>> as the prototype code):
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>> StreamTask task = new MyStreamTask(configs);
> Thread
> >> >> > thread
> >> >> > > =
> >> >> > > > > new
> >> >> > > > > >> > >>>>>> Thread(task); thread.start();
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>> I think both of these deployment modes are
> important
> >> >> for
> >> >> > > > > >> different
> >> >> > > > > >> > >>>>>> types
> >> >> > > > > >> > >>>>> of
> >> >> > > > > >> > >>>>>> users. That said, I think making Samza purely
> >> >> standalone
> >> >> > is
> >> >> > > > > still
> >> >> > > > > >> > >>>>>> sufficient for either runnable or library modes.
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>> Guozhang
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> >> >> > > > jay@confluent.io>
> >> >> > > > > >> > wrote:
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>>> Looks like gmail mangled the code example, it
> was
> >> >> > supposed
> >> >> > > > to
> >> >> > > > > >> look
> >> >> > > > > >> > >>>>>>> like
> >> >> > > > > >> > >>>>>>> this:
> >> >> > > > > >> > >>>>>>>
> >> >> > > > > >> > >>>>>>> Properties props = new Properties();
> >> >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> "localhost:4242");
> >> >> > > > > >> StreamingConfig
> >> >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> >> >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> "test-topic-2");
> >> >> > > > > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> >> >> > > > > >> > >>>>>>> config.serialization(new StringSerializer(), new
> >> >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming container
> =
> >> new
> >> >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
> >> >> > > > > >> > >>>>>>>
> >> >> > > > > >> > >>>>>>> -Jay
> >> >> > > > > >> > >>>>>>>
> >> >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> >> >> > > > jay@confluent.io
> >> >> > > > > >
> >> >> > > > > >> > >>>> wrote:
> >> >> > > > > >> > >>>>>>>
> >> >> > > > > >> > >>>>>>>> Hey guys,
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> This came out of some conversations Chris and I
> >> were
> >> >> > > having
> >> >> > > > > >> > >>>>>>>> around
> >> >> > > > > >> > >>>>>>> whether
> >> >> > > > > >> > >>>>>>>> it would make sense to use Samza as a kind of
> data
> >> >> > > > ingestion
> >> >> > > > > >> > >>>>> framework
> >> >> > > > > >> > >>>>>>> for
> >> >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26
> "copycat").
> >> >> This
> >> >> > > > kind
> >> >> > > > > of
> >> >> > > > > >> > >>>>>> combined
> >> >> > > > > >> > >>>>>>>> with complaints around config and YARN and the
> >> >> > discussion
> >> >> > > > > >> around
> >> >> > > > > >> > >>>>>>>> how
> >> >> > > > > >> > >>>>> to
> >> >> > > > > >> > >>>>>>>> best do a standalone mode.
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> So the thought experiment was, given that Samza
> >> was
> >> >> > > > basically
> >> >> > > > > >> > >>>>>>>> already totally Kafka specific, what if you
> just
> >> >> > embraced
> >> >> > > > > that
> >> >> > > > > >> > >>>>>>>> and turned it
> >> >> > > > > >> > >>>>>> into
> >> >> > > > > >> > >>>>>>>> something less like a heavyweight framework and
> >> more
> >> >> > > like a
> >> >> > > > > >> > >>>>>>>> third
> >> >> > > > > >> > >>>>> Kafka
> >> >> > > > > >> > >>>>>>>> client--a kind of "producing consumer" with
> state
> >> >> > > > management
> >> >> > > > > >> > >>>>>> facilities.
> >> >> > > > > >> > >>>>>>>> Basically a library. Instead of a complex
> stream
> >> >> > > processing
> >> >> > > > > >> > >>>>>>>> framework
> >> >> > > > > >> > >>>>>>> this
> >> >> > > > > >> > >>>>>>>> would actually be a very simple thing, not much
> >> more
> >> >> > > > > >> complicated
> >> >> > > > > >> > >>>>>>>> to
> >> >> > > > > >> > >>>>> use
> >> >> > > > > >> > >>>>>>> or
> >> >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we
> >> >> thought
> >> >> > > > about
> >> >> > > > > >> it
> >> >> > > > > >> > >>>>>>>> a
> >> >> > > > > >> > >>>>> lot
> >> >> > > > > >> > >>>>>> of
> >> >> > > > > >> > >>>>>>>> what Samza (and the other stream processing
> >> systems
> >> >> > were
> >> >> > > > > doing)
> >> >> > > > > >> > >>>>> seemed
> >> >> > > > > >> > >>>>>>> like
> >> >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> Of course you need to ingest/output data to and
> >> from
> >> >> > the
> >> >> > > > > stream
> >> >> > > > > >> > >>>>>>>> processing. But when we actually looked into
> how
> >> that
> >> >> > > would
> >> >> > > > > >> > >>>>>>>> work,
> >> >> > > > > >> > >>>>> Samza
> >> >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion framework
> >> for a
> >> >> > > bunch
> >> >> > > > of
> >> >> > > > > >> > >>>>> reasons.
> >> >> > > > > >> > >>>>>> To
> >> >> > > > > >> > >>>>>>>> really do that right you need a pretty
> different
> >> >> > internal
> >> >> > > > > data
> >> >> > > > > >> > >>>>>>>> model
> >> >> > > > > >> > >>>>>> and
> >> >> > > > > >> > >>>>>>>> set of apis. So what if you split them and had
> an
> >> api
> >> >> > for
> >> >> > > > > Kafka
> >> >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a
> separate
> >> >> api
> >> >> > > for
> >> >> > > > > >> Kafka
> >> >> > > > > >> > >>>>>>>> transformation (Samza).
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> This would also allow really embracing the same
> >> >> > > terminology
> >> >> > > > > and
> >> >> > > > > >> > >>>>>>>> conventions. One complaint about the current
> >> state is
> >> >> > > that
> >> >> > > > > the
> >> >> > > > > >> > >>>>>>>> two
> >> >> > > > > >> > >>>>>>> systems
> >> >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like
> "stream"
> >> vs
> >> >> > > > "topic"
> >> >> > > > > >> and
> >> >> > > > > >> > >>>>>>> different
> >> >> > > > > >> > >>>>>>>> config and monitoring systems means you kind of
> >> have
> >> >> to
> >> >> > > > learn
> >> >> > > > > >> > >>>>>>>> Kafka's
> >> >> > > > > >> > >>>>>>> way,
> >> >> > > > > >> > >>>>>>>> then learn Samza's slightly different way, then
> >> kind
> >> >> of
> >> >> > > > > >> > >>>>>>>> understand
> >> >> > > > > >> > >>>>> how
> >> >> > > > > >> > >>>>>>> they
> >> >> > > > > >> > >>>>>>>> map to each other, which having walked a few
> >> people
> >> >> > > through
> >> >> > > > > >> this
> >> >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> Since I have been spending a lot of time on
> >> >> airplanes I
> >> >> > > > > hacked
> >> >> > > > > >> > >>>>>>>> up an ernest but still somewhat incomplete
> >> prototype
> >> >> of
> >> >> > > > what
> >> >> > > > > >> > >>>>>>>> this would
> >> >> > > > > >> > >>>>> look
> >> >> > > > > >> > >>>>>>>> like. This is just unceremoniously dumped into
> >> Kafka
> >> >> as
> >> >> > > it
> >> >> > > > > >> > >>>>>>>> required a
> >> >> > > > > >> > >>>>>> few
> >> >> > > > > >> > >>>>>>>> changes to the new consumer. Here is the code:
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> >
> >> >> > > > >
> >> >> >
> >> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >> >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> For the purpose of the prototype I just
> liberally
> >> >> > renamed
> >> >> > > > > >> > >>>>>>>> everything
> >> >> > > > > >> > >>>>> to
> >> >> > > > > >> > >>>>>>>> try to align it with Kafka with no regard for
> >> >> > > > compatibility.
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> To use this would be something like this:
> >> >> > > > > >> > >>>>>>>> Properties props = new Properties();
> >> >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> "localhost:4242");
> >> >> > > > > >> > >>>>>>>> StreamingConfig config = new
> >> >> > > > > >> > >>>>> StreamingConfig(props);
> >> >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> >> >> > > > > >> > >>>>>>>> "test-topic-2");
> >> >> > > > > >> config.processor(ExampleStreamProcessor.class);
> >> >> > > > > >> > >>>>>>> config.serialization(new
> >> >> > > > > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
> >> >> > > > KafkaStreaming
> >> >> > > > > >> > >>>>>> container =
> >> >> > > > > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> >> >> > > > > StreamProcessor
> >> >> > > > > >> > >>>>>>>> is basically StreamTask.
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> So rather than putting all the class names in a
> >> file
> >> >> > and
> >> >> > > > then
> >> >> > > > > >> > >>>>>>>> having
> >> >> > > > > >> > >>>>>> the
> >> >> > > > > >> > >>>>>>>> job assembled by reflection, you just
> instantiate
> >> the
> >> >> > > > > container
> >> >> > > > > >> > >>>>>>>> programmatically. Work is balanced over however
> >> many
> >> >> > > > > instances
> >> >> > > > > >> > >>>>>>>> of
> >> >> > > > > >> > >>>>> this
> >> >> > > > > >> > >>>>>>> are
> >> >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance dies,
> new
> >> >> tasks
> >> >> > > are
> >> >> > > > > >> added
> >> >> > > > > >> > >>>>>>>> to
> >> >> > > > > >> > >>>>> the
> >> >> > > > > >> > >>>>>>>> existing containers without shutting them
> down).
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> We would provide some glue for running this
> stuff
> >> in
> >> >> > YARN
> >> >> > > > via
> >> >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some
> of
> >> >> their
> >> >> > > > tools
> >> >> > > > > >> > >>>>>>>> but from the
> >> >> > > > > >> > >>>>>> point
> >> >> > > > > >> > >>>>>>> of
> >> >> > > > > >> > >>>>>>>> view of these frameworks these stream
> processing
> >> jobs
> >> >> > are
> >> >> > > > > just
> >> >> > > > > >> > >>>>>> stateless
> >> >> > > > > >> > >>>>>>>> services that can come and go and expand and
> >> contract
> >> >> > at
> >> >> > > > > will.
> >> >> > > > > >> > >>>>>>>> There
> >> >> > > > > >> > >>>>> is
> >> >> > > > > >> > >>>>>>> no
> >> >> > > > > >> > >>>>>>>> more custom scheduler.
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> Here are some relevant details:
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would
> get
> >> >> larger
> >> >> > > if
> >> >> > > > we
> >> >> > > > > >> > >>>>>>>>  productionized but not vastly larger. We
> really
> >> do
> >> >> > get a
> >> >> > > > ton
> >> >> > > > > >> > >>>>>>>> of
> >> >> > > > > >> > >>>>>>> leverage
> >> >> > > > > >> > >>>>>>>>  out of Kafka.
> >> >> > > > > >> > >>>>>>>>  2. Partition management is fully delegated to
> the
> >> >> new
> >> >> > > > > >> consumer.
> >> >> > > > > >> > >>>>> This
> >> >> > > > > >> > >>>>>>>>  is nice since now any partition management
> >> strategy
> >> >> > > > > available
> >> >> > > > > >> > >>>>>>>> to
> >> >> > > > > >> > >>>>>> Kafka
> >> >> > > > > >> > >>>>>>>>  consumer is also available to Samza (and vice
> >> versa)
> >> >> > and
> >> >> > > > > with
> >> >> > > > > >> > >>>>>>>> the
> >> >> > > > > >> > >>>>>>> exact
> >> >> > > > > >> > >>>>>>>>  same configs.
> >> >> > > > > >> > >>>>>>>>  3. It supports state as well as state reuse
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is thought
> >> >> provoking.
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> -Jay
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris
> Riccomini <
> >> >> > > > > >> > >>>>>> criccomini@apache.org>
> >> >> > > > > >> > >>>>>>>> wrote:
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Hey all,
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> I have had some discussions with Samza
> engineers
> >> at
> >> >> > > > LinkedIn
> >> >> > > > > >> > >>>>>>>>> and
> >> >> > > > > >> > >>>>>>> Confluent
> >> >> > > > > >> > >>>>>>>>> and we came up with a few observations and
> would
> >> >> like
> >> >> > to
> >> >> > > > > >> > >>>>>>>>> propose
> >> >> > > > > >> > >>>>> some
> >> >> > > > > >> > >>>>>>>>> changes.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> We've observed some things that I want to call
> >> out
> >> >> > about
> >> >> > > > > >> > >>>>>>>>> Samza's
> >> >> > > > > >> > >>>>>> design,
> >> >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment
> >> >> system.
> >> >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> >> >> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and
> >> Kafka's
> >> >> > > > consumer
> >> >> > > > > >> > >>>>>>>>> APIs
> >> >> > > > > >> > >>>>> are
> >> >> > > > > >> > >>>>>>>>> trying to solve a lot of the same problems.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> All three of these issues are related, but
> I'll
> >> >> > address
> >> >> > > > them
> >> >> > > > > >> in
> >> >> > > > > >> > >>>>> order.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Deployment
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic
> >> >> > > deployment
> >> >> > > > > >> > >>>>>>>>> scheduler
> >> >> > > > > >> > >>>>>> such
> >> >> > > > > >> > >>>>>>>>> as
> >> >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built
> Samza,
> >> we
> >> >> > bet
> >> >> > > > that
> >> >> > > > > >> > >>>>>>>>> there
> >> >> > > > > >> > >>>>>> would
> >> >> > > > > >> > >>>>>>>>> be
> >> >> > > > > >> > >>>>>>>>> one or two winners in this area, and we could
> >> >> support
> >> >> > > > them,
> >> >> > > > > >> and
> >> >> > > > > >> > >>>>>>>>> the
> >> >> > > > > >> > >>>>>> rest
> >> >> > > > > >> > >>>>>>>>> would go away. In reality, there are many
> >> >> variations.
> >> >> > > > > >> > >>>>>>>>> Furthermore,
> >> >> > > > > >> > >>>>>> many
> >> >> > > > > >> > >>>>>>>>> people still prefer to just start their
> >> processors
> >> >> > like
> >> >> > > > > normal
> >> >> > > > > >> > >>>>>>>>> Java processes, and use traditional deployment
> >> >> scripts
> >> >> > > > such
> >> >> > > > > as
> >> >> > > > > >> > >>>>>>>>> Fabric,
> >> >> > > > > >> > >>>>>> Chef,
> >> >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on
> >> users
> >> >> > makes
> >> >> > > > the
> >> >> > > > > >> > >>>>>>>>> Samza start-up process really painful for
> first
> >> time
> >> >> > > > users.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was also a
> >> bit
> >> >> of
> >> >> > a
> >> >> > > > > >> > >>>>>>>>> mis-fire
> >> >> > > > > >> > >>>>>> because
> >> >> > > > > >> > >>>>>>>>> of
> >> >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between the
> >> nature of
> >> >> > > batch
> >> >> > > > > >> jobs
> >> >> > > > > >> > >>>>>>>>> and
> >> >> > > > > >> > >>>>>>> stream
> >> >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made conscious
> >> effort
> >> >> to
> >> >> > > > favor
> >> >> > > > > >> > >>>>>>>>> the
> >> >> > > > > >> > >>>>>> Hadoop
> >> >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it
> worked
> >> >> and
> >> >> > > was
> >> >> > > > > well
> >> >> > > > > >> > >>>>>>> understood.
> >> >> > > > > >> > >>>>>>>>> One thing that we missed was that batch jobs
> >> have a
> >> >> > > > definite
> >> >> > > > > >> > >>>>>> beginning,
> >> >> > > > > >> > >>>>>>>>> and
> >> >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't
> (usually).
> >> >> This
> >> >> > > > leads
> >> >> > > > > to
> >> >> > > > > >> > >>>>>>>>> a
> >> >> > > > > >> > >>>>> much
> >> >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream
> processors.
> >> >> You
> >> >> > > > > >> basically
> >> >> > > > > >> > >>>>>>>>> just
> >> >> > > > > >> > >>>>>>> need
> >> >> > > > > >> > >>>>>>>>> to find a place to start the processor, and
> start
> >> >> it.
> >> >> > > The
> >> >> > > > > way
> >> >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept
> of
> >> a
> >> >> > > cluster
> >> >> > > > > >> > >>>>>>>>> being "full". We always
> >> >> > > > > >> > >>>>>> add
> >> >> > > > > >> > >>>>>>>>> more machines. The problem with coupling Samza
> >> with
> >> >> a
> >> >> > > > > >> scheduler
> >> >> > > > > >> > >>>>>>>>> is
> >> >> > > > > >> > >>>>>> that
> >> >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to handle
> >> deployment.
> >> >> > > This
> >> >> > > > > >> pulls
> >> >> > > > > >> > >>>>>>>>> in a
> >> >> > > > > >> > >>>>>>> bunch
> >> >> > > > > >> > >>>>>>>>> of things such as configuration distribution
> >> (config
> >> >> > > > > stream),
> >> >> > > > > >> > >>>>>>>>> shell
> >> >> > > > > >> > >>>>>>> scrips
> >> >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all
> the
> >> .tgz
> >> >> > > > stuff),
> >> >> > > > > >> etc.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
> deployment
> >> was
> >> >> to
> >> >> > > > > support
> >> >> > > > > >> > >>>>>>>>> data locality. If you want to have locality,
> you
> >> >> need
> >> >> > to
> >> >> > > > put
> >> >> > > > > >> > >>>>>>>>> your
> >> >> > > > > >> > >>>>>> processors
> >> >> > > > > >> > >>>>>>>>> close to the data they're processing. Upon
> >> further
> >> >> > > > > >> > >>>>>>>>> investigation,
> >> >> > > > > >> > >>>>>>> though,
> >> >> > > > > >> > >>>>>>>>> this feature is not that beneficial. There is
> >> some
> >> >> > good
> >> >> > > > > >> > >>>>>>>>> discussion
> >> >> > > > > >> > >>>>>> about
> >> >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we
> >> took
> >> >> the
> >> >> > > > > >> > >>>>>>>>> Map/Reduce
> >> >> > > > > >> > >>>>>> path,
> >> >> > > > > >> > >>>>>>>>> but
> >> >> > > > > >> > >>>>>>>>> there are some fundamental differences between
> >> HDFS
> >> >> > and
> >> >> > > > > Kafka.
> >> >> > > > > >> > >>>>>>>>> HDFS
> >> >> > > > > >> > >>>>>> has
> >> >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions. This
> leads to
> >> >> less
> >> >> > > > > >> > >>>>>>>>> optimization potential with stream processors
> on
> >> top
> >> >> > of
> >> >> > > > > Kafka.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> This feature is also used as a crutch. Samza
> >> doesn't
> >> >> > > have
> >> >> > > > > any
> >> >> > > > > >> > >>>>>>>>> built
> >> >> > > > > >> > >>>>> in
> >> >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on
> the
> >> >> > > dynamic
> >> >> > > > > >> > >>>>>>>>> deployment scheduling system to handle
> restarts
> >> >> when a
> >> >> > > > > >> > >>>>>>>>> processor dies. This has
> >> >> > > > > >> > >>>>>>> made
> >> >> > > > > >> > >>>>>>>>> it very difficult to write a standalone Samza
> >> >> > container
> >> >> > > > > >> > >>>> (SAMZA-516).
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Pluggability
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> In some cases pluggability is good, but I
> think
> >> that
> >> >> > > we've
> >> >> > > > > >> gone
> >> >> > > > > >> > >>>>>>>>> too
> >> >> > > > > >> > >>>>>> far
> >> >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> * Pluggable config.
> >> >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> >> >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> >> >> > > > > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> >> >> > > > > SystemProducer,
> >> >> > > > > >> > >>>> etc).
> >> >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> >> >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> >> >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about every
> >> >> component
> >> >> > > > > >> > >>>>> (MessageChooser,
> >> >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter,
> >> etc).
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> There's probably more that I've forgotten, as
> >> well.
> >> >> > Some
> >> >> > > > of
> >> >> > > > > >> > >>>>>>>>> these
> >> >> > > > > >> > >>>>> are
> >> >> > > > > >> > >>>>>>>>> useful, but some have proven not to be. This
> all
> >> >> comes
> >> >> > > at
> >> >> > > > a
> >> >> > > > > >> cost:
> >> >> > > > > >> > >>>>>>>>> complexity. This complexity is making it
> harder
> >> for
> >> >> > our
> >> >> > > > > users
> >> >> > > > > >> > >>>>>>>>> to
> >> >> > > > > >> > >>>>> pick
> >> >> > > > > >> > >>>>>> up
> >> >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also makes it
> >> >> > difficult
> >> >> > > > for
> >> >> > > > > >> > >>>>>>>>> Samza developers to reason about what the
> >> >> > > characteristics
> >> >> > > > of
> >> >> > > > > >> > >>>>>>>>> the container (since the characteristics
> change
> >> >> > > depending
> >> >> > > > on
> >> >> > > > > >> > >>>>>>>>> which plugins are use).
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> The issues with pluggability are most visible
> in
> >> the
> >> >> > > > System
> >> >> > > > > >> APIs.
> >> >> > > > > >> > >>>>> What
> >> >> > > > > >> > >>>>>>>>> Samza really requires to be functional is
> Kafka
> >> as
> >> >> its
> >> >> > > > > >> > >>>>>>>>> transport
> >> >> > > > > >> > >>>>>> layer.
> >> >> > > > > >> > >>>>>>>>> But
> >> >> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases into
> one
> >> >> API:
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> >> >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> The current System API supports both of these
> use
> >> >> > cases.
> >> >> > > > The
> >> >> > > > > >> > >>>>>>>>> problem
> >> >> > > > > >> > >>>>>> is,
> >> >> > > > > >> > >>>>>>>>> we
> >> >> > > > > >> > >>>>>>>>> actually want different features for each use
> >> case.
> >> >> By
> >> >> > > > > >> papering
> >> >> > > > > >> > >>>>>>>>> over
> >> >> > > > > >> > >>>>>>> these
> >> >> > > > > >> > >>>>>>>>> two use cases, and providing a single API,
> we've
> >> >> > > > introduced
> >> >> > > > > a
> >> >> > > > > >> > >>>>>>>>> ton of
> >> >> > > > > >> > >>>>>>> leaky
> >> >> > > > > >> > >>>>>>>>> abstractions.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> For example, what we'd really like in (2) is
> to
> >> have
> >> >> > > > > >> > >>>>>>>>> monotonically increasing longs for offsets
> (like
> >> >> > Kafka).
> >> >> > > > > This
> >> >> > > > > >> > >>>>>>>>> would be at odds
> >> >> > > > > >> > >>>>> with
> >> >> > > > > >> > >>>>>>> (1),
> >> >> > > > > >> > >>>>>>>>> though, since different systems have different
> >> >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> >> >> > > > > >> > >>>>>>>>> There was discussion both on the mailing list
> and
> >> >> the
> >> >> > > SQL
> >> >> > > > > >> JIRAs
> >> >> > > > > >> > >>>>> about
> >> >> > > > > >> > >>>>>>> the
> >> >> > > > > >> > >>>>>>>>> need for this.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> The same thing holds true for replayability.
> >> Kafka
> >> >> > > allows
> >> >> > > > us
> >> >> > > > > >> to
> >> >> > > > > >> > >>>>> rewind
> >> >> > > > > >> > >>>>>>>>> when
> >> >> > > > > >> > >>>>>>>>> we have a failure. Many other systems don't.
> In
> >> some
> >> >> > > > cases,
> >> >> > > > > >> > >>>>>>>>> systems
> >> >> > > > > >> > >>>>>>> return
> >> >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> >> >> WikipediaSystemConsumer)
> >> >> > > > > because
> >> >> > > > > >> > >>>>>>>>> they
> >> >> > > > > >> > >>>>>> have
> >> >> > > > > >> > >>>>>>> no
> >> >> > > > > >> > >>>>>>>>> offsets.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka
> supports
> >> >> > > > > partitioning,
> >> >> > > > > >> > >>>>>>>>> but
> >> >> > > > > >> > >>>>> many
> >> >> > > > > >> > >>>>>>>>> systems don't. We model this by having a
> single
> >> >> > > partition
> >> >> > > > > for
> >> >> > > > > >> > >>>>>>>>> those systems. Still, other systems model
> >> >> partitioning
> >> >> > > > > >> > >>>> differently (e.g.
> >> >> > > > > >> > >>>>>>>>> Kinesis).
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a mess.
> >> Creating
> >> >> > > streams
> >> >> > > > > in
> >> >> > > > > >> a
> >> >> > > > > >> > >>>>>>>>> system-agnostic way is almost impossible. As
> is
> >> >> > modeling
> >> >> > > > > >> > >>>>>>>>> metadata
> >> >> > > > > >> > >>>>> for
> >> >> > > > > >> > >>>>>>> the
> >> >> > > > > >> > >>>>>>>>> system (replication factor, partitions,
> location,
> >> >> > etc).
> >> >> > > > The
> >> >> > > > > >> > >>>>>>>>> list
> >> >> > > > > >> > >>>>> goes
> >> >> > > > > >> > >>>>>>> on.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Duplicate work
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> At the time that we began writing Samza,
> Kafka's
> >> >> > > consumer
> >> >> > > > > and
> >> >> > > > > >> > >>>>> producer
> >> >> > > > > >> > >>>>>>>>> APIs
> >> >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On the
> >> >> > consumer-side,
> >> >> > > > you
> >> >> > > > > >> > >>>>>>>>> had two
> >> >> > > > > >> > >>>>>>>>> options: use the high level consumer, or the
> >> simple
> >> >> > > > > consumer.
> >> >> > > > > >> > >>>>>>>>> The
> >> >> > > > > >> > >>>>>>> problem
> >> >> > > > > >> > >>>>>>>>> with the high-level consumer was that it
> >> controlled
> >> >> > your
> >> >> > > > > >> > >>>>>>>>> offsets, partition assignments, and the order
> in
> >> >> which
> >> >> > > you
> >> >> > > > > >> > >>>>>>>>> received messages. The
> >> >> > > > > >> > >>>>> problem
> >> >> > > > > >> > >>>>>>>>> with
> >> >> > > > > >> > >>>>>>>>> the simple consumer is that it's not simple.
> It's
> >> >> > basic.
> >> >> > > > You
> >> >> > > > > >> > >>>>>>>>> end up
> >> >> > > > > >> > >>>>>>> having
> >> >> > > > > >> > >>>>>>>>> to handle a lot of really low-level stuff that
> >> you
> >> >> > > > > shouldn't.
> >> >> > > > > >> > >>>>>>>>> We
> >> >> > > > > >> > >>>>>> spent a
> >> >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> KafkaSystemConsumer
> >> very
> >> >> > > > robust.
> >> >> > > > > >> It
> >> >> > > > > >> > >>>>>>>>> also allows us to support some cool features:
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
> >> prioritization.
> >> >> > > > > >> > >>>>>>>>> * Tight control over partition assignment to
> >> support
> >> >> > > > joins,
> >> >> > > > > >> > >>>>>>>>> global
> >> >> > > > > >> > >>>>>> state
> >> >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> >> >> > > > > >> > >>>>>>>>> * Tight control over offset checkpointing.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> What we didn't realize at the time is that
> these
> >> >> > > features
> >> >> > > > > >> > >>>>>>>>> should
> >> >> > > > > >> > >>>>>>> actually
> >> >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not
> just
> >> >> Samza
> >> >> > > > stream
> >> >> > > > > >> > >>>>>> processors)
> >> >> > > > > >> > >>>>>>>>> end up wanting to do things like joins and
> >> partition
> >> >> > > > > >> > >>>>>>>>> assignment. The
> >> >> > > > > >> > >>>>>>> Kafka
> >> >> > > > > >> > >>>>>>>>> community has come to the same conclusion.
> >> They're
> >> >> > > adding
> >> >> > > > a
> >> >> > > > > >> ton
> >> >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka consumer
> >> >> > > implementation.
> >> >> > > > > To a
> >> >> > > > > >> > >>>>>>>>> large extent,
> >> >> > > > > >> > >>>>> it's
> >> >> > > > > >> > >>>>>>>>> duplicate work to what we've already done in
> >> Samza.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a very
> >> similar
> >> >> > > > > approach
> >> >> > > > > >> > >>>>>>>>> to
> >> >> > > > > >> > >>>>>> Samza's
> >> >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation for
> >> handling
> >> >> > > offset
> >> >> > > > > >> > >>>>>> checkpointing.
> >> >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset management
> feature
> >> >> > stores
> >> >> > > > > >> offset
> >> >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you to
> fetch
> >> them
> >> >> > > from
> >> >> > > > > the
> >> >> > > > > >> > >>>>>>>>> broker.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since we
> could
> >> >> have
> >> >> > > > shared
> >> >> > > > > >> > >>>>>>>>> the
> >> >> > > > > >> > >>>>> work
> >> >> > > > > >> > >>>>>> if
> >> >> > > > > >> > >>>>>>>>> it
> >> >> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Vision
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> All of this leads me to a rather radical
> >> proposal.
> >> >> > Samza
> >> >> > > > is
> >> >> > > > > >> > >>>>> relatively
> >> >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to say that
> >> we're
> >> >> > > near a
> >> >> > > > > 1.0
> >> >> > > > > >> > >>>>>> release.
> >> >> > > > > >> > >>>>>>>>> I'd
> >> >> > > > > >> > >>>>>>>>> like to propose that we take what we've
> learned,
> >> and
> >> >> > > begin
> >> >> > > > > >> > >>>>>>>>> thinking
> >> >> > > > > >> > >>>>>>> about
> >> >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we
> were
> >> >> > > starting
> >> >> > > > > >> from
> >> >> > > > > >> > >>>>>> scratch?
> >> >> > > > > >> > >>>>>>>>> My
> >> >> > > > > >> > >>>>>>>>> proposal is to:
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run
> >> Samza
> >> >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
> dependences
> >> on
> >> >> > > YARN,
> >> >> > > > > >> Mesos,
> >> >> > > > > >> > >>>> etc.
> >> >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support only
> Kafka
> >> as
> >> >> the
> >> >> > > > > stream
> >> >> > > > > >> > >>>>>> processing
> >> >> > > > > >> > >>>>>>>>> layer.
> >> >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
> >> >> serialization,
> >> >> > > and
> >> >> > > > > >> > >>>>>>>>> config
> >> >> > > > > >> > >>>>>>> systems,
> >> >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> This would fix all of the issues that I
> outlined
> >> >> > above.
> >> >> > > It
> >> >> > > > > >> > >>>>>>>>> should
> >> >> > > > > >> > >>>>> also
> >> >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> dramatically.
> >> >> > > Supporting
> >> >> > > > > >> only
> >> >> > > > > >> > >>>>>>>>> a standalone container will allow Samza to be
> >> >> executed
> >> >> > > on
> >> >> > > > > YARN
> >> >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> Marathon/Aurora), or
> >> >> most
> >> >> > > > other
> >> >> > > > > >> > >>>>>>>>> in-house
> >> >> > > > > >> > >>>>>>> deployment
> >> >> > > > > >> > >>>>>>>>> systems. This should make life a lot easier
> for
> >> new
> >> >> > > users.
> >> >> > > > > >> > >>>>>>>>> Imagine
> >> >> > > > > >> > >>>>>>> having
> >> >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN. The
> drop
> >> in
> >> >> > > mailing
> >> >> > > > > >> list
> >> >> > > > > >> > >>>>>> traffic
> >> >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me.
> The
> >> >> > > reality
> >> >> > > > > is,
> >> >> > > > > >> > >>>>> everyone
> >> >> > > > > >> > >>>>>>>>> that
> >> >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We
> >> basically
> >> >> > > > require
> >> >> > > > > >> it
> >> >> > > > > >> > >>>>>> already
> >> >> > > > > >> > >>>>>>> in
> >> >> > > > > >> > >>>>>>>>> order for most features to work. Those that
> are
> >> >> using
> >> >> > > > other
> >> >> > > > > >> > >>>>>>>>> systems
> >> >> > > > > >> > >>>>>> are
> >> >> > > > > >> > >>>>>>>>> generally using it for ingest into Kafka (1),
> and
> >> >> then
> >> >> > > > they
> >> >> > > > > do
> >> >> > > > > >> > >>>>>>>>> the processing on top. There is already
> >> discussion (
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> >
> >> >> > > > >
> >> >> >
> >> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >> >> > > > > >> > >>>>> 767
> >> >> > > > > >> > >>>>>>>>> )
> >> >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka
> extremely
> >> >> easy.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Once we make the call to couple with Kafka, we
> >> can
> >> >> > > > leverage
> >> >> > > > > a
> >> >> > > > > >> > >>>>>>>>> ton of
> >> >> > > > > >> > >>>>>>> their
> >> >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to maintain our
> own
> >> >> > config,
> >> >> > > > > >> > >>>>>>>>> metrics,
> >> >> > > > > >> > >>>>> etc.
> >> >> > > > > >> > >>>>>>> We
> >> >> > > > > >> > >>>>>>>>> can all share the same libraries, and make
> them
> >> >> > better.
> >> >> > > > This
> >> >> > > > > >> > >>>>>>>>> will
> >> >> > > > > >> > >>>>> also
> >> >> > > > > >> > >>>>>>>>> allow us to share the consumer/producer APIs,
> and
> >> >> will
> >> >> > > let
> >> >> > > > > us
> >> >> > > > > >> > >>>>> leverage
> >> >> > > > > >> > >>>>>>>>> their offset management and partition
> management,
> >> >> > rather
> >> >> > > > > than
> >> >> > > > > >> > >>>>>>>>> having
> >> >> > > > > >> > >>>>>> our
> >> >> > > > > >> > >>>>>>>>> own. All of the coordinator stream code would
> go
> >> >> away,
> >> >> > > as
> >> >> > > > > >> would
> >> >> > > > > >> > >>>>>>>>> most
> >> >> > > > > >> > >>>>>> of
> >> >> > > > > >> > >>>>>>>>> the
> >> >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to
> push
> >> some
> >> >> > > > > partition
> >> >> > > > > >> > >>>>>>> management
> >> >> > > > > >> > >>>>>>>>> features into the Kafka broker, but they're
> >> already
> >> >> > > moving
> >> >> > > > > in
> >> >> > > > > >> > >>>>>>>>> that direction with the new consumer API. The
> >> >> features
> >> >> > > we
> >> >> > > > > have
> >> >> > > > > >> > >>>>>>>>> for
> >> >> > > > > >> > >>>>>> partition
> >> >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and seem
> like
> >> >> they
> >> >> > > > should
> >> >> > > > > >> be
> >> >> > > > > >> > >>>>>>>>> in
> >> >> > > > > >> > >>>>>> Kafka
> >> >> > > > > >> > >>>>>>>>> anyway. There will always be some niche usages
> >> which
> >> >> > > will
> >> >> > > > > >> > >>>>>>>>> require
> >> >> > > > > >> > >>>>>> extra
> >> >> > > > > >> > >>>>>>>>> care and hence full control over partition
> >> >> assignments
> >> >> > > > much
> >> >> > > > > >> > >>>>>>>>> like the
> >> >> > > > > >> > >>>>>>> Kafka
> >> >> > > > > >> > >>>>>>>>> low level consumer api. These would continue
> to
> >> be
> >> >> > > > > supported.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> These items will be good for the Samza
> community.
> >> >> > > They'll
> >> >> > > > > make
> >> >> > > > > >> > >>>>>>>>> Samza easier to use, and make it easier for
> >> >> developers
> >> >> > > to
> >> >> > > > > add
> >> >> > > > > >> > >>>>>>>>> new features.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat
> >> >> > backwards
> >> >> > > > > >> > >>>>> incompatible
> >> >> > > > > >> > >>>>>>>>> change). If we choose to go this route, it's
> >> >> important
> >> >> > > > that
> >> >> > > > > we
> >> >> > > > > >> > >>>>> openly
> >> >> > > > > >> > >>>>>>>>> communicate how we're going to provide a
> >> migration
> >> >> > path
> >> >> > > > from
> >> >> > > > > >> > >>>>>>>>> the
> >> >> > > > > >> > >>>>>>> existing
> >> >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make incompatible
> >> >> > changes).
> >> >> > > I
> >> >> > > > > >> think
> >> >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to provide a
> >> >> wrapper
> >> >> > to
> >> >> > > > > allow
> >> >> > > > > >> > >>>>>>>>> existing StreamTask implementations to
> continue
> >> >> > running
> >> >> > > on
> >> >> > > > > the
> >> >> > > > > >> > >>>> new container.
> >> >> > > > > >> > >>>>>>> It's
> >> >> > > > > >> > >>>>>>>>> also important that we openly communicate
> about
> >> >> > timing,
> >> >> > > > and
> >> >> > > > > >> > >>>>>>>>> stages
> >> >> > > > > >> > >>>>> of
> >> >> > > > > >> > >>>>>>> the
> >> >> > > > > >> > >>>>>>>>> migration.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you have
> >> opinions.
> >> >> > :)
> >> >> > > > > Please
> >> >> > > > > >> > >>>>>>>>> send
> >> >> > > > > >> > >>>>>> your
> >> >> > > > > >> > >>>>>>>>> thoughts and feedback.
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>> Cheers,
> >> >> > > > > >> > >>>>>>>>> Chris
> >> >> > > > > >> > >>>>>>>>>
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>>
> >> >> > > > > >> > >>>>>>>
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>> --
> >> >> > > > > >> > >>>>>> -- Guozhang
> >> >> > > > > >> > >>>>>>
> >> >> > > > > >> > >>>>>
> >> >> > > > > >> > >>>>
> >> >> > > > > >> > >>
> >> >> > > > > >> > >>
> >> >> > > > > >> >
> >> >> > > > > >> >
> >> >> > > > > >> >
> >> >> > > > > >>
> >> >> > > > > >
> >> >> > > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >> >
> >> >>
> >>
>

Re: Thoughts and obesrvations on Samza

Posted by Julian Hyde <jh...@apache.org>.
I broadly support it, with one big proviso.

One of the attractive things about Kafka has been its minimalism --
the fact that it solves one part of the problem, simply, and very
well. It is very important that it continues to do that, and that
people continue to perceive it that way. Make Kafka into a stack, if
you must, but make sure that the architectural layers remain clear.

Making perfectly good software components into a muddled stack is one
of the pitfalls of success, and in particular a pitfall of being a
software company with a sales team looking for a bigger pieces to sell
and customers looking for "simpler" large components to install. Avoid
at all costs!


On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io> wrote:
> Yeah I agree with this summary. I think there are kind of two questions
> here:
> 1. Technically does alignment/reliance on Kafka make sense
> 2. Branding wise (naming, website, concepts, etc) does alignment with Kafka
> make sense
>
> Personally I do think both of these things would be really valuable, and
> would dramatically alter the trajectory of the project.
>
> My preference would be to see if people can mostly agree on a direction
> rather than splintering things off. From my point of view the ideal outcome
> of all the options discussed would be to make Samza a closely aligned
> subproject, maintained in a separate repository and retaining the existing
> committership but sharing as much else as possible (website, etc). No idea
> about how these things work, Jacob, you probably know more.
>
> No discussion amongst the Kafka folks has happened on this, but likely we
> should figure out what the Samza community actually wants first.
>
> I admit that this is a fairly radical departure from how things are.
>
> If that doesn't fly, I think, yeah we could leave Samza as it is and do the
> more radical reboot inside Kafka. From my point of view that does leave
> things in a somewhat confusing state since now there are two stream
> processing systems more or less coupled to Kafka in large part made by the
> same people. But, arguably that might be a cleaner way to make the cut-over
> and perhaps less risky for Samza community since if it works people can
> switch and if it doesn't nothing will have changed. Dunno, how do people
> feel about this?
>
> -Jay
>
> On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com> wrote:
>
>> >  This leads me to thinking that merging projects and communities might
>> be a good idea: with the union of experience from both communities, we will
>> probably build a better system that is better for users.
>> Is this what's being proposed though? Merging the projects seems like
>> a consequence of at most one of the three directions under discussion:
>> 1) Samza 2.0: The Samza community relies more heavily on Kafka for
>> configuration, etc. (to a greater or lesser extent to be determined)
>> but the Samza community would not automatically merge withe Kafka
>> community (the Phoenix/HBase example is a good one here).
>> 2) Samza Reboot: The Samza community continues to exist with a limited
>> project scope, but similarly would not need to be part of the Kafka
>> community (ie given committership) to progress.  Here, maybe the Samza
>> team would become a subproject of Kafka (the Board frowns on
>> subprojects at the moment, so I'm not sure if that's even feasible),
>> but that would not be required.
>> 3) Hey Samza! FYI, Kafka does streaming now: In this option the Kafka
>> team builds its own streaming library, possibly off of Jay's
>> prototype, which has not direct lineage to the Samza team.  There's no
>> reason for the Kafka team to bring in the Samza team.
>>
>> Is the Kafka community on board with this?
>>
>> To be clear, all three options under discussion are interesting,
>> technically valid and likely healthy directions for the project.
>> Also, they are not mutually exclusive.  The Samza community could
>> decide to pursue, say, 'Samza 2.0', while the Kafka community went
>> forward with 'Hey Samza!'  My points above are directed entirely at
>> the community aspect of these choices.
>> -Jakob
>>
>> On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com> wrote:
>> > That's great.  Thanks, Jay.
>> >
>> > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io> wrote:
>> >
>> >> Yeah totally agree. I think you have this issue even today, right? I.e.
>> if
>> >> you need to make a simple config change and you're running in YARN today
>> >> you end up bouncing the job which then rebuilds state. I think the fix
>> is
>> >> exactly what you described which is to have a long timeout on partition
>> >> movement for stateful jobs so that if a job is just getting bounced, and
>> >> the cluster manager (or admin) is smart enough to restart it on the same
>> >> host when possible, it can optimistically reuse any existing state it
>> finds
>> >> on disk (if it is valid).
>> >>
>> >> So in this model the charter of the CM is to place processes as
>> stickily as
>> >> possible and to restart or re-place failed processes. The charter of the
>> >> partition management system is to control the assignment of work to
>> these
>> >> processes. The nice thing about this is that the work assignment,
>> timeouts,
>> >> behavior, configs, and code will all be the same across all cluster
>> >> managers.
>> >>
>> >> So I think that prototype would actually give you exactly what you want
>> >> today for any cluster manager (or manual placement + restart script)
>> that
>> >> was sticky in terms of host placement since there is already a
>> configurable
>> >> partition movement timeout and task-by-task state reuse with a check on
>> >> state validity.
>> >>
>> >> -Jay
>> >>
>> >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <ro...@gmail.com>
>> >> wrote:
>> >>
>> >> > That would be great to let Kafka do as much heavy lifting as possible
>> and
>> >> > make it easier for other languages to implement Samza apis.
>> >> >
>> >> > One thing to watch out for is the interplay between Kafka's group
>> >> > management and the external scheduler/process manager's fault
>> tolerance.
>> >> > If a container dies, the Kafka group membership protocol will try to
>> >> assign
>> >> > it's tasks to other containers while at the same time the process
>> manager
>> >> > is trying to relaunch the container.  Without some consideration for
>> this
>> >> > (like a configurable amount of time to wait before Kafka alters the
>> group
>> >> > membership), there may be thrashing going on which is especially bad
>> for
>> >> > containers with large amounts of local state.
>> >> >
>> >> > Someone else pointed this out already but I thought it might be worth
>> >> > calling out again.
>> >> >
>> >> > Cheers,
>> >> >
>> >> > Roger
>> >> >
>> >> >
>> >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <ja...@confluent.io> wrote:
>> >> >
>> >> > > Hey Roger,
>> >> > >
>> >> > > I couldn't agree more. We spent a bunch of time talking to people
>> and
>> >> > that
>> >> > > is exactly the stuff we heard time and again. What makes it hard, of
>> >> > > course, is that there is some tension between compatibility with
>> what's
>> >> > > there now and making things better for new users.
>> >> > >
>> >> > > I also strongly agree with the importance of multi-language
>> support. We
>> >> > are
>> >> > > talking now about Java, but for application development use cases
>> >> people
>> >> > > want to work in whatever language they are using elsewhere. I think
>> >> > moving
>> >> > > to a model where Kafka itself does the group membership, lifecycle
>> >> > control,
>> >> > > and partition assignment has the advantage of putting all that
>> complex
>> >> > > stuff behind a clean api that the clients are already going to be
>> >> > > implementing for their consumer, so the added functionality for
>> stream
>> >> > > processing beyond a consumer becomes very minor.
>> >> > >
>> >> > > -Jay
>> >> > >
>> >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
>> roger.hoover@gmail.com>
>> >> > > wrote:
>> >> > >
>> >> > > > Metamorphosis...nice. :)
>> >> > > >
>> >> > > > This has been a great discussion.  As a user of Samza who's
>> recently
>> >> > > > integrated it into a relatively large organization, I just want to
>> >> add
>> >> > > > support to a few points already made.
>> >> > > >
>> >> > > > The biggest hurdles to adoption of Samza as it currently exists
>> that
>> >> > I've
>> >> > > > experienced are:
>> >> > > > 1) YARN - YARN is overly complex in many environments where Puppet
>> >> > would
>> >> > > do
>> >> > > > just fine but it was the only mechanism to get fault tolerance.
>> >> > > > 2) Configuration - I think I like the idea of configuring most of
>> the
>> >> > job
>> >> > > > in code rather than config files.  In general, I think the goal
>> >> should
>> >> > be
>> >> > > > to make it harder to make mistakes, especially of the kind where
>> the
>> >> > code
>> >> > > > expects something and the config doesn't match.  The current
>> config
>> >> is
>> >> > > > quite intricate and error-prone.  For example, the application
>> logic
>> >> > may
>> >> > > > depend on bootstrapping a topic but rather than asserting that in
>> the
>> >> > > code,
>> >> > > > you have to rely on getting the config right.  Likewise with
>> serdes,
>> >> > the
>> >> > > > Java representations produced by various serdes (JSON, Avro, etc.)
>> >> are
>> >> > > not
>> >> > > > equivalent so you cannot just reconfigure a serde without changing
>> >> the
>> >> > > > code.   It would be nice for jobs to be able to assert what they
>> >> expect
>> >> > > > from their input topics in terms of partitioning.  This is
>> getting a
>> >> > > little
>> >> > > > off topic but I was even thinking about creating a "Samza config
>> >> > linter"
>> >> > > > that would sanity check a set of configs.  Especially in
>> >> organizations
>> >> > > > where config is managed by a different team than the application
>> >> > > developer,
>> >> > > > it's very hard to get avoid config mistakes.
>> >> > > > 3) Java/Scala centric - for many teams (especially DevOps-type
>> >> folks),
>> >> > > the
>> >> > > > pain of the Java toolchain (maven, slow builds, weak command line
>> >> > > support,
>> >> > > > configuration over convention) really inhibits productivity.  As
>> more
>> >> > and
>> >> > > > more high-quality clients become available for Kafka, I hope
>> they'll
>> >> > > follow
>> >> > > > Samza's model.  Not sure how much it affects the proposals in this
>> >> > thread
>> >> > > > but please consider other languages in the ecosystem as well.
>> From
>> >> > what
>> >> > > > I've heard, Spark has more Python users than Java/Scala.
>> >> > > > (FYI, we added a Jython wrapper for the Samza API
>> >> > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
>> >> > > > and are working on a Yeoman generator
>> >> > > > https://github.com/Quantiply/generator-rico for Jython/Samza
>> >> projects
>> >> > to
>> >> > > > alleviate some of the pain)
>> >> > > >
>> >> > > > I also want to underscore Jay's point about improving the user
>> >> > > experience.
>> >> > > > That's a very important factor for adoption.  I think the goal
>> should
>> >> > be
>> >> > > to
>> >> > > > make Samza as easy to get started with as something like Logstash.
>> >> > > > Logstash is vastly inferior in terms of capabilities to Samza but
>> >> it's
>> >> > > easy
>> >> > > > to get started and that makes a big difference.
>> >> > > >
>> >> > > > Cheers,
>> >> > > >
>> >> > > > Roger
>> >> > > >
>> >> > > >
>> >> > > >
>> >> > > >
>> >> > > >
>> >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
>> >> > > > gdfm@apache.org> wrote:
>> >> > > >
>> >> > > > > Forgot to add. On the naming issues, Kafka Metamorphosis is a
>> clear
>> >> > > > winner
>> >> > > > > :)
>> >> > > > >
>> >> > > > > --
>> >> > > > > Gianmarco
>> >> > > > >
>> >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
>> >> > > gdfm@apache.org
>> >> > > > >
>> >> > > > > wrote:
>> >> > > > >
>> >> > > > > > Hi,
>> >> > > > > >
>> >> > > > > > @Martin, thanks for you comments.
>> >> > > > > > Maybe I'm missing some important point, but I think coupling
>> the
>> >> > > > releases
>> >> > > > > > is actually a *good* thing.
>> >> > > > > > To make an example, would it be better if the MR and HDFS
>> >> > components
>> >> > > of
>> >> > > > > > Hadoop had different release schedules?
>> >> > > > > >
>> >> > > > > > Actually, keeping the discussion in a single place would make
>> >> > > agreeing
>> >> > > > on
>> >> > > > > > releases (and backwards compatibility) much easier, as
>> everybody
>> >> > > would
>> >> > > > be
>> >> > > > > > responsible for the whole codebase.
>> >> > > > > >
>> >> > > > > > That said, I like the idea of absorbing samza-core as a
>> >> > sub-project,
>> >> > > > and
>> >> > > > > > leave the fancy stuff separate.
>> >> > > > > > It probably gives 90% of the benefits we have been discussing
>> >> here.
>> >> > > > > >
>> >> > > > > > Cheers,
>> >> > > > > >
>> >> > > > > > --
>> >> > > > > > Gianmarco
>> >> > > > > >
>> >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com>
>> wrote:
>> >> > > > > >
>> >> > > > > >> Hey Martin,
>> >> > > > > >>
>> >> > > > > >> I agree coupling release schedules is a downside.
>> >> > > > > >>
>> >> > > > > >> Definitely we can try to solve some of the integration
>> problems
>> >> in
>> >> > > > > >> Confluent Platform or in other distributions. But I think
>> this
>> >> > ends
>> >> > > up
>> >> > > > > >> being really shallow. I guess I feel to really get a good
>> user
>> >> > > > > experience
>> >> > > > > >> the two systems have to kind of feel like part of the same
>> thing
>> >> > and
>> >> > > > you
>> >> > > > > >> can't really add that in later--you can put both in the same
>> >> > > > > downloadable
>> >> > > > > >> tar file but it doesn't really give a very cohesive feeling.
>> I
>> >> > agree
>> >> > > > > that
>> >> > > > > >> ultimately any of the project stuff is as much social and
>> naming
>> >> > as
>> >> > > > > >> anything else--theoretically two totally independent projects
>> >> > could
>> >> > > > work
>> >> > > > > >> to
>> >> > > > > >> tightly align. In practice this seems to be quite difficult
>> >> > though.
>> >> > > > > >>
>> >> > > > > >> For the frameworks--totally agree it would be good to
>> maintain
>> >> the
>> >> > > > > >> framework support with the project. In some cases there may
>> not
>> >> be
>> >> > > too
>> >> > > > > >> much
>> >> > > > > >> there since the integration gets lighter but I think whatever
>> >> > stubs
>> >> > > > you
>> >> > > > > >> need should be included. So no I definitely wasn't trying to
>> >> imply
>> >> > > > > >> dropping
>> >> > > > > >> support for these frameworks, just making the integration
>> >> lighter
>> >> > by
>> >> > > > > >> separating process management from partition management.
>> >> > > > > >>
>> >> > > > > >> You raise two good points we would have to figure out if we
>> went
>> >> > > down
>> >> > > > > the
>> >> > > > > >> alignment path:
>> >> > > > > >> 1. With respect to the name, yeah I think the first question
>> is
>> >> > > > whether
>> >> > > > > >> some "re-branding" would be worth it. If so then I think we
>> can
>> >> > > have a
>> >> > > > > big
>> >> > > > > >> thread on the name. I'm definitely not set on Kafka
>> Streaming or
>> >> > > Kafka
>> >> > > > > >> Streams I was just using them to be kind of illustrative. I
>> >> agree
>> >> > > with
>> >> > > > > >> your
>> >> > > > > >> critique of these names, though I think people would get the
>> >> idea.
>> >> > > > > >> 2. Yeah you also raise a good point about how to "factor" it.
>> >> Here
>> >> > > are
>> >> > > > > the
>> >> > > > > >> options I see (I could get enthusiastic about any of them):
>> >> > > > > >>    a. One repo for both Kafka and Samza
>> >> > > > > >>    b. Two repos, retaining the current seperation
>> >> > > > > >>    c. Two repos, the equivalent of samza-api and samza-core
>> is
>> >> > > > absorbed
>> >> > > > > >> almost like a third client
>> >> > > > > >>
>> >> > > > > >> Cheers,
>> >> > > > > >>
>> >> > > > > >> -Jay
>> >> > > > > >>
>> >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
>> >> > > > martin@kleppmann.com>
>> >> > > > > >> wrote:
>> >> > > > > >>
>> >> > > > > >> > Ok, thanks for the clarifications. Just a few follow-up
>> >> > comments.
>> >> > > > > >> >
>> >> > > > > >> > - I see the appeal of merging with Kafka or becoming a
>> >> > subproject:
>> >> > > > the
>> >> > > > > >> > reasons you mention are good. The risk I see is that
>> release
>> >> > > > schedules
>> >> > > > > >> > become coupled to each other, which can slow everyone down,
>> >> and
>> >> > > > large
>> >> > > > > >> > projects with many contributors are harder to manage.
>> (Jakob,
>> >> > can
>> >> > > > you
>> >> > > > > >> speak
>> >> > > > > >> > from experience, having seen a wider range of Hadoop
>> ecosystem
>> >> > > > > >> projects?)
>> >> > > > > >> >
>> >> > > > > >> > Some of the goals of a better unified developer experience
>> >> could
>> >> > > > also
>> >> > > > > be
>> >> > > > > >> > solved by integrating Samza nicely into a Kafka
>> distribution
>> >> > (such
>> >> > > > as
>> >> > > > > >> > Confluent's). I'm not against merging projects if we decide
>> >> > that's
>> >> > > > the
>> >> > > > > >> way
>> >> > > > > >> > to go, just pointing out the same goals can perhaps also be
>> >> > > achieved
>> >> > > > > in
>> >> > > > > >> > other ways.
>> >> > > > > >> >
>> >> > > > > >> > - With regard to dropping the YARN dependency: are you
>> >> proposing
>> >> > > > that
>> >> > > > > >> > Samza doesn't give any help to people wanting to run on
>> >> > > > > >> YARN/Mesos/AWS/etc?
>> >> > > > > >> > So the docs would basically have a link to Slider and
>> nothing
>> >> > > else?
>> >> > > > Or
>> >> > > > > >> > would we maintain integrations with a bunch of popular
>> >> > deployment
>> >> > > > > >> methods
>> >> > > > > >> > (e.g. the necessary glue and shell scripts to make Samza
>> work
>> >> > with
>> >> > > > > >> Slider)?
>> >> > > > > >> >
>> >> > > > > >> > I absolutely think it's a good idea to have the "as a
>> library"
>> >> > and
>> >> > > > > "as a
>> >> > > > > >> > process" (using Yi's taxonomy) options for people who want
>> >> them,
>> >> > > > but I
>> >> > > > > >> > think there should also be a low-friction path for common
>> "as
>> >> a
>> >> > > > > service"
>> >> > > > > >> > deployment methods, for which we probably need to maintain
>> >> > > > > integrations.
>> >> > > > > >> >
>> >> > > > > >> > - Project naming: "Kafka Streams" seems odd to me, because
>> >> Kafka
>> >> > > is
>> >> > > > > all
>> >> > > > > >> > about streams already. Perhaps "Kafka Transformers" or
>> "Kafka
>> >> > > > Filters"
>> >> > > > > >> > would be more apt?
>> >> > > > > >> >
>> >> > > > > >> > One suggestion: perhaps the core of Samza (stream
>> >> transformation
>> >> > > > with
>> >> > > > > >> > state management -- i.e. the "Samza as a library" bit)
>> could
>> >> > > become
>> >> > > > > >> part of
>> >> > > > > >> > Kafka, while higher-level tools such as streaming SQL and
>> >> > > > integrations
>> >> > > > > >> with
>> >> > > > > >> > deployment frameworks remain in a separate project? In
>> other
>> >> > > words,
>> >> > > > > >> Kafka
>> >> > > > > >> > would absorb the proven, stable core of Samza, which would
>> >> > become
>> >> > > > the
>> >> > > > > >> > "third Kafka client" mentioned early in this thread. The
>> Samza
>> >> > > > project
>> >> > > > > >> > would then target that third Kafka client as its base API,
>> and
>> >> > the
>> >> > > > > >> project
>> >> > > > > >> > would be freed up to explore more experimental new
>> horizons.
>> >> > > > > >> >
>> >> > > > > >> > Martin
>> >> > > > > >> >
>> >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com>
>> >> wrote:
>> >> > > > > >> >
>> >> > > > > >> > > Hey Martin,
>> >> > > > > >> > >
>> >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually don't think
>> it
>> >> > ties
>> >> > > > our
>> >> > > > > >> > hands
>> >> > > > > >> > > at all, all it does is refactor things. The division of
>> >> > > > > >> responsibility is
>> >> > > > > >> > > that Samza core is responsible for task lifecycle, state,
>> >> and
>> >> > > > > >> partition
>> >> > > > > >> > > management (using the Kafka co-ordinator) but it is NOT
>> >> > > > responsible
>> >> > > > > >> for
>> >> > > > > >> > > packaging, configuration deployment or execution of
>> >> processes.
>> >> > > The
>> >> > > > > >> > problem
>> >> > > > > >> > > of packaging and starting these processes is
>> >> > > > > >> > > framework/environment-specific. This leaves individual
>> >> > > frameworks
>> >> > > > to
>> >> > > > > >> be
>> >> > > > > >> > as
>> >> > > > > >> > > fancy or vanilla as they like. So you can get simple
>> >> stateless
>> >> > > > > >> support in
>> >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app framework
>> >> > > (Slider,
>> >> > > > > >> > Marathon,
>> >> > > > > >> > > etc). These are well known by people and have nice UIs
>> and a
>> >> > lot
>> >> > > > of
>> >> > > > > >> > > flexibility. I don't think they have node affinity as a
>> >> built
>> >> > in
>> >> > > > > >> option
>> >> > > > > >> > > (though I could be wrong). So if we want that we can
>> either
>> >> > wait
>> >> > > > for
>> >> > > > > >> them
>> >> > > > > >> > > to add it or do a custom framework to add that feature
>> (as
>> >> > now).
>> >> > > > > >> > Obviously
>> >> > > > > >> > > if you manage things with old-school ops tools
>> >> > (puppet/chef/etc)
>> >> > > > you
>> >> > > > > >> get
>> >> > > > > >> > > locality easily. The nice thing, though, is that all the
>> >> samza
>> >> > > > > >> "business
>> >> > > > > >> > > logic" around partition management and fault tolerance
>> is in
>> >> > > Samza
>> >> > > > > >> core
>> >> > > > > >> > so
>> >> > > > > >> > > it is shared across frameworks and the framework specific
>> >> bit
>> >> > is
>> >> > > > > just
>> >> > > > > >> > > whether it is smart enough to try to get the same host
>> when
>> >> a
>> >> > > job
>> >> > > > is
>> >> > > > > >> > > restarted.
>> >> > > > > >> > >
>> >> > > > > >> > > With respect to the Kafka-alignment, yeah I think the
>> goal
>> >> > would
>> >> > > > be
>> >> > > > > >> (a)
>> >> > > > > >> > > actually get better alignment in user experience, and (b)
>> >> > > express
>> >> > > > > >> this in
>> >> > > > > >> > > the naming and project branding. Specifically:
>> >> > > > > >> > > 1. Website/docs, it would be nice for the
>> "transformation"
>> >> api
>> >> > > to
>> >> > > > be
>> >> > > > > >> > > discoverable in the main Kafka docs--i.e. be able to
>> explain
>> >> > > when
>> >> > > > to
>> >> > > > > >> use
>> >> > > > > >> > > the consumer and when to use the stream processing
>> >> > functionality
>> >> > > > and
>> >> > > > > >> lead
>> >> > > > > >> > > people into that experience.
>> >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or
>> whatever)
>> >> > that
>> >> > > > has
>> >> > > > > >> both
>> >> > > > > >> > > Kafka and the stream processing part and they actually
>> work
>> >> > > > > together.
>> >> > > > > >> > > 3. Unify the programming experience so the client and
>> Samza
>> >> > api
>> >> > > > > share
>> >> > > > > >> > > config/monitoring/naming/packaging/etc.
>> >> > > > > >> > >
>> >> > > > > >> > > I think sub-projects keep separate committers and can
>> have a
>> >> > > > > separate
>> >> > > > > >> > repo,
>> >> > > > > >> > > but I'm actually not really sure (I can't find a
>> definition
>> >> > of a
>> >> > > > > >> > subproject
>> >> > > > > >> > > in Apache).
>> >> > > > > >> > >
>> >> > > > > >> > > Basically at a high-level you want the experience to
>> "feel"
>> >> > > like a
>> >> > > > > >> single
>> >> > > > > >> > > system, not to relatively independent things that are
>> kind
>> >> of
>> >> > > > > >> awkwardly
>> >> > > > > >> > > glued together.
>> >> > > > > >> > >
>> >> > > > > >> > > I think if we did that they having naming or branding
>> like
>> >> > > "kafka
>> >> > > > > >> > > streaming" or "kafka streams" or something like that
>> would
>> >> > > > actually
>> >> > > > > >> do a
>> >> > > > > >> > > good job of conveying what it is. I do that this would
>> help
>> >> > > > adoption
>> >> > > > > >> > quite
>> >> > > > > >> > > a lot as it would correctly convey that using Kafka
>> >> Streaming
>> >> > > with
>> >> > > > > >> Kafka
>> >> > > > > >> > is
>> >> > > > > >> > > a fairly seamless experience and Kafka is pretty heavily
>> >> > adopted
>> >> > > > at
>> >> > > > > >> this
>> >> > > > > >> > > point.
>> >> > > > > >> > >
>> >> > > > > >> > > Fwiw we actually considered this model originally when
>> open
>> >> > > > sourcing
>> >> > > > > >> > Samza,
>> >> > > > > >> > > however at that time Kafka was relatively unknown and we
>> >> > decided
>> >> > > > not
>> >> > > > > >> to
>> >> > > > > >> > do
>> >> > > > > >> > > it since we felt it would be limiting. From my point of
>> view
>> >> > the
>> >> > > > > three
>> >> > > > > >> > > things have changed (1) Kafka is now really heavily used
>> for
>> >> > > > stream
>> >> > > > > >> > > processing, (2) we learned that abstracting out the
>> stream
>> >> > well
>> >> > > is
>> >> > > > > >> > > basically impossible, (3) we learned it is really hard to
>> >> keep
>> >> > > the
>> >> > > > > two
>> >> > > > > >> > > things feeling like a single product.
>> >> > > > > >> > >
>> >> > > > > >> > > -Jay
>> >> > > > > >> > >
>> >> > > > > >> > >
>> >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
>> >> > > > > >> martin@kleppmann.com>
>> >> > > > > >> > > wrote:
>> >> > > > > >> > >
>> >> > > > > >> > >> Hi all,
>> >> > > > > >> > >>
>> >> > > > > >> > >> Lots of good thoughts here.
>> >> > > > > >> > >>
>> >> > > > > >> > >> I agree with the general philosophy of tying Samza more
>> >> > firmly
>> >> > > to
>> >> > > > > >> Kafka.
>> >> > > > > >> > >> After I spent a while looking at integrating other
>> message
>> >> > > > brokers
>> >> > > > > >> (e.g.
>> >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the conclusion
>> that
>> >> > > > > >> > SystemConsumer
>> >> > > > > >> > >> tacitly assumes a model so much like Kafka's that pretty
>> >> much
>> >> > > > > nobody
>> >> > > > > >> but
>> >> > > > > >> > >> Kafka actually implements it. (Databus is perhaps an
>> >> > exception,
>> >> > > > but
>> >> > > > > >> it
>> >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus, making
>> Samza
>> >> > > fully
>> >> > > > > >> > dependent
>> >> > > > > >> > >> on Kafka acknowledges that the system-independence was
>> >> never
>> >> > as
>> >> > > > > real
>> >> > > > > >> as
>> >> > > > > >> > we
>> >> > > > > >> > >> perhaps made it out to be. The gains of code reuse are
>> >> real.
>> >> > > > > >> > >>
>> >> > > > > >> > >> The idea of decoupling Samza from YARN has also always
>> been
>> >> > > > > >> appealing to
>> >> > > > > >> > >> me, for various reasons already mentioned in this
>> thread.
>> >> > > > Although
>> >> > > > > >> > making
>> >> > > > > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc)
>> >> seems
>> >> > > > > >> laudable,
>> >> > > > > >> > I am
>> >> > > > > >> > >> a little concerned that it will restrict us to a lowest
>> >> > common
>> >> > > > > >> > denominator.
>> >> > > > > >> > >> For example, would host affinity (SAMZA-617) still be
>> >> > possible?
>> >> > > > For
>> >> > > > > >> jobs
>> >> > > > > >> > >> with large amounts of state, I think SAMZA-617 would be
>> a
>> >> big
>> >> > > > boon,
>> >> > > > > >> > since
>> >> > > > > >> > >> restoring state off the changelog on every single
>> restart
>> >> is
>> >> > > > > painful,
>> >> > > > > >> > due
>> >> > > > > >> > >> to long recovery times. It would be a shame if the
>> >> decoupling
>> >> > > > from
>> >> > > > > >> YARN
>> >> > > > > >> > >> made host affinity impossible.
>> >> > > > > >> > >>
>> >> > > > > >> > >> Jay, a question about the proposed API for
>> instantiating a
>> >> > job
>> >> > > in
>> >> > > > > >> code
>> >> > > > > >> > >> (rather than a properties file): when submitting a job
>> to a
>> >> > > > > cluster,
>> >> > > > > >> is
>> >> > > > > >> > the
>> >> > > > > >> > >> idea that the instantiation code runs on a client
>> >> somewhere,
>> >> > > > which
>> >> > > > > >> then
>> >> > > > > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or
>> >> does
>> >> > > that
>> >> > > > > >> code
>> >> > > > > >> > run
>> >> > > > > >> > >> on each container that is part of the job (in which
>> case,
>> >> how
>> >> > > > does
>> >> > > > > >> the
>> >> > > > > >> > job
>> >> > > > > >> > >> submission to the cluster work)?
>> >> > > > > >> > >>
>> >> > > > > >> > >> I agree with Garry that it doesn't feel right to make a
>> 1.0
>> >> > > > release
>> >> > > > > >> > with a
>> >> > > > > >> > >> plan for it to be immediately obsolete. So if this is
>> going
>> >> > to
>> >> > > > > >> happen, I
>> >> > > > > >> > >> think it would be more honest to stick with 0.* version
>> >> > numbers
>> >> > > > > until
>> >> > > > > >> > the
>> >> > > > > >> > >> library-ified Samza has been implemented, is stable and
>> >> > widely
>> >> > > > > used.
>> >> > > > > >> > >>
>> >> > > > > >> > >> Should the new Samza be a subproject of Kafka? There is
>> >> > > precedent
>> >> > > > > for
>> >> > > > > >> > >> tight coupling between different Apache projects (e.g.
>> >> > Curator
>> >> > > > and
>> >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think remaining
>> >> separate
>> >> > > > would
>> >> > > > > >> be
>> >> > > > > >> > ok.
>> >> > > > > >> > >> Even if Samza is fully dependent on Kafka, there is
>> enough
>> >> > > > > substance
>> >> > > > > >> in
>> >> > > > > >> > >> Samza that it warrants being a separate project. An
>> >> argument
>> >> > in
>> >> > > > > >> favour
>> >> > > > > >> > of
>> >> > > > > >> > >> merging would be if we think Kafka has a much stronger
>> >> "brand
>> >> > > > > >> presence"
>> >> > > > > >> > >> than Samza; I'm ambivalent on that one. If the Kafka
>> >> project
>> >> > is
>> >> > > > > >> willing
>> >> > > > > >> > to
>> >> > > > > >> > >> endorse Samza as the "official" way of doing stateful
>> >> stream
>> >> > > > > >> > >> transformations, that would probably have much the same
>> >> > effect
>> >> > > as
>> >> > > > > >> > >> re-branding Samza as "Kafka Stream Processors" or
>> suchlike.
>> >> > > Close
>> >> > > > > >> > >> collaboration between the two projects will be needed in
>> >> any
>> >> > > > case.
>> >> > > > > >> > >>
>> >> > > > > >> > >> From a project management perspective, I guess the "new
>> >> > Samza"
>> >> > > > > would
>> >> > > > > >> > have
>> >> > > > > >> > >> to be developed on a branch alongside ongoing
>> maintenance
>> >> of
>> >> > > the
>> >> > > > > >> current
>> >> > > > > >> > >> line of development? I think it would be important to
>> >> > continue
>> >> > > > > >> > supporting
>> >> > > > > >> > >> existing users, and provide a graceful migration path to
>> >> the
>> >> > > new
>> >> > > > > >> > version.
>> >> > > > > >> > >> Leaving the current versions unsupported and forcing
>> people
>> >> > to
>> >> > > > > >> rewrite
>> >> > > > > >> > >> their jobs would send a bad signal.
>> >> > > > > >> > >>
>> >> > > > > >> > >> Best,
>> >> > > > > >> > >> Martin
>> >> > > > > >> > >>
>> >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io>
>> >> wrote:
>> >> > > > > >> > >>
>> >> > > > > >> > >>> Hey Garry,
>> >> > > > > >> > >>>
>> >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy to chat
>> more
>> >> > about
>> >> > > > > this
>> >> > > > > >> if
>> >> > > > > >> > >>> you'd be interested. I think Chris and I started with
>> the
>> >> > idea
>> >> > > > of
>> >> > > > > >> "what
>> >> > > > > >> > >>> would it take to make Samza a kick-ass ingestion tool"
>> but
>> >> > > > > >> ultimately
>> >> > > > > >> > we
>> >> > > > > >> > >>> kind of came around to the idea that ingestion and
>> >> > > > transformation
>> >> > > > > >> had
>> >> > > > > >> > >>> pretty different needs and coupling the two made things
>> >> > hard.
>> >> > > > > >> > >>>
>> >> > > > > >> > >>> For what it's worth I think copycat (KIP-26) actually
>> will
>> >> > do
>> >> > > > what
>> >> > > > > >> you
>> >> > > > > >> > >> are
>> >> > > > > >> > >>> looking for.
>> >> > > > > >> > >>>
>> >> > > > > >> > >>> With regard to your point about slider, I don't
>> >> necessarily
>> >> > > > > >> disagree.
>> >> > > > > >> > >> But I
>> >> > > > > >> > >>> think getting good YARN support is quite doable and I
>> >> think
>> >> > we
>> >> > > > can
>> >> > > > > >> make
>> >> > > > > >> > >>> that work well. I think the issue this proposal solves
>> is
>> >> > that
>> >> > > > > >> > >> technically
>> >> > > > > >> > >>> it is pretty hard to support multiple cluster
>> management
>> >> > > systems
>> >> > > > > the
>> >> > > > > >> > way
>> >> > > > > >> > >>> things are now, you need to write an "app master" or
>> >> > > "framework"
>> >> > > > > for
>> >> > > > > >> > each
>> >> > > > > >> > >>> and they are all a little different so testing is
>> really
>> >> > hard.
>> >> > > > In
>> >> > > > > >> the
>> >> > > > > >> > >>> absence of this we have been stuck with just YARN which
>> >> has
>> >> > > > > >> fantastic
>> >> > > > > >> > >>> penetration in the Hadoopy part of the org, but zero
>> >> > > penetration
>> >> > > > > >> > >> elsewhere.
>> >> > > > > >> > >>> Given the huge amount of work being put in to slider,
>> >> > > marathon,
>> >> > > > > aws
>> >> > > > > >> > >>> tooling, not to mention the umpteen related packaging
>> >> > > > technologies
>> >> > > > > >> > people
>> >> > > > > >> > >>> want to use (Docker, Kubernetes, various cloud-specific
>> >> > deploy
>> >> > > > > >> tools,
>> >> > > > > >> > >> etc)
>> >> > > > > >> > >>> I really think it is important to get this right.
>> >> > > > > >> > >>>
>> >> > > > > >> > >>> -Jay
>> >> > > > > >> > >>>
>> >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>> >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
>> >> > > > > >> > >>>
>> >> > > > > >> > >>>> Hi all,
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>> I think the question below re does Samza become a
>> >> > sub-project
>> >> > > > of
>> >> > > > > >> Kafka
>> >> > > > > >> > >>>> highlights the broader point around migration. Chris
>> >> > mentions
>> >> > > > > >> Samza's
>> >> > > > > >> > >>>> maturity is heading towards a v1 release but I'm not
>> sure
>> >> > it
>> >> > > > > feels
>> >> > > > > >> > >> right to
>> >> > > > > >> > >>>> launch a v1 then immediately plan to deprecate most of
>> >> it.
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>> From a selfish perspective I have some guys who have
>> >> > started
>> >> > > > > >> working
>> >> > > > > >> > >> with
>> >> > > > > >> > >>>> Samza and building some new consumers/producers was
>> next
>> >> > up.
>> >> > > > > Sounds
>> >> > > > > >> > like
>> >> > > > > >> > >>>> that is absolutely not the direction to go. I need to
>> >> look
>> >> > > into
>> >> > > > > the
>> >> > > > > >> > KIP
>> >> > > > > >> > >> in
>> >> > > > > >> > >>>> more detail but for me the attractiveness of adding
>> new
>> >> > Samza
>> >> > > > > >> > >>>> consumer/producers -- even if yes all they were doing
>> was
>> >> > > > really
>> >> > > > > >> > getting
>> >> > > > > >> > >>>> data into and out of Kafka --  was to avoid  having to
>> >> > worry
>> >> > > > > about
>> >> > > > > >> the
>> >> > > > > >> > >>>> lifecycle management of external clients. If there is
>> a
>> >> > > generic
>> >> > > > > >> Kafka
>> >> > > > > >> > >>>> ingress/egress layer that I can plug a new connector
>> into
>> >> > and
>> >> > > > > have
>> >> > > > > >> a
>> >> > > > > >> > >> lot of
>> >> > > > > >> > >>>> the heavy lifting re scale and reliability done for me
>> >> then
>> >> > > it
>> >> > > > > >> gives
>> >> > > > > >> > me
>> >> > > > > >> > >> all
>> >> > > > > >> > >>>> the pushing new consumers/producers would. If not
>> then it
>> >> > > > > >> complicates
>> >> > > > > >> > my
>> >> > > > > >> > >>>> operational deployments.
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>> Which is similar to my other question with the
>> proposal
>> >> --
>> >> > if
>> >> > > > we
>> >> > > > > >> > build a
>> >> > > > > >> > >>>> fully available/stand-alone Samza plus the requisite
>> >> shims
>> >> > to
>> >> > > > > >> > integrate
>> >> > > > > >> > >>>> with Slider etc I suspect the former may be a lot more
>> >> work
>> >> > > > than
>> >> > > > > we
>> >> > > > > >> > >> think.
>> >> > > > > >> > >>>> We may make it much easier for a newcomer to get
>> >> something
>> >> > > > > running
>> >> > > > > >> but
>> >> > > > > >> > >>>> having them step up and get a reliable production
>> >> > deployment
>> >> > > > may
>> >> > > > > >> still
>> >> > > > > >> > >>>> dominate mailing list  traffic, if for different
>> reasons
>> >> > than
>> >> > > > > >> today.
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with making the
>> >> Samza
>> >> > > > > >> dependency
>> >> > > > > >> > >> on
>> >> > > > > >> > >>>> Kafka much more explicit and I absolutely see the
>> >> benefits
>> >> > > in
>> >> > > > > the
>> >> > > > > >> > >>>> reduction of duplication and clashing
>> >> > > > terminologies/abstractions
>> >> > > > > >> that
>> >> > > > > >> > >>>> Chris/Jay describe. Samza as a library would likely
>> be a
>> >> > very
>> >> > > > > nice
>> >> > > > > >> > tool
>> >> > > > > >> > >> to
>> >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the concerns
>> >> above
>> >> > re
>> >> > > > the
>> >> > > > > >> > >>>> operational side.
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>> Garry
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>> -----Original Message-----
>> >> > > > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
>> >> > gdfm@apache.org
>> >> > > ]
>> >> > > > > >> > >>>> Sent: 02 July 2015 12:56
>> >> > > > > >> > >>>> To: dev@samza.apache.org
>> >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>> Very interesting thoughts.
>> >> > > > > >> > >>>> From outside, I have always perceived Samza as a
>> >> computing
>> >> > > > layer
>> >> > > > > >> over
>> >> > > > > >> > >>>> Kafka.
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>> The question, maybe a bit provocative, is "should
>> Samza
>> >> be
>> >> > a
>> >> > > > > >> > sub-project
>> >> > > > > >> > >>>> of Kafka then?"
>> >> > > > > >> > >>>> Or does it make sense to keep it as a separate project
>> >> > with a
>> >> > > > > >> separate
>> >> > > > > >> > >>>> governance?
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>> Cheers,
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>> --
>> >> > > > > >> > >>>> Gianmarco
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
>> yanfang724@gmail.com>
>> >> > > > wrote:
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more tightly.
>> >> > Because
>> >> > > > > Samza
>> >> > > > > >> de
>> >> > > > > >> > >>>>> facto is based on Kafka, and it should leverage what
>> >> Kafka
>> >> > > > has.
>> >> > > > > At
>> >> > > > > >> > the
>> >> > > > > >> > >>>>> same time, Kafka does not need to reinvent what Samza
>> >> > > already
>> >> > > > > >> has. I
>> >> > > > > >> > >>>>> also like the idea of separating the ingestion and
>> >> > > > > transformation.
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>> But it is a little difficult for me to image how the
>> >> Samza
>> >> > > > will
>> >> > > > > >> look
>> >> > > > > >> > >>>> like.
>> >> > > > > >> > >>>>> And I feel Chris and Jay have a little difference in
>> >> terms
>> >> > > of
>> >> > > > > how
>> >> > > > > >> > >>>>> Samza should look like.
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>> *** Will it look like what Jay's code shows (A
>> client of
>> >> > > > Kakfa)
>> >> > > > > ?
>> >> > > > > >> And
>> >> > > > > >> > >>>>> user's application code calls this client?
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka (like what
>> the
>> >> > > code
>> >> > > > > >> shows),
>> >> > > > > >> > >>>>> how do we implement auto-balance and fault-tolerance?
>> >> Are
>> >> > > they
>> >> > > > > >> taken
>> >> > > > > >> > >>>>> care by the Kafka broker or other mechanism, such as
>> >> > "Samza
>> >> > > > > >> worker"
>> >> > > > > >> > >>>>> (just make up the name) ?
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>> 2. What about other features, such as auto-scaling,
>> >> shared
>> >> > > > > state,
>> >> > > > > >> > >>>>> monitoring?
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>> *** If we have Samza standalone, (is this what Chris
>> >> > > > suggests?)
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa and
>> produce
>> >> to
>> >> > > it.
>> >> > > > > >> Then it
>> >> > > > > >> > >>>>> becomes the same as what Samza looks like now,
>> except it
>> >> > > does
>> >> > > > > not
>> >> > > > > >> > rely
>> >> > > > > >> > >>>>> on Yarn anymore.
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's
>> >> > metrics,
>> >> > > > > logs,
>> >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>> Thanks,
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>> Fang, Yan
>> >> > > > > >> > >>>>> yanfang724@gmail.com
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
>> >> > > > > wangguoz@gmail.com
>> >> > > > > >> >
>> >> > > > > >> > >>>> wrote:
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>>> Read through the code example and it looks good to
>> me.
>> >> A
>> >> > > few
>> >> > > > > >> > >>>>>> thoughts regarding deployment:
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>> Today Samza deploys as executable runnable like:
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
>> >> > > > > >> > >>>> --config-path=file://...
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>> And this proposal advocate for deploying Samza more
>> as
>> >> > > > embedded
>> >> > > > > >> > >>>>>> libraries in user application code (ignoring the
>> >> > > terminology
>> >> > > > > >> since
>> >> > > > > >> > >>>>>> it is not the
>> >> > > > > >> > >>>>> same
>> >> > > > > >> > >>>>>> as the prototype code):
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread
>> >> > thread
>> >> > > =
>> >> > > > > new
>> >> > > > > >> > >>>>>> Thread(task); thread.start();
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>> I think both of these deployment modes are important
>> >> for
>> >> > > > > >> different
>> >> > > > > >> > >>>>>> types
>> >> > > > > >> > >>>>> of
>> >> > > > > >> > >>>>>> users. That said, I think making Samza purely
>> >> standalone
>> >> > is
>> >> > > > > still
>> >> > > > > >> > >>>>>> sufficient for either runnable or library modes.
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>> Guozhang
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
>> >> > > > jay@confluent.io>
>> >> > > > > >> > wrote:
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>>> Looks like gmail mangled the code example, it was
>> >> > supposed
>> >> > > > to
>> >> > > > > >> look
>> >> > > > > >> > >>>>>>> like
>> >> > > > > >> > >>>>>>> this:
>> >> > > > > >> > >>>>>>>
>> >> > > > > >> > >>>>>>> Properties props = new Properties();
>> >> > > > > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
>> >> > > > > >> StreamingConfig
>> >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
>> >> > > > > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
>> >> > > > > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
>> >> > > > > >> > >>>>>>> config.serialization(new StringSerializer(), new
>> >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming container =
>> new
>> >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
>> >> > > > > >> > >>>>>>>
>> >> > > > > >> > >>>>>>> -Jay
>> >> > > > > >> > >>>>>>>
>> >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
>> >> > > > jay@confluent.io
>> >> > > > > >
>> >> > > > > >> > >>>> wrote:
>> >> > > > > >> > >>>>>>>
>> >> > > > > >> > >>>>>>>> Hey guys,
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> This came out of some conversations Chris and I
>> were
>> >> > > having
>> >> > > > > >> > >>>>>>>> around
>> >> > > > > >> > >>>>>>> whether
>> >> > > > > >> > >>>>>>>> it would make sense to use Samza as a kind of data
>> >> > > > ingestion
>> >> > > > > >> > >>>>> framework
>> >> > > > > >> > >>>>>>> for
>> >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat").
>> >> This
>> >> > > > kind
>> >> > > > > of
>> >> > > > > >> > >>>>>> combined
>> >> > > > > >> > >>>>>>>> with complaints around config and YARN and the
>> >> > discussion
>> >> > > > > >> around
>> >> > > > > >> > >>>>>>>> how
>> >> > > > > >> > >>>>> to
>> >> > > > > >> > >>>>>>>> best do a standalone mode.
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> So the thought experiment was, given that Samza
>> was
>> >> > > > basically
>> >> > > > > >> > >>>>>>>> already totally Kafka specific, what if you just
>> >> > embraced
>> >> > > > > that
>> >> > > > > >> > >>>>>>>> and turned it
>> >> > > > > >> > >>>>>> into
>> >> > > > > >> > >>>>>>>> something less like a heavyweight framework and
>> more
>> >> > > like a
>> >> > > > > >> > >>>>>>>> third
>> >> > > > > >> > >>>>> Kafka
>> >> > > > > >> > >>>>>>>> client--a kind of "producing consumer" with state
>> >> > > > management
>> >> > > > > >> > >>>>>> facilities.
>> >> > > > > >> > >>>>>>>> Basically a library. Instead of a complex stream
>> >> > > processing
>> >> > > > > >> > >>>>>>>> framework
>> >> > > > > >> > >>>>>>> this
>> >> > > > > >> > >>>>>>>> would actually be a very simple thing, not much
>> more
>> >> > > > > >> complicated
>> >> > > > > >> > >>>>>>>> to
>> >> > > > > >> > >>>>> use
>> >> > > > > >> > >>>>>>> or
>> >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we
>> >> thought
>> >> > > > about
>> >> > > > > >> it
>> >> > > > > >> > >>>>>>>> a
>> >> > > > > >> > >>>>> lot
>> >> > > > > >> > >>>>>> of
>> >> > > > > >> > >>>>>>>> what Samza (and the other stream processing
>> systems
>> >> > were
>> >> > > > > doing)
>> >> > > > > >> > >>>>> seemed
>> >> > > > > >> > >>>>>>> like
>> >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> Of course you need to ingest/output data to and
>> from
>> >> > the
>> >> > > > > stream
>> >> > > > > >> > >>>>>>>> processing. But when we actually looked into how
>> that
>> >> > > would
>> >> > > > > >> > >>>>>>>> work,
>> >> > > > > >> > >>>>> Samza
>> >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion framework
>> for a
>> >> > > bunch
>> >> > > > of
>> >> > > > > >> > >>>>> reasons.
>> >> > > > > >> > >>>>>> To
>> >> > > > > >> > >>>>>>>> really do that right you need a pretty different
>> >> > internal
>> >> > > > > data
>> >> > > > > >> > >>>>>>>> model
>> >> > > > > >> > >>>>>> and
>> >> > > > > >> > >>>>>>>> set of apis. So what if you split them and had an
>> api
>> >> > for
>> >> > > > > Kafka
>> >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate
>> >> api
>> >> > > for
>> >> > > > > >> Kafka
>> >> > > > > >> > >>>>>>>> transformation (Samza).
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> This would also allow really embracing the same
>> >> > > terminology
>> >> > > > > and
>> >> > > > > >> > >>>>>>>> conventions. One complaint about the current
>> state is
>> >> > > that
>> >> > > > > the
>> >> > > > > >> > >>>>>>>> two
>> >> > > > > >> > >>>>>>> systems
>> >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream"
>> vs
>> >> > > > "topic"
>> >> > > > > >> and
>> >> > > > > >> > >>>>>>> different
>> >> > > > > >> > >>>>>>>> config and monitoring systems means you kind of
>> have
>> >> to
>> >> > > > learn
>> >> > > > > >> > >>>>>>>> Kafka's
>> >> > > > > >> > >>>>>>> way,
>> >> > > > > >> > >>>>>>>> then learn Samza's slightly different way, then
>> kind
>> >> of
>> >> > > > > >> > >>>>>>>> understand
>> >> > > > > >> > >>>>> how
>> >> > > > > >> > >>>>>>> they
>> >> > > > > >> > >>>>>>>> map to each other, which having walked a few
>> people
>> >> > > through
>> >> > > > > >> this
>> >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> Since I have been spending a lot of time on
>> >> airplanes I
>> >> > > > > hacked
>> >> > > > > >> > >>>>>>>> up an ernest but still somewhat incomplete
>> prototype
>> >> of
>> >> > > > what
>> >> > > > > >> > >>>>>>>> this would
>> >> > > > > >> > >>>>> look
>> >> > > > > >> > >>>>>>>> like. This is just unceremoniously dumped into
>> Kafka
>> >> as
>> >> > > it
>> >> > > > > >> > >>>>>>>> required a
>> >> > > > > >> > >>>>>> few
>> >> > > > > >> > >>>>>>>> changes to the new consumer. Here is the code:
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>
>> >> > > > > >> >
>> >> > > > >
>> >> >
>> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> >> > > > > >> > >>>>> /apache/kafka/clients/streaming
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> For the purpose of the prototype I just liberally
>> >> > renamed
>> >> > > > > >> > >>>>>>>> everything
>> >> > > > > >> > >>>>> to
>> >> > > > > >> > >>>>>>>> try to align it with Kafka with no regard for
>> >> > > > compatibility.
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> To use this would be something like this:
>> >> > > > > >> > >>>>>>>> Properties props = new Properties();
>> >> > > > > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
>> >> > > > > >> > >>>>>>>> StreamingConfig config = new
>> >> > > > > >> > >>>>> StreamingConfig(props);
>> >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
>> >> > > > > >> > >>>>>>>> "test-topic-2");
>> >> > > > > >> config.processor(ExampleStreamProcessor.class);
>> >> > > > > >> > >>>>>>> config.serialization(new
>> >> > > > > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
>> >> > > > KafkaStreaming
>> >> > > > > >> > >>>>>> container =
>> >> > > > > >> > >>>>>>>> new KafkaStreaming(config); container.run();
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
>> >> > > > > StreamProcessor
>> >> > > > > >> > >>>>>>>> is basically StreamTask.
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> So rather than putting all the class names in a
>> file
>> >> > and
>> >> > > > then
>> >> > > > > >> > >>>>>>>> having
>> >> > > > > >> > >>>>>> the
>> >> > > > > >> > >>>>>>>> job assembled by reflection, you just instantiate
>> the
>> >> > > > > container
>> >> > > > > >> > >>>>>>>> programmatically. Work is balanced over however
>> many
>> >> > > > > instances
>> >> > > > > >> > >>>>>>>> of
>> >> > > > > >> > >>>>> this
>> >> > > > > >> > >>>>>>> are
>> >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new
>> >> tasks
>> >> > > are
>> >> > > > > >> added
>> >> > > > > >> > >>>>>>>> to
>> >> > > > > >> > >>>>> the
>> >> > > > > >> > >>>>>>>> existing containers without shutting them down).
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> We would provide some glue for running this stuff
>> in
>> >> > YARN
>> >> > > > via
>> >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of
>> >> their
>> >> > > > tools
>> >> > > > > >> > >>>>>>>> but from the
>> >> > > > > >> > >>>>>> point
>> >> > > > > >> > >>>>>>> of
>> >> > > > > >> > >>>>>>>> view of these frameworks these stream processing
>> jobs
>> >> > are
>> >> > > > > just
>> >> > > > > >> > >>>>>> stateless
>> >> > > > > >> > >>>>>>>> services that can come and go and expand and
>> contract
>> >> > at
>> >> > > > > will.
>> >> > > > > >> > >>>>>>>> There
>> >> > > > > >> > >>>>> is
>> >> > > > > >> > >>>>>>> no
>> >> > > > > >> > >>>>>>>> more custom scheduler.
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> Here are some relevant details:
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get
>> >> larger
>> >> > > if
>> >> > > > we
>> >> > > > > >> > >>>>>>>>  productionized but not vastly larger. We really
>> do
>> >> > get a
>> >> > > > ton
>> >> > > > > >> > >>>>>>>> of
>> >> > > > > >> > >>>>>>> leverage
>> >> > > > > >> > >>>>>>>>  out of Kafka.
>> >> > > > > >> > >>>>>>>>  2. Partition management is fully delegated to the
>> >> new
>> >> > > > > >> consumer.
>> >> > > > > >> > >>>>> This
>> >> > > > > >> > >>>>>>>>  is nice since now any partition management
>> strategy
>> >> > > > > available
>> >> > > > > >> > >>>>>>>> to
>> >> > > > > >> > >>>>>> Kafka
>> >> > > > > >> > >>>>>>>>  consumer is also available to Samza (and vice
>> versa)
>> >> > and
>> >> > > > > with
>> >> > > > > >> > >>>>>>>> the
>> >> > > > > >> > >>>>>>> exact
>> >> > > > > >> > >>>>>>>>  same configs.
>> >> > > > > >> > >>>>>>>>  3. It supports state as well as state reuse
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is thought
>> >> provoking.
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> -Jay
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>> >> > > > > >> > >>>>>> criccomini@apache.org>
>> >> > > > > >> > >>>>>>>> wrote:
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>>> Hey all,
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> I have had some discussions with Samza engineers
>> at
>> >> > > > LinkedIn
>> >> > > > > >> > >>>>>>>>> and
>> >> > > > > >> > >>>>>>> Confluent
>> >> > > > > >> > >>>>>>>>> and we came up with a few observations and would
>> >> like
>> >> > to
>> >> > > > > >> > >>>>>>>>> propose
>> >> > > > > >> > >>>>> some
>> >> > > > > >> > >>>>>>>>> changes.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> We've observed some things that I want to call
>> out
>> >> > about
>> >> > > > > >> > >>>>>>>>> Samza's
>> >> > > > > >> > >>>>>> design,
>> >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment
>> >> system.
>> >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
>> >> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and
>> Kafka's
>> >> > > > consumer
>> >> > > > > >> > >>>>>>>>> APIs
>> >> > > > > >> > >>>>> are
>> >> > > > > >> > >>>>>>>>> trying to solve a lot of the same problems.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> All three of these issues are related, but I'll
>> >> > address
>> >> > > > them
>> >> > > > > >> in
>> >> > > > > >> > >>>>> order.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Deployment
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic
>> >> > > deployment
>> >> > > > > >> > >>>>>>>>> scheduler
>> >> > > > > >> > >>>>>> such
>> >> > > > > >> > >>>>>>>>> as
>> >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza,
>> we
>> >> > bet
>> >> > > > that
>> >> > > > > >> > >>>>>>>>> there
>> >> > > > > >> > >>>>>> would
>> >> > > > > >> > >>>>>>>>> be
>> >> > > > > >> > >>>>>>>>> one or two winners in this area, and we could
>> >> support
>> >> > > > them,
>> >> > > > > >> and
>> >> > > > > >> > >>>>>>>>> the
>> >> > > > > >> > >>>>>> rest
>> >> > > > > >> > >>>>>>>>> would go away. In reality, there are many
>> >> variations.
>> >> > > > > >> > >>>>>>>>> Furthermore,
>> >> > > > > >> > >>>>>> many
>> >> > > > > >> > >>>>>>>>> people still prefer to just start their
>> processors
>> >> > like
>> >> > > > > normal
>> >> > > > > >> > >>>>>>>>> Java processes, and use traditional deployment
>> >> scripts
>> >> > > > such
>> >> > > > > as
>> >> > > > > >> > >>>>>>>>> Fabric,
>> >> > > > > >> > >>>>>> Chef,
>> >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on
>> users
>> >> > makes
>> >> > > > the
>> >> > > > > >> > >>>>>>>>> Samza start-up process really painful for first
>> time
>> >> > > > users.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was also a
>> bit
>> >> of
>> >> > a
>> >> > > > > >> > >>>>>>>>> mis-fire
>> >> > > > > >> > >>>>>> because
>> >> > > > > >> > >>>>>>>>> of
>> >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between the
>> nature of
>> >> > > batch
>> >> > > > > >> jobs
>> >> > > > > >> > >>>>>>>>> and
>> >> > > > > >> > >>>>>>> stream
>> >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made conscious
>> effort
>> >> to
>> >> > > > favor
>> >> > > > > >> > >>>>>>>>> the
>> >> > > > > >> > >>>>>> Hadoop
>> >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked
>> >> and
>> >> > > was
>> >> > > > > well
>> >> > > > > >> > >>>>>>> understood.
>> >> > > > > >> > >>>>>>>>> One thing that we missed was that batch jobs
>> have a
>> >> > > > definite
>> >> > > > > >> > >>>>>> beginning,
>> >> > > > > >> > >>>>>>>>> and
>> >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't (usually).
>> >> This
>> >> > > > leads
>> >> > > > > to
>> >> > > > > >> > >>>>>>>>> a
>> >> > > > > >> > >>>>> much
>> >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream processors.
>> >> You
>> >> > > > > >> basically
>> >> > > > > >> > >>>>>>>>> just
>> >> > > > > >> > >>>>>>> need
>> >> > > > > >> > >>>>>>>>> to find a place to start the processor, and start
>> >> it.
>> >> > > The
>> >> > > > > way
>> >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of
>> a
>> >> > > cluster
>> >> > > > > >> > >>>>>>>>> being "full". We always
>> >> > > > > >> > >>>>>> add
>> >> > > > > >> > >>>>>>>>> more machines. The problem with coupling Samza
>> with
>> >> a
>> >> > > > > >> scheduler
>> >> > > > > >> > >>>>>>>>> is
>> >> > > > > >> > >>>>>> that
>> >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to handle
>> deployment.
>> >> > > This
>> >> > > > > >> pulls
>> >> > > > > >> > >>>>>>>>> in a
>> >> > > > > >> > >>>>>>> bunch
>> >> > > > > >> > >>>>>>>>> of things such as configuration distribution
>> (config
>> >> > > > > stream),
>> >> > > > > >> > >>>>>>>>> shell
>> >> > > > > >> > >>>>>>> scrips
>> >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the
>> .tgz
>> >> > > > stuff),
>> >> > > > > >> etc.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic deployment
>> was
>> >> to
>> >> > > > > support
>> >> > > > > >> > >>>>>>>>> data locality. If you want to have locality, you
>> >> need
>> >> > to
>> >> > > > put
>> >> > > > > >> > >>>>>>>>> your
>> >> > > > > >> > >>>>>> processors
>> >> > > > > >> > >>>>>>>>> close to the data they're processing. Upon
>> further
>> >> > > > > >> > >>>>>>>>> investigation,
>> >> > > > > >> > >>>>>>> though,
>> >> > > > > >> > >>>>>>>>> this feature is not that beneficial. There is
>> some
>> >> > good
>> >> > > > > >> > >>>>>>>>> discussion
>> >> > > > > >> > >>>>>> about
>> >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we
>> took
>> >> the
>> >> > > > > >> > >>>>>>>>> Map/Reduce
>> >> > > > > >> > >>>>>> path,
>> >> > > > > >> > >>>>>>>>> but
>> >> > > > > >> > >>>>>>>>> there are some fundamental differences between
>> HDFS
>> >> > and
>> >> > > > > Kafka.
>> >> > > > > >> > >>>>>>>>> HDFS
>> >> > > > > >> > >>>>>> has
>> >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to
>> >> less
>> >> > > > > >> > >>>>>>>>> optimization potential with stream processors on
>> top
>> >> > of
>> >> > > > > Kafka.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> This feature is also used as a crutch. Samza
>> doesn't
>> >> > > have
>> >> > > > > any
>> >> > > > > >> > >>>>>>>>> built
>> >> > > > > >> > >>>>> in
>> >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the
>> >> > > dynamic
>> >> > > > > >> > >>>>>>>>> deployment scheduling system to handle restarts
>> >> when a
>> >> > > > > >> > >>>>>>>>> processor dies. This has
>> >> > > > > >> > >>>>>>> made
>> >> > > > > >> > >>>>>>>>> it very difficult to write a standalone Samza
>> >> > container
>> >> > > > > >> > >>>> (SAMZA-516).
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Pluggability
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> In some cases pluggability is good, but I think
>> that
>> >> > > we've
>> >> > > > > >> gone
>> >> > > > > >> > >>>>>>>>> too
>> >> > > > > >> > >>>>>> far
>> >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> * Pluggable config.
>> >> > > > > >> > >>>>>>>>> * Pluggable metrics.
>> >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
>> >> > > > > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
>> >> > > > > SystemProducer,
>> >> > > > > >> > >>>> etc).
>> >> > > > > >> > >>>>>>>>> * Pluggable serdes.
>> >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
>> >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about every
>> >> component
>> >> > > > > >> > >>>>> (MessageChooser,
>> >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter,
>> etc).
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> There's probably more that I've forgotten, as
>> well.
>> >> > Some
>> >> > > > of
>> >> > > > > >> > >>>>>>>>> these
>> >> > > > > >> > >>>>> are
>> >> > > > > >> > >>>>>>>>> useful, but some have proven not to be. This all
>> >> comes
>> >> > > at
>> >> > > > a
>> >> > > > > >> cost:
>> >> > > > > >> > >>>>>>>>> complexity. This complexity is making it harder
>> for
>> >> > our
>> >> > > > > users
>> >> > > > > >> > >>>>>>>>> to
>> >> > > > > >> > >>>>> pick
>> >> > > > > >> > >>>>>> up
>> >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also makes it
>> >> > difficult
>> >> > > > for
>> >> > > > > >> > >>>>>>>>> Samza developers to reason about what the
>> >> > > characteristics
>> >> > > > of
>> >> > > > > >> > >>>>>>>>> the container (since the characteristics change
>> >> > > depending
>> >> > > > on
>> >> > > > > >> > >>>>>>>>> which plugins are use).
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> The issues with pluggability are most visible in
>> the
>> >> > > > System
>> >> > > > > >> APIs.
>> >> > > > > >> > >>>>> What
>> >> > > > > >> > >>>>>>>>> Samza really requires to be functional is Kafka
>> as
>> >> its
>> >> > > > > >> > >>>>>>>>> transport
>> >> > > > > >> > >>>>>> layer.
>> >> > > > > >> > >>>>>>>>> But
>> >> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases into one
>> >> API:
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
>> >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> The current System API supports both of these use
>> >> > cases.
>> >> > > > The
>> >> > > > > >> > >>>>>>>>> problem
>> >> > > > > >> > >>>>>> is,
>> >> > > > > >> > >>>>>>>>> we
>> >> > > > > >> > >>>>>>>>> actually want different features for each use
>> case.
>> >> By
>> >> > > > > >> papering
>> >> > > > > >> > >>>>>>>>> over
>> >> > > > > >> > >>>>>>> these
>> >> > > > > >> > >>>>>>>>> two use cases, and providing a single API, we've
>> >> > > > introduced
>> >> > > > > a
>> >> > > > > >> > >>>>>>>>> ton of
>> >> > > > > >> > >>>>>>> leaky
>> >> > > > > >> > >>>>>>>>> abstractions.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> For example, what we'd really like in (2) is to
>> have
>> >> > > > > >> > >>>>>>>>> monotonically increasing longs for offsets (like
>> >> > Kafka).
>> >> > > > > This
>> >> > > > > >> > >>>>>>>>> would be at odds
>> >> > > > > >> > >>>>> with
>> >> > > > > >> > >>>>>>> (1),
>> >> > > > > >> > >>>>>>>>> though, since different systems have different
>> >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
>> >> > > > > >> > >>>>>>>>> There was discussion both on the mailing list and
>> >> the
>> >> > > SQL
>> >> > > > > >> JIRAs
>> >> > > > > >> > >>>>> about
>> >> > > > > >> > >>>>>>> the
>> >> > > > > >> > >>>>>>>>> need for this.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> The same thing holds true for replayability.
>> Kafka
>> >> > > allows
>> >> > > > us
>> >> > > > > >> to
>> >> > > > > >> > >>>>> rewind
>> >> > > > > >> > >>>>>>>>> when
>> >> > > > > >> > >>>>>>>>> we have a failure. Many other systems don't. In
>> some
>> >> > > > cases,
>> >> > > > > >> > >>>>>>>>> systems
>> >> > > > > >> > >>>>>>> return
>> >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
>> >> WikipediaSystemConsumer)
>> >> > > > > because
>> >> > > > > >> > >>>>>>>>> they
>> >> > > > > >> > >>>>>> have
>> >> > > > > >> > >>>>>>> no
>> >> > > > > >> > >>>>>>>>> offsets.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka supports
>> >> > > > > partitioning,
>> >> > > > > >> > >>>>>>>>> but
>> >> > > > > >> > >>>>> many
>> >> > > > > >> > >>>>>>>>> systems don't. We model this by having a single
>> >> > > partition
>> >> > > > > for
>> >> > > > > >> > >>>>>>>>> those systems. Still, other systems model
>> >> partitioning
>> >> > > > > >> > >>>> differently (e.g.
>> >> > > > > >> > >>>>>>>>> Kinesis).
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a mess.
>> Creating
>> >> > > streams
>> >> > > > > in
>> >> > > > > >> a
>> >> > > > > >> > >>>>>>>>> system-agnostic way is almost impossible. As is
>> >> > modeling
>> >> > > > > >> > >>>>>>>>> metadata
>> >> > > > > >> > >>>>> for
>> >> > > > > >> > >>>>>>> the
>> >> > > > > >> > >>>>>>>>> system (replication factor, partitions, location,
>> >> > etc).
>> >> > > > The
>> >> > > > > >> > >>>>>>>>> list
>> >> > > > > >> > >>>>> goes
>> >> > > > > >> > >>>>>>> on.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Duplicate work
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's
>> >> > > consumer
>> >> > > > > and
>> >> > > > > >> > >>>>> producer
>> >> > > > > >> > >>>>>>>>> APIs
>> >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On the
>> >> > consumer-side,
>> >> > > > you
>> >> > > > > >> > >>>>>>>>> had two
>> >> > > > > >> > >>>>>>>>> options: use the high level consumer, or the
>> simple
>> >> > > > > consumer.
>> >> > > > > >> > >>>>>>>>> The
>> >> > > > > >> > >>>>>>> problem
>> >> > > > > >> > >>>>>>>>> with the high-level consumer was that it
>> controlled
>> >> > your
>> >> > > > > >> > >>>>>>>>> offsets, partition assignments, and the order in
>> >> which
>> >> > > you
>> >> > > > > >> > >>>>>>>>> received messages. The
>> >> > > > > >> > >>>>> problem
>> >> > > > > >> > >>>>>>>>> with
>> >> > > > > >> > >>>>>>>>> the simple consumer is that it's not simple. It's
>> >> > basic.
>> >> > > > You
>> >> > > > > >> > >>>>>>>>> end up
>> >> > > > > >> > >>>>>>> having
>> >> > > > > >> > >>>>>>>>> to handle a lot of really low-level stuff that
>> you
>> >> > > > > shouldn't.
>> >> > > > > >> > >>>>>>>>> We
>> >> > > > > >> > >>>>>> spent a
>> >> > > > > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer
>> very
>> >> > > > robust.
>> >> > > > > >> It
>> >> > > > > >> > >>>>>>>>> also allows us to support some cool features:
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
>> prioritization.
>> >> > > > > >> > >>>>>>>>> * Tight control over partition assignment to
>> support
>> >> > > > joins,
>> >> > > > > >> > >>>>>>>>> global
>> >> > > > > >> > >>>>>> state
>> >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
>> >> > > > > >> > >>>>>>>>> * Tight control over offset checkpointing.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> What we didn't realize at the time is that these
>> >> > > features
>> >> > > > > >> > >>>>>>>>> should
>> >> > > > > >> > >>>>>>> actually
>> >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just
>> >> Samza
>> >> > > > stream
>> >> > > > > >> > >>>>>> processors)
>> >> > > > > >> > >>>>>>>>> end up wanting to do things like joins and
>> partition
>> >> > > > > >> > >>>>>>>>> assignment. The
>> >> > > > > >> > >>>>>>> Kafka
>> >> > > > > >> > >>>>>>>>> community has come to the same conclusion.
>> They're
>> >> > > adding
>> >> > > > a
>> >> > > > > >> ton
>> >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka consumer
>> >> > > implementation.
>> >> > > > > To a
>> >> > > > > >> > >>>>>>>>> large extent,
>> >> > > > > >> > >>>>> it's
>> >> > > > > >> > >>>>>>>>> duplicate work to what we've already done in
>> Samza.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a very
>> similar
>> >> > > > > approach
>> >> > > > > >> > >>>>>>>>> to
>> >> > > > > >> > >>>>>> Samza's
>> >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation for
>> handling
>> >> > > offset
>> >> > > > > >> > >>>>>> checkpointing.
>> >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature
>> >> > stores
>> >> > > > > >> offset
>> >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch
>> them
>> >> > > from
>> >> > > > > the
>> >> > > > > >> > >>>>>>>>> broker.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since we could
>> >> have
>> >> > > > shared
>> >> > > > > >> > >>>>>>>>> the
>> >> > > > > >> > >>>>> work
>> >> > > > > >> > >>>>>> if
>> >> > > > > >> > >>>>>>>>> it
>> >> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Vision
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> All of this leads me to a rather radical
>> proposal.
>> >> > Samza
>> >> > > > is
>> >> > > > > >> > >>>>> relatively
>> >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to say that
>> we're
>> >> > > near a
>> >> > > > > 1.0
>> >> > > > > >> > >>>>>> release.
>> >> > > > > >> > >>>>>>>>> I'd
>> >> > > > > >> > >>>>>>>>> like to propose that we take what we've learned,
>> and
>> >> > > begin
>> >> > > > > >> > >>>>>>>>> thinking
>> >> > > > > >> > >>>>>>> about
>> >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were
>> >> > > starting
>> >> > > > > >> from
>> >> > > > > >> > >>>>>> scratch?
>> >> > > > > >> > >>>>>>>>> My
>> >> > > > > >> > >>>>>>>>> proposal is to:
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run
>> Samza
>> >> > > > > >> > >>>>>>>>> processors, and eliminate all direct dependences
>> on
>> >> > > YARN,
>> >> > > > > >> Mesos,
>> >> > > > > >> > >>>> etc.
>> >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka
>> as
>> >> the
>> >> > > > > stream
>> >> > > > > >> > >>>>>> processing
>> >> > > > > >> > >>>>>>>>> layer.
>> >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
>> >> serialization,
>> >> > > and
>> >> > > > > >> > >>>>>>>>> config
>> >> > > > > >> > >>>>>>> systems,
>> >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> This would fix all of the issues that I outlined
>> >> > above.
>> >> > > It
>> >> > > > > >> > >>>>>>>>> should
>> >> > > > > >> > >>>>> also
>> >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty dramatically.
>> >> > > Supporting
>> >> > > > > >> only
>> >> > > > > >> > >>>>>>>>> a standalone container will allow Samza to be
>> >> executed
>> >> > > on
>> >> > > > > YARN
>> >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or
>> >> most
>> >> > > > other
>> >> > > > > >> > >>>>>>>>> in-house
>> >> > > > > >> > >>>>>>> deployment
>> >> > > > > >> > >>>>>>>>> systems. This should make life a lot easier for
>> new
>> >> > > users.
>> >> > > > > >> > >>>>>>>>> Imagine
>> >> > > > > >> > >>>>>>> having
>> >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop
>> in
>> >> > > mailing
>> >> > > > > >> list
>> >> > > > > >> > >>>>>> traffic
>> >> > > > > >> > >>>>>>>>> will be pretty dramatic.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The
>> >> > > reality
>> >> > > > > is,
>> >> > > > > >> > >>>>> everyone
>> >> > > > > >> > >>>>>>>>> that
>> >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We
>> basically
>> >> > > > require
>> >> > > > > >> it
>> >> > > > > >> > >>>>>> already
>> >> > > > > >> > >>>>>>> in
>> >> > > > > >> > >>>>>>>>> order for most features to work. Those that are
>> >> using
>> >> > > > other
>> >> > > > > >> > >>>>>>>>> systems
>> >> > > > > >> > >>>>>> are
>> >> > > > > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and
>> >> then
>> >> > > > they
>> >> > > > > do
>> >> > > > > >> > >>>>>>>>> the processing on top. There is already
>> discussion (
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>
>> >> > > > > >> >
>> >> > > > >
>> >> >
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> >> > > > > >> > >>>>> 767
>> >> > > > > >> > >>>>>>>>> )
>> >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely
>> >> easy.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Once we make the call to couple with Kafka, we
>> can
>> >> > > > leverage
>> >> > > > > a
>> >> > > > > >> > >>>>>>>>> ton of
>> >> > > > > >> > >>>>>>> their
>> >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own
>> >> > config,
>> >> > > > > >> > >>>>>>>>> metrics,
>> >> > > > > >> > >>>>> etc.
>> >> > > > > >> > >>>>>>> We
>> >> > > > > >> > >>>>>>>>> can all share the same libraries, and make them
>> >> > better.
>> >> > > > This
>> >> > > > > >> > >>>>>>>>> will
>> >> > > > > >> > >>>>> also
>> >> > > > > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and
>> >> will
>> >> > > let
>> >> > > > > us
>> >> > > > > >> > >>>>> leverage
>> >> > > > > >> > >>>>>>>>> their offset management and partition management,
>> >> > rather
>> >> > > > > than
>> >> > > > > >> > >>>>>>>>> having
>> >> > > > > >> > >>>>>> our
>> >> > > > > >> > >>>>>>>>> own. All of the coordinator stream code would go
>> >> away,
>> >> > > as
>> >> > > > > >> would
>> >> > > > > >> > >>>>>>>>> most
>> >> > > > > >> > >>>>>> of
>> >> > > > > >> > >>>>>>>>> the
>> >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push
>> some
>> >> > > > > partition
>> >> > > > > >> > >>>>>>> management
>> >> > > > > >> > >>>>>>>>> features into the Kafka broker, but they're
>> already
>> >> > > moving
>> >> > > > > in
>> >> > > > > >> > >>>>>>>>> that direction with the new consumer API. The
>> >> features
>> >> > > we
>> >> > > > > have
>> >> > > > > >> > >>>>>>>>> for
>> >> > > > > >> > >>>>>> partition
>> >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like
>> >> they
>> >> > > > should
>> >> > > > > >> be
>> >> > > > > >> > >>>>>>>>> in
>> >> > > > > >> > >>>>>> Kafka
>> >> > > > > >> > >>>>>>>>> anyway. There will always be some niche usages
>> which
>> >> > > will
>> >> > > > > >> > >>>>>>>>> require
>> >> > > > > >> > >>>>>> extra
>> >> > > > > >> > >>>>>>>>> care and hence full control over partition
>> >> assignments
>> >> > > > much
>> >> > > > > >> > >>>>>>>>> like the
>> >> > > > > >> > >>>>>>> Kafka
>> >> > > > > >> > >>>>>>>>> low level consumer api. These would continue to
>> be
>> >> > > > > supported.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> These items will be good for the Samza community.
>> >> > > They'll
>> >> > > > > make
>> >> > > > > >> > >>>>>>>>> Samza easier to use, and make it easier for
>> >> developers
>> >> > > to
>> >> > > > > add
>> >> > > > > >> > >>>>>>>>> new features.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat
>> >> > backwards
>> >> > > > > >> > >>>>> incompatible
>> >> > > > > >> > >>>>>>>>> change). If we choose to go this route, it's
>> >> important
>> >> > > > that
>> >> > > > > we
>> >> > > > > >> > >>>>> openly
>> >> > > > > >> > >>>>>>>>> communicate how we're going to provide a
>> migration
>> >> > path
>> >> > > > from
>> >> > > > > >> > >>>>>>>>> the
>> >> > > > > >> > >>>>>>> existing
>> >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make incompatible
>> >> > changes).
>> >> > > I
>> >> > > > > >> think
>> >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to provide a
>> >> wrapper
>> >> > to
>> >> > > > > allow
>> >> > > > > >> > >>>>>>>>> existing StreamTask implementations to continue
>> >> > running
>> >> > > on
>> >> > > > > the
>> >> > > > > >> > >>>> new container.
>> >> > > > > >> > >>>>>>> It's
>> >> > > > > >> > >>>>>>>>> also important that we openly communicate about
>> >> > timing,
>> >> > > > and
>> >> > > > > >> > >>>>>>>>> stages
>> >> > > > > >> > >>>>> of
>> >> > > > > >> > >>>>>>> the
>> >> > > > > >> > >>>>>>>>> migration.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you have
>> opinions.
>> >> > :)
>> >> > > > > Please
>> >> > > > > >> > >>>>>>>>> send
>> >> > > > > >> > >>>>>> your
>> >> > > > > >> > >>>>>>>>> thoughts and feedback.
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>> Cheers,
>> >> > > > > >> > >>>>>>>>> Chris
>> >> > > > > >> > >>>>>>>>>
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>>
>> >> > > > > >> > >>>>>>>
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>> --
>> >> > > > > >> > >>>>>> -- Guozhang
>> >> > > > > >> > >>>>>>
>> >> > > > > >> > >>>>>
>> >> > > > > >> > >>>>
>> >> > > > > >> > >>
>> >> > > > > >> > >>
>> >> > > > > >> >
>> >> > > > > >> >
>> >> > > > > >> >
>> >> > > > > >>
>> >> > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>>

Re: Thoughts and obesrvations on Samza

Posted by Guozhang Wang <wa...@gmail.com>.
>From peanut gallery..

I like Yi's proposal in re-scoping the Samza project / code-base as "Stream
Processing as a Service" that will potentially include:

1. A service manager with some REST / Web UI to accept stream processing
jobs in terms of tgz / configs and schedule them as for:

a. partitions -> tasks assignment.
b. tasks -> containers assignment, including integration with different
underlying resource management systems like YARN / slider, Mesos /
Marathon, AWS, etc for auto scaling.

2. High-level SQL support, including parsing the SQL query, optimizing the
logical plan into physical DAGs and transform into streaming jobs (again,
in tgz / configs) and then submitting to the Samza service.

3. Either retain the current pluggable SystemProducer / Consumer APIs, or
drop them and rely on Kafka Copycat to make all the data available in Kafka
(as a Kafka committer I would personally of course prefer the second option
to bet on Kafka's becoming THE data transfer layer :).

The only radical change it would be asking for is to replace its own
processing framework with Kafka's standalone streaming processors (including
the processing API, k-v storage, metrics, configs, etc and optionally
providing API wrappers from the Kafka standalone streaming to the current
StreamTask API; this is of course totally up to the community to decide),
which already solves a large chunk of the problems raised in Chris'
original email.

Admittedly this is still a large change for Samza. But this change, to me,
is mainly a project / code change rather than a community change, which
regards largely to review process / coding style / code-repo / culture /
etc, as mentioned in Jakob/Julian's email. I agrees that a community change
would probably be much larger / quicker of a change for Samza right now,
and we should probably be more careful for those changes.

Some more technical details:

1. Regarding partition -> tasks assignment and tasks -> containers
assignment mentioned above, similar discussions have already been brought
up at the Kafka community for Copycat, and I think enabling client-side
partition assignments would be possibly be common logic sharing for Samza
and Copycat moving forward.

2. Regarding the REST interface of the Samza service, this has also been
proposed in Copycat (some details can be found in the wiki pages), and I
also think it is doable for design the APIs so that the Copycat and Samza
can be much aligned in a shared cluster environment.

Guozhang


On Mon, Jul 13, 2015 at 11:25 AM, Jordan Shaw <jo...@pubnub.com> wrote:

> Jay,
> I think doing this iteratively in smaller chunks is a better way to go as
> new issues arise. As Navina said Kafka is a "stream system" and Samza is a
> "stream processor" and those two ideas should be mutually exclusive.
>
> -Jordan
>
> On Mon, Jul 13, 2015 at 10:06 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hmm, thought about this more. Maybe this is just too much too quick.
> > Overall I think there is some enthusiasm for the proposal but it's not
> > really unanimous enough to make any kind of change this big cleanly. The
> > board doesn't really like the merging stuff, user's are concerned about
> > compatibility, I didn't feel there was unanimous agreement on dropping
> > SystemConsumer, etc. Even if this is the right end state to get to,
> > probably trying to push all this through at once isn't the right way to
> do
> > it.
> >
> > So let me propose a kind of fifth (?) option which I think is less
> dramatic
> > and let's things happen gradually. I think this is kind of like combining
> > the first part of Yi's proposal and Jakob's third option, leaving the
> rest
> > to be figured out incrementally:
> >
> > Option 5: We continue the prototype I shared and propose that as a kind
> of
> > "transformer" client API in Kafka. This isn't really a full-fledged
> stream
> > processing layer, more like a supped up consumer api for munging topics.
> > This would let us figure out some of the technical bits, how to do this
> on
> > Kafka's group management features, how to integrate the txn feature to do
> > the exactly-once stuff in these transformations, and get all this stuff
> > solid. This api would have valid uses in it's own right, especially when
> > your transformation will be embedded inside an existing service or
> > application which isn't possible with Samza (or other existing systems
> that
> > I know of).
> >
> > Independently we can iterate on some of the ideas of the original
> proposal
> > individually and figure out how (if at all) to make use of this
> > functionality. This can be done bit-by-bit:
> > - Could be that the existing StreamTask API ends up wrapping this
> > - Could end up exposed directly in Samza as Yi proposed
> > - Could be that just the lower-level group-management stuff get's used,
> and
> > in this case it could be either just for standalone mode, or always
> > - Could be that it stays as-is
> >
> > The advantage of this is it is lower risk...we basically don't have to
> make
> > 12 major decisions all at once that kind of hinge on what amounts to a
> > pretty aggressive rewrite. The disadvantage of this is it is a bit more
> > confusing as all this is getting figured out.
> >
> > As with some of the other stuff, this would require a further discussion
> in
> > the Kafka community if people do like this approach.
> >
> > Thoughts?
> >
> > -Jay
> >
> >
> >
> >
> > On Sun, Jul 12, 2015 at 10:52 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Chris,
> > >
> > > Yeah, I'm obviously in favor of this.
> > >
> > > The sub-project approach seems the ideal way to take a graceful step in
> > > this direction, so I will ping the board folks and see why they are
> > > discouraged, it would be good to understand that. If we go that route
> we
> > > would need to do a similar discussion in the Kafka list (but makes
> sense
> > to
> > > figure out first if it is what Samza wants).
> > >
> > > Irrespective of how it's implemented, though, to me the important
> things
> > > are the following:
> > > 1. Unify the website, config, naming, docs, metrics, etc--basically fix
> > > the product experience so the "stream" and the "processing" feel like a
> > > single user experience and brand. This seems minor but I think is a
> > really
> > > big deal.
> > > 2. Make "standalone" mode a first class citizen and have a real
> technical
> > > plan to be able to support cluster managers other than YARN.
> > > 3. Make the config and out-of-the-box experience more usable
> > >
> > > I think that prototype gives a practical example of how 1-3 could be
> done
> > > and we should pursue it. This is a pretty radical change, so I wouldn't
> > be
> > > shocked if people didn't want to take a step like that.
> > >
> > > Maybe it would make sense to see if people are on board with that
> general
> > > idea, and then try to get some advice on sub-projects in parallel and
> > nail
> > > down those details?
> > >
> > > -Jay
> > >
> > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
> criccomini@apache.org>
> > > wrote:
> > >
> > >> Hey all,
> > >>
> > >> I want to start by saying that I'm absolutely thrilled to be a part of
> > >> this
> > >> community. The amount of level-headed, thoughtful, educated discussion
> > >> that's gone on over the past ~10 days is overwhelming. Wonderful.
> > >>
> > >> It seems like discussion is waning a bit, and we've reached some
> > >> conclusions. There are several key emails in this threat, which I want
> > to
> > >> call out:
> > >>
> > >> 1. Jakob's summary of the three potential ways forward.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > >> 2. Julian's call out that we should be focusing on community over
> code.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > >> 3. Martin's summary about the benefits of merging communities.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > >> 4. Jakob's comments about the distinction between community and code
> > >> paths.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> > >>
> > >> I agree with the comments on all of these emails. I think Martin's
> > summary
> > >> of his position aligns very closely with my own. To that end, I think
> we
> > >> should get concrete about what the proposal is, and call a vote on it.
> > >> Given that Jay, Martin, and I seem to be aligning fairly closely, I
> > think
> > >> we should start with:
> > >>
> > >> 1. [community] Make Samza a subproject of Kafka.
> > >> 2. [community] Make all Samza PMC/committers committers of the
> > subproject.
> > >> 3. [community] Migrate Samza's website/documentation into Kafka's.
> > >> 4. [code] Have the Samza community and the Kafka community start a
> > >> from-scratch reboot together in the new Kafka subproject. We can
> > >> borrow/copy &  paste significant chunks of code from Samza's code
> base.
> > >> 5. [code] The subproject would intentionally eliminate support for
> both
> > >> other streaming systems and all deployment systems.
> > >> 6. [code] Attempt to provide a bridge from our SystemConsumer to
> KIP-26
> > >> (copy cat)
> > >> 7. [code] Attempt to provide a bridge from the new subproject's
> > processor
> > >> interface to our legacy StreamTask interface.
> > >> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> > >> subproject that has a fault-tolerant container with state management.
> > >>
> > >> It's likely that (6) and (7) won't be fully drop-in. Still, the closer
> > we
> > >> can get, the better it's going to be for our existing community.
> > >>
> > >> One thing that I didn't touch on with (2) is whether any Samza PMC
> > members
> > >> should be rolled into Kafka PMC membership as well (though, Jay and
> > Jakob
> > >> are already PMC members on both). I think that Samza's community
> > deserves
> > >> a
> > >> voice on the PMC, so I'd propose that we roll at least a few PMC
> members
> > >> into the Kafka PMC, but I don't have a strong framework for which
> people
> > >> to
> > >> pick.
> > >>
> > >> Before (8), I think that Samza's TLP can continue to commit bug fixes
> > and
> > >> patches as it sees fit, provided that we openly communicate that we
> > won't
> > >> necessarily migrate new features to the new subproject, and that the
> TLP
> > >> will be shut down after the migration to the Kafka subproject occurs.
> > >>
> > >> Jakob, I could use your guidance here about about how to achieve this
> > from
> > >> an Apache process perspective (sorry).
> > >>
> > >> * Should I just call a vote on this proposal?
> > >> * Should it happen on dev or private?
> > >> * Do committers have binding votes, or just PMC?
> > >>
> > >> Having trouble finding much detail on the Apache wikis. :(
> > >>
> > >> Cheers,
> > >> Chris
> > >>
> > >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com>
> wrote:
> > >>
> > >> > Thanks, Jay. This argument persuaded me actually. :)
> > >> >
> > >> > Fang, Yan
> > >> > yanfang724@gmail.com
> > >> >
> > >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > >> >
> > >> > > Hey Yan,
> > >> > >
> > >> > > Yeah philosophically I think the argument is that you should
> capture
> > >> the
> > >> > > stream in Kafka independent of the transformation. This is
> > obviously a
> > >> > > Kafka-centric view point.
> > >> > >
> > >> > > Advantages of this:
> > >> > > - In practice I think this is what e.g. Storm people often end up
> > >> doing
> > >> > > anyway. You usually need to throttle any access to a live serving
> > >> > database.
> > >> > > - Can have multiple subscribers and they get the same thing
> without
> > >> > > additional load on the source system.
> > >> > > - Applications can tap into the stream if need be by subscribing.
> > >> > > - You can debug your transformation by tailing the Kafka topic
> with
> > >> the
> > >> > > console consumer
> > >> > > - Can tee off the same data stream for batch analysis or Lambda
> arch
> > >> > style
> > >> > > re-processing
> > >> > >
> > >> > > The disadvantage is that it will use Kafka resources. But the idea
> > is
> > >> > > eventually you will have multiple subscribers to any data source
> (at
> > >> > least
> > >> > > for monitoring) so you will end up there soon enough anyway.
> > >> > >
> > >> > > Down the road the technical benefit is that I think it gives us a
> > good
> > >> > path
> > >> > > towards end-to-end exactly once semantics from source to
> > destination.
> > >> > > Basically the connectors need to support idempotence when talking
> to
> > >> > Kafka
> > >> > > and we need the transactional write feature in Kafka to make the
> > >> > > transformation atomic. This is actually pretty doable if you
> > separate
> > >> > > connector=>kafka problem from the generic transformations which
> are
> > >> > always
> > >> > > kafka=>kafka. However I think it is quite impossible to do in a
> > >> > all_things
> > >> > > => all_things environment. Today you can say "well the semantics
> of
> > >> the
> > >> > > Samza APIs depend on the connectors you use" but it is actually
> > worse
> > >> > then
> > >> > > that because the semantics actually depend on the pairing of
> > >> > connectors--so
> > >> > > not only can you probably not get a usable "exactly once"
> guarantee
> > >> > > end-to-end it can actually be quite hard to reverse engineer what
> > >> > property
> > >> > > (if any) your end-to-end flow has if you have heterogenous
> systems.
> > >> > >
> > >> > > -Jay
> > >> > >
> > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > > > {quote}
> > >> > > > maintained in a separate repository and retaining the existing
> > >> > > > committership but sharing as much else as possible (website,
> etc)
> > >> > > > {quote}
> > >> > > >
> > >> > > > Overall, I agree on this idea. Now the question is more about
> "how
> > >> to
> > >> > do
> > >> > > > it".
> > >> > > >
> > >> > > > On the other hand, one thing I want to point out is that, if we
> > >> decide
> > >> > to
> > >> > > > go this way, how do we want to support
> > >> > > > otherSystem-transformation-otherSystem use case?
> > >> > > >
> > >> > > > Basically, there are four user groups here:
> > >> > > >
> > >> > > > 1. Kafka-transformation-Kafka
> > >> > > > 2. Kafka-transformation-otherSystem
> > >> > > > 3. otherSystem-transformation-Kafka
> > >> > > > 4. otherSystem-transformation-otherSystem
> > >> > > >
> > >> > > > For group 1, they can easily use the new Samza library to
> achieve.
> > >> For
> > >> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka
> or
> > >> > Kafka->
> > >> > > > transformation -> copyCat.
> > >> > > >
> > >> > > > The problem is for group 4. Do we want to abandon this or still
> > >> support
> > >> > > it?
> > >> > > > Of course, this use case can be achieved by using copyCat ->
> > >> > > transformation
> > >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> > persuade
> > >> > them
> > >> > > to
> > >> > > > do this long chain. If yes, it will also be a win for Kafka too.
> > Or
> > >> if
> > >> > > > there is no one in this community actually doing this so far,
> > maybe
> > >> ok
> > >> > to
> > >> > > > not support the group 4 directly.
> > >> > > >
> > >> > > > Thanks,
> > >> > > >
> > >> > > > Fang, Yan
> > >> > > > yanfang724@gmail.com
> > >> > > >
> > >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
> > >> wrote:
> > >> > > >
> > >> > > > > Yeah I agree with this summary. I think there are kind of two
> > >> > questions
> > >> > > > > here:
> > >> > > > > 1. Technically does alignment/reliance on Kafka make sense
> > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
> alignment
> > >> with
> > >> > > > Kafka
> > >> > > > > make sense
> > >> > > > >
> > >> > > > > Personally I do think both of these things would be really
> > >> valuable,
> > >> > > and
> > >> > > > > would dramatically alter the trajectory of the project.
> > >> > > > >
> > >> > > > > My preference would be to see if people can mostly agree on a
> > >> > direction
> > >> > > > > rather than splintering things off. From my point of view the
> > >> ideal
> > >> > > > outcome
> > >> > > > > of all the options discussed would be to make Samza a closely
> > >> aligned
> > >> > > > > subproject, maintained in a separate repository and retaining
> > the
> > >> > > > existing
> > >> > > > > committership but sharing as much else as possible (website,
> > >> etc). No
> > >> > > > idea
> > >> > > > > about how these things work, Jacob, you probably know more.
> > >> > > > >
> > >> > > > > No discussion amongst the Kafka folks has happened on this,
> but
> > >> > likely
> > >> > > we
> > >> > > > > should figure out what the Samza community actually wants
> first.
> > >> > > > >
> > >> > > > > I admit that this is a fairly radical departure from how
> things
> > >> are.
> > >> > > > >
> > >> > > > > If that doesn't fly, I think, yeah we could leave Samza as it
> is
> > >> and
> > >> > do
> > >> > > > the
> > >> > > > > more radical reboot inside Kafka. From my point of view that
> > does
> > >> > leave
> > >> > > > > things in a somewhat confusing state since now there are two
> > >> stream
> > >> > > > > processing systems more or less coupled to Kafka in large part
> > >> made
> > >> > by
> > >> > > > the
> > >> > > > > same people. But, arguably that might be a cleaner way to make
> > the
> > >> > > > cut-over
> > >> > > > > and perhaps less risky for Samza community since if it works
> > >> people
> > >> > can
> > >> > > > > switch and if it doesn't nothing will have changed. Dunno, how
> > do
> > >> > > people
> > >> > > > > feel about this?
> > >> > > > >
> > >> > > > > -Jay
> > >> > > > >
> > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> > jghoman@gmail.com>
> > >> > > wrote:
> > >> > > > >
> > >> > > > > > >  This leads me to thinking that merging projects and
> > >> communities
> > >> > > > might
> > >> > > > > > be a good idea: with the union of experience from both
> > >> communities,
> > >> > > we
> > >> > > > > will
> > >> > > > > > probably build a better system that is better for users.
> > >> > > > > > Is this what's being proposed though? Merging the projects
> > seems
> > >> > like
> > >> > > > > > a consequence of at most one of the three directions under
> > >> > > discussion:
> > >> > > > > > 1) Samza 2.0: The Samza community relies more heavily on
> Kafka
> > >> for
> > >> > > > > > configuration, etc. (to a greater or lesser extent to be
> > >> > determined)
> > >> > > > > > but the Samza community would not automatically merge withe
> > >> Kafka
> > >> > > > > > community (the Phoenix/HBase example is a good one here).
> > >> > > > > > 2) Samza Reboot: The Samza community continues to exist
> with a
> > >> > > limited
> > >> > > > > > project scope, but similarly would not need to be part of
> the
> > >> Kafka
> > >> > > > > > community (ie given committership) to progress.  Here, maybe
> > the
> > >> > > Samza
> > >> > > > > > team would become a subproject of Kafka (the Board frowns on
> > >> > > > > > subprojects at the moment, so I'm not sure if that's even
> > >> > feasible),
> > >> > > > > > but that would not be required.
> > >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option
> > the
> > >> > Kafka
> > >> > > > > > team builds its own streaming library, possibly off of Jay's
> > >> > > > > > prototype, which has not direct lineage to the Samza team.
> > >> There's
> > >> > > no
> > >> > > > > > reason for the Kafka team to bring in the Samza team.
> > >> > > > > >
> > >> > > > > > Is the Kafka community on board with this?
> > >> > > > > >
> > >> > > > > > To be clear, all three options under discussion are
> > interesting,
> > >> > > > > > technically valid and likely healthy directions for the
> > project.
> > >> > > > > > Also, they are not mutually exclusive.  The Samza community
> > >> could
> > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
> community
> > >> went
> > >> > > > > > forward with 'Hey Samza!'  My points above are directed
> > >> entirely at
> > >> > > > > > the community aspect of these choices.
> > >> > > > > > -Jakob
> > >> > > > > >
> > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> > roger.hoover@gmail.com>
> > >> > > wrote:
> > >> > > > > > > That's great.  Thanks, Jay.
> > >> > > > > > >
> > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> > jay@confluent.io>
> > >> > > wrote:
> > >> > > > > > >
> > >> > > > > > >> Yeah totally agree. I think you have this issue even
> today,
> > >> > right?
> > >> > > > > I.e.
> > >> > > > > > if
> > >> > > > > > >> you need to make a simple config change and you're
> running
> > in
> > >> > YARN
> > >> > > > > today
> > >> > > > > > >> you end up bouncing the job which then rebuilds state. I
> > >> think
> > >> > the
> > >> > > > fix
> > >> > > > > > is
> > >> > > > > > >> exactly what you described which is to have a long
> timeout
> > on
> > >> > > > > partition
> > >> > > > > > >> movement for stateful jobs so that if a job is just
> getting
> > >> > > bounced,
> > >> > > > > and
> > >> > > > > > >> the cluster manager (or admin) is smart enough to restart
> > it
> > >> on
> > >> > > the
> > >> > > > > same
> > >> > > > > > >> host when possible, it can optimistically reuse any
> > existing
> > >> > state
> > >> > > > it
> > >> > > > > > finds
> > >> > > > > > >> on disk (if it is valid).
> > >> > > > > > >>
> > >> > > > > > >> So in this model the charter of the CM is to place
> > processes
> > >> as
> > >> > > > > > stickily as
> > >> > > > > > >> possible and to restart or re-place failed processes. The
> > >> > charter
> > >> > > of
> > >> > > > > the
> > >> > > > > > >> partition management system is to control the assignment
> of
> > >> work
> > >> > > to
> > >> > > > > > these
> > >> > > > > > >> processes. The nice thing about this is that the work
> > >> > assignment,
> > >> > > > > > timeouts,
> > >> > > > > > >> behavior, configs, and code will all be the same across
> all
> > >> > > cluster
> > >> > > > > > >> managers.
> > >> > > > > > >>
> > >> > > > > > >> So I think that prototype would actually give you exactly
> > >> what
> > >> > you
> > >> > > > > want
> > >> > > > > > >> today for any cluster manager (or manual placement +
> > restart
> > >> > > script)
> > >> > > > > > that
> > >> > > > > > >> was sticky in terms of host placement since there is
> > already
> > >> a
> > >> > > > > > configurable
> > >> > > > > > >> partition movement timeout and task-by-task state reuse
> > with
> > >> a
> > >> > > check
> > >> > > > > on
> > >> > > > > > >> state validity.
> > >> > > > > > >>
> > >> > > > > > >> -Jay
> > >> > > > > > >>
> > >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > >> > > > roger.hoover@gmail.com
> > >> > > > > >
> > >> > > > > > >> wrote:
> > >> > > > > > >>
> > >> > > > > > >> > That would be great to let Kafka do as much heavy
> lifting
> > >> as
> > >> > > > > possible
> > >> > > > > > and
> > >> > > > > > >> > make it easier for other languages to implement Samza
> > apis.
> > >> > > > > > >> >
> > >> > > > > > >> > One thing to watch out for is the interplay between
> > Kafka's
> > >> > > group
> > >> > > > > > >> > management and the external scheduler/process manager's
> > >> fault
> > >> > > > > > tolerance.
> > >> > > > > > >> > If a container dies, the Kafka group membership
> protocol
> > >> will
> > >> > > try
> > >> > > > to
> > >> > > > > > >> assign
> > >> > > > > > >> > it's tasks to other containers while at the same time
> the
> > >> > > process
> > >> > > > > > manager
> > >> > > > > > >> > is trying to relaunch the container.  Without some
> > >> > consideration
> > >> > > > for
> > >> > > > > > this
> > >> > > > > > >> > (like a configurable amount of time to wait before
> Kafka
> > >> > alters
> > >> > > > the
> > >> > > > > > group
> > >> > > > > > >> > membership), there may be thrashing going on which is
> > >> > especially
> > >> > > > bad
> > >> > > > > > for
> > >> > > > > > >> > containers with large amounts of local state.
> > >> > > > > > >> >
> > >> > > > > > >> > Someone else pointed this out already but I thought it
> > >> might
> > >> > be
> > >> > > > > worth
> > >> > > > > > >> > calling out again.
> > >> > > > > > >> >
> > >> > > > > > >> > Cheers,
> > >> > > > > > >> >
> > >> > > > > > >> > Roger
> > >> > > > > > >> >
> > >> > > > > > >> >
> > >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> > >> jay@confluent.io>
> > >> > > > > wrote:
> > >> > > > > > >> >
> > >> > > > > > >> > > Hey Roger,
> > >> > > > > > >> > >
> > >> > > > > > >> > > I couldn't agree more. We spent a bunch of time
> talking
> > >> to
> > >> > > > people
> > >> > > > > > and
> > >> > > > > > >> > that
> > >> > > > > > >> > > is exactly the stuff we heard time and again. What
> > makes
> > >> it
> > >> > > > hard,
> > >> > > > > of
> > >> > > > > > >> > > course, is that there is some tension between
> > >> compatibility
> > >> > > with
> > >> > > > > > what's
> > >> > > > > > >> > > there now and making things better for new users.
> > >> > > > > > >> > >
> > >> > > > > > >> > > I also strongly agree with the importance of
> > >> multi-language
> > >> > > > > > support. We
> > >> > > > > > >> > are
> > >> > > > > > >> > > talking now about Java, but for application
> development
> > >> use
> > >> > > > cases
> > >> > > > > > >> people
> > >> > > > > > >> > > want to work in whatever language they are using
> > >> elsewhere.
> > >> > I
> > >> > > > > think
> > >> > > > > > >> > moving
> > >> > > > > > >> > > to a model where Kafka itself does the group
> > membership,
> > >> > > > lifecycle
> > >> > > > > > >> > control,
> > >> > > > > > >> > > and partition assignment has the advantage of putting
> > all
> > >> > that
> > >> > > > > > complex
> > >> > > > > > >> > > stuff behind a clean api that the clients are already
> > >> going
> > >> > to
> > >> > > > be
> > >> > > > > > >> > > implementing for their consumer, so the added
> > >> functionality
> > >> > > for
> > >> > > > > > stream
> > >> > > > > > >> > > processing beyond a consumer becomes very minor.
> > >> > > > > > >> > >
> > >> > > > > > >> > > -Jay
> > >> > > > > > >> > >
> > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > >> > > > > > roger.hoover@gmail.com>
> > >> > > > > > >> > > wrote:
> > >> > > > > > >> > >
> > >> > > > > > >> > > > Metamorphosis...nice. :)
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > This has been a great discussion.  As a user of
> Samza
> > >> > who's
> > >> > > > > > recently
> > >> > > > > > >> > > > integrated it into a relatively large
> organization, I
> > >> just
> > >> > > > want
> > >> > > > > to
> > >> > > > > > >> add
> > >> > > > > > >> > > > support to a few points already made.
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
> > >> currently
> > >> > > > exists
> > >> > > > > > that
> > >> > > > > > >> > I've
> > >> > > > > > >> > > > experienced are:
> > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
> environments
> > >> > where
> > >> > > > > Puppet
> > >> > > > > > >> > would
> > >> > > > > > >> > > do
> > >> > > > > > >> > > > just fine but it was the only mechanism to get
> fault
> > >> > > > tolerance.
> > >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> > >> configuring
> > >> > > most
> > >> > > > > of
> > >> > > > > > the
> > >> > > > > > >> > job
> > >> > > > > > >> > > > in code rather than config files.  In general, I
> > think
> > >> the
> > >> > > > goal
> > >> > > > > > >> should
> > >> > > > > > >> > be
> > >> > > > > > >> > > > to make it harder to make mistakes, especially of
> the
> > >> kind
> > >> > > > where
> > >> > > > > > the
> > >> > > > > > >> > code
> > >> > > > > > >> > > > expects something and the config doesn't match.
> The
> > >> > current
> > >> > > > > > config
> > >> > > > > > >> is
> > >> > > > > > >> > > > quite intricate and error-prone.  For example, the
> > >> > > application
> > >> > > > > > logic
> > >> > > > > > >> > may
> > >> > > > > > >> > > > depend on bootstrapping a topic but rather than
> > >> asserting
> > >> > > that
> > >> > > > > in
> > >> > > > > > the
> > >> > > > > > >> > > code,
> > >> > > > > > >> > > > you have to rely on getting the config right.
> > Likewise
> > >> > with
> > >> > > > > > serdes,
> > >> > > > > > >> > the
> > >> > > > > > >> > > > Java representations produced by various serdes
> > (JSON,
> > >> > Avro,
> > >> > > > > etc.)
> > >> > > > > > >> are
> > >> > > > > > >> > > not
> > >> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> > >> without
> > >> > > > > changing
> > >> > > > > > >> the
> > >> > > > > > >> > > > code.   It would be nice for jobs to be able to
> > assert
> > >> > what
> > >> > > > they
> > >> > > > > > >> expect
> > >> > > > > > >> > > > from their input topics in terms of partitioning.
> > >> This is
> > >> > > > > > getting a
> > >> > > > > > >> > > little
> > >> > > > > > >> > > > off topic but I was even thinking about creating a
> > >> "Samza
> > >> > > > config
> > >> > > > > > >> > linter"
> > >> > > > > > >> > > > that would sanity check a set of configs.
> Especially
> > >> in
> > >> > > > > > >> organizations
> > >> > > > > > >> > > > where config is managed by a different team than
> the
> > >> > > > application
> > >> > > > > > >> > > developer,
> > >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > >> > > > > > >> > > > 3) Java/Scala centric - for many teams (especially
> > >> > > DevOps-type
> > >> > > > > > >> folks),
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > pain of the Java toolchain (maven, slow builds,
> weak
> > >> > command
> > >> > > > > line
> > >> > > > > > >> > > support,
> > >> > > > > > >> > > > configuration over convention) really inhibits
> > >> > productivity.
> > >> > > > As
> > >> > > > > > more
> > >> > > > > > >> > and
> > >> > > > > > >> > > > more high-quality clients become available for
> > Kafka, I
> > >> > hope
> > >> > > > > > they'll
> > >> > > > > > >> > > follow
> > >> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> > >> proposals
> > >> > > in
> > >> > > > > this
> > >> > > > > > >> > thread
> > >> > > > > > >> > > > but please consider other languages in the
> ecosystem
> > as
> > >> > > well.
> > >> > > > > > From
> > >> > > > > > >> > what
> > >> > > > > > >> > > > I've heard, Spark has more Python users than
> > >> Java/Scala.
> > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > >
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > >> > > > > > >> > > > and are working on a Yeoman generator
> > >> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> > >> > > Jython/Samza
> > >> > > > > > >> projects
> > >> > > > > > >> > to
> > >> > > > > > >> > > > alleviate some of the pain)
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > I also want to underscore Jay's point about
> improving
> > >> the
> > >> > > user
> > >> > > > > > >> > > experience.
> > >> > > > > > >> > > > That's a very important factor for adoption.  I
> think
> > >> the
> > >> > > goal
> > >> > > > > > should
> > >> > > > > > >> > be
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > make Samza as easy to get started with as something
> > >> like
> > >> > > > > Logstash.
> > >> > > > > > >> > > > Logstash is vastly inferior in terms of
> capabilities
> > to
> > >> > > Samza
> > >> > > > > but
> > >> > > > > > >> it's
> > >> > > > > > >> > > easy
> > >> > > > > > >> > > > to get started and that makes a big difference.
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > Cheers,
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > Roger
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> > Francisci
> > >> > > > Morales <
> > >> > > > > > >> > > > gdfm@apache.org> wrote:
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> > >> Metamorphosis
> > >> > > is
> > >> > > > a
> > >> > > > > > clear
> > >> > > > > > >> > > > winner
> > >> > > > > > >> > > > > :)
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > --
> > >> > > > > > >> > > > > Gianmarco
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> > >> Morales
> > >> > <
> > >> > > > > > >> > > gdfm@apache.org
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > wrote:
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > > Hi,
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > @Martin, thanks for you comments.
> > >> > > > > > >> > > > > > Maybe I'm missing some important point, but I
> > think
> > >> > > > coupling
> > >> > > > > > the
> > >> > > > > > >> > > > releases
> > >> > > > > > >> > > > > > is actually a *good* thing.
> > >> > > > > > >> > > > > > To make an example, would it be better if the
> MR
> > >> and
> > >> > > HDFS
> > >> > > > > > >> > components
> > >> > > > > > >> > > of
> > >> > > > > > >> > > > > > Hadoop had different release schedules?
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > Actually, keeping the discussion in a single
> > place
> > >> > would
> > >> > > > > make
> > >> > > > > > >> > > agreeing
> > >> > > > > > >> > > > on
> > >> > > > > > >> > > > > > releases (and backwards compatibility) much
> > >> easier, as
> > >> > > > > > everybody
> > >> > > > > > >> > > would
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > > responsible for the whole codebase.
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > That said, I like the idea of absorbing
> > samza-core
> > >> as
> > >> > a
> > >> > > > > > >> > sub-project,
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > > leave the fancy stuff separate.
> > >> > > > > > >> > > > > > It probably gives 90% of the benefits we have
> > been
> > >> > > > > discussing
> > >> > > > > > >> here.
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > Cheers,
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > --
> > >> > > > > > >> > > > > > Gianmarco
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > >> > jay.kreps@gmail.com
> > >> > > >
> > >> > > > > > wrote:
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >> Hey Martin,
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> I agree coupling release schedules is a
> > downside.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> Definitely we can try to solve some of the
> > >> > integration
> > >> > > > > > problems
> > >> > > > > > >> in
> > >> > > > > > >> > > > > >> Confluent Platform or in other distributions.
> > But
> > >> I
> > >> > > think
> > >> > > > > > this
> > >> > > > > > >> > ends
> > >> > > > > > >> > > up
> > >> > > > > > >> > > > > >> being really shallow. I guess I feel to really
> > >> get a
> > >> > > good
> > >> > > > > > user
> > >> > > > > > >> > > > > experience
> > >> > > > > > >> > > > > >> the two systems have to kind of feel like part
> > of
> > >> the
> > >> > > > same
> > >> > > > > > thing
> > >> > > > > > >> > and
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> can't really add that in later--you can put
> both
> > >> in
> > >> > the
> > >> > > > > same
> > >> > > > > > >> > > > > downloadable
> > >> > > > > > >> > > > > >> tar file but it doesn't really give a very
> > >> cohesive
> > >> > > > > feeling.
> > >> > > > > > I
> > >> > > > > > >> > agree
> > >> > > > > > >> > > > > that
> > >> > > > > > >> > > > > >> ultimately any of the project stuff is as much
> > >> social
> > >> > > and
> > >> > > > > > naming
> > >> > > > > > >> > as
> > >> > > > > > >> > > > > >> anything else--theoretically two totally
> > >> independent
> > >> > > > > projects
> > >> > > > > > >> > could
> > >> > > > > > >> > > > work
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> tightly align. In practice this seems to be
> > quite
> > >> > > > difficult
> > >> > > > > > >> > though.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> For the frameworks--totally agree it would be
> > >> good to
> > >> > > > > > maintain
> > >> > > > > > >> the
> > >> > > > > > >> > > > > >> framework support with the project. In some
> > cases
> > >> > there
> > >> > > > may
> > >> > > > > > not
> > >> > > > > > >> be
> > >> > > > > > >> > > too
> > >> > > > > > >> > > > > >> much
> > >> > > > > > >> > > > > >> there since the integration gets lighter but I
> > >> think
> > >> > > > > whatever
> > >> > > > > > >> > stubs
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> need should be included. So no I definitely
> > wasn't
> > >> > > trying
> > >> > > > > to
> > >> > > > > > >> imply
> > >> > > > > > >> > > > > >> dropping
> > >> > > > > > >> > > > > >> support for these frameworks, just making the
> > >> > > integration
> > >> > > > > > >> lighter
> > >> > > > > > >> > by
> > >> > > > > > >> > > > > >> separating process management from partition
> > >> > > management.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> You raise two good points we would have to
> > figure
> > >> out
> > >> > > if
> > >> > > > we
> > >> > > > > > went
> > >> > > > > > >> > > down
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> alignment path:
> > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think the
> > >> first
> > >> > > > > question
> > >> > > > > > is
> > >> > > > > > >> > > > whether
> > >> > > > > > >> > > > > >> some "re-branding" would be worth it. If so
> > then I
> > >> > > think
> > >> > > > we
> > >> > > > > > can
> > >> > > > > > >> > > have a
> > >> > > > > > >> > > > > big
> > >> > > > > > >> > > > > >> thread on the name. I'm definitely not set on
> > >> Kafka
> > >> > > > > > Streaming or
> > >> > > > > > >> > > Kafka
> > >> > > > > > >> > > > > >> Streams I was just using them to be kind of
> > >> > > > illustrative. I
> > >> > > > > > >> agree
> > >> > > > > > >> > > with
> > >> > > > > > >> > > > > >> your
> > >> > > > > > >> > > > > >> critique of these names, though I think people
> > >> would
> > >> > > get
> > >> > > > > the
> > >> > > > > > >> idea.
> > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about how
> to
> > >> > > "factor"
> > >> > > > > it.
> > >> > > > > > >> Here
> > >> > > > > > >> > > are
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> options I see (I could get enthusiastic about
> > any
> > >> of
> > >> > > > them):
> > >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> > seperation
> > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api
> and
> > >> > > > samza-core
> > >> > > > > > is
> > >> > > > > > >> > > > absorbed
> > >> > > > > > >> > > > > >> almost like a third client
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> Cheers,
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> -Jay
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> > Kleppmann <
> > >> > > > > > >> > > > martin@kleppmann.com>
> > >> > > > > > >> > > > > >> wrote:
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a
> few
> > >> > > follow-up
> > >> > > > > > >> > comments.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
> > >> > becoming
> > >> > > a
> > >> > > > > > >> > subproject:
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > reasons you mention are good. The risk I see
> > is
> > >> > that
> > >> > > > > > release
> > >> > > > > > >> > > > schedules
> > >> > > > > > >> > > > > >> > become coupled to each other, which can slow
> > >> > everyone
> > >> > > > > down,
> > >> > > > > > >> and
> > >> > > > > > >> > > > large
> > >> > > > > > >> > > > > >> > projects with many contributors are harder
> to
> > >> > manage.
> > >> > > > > > (Jakob,
> > >> > > > > > >> > can
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> speak
> > >> > > > > > >> > > > > >> > from experience, having seen a wider range
> of
> > >> > Hadoop
> > >> > > > > > ecosystem
> > >> > > > > > >> > > > > >> projects?)
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > Some of the goals of a better unified
> > developer
> > >> > > > > experience
> > >> > > > > > >> could
> > >> > > > > > >> > > > also
> > >> > > > > > >> > > > > be
> > >> > > > > > >> > > > > >> > solved by integrating Samza nicely into a
> > Kafka
> > >> > > > > > distribution
> > >> > > > > > >> > (such
> > >> > > > > > >> > > > as
> > >> > > > > > >> > > > > >> > Confluent's). I'm not against merging
> projects
> > >> if
> > >> > we
> > >> > > > > decide
> > >> > > > > > >> > that's
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> way
> > >> > > > > > >> > > > > >> > to go, just pointing out the same goals can
> > >> perhaps
> > >> > > > also
> > >> > > > > be
> > >> > > > > > >> > > achieved
> > >> > > > > > >> > > > > in
> > >> > > > > > >> > > > > >> > other ways.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
> dependency:
> > >> are
> > >> > > you
> > >> > > > > > >> proposing
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > >> > Samza doesn't give any help to people
> wanting
> > to
> > >> > run
> > >> > > on
> > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > >> > > > > > >> > > > > >> > So the docs would basically have a link to
> > >> Slider
> > >> > and
> > >> > > > > > nothing
> > >> > > > > > >> > > else?
> > >> > > > > > >> > > > Or
> > >> > > > > > >> > > > > >> > would we maintain integrations with a bunch
> of
> > >> > > popular
> > >> > > > > > >> > deployment
> > >> > > > > > >> > > > > >> methods
> > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts
> to
> > >> make
> > >> > > > Samza
> > >> > > > > > work
> > >> > > > > > >> > with
> > >> > > > > > >> > > > > >> Slider)?
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > I absolutely think it's a good idea to have
> > the
> > >> > "as a
> > >> > > > > > library"
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > "as a
> > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
> > >> people
> > >> > who
> > >> > > > > want
> > >> > > > > > >> them,
> > >> > > > > > >> > > > but I
> > >> > > > > > >> > > > > >> > think there should also be a low-friction
> path
> > >> for
> > >> > > > common
> > >> > > > > > "as
> > >> > > > > > >> a
> > >> > > > > > >> > > > > service"
> > >> > > > > > >> > > > > >> > deployment methods, for which we probably
> need
> > >> to
> > >> > > > > maintain
> > >> > > > > > >> > > > > integrations.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd
> to
> > >> me,
> > >> > > > > because
> > >> > > > > > >> Kafka
> > >> > > > > > >> > > is
> > >> > > > > > >> > > > > all
> > >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> > >> Transformers"
> > >> > > or
> > >> > > > > > "Kafka
> > >> > > > > > >> > > > Filters"
> > >> > > > > > >> > > > > >> > would be more apt?
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
> > >> (stream
> > >> > > > > > >> transformation
> > >> > > > > > >> > > > with
> > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> > >> library"
> > >> > > bit)
> > >> > > > > > could
> > >> > > > > > >> > > become
> > >> > > > > > >> > > > > >> part of
> > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
> > >> streaming
> > >> > SQL
> > >> > > > and
> > >> > > > > > >> > > > integrations
> > >> > > > > > >> > > > > >> with
> > >> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> > >> project?
> > >> > > In
> > >> > > > > > other
> > >> > > > > > >> > > words,
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > would absorb the proven, stable core of
> Samza,
> > >> > which
> > >> > > > > would
> > >> > > > > > >> > become
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in this
> > >> > thread.
> > >> > > > The
> > >> > > > > > Samza
> > >> > > > > > >> > > > project
> > >> > > > > > >> > > > > >> > would then target that third Kafka client as
> > its
> > >> > base
> > >> > > > > API,
> > >> > > > > > and
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > >> project
> > >> > > > > > >> > > > > >> > would be freed up to explore more
> experimental
> > >> new
> > >> > > > > > horizons.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > Martin
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > >> > > > jay.kreps@gmail.com>
> > >> > > > > > >> wrote:
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > > Hey Martin,
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I
> actually
> > >> > don't
> > >> > > > > think
> > >> > > > > > it
> > >> > > > > > >> > ties
> > >> > > > > > >> > > > our
> > >> > > > > > >> > > > > >> > hands
> > >> > > > > > >> > > > > >> > > at all, all it does is refactor things.
> The
> > >> > > division
> > >> > > > of
> > >> > > > > > >> > > > > >> responsibility is
> > >> > > > > > >> > > > > >> > > that Samza core is responsible for task
> > >> > lifecycle,
> > >> > > > > state,
> > >> > > > > > >> and
> > >> > > > > > >> > > > > >> partition
> > >> > > > > > >> > > > > >> > > management (using the Kafka co-ordinator)
> > but
> > >> it
> > >> > is
> > >> > > > NOT
> > >> > > > > > >> > > > responsible
> > >> > > > > > >> > > > > >> for
> > >> > > > > > >> > > > > >> > > packaging, configuration deployment or
> > >> execution
> > >> > of
> > >> > > > > > >> processes.
> > >> > > > > > >> > > The
> > >> > > > > > >> > > > > >> > problem
> > >> > > > > > >> > > > > >> > > of packaging and starting these processes
> is
> > >> > > > > > >> > > > > >> > > framework/environment-specific. This
> leaves
> > >> > > > individual
> > >> > > > > > >> > > frameworks
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > as
> > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you can
> > get
> > >> > > simple
> > >> > > > > > >> stateless
> > >> > > > > > >> > > > > >> support in
> > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf
> > app
> > >> > > > > framework
> > >> > > > > > >> > > (Slider,
> > >> > > > > > >> > > > > >> > Marathon,
> > >> > > > > > >> > > > > >> > > etc). These are well known by people and
> > have
> > >> > nice
> > >> > > > UIs
> > >> > > > > > and a
> > >> > > > > > >> > lot
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > > flexibility. I don't think they have node
> > >> > affinity
> > >> > > > as a
> > >> > > > > > >> built
> > >> > > > > > >> > in
> > >> > > > > > >> > > > > >> option
> > >> > > > > > >> > > > > >> > > (though I could be wrong). So if we want
> > that
> > >> we
> > >> > > can
> > >> > > > > > either
> > >> > > > > > >> > wait
> > >> > > > > > >> > > > for
> > >> > > > > > >> > > > > >> them
> > >> > > > > > >> > > > > >> > > to add it or do a custom framework to add
> > that
> > >> > > > feature
> > >> > > > > > (as
> > >> > > > > > >> > now).
> > >> > > > > > >> > > > > >> > Obviously
> > >> > > > > > >> > > > > >> > > if you manage things with old-school ops
> > tools
> > >> > > > > > >> > (puppet/chef/etc)
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> get
> > >> > > > > > >> > > > > >> > > locality easily. The nice thing, though,
> is
> > >> that
> > >> > > all
> > >> > > > > the
> > >> > > > > > >> samza
> > >> > > > > > >> > > > > >> "business
> > >> > > > > > >> > > > > >> > > logic" around partition management and
> fault
> > >> > > > tolerance
> > >> > > > > > is in
> > >> > > > > > >> > > Samza
> > >> > > > > > >> > > > > >> core
> > >> > > > > > >> > > > > >> > so
> > >> > > > > > >> > > > > >> > > it is shared across frameworks and the
> > >> framework
> > >> > > > > specific
> > >> > > > > > >> bit
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > just
> > >> > > > > > >> > > > > >> > > whether it is smart enough to try to get
> the
> > >> same
> > >> > > > host
> > >> > > > > > when
> > >> > > > > > >> a
> > >> > > > > > >> > > job
> > >> > > > > > >> > > > is
> > >> > > > > > >> > > > > >> > > restarted.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah
> I
> > >> think
> > >> > > the
> > >> > > > > > goal
> > >> > > > > > >> > would
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > >> (a)
> > >> > > > > > >> > > > > >> > > actually get better alignment in user
> > >> experience,
> > >> > > and
> > >> > > > > (b)
> > >> > > > > > >> > > express
> > >> > > > > > >> > > > > >> this in
> > >> > > > > > >> > > > > >> > > the naming and project branding.
> > Specifically:
> > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> > >> > > > > > "transformation"
> > >> > > > > > >> api
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e.
> be
> > >> able
> > >> > > to
> > >> > > > > > explain
> > >> > > > > > >> > > when
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> use
> > >> > > > > > >> > > > > >> > > the consumer and when to use the stream
> > >> > processing
> > >> > > > > > >> > functionality
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> lead
> > >> > > > > > >> > > > > >> > > people into that experience.
> > >> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza
> 1.4.2
> > >> (or
> > >> > > > > > whatever)
> > >> > > > > > >> > that
> > >> > > > > > >> > > > has
> > >> > > > > > >> > > > > >> both
> > >> > > > > > >> > > > > >> > > Kafka and the stream processing part and
> > they
> > >> > > > actually
> > >> > > > > > work
> > >> > > > > > >> > > > > together.
> > >> > > > > > >> > > > > >> > > 3. Unify the programming experience so the
> > >> client
> > >> > > and
> > >> > > > > > Samza
> > >> > > > > > >> > api
> > >> > > > > > >> > > > > share
> > >> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> > committers
> > >> and
> > >> > > can
> > >> > > > > > have a
> > >> > > > > > >> > > > > separate
> > >> > > > > > >> > > > > >> > repo,
> > >> > > > > > >> > > > > >> > > but I'm actually not really sure (I can't
> > >> find a
> > >> > > > > > definition
> > >> > > > > > >> > of a
> > >> > > > > > >> > > > > >> > subproject
> > >> > > > > > >> > > > > >> > > in Apache).
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > Basically at a high-level you want the
> > >> experience
> > >> > > to
> > >> > > > > > "feel"
> > >> > > > > > >> > > like a
> > >> > > > > > >> > > > > >> single
> > >> > > > > > >> > > > > >> > > system, not to relatively independent
> things
> > >> that
> > >> > > are
> > >> > > > > > kind
> > >> > > > > > >> of
> > >> > > > > > >> > > > > >> awkwardly
> > >> > > > > > >> > > > > >> > > glued together.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > I think if we did that they having naming
> or
> > >> > > branding
> > >> > > > > > like
> > >> > > > > > >> > > "kafka
> > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or something
> > >> like
> > >> > > that
> > >> > > > > > would
> > >> > > > > > >> > > > actually
> > >> > > > > > >> > > > > >> do a
> > >> > > > > > >> > > > > >> > > good job of conveying what it is. I do
> that
> > >> this
> > >> > > > would
> > >> > > > > > help
> > >> > > > > > >> > > > adoption
> > >> > > > > > >> > > > > >> > quite
> > >> > > > > > >> > > > > >> > > a lot as it would correctly convey that
> > using
> > >> > Kafka
> > >> > > > > > >> Streaming
> > >> > > > > > >> > > with
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > is
> > >> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
> > >> pretty
> > >> > > > > heavily
> > >> > > > > > >> > adopted
> > >> > > > > > >> > > > at
> > >> > > > > > >> > > > > >> this
> > >> > > > > > >> > > > > >> > > point.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> > >> originally
> > >> > > > when
> > >> > > > > > open
> > >> > > > > > >> > > > sourcing
> > >> > > > > > >> > > > > >> > Samza,
> > >> > > > > > >> > > > > >> > > however at that time Kafka was relatively
> > >> unknown
> > >> > > and
> > >> > > > > we
> > >> > > > > > >> > decided
> > >> > > > > > >> > > > not
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> > do
> > >> > > > > > >> > > > > >> > > it since we felt it would be limiting.
> From
> > my
> > >> > > point
> > >> > > > of
> > >> > > > > > view
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > three
> > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now
> really
> > >> > heavily
> > >> > > > > used
> > >> > > > > > for
> > >> > > > > > >> > > > stream
> > >> > > > > > >> > > > > >> > > processing, (2) we learned that
> abstracting
> > >> out
> > >> > the
> > >> > > > > > stream
> > >> > > > > > >> > well
> > >> > > > > > >> > > is
> > >> > > > > > >> > > > > >> > > basically impossible, (3) we learned it is
> > >> really
> > >> > > > hard
> > >> > > > > to
> > >> > > > > > >> keep
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > > two
> > >> > > > > > >> > > > > >> > > things feeling like a single product.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > -Jay
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> > >> Kleppmann
> > >> > <
> > >> > > > > > >> > > > > >> martin@kleppmann.com>
> > >> > > > > > >> > > > > >> > > wrote:
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > >> Hi all,
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> I agree with the general philosophy of
> > tying
> > >> > Samza
> > >> > > > > more
> > >> > > > > > >> > firmly
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > > >> Kafka.
> > >> > > > > > >> > > > > >> > >> After I spent a while looking at
> > integrating
> > >> > other
> > >> > > > > > message
> > >> > > > > > >> > > > brokers
> > >> > > > > > >> > > > > >> (e.g.
> > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to
> the
> > >> > > > conclusion
> > >> > > > > > that
> > >> > > > > > >> > > > > >> > SystemConsumer
> > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
> > Kafka's
> > >> > that
> > >> > > > > pretty
> > >> > > > > > >> much
> > >> > > > > > >> > > > > nobody
> > >> > > > > > >> > > > > >> but
> > >> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
> > >> > perhaps
> > >> > > an
> > >> > > > > > >> > exception,
> > >> > > > > > >> > > > but
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.)
> > Thus,
> > >> > > making
> > >> > > > > > Samza
> > >> > > > > > >> > > fully
> > >> > > > > > >> > > > > >> > dependent
> > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > >> > system-independence
> > >> > > > was
> > >> > > > > > >> never
> > >> > > > > > >> > as
> > >> > > > > > >> > > > > real
> > >> > > > > > >> > > > > >> as
> > >> > > > > > >> > > > > >> > we
> > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of
> > code
> > >> > reuse
> > >> > > > are
> > >> > > > > > >> real.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN
> has
> > >> also
> > >> > > > always
> > >> > > > > > been
> > >> > > > > > >> > > > > >> appealing to
> > >> > > > > > >> > > > > >> > >> me, for various reasons already mentioned
> > in
> > >> > this
> > >> > > > > > thread.
> > >> > > > > > >> > > > Although
> > >> > > > > > >> > > > > >> > making
> > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > >> > > > (YARN/Mesos/AWS/etc)
> > >> > > > > > >> seems
> > >> > > > > > >> > > > > >> laudable,
> > >> > > > > > >> > > > > >> > I am
> > >> > > > > > >> > > > > >> > >> a little concerned that it will restrict
> us
> > >> to a
> > >> > > > > lowest
> > >> > > > > > >> > common
> > >> > > > > > >> > > > > >> > denominator.
> > >> > > > > > >> > > > > >> > >> For example, would host affinity
> > (SAMZA-617)
> > >> > still
> > >> > > > be
> > >> > > > > > >> > possible?
> > >> > > > > > >> > > > For
> > >> > > > > > >> > > > > >> jobs
> > >> > > > > > >> > > > > >> > >> with large amounts of state, I think
> > >> SAMZA-617
> > >> > > would
> > >> > > > > be
> > >> > > > > > a
> > >> > > > > > >> big
> > >> > > > > > >> > > > boon,
> > >> > > > > > >> > > > > >> > since
> > >> > > > > > >> > > > > >> > >> restoring state off the changelog on
> every
> > >> > single
> > >> > > > > > restart
> > >> > > > > > >> is
> > >> > > > > > >> > > > > painful,
> > >> > > > > > >> > > > > >> > due
> > >> > > > > > >> > > > > >> > >> to long recovery times. It would be a
> shame
> > >> if
> > >> > the
> > >> > > > > > >> decoupling
> > >> > > > > > >> > > > from
> > >> > > > > > >> > > > > >> YARN
> > >> > > > > > >> > > > > >> > >> made host affinity impossible.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Jay, a question about the proposed API
> for
> > >> > > > > > instantiating a
> > >> > > > > > >> > job
> > >> > > > > > >> > > in
> > >> > > > > > >> > > > > >> code
> > >> > > > > > >> > > > > >> > >> (rather than a properties file): when
> > >> > submitting a
> > >> > > > job
> > >> > > > > > to a
> > >> > > > > > >> > > > > cluster,
> > >> > > > > > >> > > > > >> is
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >> idea that the instantiation code runs on
> a
> > >> > client
> > >> > > > > > >> somewhere,
> > >> > > > > > >> > > > which
> > >> > > > > > >> > > > > >> then
> > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > >> > > YARN/Mesos/AWS/etc?
> > >> > > > > Or
> > >> > > > > > >> does
> > >> > > > > > >> > > that
> > >> > > > > > >> > > > > >> code
> > >> > > > > > >> > > > > >> > run
> > >> > > > > > >> > > > > >> > >> on each container that is part of the job
> > (in
> > >> > > which
> > >> > > > > > case,
> > >> > > > > > >> how
> > >> > > > > > >> > > > does
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > job
> > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
> > >> right to
> > >> > > > make
> > >> > > > > a
> > >> > > > > > 1.0
> > >> > > > > > >> > > > release
> > >> > > > > > >> > > > > >> > with a
> > >> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete.
> So
> > if
> > >> > this
> > >> > > > is
> > >> > > > > > going
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> happen, I
> > >> > > > > > >> > > > > >> > >> think it would be more honest to stick
> with
> > >> 0.*
> > >> > > > > version
> > >> > > > > > >> > numbers
> > >> > > > > > >> > > > > until
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >> library-ified Samza has been implemented,
> > is
> > >> > > stable
> > >> > > > > and
> > >> > > > > > >> > widely
> > >> > > > > > >> > > > > used.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
> > >> Kafka?
> > >> > > There
> > >> > > > > is
> > >> > > > > > >> > > precedent
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > >> tight coupling between different Apache
> > >> projects
> > >> > > > (e.g.
> > >> > > > > > >> > Curator
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I
> think
> > >> > > remaining
> > >> > > > > > >> separate
> > >> > > > > > >> > > > would
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > ok.
> > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on
> Kafka,
> > >> there
> > >> > > is
> > >> > > > > > enough
> > >> > > > > > >> > > > > substance
> > >> > > > > > >> > > > > >> in
> > >> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> > >> project.
> > >> > > An
> > >> > > > > > >> argument
> > >> > > > > > >> > in
> > >> > > > > > >> > > > > >> favour
> > >> > > > > > >> > > > > >> > of
> > >> > > > > > >> > > > > >> > >> merging would be if we think Kafka has a
> > much
> > >> > > > stronger
> > >> > > > > > >> "brand
> > >> > > > > > >> > > > > >> presence"
> > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one.
> If
> > >> the
> > >> > > Kafka
> > >> > > > > > >> project
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > >> willing
> > >> > > > > > >> > > > > >> > to
> > >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of
> > doing
> > >> > > > stateful
> > >> > > > > > >> stream
> > >> > > > > > >> > > > > >> > >> transformations, that would probably have
> > >> much
> > >> > the
> > >> > > > > same
> > >> > > > > > >> > effect
> > >> > > > > > >> > > as
> > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> > >> Processors"
> > >> > or
> > >> > > > > > suchlike.
> > >> > > > > > >> > > Close
> > >> > > > > > >> > > > > >> > >> collaboration between the two projects
> will
> > >> be
> > >> > > > needed
> > >> > > > > in
> > >> > > > > > >> any
> > >> > > > > > >> > > > case.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> From a project management perspective, I
> > >> guess
> > >> > the
> > >> > > > > "new
> > >> > > > > > >> > Samza"
> > >> > > > > > >> > > > > would
> > >> > > > > > >> > > > > >> > have
> > >> > > > > > >> > > > > >> > >> to be developed on a branch alongside
> > ongoing
> > >> > > > > > maintenance
> > >> > > > > > >> of
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > > >> current
> > >> > > > > > >> > > > > >> > >> line of development? I think it would be
> > >> > important
> > >> > > > to
> > >> > > > > > >> > continue
> > >> > > > > > >> > > > > >> > supporting
> > >> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> > >> migration
> > >> > > > path
> > >> > > > > to
> > >> > > > > > >> the
> > >> > > > > > >> > > new
> > >> > > > > > >> > > > > >> > version.
> > >> > > > > > >> > > > > >> > >> Leaving the current versions unsupported
> > and
> > >> > > forcing
> > >> > > > > > people
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> rewrite
> > >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Best,
> > >> > > > > > >> > > > > >> > >> Martin
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > >> > > > jay@confluent.io>
> > >> > > > > > >> wrote:
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >>> Hey Garry,
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be
> > happy
> > >> to
> > >> > > chat
> > >> > > > > > more
> > >> > > > > > >> > about
> > >> > > > > > >> > > > > this
> > >> > > > > > >> > > > > >> if
> > >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
> > >> > started
> > >> > > > with
> > >> > > > > > the
> > >> > > > > > >> > idea
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> "what
> > >> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> > >> > ingestion
> > >> > > > > tool"
> > >> > > > > > but
> > >> > > > > > >> > > > > >> ultimately
> > >> > > > > > >> > > > > >> > we
> > >> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> > >> ingestion
> > >> > > and
> > >> > > > > > >> > > > transformation
> > >> > > > > > >> > > > > >> had
> > >> > > > > > >> > > > > >> > >>> pretty different needs and coupling the
> > two
> > >> > made
> > >> > > > > things
> > >> > > > > > >> > hard.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
> > (KIP-26)
> > >> > > > actually
> > >> > > > > > will
> > >> > > > > > >> > do
> > >> > > > > > >> > > > what
> > >> > > > > > >> > > > > >> you
> > >> > > > > > >> > > > > >> > >> are
> > >> > > > > > >> > > > > >> > >>> looking for.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> With regard to your point about slider,
> I
> > >> don't
> > >> > > > > > >> necessarily
> > >> > > > > > >> > > > > >> disagree.
> > >> > > > > > >> > > > > >> > >> But I
> > >> > > > > > >> > > > > >> > >>> think getting good YARN support is quite
> > >> doable
> > >> > > > and I
> > >> > > > > > >> think
> > >> > > > > > >> > we
> > >> > > > > > >> > > > can
> > >> > > > > > >> > > > > >> make
> > >> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> > >> proposal
> > >> > > > > solves
> > >> > > > > > is
> > >> > > > > > >> > that
> > >> > > > > > >> > > > > >> > >> technically
> > >> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
> > >> cluster
> > >> > > > > > management
> > >> > > > > > >> > > systems
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > way
> > >> > > > > > >> > > > > >> > >>> things are now, you need to write an
> "app
> > >> > master"
> > >> > > > or
> > >> > > > > > >> > > "framework"
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > each
> > >> > > > > > >> > > > > >> > >>> and they are all a little different so
> > >> testing
> > >> > is
> > >> > > > > > really
> > >> > > > > > >> > hard.
> > >> > > > > > >> > > > In
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > >>> absence of this we have been stuck with
> > just
> > >> > YARN
> > >> > > > > which
> > >> > > > > > >> has
> > >> > > > > > >> > > > > >> fantastic
> > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the
> > org,
> > >> but
> > >> > > > zero
> > >> > > > > > >> > > penetration
> > >> > > > > > >> > > > > >> > >> elsewhere.
> > >> > > > > > >> > > > > >> > >>> Given the huge amount of work being put
> in
> > >> to
> > >> > > > slider,
> > >> > > > > > >> > > marathon,
> > >> > > > > > >> > > > > aws
> > >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
> > related
> > >> > > > packaging
> > >> > > > > > >> > > > technologies
> > >> > > > > > >> > > > > >> > people
> > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> > >> > > > > cloud-specific
> > >> > > > > > >> > deploy
> > >> > > > > > >> > > > > >> tools,
> > >> > > > > > >> > > > > >> > >> etc)
> > >> > > > > > >> > > > > >> > >>> I really think it is important to get
> this
> > >> > right.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> -Jay
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> > >> > Turkington
> > >> > > <
> > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>>> Hi all,
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> I think the question below re does
> Samza
> > >> > become
> > >> > > a
> > >> > > > > > >> > sub-project
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>> highlights the broader point around
> > >> migration.
> > >> > > > Chris
> > >> > > > > > >> > mentions
> > >> > > > > > >> > > > > >> Samza's
> > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1
> release
> > >> but
> > >> > I'm
> > >> > > > not
> > >> > > > > > sure
> > >> > > > > > >> > it
> > >> > > > > > >> > > > > feels
> > >> > > > > > >> > > > > >> > >> right to
> > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> > >> deprecate
> > >> > > > most
> > >> > > > > of
> > >> > > > > > >> it.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have some
> > guys
> > >> > who
> > >> > > > have
> > >> > > > > > >> > started
> > >> > > > > > >> > > > > >> working
> > >> > > > > > >> > > > > >> > >> with
> > >> > > > > > >> > > > > >> > >>>> Samza and building some new
> > >> > consumers/producers
> > >> > > > was
> > >> > > > > > next
> > >> > > > > > >> > up.
> > >> > > > > > >> > > > > Sounds
> > >> > > > > > >> > > > > >> > like
> > >> > > > > > >> > > > > >> > >>>> that is absolutely not the direction to
> > >> go. I
> > >> > > need
> > >> > > > > to
> > >> > > > > > >> look
> > >> > > > > > >> > > into
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > KIP
> > >> > > > > > >> > > > > >> > >> in
> > >> > > > > > >> > > > > >> > >>>> more detail but for me the
> attractiveness
> > >> of
> > >> > > > adding
> > >> > > > > > new
> > >> > > > > > >> > Samza
> > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all
> > they
> > >> > were
> > >> > > > > doing
> > >> > > > > > was
> > >> > > > > > >> > > > really
> > >> > > > > > >> > > > > >> > getting
> > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to
> > avoid
> > >> > > > having
> > >> > > > > to
> > >> > > > > > >> > worry
> > >> > > > > > >> > > > > about
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > >>>> lifecycle management of external
> clients.
> > >> If
> > >> > > there
> > >> > > > > is
> > >> > > > > > a
> > >> > > > > > >> > > generic
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a
> > new
> > >> > > > connector
> > >> > > > > > into
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > have
> > >> > > > > > >> > > > > >> a
> > >> > > > > > >> > > > > >> > >> lot of
> > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> > reliability
> > >> > done
> > >> > > > for
> > >> > > > > me
> > >> > > > > > >> then
> > >> > > > > > >> > > it
> > >> > > > > > >> > > > > >> gives
> > >> > > > > > >> > > > > >> > me
> > >> > > > > > >> > > > > >> > >> all
> > >> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers
> > would.
> > >> If
> > >> > > not
> > >> > > > > > then it
> > >> > > > > > >> > > > > >> complicates
> > >> > > > > > >> > > > > >> > my
> > >> > > > > > >> > > > > >> > >>>> operational deployments.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Which is similar to my other question
> > with
> > >> the
> > >> > > > > > proposal
> > >> > > > > > >> --
> > >> > > > > > >> > if
> > >> > > > > > >> > > > we
> > >> > > > > > >> > > > > >> > build a
> > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus
> > the
> > >> > > > requisite
> > >> > > > > > >> shims
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> > integrate
> > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former
> may
> > >> be a
> > >> > > lot
> > >> > > > > more
> > >> > > > > > >> work
> > >> > > > > > >> > > > than
> > >> > > > > > >> > > > > we
> > >> > > > > > >> > > > > >> > >> think.
> > >> > > > > > >> > > > > >> > >>>> We may make it much easier for a
> newcomer
> > >> to
> > >> > get
> > >> > > > > > >> something
> > >> > > > > > >> > > > > running
> > >> > > > > > >> > > > > >> but
> > >> > > > > > >> > > > > >> > >>>> having them step up and get a reliable
> > >> > > production
> > >> > > > > > >> > deployment
> > >> > > > > > >> > > > may
> > >> > > > > > >> > > > > >> still
> > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
> > >> > different
> > >> > > > > > reasons
> > >> > > > > > >> > than
> > >> > > > > > >> > > > > >> today.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable
> > with
> > >> > > making
> > >> > > > > the
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > > >> dependency
> > >> > > > > > >> > > > > >> > >> on
> > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
> absolutely
> > >> see
> > >> > > the
> > >> > > > > > >> benefits
> > >> > > > > > >> > > in
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > >> > > > > > >> > > > terminologies/abstractions
> > >> > > > > > >> > > > > >> that
> > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library
> > >> would
> > >> > > > likely
> > >> > > > > > be a
> > >> > > > > > >> > very
> > >> > > > > > >> > > > > nice
> > >> > > > > > >> > > > > >> > tool
> > >> > > > > > >> > > > > >> > >> to
> > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have
> > the
> > >> > > > concerns
> > >> > > > > > >> above
> > >> > > > > > >> > re
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > >>>> operational side.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Garry
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> > >> [mailto:
> > >> > > > > > >> > gdfm@apache.org
> > >> > > > > > >> > > ]
> > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations
> on
> > >> > Samza
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > >> > > > > > >> > > > > >> > >>>> From outside, I have always perceived
> > Samza
> > >> > as a
> > >> > > > > > >> computing
> > >> > > > > > >> > > > layer
> > >> > > > > > >> > > > > >> over
> > >> > > > > > >> > > > > >> > >>>> Kafka.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit provocative,
> is
> > >> > > "should
> > >> > > > > > Samza
> > >> > > > > > >> be
> > >> > > > > > >> > a
> > >> > > > > > >> > > > > >> > sub-project
> > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> > >> separate
> > >> > > > > project
> > >> > > > > > >> > with a
> > >> > > > > > >> > > > > >> separate
> > >> > > > > > >> > > > > >> > >>>> governance?
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Cheers,
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> --
> > >> > > > > > >> > > > > >> > >>>> Gianmarco
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > >> > > > > > yanfang724@gmail.com>
> > >> > > > > > >> > > > wrote:
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka
> > more
> > >> > > > tightly.
> > >> > > > > > >> > Because
> > >> > > > > > >> > > > > Samza
> > >> > > > > > >> > > > > >> de
> > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
> > >> > leverage
> > >> > > > > what
> > >> > > > > > >> Kafka
> > >> > > > > > >> > > > has.
> > >> > > > > > >> > > > > At
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
> > reinvent
> > >> > what
> > >> > > > > Samza
> > >> > > > > > >> > > already
> > >> > > > > > >> > > > > >> has. I
> > >> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> > >> > ingestion
> > >> > > > and
> > >> > > > > > >> > > > > transformation.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
> > >> image
> > >> > > how
> > >> > > > > the
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > will
> > >> > > > > > >> > > > > >> look
> > >> > > > > > >> > > > > >> > >>>> like.
> > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> > >> > > difference
> > >> > > > > in
> > >> > > > > > >> terms
> > >> > > > > > >> > > of
> > >> > > > > > >> > > > > how
> > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code
> > >> shows
> > >> > (A
> > >> > > > > > client of
> > >> > > > > > >> > > > Kakfa)
> > >> > > > > > >> > > > > ?
> > >> > > > > > >> > > > > >> And
> > >> > > > > > >> > > > > >> > >>>>> user's application code calls this
> > client?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of
> > Kafka
> > >> > (like
> > >> > > > > what
> > >> > > > > > the
> > >> > > > > > >> > > code
> > >> > > > > > >> > > > > >> shows),
> > >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > >> > > > > fault-tolerance?
> > >> > > > > > >> Are
> > >> > > > > > >> > > they
> > >> > > > > > >> > > > > >> taken
> > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> > >> mechanism,
> > >> > > such
> > >> > > > > as
> > >> > > > > > >> > "Samza
> > >> > > > > > >> > > > > >> worker"
> > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 2. What about other features, such as
> > >> > > > auto-scaling,
> > >> > > > > > >> shared
> > >> > > > > > >> > > > > state,
> > >> > > > > > >> > > > > >> > >>>>> monitoring?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is
> > this
> > >> > what
> > >> > > > > Chris
> > >> > > > > > >> > > > suggests?)
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from
> > Kakfa
> > >> > and
> > >> > > > > > produce
> > >> > > > > > >> to
> > >> > > > > > >> > > it.
> > >> > > > > > >> > > > > >> Then it
> > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks
> > like
> > >> > now,
> > >> > > > > > except it
> > >> > > > > > >> > > does
> > >> > > > > > >> > > > > not
> > >> > > > > > >> > > > > >> > rely
> > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> > >> leverage
> > >> > > > Kafka's
> > >> > > > > > >> > metrics,
> > >> > > > > > >> > > > > logs,
> > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> Thanks,
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM,
> Guozhang
> > >> > Wang <
> > >> > > > > > >> > > > > wangguoz@gmail.com
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > >>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Read through the code example and it
> > >> looks
> > >> > > good
> > >> > > > to
> > >> > > > > > me.
> > >> > > > > > >> A
> > >> > > > > > >> > > few
> > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> > >> runnable
> > >> > > like:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > >> > > --config-factory=...
> > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> > deploying
> > >> > Samza
> > >> > > > > more
> > >> > > > > > as
> > >> > > > > > >> > > > embedded
> > >> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> > >> (ignoring
> > >> > > the
> > >> > > > > > >> > > terminology
> > >> > > > > > >> > > > > >> since
> > >> > > > > > >> > > > > >> > >>>>>> it is not the
> > >> > > > > > >> > > > > >> > >>>>> same
> > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> > >> MyStreamTask(configs);
> > >> > > > > Thread
> > >> > > > > > >> > thread
> > >> > > > > > >> > > =
> > >> > > > > > >> > > > > new
> > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment
> modes
> > >> are
> > >> > > > > important
> > >> > > > > > >> for
> > >> > > > > > >> > > > > >> different
> > >> > > > > > >> > > > > >> > >>>>>> types
> > >> > > > > > >> > > > > >> > >>>>> of
> > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making
> Samza
> > >> > purely
> > >> > > > > > >> standalone
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > still
> > >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or
> > library
> > >> > > modes.
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Guozhang
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
> > >> Kreps
> > >> > <
> > >> > > > > > >> > > > jay@confluent.io>
> > >> > > > > > >> > > > > >> > wrote:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> > >> example,
> > >> > it
> > >> > > > was
> > >> > > > > > >> > supposed
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> look
> > >> > > > > > >> > > > > >> > >>>>>>> like
> > >> > > > > > >> > > > > >> > >>>>>>> this:
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> > >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > >> > > > "localhost:4242");
> > >> > > > > > >> > > > > >> StreamingConfig
> > >> > > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > > > "test-topic-2");
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > config.processor(ExampleStreamProcessor.class);
> > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > >> > StringSerializer(),
> > >> > > > new
> > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
> KafkaStreaming
> > >> > > > container =
> > >> > > > > > new
> > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> > container.run();
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> -Jay
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM,
> Jay
> > >> > Kreps <
> > >> > > > > > >> > > > jay@confluent.io
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >> > >>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
> > >> Chris
> > >> > > and
> > >> > > > I
> > >> > > > > > were
> > >> > > > > > >> > > having
> > >> > > > > > >> > > > > >> > >>>>>>>> around
> > >> > > > > > >> > > > > >> > >>>>>>> whether
> > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza
> as a
> > >> kind
> > >> > > of
> > >> > > > > data
> > >> > > > > > >> > > > ingestion
> > >> > > > > > >> > > > > >> > >>>>> framework
> > >> > > > > > >> > > > > >> > >>>>>>> for
> > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to
> > KIP-26
> > >> > > > > "copycat").
> > >> > > > > > >> This
> > >> > > > > > >> > > > kind
> > >> > > > > > >> > > > > of
> > >> > > > > > >> > > > > >> > >>>>>> combined
> > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and
> > YARN
> > >> and
> > >> > > the
> > >> > > > > > >> > discussion
> > >> > > > > > >> > > > > >> around
> > >> > > > > > >> > > > > >> > >>>>>>>> how
> > >> > > > > > >> > > > > >> > >>>>> to
> > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was,
> given
> > >> that
> > >> > > > Samza
> > >> > > > > > was
> > >> > > > > > >> > > > basically
> > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific,
> what
> > if
> > >> > you
> > >> > > > just
> > >> > > > > > >> > embraced
> > >> > > > > > >> > > > > that
> > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > >> > > > > > >> > > > > >> > >>>>>> into
> > >> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> > >> > framework
> > >> > > > and
> > >> > > > > > more
> > >> > > > > > >> > > like a
> > >> > > > > > >> > > > > >> > >>>>>>>> third
> > >> > > > > > >> > > > > >> > >>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> > consumer"
> > >> > with
> > >> > > > > state
> > >> > > > > > >> > > > management
> > >> > > > > > >> > > > > >> > >>>>>> facilities.
> > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> > >> complex
> > >> > > > stream
> > >> > > > > > >> > > processing
> > >> > > > > > >> > > > > >> > >>>>>>>> framework
> > >> > > > > > >> > > > > >> > >>>>>>> this
> > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple
> > thing,
> > >> not
> > >> > > > much
> > >> > > > > > more
> > >> > > > > > >> > > > > >> complicated
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> use
> > >> > > > > > >> > > > > >> > >>>>>>> or
> > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As
> > Chris
> > >> > said
> > >> > > > we
> > >> > > > > > >> thought
> > >> > > > > > >> > > > about
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >>>>>>>> a
> > >> > > > > > >> > > > > >> > >>>>> lot
> > >> > > > > > >> > > > > >> > >>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> > >> > processing
> > >> > > > > > systems
> > >> > > > > > >> > were
> > >> > > > > > >> > > > > doing)
> > >> > > > > > >> > > > > >> > >>>>> seemed
> > >> > > > > > >> > > > > >> > >>>>>>> like
> > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output
> > >> data
> > >> > to
> > >> > > > and
> > >> > > > > > from
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > stream
> > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually
> > looked
> > >> > into
> > >> > > > how
> > >> > > > > > that
> > >> > > > > > >> > > would
> > >> > > > > > >> > > > > >> > >>>>>>>> work,
> > >> > > > > > >> > > > > >> > >>>>> Samza
> > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
> ingestion
> > >> > > framework
> > >> > > > > > for a
> > >> > > > > > >> > > bunch
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>> reasons.
> > >> > > > > > >> > > > > >> > >>>>>> To
> > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a
> > pretty
> > >> > > > different
> > >> > > > > > >> > internal
> > >> > > > > > >> > > > > data
> > >> > > > > > >> > > > > >> > >>>>>>>> model
> > >> > > > > > >> > > > > >> > >>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split
> > them
> > >> and
> > >> > > had
> > >> > > > > an
> > >> > > > > > api
> > >> > > > > > >> > for
> > >> > > > > > >> > > > > Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26)
> > >> and a
> > >> > > > > separate
> > >> > > > > > >> api
> > >> > > > > > >> > > for
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> > embracing
> > >> the
> > >> > > > same
> > >> > > > > > >> > > terminology
> > >> > > > > > >> > > > > and
> > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about
> the
> > >> > current
> > >> > > > > > state is
> > >> > > > > > >> > > that
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>> two
> > >> > > > > > >> > > > > >> > >>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology
> > >> like
> > >> > > > > "stream"
> > >> > > > > > vs
> > >> > > > > > >> > > > "topic"
> > >> > > > > > >> > > > > >> and
> > >> > > > > > >> > > > > >> > >>>>>>> different
> > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means
> > you
> > >> > kind
> > >> > > > of
> > >> > > > > > have
> > >> > > > > > >> to
> > >> > > > > > >> > > > learn
> > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > >> > > > > > >> > > > > >> > >>>>>>> way,
> > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
> different
> > >> way,
> > >> > > > then
> > >> > > > > > kind
> > >> > > > > > >> of
> > >> > > > > > >> > > > > >> > >>>>>>>> understand
> > >> > > > > > >> > > > > >> > >>>>> how
> > >> > > > > > >> > > > > >> > >>>>>>> they
> > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having
> > walked
> > >> a
> > >> > few
> > >> > > > > > people
> > >> > > > > > >> > > through
> > >> > > > > > >> > > > > >> this
> > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to
> > >> get.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of
> > >> time
> > >> > on
> > >> > > > > > >> airplanes I
> > >> > > > > > >> > > > > hacked
> > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> > >> incomplete
> > >> > > > > > prototype
> > >> > > > > > >> of
> > >> > > > > > >> > > > what
> > >> > > > > > >> > > > > >> > >>>>>>>> this would
> > >> > > > > > >> > > > > >> > >>>>> look
> > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
> > >> dumped
> > >> > > into
> > >> > > > > > Kafka
> > >> > > > > > >> as
> > >> > > > > > >> > > it
> > >> > > > > > >> > > > > >> > >>>>>>>> required a
> > >> > > > > > >> > > > > >> > >>>>>> few
> > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here
> is
> > >> the
> > >> > > code:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > >
> > >> > > > > > >> >
> > >> > > > > >
> > >> > >
> > >>
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I
> > just
> > >> > > > > liberally
> > >> > > > > > >> > renamed
> > >> > > > > > >> > > > > >> > >>>>>>>> everything
> > >> > > > > > >> > > > > >> > >>>>> to
> > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
> > >> regard
> > >> > > for
> > >> > > > > > >> > > > compatibility.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something like
> > >> this:
> > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
> Properties();
> > >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > >> > > > > "localhost:4242");
> > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > >> > > > > > >> > > > > >>
> config.processor(ExampleStreamProcessor.class);
> > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > >> > > StringDeserializer());
> > >> > > > > > >> > > > KafkaStreaming
> > >> > > > > > >> > > > > >> > >>>>>> container =
> > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > >> > container.run();
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > >> > > > SamzaContainer;
> > >> > > > > > >> > > > > StreamProcessor
> > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the
> class
> > >> names
> > >> > > in
> > >> > > > a
> > >> > > > > > file
> > >> > > > > > >> > and
> > >> > > > > > >> > > > then
> > >> > > > > > >> > > > > >> > >>>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you
> just
> > >> > > > > instantiate
> > >> > > > > > the
> > >> > > > > > >> > > > > container
> > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced
> > over
> > >> > > > however
> > >> > > > > > many
> > >> > > > > > >> > > > > instances
> > >> > > > > > >> > > > > >> > >>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>> this
> > >> > > > > > >> > > > > >> > >>>>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> > instance
> > >> > dies,
> > >> > > > new
> > >> > > > > > >> tasks
> > >> > > > > > >> > > are
> > >> > > > > > >> > > > > >> added
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
> shutting
> > >> them
> > >> > > > down).
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
> > running
> > >> > this
> > >> > > > > stuff
> > >> > > > > > in
> > >> > > > > > >> > YARN
> > >> > > > > > >> > > > via
> > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
> > >> using
> > >> > > some
> > >> > > > > of
> > >> > > > > > >> their
> > >> > > > > > >> > > > tools
> > >> > > > > > >> > > > > >> > >>>>>>>> but from the
> > >> > > > > > >> > > > > >> > >>>>>> point
> > >> > > > > > >> > > > > >> > >>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these
> stream
> > >> > > > processing
> > >> > > > > > jobs
> > >> > > > > > >> > are
> > >> > > > > > >> > > > > just
> > >> > > > > > >> > > > > >> > >>>>>> stateless
> > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
> > >> expand
> > >> > and
> > >> > > > > > contract
> > >> > > > > > >> > at
> > >> > > > > > >> > > > > will.
> > >> > > > > > >> > > > > >> > >>>>>>>> There
> > >> > > > > > >> > > > > >> > >>>>> is
> > >> > > > > > >> > > > > >> > >>>>>>> no
> > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code,
> > it
> > >> > would
> > >> > > > get
> > >> > > > > > >> larger
> > >> > > > > > >> > > if
> > >> > > > > > >> > > > we
> > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
> > larger.
> > >> We
> > >> > > > really
> > >> > > > > > do
> > >> > > > > > >> > get a
> > >> > > > > > >> > > > ton
> > >> > > > > > >> > > > > >> > >>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>> leverage
> > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> > >> > delegated
> > >> > > to
> > >> > > > > the
> > >> > > > > > >> new
> > >> > > > > > >> > > > > >> consumer.
> > >> > > > > > >> > > > > >> > >>>>> This
> > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> > >> > management
> > >> > > > > > strategy
> > >> > > > > > >> > > > > available
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to
> Samza
> > >> (and
> > >> > > vice
> > >> > > > > > versa)
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > with
> > >> > > > > > >> > > > > >> > >>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>> exact
> > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as
> > state
> > >> > reuse
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
> > >> > thought
> > >> > > > > > >> provoking.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM,
> > Chris
> > >> > > > > Riccomini <
> > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with
> > Samza
> > >> > > > > engineers
> > >> > > > > > at
> > >> > > > > > >> > > > LinkedIn
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>> Confluent
> > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> > observations
> > >> > and
> > >> > > > > would
> > >> > > > > > >> like
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> > >>>>>>>>> propose
> > >> > > > > > >> > > > > >> > >>>>> some
> > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I
> > >> want to
> > >> > > > call
> > >> > > > > > out
> > >> > > > > > >> > about
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > >> > > > > > >> > > > > >> > >>>>>> design,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> > changes.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a
> dynamic
> > >> > > > deployment
> > >> > > > > > >> system.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> > >> SystemConsumer/SystemProducer
> > >> > and
> > >> > > > > > Kafka's
> > >> > > > > > >> > > > consumer
> > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > >> > > > > > >> > > > > >> > >>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
> > >> > problems.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
> > related,
> > >> > but
> > >> > > > I'll
> > >> > > > > > >> > address
> > >> > > > > > >> > > > them
> > >> > > > > > >> > > > > >> in
> > >> > > > > > >> > > > > >> > >>>>> order.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use
> > of a
> > >> > > > dynamic
> > >> > > > > > >> > > deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > >> > > > > > >> > > > > >> > >>>>>> such
> > >> > > > > > >> > > > > >> > >>>>>>>>> as
> > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
> initially
> > >> built
> > >> > > > > Samza,
> > >> > > > > > we
> > >> > > > > > >> > bet
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > >> > >>>>>>>>> there
> > >> > > > > > >> > > > > >> > >>>>>> would
> > >> > > > > > >> > > > > >> > >>>>>>>>> be
> > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area,
> and
> > >> we
> > >> > > could
> > >> > > > > > >> support
> > >> > > > > > >> > > > them,
> > >> > > > > > >> > > > > >> and
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>> rest
> > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there
> are
> > >> many
> > >> > > > > > >> variations.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > >> > > > > > >> > > > > >> > >>>>>> many
> > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start
> > >> their
> > >> > > > > > processors
> > >> > > > > > >> > like
> > >> > > > > > >> > > > > normal
> > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
> traditional
> > >> > > > deployment
> > >> > > > > > >> scripts
> > >> > > > > > >> > > > such
> > >> > > > > > >> > > > > as
> > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > >> > > > > > >> > > > > >> > >>>>>> Chef,
> > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
> > >> system
> > >> > > on
> > >> > > > > > users
> > >> > > > > > >> > makes
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
> > painful
> > >> for
> > >> > > > first
> > >> > > > > > time
> > >> > > > > > >> > > > users.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
> requirement
> > >> was
> > >> > > also
> > >> > > > a
> > >> > > > > > bit
> > >> > > > > > >> of
> > >> > > > > > >> > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > >> > > > > > >> > > > > >> > >>>>>> because
> > >> > > > > > >> > > > > >> > >>>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding
> > between
> > >> > the
> > >> > > > > > nature of
> > >> > > > > > >> > > batch
> > >> > > > > > >> > > > > >> jobs
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>> stream
> > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> > >> > > conscious
> > >> > > > > > effort
> > >> > > > > > >> to
> > >> > > > > > >> > > > favor
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>> Hadoop
> > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things,
> > >> since
> > >> > it
> > >> > > > > worked
> > >> > > > > > >> and
> > >> > > > > > >> > > was
> > >> > > > > > >> > > > > well
> > >> > > > > > >> > > > > >> > >>>>>>> understood.
> > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that
> > >> batch
> > >> > > jobs
> > >> > > > > > have a
> > >> > > > > > >> > > > definite
> > >> > > > > > >> > > > > >> > >>>>>> beginning,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs
> > don't
> > >> > > > > (usually).
> > >> > > > > > >> This
> > >> > > > > > >> > > > leads
> > >> > > > > > >> > > > > to
> > >> > > > > > >> > > > > >> > >>>>>>>>> a
> > >> > > > > > >> > > > > >> > >>>>> much
> > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for
> > stream
> > >> > > > > processors.
> > >> > > > > > >> You
> > >> > > > > > >> > > > > >> basically
> > >> > > > > > >> > > > > >> > >>>>>>>>> just
> > >> > > > > > >> > > > > >> > >>>>>>> need
> > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> > >> processor,
> > >> > and
> > >> > > > > start
> > >> > > > > > >> it.
> > >> > > > > > >> > > The
> > >> > > > > > >> > > > > way
> > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's
> > no
> > >> > > concept
> > >> > > > > of
> > >> > > > > > a
> > >> > > > > > >> > > cluster
> > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > >> > > > > > >> > > > > >> > >>>>>> add
> > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> > >> coupling
> > >> > > > Samza
> > >> > > > > > with
> > >> > > > > > >> a
> > >> > > > > > >> > > > > >> scheduler
> > >> > > > > > >> > > > > >> > >>>>>>>>> is
> > >> > > > > > >> > > > > >> > >>>>>> that
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
> > >> handle
> > >> > > > > > deployment.
> > >> > > > > > >> > > This
> > >> > > > > > >> > > > > >> pulls
> > >> > > > > > >> > > > > >> > >>>>>>>>> in a
> > >> > > > > > >> > > > > >> > >>>>>>> bunch
> > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> > >> > > distribution
> > >> > > > > > (config
> > >> > > > > > >> > > > > stream),
> > >> > > > > > >> > > > > >> > >>>>>>>>> shell
> > >> > > > > > >> > > > > >> > >>>>>>> scrips
> > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> > packaging
> > >> > (all
> > >> > > > the
> > >> > > > > > .tgz
> > >> > > > > > >> > > > stuff),
> > >> > > > > > >> > > > > >> etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring
> dynamic
> > >> > > > deployment
> > >> > > > > > was
> > >> > > > > > >> to
> > >> > > > > > >> > > > > support
> > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> > >> > > locality,
> > >> > > > > you
> > >> > > > > > >> need
> > >> > > > > > >> > to
> > >> > > > > > >> > > > put
> > >> > > > > > >> > > > > >> > >>>>>>>>> your
> > >> > > > > > >> > > > > >> > >>>>>> processors
> > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> > processing.
> > >> > Upon
> > >> > > > > > further
> > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > >> > > > > > >> > > > > >> > >>>>>>> though,
> > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
> beneficial.
> > >> > There
> > >> > > is
> > >> > > > > > some
> > >> > > > > > >> > good
> > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > >> > > > > > >> > > > > >> > >>>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
> SAMZA-335.
> > >> > Again,
> > >> > > we
> > >> > > > > > took
> > >> > > > > > >> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > >> > > > > > >> > > > > >> > >>>>>> path,
> > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> > differences
> > >> > > > between
> > >> > > > > > HDFS
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > >> > > > > > >> > > > > >> > >>>>>>
> ...
>
> [Message clipped]




-- 
-- Guozhang

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
Hi, all,


Since there wasn't any further discussion on this, I want to summarize this
from LinkedIn's point of view here.


First of all, thanks all and it has been a great discussion and I
personally learned a lot.   We surely had started off with a set of ideas
but all the discussion on this thread has surely shaped up a more practical
and less dramatic outcome. Looks like most people are in agreement with the
option proposed below.


Here I just want to state a few points in more concrete details from
LinkedIn’s point of view:


- We will continue to focus on "Stream processing as a service" with Samza.

   * We will continue the effort to support host-affinity, auto-scaling,
and other cluster-manager related features

   * We will refactor the code to ensure

      * Partition management is pluggable and separate from the deployment
of Samza process

      * Make Samza process deployment to be independent from any cluster
management system such as YARN (SAMZA-516)

- We will continue the development in user-facing new features s.t. SQL
support, windowing/join operators in Samza

- We will participate in the of Kafka Stream API initiative in Kafka.  Once
it is ready, we will work with the Samza community on leveraging it in
Samza.


Thanks a lot!

On Tue, Jul 14, 2015 at 8:56 AM, Dan <da...@gmail.com> wrote:

> A lot of what I've used Samza for so far is unifying the input and output
> of data with Kafka, that's one of the reasons I've added the
> ElasticsearchProducer which we use at State and have more we'd want to add
> there.
>
> As with Garry and others I didn't know about CopyCat before this discussion
> but it seems the Samza community has a lot of experience working on getting
> data reliability out of and into other systems. Removing
> SystemConsumer/Producer
> may fragment that community or make it less clear how to use Samza end to
> end. Would it make sense to develop CopyCat inside Samza? Probably not...
> but it would probably be good to build that out working with the Samza
> community first, then once that's in Kafka it would probably be far more
> clear how we could remove the SystemConsumer/Producer from Samza showing
> the range of intput/export that Kafka provides.
>
> As a side point, we don't use Samza for input as Martin has pointed out it
> doesn't really fit with consuming sources like databases (MongoDB in our
> case) so we've written our own custom consumers that we'd happily work into
> a tool like CopyCat. There hasn't really been much discussion like this in
> the Kafka community and I'm not sure we'd have known about this as the
> Kafka project is seen as being lightweight from our point of view (and I
> think most people externally). Is Kafka the best project / community to
> start CopyCat? it does have a good momentum behind it but that can be
> directed at other Apache projects with marketing.
>
>  - Dan
>
>
> On 14 July 2015 at 01:33, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey Yi,
> >
> > Cool.
> >
> > The Kafka design process is very open. We usually do a very detailed wiki
> > along with any code and do a pretty extensive discussion around these.
> > Details here:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >
> > It's wide open and tons of people participate. If we went down this route
> > the next step would be to discuss there.
> >
> > For coding it's normal Apache style, JIRAs, RBC, etc.
> >
> > With respect to state, you definitely need the core support in the API as
> > that has implications all the way down. I suspect you need at least some
> > key-value store impl to ship with the client so you have one plugin to
> test
> > against and use out of the box. But the whole point of that stuff is it
> is
> > pluggable and state is in any case recreated off the Kafka topic, so
> Samza
> > can have its own storage engines too. I don't think these need to be
> > maintained centrally at all.
> >
> > -Jay
> >
> >
> >
> > On Mon, Jul 13, 2015 at 3:23 PM, Yi Pan <ni...@gmail.com> wrote:
> >
> > > Hi, Jay,
> > >
> > > Given all the user concerns, the board disagreement on sub-projects, I
> am
> > > supporting your 5th option as well. As you said, even the end goal is
> the
> > > same, it might help to pave a smooth path forward. One thing I learned
> > over
> > > the years is that what we planned for may not be the final product, and
> > the
> > > unexpected product may be even better if we learn and adapt along the
> > way.
> > > :)
> > >
> > > So, since I assume that in option 5, Samza will fully embrace the new
> > Kafka
> > > Streams API as the core and heavily depends on it, I want to raise up
> > some
> > > detailed logistic questions:
> > > 1. How do Samza community contribute to the design and development of
> the
> > > new Kafka Streams API? As Kartik mentioned, if there is a model for
> Samza
> > > community to contribute to just this part of Kafka code base, it would
> > be a
> > > huge plus point to the integration.
> > > 2. What's the scope of the new Kafka Streams API? Is it just focused on
> > > message consumption, producing, Kafka-based partition distribution,
> > offset
> > > management, message selection and delivery to StreamProcessor? In other
> > > words, I have a question regarding to whether we should put
> > samza-kv-store
> > > in the scope? The reasons that I think that it might be better to stay
> in
> > > Samza initially are: a) KV-store libraries does not directly interact
> w/
> > > Kafka brokers, it optionally uses Kafka consumers and producers like a
> > > client program; b) there are a tons of experiments / tune-ups on
> RocksDB
> > > that we want to have a faster iteration on this library (e.g. there is
> an
> > > experimental time-sequence KV store implementation from LinkedIn we
> also
> > > want to experiment on in window operator in SQL). The down-side I can
> see
> > > is that w/o this in Kafka Streams API, the as-a-library mode may not
> get
> > > the state management support. If we can find a way to make sure that
> the
> > > current Samza community can contribute to this library in a faster
> > > velocity, I can be convinced otherwise as well. What's your opinion on
> > > this?
> > >
> > > Overall, thanks a lot for pushing forward the whole discussion!
> > >
> > > -Yi
> > >
> > > On Mon, Jul 13, 2015 at 12:56 PM, Garry Turkington <
> > > g.turkington@improvedigital.com> wrote:
> > >
> > > > Hi,
> > > >
> > > > I'm also supportive of Jay's option 5. There is a risk the
> "transformer
> > > > API" -- I'd have preferred Metamorphosis but it's too hard to type!
> --
> > > > takes on a life of its own and we end up with two very different
> things
> > > but
> > > > given how good the Kafka community has been at introducing new
> producer
> > > and
> > > > consumer clients and giving very clear guidance on when they are
> > > production
> > > > ready this is a danger I believe can be managed. It'd also be
> excellent
> > > to
> > > > get some working code to beat around the notions of stream processing
> > > atop
> > > > a system with transacdtional messages.
> > > >
> > > > On the question of whether to keep or deprecate
> > SystemConsumer/Producer I
> > > > believe we need get a better understanding over the next while of
> just
> > > what
> > > > the Samza community is looking for in such connectivity. For my own
> use
> > > > cases I have been looking to add additional implementations primarily
> > to
> > > > use Samza as the data ingress and egress component around Kafka.
> > Writing
> > > > external clients that require their own reliability and scalability
> > > > management gets old real fast and pushing this into a simple Samza
> job
> > > that
> > > > reads from system X and pushes into Kafka (or vice versa) was the
> > obvious
> > > > choice for me in the current model. For this type of usage though
> > copycat
> > > > is likely much superior (obviously needs proven) and the question
> then
> > is
> > > > if most Samza users look to the system implementations to also act
> as a
> > > > front-end into Kafka or if significant usage is indeed intended to
> have
> > > the
> > > > alternative systems as the primary message source. That understanding
> > > will
> > > > I think give much clarity in just what value the abstraction overhead
> > of
> > > > the current model brings.
> > > >
> > > > Garry
> > > >
> > > > -----Original Message-----
> > > > From: Yan Fang [mailto:yanfang724@gmail.com]
> > > > Sent: 13 July 2015 19:58
> > > > To: dev@samza.apache.org
> > > > Subject: Re: Thoughts and obesrvations on Samza
> > > >
> > > > I am leaning to Jay's fifth approach. It is not radical and gives us
> > some
> > > > time to see the outcome.
> > > >
> > > > In addition, I would suggest:
> > > >
> > > > 1) Keep the SystemConsumer/SystemProducer API. Because current
> > > > SystemConsumer/SystemProducer API satisfies the usage (From Joardan,
> > and
> > > > even Garry's feedback) and is not so broken that we want to deprecate
> > it.
> > > > Though there are some issues in implemnting the Kinesis, they are not
> > > > unfixable. Nothing should prevent Samza, as a stream processing
> system,
> > > to
> > > > support other systems. In addition, there already are some systems
> > > > exiting besides Kafka: ElasticSearch (committed to the master), HDFS
> > > > (patch-available), S3( from the mailing list), Kinesis (developing in
> > > > another repository), ActiveMQ (in two months). We may want to see how
> > > those
> > > > go before we "kill" them.
> > > >
> > > > 2) Can have some Samza devs involved in Kafka's "transformer" client
> > API.
> > > > This can not only help the future integration (if any) much easier,
> > > because
> > > > they have knowledge about both systems, but also good for Kafka's
> > > > community, because Samza devs have the streaming process experience
> > that
> > > > Kafka devs may miss.
> > > >
> > > > 3) Samza's partition management system may still support other
> systems.
> > > > Though the partition management logic in samza-kafka will be moved to
> > > > Kafka, its still useful for other systems that do not have the
> > partition
> > > > management layer.
> > > >
> > > > 4) Start sharing the docs/websites and using the same terminology
> > (though
> > > > do not know how to do this exactly. :). This will reduce the future
> > > > confusion and does not hurt Samza's independency.
> > > >
> > > > In my opinion, Samza, as a standalone project, still can (and
> already)
> > > > heavily replying on Kafka, and even more tuned for Kafka-specific
> > usage.
> > > > Kafka, also can embed Samza in the document, I do not see anything
> > > prevent
> > > > doing this.
> > > >
> > > > Thanks,
> > > >
> > > > Fang, Yan
> > > > yanfang724@gmail.com
> > > >
> > > > On Mon, Jul 13, 2015 at 11:25 AM, Jordan Shaw <jo...@pubnub.com>
> > wrote:
> > > >
> > > > > Jay,
> > > > > I think doing this iteratively in smaller chunks is a better way to
> > go
> > > as
> > > > > new issues arise. As Navina said Kafka is a "stream system" and
> Samza
> > > is
> > > > a
> > > > > "stream processor" and those two ideas should be mutually
> exclusive.
> > > > >
> > > > > -Jordan
> > > > >
> > > > > On Mon, Jul 13, 2015 at 10:06 AM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Hmm, thought about this more. Maybe this is just too much too
> > quick.
> > > > > > Overall I think there is some enthusiasm for the proposal but
> it's
> > > not
> > > > > > really unanimous enough to make any kind of change this big
> > cleanly.
> > > > The
> > > > > > board doesn't really like the merging stuff, user's are concerned
> > > about
> > > > > > compatibility, I didn't feel there was unanimous agreement on
> > > dropping
> > > > > > SystemConsumer, etc. Even if this is the right end state to get
> to,
> > > > > > probably trying to push all this through at once isn't the right
> > way
> > > to
> > > > > do
> > > > > > it.
> > > > > >
> > > > > > So let me propose a kind of fifth (?) option which I think is
> less
> > > > > dramatic
> > > > > > and let's things happen gradually. I think this is kind of like
> > > > combining
> > > > > > the first part of Yi's proposal and Jakob's third option, leaving
> > the
> > > > > rest
> > > > > > to be figured out incrementally:
> > > > > >
> > > > > > Option 5: We continue the prototype I shared and propose that as
> a
> > > kind
> > > > > of
> > > > > > "transformer" client API in Kafka. This isn't really a
> full-fledged
> > > > > stream
> > > > > > processing layer, more like a supped up consumer api for munging
> > > > topics.
> > > > > > This would let us figure out some of the technical bits, how to
> do
> > > this
> > > > > on
> > > > > > Kafka's group management features, how to integrate the txn
> feature
> > > to
> > > > do
> > > > > > the exactly-once stuff in these transformations, and get all this
> > > stuff
> > > > > > solid. This api would have valid uses in it's own right,
> especially
> > > > when
> > > > > > your transformation will be embedded inside an existing service
> or
> > > > > > application which isn't possible with Samza (or other existing
> > > systems
> > > > > that
> > > > > > I know of).
> > > > > >
> > > > > > Independently we can iterate on some of the ideas of the original
> > > > > proposal
> > > > > > individually and figure out how (if at all) to make use of this
> > > > > > functionality. This can be done bit-by-bit:
> > > > > > - Could be that the existing StreamTask API ends up wrapping this
> > > > > > - Could end up exposed directly in Samza as Yi proposed
> > > > > > - Could be that just the lower-level group-management stuff get's
> > > used,
> > > > > and
> > > > > > in this case it could be either just for standalone mode, or
> always
> > > > > > - Could be that it stays as-is
> > > > > >
> > > > > > The advantage of this is it is lower risk...we basically don't
> have
> > > to
> > > > > make
> > > > > > 12 major decisions all at once that kind of hinge on what amounts
> > to
> > > a
> > > > > > pretty aggressive rewrite. The disadvantage of this is it is a
> bit
> > > more
> > > > > > confusing as all this is getting figured out.
> > > > > >
> > > > > > As with some of the other stuff, this would require a further
> > > > discussion
> > > > > in
> > > > > > the Kafka community if people do like this approach.
> > > > > >
> > > > > > Thoughts?
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Sun, Jul 12, 2015 at 10:52 PM, Jay Kreps <jay.kreps@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > Hey Chris,
> > > > > > >
> > > > > > > Yeah, I'm obviously in favor of this.
> > > > > > >
> > > > > > > The sub-project approach seems the ideal way to take a graceful
> > > step
> > > > in
> > > > > > > this direction, so I will ping the board folks and see why they
> > are
> > > > > > > discouraged, it would be good to understand that. If we go that
> > > route
> > > > > we
> > > > > > > would need to do a similar discussion in the Kafka list (but
> > makes
> > > > > sense
> > > > > > to
> > > > > > > figure out first if it is what Samza wants).
> > > > > > >
> > > > > > > Irrespective of how it's implemented, though, to me the
> important
> > > > > things
> > > > > > > are the following:
> > > > > > > 1. Unify the website, config, naming, docs, metrics,
> > etc--basically
> > > > fix
> > > > > > > the product experience so the "stream" and the "processing"
> feel
> > > > like a
> > > > > > > single user experience and brand. This seems minor but I think
> > is a
> > > > > > really
> > > > > > > big deal.
> > > > > > > 2. Make "standalone" mode a first class citizen and have a real
> > > > > technical
> > > > > > > plan to be able to support cluster managers other than YARN.
> > > > > > > 3. Make the config and out-of-the-box experience more usable
> > > > > > >
> > > > > > > I think that prototype gives a practical example of how 1-3
> could
> > > be
> > > > > done
> > > > > > > and we should pursue it. This is a pretty radical change, so I
> > > > wouldn't
> > > > > > be
> > > > > > > shocked if people didn't want to take a step like that.
> > > > > > >
> > > > > > > Maybe it would make sense to see if people are on board with
> that
> > > > > general
> > > > > > > idea, and then try to get some advice on sub-projects in
> parallel
> > > and
> > > > > > nail
> > > > > > > down those details?
> > > > > > >
> > > > > > > -Jay
> > > > > > >
> > > > > > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
> > > > > criccomini@apache.org>
> > > > > > > wrote:
> > > > > > >
> > > > > > >> Hey all,
> > > > > > >>
> > > > > > >> I want to start by saying that I'm absolutely thrilled to be a
> > > part
> > > > of
> > > > > > >> this
> > > > > > >> community. The amount of level-headed, thoughtful, educated
> > > > discussion
> > > > > > >> that's gone on over the past ~10 days is overwhelming.
> > Wonderful.
> > > > > > >>
> > > > > > >> It seems like discussion is waning a bit, and we've reached
> some
> > > > > > >> conclusions. There are several key emails in this threat,
> which
> > I
> > > > want
> > > > > > to
> > > > > > >> call out:
> > > > > > >>
> > > > > > >> 1. Jakob's summary of the three potential ways forward.
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > > > > > >> 2. Julian's call out that we should be focusing on community
> > over
> > > > > code.
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > > > > > >> 3. Martin's summary about the benefits of merging communities.
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > > > > > >> 4. Jakob's comments about the distinction between community
> and
> > > code
> > > > > > >> paths.
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> > > > > > >>
> > > > > > >> I agree with the comments on all of these emails. I think
> > Martin's
> > > > > > summary
> > > > > > >> of his position aligns very closely with my own. To that end,
> I
> > > > think
> > > > > we
> > > > > > >> should get concrete about what the proposal is, and call a
> vote
> > on
> > > > it.
> > > > > > >> Given that Jay, Martin, and I seem to be aligning fairly
> > closely,
> > > I
> > > > > > think
> > > > > > >> we should start with:
> > > > > > >>
> > > > > > >> 1. [community] Make Samza a subproject of Kafka.
> > > > > > >> 2. [community] Make all Samza PMC/committers committers of the
> > > > > > subproject.
> > > > > > >> 3. [community] Migrate Samza's website/documentation into
> > Kafka's.
> > > > > > >> 4. [code] Have the Samza community and the Kafka community
> > start a
> > > > > > >> from-scratch reboot together in the new Kafka subproject. We
> can
> > > > > > >> borrow/copy &  paste significant chunks of code from Samza's
> > code
> > > > > base.
> > > > > > >> 5. [code] The subproject would intentionally eliminate support
> > for
> > > > > both
> > > > > > >> other streaming systems and all deployment systems.
> > > > > > >> 6. [code] Attempt to provide a bridge from our SystemConsumer
> to
> > > > > KIP-26
> > > > > > >> (copy cat)
> > > > > > >> 7. [code] Attempt to provide a bridge from the new
> subproject's
> > > > > > processor
> > > > > > >> interface to our legacy StreamTask interface.
> > > > > > >> 8. [code/community] Sunset Samza as a TLP when we have a
> working
> > > > Kafka
> > > > > > >> subproject that has a fault-tolerant container with state
> > > > management.
> > > > > > >>
> > > > > > >> It's likely that (6) and (7) won't be fully drop-in. Still,
> the
> > > > closer
> > > > > > we
> > > > > > >> can get, the better it's going to be for our existing
> community.
> > > > > > >>
> > > > > > >> One thing that I didn't touch on with (2) is whether any Samza
> > PMC
> > > > > > members
> > > > > > >> should be rolled into Kafka PMC membership as well (though,
> Jay
> > > and
> > > > > > Jakob
> > > > > > >> are already PMC members on both). I think that Samza's
> community
> > > > > > deserves
> > > > > > >> a
> > > > > > >> voice on the PMC, so I'd propose that we roll at least a few
> PMC
> > > > > members
> > > > > > >> into the Kafka PMC, but I don't have a strong framework for
> > which
> > > > > people
> > > > > > >> to
> > > > > > >> pick.
> > > > > > >>
> > > > > > >> Before (8), I think that Samza's TLP can continue to commit
> bug
> > > > fixes
> > > > > > and
> > > > > > >> patches as it sees fit, provided that we openly communicate
> that
> > > we
> > > > > > won't
> > > > > > >> necessarily migrate new features to the new subproject, and
> that
> > > the
> > > > > TLP
> > > > > > >> will be shut down after the migration to the Kafka subproject
> > > > occurs.
> > > > > > >>
> > > > > > >> Jakob, I could use your guidance here about about how to
> achieve
> > > > this
> > > > > > from
> > > > > > >> an Apache process perspective (sorry).
> > > > > > >>
> > > > > > >> * Should I just call a vote on this proposal?
> > > > > > >> * Should it happen on dev or private?
> > > > > > >> * Do committers have binding votes, or just PMC?
> > > > > > >>
> > > > > > >> Having trouble finding much detail on the Apache wikis. :(
> > > > > > >>
> > > > > > >> Cheers,
> > > > > > >> Chris
> > > > > > >>
> > > > > > >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <
> yanfang724@gmail.com
> > >
> > > > > wrote:
> > > > > > >>
> > > > > > >> > Thanks, Jay. This argument persuaded me actually. :)
> > > > > > >> >
> > > > > > >> > Fang, Yan
> > > > > > >> > yanfang724@gmail.com
> > > > > > >> >
> > > > > > >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <
> jay@confluent.io>
> > > > > wrote:
> > > > > > >> >
> > > > > > >> > > Hey Yan,
> > > > > > >> > >
> > > > > > >> > > Yeah philosophically I think the argument is that you
> should
> > > > > capture
> > > > > > >> the
> > > > > > >> > > stream in Kafka independent of the transformation. This is
> > > > > > obviously a
> > > > > > >> > > Kafka-centric view point.
> > > > > > >> > >
> > > > > > >> > > Advantages of this:
> > > > > > >> > > - In practice I think this is what e.g. Storm people often
> > end
> > > > up
> > > > > > >> doing
> > > > > > >> > > anyway. You usually need to throttle any access to a live
> > > > serving
> > > > > > >> > database.
> > > > > > >> > > - Can have multiple subscribers and they get the same
> thing
> > > > > without
> > > > > > >> > > additional load on the source system.
> > > > > > >> > > - Applications can tap into the stream if need be by
> > > > subscribing.
> > > > > > >> > > - You can debug your transformation by tailing the Kafka
> > topic
> > > > > with
> > > > > > >> the
> > > > > > >> > > console consumer
> > > > > > >> > > - Can tee off the same data stream for batch analysis or
> > > Lambda
> > > > > arch
> > > > > > >> > style
> > > > > > >> > > re-processing
> > > > > > >> > >
> > > > > > >> > > The disadvantage is that it will use Kafka resources. But
> > the
> > > > idea
> > > > > > is
> > > > > > >> > > eventually you will have multiple subscribers to any data
> > > source
> > > > > (at
> > > > > > >> > least
> > > > > > >> > > for monitoring) so you will end up there soon enough
> anyway.
> > > > > > >> > >
> > > > > > >> > > Down the road the technical benefit is that I think it
> gives
> > > us
> > > > a
> > > > > > good
> > > > > > >> > path
> > > > > > >> > > towards end-to-end exactly once semantics from source to
> > > > > > destination.
> > > > > > >> > > Basically the connectors need to support idempotence when
> > > > talking
> > > > > to
> > > > > > >> > Kafka
> > > > > > >> > > and we need the transactional write feature in Kafka to
> make
> > > the
> > > > > > >> > > transformation atomic. This is actually pretty doable if
> you
> > > > > > separate
> > > > > > >> > > connector=>kafka problem from the generic transformations
> > > which
> > > > > are
> > > > > > >> > always
> > > > > > >> > > kafka=>kafka. However I think it is quite impossible to do
> > in
> > > a
> > > > > > >> > all_things
> > > > > > >> > > => all_things environment. Today you can say "well the
> > > semantics
> > > > > of
> > > > > > >> the
> > > > > > >> > > Samza APIs depend on the connectors you use" but it is
> > > actually
> > > > > > worse
> > > > > > >> > then
> > > > > > >> > > that because the semantics actually depend on the pairing
> of
> > > > > > >> > connectors--so
> > > > > > >> > > not only can you probably not get a usable "exactly once"
> > > > > guarantee
> > > > > > >> > > end-to-end it can actually be quite hard to reverse
> engineer
> > > > what
> > > > > > >> > property
> > > > > > >> > > (if any) your end-to-end flow has if you have heterogenous
> > > > > systems.
> > > > > > >> > >
> > > > > > >> > > -Jay
> > > > > > >> > >
> > > > > > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <
> > > yanfang724@gmail.com
> > > > >
> > > > > > >> wrote:
> > > > > > >> > >
> > > > > > >> > > > {quote}
> > > > > > >> > > > maintained in a separate repository and retaining the
> > > existing
> > > > > > >> > > > committership but sharing as much else as possible
> > (website,
> > > > > etc)
> > > > > > >> > > > {quote}
> > > > > > >> > > >
> > > > > > >> > > > Overall, I agree on this idea. Now the question is more
> > > about
> > > > > "how
> > > > > > >> to
> > > > > > >> > do
> > > > > > >> > > > it".
> > > > > > >> > > >
> > > > > > >> > > > On the other hand, one thing I want to point out is
> that,
> > if
> > > > we
> > > > > > >> decide
> > > > > > >> > to
> > > > > > >> > > > go this way, how do we want to support
> > > > > > >> > > > otherSystem-transformation-otherSystem use case?
> > > > > > >> > > >
> > > > > > >> > > > Basically, there are four user groups here:
> > > > > > >> > > >
> > > > > > >> > > > 1. Kafka-transformation-Kafka
> > > > > > >> > > > 2. Kafka-transformation-otherSystem
> > > > > > >> > > > 3. otherSystem-transformation-Kafka
> > > > > > >> > > > 4. otherSystem-transformation-otherSystem
> > > > > > >> > > >
> > > > > > >> > > > For group 1, they can easily use the new Samza library
> to
> > > > > achieve.
> > > > > > >> For
> > > > > > >> > > > group 2 and 3, they can use copyCat -> transformation ->
> > > Kafka
> > > > > or
> > > > > > >> > Kafka->
> > > > > > >> > > > transformation -> copyCat.
> > > > > > >> > > >
> > > > > > >> > > > The problem is for group 4. Do we want to abandon this
> or
> > > > still
> > > > > > >> support
> > > > > > >> > > it?
> > > > > > >> > > > Of course, this use case can be achieved by using
> copyCat
> > ->
> > > > > > >> > > transformation
> > > > > > >> > > > -> Kafka -> transformation -> copyCat, the thing is how
> we
> > > > > > persuade
> > > > > > >> > them
> > > > > > >> > > to
> > > > > > >> > > > do this long chain. If yes, it will also be a win for
> > Kafka
> > > > too.
> > > > > > Or
> > > > > > >> if
> > > > > > >> > > > there is no one in this community actually doing this so
> > > far,
> > > > > > maybe
> > > > > > >> ok
> > > > > > >> > to
> > > > > > >> > > > not support the group 4 directly.
> > > > > > >> > > >
> > > > > > >> > > > Thanks,
> > > > > > >> > > >
> > > > > > >> > > > Fang, Yan
> > > > > > >> > > > yanfang724@gmail.com
> > > > > > >> > > >
> > > > > > >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <
> > > jay@confluent.io
> > > > >
> > > > > > >> wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Yeah I agree with this summary. I think there are kind
> > of
> > > > two
> > > > > > >> > questions
> > > > > > >> > > > > here:
> > > > > > >> > > > > 1. Technically does alignment/reliance on Kafka make
> > sense
> > > > > > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
> > > > > alignment
> > > > > > >> with
> > > > > > >> > > > Kafka
> > > > > > >> > > > > make sense
> > > > > > >> > > > >
> > > > > > >> > > > > Personally I do think both of these things would be
> > really
> > > > > > >> valuable,
> > > > > > >> > > and
> > > > > > >> > > > > would dramatically alter the trajectory of the
> project.
> > > > > > >> > > > >
> > > > > > >> > > > > My preference would be to see if people can mostly
> agree
> > > on
> > > > a
> > > > > > >> > direction
> > > > > > >> > > > > rather than splintering things off. From my point of
> > view
> > > > the
> > > > > > >> ideal
> > > > > > >> > > > outcome
> > > > > > >> > > > > of all the options discussed would be to make Samza a
> > > > closely
> > > > > > >> aligned
> > > > > > >> > > > > subproject, maintained in a separate repository and
> > > > retaining
> > > > > > the
> > > > > > >> > > > existing
> > > > > > >> > > > > committership but sharing as much else as possible
> > > (website,
> > > > > > >> etc). No
> > > > > > >> > > > idea
> > > > > > >> > > > > about how these things work, Jacob, you probably know
> > > more.
> > > > > > >> > > > >
> > > > > > >> > > > > No discussion amongst the Kafka folks has happened on
> > > this,
> > > > > but
> > > > > > >> > likely
> > > > > > >> > > we
> > > > > > >> > > > > should figure out what the Samza community actually
> > wants
> > > > > first.
> > > > > > >> > > > >
> > > > > > >> > > > > I admit that this is a fairly radical departure from
> how
> > > > > things
> > > > > > >> are.
> > > > > > >> > > > >
> > > > > > >> > > > > If that doesn't fly, I think, yeah we could leave
> Samza
> > as
> > > > it
> > > > > is
> > > > > > >> and
> > > > > > >> > do
> > > > > > >> > > > the
> > > > > > >> > > > > more radical reboot inside Kafka. From my point of
> view
> > > that
> > > > > > does
> > > > > > >> > leave
> > > > > > >> > > > > things in a somewhat confusing state since now there
> are
> > > two
> > > > > > >> stream
> > > > > > >> > > > > processing systems more or less coupled to Kafka in
> > large
> > > > part
> > > > > > >> made
> > > > > > >> > by
> > > > > > >> > > > the
> > > > > > >> > > > > same people. But, arguably that might be a cleaner way
> > to
> > > > make
> > > > > > the
> > > > > > >> > > > cut-over
> > > > > > >> > > > > and perhaps less risky for Samza community since if it
> > > works
> > > > > > >> people
> > > > > > >> > can
> > > > > > >> > > > > switch and if it doesn't nothing will have changed.
> > Dunno,
> > > > how
> > > > > > do
> > > > > > >> > > people
> > > > > > >> > > > > feel about this?
> > > > > > >> > > > >
> > > > > > >> > > > > -Jay
> > > > > > >> > > > >
> > > > > > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> > > > > > jghoman@gmail.com>
> > > > > > >> > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > >  This leads me to thinking that merging projects
> and
> > > > > > >> communities
> > > > > > >> > > > might
> > > > > > >> > > > > > be a good idea: with the union of experience from
> both
> > > > > > >> communities,
> > > > > > >> > > we
> > > > > > >> > > > > will
> > > > > > >> > > > > > probably build a better system that is better for
> > users.
> > > > > > >> > > > > > Is this what's being proposed though? Merging the
> > > projects
> > > > > > seems
> > > > > > >> > like
> > > > > > >> > > > > > a consequence of at most one of the three directions
> > > under
> > > > > > >> > > discussion:
> > > > > > >> > > > > > 1) Samza 2.0: The Samza community relies more
> heavily
> > on
> > > > > Kafka
> > > > > > >> for
> > > > > > >> > > > > > configuration, etc. (to a greater or lesser extent
> to
> > be
> > > > > > >> > determined)
> > > > > > >> > > > > > but the Samza community would not automatically
> merge
> > > > withe
> > > > > > >> Kafka
> > > > > > >> > > > > > community (the Phoenix/HBase example is a good one
> > > here).
> > > > > > >> > > > > > 2) Samza Reboot: The Samza community continues to
> > exist
> > > > > with a
> > > > > > >> > > limited
> > > > > > >> > > > > > project scope, but similarly would not need to be
> part
> > > of
> > > > > the
> > > > > > >> Kafka
> > > > > > >> > > > > > community (ie given committership) to progress.
> Here,
> > > > maybe
> > > > > > the
> > > > > > >> > > Samza
> > > > > > >> > > > > > team would become a subproject of Kafka (the Board
> > > frowns
> > > > on
> > > > > > >> > > > > > subprojects at the moment, so I'm not sure if that's
> > > even
> > > > > > >> > feasible),
> > > > > > >> > > > > > but that would not be required.
> > > > > > >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this
> > > > option
> > > > > > the
> > > > > > >> > Kafka
> > > > > > >> > > > > > team builds its own streaming library, possibly off
> of
> > > > Jay's
> > > > > > >> > > > > > prototype, which has not direct lineage to the Samza
> > > team.
> > > > > > >> There's
> > > > > > >> > > no
> > > > > > >> > > > > > reason for the Kafka team to bring in the Samza
> team.
> > > > > > >> > > > > >
> > > > > > >> > > > > > Is the Kafka community on board with this?
> > > > > > >> > > > > >
> > > > > > >> > > > > > To be clear, all three options under discussion are
> > > > > > interesting,
> > > > > > >> > > > > > technically valid and likely healthy directions for
> > the
> > > > > > project.
> > > > > > >> > > > > > Also, they are not mutually exclusive.  The Samza
> > > > community
> > > > > > >> could
> > > > > > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
> > > > > community
> > > > > > >> went
> > > > > > >> > > > > > forward with 'Hey Samza!'  My points above are
> > directed
> > > > > > >> entirely at
> > > > > > >> > > > > > the community aspect of these choices.
> > > > > > >> > > > > > -Jakob
> > > > > > >> > > > > >
> > > > > > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> > > > > > roger.hoover@gmail.com>
> > > > > > >> > > wrote:
> > > > > > >> > > > > > > That's great.  Thanks, Jay.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> > > > > > jay@confluent.io>
> > > > > > >> > > wrote:
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >> Yeah totally agree. I think you have this issue
> > even
> > > > > today,
> > > > > > >> > right?
> > > > > > >> > > > > I.e.
> > > > > > >> > > > > > if
> > > > > > >> > > > > > >> you need to make a simple config change and
> you're
> > > > > running
> > > > > > in
> > > > > > >> > YARN
> > > > > > >> > > > > today
> > > > > > >> > > > > > >> you end up bouncing the job which then rebuilds
> > > state.
> > > > I
> > > > > > >> think
> > > > > > >> > the
> > > > > > >> > > > fix
> > > > > > >> > > > > > is
> > > > > > >> > > > > > >> exactly what you described which is to have a
> long
> > > > > timeout
> > > > > > on
> > > > > > >> > > > > partition
> > > > > > >> > > > > > >> movement for stateful jobs so that if a job is
> just
> > > > > getting
> > > > > > >> > > bounced,
> > > > > > >> > > > > and
> > > > > > >> > > > > > >> the cluster manager (or admin) is smart enough to
> > > > restart
> > > > > > it
> > > > > > >> on
> > > > > > >> > > the
> > > > > > >> > > > > same
> > > > > > >> > > > > > >> host when possible, it can optimistically reuse
> any
> > > > > > existing
> > > > > > >> > state
> > > > > > >> > > > it
> > > > > > >> > > > > > finds
> > > > > > >> > > > > > >> on disk (if it is valid).
> > > > > > >> > > > > > >>
> > > > > > >> > > > > > >> So in this model the charter of the CM is to
> place
> > > > > > processes
> > > > > > >> as
> > > > > > >> > > > > > stickily as
> > > > > > >> > > > > > >> possible and to restart or re-place failed
> > processes.
> > > > The
> > > > > > >> > charter
> > > > > > >> > > of
> > > > > > >> > > > > the
> > > > > > >> > > > > > >> partition management system is to control the
> > > > assignment
> > > > > of
> > > > > > >> work
> > > > > > >> > > to
> > > > > > >> > > > > > these
> > > > > > >> > > > > > >> processes. The nice thing about this is that the
> > work
> > > > > > >> > assignment,
> > > > > > >> > > > > > timeouts,
> > > > > > >> > > > > > >> behavior, configs, and code will all be the same
> > > across
> > > > > all
> > > > > > >> > > cluster
> > > > > > >> > > > > > >> managers.
> > > > > > >> > > > > > >>
> > > > > > >> > > > > > >> So I think that prototype would actually give you
> > > > exactly
> > > > > > >> what
> > > > > > >> > you
> > > > > > >> > > > > want
> > > > > > >> > > > > > >> today for any cluster manager (or manual
> placement
> > +
> > > > > > restart
> > > > > > >> > > script)
> > > > > > >> > > > > > that
> > > > > > >> > > > > > >> was sticky in terms of host placement since there
> > is
> > > > > > already
> > > > > > >> a
> > > > > > >> > > > > > configurable
> > > > > > >> > > > > > >> partition movement timeout and task-by-task state
> > > reuse
> > > > > > with
> > > > > > >> a
> > > > > > >> > > check
> > > > > > >> > > > > on
> > > > > > >> > > > > > >> state validity.
> > > > > > >> > > > > > >>
> > > > > > >> > > > > > >> -Jay
> > > > > > >> > > > > > >>
> > > > > > >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > > > > > >> > > > roger.hoover@gmail.com
> > > > > > >> > > > > >
> > > > > > >> > > > > > >> wrote:
> > > > > > >> > > > > > >>
> > > > > > >> > > > > > >> > That would be great to let Kafka do as much
> heavy
> > > > > lifting
> > > > > > >> as
> > > > > > >> > > > > possible
> > > > > > >> > > > > > and
> > > > > > >> > > > > > >> > make it easier for other languages to implement
> > > Samza
> > > > > > apis.
> > > > > > >> > > > > > >> >
> > > > > > >> > > > > > >> > One thing to watch out for is the interplay
> > between
> > > > > > Kafka's
> > > > > > >> > > group
> > > > > > >> > > > > > >> > management and the external scheduler/process
> > > > manager's
> > > > > > >> fault
> > > > > > >> > > > > > tolerance.
> > > > > > >> > > > > > >> > If a container dies, the Kafka group membership
> > > > > protocol
> > > > > > >> will
> > > > > > >> > > try
> > > > > > >> > > > to
> > > > > > >> > > > > > >> assign
> > > > > > >> > > > > > >> > it's tasks to other containers while at the
> same
> > > time
> > > > > the
> > > > > > >> > > process
> > > > > > >> > > > > > manager
> > > > > > >> > > > > > >> > is trying to relaunch the container.  Without
> > some
> > > > > > >> > consideration
> > > > > > >> > > > for
> > > > > > >> > > > > > this
> > > > > > >> > > > > > >> > (like a configurable amount of time to wait
> > before
> > > > > Kafka
> > > > > > >> > alters
> > > > > > >> > > > the
> > > > > > >> > > > > > group
> > > > > > >> > > > > > >> > membership), there may be thrashing going on
> > which
> > > is
> > > > > > >> > especially
> > > > > > >> > > > bad
> > > > > > >> > > > > > for
> > > > > > >> > > > > > >> > containers with large amounts of local state.
> > > > > > >> > > > > > >> >
> > > > > > >> > > > > > >> > Someone else pointed this out already but I
> > thought
> > > > it
> > > > > > >> might
> > > > > > >> > be
> > > > > > >> > > > > worth
> > > > > > >> > > > > > >> > calling out again.
> > > > > > >> > > > > > >> >
> > > > > > >> > > > > > >> > Cheers,
> > > > > > >> > > > > > >> >
> > > > > > >> > > > > > >> > Roger
> > > > > > >> > > > > > >> >
> > > > > > >> > > > > > >> >
> > > > > > >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> > > > > > >> jay@confluent.io>
> > > > > > >> > > > > wrote:
> > > > > > >> > > > > > >> >
> > > > > > >> > > > > > >> > > Hey Roger,
> > > > > > >> > > > > > >> > >
> > > > > > >> > > > > > >> > > I couldn't agree more. We spent a bunch of
> time
> > > > > talking
> > > > > > >> to
> > > > > > >> > > > people
> > > > > > >> > > > > > and
> > > > > > >> > > > > > >> > that
> > > > > > >> > > > > > >> > > is exactly the stuff we heard time and again.
> > > What
> > > > > > makes
> > > > > > >> it
> > > > > > >> > > > hard,
> > > > > > >> > > > > of
> > > > > > >> > > > > > >> > > course, is that there is some tension between
> > > > > > >> compatibility
> > > > > > >> > > with
> > > > > > >> > > > > > what's
> > > > > > >> > > > > > >> > > there now and making things better for new
> > users.
> > > > > > >> > > > > > >> > >
> > > > > > >> > > > > > >> > > I also strongly agree with the importance of
> > > > > > >> multi-language
> > > > > > >> > > > > > support. We
> > > > > > >> > > > > > >> > are
> > > > > > >> > > > > > >> > > talking now about Java, but for application
> > > > > development
> > > > > > >> use
> > > > > > >> > > > cases
> > > > > > >> > > > > > >> people
> > > > > > >> > > > > > >> > > want to work in whatever language they are
> > using
> > > > > > >> elsewhere.
> > > > > > >> > I
> > > > > > >> > > > > think
> > > > > > >> > > > > > >> > moving
> > > > > > >> > > > > > >> > > to a model where Kafka itself does the group
> > > > > > membership,
> > > > > > >> > > > lifecycle
> > > > > > >> > > > > > >> > control,
> > > > > > >> > > > > > >> > > and partition assignment has the advantage of
> > > > putting
> > > > > > all
> > > > > > >> > that
> > > > > > >> > > > > > complex
> > > > > > >> > > > > > >> > > stuff behind a clean api that the clients are
> > > > already
> > > > > > >> going
> > > > > > >> > to
> > > > > > >> > > > be
> > > > > > >> > > > > > >> > > implementing for their consumer, so the added
> > > > > > >> functionality
> > > > > > >> > > for
> > > > > > >> > > > > > stream
> > > > > > >> > > > > > >> > > processing beyond a consumer becomes very
> > minor.
> > > > > > >> > > > > > >> > >
> > > > > > >> > > > > > >> > > -Jay
> > > > > > >> > > > > > >> > >
> > > > > > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger
> Hoover <
> > > > > > >> > > > > > roger.hoover@gmail.com>
> > > > > > >> > > > > > >> > > wrote:
> > > > > > >> > > > > > >> > >
> > > > > > >> > > > > > >> > > > Metamorphosis...nice. :)
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > > This has been a great discussion.  As a
> user
> > of
> > > > > Samza
> > > > > > >> > who's
> > > > > > >> > > > > > recently
> > > > > > >> > > > > > >> > > > integrated it into a relatively large
> > > > > organization, I
> > > > > > >> just
> > > > > > >> > > > want
> > > > > > >> > > > > to
> > > > > > >> > > > > > >> add
> > > > > > >> > > > > > >> > > > support to a few points already made.
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as
> > it
> > > > > > >> currently
> > > > > > >> > > > exists
> > > > > > >> > > > > > that
> > > > > > >> > > > > > >> > I've
> > > > > > >> > > > > > >> > > > experienced are:
> > > > > > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
> > > > > environments
> > > > > > >> > where
> > > > > > >> > > > > Puppet
> > > > > > >> > > > > > >> > would
> > > > > > >> > > > > > >> > > do
> > > > > > >> > > > > > >> > > > just fine but it was the only mechanism to
> > get
> > > > > fault
> > > > > > >> > > > tolerance.
> > > > > > >> > > > > > >> > > > 2) Configuration - I think I like the idea
> of
> > > > > > >> configuring
> > > > > > >> > > most
> > > > > > >> > > > > of
> > > > > > >> > > > > > the
> > > > > > >> > > > > > >> > job
> > > > > > >> > > > > > >> > > > in code rather than config files.  In
> > general,
> > > I
> > > > > > think
> > > > > > >> the
> > > > > > >> > > > goal
> > > > > > >> > > > > > >> should
> > > > > > >> > > > > > >> > be
> > > > > > >> > > > > > >> > > > to make it harder to make mistakes,
> > especially
> > > of
> > > > > the
> > > > > > >> kind
> > > > > > >> > > > where
> > > > > > >> > > > > > the
> > > > > > >> > > > > > >> > code
> > > > > > >> > > > > > >> > > > expects something and the config doesn't
> > match.
> > > > > The
> > > > > > >> > current
> > > > > > >> > > > > > config
> > > > > > >> > > > > > >> is
> > > > > > >> > > > > > >> > > > quite intricate and error-prone.  For
> > example,
> > > > the
> > > > > > >> > > application
> > > > > > >> > > > > > logic
> > > > > > >> > > > > > >> > may
> > > > > > >> > > > > > >> > > > depend on bootstrapping a topic but rather
> > than
> > > > > > >> asserting
> > > > > > >> > > that
> > > > > > >> > > > > in
> > > > > > >> > > > > > the
> > > > > > >> > > > > > >> > > code,
> > > > > > >> > > > > > >> > > > you have to rely on getting the config
> right.
> > > > > > Likewise
> > > > > > >> > with
> > > > > > >> > > > > > serdes,
> > > > > > >> > > > > > >> > the
> > > > > > >> > > > > > >> > > > Java representations produced by various
> > serdes
> > > > > > (JSON,
> > > > > > >> > Avro,
> > > > > > >> > > > > etc.)
> > > > > > >> > > > > > >> are
> > > > > > >> > > > > > >> > > not
> > > > > > >> > > > > > >> > > > equivalent so you cannot just reconfigure a
> > > serde
> > > > > > >> without
> > > > > > >> > > > > changing
> > > > > > >> > > > > > >> the
> > > > > > >> > > > > > >> > > > code.   It would be nice for jobs to be
> able
> > to
> > > > > > assert
> > > > > > >> > what
> > > > > > >> > > > they
> > > > > > >> > > > > > >> expect
> > > > > > >> > > > > > >> > > > from their input topics in terms of
> > > partitioning.
> > > > > > >> This is
> > > > > > >> > > > > > getting a
> > > > > > >> > > > > > >> > > little
> > > > > > >> > > > > > >> > > > off topic but I was even thinking about
> > > creating
> > > > a
> > > > > > >> "Samza
> > > > > > >> > > > config
> > > > > > >> > > > > > >> > linter"
> > > > > > >> > > > > > >> > > > that would sanity check a set of configs.
> > > > > Especially
> > > > > > >> in
> > > > > > >> > > > > > >> organizations
> > > > > > >> > > > > > >> > > > where config is managed by a different team
> > > than
> > > > > the
> > > > > > >> > > > application
> > > > > > >> > > > > > >> > > developer,
> > > > > > >> > > > > > >> > > > it's very hard to get avoid config
> mistakes.
> > > > > > >> > > > > > >> > > > 3) Java/Scala centric - for many teams
> > > > (especially
> > > > > > >> > > DevOps-type
> > > > > > >> > > > > > >> folks),
> > > > > > >> > > > > > >> > > the
> > > > > > >> > > > > > >> > > > pain of the Java toolchain (maven, slow
> > builds,
> > > > > weak
> > > > > > >> > command
> > > > > > >> > > > > line
> > > > > > >> > > > > > >> > > support,
> > > > > > >> > > > > > >> > > > configuration over convention) really
> > inhibits
> > > > > > >> > productivity.
> > > > > > >> > > > As
> > > > > > >> > > > > > more
> > > > > > >> > > > > > >> > and
> > > > > > >> > > > > > >> > > > more high-quality clients become available
> > for
> > > > > > Kafka, I
> > > > > > >> > hope
> > > > > > >> > > > > > they'll
> > > > > > >> > > > > > >> > > follow
> > > > > > >> > > > > > >> > > > Samza's model.  Not sure how much it
> affects
> > > the
> > > > > > >> proposals
> > > > > > >> > > in
> > > > > > >> > > > > this
> > > > > > >> > > > > > >> > thread
> > > > > > >> > > > > > >> > > > but please consider other languages in the
> > > > > ecosystem
> > > > > > as
> > > > > > >> > > well.
> > > > > > >> > > > > > From
> > > > > > >> > > > > > >> > what
> > > > > > >> > > > > > >> > > > I've heard, Spark has more Python users
> than
> > > > > > >> Java/Scala.
> > > > > > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the
> Samza
> > > API
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > >
> > > > > > >> > > > > > >> >
> > > > > > >> > > > > > >>
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > > > > >> > > > > > >> > > > and are working on a Yeoman generator
> > > > > > >> > > > > > >> > > >
> https://github.com/Quantiply/generator-rico
> > > for
> > > > > > >> > > Jython/Samza
> > > > > > >> > > > > > >> projects
> > > > > > >> > > > > > >> > to
> > > > > > >> > > > > > >> > > > alleviate some of the pain)
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > > I also want to underscore Jay's point about
> > > > > improving
> > > > > > >> the
> > > > > > >> > > user
> > > > > > >> > > > > > >> > > experience.
> > > > > > >> > > > > > >> > > > That's a very important factor for
> > adoption.  I
> > > > > think
> > > > > > >> the
> > > > > > >> > > goal
> > > > > > >> > > > > > should
> > > > > > >> > > > > > >> > be
> > > > > > >> > > > > > >> > > to
> > > > > > >> > > > > > >> > > > make Samza as easy to get started with as
> > > > something
> > > > > > >> like
> > > > > > >> > > > > Logstash.
> > > > > > >> > > > > > >> > > > Logstash is vastly inferior in terms of
> > > > > capabilities
> > > > > > to
> > > > > > >> > > Samza
> > > > > > >> > > > > but
> > > > > > >> > > > > > >> it's
> > > > > > >> > > > > > >> > > easy
> > > > > > >> > > > > > >> > > > to get started and that makes a big
> > difference.
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > > Cheers,
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > > Roger
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco
> De
> > > > > > Francisci
> > > > > > >> > > > Morales <
> > > > > > >> > > > > > >> > > > gdfm@apache.org> wrote:
> > > > > > >> > > > > > >> > > >
> > > > > > >> > > > > > >> > > > > Forgot to add. On the naming issues,
> Kafka
> > > > > > >> Metamorphosis
> > > > > > >> > > is
> > > > > > >> > > > a
> > > > > > >> > > > > > clear
> > > > > > >> > > > > > >> > > > winner
> > > > > > >> > > > > > >> > > > > :)
> > > > > > >> > > > > > >> > > > >
> > > > > > >> > > > > > >> > > > > --
> > > > > > >> > > > > > >> > > > > Gianmarco
> > > > > > >> > > > > > >> > > > >
> > > > > > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De
> > > Francisci
> > > > > > >> Morales
> > > > > > >> > <
> > > > > > >> > > > > > >> > > gdfm@apache.org
> > > > > > >> > > > > > >> > > > >
> > > > > > >> > > > > > >> > > > > wrote:
> > > > > > >> > > > > > >> > > > >
> > > > > > >> > > > > > >> > > > > > Hi,
> > > > > > >> > > > > > >> > > > > >
> > > > > > >> > > > > > >> > > > > > @Martin, thanks for you comments.
> > > > > > >> > > > > > >> > > > > > Maybe I'm missing some important point,
> > > but I
> > > > > > think
> > > > > > >> > > > coupling
> > > > > > >> > > > > > the
> > > > > > >> > > > > > >> > > > releases
> > > > > > >> > > > > > >> > > > > > is actually a *good* thing.
> > > > > > >> > > > > > >> > > > > > To make an example, would it be better
> if
> > > the
> > > > > MR
> > > > > > >> and
> > > > > > >> > > HDFS
> > > > > > >> > > > > > >> > components
> > > > > > >> > > > > > >> > > of
> > > > > > >> > > > > > >> > > > > > Hadoop had different release schedules?
> > > > > > >> > > > > > >> > > > > >
> > > > > > >> > > > > > >> > > > > > Actually, keeping the discussion in a
> > > single
> > > > > > place
> > > > > > >> > would
> > > > > > >> > > > > make
> > > > > > >> > > > > > >> > > agreeing
> > > > > > >> > > > > > >> > > > on
> > > > > > >> > > > > > >> > > > > > releases (and backwards compatibility)
> > much
> > > > > > >> easier, as
> > > > > > >> > > > > > everybody
> > > > > > >> > > > > > >> > > would
> > > > > > >> > > > > > >> > > > be
> > > > > > >> > > > > > >> > > > > > responsible for the whole codebase.
> > > > > > >> > > > > > >> > > > > >
> > > > > > >> > > > > > >> > > > > > That said, I like the idea of absorbing
> > > > > > samza-core
> > > > > > >> as
> > > > > > >> > a
> > > > > > >> > > > > > >> > sub-project,
> > > > > > >> > > > > > >> > > > and
> > > > > > >> > > > > > >> > > > > > leave the fancy stuff separate.
> > > > > > >> > > > > > >> > > > > > It probably gives 90% of the benefits
> we
> > > have
> > > > > > been
> > > > > > >> > > > > discussing
> > > > > > >> > > > > > >> here.
> > > > > > >> > > > > > >> > > > > >
> > > > > > >> > > > > > >> > > > > > Cheers,
> > > > > > >> > > > > > >> > > > > >
> > > > > > >> > > > > > >> > > > > > --
> > > > > > >> > > > > > >> > > > > > Gianmarco
> > > > > > >> > > > > > >> > > > > >
> > > > > > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > > > > > >> > jay.kreps@gmail.com
> > > > > > >> > > >
> > > > > > >> > > > > > wrote:
> > > > > > >> > > > > > >> > > > > >
> > > > > > >> > > > > > >> > > > > >> Hey Martin,
> > > > > > >> > > > > > >> > > > > >>
> > > > > > >> > > > > > >> > > > > >> I agree coupling release schedules is
> a
> > > > > > downside.
> > > > > > >> > > > > > >> > > > > >>
> > > > > > >> > > > > > >> > > > > >> Definitely we can try to solve some of
> > the
> > > > > > >> > integration
> > > > > > >> > > > > > problems
> > > > > > >> > > > > > >> in
> > > > > > >> > > > > > >> > > > > >> Confluent Platform or in other
> > > > distributions.
> > > > > > But
> > > > > > >> I
> > > > > > >> > > think
> > > > > > >> > > > > > this
> > > > > > >> > > > > > >> > ends
> > > > > > >> > > > > > >> > > up
> > > > > > >> > > > > > >> > > > > >> being really shallow. I guess I feel
> to
> > > > really
> > > > > > >> get a
> > > > > > >> > > good
> > > > > > >> > > > > > user
> > > > > > >> > > > > > >> > > > > experience
> > > > > > >> > > > > > >> > > > > >> the two systems have to kind of feel
> > like
> > > > part
> > > > > > of
> > > > > > >> the
> > > > > > >> > > > same
> > > > > > >> > > > > > thing
> > > > > > >> > > > > > >> > and
> > > > > > >> > > > > > >> > > > you
> > > > > > >> > > > > > >> > > > > >> can't really add that in later--you
> can
> > > put
> > > > > both
> > > > > > >> in
> > > > > > >> > the
> > > > > > >> > > > > same
> > > > > > >> > > > > > >> > > > > downloadable
> > > > > > >> > > > > > >> > > > > >> tar file but it doesn't really give a
> > very
> > > > > > >> cohesive
> > > > > > >> > > > > feeling.
> > > > > > >> > > > > > I
> > > > > > >> > > > > > >> > agree
> > > > > > >> > > > > > >> > > > > that
> > > > > > >> > > > > > >> > > > > >> ultimately any of the project stuff is
> > as
> > > > much
> > > > > > >> social
> > > > > > >> > > and
> > > > > > >> > > > > > naming
> > > > > > >> > > > > > >> > as
> > > > > > >> > > > > > >> > > > > >> anything else--theoretically two
> totally
> > > > > > >> independent
> > > > > > >> > > > > projects
> > > > > > >> > > > > > >> > could
> > > > > > >> > > > > > >> > > > work
> > > > > > >> > > > > > >> > > > > >> to
> > > > > > >> > > > > > >> > > > > >> tightly align. In practice this seems
> to
> > > be
> > > > > > quite
> > > > > > >> > > > difficult
> > > > > > >> > > > > > >> > though.
> > > > > > >> > > > > > >> > > > > >>
> > > > > > >> > > > > > >> > > > > >> For the frameworks--totally agree it
> > would
> > > > be
> > > > > > >> good to
> > > > > > >> > > > > > maintain
> > > > > > >> > > > > > >> the
> > > > > > >> > > > > > >> > > > > >> framework support with the project. In
> > > some
> > > > > > cases
> > > > > > >> > there
> > > > > > >> > > > may
> > > > > > >> > > > > > not
> > > > > > >> > > > > > >> be
> > > > > > >> > > > > > >> > > too
> > > > > > >> > > > > > >> > > > > >> much
> > > > > > >> > > > > > >> > > > > >> there since the integration gets
> lighter
> > > > but I
> > > > > > >> think
> > > > > > >> > > > > whatever
> > > > > > >> > > > > > >> > stubs
> > > > > > >> > > > > > >> > > > you
> > > > > > >> > > > > > >> > > > > >> need should be included. So no I
> > > definitely
> > > > > > wasn't
> > > > > > >> > > trying
> > > > > > >> > > > > to
> > > > > > >> > > > > > >> imply
> > > > > > >> > > > > > >> > > > > >> dropping
> > > > > > >> > > > > > >> > > > > >> support for these frameworks, just
> > making
> > > > the
> > > > > > >> > > integration
> > > > > > >> > > > > > >> lighter
> > > > > > >> > > > > > >> > by
> > > > > > >> > > > > > >> > > > > >> separating process management from
> > > partition
> > > > > > >> > > management.
> > > > > > >> > > > > > >> > > > > >>
> > > > > > >> > > > > > >> > > > > >> You raise two good points we would
> have
> > to
> > > > > > figure
> > > > > > >> out
> > > > > > >> > > if
> > > > > > >> > > > we
> > > > > > >> > > > > > went
> > > > > > >> > > > > > >> > > down
> > > > > > >> > > > > > >> > > > > the
> > > > > > >> > > > > > >> > > > > >> alignment path:
> > > > > > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I
> > think
> > > > the
> > > > > > >> first
> > > > > > >> > > > > question
> > > > > > >> > > > > > is
> > > > > > >> > > > > > >> > > > whether
> > > > > > >> > > > > > >> > > > > >> some "re-branding" would be worth it.
> If
> > > so
> > > > > > then I
> > > > > > >> > > think
> > > > > > >> > > > we
> > > > > > >> > > > > > can
> > > > > > >> > > > > > >> > > have a
> > > > > > >> > > > > > >> > > > > big
> > > > > > >> > > > > > >> > > > > >> thread on the name. I'm definitely not
> > set
> > > > on
> > > > > > >> Kafka
> > > > > > >> > > > > > Streaming or
> > > > > > >> > > > > > >> > > Kafka
> > > > > > >> > > > > > >> > > > > >> Streams I was just using them to be
> kind
> > > of
> > > > > > >> > > > illustrative. I
> > > > > > >> > > > > > >> agree
> > > > > > >> > > > > > >> > > with
> > > > > > >> > > > > > >> > > > > >> your
> > > > > > >> > > > > > >> > > > > >> critique of these names, though I
> think
> > > > people
> > > > > > >> would
> > > > > > >> > > get
> > > > > > >> > > > > the
> > > > > > >> > > > > > >> idea.
> > > > > > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point
> > about
> > > > how
> > > > > to
> > > > > > >> > > "factor"
> > > > > > >> > > > > it.
> > > > > > >> > > > > > >> Here
> > > > > > >> > > > > > >> > > are
> > > > > > >> > > > > > >> > > > > the
> > > > > > >> > > > > > >> > > > > >> options I see (I could get
> enthusiastic
> > > > about
> > > > > > any
> > > > > > >> of
> > > > > > >> > > > them):
> > > > > > >> > > > > > >> > > > > >>    a. One repo for both Kafka and
> Samza
> > > > > > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> > > > > > seperation
> > > > > > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of
> > > samza-api
> > > > > and
> > > > > > >> > > > samza-core
> > > > > > >> > > > > > is
> > > > > > >> > > > > > >> > > > absorbed
> > > > > > >> > > > > > >> > > > > >> almost like a third client
> > > > > > >> > > > > > >> > > > > >>
> > > > > > >> > > > > > >> > > > > >> Cheers,
> > > > > > >> > > > > > >> > > > > >>
> > > > > > >> > > > > > >> > > > > >> -Jay
> > > > > > >> > > > > > >> > > > > >>
> > > > > > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> > > > > > Kleppmann <
> > > > > > >> > > > > > >> > > > martin@kleppmann.com>
> > > > > > >> > > > > > >> > > > > >> wrote:
> > > > > > >> > > > > > >> > > > > >>
> > > > > > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications.
> > Just
> > > a
> > > > > few
> > > > > > >> > > follow-up
> > > > > > >> > > > > > >> > comments.
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > > >> > - I see the appeal of merging with
> > Kafka
> > > > or
> > > > > > >> > becoming
> > > > > > >> > > a
> > > > > > >> > > > > > >> > subproject:
> > > > > > >> > > > > > >> > > > the
> > > > > > >> > > > > > >> > > > > >> > reasons you mention are good. The
> > risk I
> > > > see
> > > > > > is
> > > > > > >> > that
> > > > > > >> > > > > > release
> > > > > > >> > > > > > >> > > > schedules
> > > > > > >> > > > > > >> > > > > >> > become coupled to each other, which
> > can
> > > > slow
> > > > > > >> > everyone
> > > > > > >> > > > > down,
> > > > > > >> > > > > > >> and
> > > > > > >> > > > > > >> > > > large
> > > > > > >> > > > > > >> > > > > >> > projects with many contributors are
> > > harder
> > > > > to
> > > > > > >> > manage.
> > > > > > >> > > > > > (Jakob,
> > > > > > >> > > > > > >> > can
> > > > > > >> > > > > > >> > > > you
> > > > > > >> > > > > > >> > > > > >> speak
> > > > > > >> > > > > > >> > > > > >> > from experience, having seen a wider
> > > range
> > > > > of
> > > > > > >> > Hadoop
> > > > > > >> > > > > > ecosystem
> > > > > > >> > > > > > >> > > > > >> projects?)
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > > >> > Some of the goals of a better
> unified
> > > > > > developer
> > > > > > >> > > > > experience
> > > > > > >> > > > > > >> could
> > > > > > >> > > > > > >> > > > also
> > > > > > >> > > > > > >> > > > > be
> > > > > > >> > > > > > >> > > > > >> > solved by integrating Samza nicely
> > into
> > > a
> > > > > > Kafka
> > > > > > >> > > > > > distribution
> > > > > > >> > > > > > >> > (such
> > > > > > >> > > > > > >> > > > as
> > > > > > >> > > > > > >> > > > > >> > Confluent's). I'm not against
> merging
> > > > > projects
> > > > > > >> if
> > > > > > >> > we
> > > > > > >> > > > > decide
> > > > > > >> > > > > > >> > that's
> > > > > > >> > > > > > >> > > > the
> > > > > > >> > > > > > >> > > > > >> way
> > > > > > >> > > > > > >> > > > > >> > to go, just pointing out the same
> > goals
> > > > can
> > > > > > >> perhaps
> > > > > > >> > > > also
> > > > > > >> > > > > be
> > > > > > >> > > > > > >> > > achieved
> > > > > > >> > > > > > >> > > > > in
> > > > > > >> > > > > > >> > > > > >> > other ways.
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
> > > > > dependency:
> > > > > > >> are
> > > > > > >> > > you
> > > > > > >> > > > > > >> proposing
> > > > > > >> > > > > > >> > > > that
> > > > > > >> > > > > > >> > > > > >> > Samza doesn't give any help to
> people
> > > > > wanting
> > > > > > to
> > > > > > >> > run
> > > > > > >> > > on
> > > > > > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > > > > > >> > > > > > >> > > > > >> > So the docs would basically have a
> > link
> > > to
> > > > > > >> Slider
> > > > > > >> > and
> > > > > > >> > > > > > nothing
> > > > > > >> > > > > > >> > > else?
> > > > > > >> > > > > > >> > > > Or
> > > > > > >> > > > > > >> > > > > >> > would we maintain integrations with
> a
> > > > bunch
> > > > > of
> > > > > > >> > > popular
> > > > > > >> > > > > > >> > deployment
> > > > > > >> > > > > > >> > > > > >> methods
> > > > > > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell
> > > scripts
> > > > > to
> > > > > > >> make
> > > > > > >> > > > Samza
> > > > > > >> > > > > > work
> > > > > > >> > > > > > >> > with
> > > > > > >> > > > > > >> > > > > >> Slider)?
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > > >> > I absolutely think it's a good idea
> to
> > > > have
> > > > > > the
> > > > > > >> > "as a
> > > > > > >> > > > > > library"
> > > > > > >> > > > > > >> > and
> > > > > > >> > > > > > >> > > > > "as a
> > > > > > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy)
> options
> > > for
> > > > > > >> people
> > > > > > >> > who
> > > > > > >> > > > > want
> > > > > > >> > > > > > >> them,
> > > > > > >> > > > > > >> > > > but I
> > > > > > >> > > > > > >> > > > > >> > think there should also be a
> > > low-friction
> > > > > path
> > > > > > >> for
> > > > > > >> > > > common
> > > > > > >> > > > > > "as
> > > > > > >> > > > > > >> a
> > > > > > >> > > > > > >> > > > > service"
> > > > > > >> > > > > > >> > > > > >> > deployment methods, for which we
> > > probably
> > > > > need
> > > > > > >> to
> > > > > > >> > > > > maintain
> > > > > > >> > > > > > >> > > > > integrations.
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams"
> > seems
> > > > odd
> > > > > to
> > > > > > >> me,
> > > > > > >> > > > > because
> > > > > > >> > > > > > >> Kafka
> > > > > > >> > > > > > >> > > is
> > > > > > >> > > > > > >> > > > > all
> > > > > > >> > > > > > >> > > > > >> > about streams already. Perhaps
> "Kafka
> > > > > > >> Transformers"
> > > > > > >> > > or
> > > > > > >> > > > > > "Kafka
> > > > > > >> > > > > > >> > > > Filters"
> > > > > > >> > > > > > >> > > > > >> > would be more apt?
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of
> > > Samza
> > > > > > >> (stream
> > > > > > >> > > > > > >> transformation
> > > > > > >> > > > > > >> > > > with
> > > > > > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza
> > as a
> > > > > > >> library"
> > > > > > >> > > bit)
> > > > > > >> > > > > > could
> > > > > > >> > > > > > >> > > become
> > > > > > >> > > > > > >> > > > > >> part of
> > > > > > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such
> > as
> > > > > > >> streaming
> > > > > > >> > SQL
> > > > > > >> > > > and
> > > > > > >> > > > > > >> > > > integrations
> > > > > > >> > > > > > >> > > > > >> with
> > > > > > >> > > > > > >> > > > > >> > deployment frameworks remain in a
> > > separate
> > > > > > >> project?
> > > > > > >> > > In
> > > > > > >> > > > > > other
> > > > > > >> > > > > > >> > > words,
> > > > > > >> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > > >> > > > > >> > would absorb the proven, stable core
> > of
> > > > > Samza,
> > > > > > >> > which
> > > > > > >> > > > > would
> > > > > > >> > > > > > >> > become
> > > > > > >> > > > > > >> > > > the
> > > > > > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early
> > in
> > > > this
> > > > > > >> > thread.
> > > > > > >> > > > The
> > > > > > >> > > > > > Samza
> > > > > > >> > > > > > >> > > > project
> > > > > > >> > > > > > >> > > > > >> > would then target that third Kafka
> > > client
> > > > as
> > > > > > its
> > > > > > >> > base
> > > > > > >> > > > > API,
> > > > > > >> > > > > > and
> > > > > > >> > > > > > >> > the
> > > > > > >> > > > > > >> > > > > >> project
> > > > > > >> > > > > > >> > > > > >> > would be freed up to explore more
> > > > > experimental
> > > > > > >> new
> > > > > > >> > > > > > horizons.
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > > >> > Martin
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > > > > > >> > > > jay.kreps@gmail.com>
> > > > > > >> > > > > > >> wrote:
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > > >> > > Hey Martin,
> > > > > > >> > > > > > >> > > > > >> > >
> > > > > > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling
> I
> > > > > actually
> > > > > > >> > don't
> > > > > > >> > > > > think
> > > > > > >> > > > > > it
> > > > > > >> > > > > > >> > ties
> > > > > > >> > > > > > >> > > > our
> > > > > > >> > > > > > >> > > > > >> > hands
> > > > > > >> > > > > > >> > > > > >> > > at all, all it does is refactor
> > > things.
> > > > > The
> > > > > > >> > > division
> > > > > > >> > > > of
> > > > > > >> > > > > > >> > > > > >> responsibility is
> > > > > > >> > > > > > >> > > > > >> > > that Samza core is responsible for
> > > task
> > > > > > >> > lifecycle,
> > > > > > >> > > > > state,
> > > > > > >> > > > > > >> and
> > > > > > >> > > > > > >> > > > > >> partition
> > > > > > >> > > > > > >> > > > > >> > > management (using the Kafka
> > > > co-ordinator)
> > > > > > but
> > > > > > >> it
> > > > > > >> > is
> > > > > > >> > > > NOT
> > > > > > >> > > > > > >> > > > responsible
> > > > > > >> > > > > > >> > > > > >> for
> > > > > > >> > > > > > >> > > > > >> > > packaging, configuration
> deployment
> > or
> > > > > > >> execution
> > > > > > >> > of
> > > > > > >> > > > > > >> processes.
> > > > > > >> > > > > > >> > > The
> > > > > > >> > > > > > >> > > > > >> > problem
> > > > > > >> > > > > > >> > > > > >> > > of packaging and starting these
> > > > processes
> > > > > is
> > > > > > >> > > > > > >> > > > > >> > > framework/environment-specific.
> This
> > > > > leaves
> > > > > > >> > > > individual
> > > > > > >> > > > > > >> > > frameworks
> > > > > > >> > > > > > >> > > > to
> > > > > > >> > > > > > >> > > > > >> be
> > > > > > >> > > > > > >> > > > > >> > as
> > > > > > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So
> > you
> > > > can
> > > > > > get
> > > > > > >> > > simple
> > > > > > >> > > > > > >> stateless
> > > > > > >> > > > > > >> > > > > >> support in
> > > > > > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their
> > > > off-the-shelf
> > > > > > app
> > > > > > >> > > > > framework
> > > > > > >> > > > > > >> > > (Slider,
> > > > > > >> > > > > > >> > > > > >> > Marathon,
> > > > > > >> > > > > > >> > > > > >> > > etc). These are well known by
> people
> > > and
> > > > > > have
> > > > > > >> > nice
> > > > > > >> > > > UIs
> > > > > > >> > > > > > and a
> > > > > > >> > > > > > >> > lot
> > > > > > >> > > > > > >> > > > of
> > > > > > >> > > > > > >> > > > > >> > > flexibility. I don't think they
> have
> > > > node
> > > > > > >> > affinity
> > > > > > >> > > > as a
> > > > > > >> > > > > > >> built
> > > > > > >> > > > > > >> > in
> > > > > > >> > > > > > >> > > > > >> option
> > > > > > >> > > > > > >> > > > > >> > > (though I could be wrong). So if
> we
> > > want
> > > > > > that
> > > > > > >> we
> > > > > > >> > > can
> > > > > > >> > > > > > either
> > > > > > >> > > > > > >> > wait
> > > > > > >> > > > > > >> > > > for
> > > > > > >> > > > > > >> > > > > >> them
> > > > > > >> > > > > > >> > > > > >> > > to add it or do a custom framework
> > to
> > > > add
> > > > > > that
> > > > > > >> > > > feature
> > > > > > >> > > > > > (as
> > > > > > >> > > > > > >> > now).
> > > > > > >> > > > > > >> > > > > >> > Obviously
> > > > > > >> > > > > > >> > > > > >> > > if you manage things with
> old-school
> > > ops
> > > > > > tools
> > > > > > >> > > > > > >> > (puppet/chef/etc)
> > > > > > >> > > > > > >> > > > you
> > > > > > >> > > > > > >> > > > > >> get
> > > > > > >> > > > > > >> > > > > >> > > locality easily. The nice thing,
> > > though,
> > > > > is
> > > > > > >> that
> > > > > > >> > > all
> > > > > > >> > > > > the
> > > > > > >> > > > > > >> samza
> > > > > > >> > > > > > >> > > > > >> "business
> > > > > > >> > > > > > >> > > > > >> > > logic" around partition management
> > and
> > > > > fault
> > > > > > >> > > > tolerance
> > > > > > >> > > > > > is in
> > > > > > >> > > > > > >> > > Samza
> > > > > > >> > > > > > >> > > > > >> core
> > > > > > >> > > > > > >> > > > > >> > so
> > > > > > >> > > > > > >> > > > > >> > > it is shared across frameworks and
> > the
> > > > > > >> framework
> > > > > > >> > > > > specific
> > > > > > >> > > > > > >> bit
> > > > > > >> > > > > > >> > is
> > > > > > >> > > > > > >> > > > > just
> > > > > > >> > > > > > >> > > > > >> > > whether it is smart enough to try
> to
> > > get
> > > > > the
> > > > > > >> same
> > > > > > >> > > > host
> > > > > > >> > > > > > when
> > > > > > >> > > > > > >> a
> > > > > > >> > > > > > >> > > job
> > > > > > >> > > > > > >> > > > is
> > > > > > >> > > > > > >> > > > > >> > > restarted.
> > > > > > >> > > > > > >> > > > > >> > >
> > > > > > >> > > > > > >> > > > > >> > > With respect to the
> Kafka-alignment,
> > > > yeah
> > > > > I
> > > > > > >> think
> > > > > > >> > > the
> > > > > > >> > > > > > goal
> > > > > > >> > > > > > >> > would
> > > > > > >> > > > > > >> > > > be
> > > > > > >> > > > > > >> > > > > >> (a)
> > > > > > >> > > > > > >> > > > > >> > > actually get better alignment in
> > user
> > > > > > >> experience,
> > > > > > >> > > and
> > > > > > >> > > > > (b)
> > > > > > >> > > > > > >> > > express
> > > > > > >> > > > > > >> > > > > >> this in
> > > > > > >> > > > > > >> > > > > >> > > the naming and project branding.
> > > > > > Specifically:
> > > > > > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice
> > for
> > > > the
> > > > > > >> > > > > > "transformation"
> > > > > > >> > > > > > >> api
> > > > > > >> > > > > > >> > > to
> > > > > > >> > > > > > >> > > > be
> > > > > > >> > > > > > >> > > > > >> > > discoverable in the main Kafka
> > > > docs--i.e.
> > > > > be
> > > > > > >> able
> > > > > > >> > > to
> > > > > > >> > > > > > explain
> > > > > > >> > > > > > >> > > when
> > > > > > >> > > > > > >> > > > to
> > > > > > >> > > > > > >> > > > > >> use
> > > > > > >> > > > > > >> > > > > >> > > the consumer and when to use the
> > > stream
> > > > > > >> > processing
> > > > > > >> > > > > > >> > functionality
> > > > > > >> > > > > > >> > > > and
> > > > > > >> > > > > > >> > > > > >> lead
> > > > > > >> > > > > > >> > > > > >> > > people into that experience.
> > > > > > >> > > > > > >> > > > > >> > > 2. Align releases so if you get
> > Kafkza
> > > > > 1.4.2
> > > > > > >> (or
> > > > > > >> > > > > > whatever)
> > > > > > >> > > > > > >> > that
> > > > > > >> > > > > > >> > > > has
> > > > > > >> > > > > > >> > > > > >> both
> > > > > > >> > > > > > >> > > > > >> > > Kafka and the stream processing
> part
> > > and
> > > > > > they
> > > > > > >> > > > actually
> > > > > > >> > > > > > work
> > > > > > >> > > > > > >> > > > > together.
> > > > > > >> > > > > > >> > > > > >> > > 3. Unify the programming
> experience
> > so
> > > > the
> > > > > > >> client
> > > > > > >> > > and
> > > > > > >> > > > > > Samza
> > > > > > >> > > > > > >> > api
> > > > > > >> > > > > > >> > > > > share
> > > > > > >> > > > > > >> > > > > >> > >
> > > config/monitoring/naming/packaging/etc.
> > > > > > >> > > > > > >> > > > > >> > >
> > > > > > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> > > > > > committers
> > > > > > >> and
> > > > > > >> > > can
> > > > > > >> > > > > > have a
> > > > > > >> > > > > > >> > > > > separate
> > > > > > >> > > > > > >> > > > > >> > repo,
> > > > > > >> > > > > > >> > > > > >> > > but I'm actually not really sure
> (I
> > > > can't
> > > > > > >> find a
> > > > > > >> > > > > > definition
> > > > > > >> > > > > > >> > of a
> > > > > > >> > > > > > >> > > > > >> > subproject
> > > > > > >> > > > > > >> > > > > >> > > in Apache).
> > > > > > >> > > > > > >> > > > > >> > >
> > > > > > >> > > > > > >> > > > > >> > > Basically at a high-level you want
> > the
> > > > > > >> experience
> > > > > > >> > > to
> > > > > > >> > > > > > "feel"
> > > > > > >> > > > > > >> > > like a
> > > > > > >> > > > > > >> > > > > >> single
> > > > > > >> > > > > > >> > > > > >> > > system, not to relatively
> > independent
> > > > > things
> > > > > > >> that
> > > > > > >> > > are
> > > > > > >> > > > > > kind
> > > > > > >> > > > > > >> of
> > > > > > >> > > > > > >> > > > > >> awkwardly
> > > > > > >> > > > > > >> > > > > >> > > glued together.
> > > > > > >> > > > > > >> > > > > >> > >
> > > > > > >> > > > > > >> > > > > >> > > I think if we did that they having
> > > > naming
> > > > > or
> > > > > > >> > > branding
> > > > > > >> > > > > > like
> > > > > > >> > > > > > >> > > "kafka
> > > > > > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or
> > > > something
> > > > > > >> like
> > > > > > >> > > that
> > > > > > >> > > > > > would
> > > > > > >> > > > > > >> > > > actually
> > > > > > >> > > > > > >> > > > > >> do a
> > > > > > >> > > > > > >> > > > > >> > > good job of conveying what it is.
> I
> > do
> > > > > that
> > > > > > >> this
> > > > > > >> > > > would
> > > > > > >> > > > > > help
> > > > > > >> > > > > > >> > > > adoption
> > > > > > >> > > > > > >> > > > > >> > quite
> > > > > > >> > > > > > >> > > > > >> > > a lot as it would correctly convey
> > > that
> > > > > > using
> > > > > > >> > Kafka
> > > > > > >> > > > > > >> Streaming
> > > > > > >> > > > > > >> > > with
> > > > > > >> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > > >> > > > > >> > is
> > > > > > >> > > > > > >> > > > > >> > > a fairly seamless experience and
> > Kafka
> > > > is
> > > > > > >> pretty
> > > > > > >> > > > > heavily
> > > > > > >> > > > > > >> > adopted
> > > > > > >> > > > > > >> > > > at
> > > > > > >> > > > > > >> > > > > >> this
> > > > > > >> > > > > > >> > > > > >> > > point.
> > > > > > >> > > > > > >> > > > > >> > >
> > > > > > >> > > > > > >> > > > > >> > > Fwiw we actually considered this
> > model
> > > > > > >> originally
> > > > > > >> > > > when
> > > > > > >> > > > > > open
> > > > > > >> > > > > > >> > > > sourcing
> > > > > > >> > > > > > >> > > > > >> > Samza,
> > > > > > >> > > > > > >> > > > > >> > > however at that time Kafka was
> > > > relatively
> > > > > > >> unknown
> > > > > > >> > > and
> > > > > > >> > > > > we
> > > > > > >> > > > > > >> > decided
> > > > > > >> > > > > > >> > > > not
> > > > > > >> > > > > > >> > > > > >> to
> > > > > > >> > > > > > >> > > > > >> > do
> > > > > > >> > > > > > >> > > > > >> > > it since we felt it would be
> > limiting.
> > > > > From
> > > > > > my
> > > > > > >> > > point
> > > > > > >> > > > of
> > > > > > >> > > > > > view
> > > > > > >> > > > > > >> > the
> > > > > > >> > > > > > >> > > > > three
> > > > > > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is
> now
> > > > > really
> > > > > > >> > heavily
> > > > > > >> > > > > used
> > > > > > >> > > > > > for
> > > > > > >> > > > > > >> > > > stream
> > > > > > >> > > > > > >> > > > > >> > > processing, (2) we learned that
> > > > > abstracting
> > > > > > >> out
> > > > > > >> > the
> > > > > > >> > > > > > stream
> > > > > > >> > > > > > >> > well
> > > > > > >> > > > > > >> > > is
> > > > > > >> > > > > > >> > > > > >> > > basically impossible, (3) we
> learned
> > > it
> > > > is
> > > > > > >> really
> > > > > > >> > > > hard
> > > > > > >> > > > > to
> > > > > > >> > > > > > >> keep
> > > > > > >> > > > > > >> > > the
> > > > > > >> > > > > > >> > > > > two
> > > > > > >> > > > > > >> > > > > >> > > things feeling like a single
> > product.
> > > > > > >> > > > > > >> > > > > >> > >
> > > > > > >> > > > > > >> > > > > >> > > -Jay
> > > > > > >> > > > > > >> > > > > >> > >
> > > > > > >> > > > > > >> > > > > >> > >
> > > > > > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM,
> > Martin
> > > > > > >> Kleppmann
> > > > > > >> > <
> > > > > > >> > > > > > >> > > > > >> martin@kleppmann.com>
> > > > > > >> > > > > > >> > > > > >> > > wrote:
> > > > > > >> > > > > > >> > > > > >> > >
> > > > > > >> > > > > > >> > > > > >> > >> Hi all,
> > > > > > >> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > > > > > >> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > > >> > > > > >> > >> I agree with the general
> philosophy
> > > of
> > > > > > tying
> > > > > > >> > Samza
> > > > > > >> > > > > more
> > > > > > >> > > > > > >> > firmly
> > > > > > >> > > > > > >> > > to
> > > > > > >> > > > > > >> > > > > >> Kafka.
> > > > > > >> > > > > > >> > > > > >> > >> After I spent a while looking at
> > > > > > integrating
> > > > > > >> > other
> > > > > > >> > > > > > message
> > > > > > >> > > > > > >> > > > brokers
> > > > > > >> > > > > > >> > > > > >> (e.g.
> > > > > > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I
> > came
> > > to
> > > > > the
> > > > > > >> > > > conclusion
> > > > > > >> > > > > > that
> > > > > > >> > > > > > >> > > > > >> > SystemConsumer
> > > > > > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much
> > like
> > > > > > Kafka's
> > > > > > >> > that
> > > > > > >> > > > > pretty
> > > > > > >> > > > > > >> much
> > > > > > >> > > > > > >> > > > > nobody
> > > > > > >> > > > > > >> > > > > >> but
> > > > > > >> > > > > > >> > > > > >> > >> Kafka actually implements it.
> > > (Databus
> > > > is
> > > > > > >> > perhaps
> > > > > > >> > > an
> > > > > > >> > > > > > >> > exception,
> > > > > > >> > > > > > >> > > > but
> > > > > > >> > > > > > >> > > > > >> it
> > > > > > >> > > > > > >> > > > > >> > >> isn't widely used outside of
> > > LinkedIn.)
> > > > > > Thus,
> > > > > > >> > > making
> > > > > > >> > > > > > Samza
> > > > > > >> > > > > > >> > > fully
> > > > > > >> > > > > > >> > > > > >> > dependent
> > > > > > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > > > > > >> > system-independence
> > > > > > >> > > > was
> > > > > > >> > > > > > >> never
> > > > > > >> > > > > > >> > as
> > > > > > >> > > > > > >> > > > > real
> > > > > > >> > > > > > >> > > > > >> as
> > > > > > >> > > > > > >> > > > > >> > we
> > > > > > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The
> > gains
> > > of
> > > > > > code
> > > > > > >> > reuse
> > > > > > >> > > > are
> > > > > > >> > > > > > >> real.
> > > > > > >> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from
> > > YARN
> > > > > has
> > > > > > >> also
> > > > > > >> > > > always
> > > > > > >> > > > > > been
> > > > > > >> > > > > > >> > > > > >> appealing to
> > > > > > >> > > > > > >> > > > > >> > >> me, for various reasons already
> > > > mentioned
> > > > > > in
> > > > > > >> > this
> > > > > > >> > > > > > thread.
> > > > > > >> > > > > > >> > > > Although
> > > > > > >> > > > > > >> > > > > >> > making
> > > > > > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > > > > > >> > > > (YARN/Mesos/AWS/etc)
> > > > > > >> > > > > > >> seems
> > > > > > >> > > > > > >> > > > > >> laudable,
> > > > > > >> > > > > > >> > > > > >> > I am
> > > > > > >> > > > > > >> > > > > >> > >> a little concerned that it will
> > > > restrict
> > > > > us
> > > > > > >> to a
> > > > > > >> > > > > lowest
> > > > > > >> > > > > > >> > common
> > > > > > >> > > > > > >> > > > > >> > denominator.
> > > > > > >> > > > > > >> > > > > >> > >> For example, would host affinity
> > > > > > (SAMZA-617)
> > > > > > >> > still
> > > > > > >> > > > be
> > > > > > >> > > > > > >> > possible?
> > > > > > >> > > > > > >> > > > For
> > > > > > >> > > > > > >> > > > > >> jobs
> > > > > > >> > > > > > >> > > > > >> > >> with large amounts of state, I
> > think
> > > > > > >> SAMZA-617
> > > > > > >> > > would
> > > > > > >> > > > > be
> > > > > > >> > > > > > a
> > > > > > >> > > > > > >> big
> > > > > > >> > > > > > >> > > > boon,
> > > > > > >> > > > > > >> > > > > >> > since
> > > > > > >> > > > > > >> > > > > >> > >> restoring state off the changelog
> > on
> > > > > every
> > > > > > >> > single
> > > > > > >> > > > > > restart
> > > > > > >> > > > > > >> is
> > > > > > >> > > > > > >> > > > > painful,
> > > > > > >> > > > > > >> > > > > >> > due
> > > > > > >> > > > > > >> > > > > >> > >> to long recovery times. It would
> > be a
> > > > > shame
> > > > > > >> if
> > > > > > >> > the
> > > > > > >> > > > > > >> decoupling
> > > > > > >> > > > > > >> > > > from
> > > > > > >> > > > > > >> > > > > >> YARN
> > > > > > >> > > > > > >> > > > > >> > >> made host affinity impossible.
> > > > > > >> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > > >> > > > > >> > >> Jay, a question about the
> proposed
> > > API
> > > > > for
> > > > > > >> > > > > > instantiating a
> > > > > > >> > > > > > >> > job
> > > > > > >> > > > > > >> > > in
> > > > > > >> > > > > > >> > > > > >> code
> > > > > > >> > > > > > >> > > > > >> > >> (rather than a properties file):
> > when
> > > > > > >> > submitting a
> > > > > > >> > > > job
> > > > > > >> > > > > > to a
> > > > > > >> > > > > > >> > > > > cluster,
> > > > > > >> > > > > > >> > > > > >> is
> > > > > > >> > > > > > >> > > > > >> > the
> > > > > > >> > > > > > >> > > > > >> > >> idea that the instantiation code
> > runs
> > > > on
> > > > > a
> > > > > > >> > client
> > > > > > >> > > > > > >> somewhere,
> > > > > > >> > > > > > >> > > > which
> > > > > > >> > > > > > >> > > > > >> then
> > > > > > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > > > > > >> > > YARN/Mesos/AWS/etc?
> > > > > > >> > > > > Or
> > > > > > >> > > > > > >> does
> > > > > > >> > > > > > >> > > that
> > > > > > >> > > > > > >> > > > > >> code
> > > > > > >> > > > > > >> > > > > >> > run
> > > > > > >> > > > > > >> > > > > >> > >> on each container that is part of
> > the
> > > > job
> > > > > > (in
> > > > > > >> > > which
> > > > > > >> > > > > > case,
> > > > > > >> > > > > > >> how
> > > > > > >> > > > > > >> > > > does
> > > > > > >> > > > > > >> > > > > >> the
> > > > > > >> > > > > > >> > > > > >> > job
> > > > > > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > > > > > >> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > > >> > > > > >> > >> I agree with Garry that it
> doesn't
> > > feel
> > > > > > >> right to
> > > > > > >> > > > make
> > > > > > >> > > > > a
> > > > > > >> > > > > > 1.0
> > > > > > >> > > > > > >> > > > release
> > > > > > >> > > > > > >> > > > > >> > with a
> > > > > > >> > > > > > >> > > > > >> > >> plan for it to be immediately
> > > obsolete.
> > > > > So
> > > > > > if
> > > > > > >> > this
> > > > > > >> > > > is
> > > > > > >> > > > > > going
> > > > > > >> > > > > > >> > to
> > > > > > >> > > > > > >> > > > > >> happen, I
> > > > > > >> > > > > > >> > > > > >> > >> think it would be more honest to
> > > stick
> > > > > with
> > > > > > >> 0.*
> > > > > > >> > > > > version
> > > > > > >> > > > > > >> > numbers
> > > > > > >> > > > > > >> > > > > until
> > > > > > >> > > > > > >> > > > > >> > the
> > > > > > >> > > > > > >> > > > > >> > >> library-ified Samza has been
> > > > implemented,
> > > > > > is
> > > > > > >> > > stable
> > > > > > >> > > > > and
> > > > > > >> > > > > > >> > widely
> > > > > > >> > > > > > >> > > > > used.
> > > > > > >> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > > >> > > > > >> > >> Should the new Samza be a
> > subproject
> > > of
> > > > > > >> Kafka?
> > > > > > >> > > There
> > > > > > >> > > > > is
> > > > > > >> > > > > > >> > > precedent
> > > > > > >> > > > > > >> > > > > for
> > > > > > >> > > > > > >> > > > > >> > >> tight coupling between different
> > > Apache
> > > > > > >> projects
> > > > > > >> > > > (e.g.
> > > > > > >> > > > > > >> > Curator
> > > > > > >> > > > > > >> > > > and
> > > > > > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN),
> so
> > I
> > > > > think
> > > > > > >> > > remaining
> > > > > > >> > > > > > >> separate
> > > > > > >> > > > > > >> > > > would
> > > > > > >> > > > > > >> > > > > >> be
> > > > > > >> > > > > > >> > > > > >> > ok.
> > > > > > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent
> on
> > > > > Kafka,
> > > > > > >> there
> > > > > > >> > > is
> > > > > > >> > > > > > enough
> > > > > > >> > > > > > >> > > > > substance
> > > > > > >> > > > > > >> > > > > >> in
> > > > > > >> > > > > > >> > > > > >> > >> Samza that it warrants being a
> > > separate
> > > > > > >> project.
> > > > > > >> > > An
> > > > > > >> > > > > > >> argument
> > > > > > >> > > > > > >> > in
> > > > > > >> > > > > > >> > > > > >> favour
> > > > > > >> > > > > > >> > > > > >> > of
> > > > > > >> > > > > > >> > > > > >> > >> merging would be if we think
> Kafka
> > > has
> > > > a
> > > > > > much
> > > > > > >> > > > stronger
> > > > > > >> > > > > > >> "brand
> > > > > > >> > > > > > >> > > > > >> presence"
> > > > > > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on
> that
> > > one.
> > > > > If
> > > > > > >> the
> > > > > > >> > > Kafka
> > > > > > >> > > > > > >> project
> > > > > > >> > > > > > >> > is
> > > > > > >> > > > > > >> > > > > >> willing
> > > > > > >> > > > > > >> > > > > >> > to
> > > > > > >> > > > > > >> > > > > >> > >> endorse Samza as the "official"
> way
> > > of
> > > > > > doing
> > > > > > >> > > > stateful
> > > > > > >> > > > > > >> stream
> > > > > > >> > > > > > >> > > > > >> > >> transformations, that would
> > probably
> > > > have
> > > > > > >> much
> > > > > > >> > the
> > > > > > >> > > > > same
> > > > > > >> > > > > > >> > effect
> > > > > > >> > > > > > >> > > as
> > > > > > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka
> Stream
> > > > > > >> Processors"
> > > > > > >> > or
> > > > > > >> > > > > > suchlike.
> > > > > > >> > > > > > >> > > Close
> > > > > > >> > > > > > >> > > > > >> > >> collaboration between the two
> > > projects
> > > > > will
> > > > > > >> be
> > > > > > >> > > > needed
> > > > > > >> > > > > in
> > > > > > >> > > > > > >> any
> > > > > > >> > > > > > >> > > > case.
> > > > > > >> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > > >> > > > > >> > >> From a project management
> > > perspective,
> > > > I
> > > > > > >> guess
> > > > > > >> > the
> > > > > > >> > > > > "new
> > > > > > >> > > > > > >> > Samza"
> > > > > > >> > > > > > >> > > > > would
> > > > > > >> > > > > > >> > > > > >> > have
> > > > > > >> > > > > > >> > > > > >> > >> to be developed on a branch
> > alongside
> > > > > > ongoing
> > > > > > >> > > > > > maintenance
> > > > > > >> > > > > > >> of
> > > > > > >> > > > > > >> > > the
> > > > > > >> > > > > > >> > > > > >> current
> > > > > > >> > > > > > >> > > > > >> > >> line of development? I think it
> > would
> > > > be
> > > > > > >> > important
> > > > > > >> > > > to
> > > > > > >> > > > > > >> > continue
> > > > > > >> > > > > > >> > > > > >> > supporting
> > > > > > >> > > > > > >> > > > > >> > >> existing users, and provide a
> > > graceful
> > > > > > >> migration
> > > > > > >> > > > path
> > > > > > >> > > > > to
> > > > > > >> > > > > > >> the
> > > > > > >> > > > > > >> > > new
> > > > > > >> > > > > > >> > > > > >> > version.
> > > > > > >> > > > > > >> > > > > >> > >> Leaving the current versions
> > > > unsupported
> > > > > > and
> > > > > > >> > > forcing
> > > > > > >> > > > > > people
> > > > > > >> > > > > > >> > to
> > > > > > >> > > > > > >> > > > > >> rewrite
> > > > > > >> > > > > > >> > > > > >> > >> their jobs would send a bad
> signal.
> > > > > > >> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > > >> > > > > >> > >> Best,
> > > > > > >> > > > > > >> > > > > >> > >> Martin
> > > > > > >> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay
> Kreps
> > <
> > > > > > >> > > > jay@confluent.io>
> > > > > > >> > > > > > >> wrote:
> > > > > > >> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > > >> > > > > >> > >>> Hey Garry,
> > > > > > >> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating.
> I'd
> > > be
> > > > > > happy
> > > > > > >> to
> > > > > > >> > > chat
> > > > > > >> > > > > > more
> > > > > > >> > > > > > >> > about
> > > > > > >> > > > > > >> > > > > this
> > > > > > >> > > > > > >> > > > > >> if
> > > > > > >> > > > > > >> > > > > >> > >>> you'd be interested. I think
> Chris
> > > > and I
> > > > > > >> > started
> > > > > > >> > > > with
> > > > > > >> > > > > > the
> > > > > > >> > > > > > >> > idea
> > > > > > >> > > > > > >> > > > of
> > > > > > >> > > > > > >> > > > > >> "what
> > > > > > >> > > > > > >> > > > > >> > >>> would it take to make Samza a
> > > kick-ass
> > > > > > >> > ingestion
> > > > > > >> > > > > tool"
> > > > > > >> > > > > > but
> > > > > > >> > > > > > >> > > > > >> ultimately
> > > > > > >> > > > > > >> > > > > >> > we
> > > > > > >> > > > > > >> > > > > >> > >>> kind of came around to the idea
> > that
> > > > > > >> ingestion
> > > > > > >> > > and
> > > > > > >> > > > > > >> > > > transformation
> > > > > > >> > > > > > >> > > > > >> had
> > > > > > >> > > > > > >> > > > > >> > >>> pretty different needs and
> > coupling
> > > > the
> > > > > > two
> > > > > > >> > made
> > > > > > >> > > > > things
> > > > > > >> > > > > > >> > hard.
> > > > > > >> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > > >> > > > > >> > >>> For what it's worth I think
> > copycat
> > > > > > (KIP-26)
> > > > > > >> > > > actually
> > > > > > >> > > > > > will
> > > > > > >> > > > > > >> > do
> > > > > > >> > > > > > >> > > > what
> > > > > > >> > > > > > >> > > > > >> you
> > > > > > >> > > > > > >> > > > > >> > >> are
> > > > > > >> > > > > > >> > > > > >> > >>> looking for.
> > > > > > >> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > > >> > > > > >> > >>> With regard to your point about
> > > > slider,
> > > > > I
> > > > > > >> don't
> > > > > > >> > > > > > >> necessarily
> > > > > > >> > > > > > >> > > > > >> disagree.
> > > > > > >> > > > > > >> > > > > >> > >> But I
> > > > > > >> > > > > > >> > > > > >> > >>> think getting good YARN support
> is
> > > > quite
> > > > > > >> doable
> > > > > > >> > > > and I
> > > > > > >> > > > > > >> think
> > > > > > >> > > > > > >> > we
> > > > > > >> > > > > > >> > > > can
> > > > > > >> > > > > > >> > > > > >> make
> > > > > > >> > > > > > >> > > > > >> > >>> that work well. I think the
> issue
> > > this
> > > > > > >> proposal
> > > > > > >> > > > > solves
> > > > > > >> > > > > > is
> > > > > > >> > > > > > >> > that
> > > > > > >> > > > > > >> > > > > >> > >> technically
> > > > > > >> > > > > > >> > > > > >> > >>> it is pretty hard to support
> > > multiple
> > > > > > >> cluster
> > > > > > >> > > > > > management
> > > > > > >> > > > > > >> > > systems
> > > > > > >> > > > > > >> > > > > the
> > > > > > >> > > > > > >> > > > > >> > way
> > > > > > >> > > > > > >> > > > > >> > >>> things are now, you need to
> write
> > an
> > > > > "app
> > > > > > >> > master"
> > > > > > >> > > > or
> > > > > > >> > > > > > >> > > "framework"
> > > > > > >> > > > > > >> > > > > for
> > > > > > >> > > > > > >> > > > > >> > each
> > > > > > >> > > > > > >> > > > > >> > >>> and they are all a little
> > different
> > > so
> > > > > > >> testing
> > > > > > >> > is
> > > > > > >> > > > > > really
> > > > > > >> > > > > > >> > hard.
> > > > > > >> > > > > > >> > > > In
> > > > > > >> > > > > > >> > > > > >> the
> > > > > > >> > > > > > >> > > > > >> > >>> absence of this we have been
> stuck
> > > > with
> > > > > > just
> > > > > > >> > YARN
> > > > > > >> > > > > which
> > > > > > >> > > > > > >> has
> > > > > > >> > > > > > >> > > > > >> fantastic
> > > > > > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part
> of
> > > the
> > > > > > org,
> > > > > > >> but
> > > > > > >> > > > zero
> > > > > > >> > > > > > >> > > penetration
> > > > > > >> > > > > > >> > > > > >> > >> elsewhere.
> > > > > > >> > > > > > >> > > > > >> > >>> Given the huge amount of work
> > being
> > > > put
> > > > > in
> > > > > > >> to
> > > > > > >> > > > slider,
> > > > > > >> > > > > > >> > > marathon,
> > > > > > >> > > > > > >> > > > > aws
> > > > > > >> > > > > > >> > > > > >> > >>> tooling, not to mention the
> > umpteen
> > > > > > related
> > > > > > >> > > > packaging
> > > > > > >> > > > > > >> > > > technologies
> > > > > > >> > > > > > >> > > > > >> > people
> > > > > > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes,
> > > > various
> > > > > > >> > > > > cloud-specific
> > > > > > >> > > > > > >> > deploy
> > > > > > >> > > > > > >> > > > > >> tools,
> > > > > > >> > > > > > >> > > > > >> > >> etc)
> > > > > > >> > > > > > >> > > > > >> > >>> I really think it is important
> to
> > > get
> > > > > this
> > > > > > >> > right.
> > > > > > >> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > > >> > > > > >> > >>> -Jay
> > > > > > >> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM,
> > > Garry
> > > > > > >> > Turkington
> > > > > > >> > > <
> > > > > > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com
> >
> > > > wrote:
> > > > > > >> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > > >> > > > > >> > >>>> Hi all,
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>> I think the question below re
> > does
> > > > > Samza
> > > > > > >> > become
> > > > > > >> > > a
> > > > > > >> > > > > > >> > sub-project
> > > > > > >> > > > > > >> > > > of
> > > > > > >> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > > >> > > > > >> > >>>> highlights the broader point
> > around
> > > > > > >> migration.
> > > > > > >> > > > Chris
> > > > > > >> > > > > > >> > mentions
> > > > > > >> > > > > > >> > > > > >> Samza's
> > > > > > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a
> v1
> > > > > release
> > > > > > >> but
> > > > > > >> > I'm
> > > > > > >> > > > not
> > > > > > >> > > > > > sure
> > > > > > >> > > > > > >> > it
> > > > > > >> > > > > > >> > > > > feels
> > > > > > >> > > > > > >> > > > > >> > >> right to
> > > > > > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately
> plan
> > > to
> > > > > > >> deprecate
> > > > > > >> > > > most
> > > > > > >> > > > > of
> > > > > > >> > > > > > >> it.
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I
> have
> > > > some
> > > > > > guys
> > > > > > >> > who
> > > > > > >> > > > have
> > > > > > >> > > > > > >> > started
> > > > > > >> > > > > > >> > > > > >> working
> > > > > > >> > > > > > >> > > > > >> > >> with
> > > > > > >> > > > > > >> > > > > >> > >>>> Samza and building some new
> > > > > > >> > consumers/producers
> > > > > > >> > > > was
> > > > > > >> > > > > > next
> > > > > > >> > > > > > >> > up.
> > > > > > >> > > > > > >> > > > > Sounds
> > > > > > >> > > > > > >> > > > > >> > like
> > > > > > >> > > > > > >> > > > > >> > >>>> that is absolutely not the
> > > direction
> > > > to
> > > > > > >> go. I
> > > > > > >> > > need
> > > > > > >> > > > > to
> > > > > > >> > > > > > >> look
> > > > > > >> > > > > > >> > > into
> > > > > > >> > > > > > >> > > > > the
> > > > > > >> > > > > > >> > > > > >> > KIP
> > > > > > >> > > > > > >> > > > > >> > >> in
> > > > > > >> > > > > > >> > > > > >> > >>>> more detail but for me the
> > > > > attractiveness
> > > > > > >> of
> > > > > > >> > > > adding
> > > > > > >> > > > > > new
> > > > > > >> > > > > > >> > Samza
> > > > > > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if
> yes
> > > all
> > > > > > they
> > > > > > >> > were
> > > > > > >> > > > > doing
> > > > > > >> > > > > > was
> > > > > > >> > > > > > >> > > > really
> > > > > > >> > > > > > >> > > > > >> > getting
> > > > > > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --
> > was
> > > to
> > > > > > avoid
> > > > > > >> > > > having
> > > > > > >> > > > > to
> > > > > > >> > > > > > >> > worry
> > > > > > >> > > > > > >> > > > > about
> > > > > > >> > > > > > >> > > > > >> the
> > > > > > >> > > > > > >> > > > > >> > >>>> lifecycle management of
> external
> > > > > clients.
> > > > > > >> If
> > > > > > >> > > there
> > > > > > >> > > > > is
> > > > > > >> > > > > > a
> > > > > > >> > > > > > >> > > generic
> > > > > > >> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can
> > > plug
> > > > a
> > > > > > new
> > > > > > >> > > > connector
> > > > > > >> > > > > > into
> > > > > > >> > > > > > >> > and
> > > > > > >> > > > > > >> > > > > have
> > > > > > >> > > > > > >> > > > > >> a
> > > > > > >> > > > > > >> > > > > >> > >> lot of
> > > > > > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> > > > > > reliability
> > > > > > >> > done
> > > > > > >> > > > for
> > > > > > >> > > > > me
> > > > > > >> > > > > > >> then
> > > > > > >> > > > > > >> > > it
> > > > > > >> > > > > > >> > > > > >> gives
> > > > > > >> > > > > > >> > > > > >> > me
> > > > > > >> > > > > > >> > > > > >> > >> all
> > > > > > >> > > > > > >> > > > > >> > >>>> the pushing new
> > consumers/producers
> > > > > > would.
> > > > > > >> If
> > > > > > >> > > not
> > > > > > >> > > > > > then it
> > > > > > >> > > > > > >> > > > > >> complicates
> > > > > > >> > > > > > >> > > > > >> > my
> > > > > > >> > > > > > >> > > > > >> > >>>> operational deployments.
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>> Which is similar to my other
> > > question
> > > > > > with
> > > > > > >> the
> > > > > > >> > > > > > proposal
> > > > > > >> > > > > > >> --
> > > > > > >> > > > > > >> > if
> > > > > > >> > > > > > >> > > > we
> > > > > > >> > > > > > >> > > > > >> > build a
> > > > > > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone
> Samza
> > > > plus
> > > > > > the
> > > > > > >> > > > requisite
> > > > > > >> > > > > > >> shims
> > > > > > >> > > > > > >> > to
> > > > > > >> > > > > > >> > > > > >> > integrate
> > > > > > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the
> > > former
> > > > > may
> > > > > > >> be a
> > > > > > >> > > lot
> > > > > > >> > > > > more
> > > > > > >> > > > > > >> work
> > > > > > >> > > > > > >> > > > than
> > > > > > >> > > > > > >> > > > > we
> > > > > > >> > > > > > >> > > > > >> > >> think.
> > > > > > >> > > > > > >> > > > > >> > >>>> We may make it much easier for
> a
> > > > > newcomer
> > > > > > >> to
> > > > > > >> > get
> > > > > > >> > > > > > >> something
> > > > > > >> > > > > > >> > > > > running
> > > > > > >> > > > > > >> > > > > >> but
> > > > > > >> > > > > > >> > > > > >> > >>>> having them step up and get a
> > > > reliable
> > > > > > >> > > production
> > > > > > >> > > > > > >> > deployment
> > > > > > >> > > > > > >> > > > may
> > > > > > >> > > > > > >> > > > > >> still
> > > > > > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic,
> > if
> > > > for
> > > > > > >> > different
> > > > > > >> > > > > > reasons
> > > > > > >> > > > > > >> > than
> > > > > > >> > > > > > >> > > > > >> today.
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm
> > > comfortable
> > > > > > with
> > > > > > >> > > making
> > > > > > >> > > > > the
> > > > > > >> > > > > > >> Samza
> > > > > > >> > > > > > >> > > > > >> dependency
> > > > > > >> > > > > > >> > > > > >> > >> on
> > > > > > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
> > > > > absolutely
> > > > > > >> see
> > > > > > >> > > the
> > > > > > >> > > > > > >> benefits
> > > > > > >> > > > > > >> > > in
> > > > > > >> > > > > > >> > > > > the
> > > > > > >> > > > > > >> > > > > >> > >>>> reduction of duplication and
> > > clashing
> > > > > > >> > > > > > >> > > > terminologies/abstractions
> > > > > > >> > > > > > >> > > > > >> that
> > > > > > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a
> > > > library
> > > > > > >> would
> > > > > > >> > > > likely
> > > > > > >> > > > > > be a
> > > > > > >> > > > > > >> > very
> > > > > > >> > > > > > >> > > > > nice
> > > > > > >> > > > > > >> > > > > >> > tool
> > > > > > >> > > > > > >> > > > > >> > >> to
> > > > > > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I
> > just
> > > > have
> > > > > > the
> > > > > > >> > > > concerns
> > > > > > >> > > > > > >> above
> > > > > > >> > > > > > >> > re
> > > > > > >> > > > > > >> > > > the
> > > > > > >> > > > > > >> > > > > >> > >>>> operational side.
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>> Garry
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > > > > > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci
> > > Morales
> > > > > > >> [mailto:
> > > > > > >> > > > > > >> > gdfm@apache.org
> > > > > > >> > > > > > >> > > ]
> > > > > > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > > > > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > > > > > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and
> > > > obesrvations
> > > > > on
> > > > > > >> > Samza
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > > > > > >> > > > > > >> > > > > >> > >>>> From outside, I have always
> > > perceived
> > > > > > Samza
> > > > > > >> > as a
> > > > > > >> > > > > > >> computing
> > > > > > >> > > > > > >> > > > layer
> > > > > > >> > > > > > >> > > > > >> over
> > > > > > >> > > > > > >> > > > > >> > >>>> Kafka.
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit
> > > > provocative,
> > > > > is
> > > > > > >> > > "should
> > > > > > >> > > > > > Samza
> > > > > > >> > > > > > >> be
> > > > > > >> > > > > > >> > a
> > > > > > >> > > > > > >> > > > > >> > sub-project
> > > > > > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > > > > > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep
> it
> > > as a
> > > > > > >> separate
> > > > > > >> > > > > project
> > > > > > >> > > > > > >> > with a
> > > > > > >> > > > > > >> > > > > >> separate
> > > > > > >> > > > > > >> > > > > >> > >>>> governance?
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>> Cheers,
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>> --
> > > > > > >> > > > > > >> > > > > >> > >>>> Gianmarco
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan
> > Fang <
> > > > > > >> > > > > > yanfang724@gmail.com>
> > > > > > >> > > > > > >> > > > wrote:
> > > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple
> with
> > > > Kafka
> > > > > > more
> > > > > > >> > > > tightly.
> > > > > > >> > > > > > >> > Because
> > > > > > >> > > > > > >> > > > > Samza
> > > > > > >> > > > > > >> > > > > >> de
> > > > > > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and
> it
> > > > should
> > > > > > >> > leverage
> > > > > > >> > > > > what
> > > > > > >> > > > > > >> Kafka
> > > > > > >> > > > > > >> > > > has.
> > > > > > >> > > > > > >> > > > > At
> > > > > > >> > > > > > >> > > > > >> > the
> > > > > > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need
> > to
> > > > > > reinvent
> > > > > > >> > what
> > > > > > >> > > > > Samza
> > > > > > >> > > > > > >> > > already
> > > > > > >> > > > > > >> > > > > >> has. I
> > > > > > >> > > > > > >> > > > > >> > >>>>> also like the idea of
> separating
> > > the
> > > > > > >> > ingestion
> > > > > > >> > > > and
> > > > > > >> > > > > > >> > > > > transformation.
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult
> for
> > > me
> > > > to
> > > > > > >> image
> > > > > > >> > > how
> > > > > > >> > > > > the
> > > > > > >> > > > > > >> Samza
> > > > > > >> > > > > > >> > > > will
> > > > > > >> > > > > > >> > > > > >> look
> > > > > > >> > > > > > >> > > > > >> > >>>> like.
> > > > > > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have
> a
> > > > little
> > > > > > >> > > difference
> > > > > > >> > > > > in
> > > > > > >> > > > > > >> terms
> > > > > > >> > > > > > >> > > of
> > > > > > >> > > > > > >> > > > > how
> > > > > > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what
> Jay's
> > > > code
> > > > > > >> shows
> > > > > > >> > (A
> > > > > > >> > > > > > client of
> > > > > > >> > > > > > >> > > > Kakfa)
> > > > > > >> > > > > > >> > > > > ?
> > > > > > >> > > > > > >> > > > > >> And
> > > > > > >> > > > > > >> > > > > >> > >>>>> user's application code calls
> > this
> > > > > > client?
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a
> library
> > > of
> > > > > > Kafka
> > > > > > >> > (like
> > > > > > >> > > > > what
> > > > > > >> > > > > > the
> > > > > > >> > > > > > >> > > code
> > > > > > >> > > > > > >> > > > > >> shows),
> > > > > > >> > > > > > >> > > > > >> > >>>>> how do we implement
> auto-balance
> > > and
> > > > > > >> > > > > fault-tolerance?
> > > > > > >> > > > > > >> Are
> > > > > > >> > > > > > >> > > they
> > > > > > >> > > > > > >> > > > > >> taken
> > > > > > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or
> > other
> > > > > > >> mechanism,
> > > > > > >> > > such
> > > > > > >> > > > > as
> > > > > > >> > > > > > >> > "Samza
> > > > > > >> > > > > > >> > > > > >> worker"
> > > > > > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>> 2. What about other features,
> > such
> > > > as
> > > > > > >> > > > auto-scaling,
> > > > > > >> > > > > > >> shared
> > > > > > >> > > > > > >> > > > > state,
> > > > > > >> > > > > > >> > > > > >> > >>>>> monitoring?
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza
> standalone,
> > > (is
> > > > > > this
> > > > > > >> > what
> > > > > > >> > > > > Chris
> > > > > > >> > > > > > >> > > > suggests?)
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest
> data
> > > from
> > > > > > Kakfa
> > > > > > >> > and
> > > > > > >> > > > > > produce
> > > > > > >> > > > > > >> to
> > > > > > >> > > > > > >> > > it.
> > > > > > >> > > > > > >> > > > > >> Then it
> > > > > > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza
> > > looks
> > > > > > like
> > > > > > >> > now,
> > > > > > >> > > > > > except it
> > > > > > >> > > > > > >> > > does
> > > > > > >> > > > > > >> > > > > not
> > > > > > >> > > > > > >> > > > > >> > rely
> > > > > > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how
> can
> > it
> > > > > > >> leverage
> > > > > > >> > > > Kafka's
> > > > > > >> > > > > > >> > metrics,
> > > > > > >> > > > > > >> > > > > logs,
> > > > > > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the
> > > > dependency?
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>> Thanks,
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > > > > > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46
> PM,
> > > > > Guozhang
> > > > > > >> > Wang <
> > > > > > >> > > > > > >> > > > > wangguoz@gmail.com
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>> Read through the code example
> > and
> > > > it
> > > > > > >> looks
> > > > > > >> > > good
> > > > > > >> > > > to
> > > > > > >> > > > > > me.
> > > > > > >> > > > > > >> A
> > > > > > >> > > > > > >> > > few
> > > > > > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding
> deployment:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as
> > executable
> > > > > > >> runnable
> > > > > > >> > > like:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > > > > > >> > > --config-factory=...
> > > > > > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate
> for
> > > > > > deploying
> > > > > > >> > Samza
> > > > > > >> > > > > more
> > > > > > >> > > > > > as
> > > > > > >> > > > > > >> > > > embedded
> > > > > > >> > > > > > >> > > > > >> > >>>>>> libraries in user application
> > > code
> > > > > > >> (ignoring
> > > > > > >> > > the
> > > > > > >> > > > > > >> > > terminology
> > > > > > >> > > > > > >> > > > > >> since
> > > > > > >> > > > > > >> > > > > >> > >>>>>> it is not the
> > > > > > >> > > > > > >> > > > > >> > >>>>> same
> > > > > > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> > > > > > >> MyStreamTask(configs);
> > > > > > >> > > > > Thread
> > > > > > >> > > > > > >> > thread
> > > > > > >> > > > > > >> > > =
> > > > > > >> > > > > > >> > > > > new
> > > > > > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>> I think both of these
> > deployment
> > > > > modes
> > > > > > >> are
> > > > > > >> > > > > important
> > > > > > >> > > > > > >> for
> > > > > > >> > > > > > >> > > > > >> different
> > > > > > >> > > > > > >> > > > > >> > >>>>>> types
> > > > > > >> > > > > > >> > > > > >> > >>>>> of
> > > > > > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think
> > making
> > > > > Samza
> > > > > > >> > purely
> > > > > > >> > > > > > >> standalone
> > > > > > >> > > > > > >> > is
> > > > > > >> > > > > > >> > > > > still
> > > > > > >> > > > > > >> > > > > >> > >>>>>> sufficient for either
> runnable
> > or
> > > > > > library
> > > > > > >> > > modes.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>> Guozhang
> > > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33
> > PM,
> > > > Jay
> > > > > > >> Kreps
> > > > > > >> > <
> > > > > > >> > > > > > >> > > > jay@confluent.io>
> > > > > > >> > > > > > >> > > > > >> > wrote:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the
> > > code
> > > > > > >> example,
> > > > > > >> > it
> > > > > > >> > > > was
> > > > > > >> > > > > > >> > supposed
> > > > > > >> > > > > > >> > > > to
> > > > > > >> > > > > > >> > > > > >> look
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> this:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new
> > > > Properties();
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>
> props.put("bootstrap.servers",
> > > > > > >> > > > "localhost:4242");
> > > > > > >> > > > > > >> > > > > >> StreamingConfig
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> config = new
> > > > StreamingConfig(props);
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > config.subscribe("test-topic-1",
> > > > > > >> > > > "test-topic-2");
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > config.processor(ExampleStreamProcessor.class);
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > > > > >> > StringSerializer(),
> > > > > > >> > > > new
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
> > > > > KafkaStreaming
> > > > > > >> > > > container =
> > > > > > >> > > > > > new
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> > > > > > container.run();
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> -Jay
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at
> 11:32
> > > PM,
> > > > > Jay
> > > > > > >> > Kreps <
> > > > > > >> > > > > > >> > > > jay@confluent.io
> > > > > > >> > > > > > >> > > > > >
> > > > > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some
> > > > conversations
> > > > > > >> Chris
> > > > > > >> > > and
> > > > > > >> > > > I
> > > > > > >> > > > > > were
> > > > > > >> > > > > > >> > > having
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> around
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> whether
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use
> > > Samza
> > > > > as a
> > > > > > >> kind
> > > > > > >> > > of
> > > > > > >> > > > > data
> > > > > > >> > > > > > >> > > > ingestion
> > > > > > >> > > > > > >> > > > > >> > >>>>> framework
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> for
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately
> lead
> > to
> > > > > > KIP-26
> > > > > > >> > > > > "copycat").
> > > > > > >> > > > > > >> This
> > > > > > >> > > > > > >> > > > kind
> > > > > > >> > > > > > >> > > > > of
> > > > > > >> > > > > > >> > > > > >> > >>>>>> combined
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around
> config
> > > and
> > > > > > YARN
> > > > > > >> and
> > > > > > >> > > the
> > > > > > >> > > > > > >> > discussion
> > > > > > >> > > > > > >> > > > > >> around
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> how
> > > > > > >> > > > > > >> > > > > >> > >>>>> to
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment
> > was,
> > > > > given
> > > > > > >> that
> > > > > > >> > > > Samza
> > > > > > >> > > > > > was
> > > > > > >> > > > > > >> > > > basically
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka
> > specific,
> > > > > what
> > > > > > if
> > > > > > >> > you
> > > > > > >> > > > just
> > > > > > >> > > > > > >> > embraced
> > > > > > >> > > > > > >> > > > > that
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > > > > > >> > > > > > >> > > > > >> > >>>>>> into
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> something less like a
> > > heavyweight
> > > > > > >> > framework
> > > > > > >> > > > and
> > > > > > >> > > > > > more
> > > > > > >> > > > > > >> > > like a
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> third
> > > > > > >> > > > > > >> > > > > >> > >>>>> Kafka
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of
> "producing
> > > > > > consumer"
> > > > > > >> > with
> > > > > > >> > > > > state
> > > > > > >> > > > > > >> > > > management
> > > > > > >> > > > > > >> > > > > >> > >>>>>> facilities.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library.
> Instead
> > > of a
> > > > > > >> complex
> > > > > > >> > > > stream
> > > > > > >> > > > > > >> > > processing
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> framework
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> this
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very
> > simple
> > > > > > thing,
> > > > > > >> not
> > > > > > >> > > > much
> > > > > > >> > > > > > more
> > > > > > >> > > > > > >> > > > > >> complicated
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > > >> > > > > > >> > > > > >> > >>>>> use
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> or
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka
> > consumer.
> > > As
> > > > > > Chris
> > > > > > >> > said
> > > > > > >> > > > we
> > > > > > >> > > > > > >> thought
> > > > > > >> > > > > > >> > > > about
> > > > > > >> > > > > > >> > > > > >> it
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> a
> > > > > > >> > > > > > >> > > > > >> > >>>>> lot
> > > > > > >> > > > > > >> > > > > >> > >>>>>> of
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other
> > > stream
> > > > > > >> > processing
> > > > > > >> > > > > > systems
> > > > > > >> > > > > > >> > were
> > > > > > >> > > > > > >> > > > > doing)
> > > > > > >> > > > > > >> > > > > >> > >>>>> seemed
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from
> > > > MapReduce.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to
> > > > ingest/output
> > > > > > >> data
> > > > > > >> > to
> > > > > > >> > > > and
> > > > > > >> > > > > > from
> > > > > > >> > > > > > >> > the
> > > > > > >> > > > > > >> > > > > stream
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we
> > > actually
> > > > > > looked
> > > > > > >> > into
> > > > > > >> > > > how
> > > > > > >> > > > > > that
> > > > > > >> > > > > > >> > > would
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> work,
> > > > > > >> > > > > > >> > > > > >> > >>>>> Samza
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
> > > > > ingestion
> > > > > > >> > > framework
> > > > > > >> > > > > > for a
> > > > > > >> > > > > > >> > > bunch
> > > > > > >> > > > > > >> > > > of
> > > > > > >> > > > > > >> > > > > >> > >>>>> reasons.
> > > > > > >> > > > > > >> > > > > >> > >>>>>> To
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you
> > need a
> > > > > > pretty
> > > > > > >> > > > different
> > > > > > >> > > > > > >> > internal
> > > > > > >> > > > > > >> > > > > data
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> model
> > > > > > >> > > > > > >> > > > > >> > >>>>>> and
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you
> > > split
> > > > > > them
> > > > > > >> and
> > > > > > >> > > had
> > > > > > >> > > > > an
> > > > > > >> > > > > > api
> > > > > > >> > > > > > >> > for
> > > > > > >> > > > > > >> > > > > Kafka
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA
> > > > KIP-26)
> > > > > > >> and a
> > > > > > >> > > > > separate
> > > > > > >> > > > > > >> api
> > > > > > >> > > > > > >> > > for
> > > > > > >> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow
> really
> > > > > > embracing
> > > > > > >> the
> > > > > > >> > > > same
> > > > > > >> > > > > > >> > > terminology
> > > > > > >> > > > > > >> > > > > and
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint
> > > about
> > > > > the
> > > > > > >> > current
> > > > > > >> > > > > > state is
> > > > > > >> > > > > > >> > > that
> > > > > > >> > > > > > >> > > > > the
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> two
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> systems
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on.
> > > > Terminology
> > > > > > >> like
> > > > > > >> > > > > "stream"
> > > > > > >> > > > > > vs
> > > > > > >> > > > > > >> > > > "topic"
> > > > > > >> > > > > > >> > > > > >> and
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> different
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring
> systems
> > > > means
> > > > > > you
> > > > > > >> > kind
> > > > > > >> > > > of
> > > > > > >> > > > > > have
> > > > > > >> > > > > > >> to
> > > > > > >> > > > > > >> > > > learn
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> way,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
> > > > > different
> > > > > > >> way,
> > > > > > >> > > > then
> > > > > > >> > > > > > kind
> > > > > > >> > > > > > >> of
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> understand
> > > > > > >> > > > > > >> > > > > >> > >>>>> how
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> they
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which
> > having
> > > > > > walked
> > > > > > >> a
> > > > > > >> > few
> > > > > > >> > > > > > people
> > > > > > >> > > > > > >> > > through
> > > > > > >> > > > > > >> > > > > >> this
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for
> > > folks
> > > > to
> > > > > > >> get.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending
> a
> > > lot
> > > > of
> > > > > > >> time
> > > > > > >> > on
> > > > > > >> > > > > > >> airplanes I
> > > > > > >> > > > > > >> > > > > hacked
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still
> > somewhat
> > > > > > >> incomplete
> > > > > > >> > > > > > prototype
> > > > > > >> > > > > > >> of
> > > > > > >> > > > > > >> > > > what
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> this would
> > > > > > >> > > > > > >> > > > > >> > >>>>> look
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just
> > > > unceremoniously
> > > > > > >> dumped
> > > > > > >> > > into
> > > > > > >> > > > > > Kafka
> > > > > > >> > > > > > >> as
> > > > > > >> > > > > > >> > > it
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> required a
> > > > > > >> > > > > > >> > > > > >> > >>>>>> few
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new
> consumer.
> > > Here
> > > > > is
> > > > > > >> the
> > > > > > >> > > code:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > >
> > > > > > >> > > > > > >> >
> > > > > > >> > > > > >
> > > > > > >> > >
> > > > > > >>
> > > > >
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> /apache/kafka/clients/streaming
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the
> > > prototype
> > > > I
> > > > > > just
> > > > > > >> > > > > liberally
> > > > > > >> > > > > > >> > renamed
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> everything
> > > > > > >> > > > > > >> > > > > >> > >>>>> to
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka
> > with
> > > > no
> > > > > > >> regard
> > > > > > >> > > for
> > > > > > >> > > > > > >> > > > compatibility.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be
> > something
> > > > like
> > > > > > >> this:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
> > > > > Properties();
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > props.put("bootstrap.servers",
> > > > > > >> > > > > "localhost:4242");
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config =
> new
> > > > > > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > config.subscribe("test-topic-1",
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > > > > > >> > > > > > >> > > > > >>
> > > > > config.processor(ExampleStreamProcessor.class);
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > > > > > >> > > StringDeserializer());
> > > > > > >> > > > > > >> > > > KafkaStreaming
> > > > > > >> > > > > > >> > > > > >> > >>>>>> container =
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > > > > > >> > container.run();
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically
> > the
> > > > > > >> > > > SamzaContainer;
> > > > > > >> > > > > > >> > > > > StreamProcessor
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all
> > the
> > > > > class
> > > > > > >> names
> > > > > > >> > > in
> > > > > > >> > > > a
> > > > > > >> > > > > > file
> > > > > > >> > > > > > >> > and
> > > > > > >> > > > > > >> > > > then
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> having
> > > > > > >> > > > > > >> > > > > >> > >>>>>> the
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by
> reflection,
> > > you
> > > > > just
> > > > > > >> > > > > instantiate
> > > > > > >> > > > > > the
> > > > > > >> > > > > > >> > > > > container
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is
> > > > balanced
> > > > > > over
> > > > > > >> > > > however
> > > > > > >> > > > > > many
> > > > > > >> > > > > > >> > > > > instances
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > > > > >> > > > > > >> > > > > >> > >>>>> this
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> are
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if
> an
> > > > > > instance
> > > > > > >> > dies,
> > > > > > >> > > > new
> > > > > > >> > > > > > >> tasks
> > > > > > >> > > > > > >> > > are
> > > > > > >> > > > > > >> > > > > >> added
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > > >> > > > > > >> > > > > >> > >>>>> the
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
> > > > > shutting
> > > > > > >> them
> > > > > > >> > > > down).
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue
> > for
> > > > > > running
> > > > > > >> > this
> > > > > > >> > > > > stuff
> > > > > > >> > > > > > in
> > > > > > >> > > > > > >> > YARN
> > > > > > >> > > > > > >> > > > via
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon,
> > and
> > > > AWS
> > > > > > >> using
> > > > > > >> > > some
> > > > > > >> > > > > of
> > > > > > >> > > > > > >> their
> > > > > > >> > > > > > >> > > > tools
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> but from the
> > > > > > >> > > > > > >> > > > > >> > >>>>>> point
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> of
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks
> > these
> > > > > stream
> > > > > > >> > > > processing
> > > > > > >> > > > > > jobs
> > > > > > >> > > > > > >> > are
> > > > > > >> > > > > > >> > > > > just
> > > > > > >> > > > > > >> > > > > >> > >>>>>> stateless
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and
> go
> > > and
> > > > > > >> expand
> > > > > > >> > and
> > > > > > >> > > > > > contract
> > > > > > >> > > > > > >> > at
> > > > > > >> > > > > > >> > > > > will.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> There
> > > > > > >> > > > > > >> > > > > >> > >>>>> is
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant
> > details:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines
> of
> > > > code,
> > > > > > it
> > > > > > >> > would
> > > > > > >> > > > get
> > > > > > >> > > > > > >> larger
> > > > > > >> > > > > > >> > > if
> > > > > > >> > > > > > >> > > > we
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not
> > vastly
> > > > > > larger.
> > > > > > >> We
> > > > > > >> > > > really
> > > > > > >> > > > > > do
> > > > > > >> > > > > > >> > get a
> > > > > > >> > > > > > >> > > > ton
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> leverage
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is
> > > fully
> > > > > > >> > delegated
> > > > > > >> > > to
> > > > > > >> > > > > the
> > > > > > >> > > > > > >> new
> > > > > > >> > > > > > >> > > > > >> consumer.
> > > > > > >> > > > > > >> > > > > >> > >>>>> This
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any
> > > partition
> > > > > > >> > management
> > > > > > >> > > > > > strategy
> > > > > > >> > > > > > >> > > > > available
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available
> > to
> > > > > Samza
> > > > > > >> (and
> > > > > > >> > > vice
> > > > > > >> > > > > > versa)
> > > > > > >> > > > > > >> > and
> > > > > > >> > > > > > >> > > > > with
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> the
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> exact
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as
> well
> > > as
> > > > > > state
> > > > > > >> > reuse
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look,
> hopefully
> > > it
> > > > is
> > > > > > >> > thought
> > > > > > >> > > > > > >> provoking.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at
> 6:55
> > > PM,
> > > > > > Chris
> > > > > > >> > > > > Riccomini <
> > > > > > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some
> discussions
> > > with
> > > > > > Samza
> > > > > > >> > > > > engineers
> > > > > > >> > > > > > at
> > > > > > >> > > > > > >> > > > LinkedIn
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> Confluent
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> > > > > > observations
> > > > > > >> > and
> > > > > > >> > > > > would
> > > > > > >> > > > > > >> like
> > > > > > >> > > > > > >> > to
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> propose
> > > > > > >> > > > > > >> > > > > >> > >>>>> some
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things
> > > that
> > > > I
> > > > > > >> want to
> > > > > > >> > > > call
> > > > > > >> > > > > > out
> > > > > > >> > > > > > >> > about
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > > > > > >> > > > > > >> > > > > >> > >>>>>> design,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose
> some
> > > > > > changes.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon
> a
> > > > > dynamic
> > > > > > >> > > > deployment
> > > > > > >> > > > > > >> system.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> > > > > > >> SystemConsumer/SystemProducer
> > > > > > >> > and
> > > > > > >> > > > > > Kafka's
> > > > > > >> > > > > > >> > > > consumer
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > > > >> > > > > > >> > > > > >> > >>>>> are
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of
> the
> > > > same
> > > > > > >> > problems.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues
> > are
> > > > > > related,
> > > > > > >> > but
> > > > > > >> > > > I'll
> > > > > > >> > > > > > >> > address
> > > > > > >> > > > > > >> > > > them
> > > > > > >> > > > > > >> > > > > >> in
> > > > > > >> > > > > > >> > > > > >> > >>>>> order.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on
> > the
> > > > use
> > > > > > of a
> > > > > > >> > > > dynamic
> > > > > > >> > > > > > >> > > deployment
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > > > > > >> > > > > > >> > > > > >> > >>>>>> such
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> as
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
> > > > > initially
> > > > > > >> built
> > > > > > >> > > > > Samza,
> > > > > > >> > > > > > we
> > > > > > >> > > > > > >> > bet
> > > > > > >> > > > > > >> > > > that
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> there
> > > > > > >> > > > > > >> > > > > >> > >>>>>> would
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> be
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this
> > > area,
> > > > > and
> > > > > > >> we
> > > > > > >> > > could
> > > > > > >> > > > > > >> support
> > > > > > >> > > > > > >> > > > them,
> > > > > > >> > > > > > >> > > > > >> and
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > > >> > > > > > >> > > > > >> > >>>>>> rest
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality,
> > > there
> > > > > are
> > > > > > >> many
> > > > > > >> > > > > > >> variations.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > > > > > >> > > > > > >> > > > > >> > >>>>>> many
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to
> just
> > > > start
> > > > > > >> their
> > > > > > >> > > > > > processors
> > > > > > >> > > > > > >> > like
> > > > > > >> > > > > > >> > > > > normal
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
> > > > > traditional
> > > > > > >> > > > deployment
> > > > > > >> > > > > > >> scripts
> > > > > > >> > > > > > >> > > > such
> > > > > > >> > > > > > >> > > > > as
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > > > > > >> > > > > > >> > > > > >> > >>>>>> Chef,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a
> > > > deployment
> > > > > > >> system
> > > > > > >> > > on
> > > > > > >> > > > > > users
> > > > > > >> > > > > > >> > makes
> > > > > > >> > > > > > >> > > > the
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process
> > really
> > > > > > painful
> > > > > > >> for
> > > > > > >> > > > first
> > > > > > >> > > > > > time
> > > > > > >> > > > > > >> > > > users.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
> > > > > requirement
> > > > > > >> was
> > > > > > >> > > also
> > > > > > >> > > > a
> > > > > > >> > > > > > bit
> > > > > > >> > > > > > >> of
> > > > > > >> > > > > > >> > a
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > > > > > >> > > > > > >> > > > > >> > >>>>>> because
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> of
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental
> > misunderstanding
> > > > > > between
> > > > > > >> > the
> > > > > > >> > > > > > nature of
> > > > > > >> > > > > > >> > > batch
> > > > > > >> > > > > > >> > > > > >> jobs
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> stream
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on,
> > we
> > > > made
> > > > > > >> > > conscious
> > > > > > >> > > > > > effort
> > > > > > >> > > > > > >> to
> > > > > > >> > > > > > >> > > > favor
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > > >> > > > > > >> > > > > >> > >>>>>> Hadoop
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing
> > > > things,
> > > > > > >> since
> > > > > > >> > it
> > > > > > >> > > > > worked
> > > > > > >> > > > > > >> and
> > > > > > >> > > > > > >> > > was
> > > > > > >> > > > > > >> > > > > well
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> understood.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed
> was
> > > > that
> > > > > > >> batch
> > > > > > >> > > jobs
> > > > > > >> > > > > > have a
> > > > > > >> > > > > > >> > > > definite
> > > > > > >> > > > > > >> > > > > >> > >>>>>> beginning,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing
> > > jobs
> > > > > > don't
> > > > > > >> > > > > (usually).
> > > > > > >> > > > > > >> This
> > > > > > >> > > > > > >> > > > leads
> > > > > > >> > > > > > >> > > > > to
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> a
> > > > > > >> > > > > > >> > > > > >> > >>>>> much
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem
> > for
> > > > > > stream
> > > > > > >> > > > > processors.
> > > > > > >> > > > > > >> You
> > > > > > >> > > > > > >> > > > > >> basically
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> just
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> need
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start
> the
> > > > > > >> processor,
> > > > > > >> > and
> > > > > > >> > > > > start
> > > > > > >> > > > > > >> it.
> > > > > > >> > > > > > >> > > The
> > > > > > >> > > > > > >> > > > > way
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn,
> > > > there's
> > > > > > no
> > > > > > >> > > concept
> > > > > > >> > > > > of
> > > > > > >> > > > > > a
> > > > > > >> > > > > > >> > > cluster
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > > > > > >> > > > > > >> > > > > >> > >>>>>> add
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem
> > > with
> > > > > > >> coupling
> > > > > > >> > > > Samza
> > > > > > >> > > > > > with
> > > > > > >> > > > > > >> a
> > > > > > >> > > > > > >> > > > > >> scheduler
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> is
> > > > > > >> > > > > > >> > > > > >> > >>>>>> that
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now
> > has
> > > > to
> > > > > > >> handle
> > > > > > >> > > > > > deployment.
> > > > > > >> > > > > > >> > > This
> > > > > > >> > > > > > >> > > > > >> pulls
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> in a
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> bunch
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as
> > > configuration
> > > > > > >> > > distribution
> > > > > > >> > > > > > (config
> > > > > > >> > > > > > >> > > > > stream),
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> shell
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> scrips
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh,
> JobRunner),
> > > > > > packaging
> > > > > > >> > (all
> > > > > > >> > > > the
> > > > > > >> > > > > > .tgz
> > > > > > >> > > > > > >> > > > stuff),
> > > > > > >> > > > > > >> > > > > >> etc.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for
> requiring
> > > > > dynamic
> > > > > > >> > > > deployment
> > > > > > >> > > > > > was
> > > > > > >> > > > > > >> to
> > > > > > >> > > > > > >> > > > > support
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want
> > to
> > > > have
> > > > > > >> > > locality,
> > > > > > >> > > > > you
> > > > > > >> > > > > > >> need
> > > > > > >> > > > > > >> > to
> > > > > > >> > > > > > >> > > > put
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> your
> > > > > > >> > > > > > >> > > > > >> > >>>>>> processors
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> > > > > > processing.
> > > > > > >> > Upon
> > > > > > >> > > > > > further
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> though,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
> > > > > beneficial.
> > > > > > >> > There
> > > > > > >> > > is
> > > > > > >> > > > > > some
> > > > > > >> > > > > > >> > good
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > > > > > >> > > > > > >> > > > > >> > >>>>>> about
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
> > > > > SAMZA-335.
> > > > > > >> > Again,
> > > > > > >> > > we
> > > > > > >> > > > > > took
> > > > > > >> > > > > > >> the
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > > > > > >> > > > > > >> > > > > >> > >>>>>> path,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> > > > > > differences
> > > > > > >> > > > between
> > > > > > >> > > > > > HDFS
> > > > > > >> > > > > > >> > and
> > > > > > >> > > > > > >> > > > > Kafka.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > > > > > >> > > > > > >> > > > > >> > >>>>>> has
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has
> > > > > partitions.
> > > > > > >> This
> > > > > > >> > > > leads
> > > > > > >> > > > > to
> > > > > > >> > > > > > >> less
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential
> with
> > > > stream
> > > > > > >> > > processors
> > > > > > >> > > > > on
> > > > > > >> > > > > > top
> > > > > > >> > > > > > >> > of
> > > > > > >> > > > > > >> > > > > Kafka.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used
> > as a
> > > > > > crutch.
> > > > > > >> > > Samza
> > > > > > >> > > > > > doesn't
> > > > > > >> > > > > > >> > > have
> > > > > > >> > > > > > >> > > > > any
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> built
> > > > > > >> > > > > > >> > > > > >> > >>>>> in
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic.
> > > Instead,
> > > > it
> > > > > > >> > depends
> > > > > > >> > > on
> > > > > > >> > > > > the
> > > > > > >> > > > > > >> > > dynamic
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling
> system
> > > to
> > > > > > handle
> > > > > > >> > > > restarts
> > > > > > >> > > > > > >> when a
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> made
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to
> write a
> > > > > > >> standalone
> > > > > > >> > > Samza
> > > > > > >> > > > > > >> > container
> > > > > > >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability
> > is
> > > > > good,
> > > > > > >> but I
> > > > > > >> > > > think
> > > > > > >> > > > > > that
> > > > > > >> > > > > > >> > > we've
> > > > > > >> > > > > > >> > > > > >> gone
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> too
> > > > > > >> > > > > > >> > > > > >> > >>>>>> far
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza
> > has:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment
> > > systems.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming
> > systems
> > > > > > >> > > > (SystemConsumer,
> > > > > > >> > > > > > >> > > > > SystemProducer,
> > > > > > >> > > > > > >> > > > > >> > >>>> etc).
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage
> engines.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for
> > > just
> > > > > > about
> > > > > > >> > every
> > > > > > >> > > > > > >> component
> > > > > > >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > SystemStreamPartitionGrouper,
> > > > > > >> > > ConfigRewriter,
> > > > > > >> > > > > > etc).
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that
> > > I've
> > > > > > >> > forgotten,
> > > > > > >> > > as
> > > > > > >> > > > > > well.
> > > > > > >> > > > > > >> > Some
> > > > > > >> > > > > > >> > > > of
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> these
> > > > > > >> > > > > > >> > > > > >> > >>>>> are
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have
> proven
> > > not
> > > > > to
> > > > > > >> be.
> > > > > > >> > > This
> > > > > > >> > > > > all
> > > > > > >> > > > > > >> comes
> > > > > > >> > > > > > >> > > at
> > > > > > >> > > > > > >> > > > a
> > > > > > >> > > > > > >> > > > > >> cost:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This
> complexity
> > is
> > > > > > making
> > > > > > >> it
> > > > > > >> > > > harder
> > > > > > >> > > > > > for
> > > > > > >> > > > > > >> > our
> > > > > > >> > > > > > >> > > > > users
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > > > > >> > > > > > >> > > > > >> > >>>>> pick
> > > > > > >> > > > > > >> > > > > >> > >>>>>> up
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the
> > box.
> > > It
> > > > > > also
> > > > > > >> > makes
> > > > > > >> > > > it
> > > > > > >> > > > > > >> > difficult
> > > > > > >> > > > > > >> > > > for
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason
> > > about
> > > > > > what
> > > > > > >> the
> > > > > > >> > > > > > >> > > characteristics
> > > > > > >> > > > > > >> > > > of
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> > > > > > >> characteristics
> > > > > > >> > > > change
> > > > > > >> > > > > > >> > > depending
> > > > > > >> > > > > > >> > > > on
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> The issues with
> pluggability
> > > are
> > > > > > most
> > > > > > >> > > visible
> > > > > > >> > > > > in
> > > > > > >> > > > > > the
> > > > > > >> > > > > > >> > > > System
> > > > > > >> > > > > > >> > > > > >> APIs.
> > > > > > >> > > > > > >> > > > > >> > >>>>> What
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to
> be
> > > > > > >> functional is
> > > > > > >> > > > Kafka
> > > > > > >> > > > > > as
> > > > > > >> > > > > > >> its
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> transport
> > > > > > >> > > > > > >> > > > > >> > >>>>>> layer.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> But
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two
> > unrelated
> > > > use
> > > > > > >> cases
> > > > > > >> > > into
> > > > > > >> > > > > one
> > > > > > >> > > > > > >> API:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of
> > Kafka.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in
> > Kafka.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> The current System API
> > > supports
> > > > > both
> > > > > > >> of
> > > > > > >> > > these
> > > > > > >> > > > > use
> > > > > > >> > > > > > >> > cases.
> > > > > > >> > > > > > >> > > > The
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> problem
> > > > > > >> > > > > > >> > > > > >> > >>>>>> is,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> we
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> actually want different
> > > features
> > > > > for
> > > > > > >> each
> > > > > > >> > > use
> > > > > > >> > > > > > case.
> > > > > > >> > > > > > >> By
> > > > > > >> > > > > > >> > > > > >> papering
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> over
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> these
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and
> > providing a
> > > > > > single
> > > > > > >> > API,
> > > > > > >> > > > > we've
> > > > > > >> > > > > > >> > > > introduced
> > > > > > >> > > > > > >> > > > > a
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> leaky
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd
> > really
> > > > like
> > > > > > in
> > > > > > >> (2)
> > > > > > >> > > is
> > > > > > >> > > > to
> > > > > > >> > > > > > have
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing
> > longs
> > > > for
> > > > > > >> > offsets
> > > > > > >> > > > > (like
> > > > > > >> > > > > > >> > Kafka).
> > > > > > >> > > > > > >> > > > > This
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > > > > > >> > > > > > >> > > > > >> > >>>>> with
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> (1),
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> though, since different
> > > systems
> > > > > have
> > > > > > >> > > > different
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both
> on
> > > the
> > > > > > >> mailing
> > > > > > >> > > list
> > > > > > >> > > > > and
> > > > > > >> > > > > > >> the
> > > > > > >> > > > > > >> > > SQL
> > > > > > >> > > > > > >> > > > > >> JIRAs
> > > > > > >> > > > > > >> > > > > >> > >>>>> about
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true
> > for
> > > > > > >> > > replayability.
> > > > > > >> > > > > > Kafka
> > > > > > >> > > > > > >> > > allows
> > > > > > >> > > > > > >> > > > us
> > > > > > >> > > > > > >> > > > > >> to
> > > > > > >> > > > > > >> > > > > >> > >>>>> rewind
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> when
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many
> > other
> > > > > > systems
> > > > > > >> > > don't.
> > > > > > >> > > > In
> > > > > > >> > > > > > some
> > > > > > >> > > > > > >> > > > cases,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> return
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets
> (e.g.
> > > > > > >> > > > > > >> WikipediaSystemConsumer)
> > > > > > >> > > > > > >> > > > > because
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> they
> > > > > > >> > > > > > >> > > > > >> > >>>>>> have
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another
> > > example.
> > > > > > Kafka
> > > > > > >> > > > supports
> > > > > > >> > > > > > >> > > > > partitioning,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > > > > >> > > > > > >> > > > > >> > >>>>> many
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model
> this
> > > by
> > > > > > >> having a
> > > > > > >> > > > single
> > > > > > >> > > > > > >> > > partition
> > > > > > >> > > > > > >> > > > > for
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still,
> other
> > > > > systems
> > > > > > >> model
> > > > > > >> > > > > > >> partitioning
> > > > > > >> > > > > > >> > > > > >> > >>>> differently (e.g.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface
> is
> > > > also
> > > > > a
> > > > > > >> mess.
> > > > > > >> > > > > > Creating
> > > > > > >> > > > > > >> > > streams
> > > > > > >> > > > > > >> > > > > in
> > > > > > >> > > > > > >> > > > > >> a
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is
> > almost
> > > > > > >> impossible.
> > > > > > >> > > As
> > > > > > >> > > > is
> > > > > > >> > > > > > >> > modeling
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > > > > > >> > > > > > >> > > > > >> > >>>>> for
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> system (replication
> factor,
> > > > > > >> partitions,
> > > > > > >> > > > > location,
> > > > > > >> > > > > > >> > etc).
> > > > > > >> > > > > > >> > > > The
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> list
> > > > > > >> > > > > > >> > > > > >> > >>>>> goes
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> on.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began
> > > > writing
> > > > > > >> Samza,
> > > > > > >> > > > > Kafka's
> > > > > > >> > > > > > >> > > consumer
> > > > > > >> > > > > > >> > > > > and
> > > > > > >> > > > > > >> > > > > >> > >>>>> producer
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak
> > feature
> > > > set.
> > > > > > On
> > > > > > >> the
> > > > > > >> > > > > > >> > consumer-side,
> > > > > > >> > > > > > >> > > > you
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> had two
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high
> level
> > > > > > consumer,
> > > > > > >> or
> > > > > > >> > > the
> > > > > > >> > > > > > simple
> > > > > > >> > > > > > >> > > > > consumer.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> The
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> problem
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level
> consumer
> > > was
> > > > > > that
> > > > > > >> it
> > > > > > >> > > > > > controlled
> > > > > > >> > > > > > >> > your
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition
> > > assignments,
> > > > > and
> > > > > > >> the
> > > > > > >> > > order
> > > > > > >> > > > > in
> > > > > > >> > > > > > >> which
> > > > > > >> > > > > > >> > > you
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > > > > > >> > > > > > >> > > > > >> > >>>>> problem
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> with
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is
> that
> > > it's
> > > > > not
> > > > > > >> > > simple.
> > > > > > >> > > > > It's
> > > > > > >> > > > > > >> > basic.
> > > > > > >> > > > > > >> > > > You
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> end up
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really
> > > > > low-level
> > > > > > >> stuff
> > > > > > >> > > > that
> > > > > > >> > > > > > you
> > > > > > >> > > > > > >> > > > > shouldn't.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> We
> > > > > > >> > > > > > >> > > > > >> > >>>>>> spent a
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make
> Samza's
> > > > > > >> > > > KafkaSystemConsumer
> > > > > > >> > > > > > very
> > > > > > >> > > > > > >> > > > robust.
> > > > > > >> > > > > > >> > > > > >> It
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support
> > some
> > > > > cool
> > > > > > >> > > features:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message
> > > ordering
> > > > > and
> > > > > > >> > > > > > prioritization.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over
> > partition
> > > > > > >> assignment
> > > > > > >> > > to
> > > > > > >> > > > > > support
> > > > > > >> > > > > > >> > > > joins,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> global
> > > > > > >> > > > > > >> > > > > >> > >>>>>> state
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement
> it
> > > :)),
> > > > > > etc.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over
> offset
> > > > > > >> > checkpointing.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at
> > the
> > > > time
> > > > > > is
> > > > > > >> > that
> > > > > > >> > > > > these
> > > > > > >> > > > > > >> > > features
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> actually
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of
> Kafka
> > > > > > consumers
> > > > > > >> > (not
> > > > > > >> > > > just
> > > > > > >> > > > > > >> Samza
> > > > > > >> > > > > > >> > > > stream
> > > > > > >> > > > > > >> > > > > >> > >>>>>> processors)
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do
> things
> > > like
> > > > > > joins
> > > > > > >> > and
> > > > > > >> > > > > > partition
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the
> > same
> > > > > > >> > conclusion.
> > > > > > >> > > > > > They're
> > > > > > >> > > > > > >> > > adding
> > > > > > >> > > > > > >> > > > a
> > > > > > >> > > > > > >> > > > > >> ton
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new
> > > Kafka
> > > > > > >> consumer
> > > > > > >> > > > > > >> > > implementation.
> > > > > > >> > > > > > >> > > > > To a
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > > > > > >> > > > > > >> > > > > >> > >>>>> it's
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what
> we've
> > > > > already
> > > > > > >> done
> > > > > > >> > > in
> > > > > > >> > > > > > Samza.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka
> ended
> > up
> > > > > > taking
> > > > > > >> a
> > > > > > >> > > very
> > > > > > >> > > > > > similar
> > > > > > >> > > > > > >> > > > > approach
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > > > > >> > > > > > >> > > > > >> > >>>>>> Samza's
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> > > > > > implementation
> > > > > > >> for
> > > > > > >> > > > > > handling
> > > > > > >> > > > > > >> > > offset
> > > > > > >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new
> > offset
> > > > > > >> management
> > > > > > >> > > > > feature
> > > > > > >> > > > > > >> > stores
> > > > > > >> > > > > > >> > > > > >> offset
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic,
> and
> > > > allows
> > > > > > >> you to
> > > > > > >> > > > fetch
> > > > > > >> > > > > > them
> > > > > > >> > > > > > >> > > from
> > > > > > >> > > > > > >> > > > > the
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a
> > > > waste,
> > > > > > >> since
> > > > > > >> > we
> > > > > > >> > > > > could
> > > > > > >> > > > > > >> have
> > > > > > >> > > > > > >> > > > shared
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > > >> > > > > > >> > > > > >> > >>>>> work
> > > > > > >> > > > > > >> > > > > >> > >>>>>> if
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> it
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka
> from
> > > the
> > > > > > >> get-go.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a
> > > rather
> > > > > > >> radical
> > > > > > >> > > > > > proposal.
> > > > > > >> > > > > > >> > Samza
> > > > > > >> > > > > > >> > > > is
> > > > > > >> > > > > > >> > > > > >> > >>>>> relatively
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd
> > > > venture
> > > > > to
> > > > > > >> say
> > > > > > >> > > that
> > > > > > >> > > > > > we're
> > > > > > >> > > > > > >> > > near a
> > > > > > >> > > > > > >> > > > > 1.0
> > > > > > >> > > > > > >> > > > > >> > >>>>>> release.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we
> take
> > > > what
> > > > > > >> we've
> > > > > > >> > > > > learned,
> > > > > > >> > > > > > and
> > > > > > >> > > > > > >> > > begin
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> about
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What
> would
> > > we
> > > > > > >> change if
> > > > > > >> > > we
> > > > > > >> > > > > were
> > > > > > >> > > > > > >> > > starting
> > > > > > >> > > > > > >> > > > > >> from
> > > > > > >> > > > > > >> > > > > >> > >>>>>> scratch?
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> My
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone
> the
> > > > > *only*
> > > > > > >> way
> > > > > > >> > to
> > > > > > >> > > > run
> > > > > > >> > > > > > Samza
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate
> > all
> > > > > direct
> > > > > > >> > > > > dependences
> > > > > > >> > > > > > on
> > > > > > >> > > > > > >> > > YARN,
> > > > > > >> > > > > > >> > > > > >> Mesos,
> > > > > > >> > > > > > >> > > > > >> > >>>> etc.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call
> to
> > > > > support
> > > > > > >> only
> > > > > > >> > > > Kafka
> > > > > > >> > > > > > as
> > > > > > >> > > > > > >> the
> > > > > > >> > > > > > >> > > > > stream
> > > > > > >> > > > > > >> > > > > >> > >>>>>> processing
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's
> > metrics,
> > > > > > logging,
> > > > > > >> > > > > > >> serialization,
> > > > > > >> > > > > > >> > > and
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> config
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> systems,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's
> > > instead.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the
> > > issues
> > > > > > that
> > > > > > >> I
> > > > > > >> > > > > outlined
> > > > > > >> > > > > > >> > above.
> > > > > > >> > > > > > >> > > It
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > > > > >> > > > > > >> > > > > >> > >>>>> also
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base
> > > > pretty
> > > > > > >> > > > dramatically.
> > > > > > >> > > > > > >> > > Supporting
> > > > > > >> > > > > > >> > > > > >> only
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container
> will
> > > > allow
> > > > > > >> Samza
> > > > > > >> > to
> > > > > > >> > > be
> > > > > > >> > > > > > >> executed
> > > > > > >> > > > > > >> > > on
> > > > > > >> > > > > > >> > > > > YARN
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos
> (using
> > > > > > >> > > > Marathon/Aurora),
> > > > > > >> > > > > or
> > > > > > >> > > > > > >> most
> > > > > > >> > > > > > >> > > > other
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> deployment
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make
> > > life a
> > > > > lot
> > > > > > >> > easier
> > > > > > >> > > > for
> > > > > > >> > > > > > new
> > > > > > >> > > > > > >> > > users.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial
> > > without
> > > > > > YARN.
> > > > > > >> > The
> > > > > > >> > > > drop
> > > > > > >> > > > > > in
> > > > > > >> > > > > > >> > > mailing
> > > > > > >> > > > > > >> > > > > >> list
> > > > > > >> > > > > > >> > > > > >> > >>>>>> traffic
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems
> > long
> > > > > > >> overdue to
> > > > > > >> > > me.
> > > > > > >> > > > > The
> > > > > > >> > > > > > >> > > reality
> > > > > > >> > > > > > >> > > > > is,
> > > > > > >> > > > > > >> > > > > >> > >>>>> everyone
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> that
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using
> Samza
> > > with
> > > > > > >> Kafka.
> > > > > > >> > We
> > > > > > >> > > > > > basically
> > > > > > >> > > > > > >> > > > require
> > > > > > >> > > > > > >> > > > > >> it
> > > > > > >> > > > > > >> > > > > >> > >>>>>> already
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> in
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to
> > > work.
> > > > > > Those
> > > > > > >> > that
> > > > > > >> > > > are
> > > > > > >> > > > > > >> using
> > > > > > >> > > > > > >> > > > other
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > > > > >> > > > > > >> > > > > >> > >>>>>> are
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for
> > ingest
> > > > into
> > > > > > >> Kafka
> > > > > > >> > > (1),
> > > > > > >> > > > > and
> > > > > > >> > > > > > >> then
> > > > > > >> > > > > > >> > > > they
> > > > > > >> > > > > > >> > > > > do
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top.
> There
> > > is
> > > > > > >> already
> > > > > > >> > > > > > discussion (
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> > > > > > >> > > > >
> > > > > > >> > > > > > >> >
> > > > > > >> > > > > >
> > > > > > >> > >
> > > > > > >>
> > > > >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > > > >> > > > > > >> > > > > >> > >>>>> 767
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> )
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting
> > > into
> > > > > > Kafka
> > > > > > >> > > > extremely
> > > > > > >> > > > > > >> easy.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to
> > > couple
> > > > > with
> > > > > > >> > Kafka,
> > > > > > >> > > > we
> > > > > > >> > > > > > can
> > > > > > >> > > > > > >> > > > leverage
> > > > > > >> > > > > > >> > > > > a
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> their
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer
> have
> > > to
> > > > > > >> maintain
> > > > > > >> > > our
> > > > > > >> > > > > own
> > > > > > >> > > > > > >> > config,
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > > > > > >> > > > > > >> > > > > >> > >>>>> etc.
> > > > > > >> > > > > > >> > > > > >> > >>>>>>> We
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same
> > > > libraries,
> > > > > > and
> > > > > > >> > make
> > > > > > >> > > > them
> > > > > > >> > > > > > >> > better.
> > > > > > >> > > > > > >> > > > This
> > > > > > >> > > > > > >> > > > > >> > >>>>>>>>> will
> > > > > > >> > > > > > >> > > > > >> >
> > > > > > >> ...
> > > > > > >>
> > > > > > >> [Message clipped]
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Jordan Shaw
> > > > > Full Stack Software Engineer
> > > > > PubNub Inc
> > > > > 1045 17th St
> > > > > San Francisco, CA 94107
> > > > >
> > > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Dan <da...@gmail.com>.
A lot of what I've used Samza for so far is unifying the input and output
of data with Kafka, that's one of the reasons I've added the
ElasticsearchProducer which we use at State and have more we'd want to add
there.

As with Garry and others I didn't know about CopyCat before this discussion
but it seems the Samza community has a lot of experience working on getting
data reliability out of and into other systems. Removing
SystemConsumer/Producer
may fragment that community or make it less clear how to use Samza end to
end. Would it make sense to develop CopyCat inside Samza? Probably not...
but it would probably be good to build that out working with the Samza
community first, then once that's in Kafka it would probably be far more
clear how we could remove the SystemConsumer/Producer from Samza showing
the range of intput/export that Kafka provides.

As a side point, we don't use Samza for input as Martin has pointed out it
doesn't really fit with consuming sources like databases (MongoDB in our
case) so we've written our own custom consumers that we'd happily work into
a tool like CopyCat. There hasn't really been much discussion like this in
the Kafka community and I'm not sure we'd have known about this as the
Kafka project is seen as being lightweight from our point of view (and I
think most people externally). Is Kafka the best project / community to
start CopyCat? it does have a good momentum behind it but that can be
directed at other Apache projects with marketing.

 - Dan


On 14 July 2015 at 01:33, Jay Kreps <ja...@confluent.io> wrote:

> Hey Yi,
>
> Cool.
>
> The Kafka design process is very open. We usually do a very detailed wiki
> along with any code and do a pretty extensive discussion around these.
> Details here:
>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>
> It's wide open and tons of people participate. If we went down this route
> the next step would be to discuss there.
>
> For coding it's normal Apache style, JIRAs, RBC, etc.
>
> With respect to state, you definitely need the core support in the API as
> that has implications all the way down. I suspect you need at least some
> key-value store impl to ship with the client so you have one plugin to test
> against and use out of the box. But the whole point of that stuff is it is
> pluggable and state is in any case recreated off the Kafka topic, so Samza
> can have its own storage engines too. I don't think these need to be
> maintained centrally at all.
>
> -Jay
>
>
>
> On Mon, Jul 13, 2015 at 3:23 PM, Yi Pan <ni...@gmail.com> wrote:
>
> > Hi, Jay,
> >
> > Given all the user concerns, the board disagreement on sub-projects, I am
> > supporting your 5th option as well. As you said, even the end goal is the
> > same, it might help to pave a smooth path forward. One thing I learned
> over
> > the years is that what we planned for may not be the final product, and
> the
> > unexpected product may be even better if we learn and adapt along the
> way.
> > :)
> >
> > So, since I assume that in option 5, Samza will fully embrace the new
> Kafka
> > Streams API as the core and heavily depends on it, I want to raise up
> some
> > detailed logistic questions:
> > 1. How do Samza community contribute to the design and development of the
> > new Kafka Streams API? As Kartik mentioned, if there is a model for Samza
> > community to contribute to just this part of Kafka code base, it would
> be a
> > huge plus point to the integration.
> > 2. What's the scope of the new Kafka Streams API? Is it just focused on
> > message consumption, producing, Kafka-based partition distribution,
> offset
> > management, message selection and delivery to StreamProcessor? In other
> > words, I have a question regarding to whether we should put
> samza-kv-store
> > in the scope? The reasons that I think that it might be better to stay in
> > Samza initially are: a) KV-store libraries does not directly interact w/
> > Kafka brokers, it optionally uses Kafka consumers and producers like a
> > client program; b) there are a tons of experiments / tune-ups on RocksDB
> > that we want to have a faster iteration on this library (e.g. there is an
> > experimental time-sequence KV store implementation from LinkedIn we also
> > want to experiment on in window operator in SQL). The down-side I can see
> > is that w/o this in Kafka Streams API, the as-a-library mode may not get
> > the state management support. If we can find a way to make sure that the
> > current Samza community can contribute to this library in a faster
> > velocity, I can be convinced otherwise as well. What's your opinion on
> > this?
> >
> > Overall, thanks a lot for pushing forward the whole discussion!
> >
> > -Yi
> >
> > On Mon, Jul 13, 2015 at 12:56 PM, Garry Turkington <
> > g.turkington@improvedigital.com> wrote:
> >
> > > Hi,
> > >
> > > I'm also supportive of Jay's option 5. There is a risk the "transformer
> > > API" -- I'd have preferred Metamorphosis but it's too hard to type! --
> > > takes on a life of its own and we end up with two very different things
> > but
> > > given how good the Kafka community has been at introducing new producer
> > and
> > > consumer clients and giving very clear guidance on when they are
> > production
> > > ready this is a danger I believe can be managed. It'd also be excellent
> > to
> > > get some working code to beat around the notions of stream processing
> > atop
> > > a system with transacdtional messages.
> > >
> > > On the question of whether to keep or deprecate
> SystemConsumer/Producer I
> > > believe we need get a better understanding over the next while of just
> > what
> > > the Samza community is looking for in such connectivity. For my own use
> > > cases I have been looking to add additional implementations primarily
> to
> > > use Samza as the data ingress and egress component around Kafka.
> Writing
> > > external clients that require their own reliability and scalability
> > > management gets old real fast and pushing this into a simple Samza job
> > that
> > > reads from system X and pushes into Kafka (or vice versa) was the
> obvious
> > > choice for me in the current model. For this type of usage though
> copycat
> > > is likely much superior (obviously needs proven) and the question then
> is
> > > if most Samza users look to the system implementations to also act as a
> > > front-end into Kafka or if significant usage is indeed intended to have
> > the
> > > alternative systems as the primary message source. That understanding
> > will
> > > I think give much clarity in just what value the abstraction overhead
> of
> > > the current model brings.
> > >
> > > Garry
> > >
> > > -----Original Message-----
> > > From: Yan Fang [mailto:yanfang724@gmail.com]
> > > Sent: 13 July 2015 19:58
> > > To: dev@samza.apache.org
> > > Subject: Re: Thoughts and obesrvations on Samza
> > >
> > > I am leaning to Jay's fifth approach. It is not radical and gives us
> some
> > > time to see the outcome.
> > >
> > > In addition, I would suggest:
> > >
> > > 1) Keep the SystemConsumer/SystemProducer API. Because current
> > > SystemConsumer/SystemProducer API satisfies the usage (From Joardan,
> and
> > > even Garry's feedback) and is not so broken that we want to deprecate
> it.
> > > Though there are some issues in implemnting the Kinesis, they are not
> > > unfixable. Nothing should prevent Samza, as a stream processing system,
> > to
> > > support other systems. In addition, there already are some systems
> > > exiting besides Kafka: ElasticSearch (committed to the master), HDFS
> > > (patch-available), S3( from the mailing list), Kinesis (developing in
> > > another repository), ActiveMQ (in two months). We may want to see how
> > those
> > > go before we "kill" them.
> > >
> > > 2) Can have some Samza devs involved in Kafka's "transformer" client
> API.
> > > This can not only help the future integration (if any) much easier,
> > because
> > > they have knowledge about both systems, but also good for Kafka's
> > > community, because Samza devs have the streaming process experience
> that
> > > Kafka devs may miss.
> > >
> > > 3) Samza's partition management system may still support other systems.
> > > Though the partition management logic in samza-kafka will be moved to
> > > Kafka, its still useful for other systems that do not have the
> partition
> > > management layer.
> > >
> > > 4) Start sharing the docs/websites and using the same terminology
> (though
> > > do not know how to do this exactly. :). This will reduce the future
> > > confusion and does not hurt Samza's independency.
> > >
> > > In my opinion, Samza, as a standalone project, still can (and already)
> > > heavily replying on Kafka, and even more tuned for Kafka-specific
> usage.
> > > Kafka, also can embed Samza in the document, I do not see anything
> > prevent
> > > doing this.
> > >
> > > Thanks,
> > >
> > > Fang, Yan
> > > yanfang724@gmail.com
> > >
> > > On Mon, Jul 13, 2015 at 11:25 AM, Jordan Shaw <jo...@pubnub.com>
> wrote:
> > >
> > > > Jay,
> > > > I think doing this iteratively in smaller chunks is a better way to
> go
> > as
> > > > new issues arise. As Navina said Kafka is a "stream system" and Samza
> > is
> > > a
> > > > "stream processor" and those two ideas should be mutually exclusive.
> > > >
> > > > -Jordan
> > > >
> > > > On Mon, Jul 13, 2015 at 10:06 AM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > Hmm, thought about this more. Maybe this is just too much too
> quick.
> > > > > Overall I think there is some enthusiasm for the proposal but it's
> > not
> > > > > really unanimous enough to make any kind of change this big
> cleanly.
> > > The
> > > > > board doesn't really like the merging stuff, user's are concerned
> > about
> > > > > compatibility, I didn't feel there was unanimous agreement on
> > dropping
> > > > > SystemConsumer, etc. Even if this is the right end state to get to,
> > > > > probably trying to push all this through at once isn't the right
> way
> > to
> > > > do
> > > > > it.
> > > > >
> > > > > So let me propose a kind of fifth (?) option which I think is less
> > > > dramatic
> > > > > and let's things happen gradually. I think this is kind of like
> > > combining
> > > > > the first part of Yi's proposal and Jakob's third option, leaving
> the
> > > > rest
> > > > > to be figured out incrementally:
> > > > >
> > > > > Option 5: We continue the prototype I shared and propose that as a
> > kind
> > > > of
> > > > > "transformer" client API in Kafka. This isn't really a full-fledged
> > > > stream
> > > > > processing layer, more like a supped up consumer api for munging
> > > topics.
> > > > > This would let us figure out some of the technical bits, how to do
> > this
> > > > on
> > > > > Kafka's group management features, how to integrate the txn feature
> > to
> > > do
> > > > > the exactly-once stuff in these transformations, and get all this
> > stuff
> > > > > solid. This api would have valid uses in it's own right, especially
> > > when
> > > > > your transformation will be embedded inside an existing service or
> > > > > application which isn't possible with Samza (or other existing
> > systems
> > > > that
> > > > > I know of).
> > > > >
> > > > > Independently we can iterate on some of the ideas of the original
> > > > proposal
> > > > > individually and figure out how (if at all) to make use of this
> > > > > functionality. This can be done bit-by-bit:
> > > > > - Could be that the existing StreamTask API ends up wrapping this
> > > > > - Could end up exposed directly in Samza as Yi proposed
> > > > > - Could be that just the lower-level group-management stuff get's
> > used,
> > > > and
> > > > > in this case it could be either just for standalone mode, or always
> > > > > - Could be that it stays as-is
> > > > >
> > > > > The advantage of this is it is lower risk...we basically don't have
> > to
> > > > make
> > > > > 12 major decisions all at once that kind of hinge on what amounts
> to
> > a
> > > > > pretty aggressive rewrite. The disadvantage of this is it is a bit
> > more
> > > > > confusing as all this is getting figured out.
> > > > >
> > > > > As with some of the other stuff, this would require a further
> > > discussion
> > > > in
> > > > > the Kafka community if people do like this approach.
> > > > >
> > > > > Thoughts?
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Sun, Jul 12, 2015 at 10:52 PM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Hey Chris,
> > > > > >
> > > > > > Yeah, I'm obviously in favor of this.
> > > > > >
> > > > > > The sub-project approach seems the ideal way to take a graceful
> > step
> > > in
> > > > > > this direction, so I will ping the board folks and see why they
> are
> > > > > > discouraged, it would be good to understand that. If we go that
> > route
> > > > we
> > > > > > would need to do a similar discussion in the Kafka list (but
> makes
> > > > sense
> > > > > to
> > > > > > figure out first if it is what Samza wants).
> > > > > >
> > > > > > Irrespective of how it's implemented, though, to me the important
> > > > things
> > > > > > are the following:
> > > > > > 1. Unify the website, config, naming, docs, metrics,
> etc--basically
> > > fix
> > > > > > the product experience so the "stream" and the "processing" feel
> > > like a
> > > > > > single user experience and brand. This seems minor but I think
> is a
> > > > > really
> > > > > > big deal.
> > > > > > 2. Make "standalone" mode a first class citizen and have a real
> > > > technical
> > > > > > plan to be able to support cluster managers other than YARN.
> > > > > > 3. Make the config and out-of-the-box experience more usable
> > > > > >
> > > > > > I think that prototype gives a practical example of how 1-3 could
> > be
> > > > done
> > > > > > and we should pursue it. This is a pretty radical change, so I
> > > wouldn't
> > > > > be
> > > > > > shocked if people didn't want to take a step like that.
> > > > > >
> > > > > > Maybe it would make sense to see if people are on board with that
> > > > general
> > > > > > idea, and then try to get some advice on sub-projects in parallel
> > and
> > > > > nail
> > > > > > down those details?
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
> > > > criccomini@apache.org>
> > > > > > wrote:
> > > > > >
> > > > > >> Hey all,
> > > > > >>
> > > > > >> I want to start by saying that I'm absolutely thrilled to be a
> > part
> > > of
> > > > > >> this
> > > > > >> community. The amount of level-headed, thoughtful, educated
> > > discussion
> > > > > >> that's gone on over the past ~10 days is overwhelming.
> Wonderful.
> > > > > >>
> > > > > >> It seems like discussion is waning a bit, and we've reached some
> > > > > >> conclusions. There are several key emails in this threat, which
> I
> > > want
> > > > > to
> > > > > >> call out:
> > > > > >>
> > > > > >> 1. Jakob's summary of the three potential ways forward.
> > > > > >>
> > > > > >>
> > > > > >>
> > > > >
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > > > > >> 2. Julian's call out that we should be focusing on community
> over
> > > > code.
> > > > > >>
> > > > > >>
> > > > > >>
> > > > >
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > > > > >> 3. Martin's summary about the benefits of merging communities.
> > > > > >>
> > > > > >>
> > > > > >>
> > > > >
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > > > > >> 4. Jakob's comments about the distinction between community and
> > code
> > > > > >> paths.
> > > > > >>
> > > > > >>
> > > > > >>
> > > > >
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> > > > > >>
> > > > > >> I agree with the comments on all of these emails. I think
> Martin's
> > > > > summary
> > > > > >> of his position aligns very closely with my own. To that end, I
> > > think
> > > > we
> > > > > >> should get concrete about what the proposal is, and call a vote
> on
> > > it.
> > > > > >> Given that Jay, Martin, and I seem to be aligning fairly
> closely,
> > I
> > > > > think
> > > > > >> we should start with:
> > > > > >>
> > > > > >> 1. [community] Make Samza a subproject of Kafka.
> > > > > >> 2. [community] Make all Samza PMC/committers committers of the
> > > > > subproject.
> > > > > >> 3. [community] Migrate Samza's website/documentation into
> Kafka's.
> > > > > >> 4. [code] Have the Samza community and the Kafka community
> start a
> > > > > >> from-scratch reboot together in the new Kafka subproject. We can
> > > > > >> borrow/copy &  paste significant chunks of code from Samza's
> code
> > > > base.
> > > > > >> 5. [code] The subproject would intentionally eliminate support
> for
> > > > both
> > > > > >> other streaming systems and all deployment systems.
> > > > > >> 6. [code] Attempt to provide a bridge from our SystemConsumer to
> > > > KIP-26
> > > > > >> (copy cat)
> > > > > >> 7. [code] Attempt to provide a bridge from the new subproject's
> > > > > processor
> > > > > >> interface to our legacy StreamTask interface.
> > > > > >> 8. [code/community] Sunset Samza as a TLP when we have a working
> > > Kafka
> > > > > >> subproject that has a fault-tolerant container with state
> > > management.
> > > > > >>
> > > > > >> It's likely that (6) and (7) won't be fully drop-in. Still, the
> > > closer
> > > > > we
> > > > > >> can get, the better it's going to be for our existing community.
> > > > > >>
> > > > > >> One thing that I didn't touch on with (2) is whether any Samza
> PMC
> > > > > members
> > > > > >> should be rolled into Kafka PMC membership as well (though, Jay
> > and
> > > > > Jakob
> > > > > >> are already PMC members on both). I think that Samza's community
> > > > > deserves
> > > > > >> a
> > > > > >> voice on the PMC, so I'd propose that we roll at least a few PMC
> > > > members
> > > > > >> into the Kafka PMC, but I don't have a strong framework for
> which
> > > > people
> > > > > >> to
> > > > > >> pick.
> > > > > >>
> > > > > >> Before (8), I think that Samza's TLP can continue to commit bug
> > > fixes
> > > > > and
> > > > > >> patches as it sees fit, provided that we openly communicate that
> > we
> > > > > won't
> > > > > >> necessarily migrate new features to the new subproject, and that
> > the
> > > > TLP
> > > > > >> will be shut down after the migration to the Kafka subproject
> > > occurs.
> > > > > >>
> > > > > >> Jakob, I could use your guidance here about about how to achieve
> > > this
> > > > > from
> > > > > >> an Apache process perspective (sorry).
> > > > > >>
> > > > > >> * Should I just call a vote on this proposal?
> > > > > >> * Should it happen on dev or private?
> > > > > >> * Do committers have binding votes, or just PMC?
> > > > > >>
> > > > > >> Having trouble finding much detail on the Apache wikis. :(
> > > > > >>
> > > > > >> Cheers,
> > > > > >> Chris
> > > > > >>
> > > > > >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <yanfang724@gmail.com
> >
> > > > wrote:
> > > > > >>
> > > > > >> > Thanks, Jay. This argument persuaded me actually. :)
> > > > > >> >
> > > > > >> > Fang, Yan
> > > > > >> > yanfang724@gmail.com
> > > > > >> >
> > > > > >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io>
> > > > wrote:
> > > > > >> >
> > > > > >> > > Hey Yan,
> > > > > >> > >
> > > > > >> > > Yeah philosophically I think the argument is that you should
> > > > capture
> > > > > >> the
> > > > > >> > > stream in Kafka independent of the transformation. This is
> > > > > obviously a
> > > > > >> > > Kafka-centric view point.
> > > > > >> > >
> > > > > >> > > Advantages of this:
> > > > > >> > > - In practice I think this is what e.g. Storm people often
> end
> > > up
> > > > > >> doing
> > > > > >> > > anyway. You usually need to throttle any access to a live
> > > serving
> > > > > >> > database.
> > > > > >> > > - Can have multiple subscribers and they get the same thing
> > > > without
> > > > > >> > > additional load on the source system.
> > > > > >> > > - Applications can tap into the stream if need be by
> > > subscribing.
> > > > > >> > > - You can debug your transformation by tailing the Kafka
> topic
> > > > with
> > > > > >> the
> > > > > >> > > console consumer
> > > > > >> > > - Can tee off the same data stream for batch analysis or
> > Lambda
> > > > arch
> > > > > >> > style
> > > > > >> > > re-processing
> > > > > >> > >
> > > > > >> > > The disadvantage is that it will use Kafka resources. But
> the
> > > idea
> > > > > is
> > > > > >> > > eventually you will have multiple subscribers to any data
> > source
> > > > (at
> > > > > >> > least
> > > > > >> > > for monitoring) so you will end up there soon enough anyway.
> > > > > >> > >
> > > > > >> > > Down the road the technical benefit is that I think it gives
> > us
> > > a
> > > > > good
> > > > > >> > path
> > > > > >> > > towards end-to-end exactly once semantics from source to
> > > > > destination.
> > > > > >> > > Basically the connectors need to support idempotence when
> > > talking
> > > > to
> > > > > >> > Kafka
> > > > > >> > > and we need the transactional write feature in Kafka to make
> > the
> > > > > >> > > transformation atomic. This is actually pretty doable if you
> > > > > separate
> > > > > >> > > connector=>kafka problem from the generic transformations
> > which
> > > > are
> > > > > >> > always
> > > > > >> > > kafka=>kafka. However I think it is quite impossible to do
> in
> > a
> > > > > >> > all_things
> > > > > >> > > => all_things environment. Today you can say "well the
> > semantics
> > > > of
> > > > > >> the
> > > > > >> > > Samza APIs depend on the connectors you use" but it is
> > actually
> > > > > worse
> > > > > >> > then
> > > > > >> > > that because the semantics actually depend on the pairing of
> > > > > >> > connectors--so
> > > > > >> > > not only can you probably not get a usable "exactly once"
> > > > guarantee
> > > > > >> > > end-to-end it can actually be quite hard to reverse engineer
> > > what
> > > > > >> > property
> > > > > >> > > (if any) your end-to-end flow has if you have heterogenous
> > > > systems.
> > > > > >> > >
> > > > > >> > > -Jay
> > > > > >> > >
> > > > > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <
> > yanfang724@gmail.com
> > > >
> > > > > >> wrote:
> > > > > >> > >
> > > > > >> > > > {quote}
> > > > > >> > > > maintained in a separate repository and retaining the
> > existing
> > > > > >> > > > committership but sharing as much else as possible
> (website,
> > > > etc)
> > > > > >> > > > {quote}
> > > > > >> > > >
> > > > > >> > > > Overall, I agree on this idea. Now the question is more
> > about
> > > > "how
> > > > > >> to
> > > > > >> > do
> > > > > >> > > > it".
> > > > > >> > > >
> > > > > >> > > > On the other hand, one thing I want to point out is that,
> if
> > > we
> > > > > >> decide
> > > > > >> > to
> > > > > >> > > > go this way, how do we want to support
> > > > > >> > > > otherSystem-transformation-otherSystem use case?
> > > > > >> > > >
> > > > > >> > > > Basically, there are four user groups here:
> > > > > >> > > >
> > > > > >> > > > 1. Kafka-transformation-Kafka
> > > > > >> > > > 2. Kafka-transformation-otherSystem
> > > > > >> > > > 3. otherSystem-transformation-Kafka
> > > > > >> > > > 4. otherSystem-transformation-otherSystem
> > > > > >> > > >
> > > > > >> > > > For group 1, they can easily use the new Samza library to
> > > > achieve.
> > > > > >> For
> > > > > >> > > > group 2 and 3, they can use copyCat -> transformation ->
> > Kafka
> > > > or
> > > > > >> > Kafka->
> > > > > >> > > > transformation -> copyCat.
> > > > > >> > > >
> > > > > >> > > > The problem is for group 4. Do we want to abandon this or
> > > still
> > > > > >> support
> > > > > >> > > it?
> > > > > >> > > > Of course, this use case can be achieved by using copyCat
> ->
> > > > > >> > > transformation
> > > > > >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> > > > > persuade
> > > > > >> > them
> > > > > >> > > to
> > > > > >> > > > do this long chain. If yes, it will also be a win for
> Kafka
> > > too.
> > > > > Or
> > > > > >> if
> > > > > >> > > > there is no one in this community actually doing this so
> > far,
> > > > > maybe
> > > > > >> ok
> > > > > >> > to
> > > > > >> > > > not support the group 4 directly.
> > > > > >> > > >
> > > > > >> > > > Thanks,
> > > > > >> > > >
> > > > > >> > > > Fang, Yan
> > > > > >> > > > yanfang724@gmail.com
> > > > > >> > > >
> > > > > >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <
> > jay@confluent.io
> > > >
> > > > > >> wrote:
> > > > > >> > > >
> > > > > >> > > > > Yeah I agree with this summary. I think there are kind
> of
> > > two
> > > > > >> > questions
> > > > > >> > > > > here:
> > > > > >> > > > > 1. Technically does alignment/reliance on Kafka make
> sense
> > > > > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
> > > > alignment
> > > > > >> with
> > > > > >> > > > Kafka
> > > > > >> > > > > make sense
> > > > > >> > > > >
> > > > > >> > > > > Personally I do think both of these things would be
> really
> > > > > >> valuable,
> > > > > >> > > and
> > > > > >> > > > > would dramatically alter the trajectory of the project.
> > > > > >> > > > >
> > > > > >> > > > > My preference would be to see if people can mostly agree
> > on
> > > a
> > > > > >> > direction
> > > > > >> > > > > rather than splintering things off. From my point of
> view
> > > the
> > > > > >> ideal
> > > > > >> > > > outcome
> > > > > >> > > > > of all the options discussed would be to make Samza a
> > > closely
> > > > > >> aligned
> > > > > >> > > > > subproject, maintained in a separate repository and
> > > retaining
> > > > > the
> > > > > >> > > > existing
> > > > > >> > > > > committership but sharing as much else as possible
> > (website,
> > > > > >> etc). No
> > > > > >> > > > idea
> > > > > >> > > > > about how these things work, Jacob, you probably know
> > more.
> > > > > >> > > > >
> > > > > >> > > > > No discussion amongst the Kafka folks has happened on
> > this,
> > > > but
> > > > > >> > likely
> > > > > >> > > we
> > > > > >> > > > > should figure out what the Samza community actually
> wants
> > > > first.
> > > > > >> > > > >
> > > > > >> > > > > I admit that this is a fairly radical departure from how
> > > > things
> > > > > >> are.
> > > > > >> > > > >
> > > > > >> > > > > If that doesn't fly, I think, yeah we could leave Samza
> as
> > > it
> > > > is
> > > > > >> and
> > > > > >> > do
> > > > > >> > > > the
> > > > > >> > > > > more radical reboot inside Kafka. From my point of view
> > that
> > > > > does
> > > > > >> > leave
> > > > > >> > > > > things in a somewhat confusing state since now there are
> > two
> > > > > >> stream
> > > > > >> > > > > processing systems more or less coupled to Kafka in
> large
> > > part
> > > > > >> made
> > > > > >> > by
> > > > > >> > > > the
> > > > > >> > > > > same people. But, arguably that might be a cleaner way
> to
> > > make
> > > > > the
> > > > > >> > > > cut-over
> > > > > >> > > > > and perhaps less risky for Samza community since if it
> > works
> > > > > >> people
> > > > > >> > can
> > > > > >> > > > > switch and if it doesn't nothing will have changed.
> Dunno,
> > > how
> > > > > do
> > > > > >> > > people
> > > > > >> > > > > feel about this?
> > > > > >> > > > >
> > > > > >> > > > > -Jay
> > > > > >> > > > >
> > > > > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> > > > > jghoman@gmail.com>
> > > > > >> > > wrote:
> > > > > >> > > > >
> > > > > >> > > > > > >  This leads me to thinking that merging projects and
> > > > > >> communities
> > > > > >> > > > might
> > > > > >> > > > > > be a good idea: with the union of experience from both
> > > > > >> communities,
> > > > > >> > > we
> > > > > >> > > > > will
> > > > > >> > > > > > probably build a better system that is better for
> users.
> > > > > >> > > > > > Is this what's being proposed though? Merging the
> > projects
> > > > > seems
> > > > > >> > like
> > > > > >> > > > > > a consequence of at most one of the three directions
> > under
> > > > > >> > > discussion:
> > > > > >> > > > > > 1) Samza 2.0: The Samza community relies more heavily
> on
> > > > Kafka
> > > > > >> for
> > > > > >> > > > > > configuration, etc. (to a greater or lesser extent to
> be
> > > > > >> > determined)
> > > > > >> > > > > > but the Samza community would not automatically merge
> > > withe
> > > > > >> Kafka
> > > > > >> > > > > > community (the Phoenix/HBase example is a good one
> > here).
> > > > > >> > > > > > 2) Samza Reboot: The Samza community continues to
> exist
> > > > with a
> > > > > >> > > limited
> > > > > >> > > > > > project scope, but similarly would not need to be part
> > of
> > > > the
> > > > > >> Kafka
> > > > > >> > > > > > community (ie given committership) to progress.  Here,
> > > maybe
> > > > > the
> > > > > >> > > Samza
> > > > > >> > > > > > team would become a subproject of Kafka (the Board
> > frowns
> > > on
> > > > > >> > > > > > subprojects at the moment, so I'm not sure if that's
> > even
> > > > > >> > feasible),
> > > > > >> > > > > > but that would not be required.
> > > > > >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this
> > > option
> > > > > the
> > > > > >> > Kafka
> > > > > >> > > > > > team builds its own streaming library, possibly off of
> > > Jay's
> > > > > >> > > > > > prototype, which has not direct lineage to the Samza
> > team.
> > > > > >> There's
> > > > > >> > > no
> > > > > >> > > > > > reason for the Kafka team to bring in the Samza team.
> > > > > >> > > > > >
> > > > > >> > > > > > Is the Kafka community on board with this?
> > > > > >> > > > > >
> > > > > >> > > > > > To be clear, all three options under discussion are
> > > > > interesting,
> > > > > >> > > > > > technically valid and likely healthy directions for
> the
> > > > > project.
> > > > > >> > > > > > Also, they are not mutually exclusive.  The Samza
> > > community
> > > > > >> could
> > > > > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
> > > > community
> > > > > >> went
> > > > > >> > > > > > forward with 'Hey Samza!'  My points above are
> directed
> > > > > >> entirely at
> > > > > >> > > > > > the community aspect of these choices.
> > > > > >> > > > > > -Jakob
> > > > > >> > > > > >
> > > > > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> > > > > roger.hoover@gmail.com>
> > > > > >> > > wrote:
> > > > > >> > > > > > > That's great.  Thanks, Jay.
> > > > > >> > > > > > >
> > > > > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> > > > > jay@confluent.io>
> > > > > >> > > wrote:
> > > > > >> > > > > > >
> > > > > >> > > > > > >> Yeah totally agree. I think you have this issue
> even
> > > > today,
> > > > > >> > right?
> > > > > >> > > > > I.e.
> > > > > >> > > > > > if
> > > > > >> > > > > > >> you need to make a simple config change and you're
> > > > running
> > > > > in
> > > > > >> > YARN
> > > > > >> > > > > today
> > > > > >> > > > > > >> you end up bouncing the job which then rebuilds
> > state.
> > > I
> > > > > >> think
> > > > > >> > the
> > > > > >> > > > fix
> > > > > >> > > > > > is
> > > > > >> > > > > > >> exactly what you described which is to have a long
> > > > timeout
> > > > > on
> > > > > >> > > > > partition
> > > > > >> > > > > > >> movement for stateful jobs so that if a job is just
> > > > getting
> > > > > >> > > bounced,
> > > > > >> > > > > and
> > > > > >> > > > > > >> the cluster manager (or admin) is smart enough to
> > > restart
> > > > > it
> > > > > >> on
> > > > > >> > > the
> > > > > >> > > > > same
> > > > > >> > > > > > >> host when possible, it can optimistically reuse any
> > > > > existing
> > > > > >> > state
> > > > > >> > > > it
> > > > > >> > > > > > finds
> > > > > >> > > > > > >> on disk (if it is valid).
> > > > > >> > > > > > >>
> > > > > >> > > > > > >> So in this model the charter of the CM is to place
> > > > > processes
> > > > > >> as
> > > > > >> > > > > > stickily as
> > > > > >> > > > > > >> possible and to restart or re-place failed
> processes.
> > > The
> > > > > >> > charter
> > > > > >> > > of
> > > > > >> > > > > the
> > > > > >> > > > > > >> partition management system is to control the
> > > assignment
> > > > of
> > > > > >> work
> > > > > >> > > to
> > > > > >> > > > > > these
> > > > > >> > > > > > >> processes. The nice thing about this is that the
> work
> > > > > >> > assignment,
> > > > > >> > > > > > timeouts,
> > > > > >> > > > > > >> behavior, configs, and code will all be the same
> > across
> > > > all
> > > > > >> > > cluster
> > > > > >> > > > > > >> managers.
> > > > > >> > > > > > >>
> > > > > >> > > > > > >> So I think that prototype would actually give you
> > > exactly
> > > > > >> what
> > > > > >> > you
> > > > > >> > > > > want
> > > > > >> > > > > > >> today for any cluster manager (or manual placement
> +
> > > > > restart
> > > > > >> > > script)
> > > > > >> > > > > > that
> > > > > >> > > > > > >> was sticky in terms of host placement since there
> is
> > > > > already
> > > > > >> a
> > > > > >> > > > > > configurable
> > > > > >> > > > > > >> partition movement timeout and task-by-task state
> > reuse
> > > > > with
> > > > > >> a
> > > > > >> > > check
> > > > > >> > > > > on
> > > > > >> > > > > > >> state validity.
> > > > > >> > > > > > >>
> > > > > >> > > > > > >> -Jay
> > > > > >> > > > > > >>
> > > > > >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > > > > >> > > > roger.hoover@gmail.com
> > > > > >> > > > > >
> > > > > >> > > > > > >> wrote:
> > > > > >> > > > > > >>
> > > > > >> > > > > > >> > That would be great to let Kafka do as much heavy
> > > > lifting
> > > > > >> as
> > > > > >> > > > > possible
> > > > > >> > > > > > and
> > > > > >> > > > > > >> > make it easier for other languages to implement
> > Samza
> > > > > apis.
> > > > > >> > > > > > >> >
> > > > > >> > > > > > >> > One thing to watch out for is the interplay
> between
> > > > > Kafka's
> > > > > >> > > group
> > > > > >> > > > > > >> > management and the external scheduler/process
> > > manager's
> > > > > >> fault
> > > > > >> > > > > > tolerance.
> > > > > >> > > > > > >> > If a container dies, the Kafka group membership
> > > > protocol
> > > > > >> will
> > > > > >> > > try
> > > > > >> > > > to
> > > > > >> > > > > > >> assign
> > > > > >> > > > > > >> > it's tasks to other containers while at the same
> > time
> > > > the
> > > > > >> > > process
> > > > > >> > > > > > manager
> > > > > >> > > > > > >> > is trying to relaunch the container.  Without
> some
> > > > > >> > consideration
> > > > > >> > > > for
> > > > > >> > > > > > this
> > > > > >> > > > > > >> > (like a configurable amount of time to wait
> before
> > > > Kafka
> > > > > >> > alters
> > > > > >> > > > the
> > > > > >> > > > > > group
> > > > > >> > > > > > >> > membership), there may be thrashing going on
> which
> > is
> > > > > >> > especially
> > > > > >> > > > bad
> > > > > >> > > > > > for
> > > > > >> > > > > > >> > containers with large amounts of local state.
> > > > > >> > > > > > >> >
> > > > > >> > > > > > >> > Someone else pointed this out already but I
> thought
> > > it
> > > > > >> might
> > > > > >> > be
> > > > > >> > > > > worth
> > > > > >> > > > > > >> > calling out again.
> > > > > >> > > > > > >> >
> > > > > >> > > > > > >> > Cheers,
> > > > > >> > > > > > >> >
> > > > > >> > > > > > >> > Roger
> > > > > >> > > > > > >> >
> > > > > >> > > > > > >> >
> > > > > >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> > > > > >> jay@confluent.io>
> > > > > >> > > > > wrote:
> > > > > >> > > > > > >> >
> > > > > >> > > > > > >> > > Hey Roger,
> > > > > >> > > > > > >> > >
> > > > > >> > > > > > >> > > I couldn't agree more. We spent a bunch of time
> > > > talking
> > > > > >> to
> > > > > >> > > > people
> > > > > >> > > > > > and
> > > > > >> > > > > > >> > that
> > > > > >> > > > > > >> > > is exactly the stuff we heard time and again.
> > What
> > > > > makes
> > > > > >> it
> > > > > >> > > > hard,
> > > > > >> > > > > of
> > > > > >> > > > > > >> > > course, is that there is some tension between
> > > > > >> compatibility
> > > > > >> > > with
> > > > > >> > > > > > what's
> > > > > >> > > > > > >> > > there now and making things better for new
> users.
> > > > > >> > > > > > >> > >
> > > > > >> > > > > > >> > > I also strongly agree with the importance of
> > > > > >> multi-language
> > > > > >> > > > > > support. We
> > > > > >> > > > > > >> > are
> > > > > >> > > > > > >> > > talking now about Java, but for application
> > > > development
> > > > > >> use
> > > > > >> > > > cases
> > > > > >> > > > > > >> people
> > > > > >> > > > > > >> > > want to work in whatever language they are
> using
> > > > > >> elsewhere.
> > > > > >> > I
> > > > > >> > > > > think
> > > > > >> > > > > > >> > moving
> > > > > >> > > > > > >> > > to a model where Kafka itself does the group
> > > > > membership,
> > > > > >> > > > lifecycle
> > > > > >> > > > > > >> > control,
> > > > > >> > > > > > >> > > and partition assignment has the advantage of
> > > putting
> > > > > all
> > > > > >> > that
> > > > > >> > > > > > complex
> > > > > >> > > > > > >> > > stuff behind a clean api that the clients are
> > > already
> > > > > >> going
> > > > > >> > to
> > > > > >> > > > be
> > > > > >> > > > > > >> > > implementing for their consumer, so the added
> > > > > >> functionality
> > > > > >> > > for
> > > > > >> > > > > > stream
> > > > > >> > > > > > >> > > processing beyond a consumer becomes very
> minor.
> > > > > >> > > > > > >> > >
> > > > > >> > > > > > >> > > -Jay
> > > > > >> > > > > > >> > >
> > > > > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > > > > >> > > > > > roger.hoover@gmail.com>
> > > > > >> > > > > > >> > > wrote:
> > > > > >> > > > > > >> > >
> > > > > >> > > > > > >> > > > Metamorphosis...nice. :)
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > > This has been a great discussion.  As a user
> of
> > > > Samza
> > > > > >> > who's
> > > > > >> > > > > > recently
> > > > > >> > > > > > >> > > > integrated it into a relatively large
> > > > organization, I
> > > > > >> just
> > > > > >> > > > want
> > > > > >> > > > > to
> > > > > >> > > > > > >> add
> > > > > >> > > > > > >> > > > support to a few points already made.
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as
> it
> > > > > >> currently
> > > > > >> > > > exists
> > > > > >> > > > > > that
> > > > > >> > > > > > >> > I've
> > > > > >> > > > > > >> > > > experienced are:
> > > > > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
> > > > environments
> > > > > >> > where
> > > > > >> > > > > Puppet
> > > > > >> > > > > > >> > would
> > > > > >> > > > > > >> > > do
> > > > > >> > > > > > >> > > > just fine but it was the only mechanism to
> get
> > > > fault
> > > > > >> > > > tolerance.
> > > > > >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> > > > > >> configuring
> > > > > >> > > most
> > > > > >> > > > > of
> > > > > >> > > > > > the
> > > > > >> > > > > > >> > job
> > > > > >> > > > > > >> > > > in code rather than config files.  In
> general,
> > I
> > > > > think
> > > > > >> the
> > > > > >> > > > goal
> > > > > >> > > > > > >> should
> > > > > >> > > > > > >> > be
> > > > > >> > > > > > >> > > > to make it harder to make mistakes,
> especially
> > of
> > > > the
> > > > > >> kind
> > > > > >> > > > where
> > > > > >> > > > > > the
> > > > > >> > > > > > >> > code
> > > > > >> > > > > > >> > > > expects something and the config doesn't
> match.
> > > > The
> > > > > >> > current
> > > > > >> > > > > > config
> > > > > >> > > > > > >> is
> > > > > >> > > > > > >> > > > quite intricate and error-prone.  For
> example,
> > > the
> > > > > >> > > application
> > > > > >> > > > > > logic
> > > > > >> > > > > > >> > may
> > > > > >> > > > > > >> > > > depend on bootstrapping a topic but rather
> than
> > > > > >> asserting
> > > > > >> > > that
> > > > > >> > > > > in
> > > > > >> > > > > > the
> > > > > >> > > > > > >> > > code,
> > > > > >> > > > > > >> > > > you have to rely on getting the config right.
> > > > > Likewise
> > > > > >> > with
> > > > > >> > > > > > serdes,
> > > > > >> > > > > > >> > the
> > > > > >> > > > > > >> > > > Java representations produced by various
> serdes
> > > > > (JSON,
> > > > > >> > Avro,
> > > > > >> > > > > etc.)
> > > > > >> > > > > > >> are
> > > > > >> > > > > > >> > > not
> > > > > >> > > > > > >> > > > equivalent so you cannot just reconfigure a
> > serde
> > > > > >> without
> > > > > >> > > > > changing
> > > > > >> > > > > > >> the
> > > > > >> > > > > > >> > > > code.   It would be nice for jobs to be able
> to
> > > > > assert
> > > > > >> > what
> > > > > >> > > > they
> > > > > >> > > > > > >> expect
> > > > > >> > > > > > >> > > > from their input topics in terms of
> > partitioning.
> > > > > >> This is
> > > > > >> > > > > > getting a
> > > > > >> > > > > > >> > > little
> > > > > >> > > > > > >> > > > off topic but I was even thinking about
> > creating
> > > a
> > > > > >> "Samza
> > > > > >> > > > config
> > > > > >> > > > > > >> > linter"
> > > > > >> > > > > > >> > > > that would sanity check a set of configs.
> > > > Especially
> > > > > >> in
> > > > > >> > > > > > >> organizations
> > > > > >> > > > > > >> > > > where config is managed by a different team
> > than
> > > > the
> > > > > >> > > > application
> > > > > >> > > > > > >> > > developer,
> > > > > >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > > > > >> > > > > > >> > > > 3) Java/Scala centric - for many teams
> > > (especially
> > > > > >> > > DevOps-type
> > > > > >> > > > > > >> folks),
> > > > > >> > > > > > >> > > the
> > > > > >> > > > > > >> > > > pain of the Java toolchain (maven, slow
> builds,
> > > > weak
> > > > > >> > command
> > > > > >> > > > > line
> > > > > >> > > > > > >> > > support,
> > > > > >> > > > > > >> > > > configuration over convention) really
> inhibits
> > > > > >> > productivity.
> > > > > >> > > > As
> > > > > >> > > > > > more
> > > > > >> > > > > > >> > and
> > > > > >> > > > > > >> > > > more high-quality clients become available
> for
> > > > > Kafka, I
> > > > > >> > hope
> > > > > >> > > > > > they'll
> > > > > >> > > > > > >> > > follow
> > > > > >> > > > > > >> > > > Samza's model.  Not sure how much it affects
> > the
> > > > > >> proposals
> > > > > >> > > in
> > > > > >> > > > > this
> > > > > >> > > > > > >> > thread
> > > > > >> > > > > > >> > > > but please consider other languages in the
> > > > ecosystem
> > > > > as
> > > > > >> > > well.
> > > > > >> > > > > > From
> > > > > >> > > > > > >> > what
> > > > > >> > > > > > >> > > > I've heard, Spark has more Python users than
> > > > > >> Java/Scala.
> > > > > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza
> > API
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > >
> > > > > >> > > > > > >> >
> > > > > >> > > > > > >>
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > > > >> > > > > > >> > > > and are working on a Yeoman generator
> > > > > >> > > > > > >> > > > https://github.com/Quantiply/generator-rico
> > for
> > > > > >> > > Jython/Samza
> > > > > >> > > > > > >> projects
> > > > > >> > > > > > >> > to
> > > > > >> > > > > > >> > > > alleviate some of the pain)
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > > I also want to underscore Jay's point about
> > > > improving
> > > > > >> the
> > > > > >> > > user
> > > > > >> > > > > > >> > > experience.
> > > > > >> > > > > > >> > > > That's a very important factor for
> adoption.  I
> > > > think
> > > > > >> the
> > > > > >> > > goal
> > > > > >> > > > > > should
> > > > > >> > > > > > >> > be
> > > > > >> > > > > > >> > > to
> > > > > >> > > > > > >> > > > make Samza as easy to get started with as
> > > something
> > > > > >> like
> > > > > >> > > > > Logstash.
> > > > > >> > > > > > >> > > > Logstash is vastly inferior in terms of
> > > > capabilities
> > > > > to
> > > > > >> > > Samza
> > > > > >> > > > > but
> > > > > >> > > > > > >> it's
> > > > > >> > > > > > >> > > easy
> > > > > >> > > > > > >> > > > to get started and that makes a big
> difference.
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > > Cheers,
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > > Roger
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> > > > > Francisci
> > > > > >> > > > Morales <
> > > > > >> > > > > > >> > > > gdfm@apache.org> wrote:
> > > > > >> > > > > > >> > > >
> > > > > >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> > > > > >> Metamorphosis
> > > > > >> > > is
> > > > > >> > > > a
> > > > > >> > > > > > clear
> > > > > >> > > > > > >> > > > winner
> > > > > >> > > > > > >> > > > > :)
> > > > > >> > > > > > >> > > > >
> > > > > >> > > > > > >> > > > > --
> > > > > >> > > > > > >> > > > > Gianmarco
> > > > > >> > > > > > >> > > > >
> > > > > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De
> > Francisci
> > > > > >> Morales
> > > > > >> > <
> > > > > >> > > > > > >> > > gdfm@apache.org
> > > > > >> > > > > > >> > > > >
> > > > > >> > > > > > >> > > > > wrote:
> > > > > >> > > > > > >> > > > >
> > > > > >> > > > > > >> > > > > > Hi,
> > > > > >> > > > > > >> > > > > >
> > > > > >> > > > > > >> > > > > > @Martin, thanks for you comments.
> > > > > >> > > > > > >> > > > > > Maybe I'm missing some important point,
> > but I
> > > > > think
> > > > > >> > > > coupling
> > > > > >> > > > > > the
> > > > > >> > > > > > >> > > > releases
> > > > > >> > > > > > >> > > > > > is actually a *good* thing.
> > > > > >> > > > > > >> > > > > > To make an example, would it be better if
> > the
> > > > MR
> > > > > >> and
> > > > > >> > > HDFS
> > > > > >> > > > > > >> > components
> > > > > >> > > > > > >> > > of
> > > > > >> > > > > > >> > > > > > Hadoop had different release schedules?
> > > > > >> > > > > > >> > > > > >
> > > > > >> > > > > > >> > > > > > Actually, keeping the discussion in a
> > single
> > > > > place
> > > > > >> > would
> > > > > >> > > > > make
> > > > > >> > > > > > >> > > agreeing
> > > > > >> > > > > > >> > > > on
> > > > > >> > > > > > >> > > > > > releases (and backwards compatibility)
> much
> > > > > >> easier, as
> > > > > >> > > > > > everybody
> > > > > >> > > > > > >> > > would
> > > > > >> > > > > > >> > > > be
> > > > > >> > > > > > >> > > > > > responsible for the whole codebase.
> > > > > >> > > > > > >> > > > > >
> > > > > >> > > > > > >> > > > > > That said, I like the idea of absorbing
> > > > > samza-core
> > > > > >> as
> > > > > >> > a
> > > > > >> > > > > > >> > sub-project,
> > > > > >> > > > > > >> > > > and
> > > > > >> > > > > > >> > > > > > leave the fancy stuff separate.
> > > > > >> > > > > > >> > > > > > It probably gives 90% of the benefits we
> > have
> > > > > been
> > > > > >> > > > > discussing
> > > > > >> > > > > > >> here.
> > > > > >> > > > > > >> > > > > >
> > > > > >> > > > > > >> > > > > > Cheers,
> > > > > >> > > > > > >> > > > > >
> > > > > >> > > > > > >> > > > > > --
> > > > > >> > > > > > >> > > > > > Gianmarco
> > > > > >> > > > > > >> > > > > >
> > > > > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > > > > >> > jay.kreps@gmail.com
> > > > > >> > > >
> > > > > >> > > > > > wrote:
> > > > > >> > > > > > >> > > > > >
> > > > > >> > > > > > >> > > > > >> Hey Martin,
> > > > > >> > > > > > >> > > > > >>
> > > > > >> > > > > > >> > > > > >> I agree coupling release schedules is a
> > > > > downside.
> > > > > >> > > > > > >> > > > > >>
> > > > > >> > > > > > >> > > > > >> Definitely we can try to solve some of
> the
> > > > > >> > integration
> > > > > >> > > > > > problems
> > > > > >> > > > > > >> in
> > > > > >> > > > > > >> > > > > >> Confluent Platform or in other
> > > distributions.
> > > > > But
> > > > > >> I
> > > > > >> > > think
> > > > > >> > > > > > this
> > > > > >> > > > > > >> > ends
> > > > > >> > > > > > >> > > up
> > > > > >> > > > > > >> > > > > >> being really shallow. I guess I feel to
> > > really
> > > > > >> get a
> > > > > >> > > good
> > > > > >> > > > > > user
> > > > > >> > > > > > >> > > > > experience
> > > > > >> > > > > > >> > > > > >> the two systems have to kind of feel
> like
> > > part
> > > > > of
> > > > > >> the
> > > > > >> > > > same
> > > > > >> > > > > > thing
> > > > > >> > > > > > >> > and
> > > > > >> > > > > > >> > > > you
> > > > > >> > > > > > >> > > > > >> can't really add that in later--you can
> > put
> > > > both
> > > > > >> in
> > > > > >> > the
> > > > > >> > > > > same
> > > > > >> > > > > > >> > > > > downloadable
> > > > > >> > > > > > >> > > > > >> tar file but it doesn't really give a
> very
> > > > > >> cohesive
> > > > > >> > > > > feeling.
> > > > > >> > > > > > I
> > > > > >> > > > > > >> > agree
> > > > > >> > > > > > >> > > > > that
> > > > > >> > > > > > >> > > > > >> ultimately any of the project stuff is
> as
> > > much
> > > > > >> social
> > > > > >> > > and
> > > > > >> > > > > > naming
> > > > > >> > > > > > >> > as
> > > > > >> > > > > > >> > > > > >> anything else--theoretically two totally
> > > > > >> independent
> > > > > >> > > > > projects
> > > > > >> > > > > > >> > could
> > > > > >> > > > > > >> > > > work
> > > > > >> > > > > > >> > > > > >> to
> > > > > >> > > > > > >> > > > > >> tightly align. In practice this seems to
> > be
> > > > > quite
> > > > > >> > > > difficult
> > > > > >> > > > > > >> > though.
> > > > > >> > > > > > >> > > > > >>
> > > > > >> > > > > > >> > > > > >> For the frameworks--totally agree it
> would
> > > be
> > > > > >> good to
> > > > > >> > > > > > maintain
> > > > > >> > > > > > >> the
> > > > > >> > > > > > >> > > > > >> framework support with the project. In
> > some
> > > > > cases
> > > > > >> > there
> > > > > >> > > > may
> > > > > >> > > > > > not
> > > > > >> > > > > > >> be
> > > > > >> > > > > > >> > > too
> > > > > >> > > > > > >> > > > > >> much
> > > > > >> > > > > > >> > > > > >> there since the integration gets lighter
> > > but I
> > > > > >> think
> > > > > >> > > > > whatever
> > > > > >> > > > > > >> > stubs
> > > > > >> > > > > > >> > > > you
> > > > > >> > > > > > >> > > > > >> need should be included. So no I
> > definitely
> > > > > wasn't
> > > > > >> > > trying
> > > > > >> > > > > to
> > > > > >> > > > > > >> imply
> > > > > >> > > > > > >> > > > > >> dropping
> > > > > >> > > > > > >> > > > > >> support for these frameworks, just
> making
> > > the
> > > > > >> > > integration
> > > > > >> > > > > > >> lighter
> > > > > >> > > > > > >> > by
> > > > > >> > > > > > >> > > > > >> separating process management from
> > partition
> > > > > >> > > management.
> > > > > >> > > > > > >> > > > > >>
> > > > > >> > > > > > >> > > > > >> You raise two good points we would have
> to
> > > > > figure
> > > > > >> out
> > > > > >> > > if
> > > > > >> > > > we
> > > > > >> > > > > > went
> > > > > >> > > > > > >> > > down
> > > > > >> > > > > > >> > > > > the
> > > > > >> > > > > > >> > > > > >> alignment path:
> > > > > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I
> think
> > > the
> > > > > >> first
> > > > > >> > > > > question
> > > > > >> > > > > > is
> > > > > >> > > > > > >> > > > whether
> > > > > >> > > > > > >> > > > > >> some "re-branding" would be worth it. If
> > so
> > > > > then I
> > > > > >> > > think
> > > > > >> > > > we
> > > > > >> > > > > > can
> > > > > >> > > > > > >> > > have a
> > > > > >> > > > > > >> > > > > big
> > > > > >> > > > > > >> > > > > >> thread on the name. I'm definitely not
> set
> > > on
> > > > > >> Kafka
> > > > > >> > > > > > Streaming or
> > > > > >> > > > > > >> > > Kafka
> > > > > >> > > > > > >> > > > > >> Streams I was just using them to be kind
> > of
> > > > > >> > > > illustrative. I
> > > > > >> > > > > > >> agree
> > > > > >> > > > > > >> > > with
> > > > > >> > > > > > >> > > > > >> your
> > > > > >> > > > > > >> > > > > >> critique of these names, though I think
> > > people
> > > > > >> would
> > > > > >> > > get
> > > > > >> > > > > the
> > > > > >> > > > > > >> idea.
> > > > > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point
> about
> > > how
> > > > to
> > > > > >> > > "factor"
> > > > > >> > > > > it.
> > > > > >> > > > > > >> Here
> > > > > >> > > > > > >> > > are
> > > > > >> > > > > > >> > > > > the
> > > > > >> > > > > > >> > > > > >> options I see (I could get enthusiastic
> > > about
> > > > > any
> > > > > >> of
> > > > > >> > > > them):
> > > > > >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > > > > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> > > > > seperation
> > > > > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of
> > samza-api
> > > > and
> > > > > >> > > > samza-core
> > > > > >> > > > > > is
> > > > > >> > > > > > >> > > > absorbed
> > > > > >> > > > > > >> > > > > >> almost like a third client
> > > > > >> > > > > > >> > > > > >>
> > > > > >> > > > > > >> > > > > >> Cheers,
> > > > > >> > > > > > >> > > > > >>
> > > > > >> > > > > > >> > > > > >> -Jay
> > > > > >> > > > > > >> > > > > >>
> > > > > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> > > > > Kleppmann <
> > > > > >> > > > > > >> > > > martin@kleppmann.com>
> > > > > >> > > > > > >> > > > > >> wrote:
> > > > > >> > > > > > >> > > > > >>
> > > > > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications.
> Just
> > a
> > > > few
> > > > > >> > > follow-up
> > > > > >> > > > > > >> > comments.
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > > >> > - I see the appeal of merging with
> Kafka
> > > or
> > > > > >> > becoming
> > > > > >> > > a
> > > > > >> > > > > > >> > subproject:
> > > > > >> > > > > > >> > > > the
> > > > > >> > > > > > >> > > > > >> > reasons you mention are good. The
> risk I
> > > see
> > > > > is
> > > > > >> > that
> > > > > >> > > > > > release
> > > > > >> > > > > > >> > > > schedules
> > > > > >> > > > > > >> > > > > >> > become coupled to each other, which
> can
> > > slow
> > > > > >> > everyone
> > > > > >> > > > > down,
> > > > > >> > > > > > >> and
> > > > > >> > > > > > >> > > > large
> > > > > >> > > > > > >> > > > > >> > projects with many contributors are
> > harder
> > > > to
> > > > > >> > manage.
> > > > > >> > > > > > (Jakob,
> > > > > >> > > > > > >> > can
> > > > > >> > > > > > >> > > > you
> > > > > >> > > > > > >> > > > > >> speak
> > > > > >> > > > > > >> > > > > >> > from experience, having seen a wider
> > range
> > > > of
> > > > > >> > Hadoop
> > > > > >> > > > > > ecosystem
> > > > > >> > > > > > >> > > > > >> projects?)
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > > >> > Some of the goals of a better unified
> > > > > developer
> > > > > >> > > > > experience
> > > > > >> > > > > > >> could
> > > > > >> > > > > > >> > > > also
> > > > > >> > > > > > >> > > > > be
> > > > > >> > > > > > >> > > > > >> > solved by integrating Samza nicely
> into
> > a
> > > > > Kafka
> > > > > >> > > > > > distribution
> > > > > >> > > > > > >> > (such
> > > > > >> > > > > > >> > > > as
> > > > > >> > > > > > >> > > > > >> > Confluent's). I'm not against merging
> > > > projects
> > > > > >> if
> > > > > >> > we
> > > > > >> > > > > decide
> > > > > >> > > > > > >> > that's
> > > > > >> > > > > > >> > > > the
> > > > > >> > > > > > >> > > > > >> way
> > > > > >> > > > > > >> > > > > >> > to go, just pointing out the same
> goals
> > > can
> > > > > >> perhaps
> > > > > >> > > > also
> > > > > >> > > > > be
> > > > > >> > > > > > >> > > achieved
> > > > > >> > > > > > >> > > > > in
> > > > > >> > > > > > >> > > > > >> > other ways.
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
> > > > dependency:
> > > > > >> are
> > > > > >> > > you
> > > > > >> > > > > > >> proposing
> > > > > >> > > > > > >> > > > that
> > > > > >> > > > > > >> > > > > >> > Samza doesn't give any help to people
> > > > wanting
> > > > > to
> > > > > >> > run
> > > > > >> > > on
> > > > > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > > > > >> > > > > > >> > > > > >> > So the docs would basically have a
> link
> > to
> > > > > >> Slider
> > > > > >> > and
> > > > > >> > > > > > nothing
> > > > > >> > > > > > >> > > else?
> > > > > >> > > > > > >> > > > Or
> > > > > >> > > > > > >> > > > > >> > would we maintain integrations with a
> > > bunch
> > > > of
> > > > > >> > > popular
> > > > > >> > > > > > >> > deployment
> > > > > >> > > > > > >> > > > > >> methods
> > > > > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell
> > scripts
> > > > to
> > > > > >> make
> > > > > >> > > > Samza
> > > > > >> > > > > > work
> > > > > >> > > > > > >> > with
> > > > > >> > > > > > >> > > > > >> Slider)?
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > > >> > I absolutely think it's a good idea to
> > > have
> > > > > the
> > > > > >> > "as a
> > > > > >> > > > > > library"
> > > > > >> > > > > > >> > and
> > > > > >> > > > > > >> > > > > "as a
> > > > > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options
> > for
> > > > > >> people
> > > > > >> > who
> > > > > >> > > > > want
> > > > > >> > > > > > >> them,
> > > > > >> > > > > > >> > > > but I
> > > > > >> > > > > > >> > > > > >> > think there should also be a
> > low-friction
> > > > path
> > > > > >> for
> > > > > >> > > > common
> > > > > >> > > > > > "as
> > > > > >> > > > > > >> a
> > > > > >> > > > > > >> > > > > service"
> > > > > >> > > > > > >> > > > > >> > deployment methods, for which we
> > probably
> > > > need
> > > > > >> to
> > > > > >> > > > > maintain
> > > > > >> > > > > > >> > > > > integrations.
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams"
> seems
> > > odd
> > > > to
> > > > > >> me,
> > > > > >> > > > > because
> > > > > >> > > > > > >> Kafka
> > > > > >> > > > > > >> > > is
> > > > > >> > > > > > >> > > > > all
> > > > > >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> > > > > >> Transformers"
> > > > > >> > > or
> > > > > >> > > > > > "Kafka
> > > > > >> > > > > > >> > > > Filters"
> > > > > >> > > > > > >> > > > > >> > would be more apt?
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of
> > Samza
> > > > > >> (stream
> > > > > >> > > > > > >> transformation
> > > > > >> > > > > > >> > > > with
> > > > > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza
> as a
> > > > > >> library"
> > > > > >> > > bit)
> > > > > >> > > > > > could
> > > > > >> > > > > > >> > > become
> > > > > >> > > > > > >> > > > > >> part of
> > > > > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such
> as
> > > > > >> streaming
> > > > > >> > SQL
> > > > > >> > > > and
> > > > > >> > > > > > >> > > > integrations
> > > > > >> > > > > > >> > > > > >> with
> > > > > >> > > > > > >> > > > > >> > deployment frameworks remain in a
> > separate
> > > > > >> project?
> > > > > >> > > In
> > > > > >> > > > > > other
> > > > > >> > > > > > >> > > words,
> > > > > >> > > > > > >> > > > > >> Kafka
> > > > > >> > > > > > >> > > > > >> > would absorb the proven, stable core
> of
> > > > Samza,
> > > > > >> > which
> > > > > >> > > > > would
> > > > > >> > > > > > >> > become
> > > > > >> > > > > > >> > > > the
> > > > > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early
> in
> > > this
> > > > > >> > thread.
> > > > > >> > > > The
> > > > > >> > > > > > Samza
> > > > > >> > > > > > >> > > > project
> > > > > >> > > > > > >> > > > > >> > would then target that third Kafka
> > client
> > > as
> > > > > its
> > > > > >> > base
> > > > > >> > > > > API,
> > > > > >> > > > > > and
> > > > > >> > > > > > >> > the
> > > > > >> > > > > > >> > > > > >> project
> > > > > >> > > > > > >> > > > > >> > would be freed up to explore more
> > > > experimental
> > > > > >> new
> > > > > >> > > > > > horizons.
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > > >> > Martin
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > > > > >> > > > jay.kreps@gmail.com>
> > > > > >> > > > > > >> wrote:
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > > >> > > Hey Martin,
> > > > > >> > > > > > >> > > > > >> > >
> > > > > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I
> > > > actually
> > > > > >> > don't
> > > > > >> > > > > think
> > > > > >> > > > > > it
> > > > > >> > > > > > >> > ties
> > > > > >> > > > > > >> > > > our
> > > > > >> > > > > > >> > > > > >> > hands
> > > > > >> > > > > > >> > > > > >> > > at all, all it does is refactor
> > things.
> > > > The
> > > > > >> > > division
> > > > > >> > > > of
> > > > > >> > > > > > >> > > > > >> responsibility is
> > > > > >> > > > > > >> > > > > >> > > that Samza core is responsible for
> > task
> > > > > >> > lifecycle,
> > > > > >> > > > > state,
> > > > > >> > > > > > >> and
> > > > > >> > > > > > >> > > > > >> partition
> > > > > >> > > > > > >> > > > > >> > > management (using the Kafka
> > > co-ordinator)
> > > > > but
> > > > > >> it
> > > > > >> > is
> > > > > >> > > > NOT
> > > > > >> > > > > > >> > > > responsible
> > > > > >> > > > > > >> > > > > >> for
> > > > > >> > > > > > >> > > > > >> > > packaging, configuration deployment
> or
> > > > > >> execution
> > > > > >> > of
> > > > > >> > > > > > >> processes.
> > > > > >> > > > > > >> > > The
> > > > > >> > > > > > >> > > > > >> > problem
> > > > > >> > > > > > >> > > > > >> > > of packaging and starting these
> > > processes
> > > > is
> > > > > >> > > > > > >> > > > > >> > > framework/environment-specific. This
> > > > leaves
> > > > > >> > > > individual
> > > > > >> > > > > > >> > > frameworks
> > > > > >> > > > > > >> > > > to
> > > > > >> > > > > > >> > > > > >> be
> > > > > >> > > > > > >> > > > > >> > as
> > > > > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So
> you
> > > can
> > > > > get
> > > > > >> > > simple
> > > > > >> > > > > > >> stateless
> > > > > >> > > > > > >> > > > > >> support in
> > > > > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their
> > > off-the-shelf
> > > > > app
> > > > > >> > > > > framework
> > > > > >> > > > > > >> > > (Slider,
> > > > > >> > > > > > >> > > > > >> > Marathon,
> > > > > >> > > > > > >> > > > > >> > > etc). These are well known by people
> > and
> > > > > have
> > > > > >> > nice
> > > > > >> > > > UIs
> > > > > >> > > > > > and a
> > > > > >> > > > > > >> > lot
> > > > > >> > > > > > >> > > > of
> > > > > >> > > > > > >> > > > > >> > > flexibility. I don't think they have
> > > node
> > > > > >> > affinity
> > > > > >> > > > as a
> > > > > >> > > > > > >> built
> > > > > >> > > > > > >> > in
> > > > > >> > > > > > >> > > > > >> option
> > > > > >> > > > > > >> > > > > >> > > (though I could be wrong). So if we
> > want
> > > > > that
> > > > > >> we
> > > > > >> > > can
> > > > > >> > > > > > either
> > > > > >> > > > > > >> > wait
> > > > > >> > > > > > >> > > > for
> > > > > >> > > > > > >> > > > > >> them
> > > > > >> > > > > > >> > > > > >> > > to add it or do a custom framework
> to
> > > add
> > > > > that
> > > > > >> > > > feature
> > > > > >> > > > > > (as
> > > > > >> > > > > > >> > now).
> > > > > >> > > > > > >> > > > > >> > Obviously
> > > > > >> > > > > > >> > > > > >> > > if you manage things with old-school
> > ops
> > > > > tools
> > > > > >> > > > > > >> > (puppet/chef/etc)
> > > > > >> > > > > > >> > > > you
> > > > > >> > > > > > >> > > > > >> get
> > > > > >> > > > > > >> > > > > >> > > locality easily. The nice thing,
> > though,
> > > > is
> > > > > >> that
> > > > > >> > > all
> > > > > >> > > > > the
> > > > > >> > > > > > >> samza
> > > > > >> > > > > > >> > > > > >> "business
> > > > > >> > > > > > >> > > > > >> > > logic" around partition management
> and
> > > > fault
> > > > > >> > > > tolerance
> > > > > >> > > > > > is in
> > > > > >> > > > > > >> > > Samza
> > > > > >> > > > > > >> > > > > >> core
> > > > > >> > > > > > >> > > > > >> > so
> > > > > >> > > > > > >> > > > > >> > > it is shared across frameworks and
> the
> > > > > >> framework
> > > > > >> > > > > specific
> > > > > >> > > > > > >> bit
> > > > > >> > > > > > >> > is
> > > > > >> > > > > > >> > > > > just
> > > > > >> > > > > > >> > > > > >> > > whether it is smart enough to try to
> > get
> > > > the
> > > > > >> same
> > > > > >> > > > host
> > > > > >> > > > > > when
> > > > > >> > > > > > >> a
> > > > > >> > > > > > >> > > job
> > > > > >> > > > > > >> > > > is
> > > > > >> > > > > > >> > > > > >> > > restarted.
> > > > > >> > > > > > >> > > > > >> > >
> > > > > >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment,
> > > yeah
> > > > I
> > > > > >> think
> > > > > >> > > the
> > > > > >> > > > > > goal
> > > > > >> > > > > > >> > would
> > > > > >> > > > > > >> > > > be
> > > > > >> > > > > > >> > > > > >> (a)
> > > > > >> > > > > > >> > > > > >> > > actually get better alignment in
> user
> > > > > >> experience,
> > > > > >> > > and
> > > > > >> > > > > (b)
> > > > > >> > > > > > >> > > express
> > > > > >> > > > > > >> > > > > >> this in
> > > > > >> > > > > > >> > > > > >> > > the naming and project branding.
> > > > > Specifically:
> > > > > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice
> for
> > > the
> > > > > >> > > > > > "transformation"
> > > > > >> > > > > > >> api
> > > > > >> > > > > > >> > > to
> > > > > >> > > > > > >> > > > be
> > > > > >> > > > > > >> > > > > >> > > discoverable in the main Kafka
> > > docs--i.e.
> > > > be
> > > > > >> able
> > > > > >> > > to
> > > > > >> > > > > > explain
> > > > > >> > > > > > >> > > when
> > > > > >> > > > > > >> > > > to
> > > > > >> > > > > > >> > > > > >> use
> > > > > >> > > > > > >> > > > > >> > > the consumer and when to use the
> > stream
> > > > > >> > processing
> > > > > >> > > > > > >> > functionality
> > > > > >> > > > > > >> > > > and
> > > > > >> > > > > > >> > > > > >> lead
> > > > > >> > > > > > >> > > > > >> > > people into that experience.
> > > > > >> > > > > > >> > > > > >> > > 2. Align releases so if you get
> Kafkza
> > > > 1.4.2
> > > > > >> (or
> > > > > >> > > > > > whatever)
> > > > > >> > > > > > >> > that
> > > > > >> > > > > > >> > > > has
> > > > > >> > > > > > >> > > > > >> both
> > > > > >> > > > > > >> > > > > >> > > Kafka and the stream processing part
> > and
> > > > > they
> > > > > >> > > > actually
> > > > > >> > > > > > work
> > > > > >> > > > > > >> > > > > together.
> > > > > >> > > > > > >> > > > > >> > > 3. Unify the programming experience
> so
> > > the
> > > > > >> client
> > > > > >> > > and
> > > > > >> > > > > > Samza
> > > > > >> > > > > > >> > api
> > > > > >> > > > > > >> > > > > share
> > > > > >> > > > > > >> > > > > >> > >
> > config/monitoring/naming/packaging/etc.
> > > > > >> > > > > > >> > > > > >> > >
> > > > > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> > > > > committers
> > > > > >> and
> > > > > >> > > can
> > > > > >> > > > > > have a
> > > > > >> > > > > > >> > > > > separate
> > > > > >> > > > > > >> > > > > >> > repo,
> > > > > >> > > > > > >> > > > > >> > > but I'm actually not really sure (I
> > > can't
> > > > > >> find a
> > > > > >> > > > > > definition
> > > > > >> > > > > > >> > of a
> > > > > >> > > > > > >> > > > > >> > subproject
> > > > > >> > > > > > >> > > > > >> > > in Apache).
> > > > > >> > > > > > >> > > > > >> > >
> > > > > >> > > > > > >> > > > > >> > > Basically at a high-level you want
> the
> > > > > >> experience
> > > > > >> > > to
> > > > > >> > > > > > "feel"
> > > > > >> > > > > > >> > > like a
> > > > > >> > > > > > >> > > > > >> single
> > > > > >> > > > > > >> > > > > >> > > system, not to relatively
> independent
> > > > things
> > > > > >> that
> > > > > >> > > are
> > > > > >> > > > > > kind
> > > > > >> > > > > > >> of
> > > > > >> > > > > > >> > > > > >> awkwardly
> > > > > >> > > > > > >> > > > > >> > > glued together.
> > > > > >> > > > > > >> > > > > >> > >
> > > > > >> > > > > > >> > > > > >> > > I think if we did that they having
> > > naming
> > > > or
> > > > > >> > > branding
> > > > > >> > > > > > like
> > > > > >> > > > > > >> > > "kafka
> > > > > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or
> > > something
> > > > > >> like
> > > > > >> > > that
> > > > > >> > > > > > would
> > > > > >> > > > > > >> > > > actually
> > > > > >> > > > > > >> > > > > >> do a
> > > > > >> > > > > > >> > > > > >> > > good job of conveying what it is. I
> do
> > > > that
> > > > > >> this
> > > > > >> > > > would
> > > > > >> > > > > > help
> > > > > >> > > > > > >> > > > adoption
> > > > > >> > > > > > >> > > > > >> > quite
> > > > > >> > > > > > >> > > > > >> > > a lot as it would correctly convey
> > that
> > > > > using
> > > > > >> > Kafka
> > > > > >> > > > > > >> Streaming
> > > > > >> > > > > > >> > > with
> > > > > >> > > > > > >> > > > > >> Kafka
> > > > > >> > > > > > >> > > > > >> > is
> > > > > >> > > > > > >> > > > > >> > > a fairly seamless experience and
> Kafka
> > > is
> > > > > >> pretty
> > > > > >> > > > > heavily
> > > > > >> > > > > > >> > adopted
> > > > > >> > > > > > >> > > > at
> > > > > >> > > > > > >> > > > > >> this
> > > > > >> > > > > > >> > > > > >> > > point.
> > > > > >> > > > > > >> > > > > >> > >
> > > > > >> > > > > > >> > > > > >> > > Fwiw we actually considered this
> model
> > > > > >> originally
> > > > > >> > > > when
> > > > > >> > > > > > open
> > > > > >> > > > > > >> > > > sourcing
> > > > > >> > > > > > >> > > > > >> > Samza,
> > > > > >> > > > > > >> > > > > >> > > however at that time Kafka was
> > > relatively
> > > > > >> unknown
> > > > > >> > > and
> > > > > >> > > > > we
> > > > > >> > > > > > >> > decided
> > > > > >> > > > > > >> > > > not
> > > > > >> > > > > > >> > > > > >> to
> > > > > >> > > > > > >> > > > > >> > do
> > > > > >> > > > > > >> > > > > >> > > it since we felt it would be
> limiting.
> > > > From
> > > > > my
> > > > > >> > > point
> > > > > >> > > > of
> > > > > >> > > > > > view
> > > > > >> > > > > > >> > the
> > > > > >> > > > > > >> > > > > three
> > > > > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now
> > > > really
> > > > > >> > heavily
> > > > > >> > > > > used
> > > > > >> > > > > > for
> > > > > >> > > > > > >> > > > stream
> > > > > >> > > > > > >> > > > > >> > > processing, (2) we learned that
> > > > abstracting
> > > > > >> out
> > > > > >> > the
> > > > > >> > > > > > stream
> > > > > >> > > > > > >> > well
> > > > > >> > > > > > >> > > is
> > > > > >> > > > > > >> > > > > >> > > basically impossible, (3) we learned
> > it
> > > is
> > > > > >> really
> > > > > >> > > > hard
> > > > > >> > > > > to
> > > > > >> > > > > > >> keep
> > > > > >> > > > > > >> > > the
> > > > > >> > > > > > >> > > > > two
> > > > > >> > > > > > >> > > > > >> > > things feeling like a single
> product.
> > > > > >> > > > > > >> > > > > >> > >
> > > > > >> > > > > > >> > > > > >> > > -Jay
> > > > > >> > > > > > >> > > > > >> > >
> > > > > >> > > > > > >> > > > > >> > >
> > > > > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM,
> Martin
> > > > > >> Kleppmann
> > > > > >> > <
> > > > > >> > > > > > >> > > > > >> martin@kleppmann.com>
> > > > > >> > > > > > >> > > > > >> > > wrote:
> > > > > >> > > > > > >> > > > > >> > >
> > > > > >> > > > > > >> > > > > >> > >> Hi all,
> > > > > >> > > > > > >> > > > > >> > >>
> > > > > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > > > > >> > > > > > >> > > > > >> > >>
> > > > > >> > > > > > >> > > > > >> > >> I agree with the general philosophy
> > of
> > > > > tying
> > > > > >> > Samza
> > > > > >> > > > > more
> > > > > >> > > > > > >> > firmly
> > > > > >> > > > > > >> > > to
> > > > > >> > > > > > >> > > > > >> Kafka.
> > > > > >> > > > > > >> > > > > >> > >> After I spent a while looking at
> > > > > integrating
> > > > > >> > other
> > > > > >> > > > > > message
> > > > > >> > > > > > >> > > > brokers
> > > > > >> > > > > > >> > > > > >> (e.g.
> > > > > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I
> came
> > to
> > > > the
> > > > > >> > > > conclusion
> > > > > >> > > > > > that
> > > > > >> > > > > > >> > > > > >> > SystemConsumer
> > > > > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much
> like
> > > > > Kafka's
> > > > > >> > that
> > > > > >> > > > > pretty
> > > > > >> > > > > > >> much
> > > > > >> > > > > > >> > > > > nobody
> > > > > >> > > > > > >> > > > > >> but
> > > > > >> > > > > > >> > > > > >> > >> Kafka actually implements it.
> > (Databus
> > > is
> > > > > >> > perhaps
> > > > > >> > > an
> > > > > >> > > > > > >> > exception,
> > > > > >> > > > > > >> > > > but
> > > > > >> > > > > > >> > > > > >> it
> > > > > >> > > > > > >> > > > > >> > >> isn't widely used outside of
> > LinkedIn.)
> > > > > Thus,
> > > > > >> > > making
> > > > > >> > > > > > Samza
> > > > > >> > > > > > >> > > fully
> > > > > >> > > > > > >> > > > > >> > dependent
> > > > > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > > > > >> > system-independence
> > > > > >> > > > was
> > > > > >> > > > > > >> never
> > > > > >> > > > > > >> > as
> > > > > >> > > > > > >> > > > > real
> > > > > >> > > > > > >> > > > > >> as
> > > > > >> > > > > > >> > > > > >> > we
> > > > > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The
> gains
> > of
> > > > > code
> > > > > >> > reuse
> > > > > >> > > > are
> > > > > >> > > > > > >> real.
> > > > > >> > > > > > >> > > > > >> > >>
> > > > > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from
> > YARN
> > > > has
> > > > > >> also
> > > > > >> > > > always
> > > > > >> > > > > > been
> > > > > >> > > > > > >> > > > > >> appealing to
> > > > > >> > > > > > >> > > > > >> > >> me, for various reasons already
> > > mentioned
> > > > > in
> > > > > >> > this
> > > > > >> > > > > > thread.
> > > > > >> > > > > > >> > > > Although
> > > > > >> > > > > > >> > > > > >> > making
> > > > > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > > > > >> > > > (YARN/Mesos/AWS/etc)
> > > > > >> > > > > > >> seems
> > > > > >> > > > > > >> > > > > >> laudable,
> > > > > >> > > > > > >> > > > > >> > I am
> > > > > >> > > > > > >> > > > > >> > >> a little concerned that it will
> > > restrict
> > > > us
> > > > > >> to a
> > > > > >> > > > > lowest
> > > > > >> > > > > > >> > common
> > > > > >> > > > > > >> > > > > >> > denominator.
> > > > > >> > > > > > >> > > > > >> > >> For example, would host affinity
> > > > > (SAMZA-617)
> > > > > >> > still
> > > > > >> > > > be
> > > > > >> > > > > > >> > possible?
> > > > > >> > > > > > >> > > > For
> > > > > >> > > > > > >> > > > > >> jobs
> > > > > >> > > > > > >> > > > > >> > >> with large amounts of state, I
> think
> > > > > >> SAMZA-617
> > > > > >> > > would
> > > > > >> > > > > be
> > > > > >> > > > > > a
> > > > > >> > > > > > >> big
> > > > > >> > > > > > >> > > > boon,
> > > > > >> > > > > > >> > > > > >> > since
> > > > > >> > > > > > >> > > > > >> > >> restoring state off the changelog
> on
> > > > every
> > > > > >> > single
> > > > > >> > > > > > restart
> > > > > >> > > > > > >> is
> > > > > >> > > > > > >> > > > > painful,
> > > > > >> > > > > > >> > > > > >> > due
> > > > > >> > > > > > >> > > > > >> > >> to long recovery times. It would
> be a
> > > > shame
> > > > > >> if
> > > > > >> > the
> > > > > >> > > > > > >> decoupling
> > > > > >> > > > > > >> > > > from
> > > > > >> > > > > > >> > > > > >> YARN
> > > > > >> > > > > > >> > > > > >> > >> made host affinity impossible.
> > > > > >> > > > > > >> > > > > >> > >>
> > > > > >> > > > > > >> > > > > >> > >> Jay, a question about the proposed
> > API
> > > > for
> > > > > >> > > > > > instantiating a
> > > > > >> > > > > > >> > job
> > > > > >> > > > > > >> > > in
> > > > > >> > > > > > >> > > > > >> code
> > > > > >> > > > > > >> > > > > >> > >> (rather than a properties file):
> when
> > > > > >> > submitting a
> > > > > >> > > > job
> > > > > >> > > > > > to a
> > > > > >> > > > > > >> > > > > cluster,
> > > > > >> > > > > > >> > > > > >> is
> > > > > >> > > > > > >> > > > > >> > the
> > > > > >> > > > > > >> > > > > >> > >> idea that the instantiation code
> runs
> > > on
> > > > a
> > > > > >> > client
> > > > > >> > > > > > >> somewhere,
> > > > > >> > > > > > >> > > > which
> > > > > >> > > > > > >> > > > > >> then
> > > > > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > > > > >> > > YARN/Mesos/AWS/etc?
> > > > > >> > > > > Or
> > > > > >> > > > > > >> does
> > > > > >> > > > > > >> > > that
> > > > > >> > > > > > >> > > > > >> code
> > > > > >> > > > > > >> > > > > >> > run
> > > > > >> > > > > > >> > > > > >> > >> on each container that is part of
> the
> > > job
> > > > > (in
> > > > > >> > > which
> > > > > >> > > > > > case,
> > > > > >> > > > > > >> how
> > > > > >> > > > > > >> > > > does
> > > > > >> > > > > > >> > > > > >> the
> > > > > >> > > > > > >> > > > > >> > job
> > > > > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > > > > >> > > > > > >> > > > > >> > >>
> > > > > >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't
> > feel
> > > > > >> right to
> > > > > >> > > > make
> > > > > >> > > > > a
> > > > > >> > > > > > 1.0
> > > > > >> > > > > > >> > > > release
> > > > > >> > > > > > >> > > > > >> > with a
> > > > > >> > > > > > >> > > > > >> > >> plan for it to be immediately
> > obsolete.
> > > > So
> > > > > if
> > > > > >> > this
> > > > > >> > > > is
> > > > > >> > > > > > going
> > > > > >> > > > > > >> > to
> > > > > >> > > > > > >> > > > > >> happen, I
> > > > > >> > > > > > >> > > > > >> > >> think it would be more honest to
> > stick
> > > > with
> > > > > >> 0.*
> > > > > >> > > > > version
> > > > > >> > > > > > >> > numbers
> > > > > >> > > > > > >> > > > > until
> > > > > >> > > > > > >> > > > > >> > the
> > > > > >> > > > > > >> > > > > >> > >> library-ified Samza has been
> > > implemented,
> > > > > is
> > > > > >> > > stable
> > > > > >> > > > > and
> > > > > >> > > > > > >> > widely
> > > > > >> > > > > > >> > > > > used.
> > > > > >> > > > > > >> > > > > >> > >>
> > > > > >> > > > > > >> > > > > >> > >> Should the new Samza be a
> subproject
> > of
> > > > > >> Kafka?
> > > > > >> > > There
> > > > > >> > > > > is
> > > > > >> > > > > > >> > > precedent
> > > > > >> > > > > > >> > > > > for
> > > > > >> > > > > > >> > > > > >> > >> tight coupling between different
> > Apache
> > > > > >> projects
> > > > > >> > > > (e.g.
> > > > > >> > > > > > >> > Curator
> > > > > >> > > > > > >> > > > and
> > > > > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so
> I
> > > > think
> > > > > >> > > remaining
> > > > > >> > > > > > >> separate
> > > > > >> > > > > > >> > > > would
> > > > > >> > > > > > >> > > > > >> be
> > > > > >> > > > > > >> > > > > >> > ok.
> > > > > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on
> > > > Kafka,
> > > > > >> there
> > > > > >> > > is
> > > > > >> > > > > > enough
> > > > > >> > > > > > >> > > > > substance
> > > > > >> > > > > > >> > > > > >> in
> > > > > >> > > > > > >> > > > > >> > >> Samza that it warrants being a
> > separate
> > > > > >> project.
> > > > > >> > > An
> > > > > >> > > > > > >> argument
> > > > > >> > > > > > >> > in
> > > > > >> > > > > > >> > > > > >> favour
> > > > > >> > > > > > >> > > > > >> > of
> > > > > >> > > > > > >> > > > > >> > >> merging would be if we think Kafka
> > has
> > > a
> > > > > much
> > > > > >> > > > stronger
> > > > > >> > > > > > >> "brand
> > > > > >> > > > > > >> > > > > >> presence"
> > > > > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that
> > one.
> > > > If
> > > > > >> the
> > > > > >> > > Kafka
> > > > > >> > > > > > >> project
> > > > > >> > > > > > >> > is
> > > > > >> > > > > > >> > > > > >> willing
> > > > > >> > > > > > >> > > > > >> > to
> > > > > >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way
> > of
> > > > > doing
> > > > > >> > > > stateful
> > > > > >> > > > > > >> stream
> > > > > >> > > > > > >> > > > > >> > >> transformations, that would
> probably
> > > have
> > > > > >> much
> > > > > >> > the
> > > > > >> > > > > same
> > > > > >> > > > > > >> > effect
> > > > > >> > > > > > >> > > as
> > > > > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> > > > > >> Processors"
> > > > > >> > or
> > > > > >> > > > > > suchlike.
> > > > > >> > > > > > >> > > Close
> > > > > >> > > > > > >> > > > > >> > >> collaboration between the two
> > projects
> > > > will
> > > > > >> be
> > > > > >> > > > needed
> > > > > >> > > > > in
> > > > > >> > > > > > >> any
> > > > > >> > > > > > >> > > > case.
> > > > > >> > > > > > >> > > > > >> > >>
> > > > > >> > > > > > >> > > > > >> > >> From a project management
> > perspective,
> > > I
> > > > > >> guess
> > > > > >> > the
> > > > > >> > > > > "new
> > > > > >> > > > > > >> > Samza"
> > > > > >> > > > > > >> > > > > would
> > > > > >> > > > > > >> > > > > >> > have
> > > > > >> > > > > > >> > > > > >> > >> to be developed on a branch
> alongside
> > > > > ongoing
> > > > > >> > > > > > maintenance
> > > > > >> > > > > > >> of
> > > > > >> > > > > > >> > > the
> > > > > >> > > > > > >> > > > > >> current
> > > > > >> > > > > > >> > > > > >> > >> line of development? I think it
> would
> > > be
> > > > > >> > important
> > > > > >> > > > to
> > > > > >> > > > > > >> > continue
> > > > > >> > > > > > >> > > > > >> > supporting
> > > > > >> > > > > > >> > > > > >> > >> existing users, and provide a
> > graceful
> > > > > >> migration
> > > > > >> > > > path
> > > > > >> > > > > to
> > > > > >> > > > > > >> the
> > > > > >> > > > > > >> > > new
> > > > > >> > > > > > >> > > > > >> > version.
> > > > > >> > > > > > >> > > > > >> > >> Leaving the current versions
> > > unsupported
> > > > > and
> > > > > >> > > forcing
> > > > > >> > > > > > people
> > > > > >> > > > > > >> > to
> > > > > >> > > > > > >> > > > > >> rewrite
> > > > > >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > > > > >> > > > > > >> > > > > >> > >>
> > > > > >> > > > > > >> > > > > >> > >> Best,
> > > > > >> > > > > > >> > > > > >> > >> Martin
> > > > > >> > > > > > >> > > > > >> > >>
> > > > > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps
> <
> > > > > >> > > > jay@confluent.io>
> > > > > >> > > > > > >> wrote:
> > > > > >> > > > > > >> > > > > >> > >>
> > > > > >> > > > > > >> > > > > >> > >>> Hey Garry,
> > > > > >> > > > > > >> > > > > >> > >>>
> > > > > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd
> > be
> > > > > happy
> > > > > >> to
> > > > > >> > > chat
> > > > > >> > > > > > more
> > > > > >> > > > > > >> > about
> > > > > >> > > > > > >> > > > > this
> > > > > >> > > > > > >> > > > > >> if
> > > > > >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris
> > > and I
> > > > > >> > started
> > > > > >> > > > with
> > > > > >> > > > > > the
> > > > > >> > > > > > >> > idea
> > > > > >> > > > > > >> > > > of
> > > > > >> > > > > > >> > > > > >> "what
> > > > > >> > > > > > >> > > > > >> > >>> would it take to make Samza a
> > kick-ass
> > > > > >> > ingestion
> > > > > >> > > > > tool"
> > > > > >> > > > > > but
> > > > > >> > > > > > >> > > > > >> ultimately
> > > > > >> > > > > > >> > > > > >> > we
> > > > > >> > > > > > >> > > > > >> > >>> kind of came around to the idea
> that
> > > > > >> ingestion
> > > > > >> > > and
> > > > > >> > > > > > >> > > > transformation
> > > > > >> > > > > > >> > > > > >> had
> > > > > >> > > > > > >> > > > > >> > >>> pretty different needs and
> coupling
> > > the
> > > > > two
> > > > > >> > made
> > > > > >> > > > > things
> > > > > >> > > > > > >> > hard.
> > > > > >> > > > > > >> > > > > >> > >>>
> > > > > >> > > > > > >> > > > > >> > >>> For what it's worth I think
> copycat
> > > > > (KIP-26)
> > > > > >> > > > actually
> > > > > >> > > > > > will
> > > > > >> > > > > > >> > do
> > > > > >> > > > > > >> > > > what
> > > > > >> > > > > > >> > > > > >> you
> > > > > >> > > > > > >> > > > > >> > >> are
> > > > > >> > > > > > >> > > > > >> > >>> looking for.
> > > > > >> > > > > > >> > > > > >> > >>>
> > > > > >> > > > > > >> > > > > >> > >>> With regard to your point about
> > > slider,
> > > > I
> > > > > >> don't
> > > > > >> > > > > > >> necessarily
> > > > > >> > > > > > >> > > > > >> disagree.
> > > > > >> > > > > > >> > > > > >> > >> But I
> > > > > >> > > > > > >> > > > > >> > >>> think getting good YARN support is
> > > quite
> > > > > >> doable
> > > > > >> > > > and I
> > > > > >> > > > > > >> think
> > > > > >> > > > > > >> > we
> > > > > >> > > > > > >> > > > can
> > > > > >> > > > > > >> > > > > >> make
> > > > > >> > > > > > >> > > > > >> > >>> that work well. I think the issue
> > this
> > > > > >> proposal
> > > > > >> > > > > solves
> > > > > >> > > > > > is
> > > > > >> > > > > > >> > that
> > > > > >> > > > > > >> > > > > >> > >> technically
> > > > > >> > > > > > >> > > > > >> > >>> it is pretty hard to support
> > multiple
> > > > > >> cluster
> > > > > >> > > > > > management
> > > > > >> > > > > > >> > > systems
> > > > > >> > > > > > >> > > > > the
> > > > > >> > > > > > >> > > > > >> > way
> > > > > >> > > > > > >> > > > > >> > >>> things are now, you need to write
> an
> > > > "app
> > > > > >> > master"
> > > > > >> > > > or
> > > > > >> > > > > > >> > > "framework"
> > > > > >> > > > > > >> > > > > for
> > > > > >> > > > > > >> > > > > >> > each
> > > > > >> > > > > > >> > > > > >> > >>> and they are all a little
> different
> > so
> > > > > >> testing
> > > > > >> > is
> > > > > >> > > > > > really
> > > > > >> > > > > > >> > hard.
> > > > > >> > > > > > >> > > > In
> > > > > >> > > > > > >> > > > > >> the
> > > > > >> > > > > > >> > > > > >> > >>> absence of this we have been stuck
> > > with
> > > > > just
> > > > > >> > YARN
> > > > > >> > > > > which
> > > > > >> > > > > > >> has
> > > > > >> > > > > > >> > > > > >> fantastic
> > > > > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of
> > the
> > > > > org,
> > > > > >> but
> > > > > >> > > > zero
> > > > > >> > > > > > >> > > penetration
> > > > > >> > > > > > >> > > > > >> > >> elsewhere.
> > > > > >> > > > > > >> > > > > >> > >>> Given the huge amount of work
> being
> > > put
> > > > in
> > > > > >> to
> > > > > >> > > > slider,
> > > > > >> > > > > > >> > > marathon,
> > > > > >> > > > > > >> > > > > aws
> > > > > >> > > > > > >> > > > > >> > >>> tooling, not to mention the
> umpteen
> > > > > related
> > > > > >> > > > packaging
> > > > > >> > > > > > >> > > > technologies
> > > > > >> > > > > > >> > > > > >> > people
> > > > > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes,
> > > various
> > > > > >> > > > > cloud-specific
> > > > > >> > > > > > >> > deploy
> > > > > >> > > > > > >> > > > > >> tools,
> > > > > >> > > > > > >> > > > > >> > >> etc)
> > > > > >> > > > > > >> > > > > >> > >>> I really think it is important to
> > get
> > > > this
> > > > > >> > right.
> > > > > >> > > > > > >> > > > > >> > >>>
> > > > > >> > > > > > >> > > > > >> > >>> -Jay
> > > > > >> > > > > > >> > > > > >> > >>>
> > > > > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM,
> > Garry
> > > > > >> > Turkington
> > > > > >> > > <
> > > > > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com>
> > > wrote:
> > > > > >> > > > > > >> > > > > >> > >>>
> > > > > >> > > > > > >> > > > > >> > >>>> Hi all,
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>> I think the question below re
> does
> > > > Samza
> > > > > >> > become
> > > > > >> > > a
> > > > > >> > > > > > >> > sub-project
> > > > > >> > > > > > >> > > > of
> > > > > >> > > > > > >> > > > > >> Kafka
> > > > > >> > > > > > >> > > > > >> > >>>> highlights the broader point
> around
> > > > > >> migration.
> > > > > >> > > > Chris
> > > > > >> > > > > > >> > mentions
> > > > > >> > > > > > >> > > > > >> Samza's
> > > > > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1
> > > > release
> > > > > >> but
> > > > > >> > I'm
> > > > > >> > > > not
> > > > > >> > > > > > sure
> > > > > >> > > > > > >> > it
> > > > > >> > > > > > >> > > > > feels
> > > > > >> > > > > > >> > > > > >> > >> right to
> > > > > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan
> > to
> > > > > >> deprecate
> > > > > >> > > > most
> > > > > >> > > > > of
> > > > > >> > > > > > >> it.
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have
> > > some
> > > > > guys
> > > > > >> > who
> > > > > >> > > > have
> > > > > >> > > > > > >> > started
> > > > > >> > > > > > >> > > > > >> working
> > > > > >> > > > > > >> > > > > >> > >> with
> > > > > >> > > > > > >> > > > > >> > >>>> Samza and building some new
> > > > > >> > consumers/producers
> > > > > >> > > > was
> > > > > >> > > > > > next
> > > > > >> > > > > > >> > up.
> > > > > >> > > > > > >> > > > > Sounds
> > > > > >> > > > > > >> > > > > >> > like
> > > > > >> > > > > > >> > > > > >> > >>>> that is absolutely not the
> > direction
> > > to
> > > > > >> go. I
> > > > > >> > > need
> > > > > >> > > > > to
> > > > > >> > > > > > >> look
> > > > > >> > > > > > >> > > into
> > > > > >> > > > > > >> > > > > the
> > > > > >> > > > > > >> > > > > >> > KIP
> > > > > >> > > > > > >> > > > > >> > >> in
> > > > > >> > > > > > >> > > > > >> > >>>> more detail but for me the
> > > > attractiveness
> > > > > >> of
> > > > > >> > > > adding
> > > > > >> > > > > > new
> > > > > >> > > > > > >> > Samza
> > > > > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes
> > all
> > > > > they
> > > > > >> > were
> > > > > >> > > > > doing
> > > > > >> > > > > > was
> > > > > >> > > > > > >> > > > really
> > > > > >> > > > > > >> > > > > >> > getting
> > > > > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --
> was
> > to
> > > > > avoid
> > > > > >> > > > having
> > > > > >> > > > > to
> > > > > >> > > > > > >> > worry
> > > > > >> > > > > > >> > > > > about
> > > > > >> > > > > > >> > > > > >> the
> > > > > >> > > > > > >> > > > > >> > >>>> lifecycle management of external
> > > > clients.
> > > > > >> If
> > > > > >> > > there
> > > > > >> > > > > is
> > > > > >> > > > > > a
> > > > > >> > > > > > >> > > generic
> > > > > >> > > > > > >> > > > > >> Kafka
> > > > > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can
> > plug
> > > a
> > > > > new
> > > > > >> > > > connector
> > > > > >> > > > > > into
> > > > > >> > > > > > >> > and
> > > > > >> > > > > > >> > > > > have
> > > > > >> > > > > > >> > > > > >> a
> > > > > >> > > > > > >> > > > > >> > >> lot of
> > > > > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> > > > > reliability
> > > > > >> > done
> > > > > >> > > > for
> > > > > >> > > > > me
> > > > > >> > > > > > >> then
> > > > > >> > > > > > >> > > it
> > > > > >> > > > > > >> > > > > >> gives
> > > > > >> > > > > > >> > > > > >> > me
> > > > > >> > > > > > >> > > > > >> > >> all
> > > > > >> > > > > > >> > > > > >> > >>>> the pushing new
> consumers/producers
> > > > > would.
> > > > > >> If
> > > > > >> > > not
> > > > > >> > > > > > then it
> > > > > >> > > > > > >> > > > > >> complicates
> > > > > >> > > > > > >> > > > > >> > my
> > > > > >> > > > > > >> > > > > >> > >>>> operational deployments.
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>> Which is similar to my other
> > question
> > > > > with
> > > > > >> the
> > > > > >> > > > > > proposal
> > > > > >> > > > > > >> --
> > > > > >> > > > > > >> > if
> > > > > >> > > > > > >> > > > we
> > > > > >> > > > > > >> > > > > >> > build a
> > > > > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza
> > > plus
> > > > > the
> > > > > >> > > > requisite
> > > > > >> > > > > > >> shims
> > > > > >> > > > > > >> > to
> > > > > >> > > > > > >> > > > > >> > integrate
> > > > > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the
> > former
> > > > may
> > > > > >> be a
> > > > > >> > > lot
> > > > > >> > > > > more
> > > > > >> > > > > > >> work
> > > > > >> > > > > > >> > > > than
> > > > > >> > > > > > >> > > > > we
> > > > > >> > > > > > >> > > > > >> > >> think.
> > > > > >> > > > > > >> > > > > >> > >>>> We may make it much easier for a
> > > > newcomer
> > > > > >> to
> > > > > >> > get
> > > > > >> > > > > > >> something
> > > > > >> > > > > > >> > > > > running
> > > > > >> > > > > > >> > > > > >> but
> > > > > >> > > > > > >> > > > > >> > >>>> having them step up and get a
> > > reliable
> > > > > >> > > production
> > > > > >> > > > > > >> > deployment
> > > > > >> > > > > > >> > > > may
> > > > > >> > > > > > >> > > > > >> still
> > > > > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic,
> if
> > > for
> > > > > >> > different
> > > > > >> > > > > > reasons
> > > > > >> > > > > > >> > than
> > > > > >> > > > > > >> > > > > >> today.
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm
> > comfortable
> > > > > with
> > > > > >> > > making
> > > > > >> > > > > the
> > > > > >> > > > > > >> Samza
> > > > > >> > > > > > >> > > > > >> dependency
> > > > > >> > > > > > >> > > > > >> > >> on
> > > > > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
> > > > absolutely
> > > > > >> see
> > > > > >> > > the
> > > > > >> > > > > > >> benefits
> > > > > >> > > > > > >> > > in
> > > > > >> > > > > > >> > > > > the
> > > > > >> > > > > > >> > > > > >> > >>>> reduction of duplication and
> > clashing
> > > > > >> > > > > > >> > > > terminologies/abstractions
> > > > > >> > > > > > >> > > > > >> that
> > > > > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a
> > > library
> > > > > >> would
> > > > > >> > > > likely
> > > > > >> > > > > > be a
> > > > > >> > > > > > >> > very
> > > > > >> > > > > > >> > > > > nice
> > > > > >> > > > > > >> > > > > >> > tool
> > > > > >> > > > > > >> > > > > >> > >> to
> > > > > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I
> just
> > > have
> > > > > the
> > > > > >> > > > concerns
> > > > > >> > > > > > >> above
> > > > > >> > > > > > >> > re
> > > > > >> > > > > > >> > > > the
> > > > > >> > > > > > >> > > > > >> > >>>> operational side.
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>> Garry
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > > > > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci
> > Morales
> > > > > >> [mailto:
> > > > > >> > > > > > >> > gdfm@apache.org
> > > > > >> > > > > > >> > > ]
> > > > > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > > > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > > > > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and
> > > obesrvations
> > > > on
> > > > > >> > Samza
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > > > > >> > > > > > >> > > > > >> > >>>> From outside, I have always
> > perceived
> > > > > Samza
> > > > > >> > as a
> > > > > >> > > > > > >> computing
> > > > > >> > > > > > >> > > > layer
> > > > > >> > > > > > >> > > > > >> over
> > > > > >> > > > > > >> > > > > >> > >>>> Kafka.
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit
> > > provocative,
> > > > is
> > > > > >> > > "should
> > > > > >> > > > > > Samza
> > > > > >> > > > > > >> be
> > > > > >> > > > > > >> > a
> > > > > >> > > > > > >> > > > > >> > sub-project
> > > > > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > > > > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it
> > as a
> > > > > >> separate
> > > > > >> > > > > project
> > > > > >> > > > > > >> > with a
> > > > > >> > > > > > >> > > > > >> separate
> > > > > >> > > > > > >> > > > > >> > >>>> governance?
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>> Cheers,
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>> --
> > > > > >> > > > > > >> > > > > >> > >>>> Gianmarco
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan
> Fang <
> > > > > >> > > > > > yanfang724@gmail.com>
> > > > > >> > > > > > >> > > > wrote:
> > > > > >> > > > > > >> > > > > >> > >>>>
> > > > > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with
> > > Kafka
> > > > > more
> > > > > >> > > > tightly.
> > > > > >> > > > > > >> > Because
> > > > > >> > > > > > >> > > > > Samza
> > > > > >> > > > > > >> > > > > >> de
> > > > > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it
> > > should
> > > > > >> > leverage
> > > > > >> > > > > what
> > > > > >> > > > > > >> Kafka
> > > > > >> > > > > > >> > > > has.
> > > > > >> > > > > > >> > > > > At
> > > > > >> > > > > > >> > > > > >> > the
> > > > > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need
> to
> > > > > reinvent
> > > > > >> > what
> > > > > >> > > > > Samza
> > > > > >> > > > > > >> > > already
> > > > > >> > > > > > >> > > > > >> has. I
> > > > > >> > > > > > >> > > > > >> > >>>>> also like the idea of separating
> > the
> > > > > >> > ingestion
> > > > > >> > > > and
> > > > > >> > > > > > >> > > > > transformation.
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for
> > me
> > > to
> > > > > >> image
> > > > > >> > > how
> > > > > >> > > > > the
> > > > > >> > > > > > >> Samza
> > > > > >> > > > > > >> > > > will
> > > > > >> > > > > > >> > > > > >> look
> > > > > >> > > > > > >> > > > > >> > >>>> like.
> > > > > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a
> > > little
> > > > > >> > > difference
> > > > > >> > > > > in
> > > > > >> > > > > > >> terms
> > > > > >> > > > > > >> > > of
> > > > > >> > > > > > >> > > > > how
> > > > > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's
> > > code
> > > > > >> shows
> > > > > >> > (A
> > > > > >> > > > > > client of
> > > > > >> > > > > > >> > > > Kakfa)
> > > > > >> > > > > > >> > > > > ?
> > > > > >> > > > > > >> > > > > >> And
> > > > > >> > > > > > >> > > > > >> > >>>>> user's application code calls
> this
> > > > > client?
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library
> > of
> > > > > Kafka
> > > > > >> > (like
> > > > > >> > > > > what
> > > > > >> > > > > > the
> > > > > >> > > > > > >> > > code
> > > > > >> > > > > > >> > > > > >> shows),
> > > > > >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance
> > and
> > > > > >> > > > > fault-tolerance?
> > > > > >> > > > > > >> Are
> > > > > >> > > > > > >> > > they
> > > > > >> > > > > > >> > > > > >> taken
> > > > > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or
> other
> > > > > >> mechanism,
> > > > > >> > > such
> > > > > >> > > > > as
> > > > > >> > > > > > >> > "Samza
> > > > > >> > > > > > >> > > > > >> worker"
> > > > > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>> 2. What about other features,
> such
> > > as
> > > > > >> > > > auto-scaling,
> > > > > >> > > > > > >> shared
> > > > > >> > > > > > >> > > > > state,
> > > > > >> > > > > > >> > > > > >> > >>>>> monitoring?
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone,
> > (is
> > > > > this
> > > > > >> > what
> > > > > >> > > > > Chris
> > > > > >> > > > > > >> > > > suggests?)
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data
> > from
> > > > > Kakfa
> > > > > >> > and
> > > > > >> > > > > > produce
> > > > > >> > > > > > >> to
> > > > > >> > > > > > >> > > it.
> > > > > >> > > > > > >> > > > > >> Then it
> > > > > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza
> > looks
> > > > > like
> > > > > >> > now,
> > > > > >> > > > > > except it
> > > > > >> > > > > > >> > > does
> > > > > >> > > > > > >> > > > > not
> > > > > >> > > > > > >> > > > > >> > rely
> > > > > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can
> it
> > > > > >> leverage
> > > > > >> > > > Kafka's
> > > > > >> > > > > > >> > metrics,
> > > > > >> > > > > > >> > > > > logs,
> > > > > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the
> > > dependency?
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>> Thanks,
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > > > > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM,
> > > > Guozhang
> > > > > >> > Wang <
> > > > > >> > > > > > >> > > > > wangguoz@gmail.com
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>> Read through the code example
> and
> > > it
> > > > > >> looks
> > > > > >> > > good
> > > > > >> > > > to
> > > > > >> > > > > > me.
> > > > > >> > > > > > >> A
> > > > > >> > > > > > >> > > few
> > > > > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as
> executable
> > > > > >> runnable
> > > > > >> > > like:
> > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > > > > >> > > --config-factory=...
> > > > > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> > > > > deploying
> > > > > >> > Samza
> > > > > >> > > > > more
> > > > > >> > > > > > as
> > > > > >> > > > > > >> > > > embedded
> > > > > >> > > > > > >> > > > > >> > >>>>>> libraries in user application
> > code
> > > > > >> (ignoring
> > > > > >> > > the
> > > > > >> > > > > > >> > > terminology
> > > > > >> > > > > > >> > > > > >> since
> > > > > >> > > > > > >> > > > > >> > >>>>>> it is not the
> > > > > >> > > > > > >> > > > > >> > >>>>> same
> > > > > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> > > > > >> MyStreamTask(configs);
> > > > > >> > > > > Thread
> > > > > >> > > > > > >> > thread
> > > > > >> > > > > > >> > > =
> > > > > >> > > > > > >> > > > > new
> > > > > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>> I think both of these
> deployment
> > > > modes
> > > > > >> are
> > > > > >> > > > > important
> > > > > >> > > > > > >> for
> > > > > >> > > > > > >> > > > > >> different
> > > > > >> > > > > > >> > > > > >> > >>>>>> types
> > > > > >> > > > > > >> > > > > >> > >>>>> of
> > > > > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think
> making
> > > > Samza
> > > > > >> > purely
> > > > > >> > > > > > >> standalone
> > > > > >> > > > > > >> > is
> > > > > >> > > > > > >> > > > > still
> > > > > >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable
> or
> > > > > library
> > > > > >> > > modes.
> > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>> Guozhang
> > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33
> PM,
> > > Jay
> > > > > >> Kreps
> > > > > >> > <
> > > > > >> > > > > > >> > > > jay@confluent.io>
> > > > > >> > > > > > >> > > > > >> > wrote:
> > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the
> > code
> > > > > >> example,
> > > > > >> > it
> > > > > >> > > > was
> > > > > >> > > > > > >> > supposed
> > > > > >> > > > > > >> > > > to
> > > > > >> > > > > > >> > > > > >> look
> > > > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > > > >> > > > > > >> > > > > >> > >>>>>>> this:
> > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new
> > > Properties();
> > > > > >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > > > > >> > > > "localhost:4242");
> > > > > >> > > > > > >> > > > > >> StreamingConfig
> > > > > >> > > > > > >> > > > > >> > >>>>>>> config = new
> > > StreamingConfig(props);
> > > > > >> > > > > > >> > > > > >> > >>>>>>>
> config.subscribe("test-topic-1",
> > > > > >> > > > "test-topic-2");
> > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > config.processor(ExampleStreamProcessor.class);
> > > > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > > > >> > StringSerializer(),
> > > > > >> > > > new
> > > > > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
> > > > KafkaStreaming
> > > > > >> > > > container =
> > > > > >> > > > > > new
> > > > > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> > > > > container.run();
> > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>> -Jay
> > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32
> > PM,
> > > > Jay
> > > > > >> > Kreps <
> > > > > >> > > > > > >> > > > jay@confluent.io
> > > > > >> > > > > > >> > > > > >
> > > > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some
> > > conversations
> > > > > >> Chris
> > > > > >> > > and
> > > > > >> > > > I
> > > > > >> > > > > > were
> > > > > >> > > > > > >> > > having
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> around
> > > > > >> > > > > > >> > > > > >> > >>>>>>> whether
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use
> > Samza
> > > > as a
> > > > > >> kind
> > > > > >> > > of
> > > > > >> > > > > data
> > > > > >> > > > > > >> > > > ingestion
> > > > > >> > > > > > >> > > > > >> > >>>>> framework
> > > > > >> > > > > > >> > > > > >> > >>>>>>> for
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead
> to
> > > > > KIP-26
> > > > > >> > > > > "copycat").
> > > > > >> > > > > > >> This
> > > > > >> > > > > > >> > > > kind
> > > > > >> > > > > > >> > > > > of
> > > > > >> > > > > > >> > > > > >> > >>>>>> combined
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config
> > and
> > > > > YARN
> > > > > >> and
> > > > > >> > > the
> > > > > >> > > > > > >> > discussion
> > > > > >> > > > > > >> > > > > >> around
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> how
> > > > > >> > > > > > >> > > > > >> > >>>>> to
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment
> was,
> > > > given
> > > > > >> that
> > > > > >> > > > Samza
> > > > > >> > > > > > was
> > > > > >> > > > > > >> > > > basically
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka
> specific,
> > > > what
> > > > > if
> > > > > >> > you
> > > > > >> > > > just
> > > > > >> > > > > > >> > embraced
> > > > > >> > > > > > >> > > > > that
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > > > > >> > > > > > >> > > > > >> > >>>>>> into
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> something less like a
> > heavyweight
> > > > > >> > framework
> > > > > >> > > > and
> > > > > >> > > > > > more
> > > > > >> > > > > > >> > > like a
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> third
> > > > > >> > > > > > >> > > > > >> > >>>>> Kafka
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> > > > > consumer"
> > > > > >> > with
> > > > > >> > > > > state
> > > > > >> > > > > > >> > > > management
> > > > > >> > > > > > >> > > > > >> > >>>>>> facilities.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead
> > of a
> > > > > >> complex
> > > > > >> > > > stream
> > > > > >> > > > > > >> > > processing
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> framework
> > > > > >> > > > > > >> > > > > >> > >>>>>>> this
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very
> simple
> > > > > thing,
> > > > > >> not
> > > > > >> > > > much
> > > > > >> > > > > > more
> > > > > >> > > > > > >> > > > > >> complicated
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > >> > > > > > >> > > > > >> > >>>>> use
> > > > > >> > > > > > >> > > > > >> > >>>>>>> or
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka
> consumer.
> > As
> > > > > Chris
> > > > > >> > said
> > > > > >> > > > we
> > > > > >> > > > > > >> thought
> > > > > >> > > > > > >> > > > about
> > > > > >> > > > > > >> > > > > >> it
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> a
> > > > > >> > > > > > >> > > > > >> > >>>>> lot
> > > > > >> > > > > > >> > > > > >> > >>>>>> of
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other
> > stream
> > > > > >> > processing
> > > > > >> > > > > > systems
> > > > > >> > > > > > >> > were
> > > > > >> > > > > > >> > > > > doing)
> > > > > >> > > > > > >> > > > > >> > >>>>> seemed
> > > > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from
> > > MapReduce.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to
> > > ingest/output
> > > > > >> data
> > > > > >> > to
> > > > > >> > > > and
> > > > > >> > > > > > from
> > > > > >> > > > > > >> > the
> > > > > >> > > > > > >> > > > > stream
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we
> > actually
> > > > > looked
> > > > > >> > into
> > > > > >> > > > how
> > > > > >> > > > > > that
> > > > > >> > > > > > >> > > would
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> work,
> > > > > >> > > > > > >> > > > > >> > >>>>> Samza
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
> > > > ingestion
> > > > > >> > > framework
> > > > > >> > > > > > for a
> > > > > >> > > > > > >> > > bunch
> > > > > >> > > > > > >> > > > of
> > > > > >> > > > > > >> > > > > >> > >>>>> reasons.
> > > > > >> > > > > > >> > > > > >> > >>>>>> To
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you
> need a
> > > > > pretty
> > > > > >> > > > different
> > > > > >> > > > > > >> > internal
> > > > > >> > > > > > >> > > > > data
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> model
> > > > > >> > > > > > >> > > > > >> > >>>>>> and
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you
> > split
> > > > > them
> > > > > >> and
> > > > > >> > > had
> > > > > >> > > > > an
> > > > > >> > > > > > api
> > > > > >> > > > > > >> > for
> > > > > >> > > > > > >> > > > > Kafka
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA
> > > KIP-26)
> > > > > >> and a
> > > > > >> > > > > separate
> > > > > >> > > > > > >> api
> > > > > >> > > > > > >> > > for
> > > > > >> > > > > > >> > > > > >> Kafka
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> > > > > embracing
> > > > > >> the
> > > > > >> > > > same
> > > > > >> > > > > > >> > > terminology
> > > > > >> > > > > > >> > > > > and
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint
> > about
> > > > the
> > > > > >> > current
> > > > > >> > > > > > state is
> > > > > >> > > > > > >> > > that
> > > > > >> > > > > > >> > > > > the
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> two
> > > > > >> > > > > > >> > > > > >> > >>>>>>> systems
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on.
> > > Terminology
> > > > > >> like
> > > > > >> > > > > "stream"
> > > > > >> > > > > > vs
> > > > > >> > > > > > >> > > > "topic"
> > > > > >> > > > > > >> > > > > >> and
> > > > > >> > > > > > >> > > > > >> > >>>>>>> different
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems
> > > means
> > > > > you
> > > > > >> > kind
> > > > > >> > > > of
> > > > > >> > > > > > have
> > > > > >> > > > > > >> to
> > > > > >> > > > > > >> > > > learn
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > > > > >> > > > > > >> > > > > >> > >>>>>>> way,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
> > > > different
> > > > > >> way,
> > > > > >> > > > then
> > > > > >> > > > > > kind
> > > > > >> > > > > > >> of
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> understand
> > > > > >> > > > > > >> > > > > >> > >>>>> how
> > > > > >> > > > > > >> > > > > >> > >>>>>>> they
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which
> having
> > > > > walked
> > > > > >> a
> > > > > >> > few
> > > > > >> > > > > > people
> > > > > >> > > > > > >> > > through
> > > > > >> > > > > > >> > > > > >> this
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for
> > folks
> > > to
> > > > > >> get.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a
> > lot
> > > of
> > > > > >> time
> > > > > >> > on
> > > > > >> > > > > > >> airplanes I
> > > > > >> > > > > > >> > > > > hacked
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still
> somewhat
> > > > > >> incomplete
> > > > > >> > > > > > prototype
> > > > > >> > > > > > >> of
> > > > > >> > > > > > >> > > > what
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> this would
> > > > > >> > > > > > >> > > > > >> > >>>>> look
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just
> > > unceremoniously
> > > > > >> dumped
> > > > > >> > > into
> > > > > >> > > > > > Kafka
> > > > > >> > > > > > >> as
> > > > > >> > > > > > >> > > it
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> required a
> > > > > >> > > > > > >> > > > > >> > >>>>>> few
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer.
> > Here
> > > > is
> > > > > >> the
> > > > > >> > > code:
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > >
> > > > > >> > > > > > >> >
> > > > > >> > > > > >
> > > > > >> > >
> > > > > >>
> > > >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > > >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the
> > prototype
> > > I
> > > > > just
> > > > > >> > > > > liberally
> > > > > >> > > > > > >> > renamed
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> everything
> > > > > >> > > > > > >> > > > > >> > >>>>> to
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka
> with
> > > no
> > > > > >> regard
> > > > > >> > > for
> > > > > >> > > > > > >> > > > compatibility.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be
> something
> > > like
> > > > > >> this:
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
> > > > Properties();
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> props.put("bootstrap.servers",
> > > > > >> > > > > "localhost:4242");
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > > > > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > > > > >> > > > > > >> > > > > >> > >>>>>>>
> config.subscribe("test-topic-1",
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > > > > >> > > > > > >> > > > > >>
> > > > config.processor(ExampleStreamProcessor.class);
> > > > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > > > > >> > > StringDeserializer());
> > > > > >> > > > > > >> > > > KafkaStreaming
> > > > > >> > > > > > >> > > > > >> > >>>>>> container =
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > > > > >> > container.run();
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically
> the
> > > > > >> > > > SamzaContainer;
> > > > > >> > > > > > >> > > > > StreamProcessor
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all
> the
> > > > class
> > > > > >> names
> > > > > >> > > in
> > > > > >> > > > a
> > > > > >> > > > > > file
> > > > > >> > > > > > >> > and
> > > > > >> > > > > > >> > > > then
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> having
> > > > > >> > > > > > >> > > > > >> > >>>>>> the
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection,
> > you
> > > > just
> > > > > >> > > > > instantiate
> > > > > >> > > > > > the
> > > > > >> > > > > > >> > > > > container
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is
> > > balanced
> > > > > over
> > > > > >> > > > however
> > > > > >> > > > > > many
> > > > > >> > > > > > >> > > > > instances
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > > > >> > > > > > >> > > > > >> > >>>>> this
> > > > > >> > > > > > >> > > > > >> > >>>>>>> are
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> > > > > instance
> > > > > >> > dies,
> > > > > >> > > > new
> > > > > >> > > > > > >> tasks
> > > > > >> > > > > > >> > > are
> > > > > >> > > > > > >> > > > > >> added
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > >> > > > > > >> > > > > >> > >>>>> the
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
> > > > shutting
> > > > > >> them
> > > > > >> > > > down).
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue
> for
> > > > > running
> > > > > >> > this
> > > > > >> > > > > stuff
> > > > > >> > > > > > in
> > > > > >> > > > > > >> > YARN
> > > > > >> > > > > > >> > > > via
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon,
> and
> > > AWS
> > > > > >> using
> > > > > >> > > some
> > > > > >> > > > > of
> > > > > >> > > > > > >> their
> > > > > >> > > > > > >> > > > tools
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> but from the
> > > > > >> > > > > > >> > > > > >> > >>>>>> point
> > > > > >> > > > > > >> > > > > >> > >>>>>>> of
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks
> these
> > > > stream
> > > > > >> > > > processing
> > > > > >> > > > > > jobs
> > > > > >> > > > > > >> > are
> > > > > >> > > > > > >> > > > > just
> > > > > >> > > > > > >> > > > > >> > >>>>>> stateless
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go
> > and
> > > > > >> expand
> > > > > >> > and
> > > > > >> > > > > > contract
> > > > > >> > > > > > >> > at
> > > > > >> > > > > > >> > > > > will.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> There
> > > > > >> > > > > > >> > > > > >> > >>>>> is
> > > > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant
> details:
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of
> > > code,
> > > > > it
> > > > > >> > would
> > > > > >> > > > get
> > > > > >> > > > > > >> larger
> > > > > >> > > > > > >> > > if
> > > > > >> > > > > > >> > > > we
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not
> vastly
> > > > > larger.
> > > > > >> We
> > > > > >> > > > really
> > > > > >> > > > > > do
> > > > > >> > > > > > >> > get a
> > > > > >> > > > > > >> > > > ton
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > > > >> > > > > > >> > > > > >> > >>>>>>> leverage
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is
> > fully
> > > > > >> > delegated
> > > > > >> > > to
> > > > > >> > > > > the
> > > > > >> > > > > > >> new
> > > > > >> > > > > > >> > > > > >> consumer.
> > > > > >> > > > > > >> > > > > >> > >>>>> This
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any
> > partition
> > > > > >> > management
> > > > > >> > > > > > strategy
> > > > > >> > > > > > >> > > > > available
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available
> to
> > > > Samza
> > > > > >> (and
> > > > > >> > > vice
> > > > > >> > > > > > versa)
> > > > > >> > > > > > >> > and
> > > > > >> > > > > > >> > > > > with
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> the
> > > > > >> > > > > > >> > > > > >> > >>>>>>> exact
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well
> > as
> > > > > state
> > > > > >> > reuse
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully
> > it
> > > is
> > > > > >> > thought
> > > > > >> > > > > > >> provoking.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55
> > PM,
> > > > > Chris
> > > > > >> > > > > Riccomini <
> > > > > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions
> > with
> > > > > Samza
> > > > > >> > > > > engineers
> > > > > >> > > > > > at
> > > > > >> > > > > > >> > > > LinkedIn
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > >> > > > > > >> > > > > >> > >>>>>>> Confluent
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> > > > > observations
> > > > > >> > and
> > > > > >> > > > > would
> > > > > >> > > > > > >> like
> > > > > >> > > > > > >> > to
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> propose
> > > > > >> > > > > > >> > > > > >> > >>>>> some
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things
> > that
> > > I
> > > > > >> want to
> > > > > >> > > > call
> > > > > >> > > > > > out
> > > > > >> > > > > > >> > about
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > > > > >> > > > > > >> > > > > >> > >>>>>> design,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> > > > > changes.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a
> > > > dynamic
> > > > > >> > > > deployment
> > > > > >> > > > > > >> system.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> > > > > >> SystemConsumer/SystemProducer
> > > > > >> > and
> > > > > >> > > > > > Kafka's
> > > > > >> > > > > > >> > > > consumer
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > > >> > > > > > >> > > > > >> > >>>>> are
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the
> > > same
> > > > > >> > problems.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues
> are
> > > > > related,
> > > > > >> > but
> > > > > >> > > > I'll
> > > > > >> > > > > > >> > address
> > > > > >> > > > > > >> > > > them
> > > > > >> > > > > > >> > > > > >> in
> > > > > >> > > > > > >> > > > > >> > >>>>> order.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on
> the
> > > use
> > > > > of a
> > > > > >> > > > dynamic
> > > > > >> > > > > > >> > > deployment
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > > > > >> > > > > > >> > > > > >> > >>>>>> such
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> as
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
> > > > initially
> > > > > >> built
> > > > > >> > > > > Samza,
> > > > > >> > > > > > we
> > > > > >> > > > > > >> > bet
> > > > > >> > > > > > >> > > > that
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> there
> > > > > >> > > > > > >> > > > > >> > >>>>>> would
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> be
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this
> > area,
> > > > and
> > > > > >> we
> > > > > >> > > could
> > > > > >> > > > > > >> support
> > > > > >> > > > > > >> > > > them,
> > > > > >> > > > > > >> > > > > >> and
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > >> > > > > > >> > > > > >> > >>>>>> rest
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality,
> > there
> > > > are
> > > > > >> many
> > > > > >> > > > > > >> variations.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > > > > >> > > > > > >> > > > > >> > >>>>>> many
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just
> > > start
> > > > > >> their
> > > > > >> > > > > > processors
> > > > > >> > > > > > >> > like
> > > > > >> > > > > > >> > > > > normal
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
> > > > traditional
> > > > > >> > > > deployment
> > > > > >> > > > > > >> scripts
> > > > > >> > > > > > >> > > > such
> > > > > >> > > > > > >> > > > > as
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > > > > >> > > > > > >> > > > > >> > >>>>>> Chef,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a
> > > deployment
> > > > > >> system
> > > > > >> > > on
> > > > > >> > > > > > users
> > > > > >> > > > > > >> > makes
> > > > > >> > > > > > >> > > > the
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process
> really
> > > > > painful
> > > > > >> for
> > > > > >> > > > first
> > > > > >> > > > > > time
> > > > > >> > > > > > >> > > > users.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
> > > > requirement
> > > > > >> was
> > > > > >> > > also
> > > > > >> > > > a
> > > > > >> > > > > > bit
> > > > > >> > > > > > >> of
> > > > > >> > > > > > >> > a
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > > > > >> > > > > > >> > > > > >> > >>>>>> because
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> of
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental
> misunderstanding
> > > > > between
> > > > > >> > the
> > > > > >> > > > > > nature of
> > > > > >> > > > > > >> > > batch
> > > > > >> > > > > > >> > > > > >> jobs
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > >> > > > > > >> > > > > >> > >>>>>>> stream
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on,
> we
> > > made
> > > > > >> > > conscious
> > > > > >> > > > > > effort
> > > > > >> > > > > > >> to
> > > > > >> > > > > > >> > > > favor
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > >> > > > > > >> > > > > >> > >>>>>> Hadoop
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing
> > > things,
> > > > > >> since
> > > > > >> > it
> > > > > >> > > > > worked
> > > > > >> > > > > > >> and
> > > > > >> > > > > > >> > > was
> > > > > >> > > > > > >> > > > > well
> > > > > >> > > > > > >> > > > > >> > >>>>>>> understood.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was
> > > that
> > > > > >> batch
> > > > > >> > > jobs
> > > > > >> > > > > > have a
> > > > > >> > > > > > >> > > > definite
> > > > > >> > > > > > >> > > > > >> > >>>>>> beginning,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing
> > jobs
> > > > > don't
> > > > > >> > > > > (usually).
> > > > > >> > > > > > >> This
> > > > > >> > > > > > >> > > > leads
> > > > > >> > > > > > >> > > > > to
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> a
> > > > > >> > > > > > >> > > > > >> > >>>>> much
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem
> for
> > > > > stream
> > > > > >> > > > > processors.
> > > > > >> > > > > > >> You
> > > > > >> > > > > > >> > > > > >> basically
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> just
> > > > > >> > > > > > >> > > > > >> > >>>>>>> need
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> > > > > >> processor,
> > > > > >> > and
> > > > > >> > > > > start
> > > > > >> > > > > > >> it.
> > > > > >> > > > > > >> > > The
> > > > > >> > > > > > >> > > > > way
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn,
> > > there's
> > > > > no
> > > > > >> > > concept
> > > > > >> > > > > of
> > > > > >> > > > > > a
> > > > > >> > > > > > >> > > cluster
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > > > > >> > > > > > >> > > > > >> > >>>>>> add
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem
> > with
> > > > > >> coupling
> > > > > >> > > > Samza
> > > > > >> > > > > > with
> > > > > >> > > > > > >> a
> > > > > >> > > > > > >> > > > > >> scheduler
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> is
> > > > > >> > > > > > >> > > > > >> > >>>>>> that
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now
> has
> > > to
> > > > > >> handle
> > > > > >> > > > > > deployment.
> > > > > >> > > > > > >> > > This
> > > > > >> > > > > > >> > > > > >> pulls
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> in a
> > > > > >> > > > > > >> > > > > >> > >>>>>>> bunch
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as
> > configuration
> > > > > >> > > distribution
> > > > > >> > > > > > (config
> > > > > >> > > > > > >> > > > > stream),
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> shell
> > > > > >> > > > > > >> > > > > >> > >>>>>>> scrips
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> > > > > packaging
> > > > > >> > (all
> > > > > >> > > > the
> > > > > >> > > > > > .tgz
> > > > > >> > > > > > >> > > > stuff),
> > > > > >> > > > > > >> > > > > >> etc.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring
> > > > dynamic
> > > > > >> > > > deployment
> > > > > >> > > > > > was
> > > > > >> > > > > > >> to
> > > > > >> > > > > > >> > > > > support
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want
> to
> > > have
> > > > > >> > > locality,
> > > > > >> > > > > you
> > > > > >> > > > > > >> need
> > > > > >> > > > > > >> > to
> > > > > >> > > > > > >> > > > put
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> your
> > > > > >> > > > > > >> > > > > >> > >>>>>> processors
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> > > > > processing.
> > > > > >> > Upon
> > > > > >> > > > > > further
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > > > > >> > > > > > >> > > > > >> > >>>>>>> though,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
> > > > beneficial.
> > > > > >> > There
> > > > > >> > > is
> > > > > >> > > > > > some
> > > > > >> > > > > > >> > good
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > > > > >> > > > > > >> > > > > >> > >>>>>> about
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
> > > > SAMZA-335.
> > > > > >> > Again,
> > > > > >> > > we
> > > > > >> > > > > > took
> > > > > >> > > > > > >> the
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > > > > >> > > > > > >> > > > > >> > >>>>>> path,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> > > > > differences
> > > > > >> > > > between
> > > > > >> > > > > > HDFS
> > > > > >> > > > > > >> > and
> > > > > >> > > > > > >> > > > > Kafka.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > > > > >> > > > > > >> > > > > >> > >>>>>> has
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has
> > > > partitions.
> > > > > >> This
> > > > > >> > > > leads
> > > > > >> > > > > to
> > > > > >> > > > > > >> less
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with
> > > stream
> > > > > >> > > processors
> > > > > >> > > > > on
> > > > > >> > > > > > top
> > > > > >> > > > > > >> > of
> > > > > >> > > > > > >> > > > > Kafka.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used
> as a
> > > > > crutch.
> > > > > >> > > Samza
> > > > > >> > > > > > doesn't
> > > > > >> > > > > > >> > > have
> > > > > >> > > > > > >> > > > > any
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> built
> > > > > >> > > > > > >> > > > > >> > >>>>> in
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic.
> > Instead,
> > > it
> > > > > >> > depends
> > > > > >> > > on
> > > > > >> > > > > the
> > > > > >> > > > > > >> > > dynamic
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system
> > to
> > > > > handle
> > > > > >> > > > restarts
> > > > > >> > > > > > >> when a
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > > > > >> > > > > > >> > > > > >> > >>>>>>> made
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> > > > > >> standalone
> > > > > >> > > Samza
> > > > > >> > > > > > >> > container
> > > > > >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability
> is
> > > > good,
> > > > > >> but I
> > > > > >> > > > think
> > > > > >> > > > > > that
> > > > > >> > > > > > >> > > we've
> > > > > >> > > > > > >> > > > > >> gone
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> too
> > > > > >> > > > > > >> > > > > >> > >>>>>> far
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza
> has:
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment
> > systems.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming
> systems
> > > > > >> > > > (SystemConsumer,
> > > > > >> > > > > > >> > > > > SystemProducer,
> > > > > >> > > > > > >> > > > > >> > >>>> etc).
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for
> > just
> > > > > about
> > > > > >> > every
> > > > > >> > > > > > >> component
> > > > > >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> SystemStreamPartitionGrouper,
> > > > > >> > > ConfigRewriter,
> > > > > >> > > > > > etc).
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that
> > I've
> > > > > >> > forgotten,
> > > > > >> > > as
> > > > > >> > > > > > well.
> > > > > >> > > > > > >> > Some
> > > > > >> > > > > > >> > > > of
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> these
> > > > > >> > > > > > >> > > > > >> > >>>>> are
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven
> > not
> > > > to
> > > > > >> be.
> > > > > >> > > This
> > > > > >> > > > > all
> > > > > >> > > > > > >> comes
> > > > > >> > > > > > >> > > at
> > > > > >> > > > > > >> > > > a
> > > > > >> > > > > > >> > > > > >> cost:
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity
> is
> > > > > making
> > > > > >> it
> > > > > >> > > > harder
> > > > > >> > > > > > for
> > > > > >> > > > > > >> > our
> > > > > >> > > > > > >> > > > > users
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > > > >> > > > > > >> > > > > >> > >>>>> pick
> > > > > >> > > > > > >> > > > > >> > >>>>>> up
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the
> box.
> > It
> > > > > also
> > > > > >> > makes
> > > > > >> > > > it
> > > > > >> > > > > > >> > difficult
> > > > > >> > > > > > >> > > > for
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason
> > about
> > > > > what
> > > > > >> the
> > > > > >> > > > > > >> > > characteristics
> > > > > >> > > > > > >> > > > of
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> > > > > >> characteristics
> > > > > >> > > > change
> > > > > >> > > > > > >> > > depending
> > > > > >> > > > > > >> > > > on
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability
> > are
> > > > > most
> > > > > >> > > visible
> > > > > >> > > > > in
> > > > > >> > > > > > the
> > > > > >> > > > > > >> > > > System
> > > > > >> > > > > > >> > > > > >> APIs.
> > > > > >> > > > > > >> > > > > >> > >>>>> What
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
> > > > > >> functional is
> > > > > >> > > > Kafka
> > > > > >> > > > > > as
> > > > > >> > > > > > >> its
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> transport
> > > > > >> > > > > > >> > > > > >> > >>>>>> layer.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> But
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two
> unrelated
> > > use
> > > > > >> cases
> > > > > >> > > into
> > > > > >> > > > > one
> > > > > >> > > > > > >> API:
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of
> Kafka.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in
> Kafka.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> The current System API
> > supports
> > > > both
> > > > > >> of
> > > > > >> > > these
> > > > > >> > > > > use
> > > > > >> > > > > > >> > cases.
> > > > > >> > > > > > >> > > > The
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> problem
> > > > > >> > > > > > >> > > > > >> > >>>>>> is,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> we
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> actually want different
> > features
> > > > for
> > > > > >> each
> > > > > >> > > use
> > > > > >> > > > > > case.
> > > > > >> > > > > > >> By
> > > > > >> > > > > > >> > > > > >> papering
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> over
> > > > > >> > > > > > >> > > > > >> > >>>>>>> these
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and
> providing a
> > > > > single
> > > > > >> > API,
> > > > > >> > > > > we've
> > > > > >> > > > > > >> > > > introduced
> > > > > >> > > > > > >> > > > > a
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > > >> > > > > > >> > > > > >> > >>>>>>> leaky
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd
> really
> > > like
> > > > > in
> > > > > >> (2)
> > > > > >> > > is
> > > > > >> > > > to
> > > > > >> > > > > > have
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing
> longs
> > > for
> > > > > >> > offsets
> > > > > >> > > > > (like
> > > > > >> > > > > > >> > Kafka).
> > > > > >> > > > > > >> > > > > This
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > > > > >> > > > > > >> > > > > >> > >>>>> with
> > > > > >> > > > > > >> > > > > >> > >>>>>>> (1),
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> though, since different
> > systems
> > > > have
> > > > > >> > > > different
> > > > > >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on
> > the
> > > > > >> mailing
> > > > > >> > > list
> > > > > >> > > > > and
> > > > > >> > > > > > >> the
> > > > > >> > > > > > >> > > SQL
> > > > > >> > > > > > >> > > > > >> JIRAs
> > > > > >> > > > > > >> > > > > >> > >>>>> about
> > > > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true
> for
> > > > > >> > > replayability.
> > > > > >> > > > > > Kafka
> > > > > >> > > > > > >> > > allows
> > > > > >> > > > > > >> > > > us
> > > > > >> > > > > > >> > > > > >> to
> > > > > >> > > > > > >> > > > > >> > >>>>> rewind
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> when
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many
> other
> > > > > systems
> > > > > >> > > don't.
> > > > > >> > > > In
> > > > > >> > > > > > some
> > > > > >> > > > > > >> > > > cases,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > > > >> > > > > > >> > > > > >> > >>>>>>> return
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > > > > >> > > > > > >> WikipediaSystemConsumer)
> > > > > >> > > > > > >> > > > > because
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> they
> > > > > >> > > > > > >> > > > > >> > >>>>>> have
> > > > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another
> > example.
> > > > > Kafka
> > > > > >> > > > supports
> > > > > >> > > > > > >> > > > > partitioning,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > > > >> > > > > > >> > > > > >> > >>>>> many
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this
> > by
> > > > > >> having a
> > > > > >> > > > single
> > > > > >> > > > > > >> > > partition
> > > > > >> > > > > > >> > > > > for
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other
> > > > systems
> > > > > >> model
> > > > > >> > > > > > >> partitioning
> > > > > >> > > > > > >> > > > > >> > >>>> differently (e.g.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is
> > > also
> > > > a
> > > > > >> mess.
> > > > > >> > > > > > Creating
> > > > > >> > > > > > >> > > streams
> > > > > >> > > > > > >> > > > > in
> > > > > >> > > > > > >> > > > > >> a
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is
> almost
> > > > > >> impossible.
> > > > > >> > > As
> > > > > >> > > > is
> > > > > >> > > > > > >> > modeling
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > > > > >> > > > > > >> > > > > >> > >>>>> for
> > > > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> > > > > >> partitions,
> > > > > >> > > > > location,
> > > > > >> > > > > > >> > etc).
> > > > > >> > > > > > >> > > > The
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> list
> > > > > >> > > > > > >> > > > > >> > >>>>> goes
> > > > > >> > > > > > >> > > > > >> > >>>>>>> on.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began
> > > writing
> > > > > >> Samza,
> > > > > >> > > > > Kafka's
> > > > > >> > > > > > >> > > consumer
> > > > > >> > > > > > >> > > > > and
> > > > > >> > > > > > >> > > > > >> > >>>>> producer
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak
> feature
> > > set.
> > > > > On
> > > > > >> the
> > > > > >> > > > > > >> > consumer-side,
> > > > > >> > > > > > >> > > > you
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> had two
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
> > > > > consumer,
> > > > > >> or
> > > > > >> > > the
> > > > > >> > > > > > simple
> > > > > >> > > > > > >> > > > > consumer.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> The
> > > > > >> > > > > > >> > > > > >> > >>>>>>> problem
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer
> > was
> > > > > that
> > > > > >> it
> > > > > >> > > > > > controlled
> > > > > >> > > > > > >> > your
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition
> > assignments,
> > > > and
> > > > > >> the
> > > > > >> > > order
> > > > > >> > > > > in
> > > > > >> > > > > > >> which
> > > > > >> > > > > > >> > > you
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > > > > >> > > > > > >> > > > > >> > >>>>> problem
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> with
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that
> > it's
> > > > not
> > > > > >> > > simple.
> > > > > >> > > > > It's
> > > > > >> > > > > > >> > basic.
> > > > > >> > > > > > >> > > > You
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> end up
> > > > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really
> > > > low-level
> > > > > >> stuff
> > > > > >> > > > that
> > > > > >> > > > > > you
> > > > > >> > > > > > >> > > > > shouldn't.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> We
> > > > > >> > > > > > >> > > > > >> > >>>>>> spent a
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > > > > >> > > > KafkaSystemConsumer
> > > > > >> > > > > > very
> > > > > >> > > > > > >> > > > robust.
> > > > > >> > > > > > >> > > > > >> It
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support
> some
> > > > cool
> > > > > >> > > features:
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message
> > ordering
> > > > and
> > > > > >> > > > > > prioritization.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over
> partition
> > > > > >> assignment
> > > > > >> > > to
> > > > > >> > > > > > support
> > > > > >> > > > > > >> > > > joins,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> global
> > > > > >> > > > > > >> > > > > >> > >>>>>> state
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it
> > :)),
> > > > > etc.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> > > > > >> > checkpointing.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at
> the
> > > time
> > > > > is
> > > > > >> > that
> > > > > >> > > > > these
> > > > > >> > > > > > >> > > features
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > > > >> > > > > > >> > > > > >> > >>>>>>> actually
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
> > > > > consumers
> > > > > >> > (not
> > > > > >> > > > just
> > > > > >> > > > > > >> Samza
> > > > > >> > > > > > >> > > > stream
> > > > > >> > > > > > >> > > > > >> > >>>>>> processors)
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things
> > like
> > > > > joins
> > > > > >> > and
> > > > > >> > > > > > partition
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > > > > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the
> same
> > > > > >> > conclusion.
> > > > > >> > > > > > They're
> > > > > >> > > > > > >> > > adding
> > > > > >> > > > > > >> > > > a
> > > > > >> > > > > > >> > > > > >> ton
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new
> > Kafka
> > > > > >> consumer
> > > > > >> > > > > > >> > > implementation.
> > > > > >> > > > > > >> > > > > To a
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > > > > >> > > > > > >> > > > > >> > >>>>> it's
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've
> > > > already
> > > > > >> done
> > > > > >> > > in
> > > > > >> > > > > > Samza.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended
> up
> > > > > taking
> > > > > >> a
> > > > > >> > > very
> > > > > >> > > > > > similar
> > > > > >> > > > > > >> > > > > approach
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > > > >> > > > > > >> > > > > >> > >>>>>> Samza's
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> > > > > implementation
> > > > > >> for
> > > > > >> > > > > > handling
> > > > > >> > > > > > >> > > offset
> > > > > >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new
> offset
> > > > > >> management
> > > > > >> > > > > feature
> > > > > >> > > > > > >> > stores
> > > > > >> > > > > > >> > > > > >> offset
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and
> > > allows
> > > > > >> you to
> > > > > >> > > > fetch
> > > > > >> > > > > > them
> > > > > >> > > > > > >> > > from
> > > > > >> > > > > > >> > > > > the
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a
> > > waste,
> > > > > >> since
> > > > > >> > we
> > > > > >> > > > > could
> > > > > >> > > > > > >> have
> > > > > >> > > > > > >> > > > shared
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > >> > > > > > >> > > > > >> > >>>>> work
> > > > > >> > > > > > >> > > > > >> > >>>>>> if
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> it
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from
> > the
> > > > > >> get-go.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a
> > rather
> > > > > >> radical
> > > > > >> > > > > > proposal.
> > > > > >> > > > > > >> > Samza
> > > > > >> > > > > > >> > > > is
> > > > > >> > > > > > >> > > > > >> > >>>>> relatively
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd
> > > venture
> > > > to
> > > > > >> say
> > > > > >> > > that
> > > > > >> > > > > > we're
> > > > > >> > > > > > >> > > near a
> > > > > >> > > > > > >> > > > > 1.0
> > > > > >> > > > > > >> > > > > >> > >>>>>> release.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take
> > > what
> > > > > >> we've
> > > > > >> > > > > learned,
> > > > > >> > > > > > and
> > > > > >> > > > > > >> > > begin
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > > > > >> > > > > > >> > > > > >> > >>>>>>> about
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would
> > we
> > > > > >> change if
> > > > > >> > > we
> > > > > >> > > > > were
> > > > > >> > > > > > >> > > starting
> > > > > >> > > > > > >> > > > > >> from
> > > > > >> > > > > > >> > > > > >> > >>>>>> scratch?
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> My
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the
> > > > *only*
> > > > > >> way
> > > > > >> > to
> > > > > >> > > > run
> > > > > >> > > > > > Samza
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate
> all
> > > > direct
> > > > > >> > > > > dependences
> > > > > >> > > > > > on
> > > > > >> > > > > > >> > > YARN,
> > > > > >> > > > > > >> > > > > >> Mesos,
> > > > > >> > > > > > >> > > > > >> > >>>> etc.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to
> > > > support
> > > > > >> only
> > > > > >> > > > Kafka
> > > > > >> > > > > > as
> > > > > >> > > > > > >> the
> > > > > >> > > > > > >> > > > > stream
> > > > > >> > > > > > >> > > > > >> > >>>>>> processing
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's
> metrics,
> > > > > logging,
> > > > > >> > > > > > >> serialization,
> > > > > >> > > > > > >> > > and
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> config
> > > > > >> > > > > > >> > > > > >> > >>>>>>> systems,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's
> > instead.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the
> > issues
> > > > > that
> > > > > >> I
> > > > > >> > > > > outlined
> > > > > >> > > > > > >> > above.
> > > > > >> > > > > > >> > > It
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > > > >> > > > > > >> > > > > >> > >>>>> also
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base
> > > pretty
> > > > > >> > > > dramatically.
> > > > > >> > > > > > >> > > Supporting
> > > > > >> > > > > > >> > > > > >> only
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will
> > > allow
> > > > > >> Samza
> > > > > >> > to
> > > > > >> > > be
> > > > > >> > > > > > >> executed
> > > > > >> > > > > > >> > > on
> > > > > >> > > > > > >> > > > > YARN
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > > > > >> > > > Marathon/Aurora),
> > > > > >> > > > > or
> > > > > >> > > > > > >> most
> > > > > >> > > > > > >> > > > other
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > > > > >> > > > > > >> > > > > >> > >>>>>>> deployment
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make
> > life a
> > > > lot
> > > > > >> > easier
> > > > > >> > > > for
> > > > > >> > > > > > new
> > > > > >> > > > > > >> > > users.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > > > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial
> > without
> > > > > YARN.
> > > > > >> > The
> > > > > >> > > > drop
> > > > > >> > > > > > in
> > > > > >> > > > > > >> > > mailing
> > > > > >> > > > > > >> > > > > >> list
> > > > > >> > > > > > >> > > > > >> > >>>>>> traffic
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems
> long
> > > > > >> overdue to
> > > > > >> > > me.
> > > > > >> > > > > The
> > > > > >> > > > > > >> > > reality
> > > > > >> > > > > > >> > > > > is,
> > > > > >> > > > > > >> > > > > >> > >>>>> everyone
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> that
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza
> > with
> > > > > >> Kafka.
> > > > > >> > We
> > > > > >> > > > > > basically
> > > > > >> > > > > > >> > > > require
> > > > > >> > > > > > >> > > > > >> it
> > > > > >> > > > > > >> > > > > >> > >>>>>> already
> > > > > >> > > > > > >> > > > > >> > >>>>>>> in
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to
> > work.
> > > > > Those
> > > > > >> > that
> > > > > >> > > > are
> > > > > >> > > > > > >> using
> > > > > >> > > > > > >> > > > other
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > > > >> > > > > > >> > > > > >> > >>>>>> are
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for
> ingest
> > > into
> > > > > >> Kafka
> > > > > >> > > (1),
> > > > > >> > > > > and
> > > > > >> > > > > > >> then
> > > > > >> > > > > > >> > > > they
> > > > > >> > > > > > >> > > > > do
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There
> > is
> > > > > >> already
> > > > > >> > > > > > discussion (
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> > > > > > >> > > > >
> > > > > >> > > > > > >> >
> > > > > >> > > > > >
> > > > > >> > >
> > > > > >>
> > > >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > > >> > > > > > >> > > > > >> > >>>>> 767
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> )
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting
> > into
> > > > > Kafka
> > > > > >> > > > extremely
> > > > > >> > > > > > >> easy.
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to
> > couple
> > > > with
> > > > > >> > Kafka,
> > > > > >> > > > we
> > > > > >> > > > > > can
> > > > > >> > > > > > >> > > > leverage
> > > > > >> > > > > > >> > > > > a
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > > >> > > > > > >> > > > > >> > >>>>>>> their
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have
> > to
> > > > > >> maintain
> > > > > >> > > our
> > > > > >> > > > > own
> > > > > >> > > > > > >> > config,
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > > > > >> > > > > > >> > > > > >> > >>>>> etc.
> > > > > >> > > > > > >> > > > > >> > >>>>>>> We
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same
> > > libraries,
> > > > > and
> > > > > >> > make
> > > > > >> > > > them
> > > > > >> > > > > > >> > better.
> > > > > >> > > > > > >> > > > This
> > > > > >> > > > > > >> > > > > >> > >>>>>>>>> will
> > > > > >> > > > > > >> > > > > >> >
> > > > > >> ...
> > > > > >>
> > > > > >> [Message clipped]
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Jordan Shaw
> > > > Full Stack Software Engineer
> > > > PubNub Inc
> > > > 1045 17th St
> > > > San Francisco, CA 94107
> > > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

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

Cool.

The Kafka design process is very open. We usually do a very detailed wiki
along with any code and do a pretty extensive discussion around these.
Details here:
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals

It's wide open and tons of people participate. If we went down this route
the next step would be to discuss there.

For coding it's normal Apache style, JIRAs, RBC, etc.

With respect to state, you definitely need the core support in the API as
that has implications all the way down. I suspect you need at least some
key-value store impl to ship with the client so you have one plugin to test
against and use out of the box. But the whole point of that stuff is it is
pluggable and state is in any case recreated off the Kafka topic, so Samza
can have its own storage engines too. I don't think these need to be
maintained centrally at all.

-Jay



On Mon, Jul 13, 2015 at 3:23 PM, Yi Pan <ni...@gmail.com> wrote:

> Hi, Jay,
>
> Given all the user concerns, the board disagreement on sub-projects, I am
> supporting your 5th option as well. As you said, even the end goal is the
> same, it might help to pave a smooth path forward. One thing I learned over
> the years is that what we planned for may not be the final product, and the
> unexpected product may be even better if we learn and adapt along the way.
> :)
>
> So, since I assume that in option 5, Samza will fully embrace the new Kafka
> Streams API as the core and heavily depends on it, I want to raise up some
> detailed logistic questions:
> 1. How do Samza community contribute to the design and development of the
> new Kafka Streams API? As Kartik mentioned, if there is a model for Samza
> community to contribute to just this part of Kafka code base, it would be a
> huge plus point to the integration.
> 2. What's the scope of the new Kafka Streams API? Is it just focused on
> message consumption, producing, Kafka-based partition distribution, offset
> management, message selection and delivery to StreamProcessor? In other
> words, I have a question regarding to whether we should put samza-kv-store
> in the scope? The reasons that I think that it might be better to stay in
> Samza initially are: a) KV-store libraries does not directly interact w/
> Kafka brokers, it optionally uses Kafka consumers and producers like a
> client program; b) there are a tons of experiments / tune-ups on RocksDB
> that we want to have a faster iteration on this library (e.g. there is an
> experimental time-sequence KV store implementation from LinkedIn we also
> want to experiment on in window operator in SQL). The down-side I can see
> is that w/o this in Kafka Streams API, the as-a-library mode may not get
> the state management support. If we can find a way to make sure that the
> current Samza community can contribute to this library in a faster
> velocity, I can be convinced otherwise as well. What's your opinion on
> this?
>
> Overall, thanks a lot for pushing forward the whole discussion!
>
> -Yi
>
> On Mon, Jul 13, 2015 at 12:56 PM, Garry Turkington <
> g.turkington@improvedigital.com> wrote:
>
> > Hi,
> >
> > I'm also supportive of Jay's option 5. There is a risk the "transformer
> > API" -- I'd have preferred Metamorphosis but it's too hard to type! --
> > takes on a life of its own and we end up with two very different things
> but
> > given how good the Kafka community has been at introducing new producer
> and
> > consumer clients and giving very clear guidance on when they are
> production
> > ready this is a danger I believe can be managed. It'd also be excellent
> to
> > get some working code to beat around the notions of stream processing
> atop
> > a system with transacdtional messages.
> >
> > On the question of whether to keep or deprecate SystemConsumer/Producer I
> > believe we need get a better understanding over the next while of just
> what
> > the Samza community is looking for in such connectivity. For my own use
> > cases I have been looking to add additional implementations primarily to
> > use Samza as the data ingress and egress component around Kafka. Writing
> > external clients that require their own reliability and scalability
> > management gets old real fast and pushing this into a simple Samza job
> that
> > reads from system X and pushes into Kafka (or vice versa) was the obvious
> > choice for me in the current model. For this type of usage though copycat
> > is likely much superior (obviously needs proven) and the question then is
> > if most Samza users look to the system implementations to also act as a
> > front-end into Kafka or if significant usage is indeed intended to have
> the
> > alternative systems as the primary message source. That understanding
> will
> > I think give much clarity in just what value the abstraction overhead of
> > the current model brings.
> >
> > Garry
> >
> > -----Original Message-----
> > From: Yan Fang [mailto:yanfang724@gmail.com]
> > Sent: 13 July 2015 19:58
> > To: dev@samza.apache.org
> > Subject: Re: Thoughts and obesrvations on Samza
> >
> > I am leaning to Jay's fifth approach. It is not radical and gives us some
> > time to see the outcome.
> >
> > In addition, I would suggest:
> >
> > 1) Keep the SystemConsumer/SystemProducer API. Because current
> > SystemConsumer/SystemProducer API satisfies the usage (From Joardan, and
> > even Garry's feedback) and is not so broken that we want to deprecate it.
> > Though there are some issues in implemnting the Kinesis, they are not
> > unfixable. Nothing should prevent Samza, as a stream processing system,
> to
> > support other systems. In addition, there already are some systems
> > exiting besides Kafka: ElasticSearch (committed to the master), HDFS
> > (patch-available), S3( from the mailing list), Kinesis (developing in
> > another repository), ActiveMQ (in two months). We may want to see how
> those
> > go before we "kill" them.
> >
> > 2) Can have some Samza devs involved in Kafka's "transformer" client API.
> > This can not only help the future integration (if any) much easier,
> because
> > they have knowledge about both systems, but also good for Kafka's
> > community, because Samza devs have the streaming process experience that
> > Kafka devs may miss.
> >
> > 3) Samza's partition management system may still support other systems.
> > Though the partition management logic in samza-kafka will be moved to
> > Kafka, its still useful for other systems that do not have the partition
> > management layer.
> >
> > 4) Start sharing the docs/websites and using the same terminology (though
> > do not know how to do this exactly. :). This will reduce the future
> > confusion and does not hurt Samza's independency.
> >
> > In my opinion, Samza, as a standalone project, still can (and already)
> > heavily replying on Kafka, and even more tuned for Kafka-specific usage.
> > Kafka, also can embed Samza in the document, I do not see anything
> prevent
> > doing this.
> >
> > Thanks,
> >
> > Fang, Yan
> > yanfang724@gmail.com
> >
> > On Mon, Jul 13, 2015 at 11:25 AM, Jordan Shaw <jo...@pubnub.com> wrote:
> >
> > > Jay,
> > > I think doing this iteratively in smaller chunks is a better way to go
> as
> > > new issues arise. As Navina said Kafka is a "stream system" and Samza
> is
> > a
> > > "stream processor" and those two ideas should be mutually exclusive.
> > >
> > > -Jordan
> > >
> > > On Mon, Jul 13, 2015 at 10:06 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hmm, thought about this more. Maybe this is just too much too quick.
> > > > Overall I think there is some enthusiasm for the proposal but it's
> not
> > > > really unanimous enough to make any kind of change this big cleanly.
> > The
> > > > board doesn't really like the merging stuff, user's are concerned
> about
> > > > compatibility, I didn't feel there was unanimous agreement on
> dropping
> > > > SystemConsumer, etc. Even if this is the right end state to get to,
> > > > probably trying to push all this through at once isn't the right way
> to
> > > do
> > > > it.
> > > >
> > > > So let me propose a kind of fifth (?) option which I think is less
> > > dramatic
> > > > and let's things happen gradually. I think this is kind of like
> > combining
> > > > the first part of Yi's proposal and Jakob's third option, leaving the
> > > rest
> > > > to be figured out incrementally:
> > > >
> > > > Option 5: We continue the prototype I shared and propose that as a
> kind
> > > of
> > > > "transformer" client API in Kafka. This isn't really a full-fledged
> > > stream
> > > > processing layer, more like a supped up consumer api for munging
> > topics.
> > > > This would let us figure out some of the technical bits, how to do
> this
> > > on
> > > > Kafka's group management features, how to integrate the txn feature
> to
> > do
> > > > the exactly-once stuff in these transformations, and get all this
> stuff
> > > > solid. This api would have valid uses in it's own right, especially
> > when
> > > > your transformation will be embedded inside an existing service or
> > > > application which isn't possible with Samza (or other existing
> systems
> > > that
> > > > I know of).
> > > >
> > > > Independently we can iterate on some of the ideas of the original
> > > proposal
> > > > individually and figure out how (if at all) to make use of this
> > > > functionality. This can be done bit-by-bit:
> > > > - Could be that the existing StreamTask API ends up wrapping this
> > > > - Could end up exposed directly in Samza as Yi proposed
> > > > - Could be that just the lower-level group-management stuff get's
> used,
> > > and
> > > > in this case it could be either just for standalone mode, or always
> > > > - Could be that it stays as-is
> > > >
> > > > The advantage of this is it is lower risk...we basically don't have
> to
> > > make
> > > > 12 major decisions all at once that kind of hinge on what amounts to
> a
> > > > pretty aggressive rewrite. The disadvantage of this is it is a bit
> more
> > > > confusing as all this is getting figured out.
> > > >
> > > > As with some of the other stuff, this would require a further
> > discussion
> > > in
> > > > the Kafka community if people do like this approach.
> > > >
> > > > Thoughts?
> > > >
> > > > -Jay
> > > >
> > > >
> > > >
> > > >
> > > > On Sun, Jul 12, 2015 at 10:52 PM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > Hey Chris,
> > > > >
> > > > > Yeah, I'm obviously in favor of this.
> > > > >
> > > > > The sub-project approach seems the ideal way to take a graceful
> step
> > in
> > > > > this direction, so I will ping the board folks and see why they are
> > > > > discouraged, it would be good to understand that. If we go that
> route
> > > we
> > > > > would need to do a similar discussion in the Kafka list (but makes
> > > sense
> > > > to
> > > > > figure out first if it is what Samza wants).
> > > > >
> > > > > Irrespective of how it's implemented, though, to me the important
> > > things
> > > > > are the following:
> > > > > 1. Unify the website, config, naming, docs, metrics, etc--basically
> > fix
> > > > > the product experience so the "stream" and the "processing" feel
> > like a
> > > > > single user experience and brand. This seems minor but I think is a
> > > > really
> > > > > big deal.
> > > > > 2. Make "standalone" mode a first class citizen and have a real
> > > technical
> > > > > plan to be able to support cluster managers other than YARN.
> > > > > 3. Make the config and out-of-the-box experience more usable
> > > > >
> > > > > I think that prototype gives a practical example of how 1-3 could
> be
> > > done
> > > > > and we should pursue it. This is a pretty radical change, so I
> > wouldn't
> > > > be
> > > > > shocked if people didn't want to take a step like that.
> > > > >
> > > > > Maybe it would make sense to see if people are on board with that
> > > general
> > > > > idea, and then try to get some advice on sub-projects in parallel
> and
> > > > nail
> > > > > down those details?
> > > > >
> > > > > -Jay
> > > > >
> > > > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
> > > criccomini@apache.org>
> > > > > wrote:
> > > > >
> > > > >> Hey all,
> > > > >>
> > > > >> I want to start by saying that I'm absolutely thrilled to be a
> part
> > of
> > > > >> this
> > > > >> community. The amount of level-headed, thoughtful, educated
> > discussion
> > > > >> that's gone on over the past ~10 days is overwhelming. Wonderful.
> > > > >>
> > > > >> It seems like discussion is waning a bit, and we've reached some
> > > > >> conclusions. There are several key emails in this threat, which I
> > want
> > > > to
> > > > >> call out:
> > > > >>
> > > > >> 1. Jakob's summary of the three potential ways forward.
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > > > >> 2. Julian's call out that we should be focusing on community over
> > > code.
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > > > >> 3. Martin's summary about the benefits of merging communities.
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > > > >> 4. Jakob's comments about the distinction between community and
> code
> > > > >> paths.
> > > > >>
> > > > >>
> > > > >>
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> > > > >>
> > > > >> I agree with the comments on all of these emails. I think Martin's
> > > > summary
> > > > >> of his position aligns very closely with my own. To that end, I
> > think
> > > we
> > > > >> should get concrete about what the proposal is, and call a vote on
> > it.
> > > > >> Given that Jay, Martin, and I seem to be aligning fairly closely,
> I
> > > > think
> > > > >> we should start with:
> > > > >>
> > > > >> 1. [community] Make Samza a subproject of Kafka.
> > > > >> 2. [community] Make all Samza PMC/committers committers of the
> > > > subproject.
> > > > >> 3. [community] Migrate Samza's website/documentation into Kafka's.
> > > > >> 4. [code] Have the Samza community and the Kafka community start a
> > > > >> from-scratch reboot together in the new Kafka subproject. We can
> > > > >> borrow/copy &  paste significant chunks of code from Samza's code
> > > base.
> > > > >> 5. [code] The subproject would intentionally eliminate support for
> > > both
> > > > >> other streaming systems and all deployment systems.
> > > > >> 6. [code] Attempt to provide a bridge from our SystemConsumer to
> > > KIP-26
> > > > >> (copy cat)
> > > > >> 7. [code] Attempt to provide a bridge from the new subproject's
> > > > processor
> > > > >> interface to our legacy StreamTask interface.
> > > > >> 8. [code/community] Sunset Samza as a TLP when we have a working
> > Kafka
> > > > >> subproject that has a fault-tolerant container with state
> > management.
> > > > >>
> > > > >> It's likely that (6) and (7) won't be fully drop-in. Still, the
> > closer
> > > > we
> > > > >> can get, the better it's going to be for our existing community.
> > > > >>
> > > > >> One thing that I didn't touch on with (2) is whether any Samza PMC
> > > > members
> > > > >> should be rolled into Kafka PMC membership as well (though, Jay
> and
> > > > Jakob
> > > > >> are already PMC members on both). I think that Samza's community
> > > > deserves
> > > > >> a
> > > > >> voice on the PMC, so I'd propose that we roll at least a few PMC
> > > members
> > > > >> into the Kafka PMC, but I don't have a strong framework for which
> > > people
> > > > >> to
> > > > >> pick.
> > > > >>
> > > > >> Before (8), I think that Samza's TLP can continue to commit bug
> > fixes
> > > > and
> > > > >> patches as it sees fit, provided that we openly communicate that
> we
> > > > won't
> > > > >> necessarily migrate new features to the new subproject, and that
> the
> > > TLP
> > > > >> will be shut down after the migration to the Kafka subproject
> > occurs.
> > > > >>
> > > > >> Jakob, I could use your guidance here about about how to achieve
> > this
> > > > from
> > > > >> an Apache process perspective (sorry).
> > > > >>
> > > > >> * Should I just call a vote on this proposal?
> > > > >> * Should it happen on dev or private?
> > > > >> * Do committers have binding votes, or just PMC?
> > > > >>
> > > > >> Having trouble finding much detail on the Apache wikis. :(
> > > > >>
> > > > >> Cheers,
> > > > >> Chris
> > > > >>
> > > > >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com>
> > > wrote:
> > > > >>
> > > > >> > Thanks, Jay. This argument persuaded me actually. :)
> > > > >> >
> > > > >> > Fang, Yan
> > > > >> > yanfang724@gmail.com
> > > > >> >
> > > > >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io>
> > > wrote:
> > > > >> >
> > > > >> > > Hey Yan,
> > > > >> > >
> > > > >> > > Yeah philosophically I think the argument is that you should
> > > capture
> > > > >> the
> > > > >> > > stream in Kafka independent of the transformation. This is
> > > > obviously a
> > > > >> > > Kafka-centric view point.
> > > > >> > >
> > > > >> > > Advantages of this:
> > > > >> > > - In practice I think this is what e.g. Storm people often end
> > up
> > > > >> doing
> > > > >> > > anyway. You usually need to throttle any access to a live
> > serving
> > > > >> > database.
> > > > >> > > - Can have multiple subscribers and they get the same thing
> > > without
> > > > >> > > additional load on the source system.
> > > > >> > > - Applications can tap into the stream if need be by
> > subscribing.
> > > > >> > > - You can debug your transformation by tailing the Kafka topic
> > > with
> > > > >> the
> > > > >> > > console consumer
> > > > >> > > - Can tee off the same data stream for batch analysis or
> Lambda
> > > arch
> > > > >> > style
> > > > >> > > re-processing
> > > > >> > >
> > > > >> > > The disadvantage is that it will use Kafka resources. But the
> > idea
> > > > is
> > > > >> > > eventually you will have multiple subscribers to any data
> source
> > > (at
> > > > >> > least
> > > > >> > > for monitoring) so you will end up there soon enough anyway.
> > > > >> > >
> > > > >> > > Down the road the technical benefit is that I think it gives
> us
> > a
> > > > good
> > > > >> > path
> > > > >> > > towards end-to-end exactly once semantics from source to
> > > > destination.
> > > > >> > > Basically the connectors need to support idempotence when
> > talking
> > > to
> > > > >> > Kafka
> > > > >> > > and we need the transactional write feature in Kafka to make
> the
> > > > >> > > transformation atomic. This is actually pretty doable if you
> > > > separate
> > > > >> > > connector=>kafka problem from the generic transformations
> which
> > > are
> > > > >> > always
> > > > >> > > kafka=>kafka. However I think it is quite impossible to do in
> a
> > > > >> > all_things
> > > > >> > > => all_things environment. Today you can say "well the
> semantics
> > > of
> > > > >> the
> > > > >> > > Samza APIs depend on the connectors you use" but it is
> actually
> > > > worse
> > > > >> > then
> > > > >> > > that because the semantics actually depend on the pairing of
> > > > >> > connectors--so
> > > > >> > > not only can you probably not get a usable "exactly once"
> > > guarantee
> > > > >> > > end-to-end it can actually be quite hard to reverse engineer
> > what
> > > > >> > property
> > > > >> > > (if any) your end-to-end flow has if you have heterogenous
> > > systems.
> > > > >> > >
> > > > >> > > -Jay
> > > > >> > >
> > > > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <
> yanfang724@gmail.com
> > >
> > > > >> wrote:
> > > > >> > >
> > > > >> > > > {quote}
> > > > >> > > > maintained in a separate repository and retaining the
> existing
> > > > >> > > > committership but sharing as much else as possible (website,
> > > etc)
> > > > >> > > > {quote}
> > > > >> > > >
> > > > >> > > > Overall, I agree on this idea. Now the question is more
> about
> > > "how
> > > > >> to
> > > > >> > do
> > > > >> > > > it".
> > > > >> > > >
> > > > >> > > > On the other hand, one thing I want to point out is that, if
> > we
> > > > >> decide
> > > > >> > to
> > > > >> > > > go this way, how do we want to support
> > > > >> > > > otherSystem-transformation-otherSystem use case?
> > > > >> > > >
> > > > >> > > > Basically, there are four user groups here:
> > > > >> > > >
> > > > >> > > > 1. Kafka-transformation-Kafka
> > > > >> > > > 2. Kafka-transformation-otherSystem
> > > > >> > > > 3. otherSystem-transformation-Kafka
> > > > >> > > > 4. otherSystem-transformation-otherSystem
> > > > >> > > >
> > > > >> > > > For group 1, they can easily use the new Samza library to
> > > achieve.
> > > > >> For
> > > > >> > > > group 2 and 3, they can use copyCat -> transformation ->
> Kafka
> > > or
> > > > >> > Kafka->
> > > > >> > > > transformation -> copyCat.
> > > > >> > > >
> > > > >> > > > The problem is for group 4. Do we want to abandon this or
> > still
> > > > >> support
> > > > >> > > it?
> > > > >> > > > Of course, this use case can be achieved by using copyCat ->
> > > > >> > > transformation
> > > > >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> > > > persuade
> > > > >> > them
> > > > >> > > to
> > > > >> > > > do this long chain. If yes, it will also be a win for Kafka
> > too.
> > > > Or
> > > > >> if
> > > > >> > > > there is no one in this community actually doing this so
> far,
> > > > maybe
> > > > >> ok
> > > > >> > to
> > > > >> > > > not support the group 4 directly.
> > > > >> > > >
> > > > >> > > > Thanks,
> > > > >> > > >
> > > > >> > > > Fang, Yan
> > > > >> > > > yanfang724@gmail.com
> > > > >> > > >
> > > > >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <
> jay@confluent.io
> > >
> > > > >> wrote:
> > > > >> > > >
> > > > >> > > > > Yeah I agree with this summary. I think there are kind of
> > two
> > > > >> > questions
> > > > >> > > > > here:
> > > > >> > > > > 1. Technically does alignment/reliance on Kafka make sense
> > > > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
> > > alignment
> > > > >> with
> > > > >> > > > Kafka
> > > > >> > > > > make sense
> > > > >> > > > >
> > > > >> > > > > Personally I do think both of these things would be really
> > > > >> valuable,
> > > > >> > > and
> > > > >> > > > > would dramatically alter the trajectory of the project.
> > > > >> > > > >
> > > > >> > > > > My preference would be to see if people can mostly agree
> on
> > a
> > > > >> > direction
> > > > >> > > > > rather than splintering things off. From my point of view
> > the
> > > > >> ideal
> > > > >> > > > outcome
> > > > >> > > > > of all the options discussed would be to make Samza a
> > closely
> > > > >> aligned
> > > > >> > > > > subproject, maintained in a separate repository and
> > retaining
> > > > the
> > > > >> > > > existing
> > > > >> > > > > committership but sharing as much else as possible
> (website,
> > > > >> etc). No
> > > > >> > > > idea
> > > > >> > > > > about how these things work, Jacob, you probably know
> more.
> > > > >> > > > >
> > > > >> > > > > No discussion amongst the Kafka folks has happened on
> this,
> > > but
> > > > >> > likely
> > > > >> > > we
> > > > >> > > > > should figure out what the Samza community actually wants
> > > first.
> > > > >> > > > >
> > > > >> > > > > I admit that this is a fairly radical departure from how
> > > things
> > > > >> are.
> > > > >> > > > >
> > > > >> > > > > If that doesn't fly, I think, yeah we could leave Samza as
> > it
> > > is
> > > > >> and
> > > > >> > do
> > > > >> > > > the
> > > > >> > > > > more radical reboot inside Kafka. From my point of view
> that
> > > > does
> > > > >> > leave
> > > > >> > > > > things in a somewhat confusing state since now there are
> two
> > > > >> stream
> > > > >> > > > > processing systems more or less coupled to Kafka in large
> > part
> > > > >> made
> > > > >> > by
> > > > >> > > > the
> > > > >> > > > > same people. But, arguably that might be a cleaner way to
> > make
> > > > the
> > > > >> > > > cut-over
> > > > >> > > > > and perhaps less risky for Samza community since if it
> works
> > > > >> people
> > > > >> > can
> > > > >> > > > > switch and if it doesn't nothing will have changed. Dunno,
> > how
> > > > do
> > > > >> > > people
> > > > >> > > > > feel about this?
> > > > >> > > > >
> > > > >> > > > > -Jay
> > > > >> > > > >
> > > > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> > > > jghoman@gmail.com>
> > > > >> > > wrote:
> > > > >> > > > >
> > > > >> > > > > > >  This leads me to thinking that merging projects and
> > > > >> communities
> > > > >> > > > might
> > > > >> > > > > > be a good idea: with the union of experience from both
> > > > >> communities,
> > > > >> > > we
> > > > >> > > > > will
> > > > >> > > > > > probably build a better system that is better for users.
> > > > >> > > > > > Is this what's being proposed though? Merging the
> projects
> > > > seems
> > > > >> > like
> > > > >> > > > > > a consequence of at most one of the three directions
> under
> > > > >> > > discussion:
> > > > >> > > > > > 1) Samza 2.0: The Samza community relies more heavily on
> > > Kafka
> > > > >> for
> > > > >> > > > > > configuration, etc. (to a greater or lesser extent to be
> > > > >> > determined)
> > > > >> > > > > > but the Samza community would not automatically merge
> > withe
> > > > >> Kafka
> > > > >> > > > > > community (the Phoenix/HBase example is a good one
> here).
> > > > >> > > > > > 2) Samza Reboot: The Samza community continues to exist
> > > with a
> > > > >> > > limited
> > > > >> > > > > > project scope, but similarly would not need to be part
> of
> > > the
> > > > >> Kafka
> > > > >> > > > > > community (ie given committership) to progress.  Here,
> > maybe
> > > > the
> > > > >> > > Samza
> > > > >> > > > > > team would become a subproject of Kafka (the Board
> frowns
> > on
> > > > >> > > > > > subprojects at the moment, so I'm not sure if that's
> even
> > > > >> > feasible),
> > > > >> > > > > > but that would not be required.
> > > > >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this
> > option
> > > > the
> > > > >> > Kafka
> > > > >> > > > > > team builds its own streaming library, possibly off of
> > Jay's
> > > > >> > > > > > prototype, which has not direct lineage to the Samza
> team.
> > > > >> There's
> > > > >> > > no
> > > > >> > > > > > reason for the Kafka team to bring in the Samza team.
> > > > >> > > > > >
> > > > >> > > > > > Is the Kafka community on board with this?
> > > > >> > > > > >
> > > > >> > > > > > To be clear, all three options under discussion are
> > > > interesting,
> > > > >> > > > > > technically valid and likely healthy directions for the
> > > > project.
> > > > >> > > > > > Also, they are not mutually exclusive.  The Samza
> > community
> > > > >> could
> > > > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
> > > community
> > > > >> went
> > > > >> > > > > > forward with 'Hey Samza!'  My points above are directed
> > > > >> entirely at
> > > > >> > > > > > the community aspect of these choices.
> > > > >> > > > > > -Jakob
> > > > >> > > > > >
> > > > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> > > > roger.hoover@gmail.com>
> > > > >> > > wrote:
> > > > >> > > > > > > That's great.  Thanks, Jay.
> > > > >> > > > > > >
> > > > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> > > > jay@confluent.io>
> > > > >> > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > >> Yeah totally agree. I think you have this issue even
> > > today,
> > > > >> > right?
> > > > >> > > > > I.e.
> > > > >> > > > > > if
> > > > >> > > > > > >> you need to make a simple config change and you're
> > > running
> > > > in
> > > > >> > YARN
> > > > >> > > > > today
> > > > >> > > > > > >> you end up bouncing the job which then rebuilds
> state.
> > I
> > > > >> think
> > > > >> > the
> > > > >> > > > fix
> > > > >> > > > > > is
> > > > >> > > > > > >> exactly what you described which is to have a long
> > > timeout
> > > > on
> > > > >> > > > > partition
> > > > >> > > > > > >> movement for stateful jobs so that if a job is just
> > > getting
> > > > >> > > bounced,
> > > > >> > > > > and
> > > > >> > > > > > >> the cluster manager (or admin) is smart enough to
> > restart
> > > > it
> > > > >> on
> > > > >> > > the
> > > > >> > > > > same
> > > > >> > > > > > >> host when possible, it can optimistically reuse any
> > > > existing
> > > > >> > state
> > > > >> > > > it
> > > > >> > > > > > finds
> > > > >> > > > > > >> on disk (if it is valid).
> > > > >> > > > > > >>
> > > > >> > > > > > >> So in this model the charter of the CM is to place
> > > > processes
> > > > >> as
> > > > >> > > > > > stickily as
> > > > >> > > > > > >> possible and to restart or re-place failed processes.
> > The
> > > > >> > charter
> > > > >> > > of
> > > > >> > > > > the
> > > > >> > > > > > >> partition management system is to control the
> > assignment
> > > of
> > > > >> work
> > > > >> > > to
> > > > >> > > > > > these
> > > > >> > > > > > >> processes. The nice thing about this is that the work
> > > > >> > assignment,
> > > > >> > > > > > timeouts,
> > > > >> > > > > > >> behavior, configs, and code will all be the same
> across
> > > all
> > > > >> > > cluster
> > > > >> > > > > > >> managers.
> > > > >> > > > > > >>
> > > > >> > > > > > >> So I think that prototype would actually give you
> > exactly
> > > > >> what
> > > > >> > you
> > > > >> > > > > want
> > > > >> > > > > > >> today for any cluster manager (or manual placement +
> > > > restart
> > > > >> > > script)
> > > > >> > > > > > that
> > > > >> > > > > > >> was sticky in terms of host placement since there is
> > > > already
> > > > >> a
> > > > >> > > > > > configurable
> > > > >> > > > > > >> partition movement timeout and task-by-task state
> reuse
> > > > with
> > > > >> a
> > > > >> > > check
> > > > >> > > > > on
> > > > >> > > > > > >> state validity.
> > > > >> > > > > > >>
> > > > >> > > > > > >> -Jay
> > > > >> > > > > > >>
> > > > >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > > > >> > > > roger.hoover@gmail.com
> > > > >> > > > > >
> > > > >> > > > > > >> wrote:
> > > > >> > > > > > >>
> > > > >> > > > > > >> > That would be great to let Kafka do as much heavy
> > > lifting
> > > > >> as
> > > > >> > > > > possible
> > > > >> > > > > > and
> > > > >> > > > > > >> > make it easier for other languages to implement
> Samza
> > > > apis.
> > > > >> > > > > > >> >
> > > > >> > > > > > >> > One thing to watch out for is the interplay between
> > > > Kafka's
> > > > >> > > group
> > > > >> > > > > > >> > management and the external scheduler/process
> > manager's
> > > > >> fault
> > > > >> > > > > > tolerance.
> > > > >> > > > > > >> > If a container dies, the Kafka group membership
> > > protocol
> > > > >> will
> > > > >> > > try
> > > > >> > > > to
> > > > >> > > > > > >> assign
> > > > >> > > > > > >> > it's tasks to other containers while at the same
> time
> > > the
> > > > >> > > process
> > > > >> > > > > > manager
> > > > >> > > > > > >> > is trying to relaunch the container.  Without some
> > > > >> > consideration
> > > > >> > > > for
> > > > >> > > > > > this
> > > > >> > > > > > >> > (like a configurable amount of time to wait before
> > > Kafka
> > > > >> > alters
> > > > >> > > > the
> > > > >> > > > > > group
> > > > >> > > > > > >> > membership), there may be thrashing going on which
> is
> > > > >> > especially
> > > > >> > > > bad
> > > > >> > > > > > for
> > > > >> > > > > > >> > containers with large amounts of local state.
> > > > >> > > > > > >> >
> > > > >> > > > > > >> > Someone else pointed this out already but I thought
> > it
> > > > >> might
> > > > >> > be
> > > > >> > > > > worth
> > > > >> > > > > > >> > calling out again.
> > > > >> > > > > > >> >
> > > > >> > > > > > >> > Cheers,
> > > > >> > > > > > >> >
> > > > >> > > > > > >> > Roger
> > > > >> > > > > > >> >
> > > > >> > > > > > >> >
> > > > >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> > > > >> jay@confluent.io>
> > > > >> > > > > wrote:
> > > > >> > > > > > >> >
> > > > >> > > > > > >> > > Hey Roger,
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > I couldn't agree more. We spent a bunch of time
> > > talking
> > > > >> to
> > > > >> > > > people
> > > > >> > > > > > and
> > > > >> > > > > > >> > that
> > > > >> > > > > > >> > > is exactly the stuff we heard time and again.
> What
> > > > makes
> > > > >> it
> > > > >> > > > hard,
> > > > >> > > > > of
> > > > >> > > > > > >> > > course, is that there is some tension between
> > > > >> compatibility
> > > > >> > > with
> > > > >> > > > > > what's
> > > > >> > > > > > >> > > there now and making things better for new users.
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > I also strongly agree with the importance of
> > > > >> multi-language
> > > > >> > > > > > support. We
> > > > >> > > > > > >> > are
> > > > >> > > > > > >> > > talking now about Java, but for application
> > > development
> > > > >> use
> > > > >> > > > cases
> > > > >> > > > > > >> people
> > > > >> > > > > > >> > > want to work in whatever language they are using
> > > > >> elsewhere.
> > > > >> > I
> > > > >> > > > > think
> > > > >> > > > > > >> > moving
> > > > >> > > > > > >> > > to a model where Kafka itself does the group
> > > > membership,
> > > > >> > > > lifecycle
> > > > >> > > > > > >> > control,
> > > > >> > > > > > >> > > and partition assignment has the advantage of
> > putting
> > > > all
> > > > >> > that
> > > > >> > > > > > complex
> > > > >> > > > > > >> > > stuff behind a clean api that the clients are
> > already
> > > > >> going
> > > > >> > to
> > > > >> > > > be
> > > > >> > > > > > >> > > implementing for their consumer, so the added
> > > > >> functionality
> > > > >> > > for
> > > > >> > > > > > stream
> > > > >> > > > > > >> > > processing beyond a consumer becomes very minor.
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > -Jay
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > > > >> > > > > > roger.hoover@gmail.com>
> > > > >> > > > > > >> > > wrote:
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > > Metamorphosis...nice. :)
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > > This has been a great discussion.  As a user of
> > > Samza
> > > > >> > who's
> > > > >> > > > > > recently
> > > > >> > > > > > >> > > > integrated it into a relatively large
> > > organization, I
> > > > >> just
> > > > >> > > > want
> > > > >> > > > > to
> > > > >> > > > > > >> add
> > > > >> > > > > > >> > > > support to a few points already made.
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
> > > > >> currently
> > > > >> > > > exists
> > > > >> > > > > > that
> > > > >> > > > > > >> > I've
> > > > >> > > > > > >> > > > experienced are:
> > > > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
> > > environments
> > > > >> > where
> > > > >> > > > > Puppet
> > > > >> > > > > > >> > would
> > > > >> > > > > > >> > > do
> > > > >> > > > > > >> > > > just fine but it was the only mechanism to get
> > > fault
> > > > >> > > > tolerance.
> > > > >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> > > > >> configuring
> > > > >> > > most
> > > > >> > > > > of
> > > > >> > > > > > the
> > > > >> > > > > > >> > job
> > > > >> > > > > > >> > > > in code rather than config files.  In general,
> I
> > > > think
> > > > >> the
> > > > >> > > > goal
> > > > >> > > > > > >> should
> > > > >> > > > > > >> > be
> > > > >> > > > > > >> > > > to make it harder to make mistakes, especially
> of
> > > the
> > > > >> kind
> > > > >> > > > where
> > > > >> > > > > > the
> > > > >> > > > > > >> > code
> > > > >> > > > > > >> > > > expects something and the config doesn't match.
> > > The
> > > > >> > current
> > > > >> > > > > > config
> > > > >> > > > > > >> is
> > > > >> > > > > > >> > > > quite intricate and error-prone.  For example,
> > the
> > > > >> > > application
> > > > >> > > > > > logic
> > > > >> > > > > > >> > may
> > > > >> > > > > > >> > > > depend on bootstrapping a topic but rather than
> > > > >> asserting
> > > > >> > > that
> > > > >> > > > > in
> > > > >> > > > > > the
> > > > >> > > > > > >> > > code,
> > > > >> > > > > > >> > > > you have to rely on getting the config right.
> > > > Likewise
> > > > >> > with
> > > > >> > > > > > serdes,
> > > > >> > > > > > >> > the
> > > > >> > > > > > >> > > > Java representations produced by various serdes
> > > > (JSON,
> > > > >> > Avro,
> > > > >> > > > > etc.)
> > > > >> > > > > > >> are
> > > > >> > > > > > >> > > not
> > > > >> > > > > > >> > > > equivalent so you cannot just reconfigure a
> serde
> > > > >> without
> > > > >> > > > > changing
> > > > >> > > > > > >> the
> > > > >> > > > > > >> > > > code.   It would be nice for jobs to be able to
> > > > assert
> > > > >> > what
> > > > >> > > > they
> > > > >> > > > > > >> expect
> > > > >> > > > > > >> > > > from their input topics in terms of
> partitioning.
> > > > >> This is
> > > > >> > > > > > getting a
> > > > >> > > > > > >> > > little
> > > > >> > > > > > >> > > > off topic but I was even thinking about
> creating
> > a
> > > > >> "Samza
> > > > >> > > > config
> > > > >> > > > > > >> > linter"
> > > > >> > > > > > >> > > > that would sanity check a set of configs.
> > > Especially
> > > > >> in
> > > > >> > > > > > >> organizations
> > > > >> > > > > > >> > > > where config is managed by a different team
> than
> > > the
> > > > >> > > > application
> > > > >> > > > > > >> > > developer,
> > > > >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > > > >> > > > > > >> > > > 3) Java/Scala centric - for many teams
> > (especially
> > > > >> > > DevOps-type
> > > > >> > > > > > >> folks),
> > > > >> > > > > > >> > > the
> > > > >> > > > > > >> > > > pain of the Java toolchain (maven, slow builds,
> > > weak
> > > > >> > command
> > > > >> > > > > line
> > > > >> > > > > > >> > > support,
> > > > >> > > > > > >> > > > configuration over convention) really inhibits
> > > > >> > productivity.
> > > > >> > > > As
> > > > >> > > > > > more
> > > > >> > > > > > >> > and
> > > > >> > > > > > >> > > > more high-quality clients become available for
> > > > Kafka, I
> > > > >> > hope
> > > > >> > > > > > they'll
> > > > >> > > > > > >> > > follow
> > > > >> > > > > > >> > > > Samza's model.  Not sure how much it affects
> the
> > > > >> proposals
> > > > >> > > in
> > > > >> > > > > this
> > > > >> > > > > > >> > thread
> > > > >> > > > > > >> > > > but please consider other languages in the
> > > ecosystem
> > > > as
> > > > >> > > well.
> > > > >> > > > > > From
> > > > >> > > > > > >> > what
> > > > >> > > > > > >> > > > I've heard, Spark has more Python users than
> > > > >> Java/Scala.
> > > > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza
> API
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> >
> > > > >> > > > > > >>
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > > >> > > > > > >> > > > and are working on a Yeoman generator
> > > > >> > > > > > >> > > > https://github.com/Quantiply/generator-rico
> for
> > > > >> > > Jython/Samza
> > > > >> > > > > > >> projects
> > > > >> > > > > > >> > to
> > > > >> > > > > > >> > > > alleviate some of the pain)
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > > I also want to underscore Jay's point about
> > > improving
> > > > >> the
> > > > >> > > user
> > > > >> > > > > > >> > > experience.
> > > > >> > > > > > >> > > > That's a very important factor for adoption.  I
> > > think
> > > > >> the
> > > > >> > > goal
> > > > >> > > > > > should
> > > > >> > > > > > >> > be
> > > > >> > > > > > >> > > to
> > > > >> > > > > > >> > > > make Samza as easy to get started with as
> > something
> > > > >> like
> > > > >> > > > > Logstash.
> > > > >> > > > > > >> > > > Logstash is vastly inferior in terms of
> > > capabilities
> > > > to
> > > > >> > > Samza
> > > > >> > > > > but
> > > > >> > > > > > >> it's
> > > > >> > > > > > >> > > easy
> > > > >> > > > > > >> > > > to get started and that makes a big difference.
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > > Cheers,
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > > Roger
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> > > > Francisci
> > > > >> > > > Morales <
> > > > >> > > > > > >> > > > gdfm@apache.org> wrote:
> > > > >> > > > > > >> > > >
> > > > >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> > > > >> Metamorphosis
> > > > >> > > is
> > > > >> > > > a
> > > > >> > > > > > clear
> > > > >> > > > > > >> > > > winner
> > > > >> > > > > > >> > > > > :)
> > > > >> > > > > > >> > > > >
> > > > >> > > > > > >> > > > > --
> > > > >> > > > > > >> > > > > Gianmarco
> > > > >> > > > > > >> > > > >
> > > > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De
> Francisci
> > > > >> Morales
> > > > >> > <
> > > > >> > > > > > >> > > gdfm@apache.org
> > > > >> > > > > > >> > > > >
> > > > >> > > > > > >> > > > > wrote:
> > > > >> > > > > > >> > > > >
> > > > >> > > > > > >> > > > > > Hi,
> > > > >> > > > > > >> > > > > >
> > > > >> > > > > > >> > > > > > @Martin, thanks for you comments.
> > > > >> > > > > > >> > > > > > Maybe I'm missing some important point,
> but I
> > > > think
> > > > >> > > > coupling
> > > > >> > > > > > the
> > > > >> > > > > > >> > > > releases
> > > > >> > > > > > >> > > > > > is actually a *good* thing.
> > > > >> > > > > > >> > > > > > To make an example, would it be better if
> the
> > > MR
> > > > >> and
> > > > >> > > HDFS
> > > > >> > > > > > >> > components
> > > > >> > > > > > >> > > of
> > > > >> > > > > > >> > > > > > Hadoop had different release schedules?
> > > > >> > > > > > >> > > > > >
> > > > >> > > > > > >> > > > > > Actually, keeping the discussion in a
> single
> > > > place
> > > > >> > would
> > > > >> > > > > make
> > > > >> > > > > > >> > > agreeing
> > > > >> > > > > > >> > > > on
> > > > >> > > > > > >> > > > > > releases (and backwards compatibility) much
> > > > >> easier, as
> > > > >> > > > > > everybody
> > > > >> > > > > > >> > > would
> > > > >> > > > > > >> > > > be
> > > > >> > > > > > >> > > > > > responsible for the whole codebase.
> > > > >> > > > > > >> > > > > >
> > > > >> > > > > > >> > > > > > That said, I like the idea of absorbing
> > > > samza-core
> > > > >> as
> > > > >> > a
> > > > >> > > > > > >> > sub-project,
> > > > >> > > > > > >> > > > and
> > > > >> > > > > > >> > > > > > leave the fancy stuff separate.
> > > > >> > > > > > >> > > > > > It probably gives 90% of the benefits we
> have
> > > > been
> > > > >> > > > > discussing
> > > > >> > > > > > >> here.
> > > > >> > > > > > >> > > > > >
> > > > >> > > > > > >> > > > > > Cheers,
> > > > >> > > > > > >> > > > > >
> > > > >> > > > > > >> > > > > > --
> > > > >> > > > > > >> > > > > > Gianmarco
> > > > >> > > > > > >> > > > > >
> > > > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > > > >> > jay.kreps@gmail.com
> > > > >> > > >
> > > > >> > > > > > wrote:
> > > > >> > > > > > >> > > > > >
> > > > >> > > > > > >> > > > > >> Hey Martin,
> > > > >> > > > > > >> > > > > >>
> > > > >> > > > > > >> > > > > >> I agree coupling release schedules is a
> > > > downside.
> > > > >> > > > > > >> > > > > >>
> > > > >> > > > > > >> > > > > >> Definitely we can try to solve some of the
> > > > >> > integration
> > > > >> > > > > > problems
> > > > >> > > > > > >> in
> > > > >> > > > > > >> > > > > >> Confluent Platform or in other
> > distributions.
> > > > But
> > > > >> I
> > > > >> > > think
> > > > >> > > > > > this
> > > > >> > > > > > >> > ends
> > > > >> > > > > > >> > > up
> > > > >> > > > > > >> > > > > >> being really shallow. I guess I feel to
> > really
> > > > >> get a
> > > > >> > > good
> > > > >> > > > > > user
> > > > >> > > > > > >> > > > > experience
> > > > >> > > > > > >> > > > > >> the two systems have to kind of feel like
> > part
> > > > of
> > > > >> the
> > > > >> > > > same
> > > > >> > > > > > thing
> > > > >> > > > > > >> > and
> > > > >> > > > > > >> > > > you
> > > > >> > > > > > >> > > > > >> can't really add that in later--you can
> put
> > > both
> > > > >> in
> > > > >> > the
> > > > >> > > > > same
> > > > >> > > > > > >> > > > > downloadable
> > > > >> > > > > > >> > > > > >> tar file but it doesn't really give a very
> > > > >> cohesive
> > > > >> > > > > feeling.
> > > > >> > > > > > I
> > > > >> > > > > > >> > agree
> > > > >> > > > > > >> > > > > that
> > > > >> > > > > > >> > > > > >> ultimately any of the project stuff is as
> > much
> > > > >> social
> > > > >> > > and
> > > > >> > > > > > naming
> > > > >> > > > > > >> > as
> > > > >> > > > > > >> > > > > >> anything else--theoretically two totally
> > > > >> independent
> > > > >> > > > > projects
> > > > >> > > > > > >> > could
> > > > >> > > > > > >> > > > work
> > > > >> > > > > > >> > > > > >> to
> > > > >> > > > > > >> > > > > >> tightly align. In practice this seems to
> be
> > > > quite
> > > > >> > > > difficult
> > > > >> > > > > > >> > though.
> > > > >> > > > > > >> > > > > >>
> > > > >> > > > > > >> > > > > >> For the frameworks--totally agree it would
> > be
> > > > >> good to
> > > > >> > > > > > maintain
> > > > >> > > > > > >> the
> > > > >> > > > > > >> > > > > >> framework support with the project. In
> some
> > > > cases
> > > > >> > there
> > > > >> > > > may
> > > > >> > > > > > not
> > > > >> > > > > > >> be
> > > > >> > > > > > >> > > too
> > > > >> > > > > > >> > > > > >> much
> > > > >> > > > > > >> > > > > >> there since the integration gets lighter
> > but I
> > > > >> think
> > > > >> > > > > whatever
> > > > >> > > > > > >> > stubs
> > > > >> > > > > > >> > > > you
> > > > >> > > > > > >> > > > > >> need should be included. So no I
> definitely
> > > > wasn't
> > > > >> > > trying
> > > > >> > > > > to
> > > > >> > > > > > >> imply
> > > > >> > > > > > >> > > > > >> dropping
> > > > >> > > > > > >> > > > > >> support for these frameworks, just making
> > the
> > > > >> > > integration
> > > > >> > > > > > >> lighter
> > > > >> > > > > > >> > by
> > > > >> > > > > > >> > > > > >> separating process management from
> partition
> > > > >> > > management.
> > > > >> > > > > > >> > > > > >>
> > > > >> > > > > > >> > > > > >> You raise two good points we would have to
> > > > figure
> > > > >> out
> > > > >> > > if
> > > > >> > > > we
> > > > >> > > > > > went
> > > > >> > > > > > >> > > down
> > > > >> > > > > > >> > > > > the
> > > > >> > > > > > >> > > > > >> alignment path:
> > > > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think
> > the
> > > > >> first
> > > > >> > > > > question
> > > > >> > > > > > is
> > > > >> > > > > > >> > > > whether
> > > > >> > > > > > >> > > > > >> some "re-branding" would be worth it. If
> so
> > > > then I
> > > > >> > > think
> > > > >> > > > we
> > > > >> > > > > > can
> > > > >> > > > > > >> > > have a
> > > > >> > > > > > >> > > > > big
> > > > >> > > > > > >> > > > > >> thread on the name. I'm definitely not set
> > on
> > > > >> Kafka
> > > > >> > > > > > Streaming or
> > > > >> > > > > > >> > > Kafka
> > > > >> > > > > > >> > > > > >> Streams I was just using them to be kind
> of
> > > > >> > > > illustrative. I
> > > > >> > > > > > >> agree
> > > > >> > > > > > >> > > with
> > > > >> > > > > > >> > > > > >> your
> > > > >> > > > > > >> > > > > >> critique of these names, though I think
> > people
> > > > >> would
> > > > >> > > get
> > > > >> > > > > the
> > > > >> > > > > > >> idea.
> > > > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about
> > how
> > > to
> > > > >> > > "factor"
> > > > >> > > > > it.
> > > > >> > > > > > >> Here
> > > > >> > > > > > >> > > are
> > > > >> > > > > > >> > > > > the
> > > > >> > > > > > >> > > > > >> options I see (I could get enthusiastic
> > about
> > > > any
> > > > >> of
> > > > >> > > > them):
> > > > >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > > > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> > > > seperation
> > > > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of
> samza-api
> > > and
> > > > >> > > > samza-core
> > > > >> > > > > > is
> > > > >> > > > > > >> > > > absorbed
> > > > >> > > > > > >> > > > > >> almost like a third client
> > > > >> > > > > > >> > > > > >>
> > > > >> > > > > > >> > > > > >> Cheers,
> > > > >> > > > > > >> > > > > >>
> > > > >> > > > > > >> > > > > >> -Jay
> > > > >> > > > > > >> > > > > >>
> > > > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> > > > Kleppmann <
> > > > >> > > > > > >> > > > martin@kleppmann.com>
> > > > >> > > > > > >> > > > > >> wrote:
> > > > >> > > > > > >> > > > > >>
> > > > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just
> a
> > > few
> > > > >> > > follow-up
> > > > >> > > > > > >> > comments.
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka
> > or
> > > > >> > becoming
> > > > >> > > a
> > > > >> > > > > > >> > subproject:
> > > > >> > > > > > >> > > > the
> > > > >> > > > > > >> > > > > >> > reasons you mention are good. The risk I
> > see
> > > > is
> > > > >> > that
> > > > >> > > > > > release
> > > > >> > > > > > >> > > > schedules
> > > > >> > > > > > >> > > > > >> > become coupled to each other, which can
> > slow
> > > > >> > everyone
> > > > >> > > > > down,
> > > > >> > > > > > >> and
> > > > >> > > > > > >> > > > large
> > > > >> > > > > > >> > > > > >> > projects with many contributors are
> harder
> > > to
> > > > >> > manage.
> > > > >> > > > > > (Jakob,
> > > > >> > > > > > >> > can
> > > > >> > > > > > >> > > > you
> > > > >> > > > > > >> > > > > >> speak
> > > > >> > > > > > >> > > > > >> > from experience, having seen a wider
> range
> > > of
> > > > >> > Hadoop
> > > > >> > > > > > ecosystem
> > > > >> > > > > > >> > > > > >> projects?)
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > > >> > Some of the goals of a better unified
> > > > developer
> > > > >> > > > > experience
> > > > >> > > > > > >> could
> > > > >> > > > > > >> > > > also
> > > > >> > > > > > >> > > > > be
> > > > >> > > > > > >> > > > > >> > solved by integrating Samza nicely into
> a
> > > > Kafka
> > > > >> > > > > > distribution
> > > > >> > > > > > >> > (such
> > > > >> > > > > > >> > > > as
> > > > >> > > > > > >> > > > > >> > Confluent's). I'm not against merging
> > > projects
> > > > >> if
> > > > >> > we
> > > > >> > > > > decide
> > > > >> > > > > > >> > that's
> > > > >> > > > > > >> > > > the
> > > > >> > > > > > >> > > > > >> way
> > > > >> > > > > > >> > > > > >> > to go, just pointing out the same goals
> > can
> > > > >> perhaps
> > > > >> > > > also
> > > > >> > > > > be
> > > > >> > > > > > >> > > achieved
> > > > >> > > > > > >> > > > > in
> > > > >> > > > > > >> > > > > >> > other ways.
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
> > > dependency:
> > > > >> are
> > > > >> > > you
> > > > >> > > > > > >> proposing
> > > > >> > > > > > >> > > > that
> > > > >> > > > > > >> > > > > >> > Samza doesn't give any help to people
> > > wanting
> > > > to
> > > > >> > run
> > > > >> > > on
> > > > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > > > >> > > > > > >> > > > > >> > So the docs would basically have a link
> to
> > > > >> Slider
> > > > >> > and
> > > > >> > > > > > nothing
> > > > >> > > > > > >> > > else?
> > > > >> > > > > > >> > > > Or
> > > > >> > > > > > >> > > > > >> > would we maintain integrations with a
> > bunch
> > > of
> > > > >> > > popular
> > > > >> > > > > > >> > deployment
> > > > >> > > > > > >> > > > > >> methods
> > > > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell
> scripts
> > > to
> > > > >> make
> > > > >> > > > Samza
> > > > >> > > > > > work
> > > > >> > > > > > >> > with
> > > > >> > > > > > >> > > > > >> Slider)?
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > > >> > I absolutely think it's a good idea to
> > have
> > > > the
> > > > >> > "as a
> > > > >> > > > > > library"
> > > > >> > > > > > >> > and
> > > > >> > > > > > >> > > > > "as a
> > > > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options
> for
> > > > >> people
> > > > >> > who
> > > > >> > > > > want
> > > > >> > > > > > >> them,
> > > > >> > > > > > >> > > > but I
> > > > >> > > > > > >> > > > > >> > think there should also be a
> low-friction
> > > path
> > > > >> for
> > > > >> > > > common
> > > > >> > > > > > "as
> > > > >> > > > > > >> a
> > > > >> > > > > > >> > > > > service"
> > > > >> > > > > > >> > > > > >> > deployment methods, for which we
> probably
> > > need
> > > > >> to
> > > > >> > > > > maintain
> > > > >> > > > > > >> > > > > integrations.
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems
> > odd
> > > to
> > > > >> me,
> > > > >> > > > > because
> > > > >> > > > > > >> Kafka
> > > > >> > > > > > >> > > is
> > > > >> > > > > > >> > > > > all
> > > > >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> > > > >> Transformers"
> > > > >> > > or
> > > > >> > > > > > "Kafka
> > > > >> > > > > > >> > > > Filters"
> > > > >> > > > > > >> > > > > >> > would be more apt?
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of
> Samza
> > > > >> (stream
> > > > >> > > > > > >> transformation
> > > > >> > > > > > >> > > > with
> > > > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> > > > >> library"
> > > > >> > > bit)
> > > > >> > > > > > could
> > > > >> > > > > > >> > > become
> > > > >> > > > > > >> > > > > >> part of
> > > > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
> > > > >> streaming
> > > > >> > SQL
> > > > >> > > > and
> > > > >> > > > > > >> > > > integrations
> > > > >> > > > > > >> > > > > >> with
> > > > >> > > > > > >> > > > > >> > deployment frameworks remain in a
> separate
> > > > >> project?
> > > > >> > > In
> > > > >> > > > > > other
> > > > >> > > > > > >> > > words,
> > > > >> > > > > > >> > > > > >> Kafka
> > > > >> > > > > > >> > > > > >> > would absorb the proven, stable core of
> > > Samza,
> > > > >> > which
> > > > >> > > > > would
> > > > >> > > > > > >> > become
> > > > >> > > > > > >> > > > the
> > > > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in
> > this
> > > > >> > thread.
> > > > >> > > > The
> > > > >> > > > > > Samza
> > > > >> > > > > > >> > > > project
> > > > >> > > > > > >> > > > > >> > would then target that third Kafka
> client
> > as
> > > > its
> > > > >> > base
> > > > >> > > > > API,
> > > > >> > > > > > and
> > > > >> > > > > > >> > the
> > > > >> > > > > > >> > > > > >> project
> > > > >> > > > > > >> > > > > >> > would be freed up to explore more
> > > experimental
> > > > >> new
> > > > >> > > > > > horizons.
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > > >> > Martin
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > > > >> > > > jay.kreps@gmail.com>
> > > > >> > > > > > >> wrote:
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > > >> > > Hey Martin,
> > > > >> > > > > > >> > > > > >> > >
> > > > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I
> > > actually
> > > > >> > don't
> > > > >> > > > > think
> > > > >> > > > > > it
> > > > >> > > > > > >> > ties
> > > > >> > > > > > >> > > > our
> > > > >> > > > > > >> > > > > >> > hands
> > > > >> > > > > > >> > > > > >> > > at all, all it does is refactor
> things.
> > > The
> > > > >> > > division
> > > > >> > > > of
> > > > >> > > > > > >> > > > > >> responsibility is
> > > > >> > > > > > >> > > > > >> > > that Samza core is responsible for
> task
> > > > >> > lifecycle,
> > > > >> > > > > state,
> > > > >> > > > > > >> and
> > > > >> > > > > > >> > > > > >> partition
> > > > >> > > > > > >> > > > > >> > > management (using the Kafka
> > co-ordinator)
> > > > but
> > > > >> it
> > > > >> > is
> > > > >> > > > NOT
> > > > >> > > > > > >> > > > responsible
> > > > >> > > > > > >> > > > > >> for
> > > > >> > > > > > >> > > > > >> > > packaging, configuration deployment or
> > > > >> execution
> > > > >> > of
> > > > >> > > > > > >> processes.
> > > > >> > > > > > >> > > The
> > > > >> > > > > > >> > > > > >> > problem
> > > > >> > > > > > >> > > > > >> > > of packaging and starting these
> > processes
> > > is
> > > > >> > > > > > >> > > > > >> > > framework/environment-specific. This
> > > leaves
> > > > >> > > > individual
> > > > >> > > > > > >> > > frameworks
> > > > >> > > > > > >> > > > to
> > > > >> > > > > > >> > > > > >> be
> > > > >> > > > > > >> > > > > >> > as
> > > > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you
> > can
> > > > get
> > > > >> > > simple
> > > > >> > > > > > >> stateless
> > > > >> > > > > > >> > > > > >> support in
> > > > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their
> > off-the-shelf
> > > > app
> > > > >> > > > > framework
> > > > >> > > > > > >> > > (Slider,
> > > > >> > > > > > >> > > > > >> > Marathon,
> > > > >> > > > > > >> > > > > >> > > etc). These are well known by people
> and
> > > > have
> > > > >> > nice
> > > > >> > > > UIs
> > > > >> > > > > > and a
> > > > >> > > > > > >> > lot
> > > > >> > > > > > >> > > > of
> > > > >> > > > > > >> > > > > >> > > flexibility. I don't think they have
> > node
> > > > >> > affinity
> > > > >> > > > as a
> > > > >> > > > > > >> built
> > > > >> > > > > > >> > in
> > > > >> > > > > > >> > > > > >> option
> > > > >> > > > > > >> > > > > >> > > (though I could be wrong). So if we
> want
> > > > that
> > > > >> we
> > > > >> > > can
> > > > >> > > > > > either
> > > > >> > > > > > >> > wait
> > > > >> > > > > > >> > > > for
> > > > >> > > > > > >> > > > > >> them
> > > > >> > > > > > >> > > > > >> > > to add it or do a custom framework to
> > add
> > > > that
> > > > >> > > > feature
> > > > >> > > > > > (as
> > > > >> > > > > > >> > now).
> > > > >> > > > > > >> > > > > >> > Obviously
> > > > >> > > > > > >> > > > > >> > > if you manage things with old-school
> ops
> > > > tools
> > > > >> > > > > > >> > (puppet/chef/etc)
> > > > >> > > > > > >> > > > you
> > > > >> > > > > > >> > > > > >> get
> > > > >> > > > > > >> > > > > >> > > locality easily. The nice thing,
> though,
> > > is
> > > > >> that
> > > > >> > > all
> > > > >> > > > > the
> > > > >> > > > > > >> samza
> > > > >> > > > > > >> > > > > >> "business
> > > > >> > > > > > >> > > > > >> > > logic" around partition management and
> > > fault
> > > > >> > > > tolerance
> > > > >> > > > > > is in
> > > > >> > > > > > >> > > Samza
> > > > >> > > > > > >> > > > > >> core
> > > > >> > > > > > >> > > > > >> > so
> > > > >> > > > > > >> > > > > >> > > it is shared across frameworks and the
> > > > >> framework
> > > > >> > > > > specific
> > > > >> > > > > > >> bit
> > > > >> > > > > > >> > is
> > > > >> > > > > > >> > > > > just
> > > > >> > > > > > >> > > > > >> > > whether it is smart enough to try to
> get
> > > the
> > > > >> same
> > > > >> > > > host
> > > > >> > > > > > when
> > > > >> > > > > > >> a
> > > > >> > > > > > >> > > job
> > > > >> > > > > > >> > > > is
> > > > >> > > > > > >> > > > > >> > > restarted.
> > > > >> > > > > > >> > > > > >> > >
> > > > >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment,
> > yeah
> > > I
> > > > >> think
> > > > >> > > the
> > > > >> > > > > > goal
> > > > >> > > > > > >> > would
> > > > >> > > > > > >> > > > be
> > > > >> > > > > > >> > > > > >> (a)
> > > > >> > > > > > >> > > > > >> > > actually get better alignment in user
> > > > >> experience,
> > > > >> > > and
> > > > >> > > > > (b)
> > > > >> > > > > > >> > > express
> > > > >> > > > > > >> > > > > >> this in
> > > > >> > > > > > >> > > > > >> > > the naming and project branding.
> > > > Specifically:
> > > > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for
> > the
> > > > >> > > > > > "transformation"
> > > > >> > > > > > >> api
> > > > >> > > > > > >> > > to
> > > > >> > > > > > >> > > > be
> > > > >> > > > > > >> > > > > >> > > discoverable in the main Kafka
> > docs--i.e.
> > > be
> > > > >> able
> > > > >> > > to
> > > > >> > > > > > explain
> > > > >> > > > > > >> > > when
> > > > >> > > > > > >> > > > to
> > > > >> > > > > > >> > > > > >> use
> > > > >> > > > > > >> > > > > >> > > the consumer and when to use the
> stream
> > > > >> > processing
> > > > >> > > > > > >> > functionality
> > > > >> > > > > > >> > > > and
> > > > >> > > > > > >> > > > > >> lead
> > > > >> > > > > > >> > > > > >> > > people into that experience.
> > > > >> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza
> > > 1.4.2
> > > > >> (or
> > > > >> > > > > > whatever)
> > > > >> > > > > > >> > that
> > > > >> > > > > > >> > > > has
> > > > >> > > > > > >> > > > > >> both
> > > > >> > > > > > >> > > > > >> > > Kafka and the stream processing part
> and
> > > > they
> > > > >> > > > actually
> > > > >> > > > > > work
> > > > >> > > > > > >> > > > > together.
> > > > >> > > > > > >> > > > > >> > > 3. Unify the programming experience so
> > the
> > > > >> client
> > > > >> > > and
> > > > >> > > > > > Samza
> > > > >> > > > > > >> > api
> > > > >> > > > > > >> > > > > share
> > > > >> > > > > > >> > > > > >> > >
> config/monitoring/naming/packaging/etc.
> > > > >> > > > > > >> > > > > >> > >
> > > > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> > > > committers
> > > > >> and
> > > > >> > > can
> > > > >> > > > > > have a
> > > > >> > > > > > >> > > > > separate
> > > > >> > > > > > >> > > > > >> > repo,
> > > > >> > > > > > >> > > > > >> > > but I'm actually not really sure (I
> > can't
> > > > >> find a
> > > > >> > > > > > definition
> > > > >> > > > > > >> > of a
> > > > >> > > > > > >> > > > > >> > subproject
> > > > >> > > > > > >> > > > > >> > > in Apache).
> > > > >> > > > > > >> > > > > >> > >
> > > > >> > > > > > >> > > > > >> > > Basically at a high-level you want the
> > > > >> experience
> > > > >> > > to
> > > > >> > > > > > "feel"
> > > > >> > > > > > >> > > like a
> > > > >> > > > > > >> > > > > >> single
> > > > >> > > > > > >> > > > > >> > > system, not to relatively independent
> > > things
> > > > >> that
> > > > >> > > are
> > > > >> > > > > > kind
> > > > >> > > > > > >> of
> > > > >> > > > > > >> > > > > >> awkwardly
> > > > >> > > > > > >> > > > > >> > > glued together.
> > > > >> > > > > > >> > > > > >> > >
> > > > >> > > > > > >> > > > > >> > > I think if we did that they having
> > naming
> > > or
> > > > >> > > branding
> > > > >> > > > > > like
> > > > >> > > > > > >> > > "kafka
> > > > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or
> > something
> > > > >> like
> > > > >> > > that
> > > > >> > > > > > would
> > > > >> > > > > > >> > > > actually
> > > > >> > > > > > >> > > > > >> do a
> > > > >> > > > > > >> > > > > >> > > good job of conveying what it is. I do
> > > that
> > > > >> this
> > > > >> > > > would
> > > > >> > > > > > help
> > > > >> > > > > > >> > > > adoption
> > > > >> > > > > > >> > > > > >> > quite
> > > > >> > > > > > >> > > > > >> > > a lot as it would correctly convey
> that
> > > > using
> > > > >> > Kafka
> > > > >> > > > > > >> Streaming
> > > > >> > > > > > >> > > with
> > > > >> > > > > > >> > > > > >> Kafka
> > > > >> > > > > > >> > > > > >> > is
> > > > >> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka
> > is
> > > > >> pretty
> > > > >> > > > > heavily
> > > > >> > > > > > >> > adopted
> > > > >> > > > > > >> > > > at
> > > > >> > > > > > >> > > > > >> this
> > > > >> > > > > > >> > > > > >> > > point.
> > > > >> > > > > > >> > > > > >> > >
> > > > >> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> > > > >> originally
> > > > >> > > > when
> > > > >> > > > > > open
> > > > >> > > > > > >> > > > sourcing
> > > > >> > > > > > >> > > > > >> > Samza,
> > > > >> > > > > > >> > > > > >> > > however at that time Kafka was
> > relatively
> > > > >> unknown
> > > > >> > > and
> > > > >> > > > > we
> > > > >> > > > > > >> > decided
> > > > >> > > > > > >> > > > not
> > > > >> > > > > > >> > > > > >> to
> > > > >> > > > > > >> > > > > >> > do
> > > > >> > > > > > >> > > > > >> > > it since we felt it would be limiting.
> > > From
> > > > my
> > > > >> > > point
> > > > >> > > > of
> > > > >> > > > > > view
> > > > >> > > > > > >> > the
> > > > >> > > > > > >> > > > > three
> > > > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now
> > > really
> > > > >> > heavily
> > > > >> > > > > used
> > > > >> > > > > > for
> > > > >> > > > > > >> > > > stream
> > > > >> > > > > > >> > > > > >> > > processing, (2) we learned that
> > > abstracting
> > > > >> out
> > > > >> > the
> > > > >> > > > > > stream
> > > > >> > > > > > >> > well
> > > > >> > > > > > >> > > is
> > > > >> > > > > > >> > > > > >> > > basically impossible, (3) we learned
> it
> > is
> > > > >> really
> > > > >> > > > hard
> > > > >> > > > > to
> > > > >> > > > > > >> keep
> > > > >> > > > > > >> > > the
> > > > >> > > > > > >> > > > > two
> > > > >> > > > > > >> > > > > >> > > things feeling like a single product.
> > > > >> > > > > > >> > > > > >> > >
> > > > >> > > > > > >> > > > > >> > > -Jay
> > > > >> > > > > > >> > > > > >> > >
> > > > >> > > > > > >> > > > > >> > >
> > > > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> > > > >> Kleppmann
> > > > >> > <
> > > > >> > > > > > >> > > > > >> martin@kleppmann.com>
> > > > >> > > > > > >> > > > > >> > > wrote:
> > > > >> > > > > > >> > > > > >> > >
> > > > >> > > > > > >> > > > > >> > >> Hi all,
> > > > >> > > > > > >> > > > > >> > >>
> > > > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > > > >> > > > > > >> > > > > >> > >>
> > > > >> > > > > > >> > > > > >> > >> I agree with the general philosophy
> of
> > > > tying
> > > > >> > Samza
> > > > >> > > > > more
> > > > >> > > > > > >> > firmly
> > > > >> > > > > > >> > > to
> > > > >> > > > > > >> > > > > >> Kafka.
> > > > >> > > > > > >> > > > > >> > >> After I spent a while looking at
> > > > integrating
> > > > >> > other
> > > > >> > > > > > message
> > > > >> > > > > > >> > > > brokers
> > > > >> > > > > > >> > > > > >> (e.g.
> > > > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came
> to
> > > the
> > > > >> > > > conclusion
> > > > >> > > > > > that
> > > > >> > > > > > >> > > > > >> > SystemConsumer
> > > > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
> > > > Kafka's
> > > > >> > that
> > > > >> > > > > pretty
> > > > >> > > > > > >> much
> > > > >> > > > > > >> > > > > nobody
> > > > >> > > > > > >> > > > > >> but
> > > > >> > > > > > >> > > > > >> > >> Kafka actually implements it.
> (Databus
> > is
> > > > >> > perhaps
> > > > >> > > an
> > > > >> > > > > > >> > exception,
> > > > >> > > > > > >> > > > but
> > > > >> > > > > > >> > > > > >> it
> > > > >> > > > > > >> > > > > >> > >> isn't widely used outside of
> LinkedIn.)
> > > > Thus,
> > > > >> > > making
> > > > >> > > > > > Samza
> > > > >> > > > > > >> > > fully
> > > > >> > > > > > >> > > > > >> > dependent
> > > > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > > > >> > system-independence
> > > > >> > > > was
> > > > >> > > > > > >> never
> > > > >> > > > > > >> > as
> > > > >> > > > > > >> > > > > real
> > > > >> > > > > > >> > > > > >> as
> > > > >> > > > > > >> > > > > >> > we
> > > > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains
> of
> > > > code
> > > > >> > reuse
> > > > >> > > > are
> > > > >> > > > > > >> real.
> > > > >> > > > > > >> > > > > >> > >>
> > > > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from
> YARN
> > > has
> > > > >> also
> > > > >> > > > always
> > > > >> > > > > > been
> > > > >> > > > > > >> > > > > >> appealing to
> > > > >> > > > > > >> > > > > >> > >> me, for various reasons already
> > mentioned
> > > > in
> > > > >> > this
> > > > >> > > > > > thread.
> > > > >> > > > > > >> > > > Although
> > > > >> > > > > > >> > > > > >> > making
> > > > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > > > >> > > > (YARN/Mesos/AWS/etc)
> > > > >> > > > > > >> seems
> > > > >> > > > > > >> > > > > >> laudable,
> > > > >> > > > > > >> > > > > >> > I am
> > > > >> > > > > > >> > > > > >> > >> a little concerned that it will
> > restrict
> > > us
> > > > >> to a
> > > > >> > > > > lowest
> > > > >> > > > > > >> > common
> > > > >> > > > > > >> > > > > >> > denominator.
> > > > >> > > > > > >> > > > > >> > >> For example, would host affinity
> > > > (SAMZA-617)
> > > > >> > still
> > > > >> > > > be
> > > > >> > > > > > >> > possible?
> > > > >> > > > > > >> > > > For
> > > > >> > > > > > >> > > > > >> jobs
> > > > >> > > > > > >> > > > > >> > >> with large amounts of state, I think
> > > > >> SAMZA-617
> > > > >> > > would
> > > > >> > > > > be
> > > > >> > > > > > a
> > > > >> > > > > > >> big
> > > > >> > > > > > >> > > > boon,
> > > > >> > > > > > >> > > > > >> > since
> > > > >> > > > > > >> > > > > >> > >> restoring state off the changelog on
> > > every
> > > > >> > single
> > > > >> > > > > > restart
> > > > >> > > > > > >> is
> > > > >> > > > > > >> > > > > painful,
> > > > >> > > > > > >> > > > > >> > due
> > > > >> > > > > > >> > > > > >> > >> to long recovery times. It would be a
> > > shame
> > > > >> if
> > > > >> > the
> > > > >> > > > > > >> decoupling
> > > > >> > > > > > >> > > > from
> > > > >> > > > > > >> > > > > >> YARN
> > > > >> > > > > > >> > > > > >> > >> made host affinity impossible.
> > > > >> > > > > > >> > > > > >> > >>
> > > > >> > > > > > >> > > > > >> > >> Jay, a question about the proposed
> API
> > > for
> > > > >> > > > > > instantiating a
> > > > >> > > > > > >> > job
> > > > >> > > > > > >> > > in
> > > > >> > > > > > >> > > > > >> code
> > > > >> > > > > > >> > > > > >> > >> (rather than a properties file): when
> > > > >> > submitting a
> > > > >> > > > job
> > > > >> > > > > > to a
> > > > >> > > > > > >> > > > > cluster,
> > > > >> > > > > > >> > > > > >> is
> > > > >> > > > > > >> > > > > >> > the
> > > > >> > > > > > >> > > > > >> > >> idea that the instantiation code runs
> > on
> > > a
> > > > >> > client
> > > > >> > > > > > >> somewhere,
> > > > >> > > > > > >> > > > which
> > > > >> > > > > > >> > > > > >> then
> > > > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > > > >> > > YARN/Mesos/AWS/etc?
> > > > >> > > > > Or
> > > > >> > > > > > >> does
> > > > >> > > > > > >> > > that
> > > > >> > > > > > >> > > > > >> code
> > > > >> > > > > > >> > > > > >> > run
> > > > >> > > > > > >> > > > > >> > >> on each container that is part of the
> > job
> > > > (in
> > > > >> > > which
> > > > >> > > > > > case,
> > > > >> > > > > > >> how
> > > > >> > > > > > >> > > > does
> > > > >> > > > > > >> > > > > >> the
> > > > >> > > > > > >> > > > > >> > job
> > > > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > > > >> > > > > > >> > > > > >> > >>
> > > > >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't
> feel
> > > > >> right to
> > > > >> > > > make
> > > > >> > > > > a
> > > > >> > > > > > 1.0
> > > > >> > > > > > >> > > > release
> > > > >> > > > > > >> > > > > >> > with a
> > > > >> > > > > > >> > > > > >> > >> plan for it to be immediately
> obsolete.
> > > So
> > > > if
> > > > >> > this
> > > > >> > > > is
> > > > >> > > > > > going
> > > > >> > > > > > >> > to
> > > > >> > > > > > >> > > > > >> happen, I
> > > > >> > > > > > >> > > > > >> > >> think it would be more honest to
> stick
> > > with
> > > > >> 0.*
> > > > >> > > > > version
> > > > >> > > > > > >> > numbers
> > > > >> > > > > > >> > > > > until
> > > > >> > > > > > >> > > > > >> > the
> > > > >> > > > > > >> > > > > >> > >> library-ified Samza has been
> > implemented,
> > > > is
> > > > >> > > stable
> > > > >> > > > > and
> > > > >> > > > > > >> > widely
> > > > >> > > > > > >> > > > > used.
> > > > >> > > > > > >> > > > > >> > >>
> > > > >> > > > > > >> > > > > >> > >> Should the new Samza be a subproject
> of
> > > > >> Kafka?
> > > > >> > > There
> > > > >> > > > > is
> > > > >> > > > > > >> > > precedent
> > > > >> > > > > > >> > > > > for
> > > > >> > > > > > >> > > > > >> > >> tight coupling between different
> Apache
> > > > >> projects
> > > > >> > > > (e.g.
> > > > >> > > > > > >> > Curator
> > > > >> > > > > > >> > > > and
> > > > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I
> > > think
> > > > >> > > remaining
> > > > >> > > > > > >> separate
> > > > >> > > > > > >> > > > would
> > > > >> > > > > > >> > > > > >> be
> > > > >> > > > > > >> > > > > >> > ok.
> > > > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on
> > > Kafka,
> > > > >> there
> > > > >> > > is
> > > > >> > > > > > enough
> > > > >> > > > > > >> > > > > substance
> > > > >> > > > > > >> > > > > >> in
> > > > >> > > > > > >> > > > > >> > >> Samza that it warrants being a
> separate
> > > > >> project.
> > > > >> > > An
> > > > >> > > > > > >> argument
> > > > >> > > > > > >> > in
> > > > >> > > > > > >> > > > > >> favour
> > > > >> > > > > > >> > > > > >> > of
> > > > >> > > > > > >> > > > > >> > >> merging would be if we think Kafka
> has
> > a
> > > > much
> > > > >> > > > stronger
> > > > >> > > > > > >> "brand
> > > > >> > > > > > >> > > > > >> presence"
> > > > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that
> one.
> > > If
> > > > >> the
> > > > >> > > Kafka
> > > > >> > > > > > >> project
> > > > >> > > > > > >> > is
> > > > >> > > > > > >> > > > > >> willing
> > > > >> > > > > > >> > > > > >> > to
> > > > >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way
> of
> > > > doing
> > > > >> > > > stateful
> > > > >> > > > > > >> stream
> > > > >> > > > > > >> > > > > >> > >> transformations, that would probably
> > have
> > > > >> much
> > > > >> > the
> > > > >> > > > > same
> > > > >> > > > > > >> > effect
> > > > >> > > > > > >> > > as
> > > > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> > > > >> Processors"
> > > > >> > or
> > > > >> > > > > > suchlike.
> > > > >> > > > > > >> > > Close
> > > > >> > > > > > >> > > > > >> > >> collaboration between the two
> projects
> > > will
> > > > >> be
> > > > >> > > > needed
> > > > >> > > > > in
> > > > >> > > > > > >> any
> > > > >> > > > > > >> > > > case.
> > > > >> > > > > > >> > > > > >> > >>
> > > > >> > > > > > >> > > > > >> > >> From a project management
> perspective,
> > I
> > > > >> guess
> > > > >> > the
> > > > >> > > > > "new
> > > > >> > > > > > >> > Samza"
> > > > >> > > > > > >> > > > > would
> > > > >> > > > > > >> > > > > >> > have
> > > > >> > > > > > >> > > > > >> > >> to be developed on a branch alongside
> > > > ongoing
> > > > >> > > > > > maintenance
> > > > >> > > > > > >> of
> > > > >> > > > > > >> > > the
> > > > >> > > > > > >> > > > > >> current
> > > > >> > > > > > >> > > > > >> > >> line of development? I think it would
> > be
> > > > >> > important
> > > > >> > > > to
> > > > >> > > > > > >> > continue
> > > > >> > > > > > >> > > > > >> > supporting
> > > > >> > > > > > >> > > > > >> > >> existing users, and provide a
> graceful
> > > > >> migration
> > > > >> > > > path
> > > > >> > > > > to
> > > > >> > > > > > >> the
> > > > >> > > > > > >> > > new
> > > > >> > > > > > >> > > > > >> > version.
> > > > >> > > > > > >> > > > > >> > >> Leaving the current versions
> > unsupported
> > > > and
> > > > >> > > forcing
> > > > >> > > > > > people
> > > > >> > > > > > >> > to
> > > > >> > > > > > >> > > > > >> rewrite
> > > > >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > > > >> > > > > > >> > > > > >> > >>
> > > > >> > > > > > >> > > > > >> > >> Best,
> > > > >> > > > > > >> > > > > >> > >> Martin
> > > > >> > > > > > >> > > > > >> > >>
> > > > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > > > >> > > > jay@confluent.io>
> > > > >> > > > > > >> wrote:
> > > > >> > > > > > >> > > > > >> > >>
> > > > >> > > > > > >> > > > > >> > >>> Hey Garry,
> > > > >> > > > > > >> > > > > >> > >>>
> > > > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd
> be
> > > > happy
> > > > >> to
> > > > >> > > chat
> > > > >> > > > > > more
> > > > >> > > > > > >> > about
> > > > >> > > > > > >> > > > > this
> > > > >> > > > > > >> > > > > >> if
> > > > >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris
> > and I
> > > > >> > started
> > > > >> > > > with
> > > > >> > > > > > the
> > > > >> > > > > > >> > idea
> > > > >> > > > > > >> > > > of
> > > > >> > > > > > >> > > > > >> "what
> > > > >> > > > > > >> > > > > >> > >>> would it take to make Samza a
> kick-ass
> > > > >> > ingestion
> > > > >> > > > > tool"
> > > > >> > > > > > but
> > > > >> > > > > > >> > > > > >> ultimately
> > > > >> > > > > > >> > > > > >> > we
> > > > >> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> > > > >> ingestion
> > > > >> > > and
> > > > >> > > > > > >> > > > transformation
> > > > >> > > > > > >> > > > > >> had
> > > > >> > > > > > >> > > > > >> > >>> pretty different needs and coupling
> > the
> > > > two
> > > > >> > made
> > > > >> > > > > things
> > > > >> > > > > > >> > hard.
> > > > >> > > > > > >> > > > > >> > >>>
> > > > >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
> > > > (KIP-26)
> > > > >> > > > actually
> > > > >> > > > > > will
> > > > >> > > > > > >> > do
> > > > >> > > > > > >> > > > what
> > > > >> > > > > > >> > > > > >> you
> > > > >> > > > > > >> > > > > >> > >> are
> > > > >> > > > > > >> > > > > >> > >>> looking for.
> > > > >> > > > > > >> > > > > >> > >>>
> > > > >> > > > > > >> > > > > >> > >>> With regard to your point about
> > slider,
> > > I
> > > > >> don't
> > > > >> > > > > > >> necessarily
> > > > >> > > > > > >> > > > > >> disagree.
> > > > >> > > > > > >> > > > > >> > >> But I
> > > > >> > > > > > >> > > > > >> > >>> think getting good YARN support is
> > quite
> > > > >> doable
> > > > >> > > > and I
> > > > >> > > > > > >> think
> > > > >> > > > > > >> > we
> > > > >> > > > > > >> > > > can
> > > > >> > > > > > >> > > > > >> make
> > > > >> > > > > > >> > > > > >> > >>> that work well. I think the issue
> this
> > > > >> proposal
> > > > >> > > > > solves
> > > > >> > > > > > is
> > > > >> > > > > > >> > that
> > > > >> > > > > > >> > > > > >> > >> technically
> > > > >> > > > > > >> > > > > >> > >>> it is pretty hard to support
> multiple
> > > > >> cluster
> > > > >> > > > > > management
> > > > >> > > > > > >> > > systems
> > > > >> > > > > > >> > > > > the
> > > > >> > > > > > >> > > > > >> > way
> > > > >> > > > > > >> > > > > >> > >>> things are now, you need to write an
> > > "app
> > > > >> > master"
> > > > >> > > > or
> > > > >> > > > > > >> > > "framework"
> > > > >> > > > > > >> > > > > for
> > > > >> > > > > > >> > > > > >> > each
> > > > >> > > > > > >> > > > > >> > >>> and they are all a little different
> so
> > > > >> testing
> > > > >> > is
> > > > >> > > > > > really
> > > > >> > > > > > >> > hard.
> > > > >> > > > > > >> > > > In
> > > > >> > > > > > >> > > > > >> the
> > > > >> > > > > > >> > > > > >> > >>> absence of this we have been stuck
> > with
> > > > just
> > > > >> > YARN
> > > > >> > > > > which
> > > > >> > > > > > >> has
> > > > >> > > > > > >> > > > > >> fantastic
> > > > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of
> the
> > > > org,
> > > > >> but
> > > > >> > > > zero
> > > > >> > > > > > >> > > penetration
> > > > >> > > > > > >> > > > > >> > >> elsewhere.
> > > > >> > > > > > >> > > > > >> > >>> Given the huge amount of work being
> > put
> > > in
> > > > >> to
> > > > >> > > > slider,
> > > > >> > > > > > >> > > marathon,
> > > > >> > > > > > >> > > > > aws
> > > > >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
> > > > related
> > > > >> > > > packaging
> > > > >> > > > > > >> > > > technologies
> > > > >> > > > > > >> > > > > >> > people
> > > > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes,
> > various
> > > > >> > > > > cloud-specific
> > > > >> > > > > > >> > deploy
> > > > >> > > > > > >> > > > > >> tools,
> > > > >> > > > > > >> > > > > >> > >> etc)
> > > > >> > > > > > >> > > > > >> > >>> I really think it is important to
> get
> > > this
> > > > >> > right.
> > > > >> > > > > > >> > > > > >> > >>>
> > > > >> > > > > > >> > > > > >> > >>> -Jay
> > > > >> > > > > > >> > > > > >> > >>>
> > > > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM,
> Garry
> > > > >> > Turkington
> > > > >> > > <
> > > > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com>
> > wrote:
> > > > >> > > > > > >> > > > > >> > >>>
> > > > >> > > > > > >> > > > > >> > >>>> Hi all,
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>> I think the question below re does
> > > Samza
> > > > >> > become
> > > > >> > > a
> > > > >> > > > > > >> > sub-project
> > > > >> > > > > > >> > > > of
> > > > >> > > > > > >> > > > > >> Kafka
> > > > >> > > > > > >> > > > > >> > >>>> highlights the broader point around
> > > > >> migration.
> > > > >> > > > Chris
> > > > >> > > > > > >> > mentions
> > > > >> > > > > > >> > > > > >> Samza's
> > > > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1
> > > release
> > > > >> but
> > > > >> > I'm
> > > > >> > > > not
> > > > >> > > > > > sure
> > > > >> > > > > > >> > it
> > > > >> > > > > > >> > > > > feels
> > > > >> > > > > > >> > > > > >> > >> right to
> > > > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan
> to
> > > > >> deprecate
> > > > >> > > > most
> > > > >> > > > > of
> > > > >> > > > > > >> it.
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have
> > some
> > > > guys
> > > > >> > who
> > > > >> > > > have
> > > > >> > > > > > >> > started
> > > > >> > > > > > >> > > > > >> working
> > > > >> > > > > > >> > > > > >> > >> with
> > > > >> > > > > > >> > > > > >> > >>>> Samza and building some new
> > > > >> > consumers/producers
> > > > >> > > > was
> > > > >> > > > > > next
> > > > >> > > > > > >> > up.
> > > > >> > > > > > >> > > > > Sounds
> > > > >> > > > > > >> > > > > >> > like
> > > > >> > > > > > >> > > > > >> > >>>> that is absolutely not the
> direction
> > to
> > > > >> go. I
> > > > >> > > need
> > > > >> > > > > to
> > > > >> > > > > > >> look
> > > > >> > > > > > >> > > into
> > > > >> > > > > > >> > > > > the
> > > > >> > > > > > >> > > > > >> > KIP
> > > > >> > > > > > >> > > > > >> > >> in
> > > > >> > > > > > >> > > > > >> > >>>> more detail but for me the
> > > attractiveness
> > > > >> of
> > > > >> > > > adding
> > > > >> > > > > > new
> > > > >> > > > > > >> > Samza
> > > > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes
> all
> > > > they
> > > > >> > were
> > > > >> > > > > doing
> > > > >> > > > > > was
> > > > >> > > > > > >> > > > really
> > > > >> > > > > > >> > > > > >> > getting
> > > > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was
> to
> > > > avoid
> > > > >> > > > having
> > > > >> > > > > to
> > > > >> > > > > > >> > worry
> > > > >> > > > > > >> > > > > about
> > > > >> > > > > > >> > > > > >> the
> > > > >> > > > > > >> > > > > >> > >>>> lifecycle management of external
> > > clients.
> > > > >> If
> > > > >> > > there
> > > > >> > > > > is
> > > > >> > > > > > a
> > > > >> > > > > > >> > > generic
> > > > >> > > > > > >> > > > > >> Kafka
> > > > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can
> plug
> > a
> > > > new
> > > > >> > > > connector
> > > > >> > > > > > into
> > > > >> > > > > > >> > and
> > > > >> > > > > > >> > > > > have
> > > > >> > > > > > >> > > > > >> a
> > > > >> > > > > > >> > > > > >> > >> lot of
> > > > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> > > > reliability
> > > > >> > done
> > > > >> > > > for
> > > > >> > > > > me
> > > > >> > > > > > >> then
> > > > >> > > > > > >> > > it
> > > > >> > > > > > >> > > > > >> gives
> > > > >> > > > > > >> > > > > >> > me
> > > > >> > > > > > >> > > > > >> > >> all
> > > > >> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers
> > > > would.
> > > > >> If
> > > > >> > > not
> > > > >> > > > > > then it
> > > > >> > > > > > >> > > > > >> complicates
> > > > >> > > > > > >> > > > > >> > my
> > > > >> > > > > > >> > > > > >> > >>>> operational deployments.
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>> Which is similar to my other
> question
> > > > with
> > > > >> the
> > > > >> > > > > > proposal
> > > > >> > > > > > >> --
> > > > >> > > > > > >> > if
> > > > >> > > > > > >> > > > we
> > > > >> > > > > > >> > > > > >> > build a
> > > > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza
> > plus
> > > > the
> > > > >> > > > requisite
> > > > >> > > > > > >> shims
> > > > >> > > > > > >> > to
> > > > >> > > > > > >> > > > > >> > integrate
> > > > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the
> former
> > > may
> > > > >> be a
> > > > >> > > lot
> > > > >> > > > > more
> > > > >> > > > > > >> work
> > > > >> > > > > > >> > > > than
> > > > >> > > > > > >> > > > > we
> > > > >> > > > > > >> > > > > >> > >> think.
> > > > >> > > > > > >> > > > > >> > >>>> We may make it much easier for a
> > > newcomer
> > > > >> to
> > > > >> > get
> > > > >> > > > > > >> something
> > > > >> > > > > > >> > > > > running
> > > > >> > > > > > >> > > > > >> but
> > > > >> > > > > > >> > > > > >> > >>>> having them step up and get a
> > reliable
> > > > >> > > production
> > > > >> > > > > > >> > deployment
> > > > >> > > > > > >> > > > may
> > > > >> > > > > > >> > > > > >> still
> > > > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if
> > for
> > > > >> > different
> > > > >> > > > > > reasons
> > > > >> > > > > > >> > than
> > > > >> > > > > > >> > > > > >> today.
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm
> comfortable
> > > > with
> > > > >> > > making
> > > > >> > > > > the
> > > > >> > > > > > >> Samza
> > > > >> > > > > > >> > > > > >> dependency
> > > > >> > > > > > >> > > > > >> > >> on
> > > > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
> > > absolutely
> > > > >> see
> > > > >> > > the
> > > > >> > > > > > >> benefits
> > > > >> > > > > > >> > > in
> > > > >> > > > > > >> > > > > the
> > > > >> > > > > > >> > > > > >> > >>>> reduction of duplication and
> clashing
> > > > >> > > > > > >> > > > terminologies/abstractions
> > > > >> > > > > > >> > > > > >> that
> > > > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a
> > library
> > > > >> would
> > > > >> > > > likely
> > > > >> > > > > > be a
> > > > >> > > > > > >> > very
> > > > >> > > > > > >> > > > > nice
> > > > >> > > > > > >> > > > > >> > tool
> > > > >> > > > > > >> > > > > >> > >> to
> > > > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just
> > have
> > > > the
> > > > >> > > > concerns
> > > > >> > > > > > >> above
> > > > >> > > > > > >> > re
> > > > >> > > > > > >> > > > the
> > > > >> > > > > > >> > > > > >> > >>>> operational side.
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>> Garry
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > > > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci
> Morales
> > > > >> [mailto:
> > > > >> > > > > > >> > gdfm@apache.org
> > > > >> > > > > > >> > > ]
> > > > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > > > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and
> > obesrvations
> > > on
> > > > >> > Samza
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > > > >> > > > > > >> > > > > >> > >>>> From outside, I have always
> perceived
> > > > Samza
> > > > >> > as a
> > > > >> > > > > > >> computing
> > > > >> > > > > > >> > > > layer
> > > > >> > > > > > >> > > > > >> over
> > > > >> > > > > > >> > > > > >> > >>>> Kafka.
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit
> > provocative,
> > > is
> > > > >> > > "should
> > > > >> > > > > > Samza
> > > > >> > > > > > >> be
> > > > >> > > > > > >> > a
> > > > >> > > > > > >> > > > > >> > sub-project
> > > > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > > > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it
> as a
> > > > >> separate
> > > > >> > > > > project
> > > > >> > > > > > >> > with a
> > > > >> > > > > > >> > > > > >> separate
> > > > >> > > > > > >> > > > > >> > >>>> governance?
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>> Cheers,
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>> --
> > > > >> > > > > > >> > > > > >> > >>>> Gianmarco
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > > > >> > > > > > yanfang724@gmail.com>
> > > > >> > > > > > >> > > > wrote:
> > > > >> > > > > > >> > > > > >> > >>>>
> > > > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with
> > Kafka
> > > > more
> > > > >> > > > tightly.
> > > > >> > > > > > >> > Because
> > > > >> > > > > > >> > > > > Samza
> > > > >> > > > > > >> > > > > >> de
> > > > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it
> > should
> > > > >> > leverage
> > > > >> > > > > what
> > > > >> > > > > > >> Kafka
> > > > >> > > > > > >> > > > has.
> > > > >> > > > > > >> > > > > At
> > > > >> > > > > > >> > > > > >> > the
> > > > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
> > > > reinvent
> > > > >> > what
> > > > >> > > > > Samza
> > > > >> > > > > > >> > > already
> > > > >> > > > > > >> > > > > >> has. I
> > > > >> > > > > > >> > > > > >> > >>>>> also like the idea of separating
> the
> > > > >> > ingestion
> > > > >> > > > and
> > > > >> > > > > > >> > > > > transformation.
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for
> me
> > to
> > > > >> image
> > > > >> > > how
> > > > >> > > > > the
> > > > >> > > > > > >> Samza
> > > > >> > > > > > >> > > > will
> > > > >> > > > > > >> > > > > >> look
> > > > >> > > > > > >> > > > > >> > >>>> like.
> > > > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a
> > little
> > > > >> > > difference
> > > > >> > > > > in
> > > > >> > > > > > >> terms
> > > > >> > > > > > >> > > of
> > > > >> > > > > > >> > > > > how
> > > > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's
> > code
> > > > >> shows
> > > > >> > (A
> > > > >> > > > > > client of
> > > > >> > > > > > >> > > > Kakfa)
> > > > >> > > > > > >> > > > > ?
> > > > >> > > > > > >> > > > > >> And
> > > > >> > > > > > >> > > > > >> > >>>>> user's application code calls this
> > > > client?
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library
> of
> > > > Kafka
> > > > >> > (like
> > > > >> > > > > what
> > > > >> > > > > > the
> > > > >> > > > > > >> > > code
> > > > >> > > > > > >> > > > > >> shows),
> > > > >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance
> and
> > > > >> > > > > fault-tolerance?
> > > > >> > > > > > >> Are
> > > > >> > > > > > >> > > they
> > > > >> > > > > > >> > > > > >> taken
> > > > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> > > > >> mechanism,
> > > > >> > > such
> > > > >> > > > > as
> > > > >> > > > > > >> > "Samza
> > > > >> > > > > > >> > > > > >> worker"
> > > > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>> 2. What about other features, such
> > as
> > > > >> > > > auto-scaling,
> > > > >> > > > > > >> shared
> > > > >> > > > > > >> > > > > state,
> > > > >> > > > > > >> > > > > >> > >>>>> monitoring?
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone,
> (is
> > > > this
> > > > >> > what
> > > > >> > > > > Chris
> > > > >> > > > > > >> > > > suggests?)
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data
> from
> > > > Kakfa
> > > > >> > and
> > > > >> > > > > > produce
> > > > >> > > > > > >> to
> > > > >> > > > > > >> > > it.
> > > > >> > > > > > >> > > > > >> Then it
> > > > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza
> looks
> > > > like
> > > > >> > now,
> > > > >> > > > > > except it
> > > > >> > > > > > >> > > does
> > > > >> > > > > > >> > > > > not
> > > > >> > > > > > >> > > > > >> > rely
> > > > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> > > > >> leverage
> > > > >> > > > Kafka's
> > > > >> > > > > > >> > metrics,
> > > > >> > > > > > >> > > > > logs,
> > > > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the
> > dependency?
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>> Thanks,
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > > > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM,
> > > Guozhang
> > > > >> > Wang <
> > > > >> > > > > > >> > > > > wangguoz@gmail.com
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>> Read through the code example and
> > it
> > > > >> looks
> > > > >> > > good
> > > > >> > > > to
> > > > >> > > > > > me.
> > > > >> > > > > > >> A
> > > > >> > > > > > >> > > few
> > > > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> > > > >> runnable
> > > > >> > > like:
> > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > > > >> > > --config-factory=...
> > > > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> > > > deploying
> > > > >> > Samza
> > > > >> > > > > more
> > > > >> > > > > > as
> > > > >> > > > > > >> > > > embedded
> > > > >> > > > > > >> > > > > >> > >>>>>> libraries in user application
> code
> > > > >> (ignoring
> > > > >> > > the
> > > > >> > > > > > >> > > terminology
> > > > >> > > > > > >> > > > > >> since
> > > > >> > > > > > >> > > > > >> > >>>>>> it is not the
> > > > >> > > > > > >> > > > > >> > >>>>> same
> > > > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> > > > >> MyStreamTask(configs);
> > > > >> > > > > Thread
> > > > >> > > > > > >> > thread
> > > > >> > > > > > >> > > =
> > > > >> > > > > > >> > > > > new
> > > > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment
> > > modes
> > > > >> are
> > > > >> > > > > important
> > > > >> > > > > > >> for
> > > > >> > > > > > >> > > > > >> different
> > > > >> > > > > > >> > > > > >> > >>>>>> types
> > > > >> > > > > > >> > > > > >> > >>>>> of
> > > > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making
> > > Samza
> > > > >> > purely
> > > > >> > > > > > >> standalone
> > > > >> > > > > > >> > is
> > > > >> > > > > > >> > > > > still
> > > > >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or
> > > > library
> > > > >> > > modes.
> > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>> Guozhang
> > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM,
> > Jay
> > > > >> Kreps
> > > > >> > <
> > > > >> > > > > > >> > > > jay@confluent.io>
> > > > >> > > > > > >> > > > > >> > wrote:
> > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the
> code
> > > > >> example,
> > > > >> > it
> > > > >> > > > was
> > > > >> > > > > > >> > supposed
> > > > >> > > > > > >> > > > to
> > > > >> > > > > > >> > > > > >> look
> > > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > > >> > > > > > >> > > > > >> > >>>>>>> this:
> > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new
> > Properties();
> > > > >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > > > >> > > > "localhost:4242");
> > > > >> > > > > > >> > > > > >> StreamingConfig
> > > > >> > > > > > >> > > > > >> > >>>>>>> config = new
> > StreamingConfig(props);
> > > > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > > >> > > > "test-topic-2");
> > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > >> > > config.processor(ExampleStreamProcessor.class);
> > > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > > >> > StringSerializer(),
> > > > >> > > > new
> > > > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
> > > KafkaStreaming
> > > > >> > > > container =
> > > > >> > > > > > new
> > > > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> > > > container.run();
> > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>> -Jay
> > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32
> PM,
> > > Jay
> > > > >> > Kreps <
> > > > >> > > > > > >> > > > jay@confluent.io
> > > > >> > > > > > >> > > > > >
> > > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some
> > conversations
> > > > >> Chris
> > > > >> > > and
> > > > >> > > > I
> > > > >> > > > > > were
> > > > >> > > > > > >> > > having
> > > > >> > > > > > >> > > > > >> > >>>>>>>> around
> > > > >> > > > > > >> > > > > >> > >>>>>>> whether
> > > > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use
> Samza
> > > as a
> > > > >> kind
> > > > >> > > of
> > > > >> > > > > data
> > > > >> > > > > > >> > > > ingestion
> > > > >> > > > > > >> > > > > >> > >>>>> framework
> > > > >> > > > > > >> > > > > >> > >>>>>>> for
> > > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to
> > > > KIP-26
> > > > >> > > > > "copycat").
> > > > >> > > > > > >> This
> > > > >> > > > > > >> > > > kind
> > > > >> > > > > > >> > > > > of
> > > > >> > > > > > >> > > > > >> > >>>>>> combined
> > > > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config
> and
> > > > YARN
> > > > >> and
> > > > >> > > the
> > > > >> > > > > > >> > discussion
> > > > >> > > > > > >> > > > > >> around
> > > > >> > > > > > >> > > > > >> > >>>>>>>> how
> > > > >> > > > > > >> > > > > >> > >>>>> to
> > > > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was,
> > > given
> > > > >> that
> > > > >> > > > Samza
> > > > >> > > > > > was
> > > > >> > > > > > >> > > > basically
> > > > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific,
> > > what
> > > > if
> > > > >> > you
> > > > >> > > > just
> > > > >> > > > > > >> > embraced
> > > > >> > > > > > >> > > > > that
> > > > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > > > >> > > > > > >> > > > > >> > >>>>>> into
> > > > >> > > > > > >> > > > > >> > >>>>>>>> something less like a
> heavyweight
> > > > >> > framework
> > > > >> > > > and
> > > > >> > > > > > more
> > > > >> > > > > > >> > > like a
> > > > >> > > > > > >> > > > > >> > >>>>>>>> third
> > > > >> > > > > > >> > > > > >> > >>>>> Kafka
> > > > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> > > > consumer"
> > > > >> > with
> > > > >> > > > > state
> > > > >> > > > > > >> > > > management
> > > > >> > > > > > >> > > > > >> > >>>>>> facilities.
> > > > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead
> of a
> > > > >> complex
> > > > >> > > > stream
> > > > >> > > > > > >> > > processing
> > > > >> > > > > > >> > > > > >> > >>>>>>>> framework
> > > > >> > > > > > >> > > > > >> > >>>>>>> this
> > > > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple
> > > > thing,
> > > > >> not
> > > > >> > > > much
> > > > >> > > > > > more
> > > > >> > > > > > >> > > > > >> complicated
> > > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > > >> > > > > > >> > > > > >> > >>>>> use
> > > > >> > > > > > >> > > > > >> > >>>>>>> or
> > > > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer.
> As
> > > > Chris
> > > > >> > said
> > > > >> > > > we
> > > > >> > > > > > >> thought
> > > > >> > > > > > >> > > > about
> > > > >> > > > > > >> > > > > >> it
> > > > >> > > > > > >> > > > > >> > >>>>>>>> a
> > > > >> > > > > > >> > > > > >> > >>>>> lot
> > > > >> > > > > > >> > > > > >> > >>>>>> of
> > > > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other
> stream
> > > > >> > processing
> > > > >> > > > > > systems
> > > > >> > > > > > >> > were
> > > > >> > > > > > >> > > > > doing)
> > > > >> > > > > > >> > > > > >> > >>>>> seemed
> > > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from
> > MapReduce.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to
> > ingest/output
> > > > >> data
> > > > >> > to
> > > > >> > > > and
> > > > >> > > > > > from
> > > > >> > > > > > >> > the
> > > > >> > > > > > >> > > > > stream
> > > > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we
> actually
> > > > looked
> > > > >> > into
> > > > >> > > > how
> > > > >> > > > > > that
> > > > >> > > > > > >> > > would
> > > > >> > > > > > >> > > > > >> > >>>>>>>> work,
> > > > >> > > > > > >> > > > > >> > >>>>> Samza
> > > > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
> > > ingestion
> > > > >> > > framework
> > > > >> > > > > > for a
> > > > >> > > > > > >> > > bunch
> > > > >> > > > > > >> > > > of
> > > > >> > > > > > >> > > > > >> > >>>>> reasons.
> > > > >> > > > > > >> > > > > >> > >>>>>> To
> > > > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a
> > > > pretty
> > > > >> > > > different
> > > > >> > > > > > >> > internal
> > > > >> > > > > > >> > > > > data
> > > > >> > > > > > >> > > > > >> > >>>>>>>> model
> > > > >> > > > > > >> > > > > >> > >>>>>> and
> > > > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you
> split
> > > > them
> > > > >> and
> > > > >> > > had
> > > > >> > > > > an
> > > > >> > > > > > api
> > > > >> > > > > > >> > for
> > > > >> > > > > > >> > > > > Kafka
> > > > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA
> > KIP-26)
> > > > >> and a
> > > > >> > > > > separate
> > > > >> > > > > > >> api
> > > > >> > > > > > >> > > for
> > > > >> > > > > > >> > > > > >> Kafka
> > > > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> > > > embracing
> > > > >> the
> > > > >> > > > same
> > > > >> > > > > > >> > > terminology
> > > > >> > > > > > >> > > > > and
> > > > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint
> about
> > > the
> > > > >> > current
> > > > >> > > > > > state is
> > > > >> > > > > > >> > > that
> > > > >> > > > > > >> > > > > the
> > > > >> > > > > > >> > > > > >> > >>>>>>>> two
> > > > >> > > > > > >> > > > > >> > >>>>>>> systems
> > > > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on.
> > Terminology
> > > > >> like
> > > > >> > > > > "stream"
> > > > >> > > > > > vs
> > > > >> > > > > > >> > > > "topic"
> > > > >> > > > > > >> > > > > >> and
> > > > >> > > > > > >> > > > > >> > >>>>>>> different
> > > > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems
> > means
> > > > you
> > > > >> > kind
> > > > >> > > > of
> > > > >> > > > > > have
> > > > >> > > > > > >> to
> > > > >> > > > > > >> > > > learn
> > > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > > > >> > > > > > >> > > > > >> > >>>>>>> way,
> > > > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
> > > different
> > > > >> way,
> > > > >> > > > then
> > > > >> > > > > > kind
> > > > >> > > > > > >> of
> > > > >> > > > > > >> > > > > >> > >>>>>>>> understand
> > > > >> > > > > > >> > > > > >> > >>>>> how
> > > > >> > > > > > >> > > > > >> > >>>>>>> they
> > > > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having
> > > > walked
> > > > >> a
> > > > >> > few
> > > > >> > > > > > people
> > > > >> > > > > > >> > > through
> > > > >> > > > > > >> > > > > >> this
> > > > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for
> folks
> > to
> > > > >> get.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a
> lot
> > of
> > > > >> time
> > > > >> > on
> > > > >> > > > > > >> airplanes I
> > > > >> > > > > > >> > > > > hacked
> > > > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> > > > >> incomplete
> > > > >> > > > > > prototype
> > > > >> > > > > > >> of
> > > > >> > > > > > >> > > > what
> > > > >> > > > > > >> > > > > >> > >>>>>>>> this would
> > > > >> > > > > > >> > > > > >> > >>>>> look
> > > > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just
> > unceremoniously
> > > > >> dumped
> > > > >> > > into
> > > > >> > > > > > Kafka
> > > > >> > > > > > >> as
> > > > >> > > > > > >> > > it
> > > > >> > > > > > >> > > > > >> > >>>>>>>> required a
> > > > >> > > > > > >> > > > > >> > >>>>>> few
> > > > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer.
> Here
> > > is
> > > > >> the
> > > > >> > > code:
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > >
> > > > >> > > > > > >> >
> > > > >> > > > > >
> > > > >> > >
> > > > >>
> > > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the
> prototype
> > I
> > > > just
> > > > >> > > > > liberally
> > > > >> > > > > > >> > renamed
> > > > >> > > > > > >> > > > > >> > >>>>>>>> everything
> > > > >> > > > > > >> > > > > >> > >>>>> to
> > > > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with
> > no
> > > > >> regard
> > > > >> > > for
> > > > >> > > > > > >> > > > compatibility.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something
> > like
> > > > >> this:
> > > > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
> > > Properties();
> > > > >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > > > >> > > > > "localhost:4242");
> > > > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > > > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > > > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > > > >> > > > > > >> > > > > >>
> > > config.processor(ExampleStreamProcessor.class);
> > > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > > > >> > > StringDeserializer());
> > > > >> > > > > > >> > > > KafkaStreaming
> > > > >> > > > > > >> > > > > >> > >>>>>> container =
> > > > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > > > >> > container.run();
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > > > >> > > > SamzaContainer;
> > > > >> > > > > > >> > > > > StreamProcessor
> > > > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the
> > > class
> > > > >> names
> > > > >> > > in
> > > > >> > > > a
> > > > >> > > > > > file
> > > > >> > > > > > >> > and
> > > > >> > > > > > >> > > > then
> > > > >> > > > > > >> > > > > >> > >>>>>>>> having
> > > > >> > > > > > >> > > > > >> > >>>>>> the
> > > > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection,
> you
> > > just
> > > > >> > > > > instantiate
> > > > >> > > > > > the
> > > > >> > > > > > >> > > > > container
> > > > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is
> > balanced
> > > > over
> > > > >> > > > however
> > > > >> > > > > > many
> > > > >> > > > > > >> > > > > instances
> > > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > > >> > > > > > >> > > > > >> > >>>>> this
> > > > >> > > > > > >> > > > > >> > >>>>>>> are
> > > > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> > > > instance
> > > > >> > dies,
> > > > >> > > > new
> > > > >> > > > > > >> tasks
> > > > >> > > > > > >> > > are
> > > > >> > > > > > >> > > > > >> added
> > > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > > >> > > > > > >> > > > > >> > >>>>> the
> > > > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
> > > shutting
> > > > >> them
> > > > >> > > > down).
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
> > > > running
> > > > >> > this
> > > > >> > > > > stuff
> > > > >> > > > > > in
> > > > >> > > > > > >> > YARN
> > > > >> > > > > > >> > > > via
> > > > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and
> > AWS
> > > > >> using
> > > > >> > > some
> > > > >> > > > > of
> > > > >> > > > > > >> their
> > > > >> > > > > > >> > > > tools
> > > > >> > > > > > >> > > > > >> > >>>>>>>> but from the
> > > > >> > > > > > >> > > > > >> > >>>>>> point
> > > > >> > > > > > >> > > > > >> > >>>>>>> of
> > > > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these
> > > stream
> > > > >> > > > processing
> > > > >> > > > > > jobs
> > > > >> > > > > > >> > are
> > > > >> > > > > > >> > > > > just
> > > > >> > > > > > >> > > > > >> > >>>>>> stateless
> > > > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go
> and
> > > > >> expand
> > > > >> > and
> > > > >> > > > > > contract
> > > > >> > > > > > >> > at
> > > > >> > > > > > >> > > > > will.
> > > > >> > > > > > >> > > > > >> > >>>>>>>> There
> > > > >> > > > > > >> > > > > >> > >>>>> is
> > > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of
> > code,
> > > > it
> > > > >> > would
> > > > >> > > > get
> > > > >> > > > > > >> larger
> > > > >> > > > > > >> > > if
> > > > >> > > > > > >> > > > we
> > > > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
> > > > larger.
> > > > >> We
> > > > >> > > > really
> > > > >> > > > > > do
> > > > >> > > > > > >> > get a
> > > > >> > > > > > >> > > > ton
> > > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > > >> > > > > > >> > > > > >> > >>>>>>> leverage
> > > > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is
> fully
> > > > >> > delegated
> > > > >> > > to
> > > > >> > > > > the
> > > > >> > > > > > >> new
> > > > >> > > > > > >> > > > > >> consumer.
> > > > >> > > > > > >> > > > > >> > >>>>> This
> > > > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any
> partition
> > > > >> > management
> > > > >> > > > > > strategy
> > > > >> > > > > > >> > > > > available
> > > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > > > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to
> > > Samza
> > > > >> (and
> > > > >> > > vice
> > > > >> > > > > > versa)
> > > > >> > > > > > >> > and
> > > > >> > > > > > >> > > > > with
> > > > >> > > > > > >> > > > > >> > >>>>>>>> the
> > > > >> > > > > > >> > > > > >> > >>>>>>> exact
> > > > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well
> as
> > > > state
> > > > >> > reuse
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully
> it
> > is
> > > > >> > thought
> > > > >> > > > > > >> provoking.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55
> PM,
> > > > Chris
> > > > >> > > > > Riccomini <
> > > > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > > > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions
> with
> > > > Samza
> > > > >> > > > > engineers
> > > > >> > > > > > at
> > > > >> > > > > > >> > > > LinkedIn
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > >> > > > > > >> > > > > >> > >>>>>>> Confluent
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> > > > observations
> > > > >> > and
> > > > >> > > > > would
> > > > >> > > > > > >> like
> > > > >> > > > > > >> > to
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> propose
> > > > >> > > > > > >> > > > > >> > >>>>> some
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things
> that
> > I
> > > > >> want to
> > > > >> > > > call
> > > > >> > > > > > out
> > > > >> > > > > > >> > about
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > > > >> > > > > > >> > > > > >> > >>>>>> design,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> > > > changes.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a
> > > dynamic
> > > > >> > > > deployment
> > > > >> > > > > > >> system.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> > > > >> SystemConsumer/SystemProducer
> > > > >> > and
> > > > >> > > > > > Kafka's
> > > > >> > > > > > >> > > > consumer
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > >> > > > > > >> > > > > >> > >>>>> are
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the
> > same
> > > > >> > problems.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
> > > > related,
> > > > >> > but
> > > > >> > > > I'll
> > > > >> > > > > > >> > address
> > > > >> > > > > > >> > > > them
> > > > >> > > > > > >> > > > > >> in
> > > > >> > > > > > >> > > > > >> > >>>>> order.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the
> > use
> > > > of a
> > > > >> > > > dynamic
> > > > >> > > > > > >> > > deployment
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > > > >> > > > > > >> > > > > >> > >>>>>> such
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> as
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
> > > initially
> > > > >> built
> > > > >> > > > > Samza,
> > > > >> > > > > > we
> > > > >> > > > > > >> > bet
> > > > >> > > > > > >> > > > that
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> there
> > > > >> > > > > > >> > > > > >> > >>>>>> would
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> be
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this
> area,
> > > and
> > > > >> we
> > > > >> > > could
> > > > >> > > > > > >> support
> > > > >> > > > > > >> > > > them,
> > > > >> > > > > > >> > > > > >> and
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > >> > > > > > >> > > > > >> > >>>>>> rest
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality,
> there
> > > are
> > > > >> many
> > > > >> > > > > > >> variations.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > > > >> > > > > > >> > > > > >> > >>>>>> many
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just
> > start
> > > > >> their
> > > > >> > > > > > processors
> > > > >> > > > > > >> > like
> > > > >> > > > > > >> > > > > normal
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
> > > traditional
> > > > >> > > > deployment
> > > > >> > > > > > >> scripts
> > > > >> > > > > > >> > > > such
> > > > >> > > > > > >> > > > > as
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > > > >> > > > > > >> > > > > >> > >>>>>> Chef,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a
> > deployment
> > > > >> system
> > > > >> > > on
> > > > >> > > > > > users
> > > > >> > > > > > >> > makes
> > > > >> > > > > > >> > > > the
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
> > > > painful
> > > > >> for
> > > > >> > > > first
> > > > >> > > > > > time
> > > > >> > > > > > >> > > > users.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
> > > requirement
> > > > >> was
> > > > >> > > also
> > > > >> > > > a
> > > > >> > > > > > bit
> > > > >> > > > > > >> of
> > > > >> > > > > > >> > a
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > > > >> > > > > > >> > > > > >> > >>>>>> because
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> of
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding
> > > > between
> > > > >> > the
> > > > >> > > > > > nature of
> > > > >> > > > > > >> > > batch
> > > > >> > > > > > >> > > > > >> jobs
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > >> > > > > > >> > > > > >> > >>>>>>> stream
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we
> > made
> > > > >> > > conscious
> > > > >> > > > > > effort
> > > > >> > > > > > >> to
> > > > >> > > > > > >> > > > favor
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > >> > > > > > >> > > > > >> > >>>>>> Hadoop
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing
> > things,
> > > > >> since
> > > > >> > it
> > > > >> > > > > worked
> > > > >> > > > > > >> and
> > > > >> > > > > > >> > > was
> > > > >> > > > > > >> > > > > well
> > > > >> > > > > > >> > > > > >> > >>>>>>> understood.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was
> > that
> > > > >> batch
> > > > >> > > jobs
> > > > >> > > > > > have a
> > > > >> > > > > > >> > > > definite
> > > > >> > > > > > >> > > > > >> > >>>>>> beginning,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing
> jobs
> > > > don't
> > > > >> > > > > (usually).
> > > > >> > > > > > >> This
> > > > >> > > > > > >> > > > leads
> > > > >> > > > > > >> > > > > to
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> a
> > > > >> > > > > > >> > > > > >> > >>>>> much
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for
> > > > stream
> > > > >> > > > > processors.
> > > > >> > > > > > >> You
> > > > >> > > > > > >> > > > > >> basically
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> just
> > > > >> > > > > > >> > > > > >> > >>>>>>> need
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> > > > >> processor,
> > > > >> > and
> > > > >> > > > > start
> > > > >> > > > > > >> it.
> > > > >> > > > > > >> > > The
> > > > >> > > > > > >> > > > > way
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn,
> > there's
> > > > no
> > > > >> > > concept
> > > > >> > > > > of
> > > > >> > > > > > a
> > > > >> > > > > > >> > > cluster
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > > > >> > > > > > >> > > > > >> > >>>>>> add
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem
> with
> > > > >> coupling
> > > > >> > > > Samza
> > > > >> > > > > > with
> > > > >> > > > > > >> a
> > > > >> > > > > > >> > > > > >> scheduler
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> is
> > > > >> > > > > > >> > > > > >> > >>>>>> that
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has
> > to
> > > > >> handle
> > > > >> > > > > > deployment.
> > > > >> > > > > > >> > > This
> > > > >> > > > > > >> > > > > >> pulls
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> in a
> > > > >> > > > > > >> > > > > >> > >>>>>>> bunch
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as
> configuration
> > > > >> > > distribution
> > > > >> > > > > > (config
> > > > >> > > > > > >> > > > > stream),
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> shell
> > > > >> > > > > > >> > > > > >> > >>>>>>> scrips
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> > > > packaging
> > > > >> > (all
> > > > >> > > > the
> > > > >> > > > > > .tgz
> > > > >> > > > > > >> > > > stuff),
> > > > >> > > > > > >> > > > > >> etc.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring
> > > dynamic
> > > > >> > > > deployment
> > > > >> > > > > > was
> > > > >> > > > > > >> to
> > > > >> > > > > > >> > > > > support
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to
> > have
> > > > >> > > locality,
> > > > >> > > > > you
> > > > >> > > > > > >> need
> > > > >> > > > > > >> > to
> > > > >> > > > > > >> > > > put
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> your
> > > > >> > > > > > >> > > > > >> > >>>>>> processors
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> > > > processing.
> > > > >> > Upon
> > > > >> > > > > > further
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > > > >> > > > > > >> > > > > >> > >>>>>>> though,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
> > > beneficial.
> > > > >> > There
> > > > >> > > is
> > > > >> > > > > > some
> > > > >> > > > > > >> > good
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > > > >> > > > > > >> > > > > >> > >>>>>> about
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
> > > SAMZA-335.
> > > > >> > Again,
> > > > >> > > we
> > > > >> > > > > > took
> > > > >> > > > > > >> the
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > > > >> > > > > > >> > > > > >> > >>>>>> path,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> > > > differences
> > > > >> > > > between
> > > > >> > > > > > HDFS
> > > > >> > > > > > >> > and
> > > > >> > > > > > >> > > > > Kafka.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > > > >> > > > > > >> > > > > >> > >>>>>> has
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has
> > > partitions.
> > > > >> This
> > > > >> > > > leads
> > > > >> > > > > to
> > > > >> > > > > > >> less
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with
> > stream
> > > > >> > > processors
> > > > >> > > > > on
> > > > >> > > > > > top
> > > > >> > > > > > >> > of
> > > > >> > > > > > >> > > > > Kafka.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a
> > > > crutch.
> > > > >> > > Samza
> > > > >> > > > > > doesn't
> > > > >> > > > > > >> > > have
> > > > >> > > > > > >> > > > > any
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> built
> > > > >> > > > > > >> > > > > >> > >>>>> in
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic.
> Instead,
> > it
> > > > >> > depends
> > > > >> > > on
> > > > >> > > > > the
> > > > >> > > > > > >> > > dynamic
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system
> to
> > > > handle
> > > > >> > > > restarts
> > > > >> > > > > > >> when a
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > > > >> > > > > > >> > > > > >> > >>>>>>> made
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> > > > >> standalone
> > > > >> > > Samza
> > > > >> > > > > > >> > container
> > > > >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is
> > > good,
> > > > >> but I
> > > > >> > > > think
> > > > >> > > > > > that
> > > > >> > > > > > >> > > we've
> > > > >> > > > > > >> > > > > >> gone
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> too
> > > > >> > > > > > >> > > > > >> > >>>>>> far
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment
> systems.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > > > >> > > > (SystemConsumer,
> > > > >> > > > > > >> > > > > SystemProducer,
> > > > >> > > > > > >> > > > > >> > >>>> etc).
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for
> just
> > > > about
> > > > >> > every
> > > > >> > > > > > >> component
> > > > >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> > > > >> > > ConfigRewriter,
> > > > >> > > > > > etc).
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that
> I've
> > > > >> > forgotten,
> > > > >> > > as
> > > > >> > > > > > well.
> > > > >> > > > > > >> > Some
> > > > >> > > > > > >> > > > of
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> these
> > > > >> > > > > > >> > > > > >> > >>>>> are
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven
> not
> > > to
> > > > >> be.
> > > > >> > > This
> > > > >> > > > > all
> > > > >> > > > > > >> comes
> > > > >> > > > > > >> > > at
> > > > >> > > > > > >> > > > a
> > > > >> > > > > > >> > > > > >> cost:
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is
> > > > making
> > > > >> it
> > > > >> > > > harder
> > > > >> > > > > > for
> > > > >> > > > > > >> > our
> > > > >> > > > > > >> > > > > users
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > > >> > > > > > >> > > > > >> > >>>>> pick
> > > > >> > > > > > >> > > > > >> > >>>>>> up
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box.
> It
> > > > also
> > > > >> > makes
> > > > >> > > > it
> > > > >> > > > > > >> > difficult
> > > > >> > > > > > >> > > > for
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason
> about
> > > > what
> > > > >> the
> > > > >> > > > > > >> > > characteristics
> > > > >> > > > > > >> > > > of
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> > > > >> characteristics
> > > > >> > > > change
> > > > >> > > > > > >> > > depending
> > > > >> > > > > > >> > > > on
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability
> are
> > > > most
> > > > >> > > visible
> > > > >> > > > > in
> > > > >> > > > > > the
> > > > >> > > > > > >> > > > System
> > > > >> > > > > > >> > > > > >> APIs.
> > > > >> > > > > > >> > > > > >> > >>>>> What
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
> > > > >> functional is
> > > > >> > > > Kafka
> > > > >> > > > > > as
> > > > >> > > > > > >> its
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> transport
> > > > >> > > > > > >> > > > > >> > >>>>>> layer.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> But
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated
> > use
> > > > >> cases
> > > > >> > > into
> > > > >> > > > > one
> > > > >> > > > > > >> API:
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> The current System API
> supports
> > > both
> > > > >> of
> > > > >> > > these
> > > > >> > > > > use
> > > > >> > > > > > >> > cases.
> > > > >> > > > > > >> > > > The
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> problem
> > > > >> > > > > > >> > > > > >> > >>>>>> is,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> we
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> actually want different
> features
> > > for
> > > > >> each
> > > > >> > > use
> > > > >> > > > > > case.
> > > > >> > > > > > >> By
> > > > >> > > > > > >> > > > > >> papering
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> over
> > > > >> > > > > > >> > > > > >> > >>>>>>> these
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a
> > > > single
> > > > >> > API,
> > > > >> > > > > we've
> > > > >> > > > > > >> > > > introduced
> > > > >> > > > > > >> > > > > a
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > >> > > > > > >> > > > > >> > >>>>>>> leaky
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really
> > like
> > > > in
> > > > >> (2)
> > > > >> > > is
> > > > >> > > > to
> > > > >> > > > > > have
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs
> > for
> > > > >> > offsets
> > > > >> > > > > (like
> > > > >> > > > > > >> > Kafka).
> > > > >> > > > > > >> > > > > This
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > > > >> > > > > > >> > > > > >> > >>>>> with
> > > > >> > > > > > >> > > > > >> > >>>>>>> (1),
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> though, since different
> systems
> > > have
> > > > >> > > > different
> > > > >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on
> the
> > > > >> mailing
> > > > >> > > list
> > > > >> > > > > and
> > > > >> > > > > > >> the
> > > > >> > > > > > >> > > SQL
> > > > >> > > > > > >> > > > > >> JIRAs
> > > > >> > > > > > >> > > > > >> > >>>>> about
> > > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> > > > >> > > replayability.
> > > > >> > > > > > Kafka
> > > > >> > > > > > >> > > allows
> > > > >> > > > > > >> > > > us
> > > > >> > > > > > >> > > > > >> to
> > > > >> > > > > > >> > > > > >> > >>>>> rewind
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> when
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other
> > > > systems
> > > > >> > > don't.
> > > > >> > > > In
> > > > >> > > > > > some
> > > > >> > > > > > >> > > > cases,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > > >> > > > > > >> > > > > >> > >>>>>>> return
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > > > >> > > > > > >> WikipediaSystemConsumer)
> > > > >> > > > > > >> > > > > because
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> they
> > > > >> > > > > > >> > > > > >> > >>>>>> have
> > > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another
> example.
> > > > Kafka
> > > > >> > > > supports
> > > > >> > > > > > >> > > > > partitioning,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > > >> > > > > > >> > > > > >> > >>>>> many
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this
> by
> > > > >> having a
> > > > >> > > > single
> > > > >> > > > > > >> > > partition
> > > > >> > > > > > >> > > > > for
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other
> > > systems
> > > > >> model
> > > > >> > > > > > >> partitioning
> > > > >> > > > > > >> > > > > >> > >>>> differently (e.g.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is
> > also
> > > a
> > > > >> mess.
> > > > >> > > > > > Creating
> > > > >> > > > > > >> > > streams
> > > > >> > > > > > >> > > > > in
> > > > >> > > > > > >> > > > > >> a
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> > > > >> impossible.
> > > > >> > > As
> > > > >> > > > is
> > > > >> > > > > > >> > modeling
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > > > >> > > > > > >> > > > > >> > >>>>> for
> > > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> > > > >> partitions,
> > > > >> > > > > location,
> > > > >> > > > > > >> > etc).
> > > > >> > > > > > >> > > > The
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> list
> > > > >> > > > > > >> > > > > >> > >>>>> goes
> > > > >> > > > > > >> > > > > >> > >>>>>>> on.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began
> > writing
> > > > >> Samza,
> > > > >> > > > > Kafka's
> > > > >> > > > > > >> > > consumer
> > > > >> > > > > > >> > > > > and
> > > > >> > > > > > >> > > > > >> > >>>>> producer
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature
> > set.
> > > > On
> > > > >> the
> > > > >> > > > > > >> > consumer-side,
> > > > >> > > > > > >> > > > you
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> had two
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
> > > > consumer,
> > > > >> or
> > > > >> > > the
> > > > >> > > > > > simple
> > > > >> > > > > > >> > > > > consumer.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> The
> > > > >> > > > > > >> > > > > >> > >>>>>>> problem
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer
> was
> > > > that
> > > > >> it
> > > > >> > > > > > controlled
> > > > >> > > > > > >> > your
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition
> assignments,
> > > and
> > > > >> the
> > > > >> > > order
> > > > >> > > > > in
> > > > >> > > > > > >> which
> > > > >> > > > > > >> > > you
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > > > >> > > > > > >> > > > > >> > >>>>> problem
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> with
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that
> it's
> > > not
> > > > >> > > simple.
> > > > >> > > > > It's
> > > > >> > > > > > >> > basic.
> > > > >> > > > > > >> > > > You
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> end up
> > > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really
> > > low-level
> > > > >> stuff
> > > > >> > > > that
> > > > >> > > > > > you
> > > > >> > > > > > >> > > > > shouldn't.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> We
> > > > >> > > > > > >> > > > > >> > >>>>>> spent a
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > > > >> > > > KafkaSystemConsumer
> > > > >> > > > > > very
> > > > >> > > > > > >> > > > robust.
> > > > >> > > > > > >> > > > > >> It
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some
> > > cool
> > > > >> > > features:
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message
> ordering
> > > and
> > > > >> > > > > > prioritization.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> > > > >> assignment
> > > > >> > > to
> > > > >> > > > > > support
> > > > >> > > > > > >> > > > joins,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> global
> > > > >> > > > > > >> > > > > >> > >>>>>> state
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it
> :)),
> > > > etc.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> > > > >> > checkpointing.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the
> > time
> > > > is
> > > > >> > that
> > > > >> > > > > these
> > > > >> > > > > > >> > > features
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > > >> > > > > > >> > > > > >> > >>>>>>> actually
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
> > > > consumers
> > > > >> > (not
> > > > >> > > > just
> > > > >> > > > > > >> Samza
> > > > >> > > > > > >> > > > stream
> > > > >> > > > > > >> > > > > >> > >>>>>> processors)
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things
> like
> > > > joins
> > > > >> > and
> > > > >> > > > > > partition
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > > > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> > > > >> > conclusion.
> > > > >> > > > > > They're
> > > > >> > > > > > >> > > adding
> > > > >> > > > > > >> > > > a
> > > > >> > > > > > >> > > > > >> ton
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new
> Kafka
> > > > >> consumer
> > > > >> > > > > > >> > > implementation.
> > > > >> > > > > > >> > > > > To a
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > > > >> > > > > > >> > > > > >> > >>>>> it's
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've
> > > already
> > > > >> done
> > > > >> > > in
> > > > >> > > > > > Samza.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up
> > > > taking
> > > > >> a
> > > > >> > > very
> > > > >> > > > > > similar
> > > > >> > > > > > >> > > > > approach
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > > >> > > > > > >> > > > > >> > >>>>>> Samza's
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> > > > implementation
> > > > >> for
> > > > >> > > > > > handling
> > > > >> > > > > > >> > > offset
> > > > >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> > > > >> management
> > > > >> > > > > feature
> > > > >> > > > > > >> > stores
> > > > >> > > > > > >> > > > > >> offset
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and
> > allows
> > > > >> you to
> > > > >> > > > fetch
> > > > >> > > > > > them
> > > > >> > > > > > >> > > from
> > > > >> > > > > > >> > > > > the
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a
> > waste,
> > > > >> since
> > > > >> > we
> > > > >> > > > > could
> > > > >> > > > > > >> have
> > > > >> > > > > > >> > > > shared
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > >> > > > > > >> > > > > >> > >>>>> work
> > > > >> > > > > > >> > > > > >> > >>>>>> if
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> it
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from
> the
> > > > >> get-go.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a
> rather
> > > > >> radical
> > > > >> > > > > > proposal.
> > > > >> > > > > > >> > Samza
> > > > >> > > > > > >> > > > is
> > > > >> > > > > > >> > > > > >> > >>>>> relatively
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd
> > venture
> > > to
> > > > >> say
> > > > >> > > that
> > > > >> > > > > > we're
> > > > >> > > > > > >> > > near a
> > > > >> > > > > > >> > > > > 1.0
> > > > >> > > > > > >> > > > > >> > >>>>>> release.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take
> > what
> > > > >> we've
> > > > >> > > > > learned,
> > > > >> > > > > > and
> > > > >> > > > > > >> > > begin
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > > > >> > > > > > >> > > > > >> > >>>>>>> about
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would
> we
> > > > >> change if
> > > > >> > > we
> > > > >> > > > > were
> > > > >> > > > > > >> > > starting
> > > > >> > > > > > >> > > > > >> from
> > > > >> > > > > > >> > > > > >> > >>>>>> scratch?
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> My
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the
> > > *only*
> > > > >> way
> > > > >> > to
> > > > >> > > > run
> > > > >> > > > > > Samza
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all
> > > direct
> > > > >> > > > > dependences
> > > > >> > > > > > on
> > > > >> > > > > > >> > > YARN,
> > > > >> > > > > > >> > > > > >> Mesos,
> > > > >> > > > > > >> > > > > >> > >>>> etc.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to
> > > support
> > > > >> only
> > > > >> > > > Kafka
> > > > >> > > > > > as
> > > > >> > > > > > >> the
> > > > >> > > > > > >> > > > > stream
> > > > >> > > > > > >> > > > > >> > >>>>>> processing
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics,
> > > > logging,
> > > > >> > > > > > >> serialization,
> > > > >> > > > > > >> > > and
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> config
> > > > >> > > > > > >> > > > > >> > >>>>>>> systems,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's
> instead.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the
> issues
> > > > that
> > > > >> I
> > > > >> > > > > outlined
> > > > >> > > > > > >> > above.
> > > > >> > > > > > >> > > It
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > > >> > > > > > >> > > > > >> > >>>>> also
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base
> > pretty
> > > > >> > > > dramatically.
> > > > >> > > > > > >> > > Supporting
> > > > >> > > > > > >> > > > > >> only
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will
> > allow
> > > > >> Samza
> > > > >> > to
> > > > >> > > be
> > > > >> > > > > > >> executed
> > > > >> > > > > > >> > > on
> > > > >> > > > > > >> > > > > YARN
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > > > >> > > > Marathon/Aurora),
> > > > >> > > > > or
> > > > >> > > > > > >> most
> > > > >> > > > > > >> > > > other
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > > > >> > > > > > >> > > > > >> > >>>>>>> deployment
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make
> life a
> > > lot
> > > > >> > easier
> > > > >> > > > for
> > > > >> > > > > > new
> > > > >> > > > > > >> > > users.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial
> without
> > > > YARN.
> > > > >> > The
> > > > >> > > > drop
> > > > >> > > > > > in
> > > > >> > > > > > >> > > mailing
> > > > >> > > > > > >> > > > > >> list
> > > > >> > > > > > >> > > > > >> > >>>>>> traffic
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
> > > > >> overdue to
> > > > >> > > me.
> > > > >> > > > > The
> > > > >> > > > > > >> > > reality
> > > > >> > > > > > >> > > > > is,
> > > > >> > > > > > >> > > > > >> > >>>>> everyone
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> that
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza
> with
> > > > >> Kafka.
> > > > >> > We
> > > > >> > > > > > basically
> > > > >> > > > > > >> > > > require
> > > > >> > > > > > >> > > > > >> it
> > > > >> > > > > > >> > > > > >> > >>>>>> already
> > > > >> > > > > > >> > > > > >> > >>>>>>> in
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to
> work.
> > > > Those
> > > > >> > that
> > > > >> > > > are
> > > > >> > > > > > >> using
> > > > >> > > > > > >> > > > other
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > > >> > > > > > >> > > > > >> > >>>>>> are
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest
> > into
> > > > >> Kafka
> > > > >> > > (1),
> > > > >> > > > > and
> > > > >> > > > > > >> then
> > > > >> > > > > > >> > > > they
> > > > >> > > > > > >> > > > > do
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There
> is
> > > > >> already
> > > > >> > > > > > discussion (
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>
> > > > >> > > > > > >> > > > > >> >
> > > > >> > > > > > >> > > > >
> > > > >> > > > > > >> >
> > > > >> > > > > >
> > > > >> > >
> > > > >>
> > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > >> > > > > > >> > > > > >> > >>>>> 767
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> )
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting
> into
> > > > Kafka
> > > > >> > > > extremely
> > > > >> > > > > > >> easy.
> > > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to
> couple
> > > with
> > > > >> > Kafka,
> > > > >> > > > we
> > > > >> > > > > > can
> > > > >> > > > > > >> > > > leverage
> > > > >> > > > > > >> > > > > a
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > >> > > > > > >> > > > > >> > >>>>>>> their
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have
> to
> > > > >> maintain
> > > > >> > > our
> > > > >> > > > > own
> > > > >> > > > > > >> > config,
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > > > >> > > > > > >> > > > > >> > >>>>> etc.
> > > > >> > > > > > >> > > > > >> > >>>>>>> We
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same
> > libraries,
> > > > and
> > > > >> > make
> > > > >> > > > them
> > > > >> > > > > > >> > better.
> > > > >> > > > > > >> > > > This
> > > > >> > > > > > >> > > > > >> > >>>>>>>>> will
> > > > >> > > > > > >> > > > > >> >
> > > > >> ...
> > > > >>
> > > > >> [Message clipped]
> > > > >
> > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > Jordan Shaw
> > > Full Stack Software Engineer
> > > PubNub Inc
> > > 1045 17th St
> > > San Francisco, CA 94107
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
Hi, Jay,

Given all the user concerns, the board disagreement on sub-projects, I am
supporting your 5th option as well. As you said, even the end goal is the
same, it might help to pave a smooth path forward. One thing I learned over
the years is that what we planned for may not be the final product, and the
unexpected product may be even better if we learn and adapt along the way.
:)

So, since I assume that in option 5, Samza will fully embrace the new Kafka
Streams API as the core and heavily depends on it, I want to raise up some
detailed logistic questions:
1. How do Samza community contribute to the design and development of the
new Kafka Streams API? As Kartik mentioned, if there is a model for Samza
community to contribute to just this part of Kafka code base, it would be a
huge plus point to the integration.
2. What's the scope of the new Kafka Streams API? Is it just focused on
message consumption, producing, Kafka-based partition distribution, offset
management, message selection and delivery to StreamProcessor? In other
words, I have a question regarding to whether we should put samza-kv-store
in the scope? The reasons that I think that it might be better to stay in
Samza initially are: a) KV-store libraries does not directly interact w/
Kafka brokers, it optionally uses Kafka consumers and producers like a
client program; b) there are a tons of experiments / tune-ups on RocksDB
that we want to have a faster iteration on this library (e.g. there is an
experimental time-sequence KV store implementation from LinkedIn we also
want to experiment on in window operator in SQL). The down-side I can see
is that w/o this in Kafka Streams API, the as-a-library mode may not get
the state management support. If we can find a way to make sure that the
current Samza community can contribute to this library in a faster
velocity, I can be convinced otherwise as well. What's your opinion on this?

Overall, thanks a lot for pushing forward the whole discussion!

-Yi

On Mon, Jul 13, 2015 at 12:56 PM, Garry Turkington <
g.turkington@improvedigital.com> wrote:

> Hi,
>
> I'm also supportive of Jay's option 5. There is a risk the "transformer
> API" -- I'd have preferred Metamorphosis but it's too hard to type! --
> takes on a life of its own and we end up with two very different things but
> given how good the Kafka community has been at introducing new producer and
> consumer clients and giving very clear guidance on when they are production
> ready this is a danger I believe can be managed. It'd also be excellent to
> get some working code to beat around the notions of stream processing atop
> a system with transacdtional messages.
>
> On the question of whether to keep or deprecate SystemConsumer/Producer I
> believe we need get a better understanding over the next while of just what
> the Samza community is looking for in such connectivity. For my own use
> cases I have been looking to add additional implementations primarily to
> use Samza as the data ingress and egress component around Kafka. Writing
> external clients that require their own reliability and scalability
> management gets old real fast and pushing this into a simple Samza job that
> reads from system X and pushes into Kafka (or vice versa) was the obvious
> choice for me in the current model. For this type of usage though copycat
> is likely much superior (obviously needs proven) and the question then is
> if most Samza users look to the system implementations to also act as a
> front-end into Kafka or if significant usage is indeed intended to have the
> alternative systems as the primary message source. That understanding will
> I think give much clarity in just what value the abstraction overhead of
> the current model brings.
>
> Garry
>
> -----Original Message-----
> From: Yan Fang [mailto:yanfang724@gmail.com]
> Sent: 13 July 2015 19:58
> To: dev@samza.apache.org
> Subject: Re: Thoughts and obesrvations on Samza
>
> I am leaning to Jay's fifth approach. It is not radical and gives us some
> time to see the outcome.
>
> In addition, I would suggest:
>
> 1) Keep the SystemConsumer/SystemProducer API. Because current
> SystemConsumer/SystemProducer API satisfies the usage (From Joardan, and
> even Garry's feedback) and is not so broken that we want to deprecate it.
> Though there are some issues in implemnting the Kinesis, they are not
> unfixable. Nothing should prevent Samza, as a stream processing system, to
> support other systems. In addition, there already are some systems
> exiting besides Kafka: ElasticSearch (committed to the master), HDFS
> (patch-available), S3( from the mailing list), Kinesis (developing in
> another repository), ActiveMQ (in two months). We may want to see how those
> go before we "kill" them.
>
> 2) Can have some Samza devs involved in Kafka's "transformer" client API.
> This can not only help the future integration (if any) much easier, because
> they have knowledge about both systems, but also good for Kafka's
> community, because Samza devs have the streaming process experience that
> Kafka devs may miss.
>
> 3) Samza's partition management system may still support other systems.
> Though the partition management logic in samza-kafka will be moved to
> Kafka, its still useful for other systems that do not have the partition
> management layer.
>
> 4) Start sharing the docs/websites and using the same terminology (though
> do not know how to do this exactly. :). This will reduce the future
> confusion and does not hurt Samza's independency.
>
> In my opinion, Samza, as a standalone project, still can (and already)
> heavily replying on Kafka, and even more tuned for Kafka-specific usage.
> Kafka, also can embed Samza in the document, I do not see anything prevent
> doing this.
>
> Thanks,
>
> Fang, Yan
> yanfang724@gmail.com
>
> On Mon, Jul 13, 2015 at 11:25 AM, Jordan Shaw <jo...@pubnub.com> wrote:
>
> > Jay,
> > I think doing this iteratively in smaller chunks is a better way to go as
> > new issues arise. As Navina said Kafka is a "stream system" and Samza is
> a
> > "stream processor" and those two ideas should be mutually exclusive.
> >
> > -Jordan
> >
> > On Mon, Jul 13, 2015 at 10:06 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hmm, thought about this more. Maybe this is just too much too quick.
> > > Overall I think there is some enthusiasm for the proposal but it's not
> > > really unanimous enough to make any kind of change this big cleanly.
> The
> > > board doesn't really like the merging stuff, user's are concerned about
> > > compatibility, I didn't feel there was unanimous agreement on dropping
> > > SystemConsumer, etc. Even if this is the right end state to get to,
> > > probably trying to push all this through at once isn't the right way to
> > do
> > > it.
> > >
> > > So let me propose a kind of fifth (?) option which I think is less
> > dramatic
> > > and let's things happen gradually. I think this is kind of like
> combining
> > > the first part of Yi's proposal and Jakob's third option, leaving the
> > rest
> > > to be figured out incrementally:
> > >
> > > Option 5: We continue the prototype I shared and propose that as a kind
> > of
> > > "transformer" client API in Kafka. This isn't really a full-fledged
> > stream
> > > processing layer, more like a supped up consumer api for munging
> topics.
> > > This would let us figure out some of the technical bits, how to do this
> > on
> > > Kafka's group management features, how to integrate the txn feature to
> do
> > > the exactly-once stuff in these transformations, and get all this stuff
> > > solid. This api would have valid uses in it's own right, especially
> when
> > > your transformation will be embedded inside an existing service or
> > > application which isn't possible with Samza (or other existing systems
> > that
> > > I know of).
> > >
> > > Independently we can iterate on some of the ideas of the original
> > proposal
> > > individually and figure out how (if at all) to make use of this
> > > functionality. This can be done bit-by-bit:
> > > - Could be that the existing StreamTask API ends up wrapping this
> > > - Could end up exposed directly in Samza as Yi proposed
> > > - Could be that just the lower-level group-management stuff get's used,
> > and
> > > in this case it could be either just for standalone mode, or always
> > > - Could be that it stays as-is
> > >
> > > The advantage of this is it is lower risk...we basically don't have to
> > make
> > > 12 major decisions all at once that kind of hinge on what amounts to a
> > > pretty aggressive rewrite. The disadvantage of this is it is a bit more
> > > confusing as all this is getting figured out.
> > >
> > > As with some of the other stuff, this would require a further
> discussion
> > in
> > > the Kafka community if people do like this approach.
> > >
> > > Thoughts?
> > >
> > > -Jay
> > >
> > >
> > >
> > >
> > > On Sun, Jul 12, 2015 at 10:52 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hey Chris,
> > > >
> > > > Yeah, I'm obviously in favor of this.
> > > >
> > > > The sub-project approach seems the ideal way to take a graceful step
> in
> > > > this direction, so I will ping the board folks and see why they are
> > > > discouraged, it would be good to understand that. If we go that route
> > we
> > > > would need to do a similar discussion in the Kafka list (but makes
> > sense
> > > to
> > > > figure out first if it is what Samza wants).
> > > >
> > > > Irrespective of how it's implemented, though, to me the important
> > things
> > > > are the following:
> > > > 1. Unify the website, config, naming, docs, metrics, etc--basically
> fix
> > > > the product experience so the "stream" and the "processing" feel
> like a
> > > > single user experience and brand. This seems minor but I think is a
> > > really
> > > > big deal.
> > > > 2. Make "standalone" mode a first class citizen and have a real
> > technical
> > > > plan to be able to support cluster managers other than YARN.
> > > > 3. Make the config and out-of-the-box experience more usable
> > > >
> > > > I think that prototype gives a practical example of how 1-3 could be
> > done
> > > > and we should pursue it. This is a pretty radical change, so I
> wouldn't
> > > be
> > > > shocked if people didn't want to take a step like that.
> > > >
> > > > Maybe it would make sense to see if people are on board with that
> > general
> > > > idea, and then try to get some advice on sub-projects in parallel and
> > > nail
> > > > down those details?
> > > >
> > > > -Jay
> > > >
> > > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
> > criccomini@apache.org>
> > > > wrote:
> > > >
> > > >> Hey all,
> > > >>
> > > >> I want to start by saying that I'm absolutely thrilled to be a part
> of
> > > >> this
> > > >> community. The amount of level-headed, thoughtful, educated
> discussion
> > > >> that's gone on over the past ~10 days is overwhelming. Wonderful.
> > > >>
> > > >> It seems like discussion is waning a bit, and we've reached some
> > > >> conclusions. There are several key emails in this threat, which I
> want
> > > to
> > > >> call out:
> > > >>
> > > >> 1. Jakob's summary of the three potential ways forward.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > > >> 2. Julian's call out that we should be focusing on community over
> > code.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > > >> 3. Martin's summary about the benefits of merging communities.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > > >> 4. Jakob's comments about the distinction between community and code
> > > >> paths.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> > > >>
> > > >> I agree with the comments on all of these emails. I think Martin's
> > > summary
> > > >> of his position aligns very closely with my own. To that end, I
> think
> > we
> > > >> should get concrete about what the proposal is, and call a vote on
> it.
> > > >> Given that Jay, Martin, and I seem to be aligning fairly closely, I
> > > think
> > > >> we should start with:
> > > >>
> > > >> 1. [community] Make Samza a subproject of Kafka.
> > > >> 2. [community] Make all Samza PMC/committers committers of the
> > > subproject.
> > > >> 3. [community] Migrate Samza's website/documentation into Kafka's.
> > > >> 4. [code] Have the Samza community and the Kafka community start a
> > > >> from-scratch reboot together in the new Kafka subproject. We can
> > > >> borrow/copy &  paste significant chunks of code from Samza's code
> > base.
> > > >> 5. [code] The subproject would intentionally eliminate support for
> > both
> > > >> other streaming systems and all deployment systems.
> > > >> 6. [code] Attempt to provide a bridge from our SystemConsumer to
> > KIP-26
> > > >> (copy cat)
> > > >> 7. [code] Attempt to provide a bridge from the new subproject's
> > > processor
> > > >> interface to our legacy StreamTask interface.
> > > >> 8. [code/community] Sunset Samza as a TLP when we have a working
> Kafka
> > > >> subproject that has a fault-tolerant container with state
> management.
> > > >>
> > > >> It's likely that (6) and (7) won't be fully drop-in. Still, the
> closer
> > > we
> > > >> can get, the better it's going to be for our existing community.
> > > >>
> > > >> One thing that I didn't touch on with (2) is whether any Samza PMC
> > > members
> > > >> should be rolled into Kafka PMC membership as well (though, Jay and
> > > Jakob
> > > >> are already PMC members on both). I think that Samza's community
> > > deserves
> > > >> a
> > > >> voice on the PMC, so I'd propose that we roll at least a few PMC
> > members
> > > >> into the Kafka PMC, but I don't have a strong framework for which
> > people
> > > >> to
> > > >> pick.
> > > >>
> > > >> Before (8), I think that Samza's TLP can continue to commit bug
> fixes
> > > and
> > > >> patches as it sees fit, provided that we openly communicate that we
> > > won't
> > > >> necessarily migrate new features to the new subproject, and that the
> > TLP
> > > >> will be shut down after the migration to the Kafka subproject
> occurs.
> > > >>
> > > >> Jakob, I could use your guidance here about about how to achieve
> this
> > > from
> > > >> an Apache process perspective (sorry).
> > > >>
> > > >> * Should I just call a vote on this proposal?
> > > >> * Should it happen on dev or private?
> > > >> * Do committers have binding votes, or just PMC?
> > > >>
> > > >> Having trouble finding much detail on the Apache wikis. :(
> > > >>
> > > >> Cheers,
> > > >> Chris
> > > >>
> > > >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com>
> > wrote:
> > > >>
> > > >> > Thanks, Jay. This argument persuaded me actually. :)
> > > >> >
> > > >> > Fang, Yan
> > > >> > yanfang724@gmail.com
> > > >> >
> > > >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io>
> > wrote:
> > > >> >
> > > >> > > Hey Yan,
> > > >> > >
> > > >> > > Yeah philosophically I think the argument is that you should
> > capture
> > > >> the
> > > >> > > stream in Kafka independent of the transformation. This is
> > > obviously a
> > > >> > > Kafka-centric view point.
> > > >> > >
> > > >> > > Advantages of this:
> > > >> > > - In practice I think this is what e.g. Storm people often end
> up
> > > >> doing
> > > >> > > anyway. You usually need to throttle any access to a live
> serving
> > > >> > database.
> > > >> > > - Can have multiple subscribers and they get the same thing
> > without
> > > >> > > additional load on the source system.
> > > >> > > - Applications can tap into the stream if need be by
> subscribing.
> > > >> > > - You can debug your transformation by tailing the Kafka topic
> > with
> > > >> the
> > > >> > > console consumer
> > > >> > > - Can tee off the same data stream for batch analysis or Lambda
> > arch
> > > >> > style
> > > >> > > re-processing
> > > >> > >
> > > >> > > The disadvantage is that it will use Kafka resources. But the
> idea
> > > is
> > > >> > > eventually you will have multiple subscribers to any data source
> > (at
> > > >> > least
> > > >> > > for monitoring) so you will end up there soon enough anyway.
> > > >> > >
> > > >> > > Down the road the technical benefit is that I think it gives us
> a
> > > good
> > > >> > path
> > > >> > > towards end-to-end exactly once semantics from source to
> > > destination.
> > > >> > > Basically the connectors need to support idempotence when
> talking
> > to
> > > >> > Kafka
> > > >> > > and we need the transactional write feature in Kafka to make the
> > > >> > > transformation atomic. This is actually pretty doable if you
> > > separate
> > > >> > > connector=>kafka problem from the generic transformations which
> > are
> > > >> > always
> > > >> > > kafka=>kafka. However I think it is quite impossible to do in a
> > > >> > all_things
> > > >> > > => all_things environment. Today you can say "well the semantics
> > of
> > > >> the
> > > >> > > Samza APIs depend on the connectors you use" but it is actually
> > > worse
> > > >> > then
> > > >> > > that because the semantics actually depend on the pairing of
> > > >> > connectors--so
> > > >> > > not only can you probably not get a usable "exactly once"
> > guarantee
> > > >> > > end-to-end it can actually be quite hard to reverse engineer
> what
> > > >> > property
> > > >> > > (if any) your end-to-end flow has if you have heterogenous
> > systems.
> > > >> > >
> > > >> > > -Jay
> > > >> > >
> > > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <yanfang724@gmail.com
> >
> > > >> wrote:
> > > >> > >
> > > >> > > > {quote}
> > > >> > > > maintained in a separate repository and retaining the existing
> > > >> > > > committership but sharing as much else as possible (website,
> > etc)
> > > >> > > > {quote}
> > > >> > > >
> > > >> > > > Overall, I agree on this idea. Now the question is more about
> > "how
> > > >> to
> > > >> > do
> > > >> > > > it".
> > > >> > > >
> > > >> > > > On the other hand, one thing I want to point out is that, if
> we
> > > >> decide
> > > >> > to
> > > >> > > > go this way, how do we want to support
> > > >> > > > otherSystem-transformation-otherSystem use case?
> > > >> > > >
> > > >> > > > Basically, there are four user groups here:
> > > >> > > >
> > > >> > > > 1. Kafka-transformation-Kafka
> > > >> > > > 2. Kafka-transformation-otherSystem
> > > >> > > > 3. otherSystem-transformation-Kafka
> > > >> > > > 4. otherSystem-transformation-otherSystem
> > > >> > > >
> > > >> > > > For group 1, they can easily use the new Samza library to
> > achieve.
> > > >> For
> > > >> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka
> > or
> > > >> > Kafka->
> > > >> > > > transformation -> copyCat.
> > > >> > > >
> > > >> > > > The problem is for group 4. Do we want to abandon this or
> still
> > > >> support
> > > >> > > it?
> > > >> > > > Of course, this use case can be achieved by using copyCat ->
> > > >> > > transformation
> > > >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> > > persuade
> > > >> > them
> > > >> > > to
> > > >> > > > do this long chain. If yes, it will also be a win for Kafka
> too.
> > > Or
> > > >> if
> > > >> > > > there is no one in this community actually doing this so far,
> > > maybe
> > > >> ok
> > > >> > to
> > > >> > > > not support the group 4 directly.
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > >
> > > >> > > > Fang, Yan
> > > >> > > > yanfang724@gmail.com
> > > >> > > >
> > > >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <jay@confluent.io
> >
> > > >> wrote:
> > > >> > > >
> > > >> > > > > Yeah I agree with this summary. I think there are kind of
> two
> > > >> > questions
> > > >> > > > > here:
> > > >> > > > > 1. Technically does alignment/reliance on Kafka make sense
> > > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
> > alignment
> > > >> with
> > > >> > > > Kafka
> > > >> > > > > make sense
> > > >> > > > >
> > > >> > > > > Personally I do think both of these things would be really
> > > >> valuable,
> > > >> > > and
> > > >> > > > > would dramatically alter the trajectory of the project.
> > > >> > > > >
> > > >> > > > > My preference would be to see if people can mostly agree on
> a
> > > >> > direction
> > > >> > > > > rather than splintering things off. From my point of view
> the
> > > >> ideal
> > > >> > > > outcome
> > > >> > > > > of all the options discussed would be to make Samza a
> closely
> > > >> aligned
> > > >> > > > > subproject, maintained in a separate repository and
> retaining
> > > the
> > > >> > > > existing
> > > >> > > > > committership but sharing as much else as possible (website,
> > > >> etc). No
> > > >> > > > idea
> > > >> > > > > about how these things work, Jacob, you probably know more.
> > > >> > > > >
> > > >> > > > > No discussion amongst the Kafka folks has happened on this,
> > but
> > > >> > likely
> > > >> > > we
> > > >> > > > > should figure out what the Samza community actually wants
> > first.
> > > >> > > > >
> > > >> > > > > I admit that this is a fairly radical departure from how
> > things
> > > >> are.
> > > >> > > > >
> > > >> > > > > If that doesn't fly, I think, yeah we could leave Samza as
> it
> > is
> > > >> and
> > > >> > do
> > > >> > > > the
> > > >> > > > > more radical reboot inside Kafka. From my point of view that
> > > does
> > > >> > leave
> > > >> > > > > things in a somewhat confusing state since now there are two
> > > >> stream
> > > >> > > > > processing systems more or less coupled to Kafka in large
> part
> > > >> made
> > > >> > by
> > > >> > > > the
> > > >> > > > > same people. But, arguably that might be a cleaner way to
> make
> > > the
> > > >> > > > cut-over
> > > >> > > > > and perhaps less risky for Samza community since if it works
> > > >> people
> > > >> > can
> > > >> > > > > switch and if it doesn't nothing will have changed. Dunno,
> how
> > > do
> > > >> > > people
> > > >> > > > > feel about this?
> > > >> > > > >
> > > >> > > > > -Jay
> > > >> > > > >
> > > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> > > jghoman@gmail.com>
> > > >> > > wrote:
> > > >> > > > >
> > > >> > > > > > >  This leads me to thinking that merging projects and
> > > >> communities
> > > >> > > > might
> > > >> > > > > > be a good idea: with the union of experience from both
> > > >> communities,
> > > >> > > we
> > > >> > > > > will
> > > >> > > > > > probably build a better system that is better for users.
> > > >> > > > > > Is this what's being proposed though? Merging the projects
> > > seems
> > > >> > like
> > > >> > > > > > a consequence of at most one of the three directions under
> > > >> > > discussion:
> > > >> > > > > > 1) Samza 2.0: The Samza community relies more heavily on
> > Kafka
> > > >> for
> > > >> > > > > > configuration, etc. (to a greater or lesser extent to be
> > > >> > determined)
> > > >> > > > > > but the Samza community would not automatically merge
> withe
> > > >> Kafka
> > > >> > > > > > community (the Phoenix/HBase example is a good one here).
> > > >> > > > > > 2) Samza Reboot: The Samza community continues to exist
> > with a
> > > >> > > limited
> > > >> > > > > > project scope, but similarly would not need to be part of
> > the
> > > >> Kafka
> > > >> > > > > > community (ie given committership) to progress.  Here,
> maybe
> > > the
> > > >> > > Samza
> > > >> > > > > > team would become a subproject of Kafka (the Board frowns
> on
> > > >> > > > > > subprojects at the moment, so I'm not sure if that's even
> > > >> > feasible),
> > > >> > > > > > but that would not be required.
> > > >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this
> option
> > > the
> > > >> > Kafka
> > > >> > > > > > team builds its own streaming library, possibly off of
> Jay's
> > > >> > > > > > prototype, which has not direct lineage to the Samza team.
> > > >> There's
> > > >> > > no
> > > >> > > > > > reason for the Kafka team to bring in the Samza team.
> > > >> > > > > >
> > > >> > > > > > Is the Kafka community on board with this?
> > > >> > > > > >
> > > >> > > > > > To be clear, all three options under discussion are
> > > interesting,
> > > >> > > > > > technically valid and likely healthy directions for the
> > > project.
> > > >> > > > > > Also, they are not mutually exclusive.  The Samza
> community
> > > >> could
> > > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
> > community
> > > >> went
> > > >> > > > > > forward with 'Hey Samza!'  My points above are directed
> > > >> entirely at
> > > >> > > > > > the community aspect of these choices.
> > > >> > > > > > -Jakob
> > > >> > > > > >
> > > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> > > roger.hoover@gmail.com>
> > > >> > > wrote:
> > > >> > > > > > > That's great.  Thanks, Jay.
> > > >> > > > > > >
> > > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> > > jay@confluent.io>
> > > >> > > wrote:
> > > >> > > > > > >
> > > >> > > > > > >> Yeah totally agree. I think you have this issue even
> > today,
> > > >> > right?
> > > >> > > > > I.e.
> > > >> > > > > > if
> > > >> > > > > > >> you need to make a simple config change and you're
> > running
> > > in
> > > >> > YARN
> > > >> > > > > today
> > > >> > > > > > >> you end up bouncing the job which then rebuilds state.
> I
> > > >> think
> > > >> > the
> > > >> > > > fix
> > > >> > > > > > is
> > > >> > > > > > >> exactly what you described which is to have a long
> > timeout
> > > on
> > > >> > > > > partition
> > > >> > > > > > >> movement for stateful jobs so that if a job is just
> > getting
> > > >> > > bounced,
> > > >> > > > > and
> > > >> > > > > > >> the cluster manager (or admin) is smart enough to
> restart
> > > it
> > > >> on
> > > >> > > the
> > > >> > > > > same
> > > >> > > > > > >> host when possible, it can optimistically reuse any
> > > existing
> > > >> > state
> > > >> > > > it
> > > >> > > > > > finds
> > > >> > > > > > >> on disk (if it is valid).
> > > >> > > > > > >>
> > > >> > > > > > >> So in this model the charter of the CM is to place
> > > processes
> > > >> as
> > > >> > > > > > stickily as
> > > >> > > > > > >> possible and to restart or re-place failed processes.
> The
> > > >> > charter
> > > >> > > of
> > > >> > > > > the
> > > >> > > > > > >> partition management system is to control the
> assignment
> > of
> > > >> work
> > > >> > > to
> > > >> > > > > > these
> > > >> > > > > > >> processes. The nice thing about this is that the work
> > > >> > assignment,
> > > >> > > > > > timeouts,
> > > >> > > > > > >> behavior, configs, and code will all be the same across
> > all
> > > >> > > cluster
> > > >> > > > > > >> managers.
> > > >> > > > > > >>
> > > >> > > > > > >> So I think that prototype would actually give you
> exactly
> > > >> what
> > > >> > you
> > > >> > > > > want
> > > >> > > > > > >> today for any cluster manager (or manual placement +
> > > restart
> > > >> > > script)
> > > >> > > > > > that
> > > >> > > > > > >> was sticky in terms of host placement since there is
> > > already
> > > >> a
> > > >> > > > > > configurable
> > > >> > > > > > >> partition movement timeout and task-by-task state reuse
> > > with
> > > >> a
> > > >> > > check
> > > >> > > > > on
> > > >> > > > > > >> state validity.
> > > >> > > > > > >>
> > > >> > > > > > >> -Jay
> > > >> > > > > > >>
> > > >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > > >> > > > roger.hoover@gmail.com
> > > >> > > > > >
> > > >> > > > > > >> wrote:
> > > >> > > > > > >>
> > > >> > > > > > >> > That would be great to let Kafka do as much heavy
> > lifting
> > > >> as
> > > >> > > > > possible
> > > >> > > > > > and
> > > >> > > > > > >> > make it easier for other languages to implement Samza
> > > apis.
> > > >> > > > > > >> >
> > > >> > > > > > >> > One thing to watch out for is the interplay between
> > > Kafka's
> > > >> > > group
> > > >> > > > > > >> > management and the external scheduler/process
> manager's
> > > >> fault
> > > >> > > > > > tolerance.
> > > >> > > > > > >> > If a container dies, the Kafka group membership
> > protocol
> > > >> will
> > > >> > > try
> > > >> > > > to
> > > >> > > > > > >> assign
> > > >> > > > > > >> > it's tasks to other containers while at the same time
> > the
> > > >> > > process
> > > >> > > > > > manager
> > > >> > > > > > >> > is trying to relaunch the container.  Without some
> > > >> > consideration
> > > >> > > > for
> > > >> > > > > > this
> > > >> > > > > > >> > (like a configurable amount of time to wait before
> > Kafka
> > > >> > alters
> > > >> > > > the
> > > >> > > > > > group
> > > >> > > > > > >> > membership), there may be thrashing going on which is
> > > >> > especially
> > > >> > > > bad
> > > >> > > > > > for
> > > >> > > > > > >> > containers with large amounts of local state.
> > > >> > > > > > >> >
> > > >> > > > > > >> > Someone else pointed this out already but I thought
> it
> > > >> might
> > > >> > be
> > > >> > > > > worth
> > > >> > > > > > >> > calling out again.
> > > >> > > > > > >> >
> > > >> > > > > > >> > Cheers,
> > > >> > > > > > >> >
> > > >> > > > > > >> > Roger
> > > >> > > > > > >> >
> > > >> > > > > > >> >
> > > >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> > > >> jay@confluent.io>
> > > >> > > > > wrote:
> > > >> > > > > > >> >
> > > >> > > > > > >> > > Hey Roger,
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > I couldn't agree more. We spent a bunch of time
> > talking
> > > >> to
> > > >> > > > people
> > > >> > > > > > and
> > > >> > > > > > >> > that
> > > >> > > > > > >> > > is exactly the stuff we heard time and again. What
> > > makes
> > > >> it
> > > >> > > > hard,
> > > >> > > > > of
> > > >> > > > > > >> > > course, is that there is some tension between
> > > >> compatibility
> > > >> > > with
> > > >> > > > > > what's
> > > >> > > > > > >> > > there now and making things better for new users.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > I also strongly agree with the importance of
> > > >> multi-language
> > > >> > > > > > support. We
> > > >> > > > > > >> > are
> > > >> > > > > > >> > > talking now about Java, but for application
> > development
> > > >> use
> > > >> > > > cases
> > > >> > > > > > >> people
> > > >> > > > > > >> > > want to work in whatever language they are using
> > > >> elsewhere.
> > > >> > I
> > > >> > > > > think
> > > >> > > > > > >> > moving
> > > >> > > > > > >> > > to a model where Kafka itself does the group
> > > membership,
> > > >> > > > lifecycle
> > > >> > > > > > >> > control,
> > > >> > > > > > >> > > and partition assignment has the advantage of
> putting
> > > all
> > > >> > that
> > > >> > > > > > complex
> > > >> > > > > > >> > > stuff behind a clean api that the clients are
> already
> > > >> going
> > > >> > to
> > > >> > > > be
> > > >> > > > > > >> > > implementing for their consumer, so the added
> > > >> functionality
> > > >> > > for
> > > >> > > > > > stream
> > > >> > > > > > >> > > processing beyond a consumer becomes very minor.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > -Jay
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > > >> > > > > > roger.hoover@gmail.com>
> > > >> > > > > > >> > > wrote:
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > > Metamorphosis...nice. :)
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > This has been a great discussion.  As a user of
> > Samza
> > > >> > who's
> > > >> > > > > > recently
> > > >> > > > > > >> > > > integrated it into a relatively large
> > organization, I
> > > >> just
> > > >> > > > want
> > > >> > > > > to
> > > >> > > > > > >> add
> > > >> > > > > > >> > > > support to a few points already made.
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
> > > >> currently
> > > >> > > > exists
> > > >> > > > > > that
> > > >> > > > > > >> > I've
> > > >> > > > > > >> > > > experienced are:
> > > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
> > environments
> > > >> > where
> > > >> > > > > Puppet
> > > >> > > > > > >> > would
> > > >> > > > > > >> > > do
> > > >> > > > > > >> > > > just fine but it was the only mechanism to get
> > fault
> > > >> > > > tolerance.
> > > >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> > > >> configuring
> > > >> > > most
> > > >> > > > > of
> > > >> > > > > > the
> > > >> > > > > > >> > job
> > > >> > > > > > >> > > > in code rather than config files.  In general, I
> > > think
> > > >> the
> > > >> > > > goal
> > > >> > > > > > >> should
> > > >> > > > > > >> > be
> > > >> > > > > > >> > > > to make it harder to make mistakes, especially of
> > the
> > > >> kind
> > > >> > > > where
> > > >> > > > > > the
> > > >> > > > > > >> > code
> > > >> > > > > > >> > > > expects something and the config doesn't match.
> > The
> > > >> > current
> > > >> > > > > > config
> > > >> > > > > > >> is
> > > >> > > > > > >> > > > quite intricate and error-prone.  For example,
> the
> > > >> > > application
> > > >> > > > > > logic
> > > >> > > > > > >> > may
> > > >> > > > > > >> > > > depend on bootstrapping a topic but rather than
> > > >> asserting
> > > >> > > that
> > > >> > > > > in
> > > >> > > > > > the
> > > >> > > > > > >> > > code,
> > > >> > > > > > >> > > > you have to rely on getting the config right.
> > > Likewise
> > > >> > with
> > > >> > > > > > serdes,
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > Java representations produced by various serdes
> > > (JSON,
> > > >> > Avro,
> > > >> > > > > etc.)
> > > >> > > > > > >> are
> > > >> > > > > > >> > > not
> > > >> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> > > >> without
> > > >> > > > > changing
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > code.   It would be nice for jobs to be able to
> > > assert
> > > >> > what
> > > >> > > > they
> > > >> > > > > > >> expect
> > > >> > > > > > >> > > > from their input topics in terms of partitioning.
> > > >> This is
> > > >> > > > > > getting a
> > > >> > > > > > >> > > little
> > > >> > > > > > >> > > > off topic but I was even thinking about creating
> a
> > > >> "Samza
> > > >> > > > config
> > > >> > > > > > >> > linter"
> > > >> > > > > > >> > > > that would sanity check a set of configs.
> > Especially
> > > >> in
> > > >> > > > > > >> organizations
> > > >> > > > > > >> > > > where config is managed by a different team than
> > the
> > > >> > > > application
> > > >> > > > > > >> > > developer,
> > > >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > > >> > > > > > >> > > > 3) Java/Scala centric - for many teams
> (especially
> > > >> > > DevOps-type
> > > >> > > > > > >> folks),
> > > >> > > > > > >> > > the
> > > >> > > > > > >> > > > pain of the Java toolchain (maven, slow builds,
> > weak
> > > >> > command
> > > >> > > > > line
> > > >> > > > > > >> > > support,
> > > >> > > > > > >> > > > configuration over convention) really inhibits
> > > >> > productivity.
> > > >> > > > As
> > > >> > > > > > more
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > more high-quality clients become available for
> > > Kafka, I
> > > >> > hope
> > > >> > > > > > they'll
> > > >> > > > > > >> > > follow
> > > >> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> > > >> proposals
> > > >> > > in
> > > >> > > > > this
> > > >> > > > > > >> > thread
> > > >> > > > > > >> > > > but please consider other languages in the
> > ecosystem
> > > as
> > > >> > > well.
> > > >> > > > > > From
> > > >> > > > > > >> > what
> > > >> > > > > > >> > > > I've heard, Spark has more Python users than
> > > >> Java/Scala.
> > > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > >
> > > >> > > > > > >> >
> > > >> > > > > > >>
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > >> > > > > > >> > > > and are working on a Yeoman generator
> > > >> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> > > >> > > Jython/Samza
> > > >> > > > > > >> projects
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > alleviate some of the pain)
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > I also want to underscore Jay's point about
> > improving
> > > >> the
> > > >> > > user
> > > >> > > > > > >> > > experience.
> > > >> > > > > > >> > > > That's a very important factor for adoption.  I
> > think
> > > >> the
> > > >> > > goal
> > > >> > > > > > should
> > > >> > > > > > >> > be
> > > >> > > > > > >> > > to
> > > >> > > > > > >> > > > make Samza as easy to get started with as
> something
> > > >> like
> > > >> > > > > Logstash.
> > > >> > > > > > >> > > > Logstash is vastly inferior in terms of
> > capabilities
> > > to
> > > >> > > Samza
> > > >> > > > > but
> > > >> > > > > > >> it's
> > > >> > > > > > >> > > easy
> > > >> > > > > > >> > > > to get started and that makes a big difference.
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > Cheers,
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > Roger
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> > > Francisci
> > > >> > > > Morales <
> > > >> > > > > > >> > > > gdfm@apache.org> wrote:
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> > > >> Metamorphosis
> > > >> > > is
> > > >> > > > a
> > > >> > > > > > clear
> > > >> > > > > > >> > > > winner
> > > >> > > > > > >> > > > > :)
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > --
> > > >> > > > > > >> > > > > Gianmarco
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> > > >> Morales
> > > >> > <
> > > >> > > > > > >> > > gdfm@apache.org
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > wrote:
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > > Hi,
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > @Martin, thanks for you comments.
> > > >> > > > > > >> > > > > > Maybe I'm missing some important point, but I
> > > think
> > > >> > > > coupling
> > > >> > > > > > the
> > > >> > > > > > >> > > > releases
> > > >> > > > > > >> > > > > > is actually a *good* thing.
> > > >> > > > > > >> > > > > > To make an example, would it be better if the
> > MR
> > > >> and
> > > >> > > HDFS
> > > >> > > > > > >> > components
> > > >> > > > > > >> > > of
> > > >> > > > > > >> > > > > > Hadoop had different release schedules?
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > Actually, keeping the discussion in a single
> > > place
> > > >> > would
> > > >> > > > > make
> > > >> > > > > > >> > > agreeing
> > > >> > > > > > >> > > > on
> > > >> > > > > > >> > > > > > releases (and backwards compatibility) much
> > > >> easier, as
> > > >> > > > > > everybody
> > > >> > > > > > >> > > would
> > > >> > > > > > >> > > > be
> > > >> > > > > > >> > > > > > responsible for the whole codebase.
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > That said, I like the idea of absorbing
> > > samza-core
> > > >> as
> > > >> > a
> > > >> > > > > > >> > sub-project,
> > > >> > > > > > >> > > > and
> > > >> > > > > > >> > > > > > leave the fancy stuff separate.
> > > >> > > > > > >> > > > > > It probably gives 90% of the benefits we have
> > > been
> > > >> > > > > discussing
> > > >> > > > > > >> here.
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > Cheers,
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > --
> > > >> > > > > > >> > > > > > Gianmarco
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > > >> > jay.kreps@gmail.com
> > > >> > > >
> > > >> > > > > > wrote:
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > >> Hey Martin,
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> I agree coupling release schedules is a
> > > downside.
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> Definitely we can try to solve some of the
> > > >> > integration
> > > >> > > > > > problems
> > > >> > > > > > >> in
> > > >> > > > > > >> > > > > >> Confluent Platform or in other
> distributions.
> > > But
> > > >> I
> > > >> > > think
> > > >> > > > > > this
> > > >> > > > > > >> > ends
> > > >> > > > > > >> > > up
> > > >> > > > > > >> > > > > >> being really shallow. I guess I feel to
> really
> > > >> get a
> > > >> > > good
> > > >> > > > > > user
> > > >> > > > > > >> > > > > experience
> > > >> > > > > > >> > > > > >> the two systems have to kind of feel like
> part
> > > of
> > > >> the
> > > >> > > > same
> > > >> > > > > > thing
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> can't really add that in later--you can put
> > both
> > > >> in
> > > >> > the
> > > >> > > > > same
> > > >> > > > > > >> > > > > downloadable
> > > >> > > > > > >> > > > > >> tar file but it doesn't really give a very
> > > >> cohesive
> > > >> > > > > feeling.
> > > >> > > > > > I
> > > >> > > > > > >> > agree
> > > >> > > > > > >> > > > > that
> > > >> > > > > > >> > > > > >> ultimately any of the project stuff is as
> much
> > > >> social
> > > >> > > and
> > > >> > > > > > naming
> > > >> > > > > > >> > as
> > > >> > > > > > >> > > > > >> anything else--theoretically two totally
> > > >> independent
> > > >> > > > > projects
> > > >> > > > > > >> > could
> > > >> > > > > > >> > > > work
> > > >> > > > > > >> > > > > >> to
> > > >> > > > > > >> > > > > >> tightly align. In practice this seems to be
> > > quite
> > > >> > > > difficult
> > > >> > > > > > >> > though.
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> For the frameworks--totally agree it would
> be
> > > >> good to
> > > >> > > > > > maintain
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > > >> framework support with the project. In some
> > > cases
> > > >> > there
> > > >> > > > may
> > > >> > > > > > not
> > > >> > > > > > >> be
> > > >> > > > > > >> > > too
> > > >> > > > > > >> > > > > >> much
> > > >> > > > > > >> > > > > >> there since the integration gets lighter
> but I
> > > >> think
> > > >> > > > > whatever
> > > >> > > > > > >> > stubs
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> need should be included. So no I definitely
> > > wasn't
> > > >> > > trying
> > > >> > > > > to
> > > >> > > > > > >> imply
> > > >> > > > > > >> > > > > >> dropping
> > > >> > > > > > >> > > > > >> support for these frameworks, just making
> the
> > > >> > > integration
> > > >> > > > > > >> lighter
> > > >> > > > > > >> > by
> > > >> > > > > > >> > > > > >> separating process management from partition
> > > >> > > management.
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> You raise two good points we would have to
> > > figure
> > > >> out
> > > >> > > if
> > > >> > > > we
> > > >> > > > > > went
> > > >> > > > > > >> > > down
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> alignment path:
> > > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think
> the
> > > >> first
> > > >> > > > > question
> > > >> > > > > > is
> > > >> > > > > > >> > > > whether
> > > >> > > > > > >> > > > > >> some "re-branding" would be worth it. If so
> > > then I
> > > >> > > think
> > > >> > > > we
> > > >> > > > > > can
> > > >> > > > > > >> > > have a
> > > >> > > > > > >> > > > > big
> > > >> > > > > > >> > > > > >> thread on the name. I'm definitely not set
> on
> > > >> Kafka
> > > >> > > > > > Streaming or
> > > >> > > > > > >> > > Kafka
> > > >> > > > > > >> > > > > >> Streams I was just using them to be kind of
> > > >> > > > illustrative. I
> > > >> > > > > > >> agree
> > > >> > > > > > >> > > with
> > > >> > > > > > >> > > > > >> your
> > > >> > > > > > >> > > > > >> critique of these names, though I think
> people
> > > >> would
> > > >> > > get
> > > >> > > > > the
> > > >> > > > > > >> idea.
> > > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about
> how
> > to
> > > >> > > "factor"
> > > >> > > > > it.
> > > >> > > > > > >> Here
> > > >> > > > > > >> > > are
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> options I see (I could get enthusiastic
> about
> > > any
> > > >> of
> > > >> > > > them):
> > > >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> > > seperation
> > > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api
> > and
> > > >> > > > samza-core
> > > >> > > > > > is
> > > >> > > > > > >> > > > absorbed
> > > >> > > > > > >> > > > > >> almost like a third client
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> Cheers,
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> -Jay
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> > > Kleppmann <
> > > >> > > > > > >> > > > martin@kleppmann.com>
> > > >> > > > > > >> > > > > >> wrote:
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a
> > few
> > > >> > > follow-up
> > > >> > > > > > >> > comments.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka
> or
> > > >> > becoming
> > > >> > > a
> > > >> > > > > > >> > subproject:
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > reasons you mention are good. The risk I
> see
> > > is
> > > >> > that
> > > >> > > > > > release
> > > >> > > > > > >> > > > schedules
> > > >> > > > > > >> > > > > >> > become coupled to each other, which can
> slow
> > > >> > everyone
> > > >> > > > > down,
> > > >> > > > > > >> and
> > > >> > > > > > >> > > > large
> > > >> > > > > > >> > > > > >> > projects with many contributors are harder
> > to
> > > >> > manage.
> > > >> > > > > > (Jakob,
> > > >> > > > > > >> > can
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> speak
> > > >> > > > > > >> > > > > >> > from experience, having seen a wider range
> > of
> > > >> > Hadoop
> > > >> > > > > > ecosystem
> > > >> > > > > > >> > > > > >> projects?)
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > Some of the goals of a better unified
> > > developer
> > > >> > > > > experience
> > > >> > > > > > >> could
> > > >> > > > > > >> > > > also
> > > >> > > > > > >> > > > > be
> > > >> > > > > > >> > > > > >> > solved by integrating Samza nicely into a
> > > Kafka
> > > >> > > > > > distribution
> > > >> > > > > > >> > (such
> > > >> > > > > > >> > > > as
> > > >> > > > > > >> > > > > >> > Confluent's). I'm not against merging
> > projects
> > > >> if
> > > >> > we
> > > >> > > > > decide
> > > >> > > > > > >> > that's
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> way
> > > >> > > > > > >> > > > > >> > to go, just pointing out the same goals
> can
> > > >> perhaps
> > > >> > > > also
> > > >> > > > > be
> > > >> > > > > > >> > > achieved
> > > >> > > > > > >> > > > > in
> > > >> > > > > > >> > > > > >> > other ways.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
> > dependency:
> > > >> are
> > > >> > > you
> > > >> > > > > > >> proposing
> > > >> > > > > > >> > > > that
> > > >> > > > > > >> > > > > >> > Samza doesn't give any help to people
> > wanting
> > > to
> > > >> > run
> > > >> > > on
> > > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > > >> > > > > > >> > > > > >> > So the docs would basically have a link to
> > > >> Slider
> > > >> > and
> > > >> > > > > > nothing
> > > >> > > > > > >> > > else?
> > > >> > > > > > >> > > > Or
> > > >> > > > > > >> > > > > >> > would we maintain integrations with a
> bunch
> > of
> > > >> > > popular
> > > >> > > > > > >> > deployment
> > > >> > > > > > >> > > > > >> methods
> > > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts
> > to
> > > >> make
> > > >> > > > Samza
> > > >> > > > > > work
> > > >> > > > > > >> > with
> > > >> > > > > > >> > > > > >> Slider)?
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > I absolutely think it's a good idea to
> have
> > > the
> > > >> > "as a
> > > >> > > > > > library"
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > "as a
> > > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
> > > >> people
> > > >> > who
> > > >> > > > > want
> > > >> > > > > > >> them,
> > > >> > > > > > >> > > > but I
> > > >> > > > > > >> > > > > >> > think there should also be a low-friction
> > path
> > > >> for
> > > >> > > > common
> > > >> > > > > > "as
> > > >> > > > > > >> a
> > > >> > > > > > >> > > > > service"
> > > >> > > > > > >> > > > > >> > deployment methods, for which we probably
> > need
> > > >> to
> > > >> > > > > maintain
> > > >> > > > > > >> > > > > integrations.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems
> odd
> > to
> > > >> me,
> > > >> > > > > because
> > > >> > > > > > >> Kafka
> > > >> > > > > > >> > > is
> > > >> > > > > > >> > > > > all
> > > >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> > > >> Transformers"
> > > >> > > or
> > > >> > > > > > "Kafka
> > > >> > > > > > >> > > > Filters"
> > > >> > > > > > >> > > > > >> > would be more apt?
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
> > > >> (stream
> > > >> > > > > > >> transformation
> > > >> > > > > > >> > > > with
> > > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> > > >> library"
> > > >> > > bit)
> > > >> > > > > > could
> > > >> > > > > > >> > > become
> > > >> > > > > > >> > > > > >> part of
> > > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
> > > >> streaming
> > > >> > SQL
> > > >> > > > and
> > > >> > > > > > >> > > > integrations
> > > >> > > > > > >> > > > > >> with
> > > >> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> > > >> project?
> > > >> > > In
> > > >> > > > > > other
> > > >> > > > > > >> > > words,
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > would absorb the proven, stable core of
> > Samza,
> > > >> > which
> > > >> > > > > would
> > > >> > > > > > >> > become
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in
> this
> > > >> > thread.
> > > >> > > > The
> > > >> > > > > > Samza
> > > >> > > > > > >> > > > project
> > > >> > > > > > >> > > > > >> > would then target that third Kafka client
> as
> > > its
> > > >> > base
> > > >> > > > > API,
> > > >> > > > > > and
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > > >> project
> > > >> > > > > > >> > > > > >> > would be freed up to explore more
> > experimental
> > > >> new
> > > >> > > > > > horizons.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > Martin
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > > >> > > > jay.kreps@gmail.com>
> > > >> > > > > > >> wrote:
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > > Hey Martin,
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I
> > actually
> > > >> > don't
> > > >> > > > > think
> > > >> > > > > > it
> > > >> > > > > > >> > ties
> > > >> > > > > > >> > > > our
> > > >> > > > > > >> > > > > >> > hands
> > > >> > > > > > >> > > > > >> > > at all, all it does is refactor things.
> > The
> > > >> > > division
> > > >> > > > of
> > > >> > > > > > >> > > > > >> responsibility is
> > > >> > > > > > >> > > > > >> > > that Samza core is responsible for task
> > > >> > lifecycle,
> > > >> > > > > state,
> > > >> > > > > > >> and
> > > >> > > > > > >> > > > > >> partition
> > > >> > > > > > >> > > > > >> > > management (using the Kafka
> co-ordinator)
> > > but
> > > >> it
> > > >> > is
> > > >> > > > NOT
> > > >> > > > > > >> > > > responsible
> > > >> > > > > > >> > > > > >> for
> > > >> > > > > > >> > > > > >> > > packaging, configuration deployment or
> > > >> execution
> > > >> > of
> > > >> > > > > > >> processes.
> > > >> > > > > > >> > > The
> > > >> > > > > > >> > > > > >> > problem
> > > >> > > > > > >> > > > > >> > > of packaging and starting these
> processes
> > is
> > > >> > > > > > >> > > > > >> > > framework/environment-specific. This
> > leaves
> > > >> > > > individual
> > > >> > > > > > >> > > frameworks
> > > >> > > > > > >> > > > to
> > > >> > > > > > >> > > > > >> be
> > > >> > > > > > >> > > > > >> > as
> > > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you
> can
> > > get
> > > >> > > simple
> > > >> > > > > > >> stateless
> > > >> > > > > > >> > > > > >> support in
> > > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their
> off-the-shelf
> > > app
> > > >> > > > > framework
> > > >> > > > > > >> > > (Slider,
> > > >> > > > > > >> > > > > >> > Marathon,
> > > >> > > > > > >> > > > > >> > > etc). These are well known by people and
> > > have
> > > >> > nice
> > > >> > > > UIs
> > > >> > > > > > and a
> > > >> > > > > > >> > lot
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > > flexibility. I don't think they have
> node
> > > >> > affinity
> > > >> > > > as a
> > > >> > > > > > >> built
> > > >> > > > > > >> > in
> > > >> > > > > > >> > > > > >> option
> > > >> > > > > > >> > > > > >> > > (though I could be wrong). So if we want
> > > that
> > > >> we
> > > >> > > can
> > > >> > > > > > either
> > > >> > > > > > >> > wait
> > > >> > > > > > >> > > > for
> > > >> > > > > > >> > > > > >> them
> > > >> > > > > > >> > > > > >> > > to add it or do a custom framework to
> add
> > > that
> > > >> > > > feature
> > > >> > > > > > (as
> > > >> > > > > > >> > now).
> > > >> > > > > > >> > > > > >> > Obviously
> > > >> > > > > > >> > > > > >> > > if you manage things with old-school ops
> > > tools
> > > >> > > > > > >> > (puppet/chef/etc)
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> get
> > > >> > > > > > >> > > > > >> > > locality easily. The nice thing, though,
> > is
> > > >> that
> > > >> > > all
> > > >> > > > > the
> > > >> > > > > > >> samza
> > > >> > > > > > >> > > > > >> "business
> > > >> > > > > > >> > > > > >> > > logic" around partition management and
> > fault
> > > >> > > > tolerance
> > > >> > > > > > is in
> > > >> > > > > > >> > > Samza
> > > >> > > > > > >> > > > > >> core
> > > >> > > > > > >> > > > > >> > so
> > > >> > > > > > >> > > > > >> > > it is shared across frameworks and the
> > > >> framework
> > > >> > > > > specific
> > > >> > > > > > >> bit
> > > >> > > > > > >> > is
> > > >> > > > > > >> > > > > just
> > > >> > > > > > >> > > > > >> > > whether it is smart enough to try to get
> > the
> > > >> same
> > > >> > > > host
> > > >> > > > > > when
> > > >> > > > > > >> a
> > > >> > > > > > >> > > job
> > > >> > > > > > >> > > > is
> > > >> > > > > > >> > > > > >> > > restarted.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment,
> yeah
> > I
> > > >> think
> > > >> > > the
> > > >> > > > > > goal
> > > >> > > > > > >> > would
> > > >> > > > > > >> > > > be
> > > >> > > > > > >> > > > > >> (a)
> > > >> > > > > > >> > > > > >> > > actually get better alignment in user
> > > >> experience,
> > > >> > > and
> > > >> > > > > (b)
> > > >> > > > > > >> > > express
> > > >> > > > > > >> > > > > >> this in
> > > >> > > > > > >> > > > > >> > > the naming and project branding.
> > > Specifically:
> > > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for
> the
> > > >> > > > > > "transformation"
> > > >> > > > > > >> api
> > > >> > > > > > >> > > to
> > > >> > > > > > >> > > > be
> > > >> > > > > > >> > > > > >> > > discoverable in the main Kafka
> docs--i.e.
> > be
> > > >> able
> > > >> > > to
> > > >> > > > > > explain
> > > >> > > > > > >> > > when
> > > >> > > > > > >> > > > to
> > > >> > > > > > >> > > > > >> use
> > > >> > > > > > >> > > > > >> > > the consumer and when to use the stream
> > > >> > processing
> > > >> > > > > > >> > functionality
> > > >> > > > > > >> > > > and
> > > >> > > > > > >> > > > > >> lead
> > > >> > > > > > >> > > > > >> > > people into that experience.
> > > >> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza
> > 1.4.2
> > > >> (or
> > > >> > > > > > whatever)
> > > >> > > > > > >> > that
> > > >> > > > > > >> > > > has
> > > >> > > > > > >> > > > > >> both
> > > >> > > > > > >> > > > > >> > > Kafka and the stream processing part and
> > > they
> > > >> > > > actually
> > > >> > > > > > work
> > > >> > > > > > >> > > > > together.
> > > >> > > > > > >> > > > > >> > > 3. Unify the programming experience so
> the
> > > >> client
> > > >> > > and
> > > >> > > > > > Samza
> > > >> > > > > > >> > api
> > > >> > > > > > >> > > > > share
> > > >> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> > > committers
> > > >> and
> > > >> > > can
> > > >> > > > > > have a
> > > >> > > > > > >> > > > > separate
> > > >> > > > > > >> > > > > >> > repo,
> > > >> > > > > > >> > > > > >> > > but I'm actually not really sure (I
> can't
> > > >> find a
> > > >> > > > > > definition
> > > >> > > > > > >> > of a
> > > >> > > > > > >> > > > > >> > subproject
> > > >> > > > > > >> > > > > >> > > in Apache).
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > Basically at a high-level you want the
> > > >> experience
> > > >> > > to
> > > >> > > > > > "feel"
> > > >> > > > > > >> > > like a
> > > >> > > > > > >> > > > > >> single
> > > >> > > > > > >> > > > > >> > > system, not to relatively independent
> > things
> > > >> that
> > > >> > > are
> > > >> > > > > > kind
> > > >> > > > > > >> of
> > > >> > > > > > >> > > > > >> awkwardly
> > > >> > > > > > >> > > > > >> > > glued together.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > I think if we did that they having
> naming
> > or
> > > >> > > branding
> > > >> > > > > > like
> > > >> > > > > > >> > > "kafka
> > > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or
> something
> > > >> like
> > > >> > > that
> > > >> > > > > > would
> > > >> > > > > > >> > > > actually
> > > >> > > > > > >> > > > > >> do a
> > > >> > > > > > >> > > > > >> > > good job of conveying what it is. I do
> > that
> > > >> this
> > > >> > > > would
> > > >> > > > > > help
> > > >> > > > > > >> > > > adoption
> > > >> > > > > > >> > > > > >> > quite
> > > >> > > > > > >> > > > > >> > > a lot as it would correctly convey that
> > > using
> > > >> > Kafka
> > > >> > > > > > >> Streaming
> > > >> > > > > > >> > > with
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > is
> > > >> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka
> is
> > > >> pretty
> > > >> > > > > heavily
> > > >> > > > > > >> > adopted
> > > >> > > > > > >> > > > at
> > > >> > > > > > >> > > > > >> this
> > > >> > > > > > >> > > > > >> > > point.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> > > >> originally
> > > >> > > > when
> > > >> > > > > > open
> > > >> > > > > > >> > > > sourcing
> > > >> > > > > > >> > > > > >> > Samza,
> > > >> > > > > > >> > > > > >> > > however at that time Kafka was
> relatively
> > > >> unknown
> > > >> > > and
> > > >> > > > > we
> > > >> > > > > > >> > decided
> > > >> > > > > > >> > > > not
> > > >> > > > > > >> > > > > >> to
> > > >> > > > > > >> > > > > >> > do
> > > >> > > > > > >> > > > > >> > > it since we felt it would be limiting.
> > From
> > > my
> > > >> > > point
> > > >> > > > of
> > > >> > > > > > view
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > > three
> > > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now
> > really
> > > >> > heavily
> > > >> > > > > used
> > > >> > > > > > for
> > > >> > > > > > >> > > > stream
> > > >> > > > > > >> > > > > >> > > processing, (2) we learned that
> > abstracting
> > > >> out
> > > >> > the
> > > >> > > > > > stream
> > > >> > > > > > >> > well
> > > >> > > > > > >> > > is
> > > >> > > > > > >> > > > > >> > > basically impossible, (3) we learned it
> is
> > > >> really
> > > >> > > > hard
> > > >> > > > > to
> > > >> > > > > > >> keep
> > > >> > > > > > >> > > the
> > > >> > > > > > >> > > > > two
> > > >> > > > > > >> > > > > >> > > things feeling like a single product.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > -Jay
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> > > >> Kleppmann
> > > >> > <
> > > >> > > > > > >> > > > > >> martin@kleppmann.com>
> > > >> > > > > > >> > > > > >> > > wrote:
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > >> Hi all,
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> I agree with the general philosophy of
> > > tying
> > > >> > Samza
> > > >> > > > > more
> > > >> > > > > > >> > firmly
> > > >> > > > > > >> > > to
> > > >> > > > > > >> > > > > >> Kafka.
> > > >> > > > > > >> > > > > >> > >> After I spent a while looking at
> > > integrating
> > > >> > other
> > > >> > > > > > message
> > > >> > > > > > >> > > > brokers
> > > >> > > > > > >> > > > > >> (e.g.
> > > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to
> > the
> > > >> > > > conclusion
> > > >> > > > > > that
> > > >> > > > > > >> > > > > >> > SystemConsumer
> > > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
> > > Kafka's
> > > >> > that
> > > >> > > > > pretty
> > > >> > > > > > >> much
> > > >> > > > > > >> > > > > nobody
> > > >> > > > > > >> > > > > >> but
> > > >> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus
> is
> > > >> > perhaps
> > > >> > > an
> > > >> > > > > > >> > exception,
> > > >> > > > > > >> > > > but
> > > >> > > > > > >> > > > > >> it
> > > >> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.)
> > > Thus,
> > > >> > > making
> > > >> > > > > > Samza
> > > >> > > > > > >> > > fully
> > > >> > > > > > >> > > > > >> > dependent
> > > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > > >> > system-independence
> > > >> > > > was
> > > >> > > > > > >> never
> > > >> > > > > > >> > as
> > > >> > > > > > >> > > > > real
> > > >> > > > > > >> > > > > >> as
> > > >> > > > > > >> > > > > >> > we
> > > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of
> > > code
> > > >> > reuse
> > > >> > > > are
> > > >> > > > > > >> real.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN
> > has
> > > >> also
> > > >> > > > always
> > > >> > > > > > been
> > > >> > > > > > >> > > > > >> appealing to
> > > >> > > > > > >> > > > > >> > >> me, for various reasons already
> mentioned
> > > in
> > > >> > this
> > > >> > > > > > thread.
> > > >> > > > > > >> > > > Although
> > > >> > > > > > >> > > > > >> > making
> > > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > > >> > > > (YARN/Mesos/AWS/etc)
> > > >> > > > > > >> seems
> > > >> > > > > > >> > > > > >> laudable,
> > > >> > > > > > >> > > > > >> > I am
> > > >> > > > > > >> > > > > >> > >> a little concerned that it will
> restrict
> > us
> > > >> to a
> > > >> > > > > lowest
> > > >> > > > > > >> > common
> > > >> > > > > > >> > > > > >> > denominator.
> > > >> > > > > > >> > > > > >> > >> For example, would host affinity
> > > (SAMZA-617)
> > > >> > still
> > > >> > > > be
> > > >> > > > > > >> > possible?
> > > >> > > > > > >> > > > For
> > > >> > > > > > >> > > > > >> jobs
> > > >> > > > > > >> > > > > >> > >> with large amounts of state, I think
> > > >> SAMZA-617
> > > >> > > would
> > > >> > > > > be
> > > >> > > > > > a
> > > >> > > > > > >> big
> > > >> > > > > > >> > > > boon,
> > > >> > > > > > >> > > > > >> > since
> > > >> > > > > > >> > > > > >> > >> restoring state off the changelog on
> > every
> > > >> > single
> > > >> > > > > > restart
> > > >> > > > > > >> is
> > > >> > > > > > >> > > > > painful,
> > > >> > > > > > >> > > > > >> > due
> > > >> > > > > > >> > > > > >> > >> to long recovery times. It would be a
> > shame
> > > >> if
> > > >> > the
> > > >> > > > > > >> decoupling
> > > >> > > > > > >> > > > from
> > > >> > > > > > >> > > > > >> YARN
> > > >> > > > > > >> > > > > >> > >> made host affinity impossible.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Jay, a question about the proposed API
> > for
> > > >> > > > > > instantiating a
> > > >> > > > > > >> > job
> > > >> > > > > > >> > > in
> > > >> > > > > > >> > > > > >> code
> > > >> > > > > > >> > > > > >> > >> (rather than a properties file): when
> > > >> > submitting a
> > > >> > > > job
> > > >> > > > > > to a
> > > >> > > > > > >> > > > > cluster,
> > > >> > > > > > >> > > > > >> is
> > > >> > > > > > >> > > > > >> > the
> > > >> > > > > > >> > > > > >> > >> idea that the instantiation code runs
> on
> > a
> > > >> > client
> > > >> > > > > > >> somewhere,
> > > >> > > > > > >> > > > which
> > > >> > > > > > >> > > > > >> then
> > > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > > >> > > YARN/Mesos/AWS/etc?
> > > >> > > > > Or
> > > >> > > > > > >> does
> > > >> > > > > > >> > > that
> > > >> > > > > > >> > > > > >> code
> > > >> > > > > > >> > > > > >> > run
> > > >> > > > > > >> > > > > >> > >> on each container that is part of the
> job
> > > (in
> > > >> > > which
> > > >> > > > > > case,
> > > >> > > > > > >> how
> > > >> > > > > > >> > > > does
> > > >> > > > > > >> > > > > >> the
> > > >> > > > > > >> > > > > >> > job
> > > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
> > > >> right to
> > > >> > > > make
> > > >> > > > > a
> > > >> > > > > > 1.0
> > > >> > > > > > >> > > > release
> > > >> > > > > > >> > > > > >> > with a
> > > >> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete.
> > So
> > > if
> > > >> > this
> > > >> > > > is
> > > >> > > > > > going
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> happen, I
> > > >> > > > > > >> > > > > >> > >> think it would be more honest to stick
> > with
> > > >> 0.*
> > > >> > > > > version
> > > >> > > > > > >> > numbers
> > > >> > > > > > >> > > > > until
> > > >> > > > > > >> > > > > >> > the
> > > >> > > > > > >> > > > > >> > >> library-ified Samza has been
> implemented,
> > > is
> > > >> > > stable
> > > >> > > > > and
> > > >> > > > > > >> > widely
> > > >> > > > > > >> > > > > used.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
> > > >> Kafka?
> > > >> > > There
> > > >> > > > > is
> > > >> > > > > > >> > > precedent
> > > >> > > > > > >> > > > > for
> > > >> > > > > > >> > > > > >> > >> tight coupling between different Apache
> > > >> projects
> > > >> > > > (e.g.
> > > >> > > > > > >> > Curator
> > > >> > > > > > >> > > > and
> > > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I
> > think
> > > >> > > remaining
> > > >> > > > > > >> separate
> > > >> > > > > > >> > > > would
> > > >> > > > > > >> > > > > >> be
> > > >> > > > > > >> > > > > >> > ok.
> > > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on
> > Kafka,
> > > >> there
> > > >> > > is
> > > >> > > > > > enough
> > > >> > > > > > >> > > > > substance
> > > >> > > > > > >> > > > > >> in
> > > >> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> > > >> project.
> > > >> > > An
> > > >> > > > > > >> argument
> > > >> > > > > > >> > in
> > > >> > > > > > >> > > > > >> favour
> > > >> > > > > > >> > > > > >> > of
> > > >> > > > > > >> > > > > >> > >> merging would be if we think Kafka has
> a
> > > much
> > > >> > > > stronger
> > > >> > > > > > >> "brand
> > > >> > > > > > >> > > > > >> presence"
> > > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one.
> > If
> > > >> the
> > > >> > > Kafka
> > > >> > > > > > >> project
> > > >> > > > > > >> > is
> > > >> > > > > > >> > > > > >> willing
> > > >> > > > > > >> > > > > >> > to
> > > >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of
> > > doing
> > > >> > > > stateful
> > > >> > > > > > >> stream
> > > >> > > > > > >> > > > > >> > >> transformations, that would probably
> have
> > > >> much
> > > >> > the
> > > >> > > > > same
> > > >> > > > > > >> > effect
> > > >> > > > > > >> > > as
> > > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> > > >> Processors"
> > > >> > or
> > > >> > > > > > suchlike.
> > > >> > > > > > >> > > Close
> > > >> > > > > > >> > > > > >> > >> collaboration between the two projects
> > will
> > > >> be
> > > >> > > > needed
> > > >> > > > > in
> > > >> > > > > > >> any
> > > >> > > > > > >> > > > case.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> From a project management perspective,
> I
> > > >> guess
> > > >> > the
> > > >> > > > > "new
> > > >> > > > > > >> > Samza"
> > > >> > > > > > >> > > > > would
> > > >> > > > > > >> > > > > >> > have
> > > >> > > > > > >> > > > > >> > >> to be developed on a branch alongside
> > > ongoing
> > > >> > > > > > maintenance
> > > >> > > > > > >> of
> > > >> > > > > > >> > > the
> > > >> > > > > > >> > > > > >> current
> > > >> > > > > > >> > > > > >> > >> line of development? I think it would
> be
> > > >> > important
> > > >> > > > to
> > > >> > > > > > >> > continue
> > > >> > > > > > >> > > > > >> > supporting
> > > >> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> > > >> migration
> > > >> > > > path
> > > >> > > > > to
> > > >> > > > > > >> the
> > > >> > > > > > >> > > new
> > > >> > > > > > >> > > > > >> > version.
> > > >> > > > > > >> > > > > >> > >> Leaving the current versions
> unsupported
> > > and
> > > >> > > forcing
> > > >> > > > > > people
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> rewrite
> > > >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Best,
> > > >> > > > > > >> > > > > >> > >> Martin
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > > >> > > > jay@confluent.io>
> > > >> > > > > > >> wrote:
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >>> Hey Garry,
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be
> > > happy
> > > >> to
> > > >> > > chat
> > > >> > > > > > more
> > > >> > > > > > >> > about
> > > >> > > > > > >> > > > > this
> > > >> > > > > > >> > > > > >> if
> > > >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris
> and I
> > > >> > started
> > > >> > > > with
> > > >> > > > > > the
> > > >> > > > > > >> > idea
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> "what
> > > >> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> > > >> > ingestion
> > > >> > > > > tool"
> > > >> > > > > > but
> > > >> > > > > > >> > > > > >> ultimately
> > > >> > > > > > >> > > > > >> > we
> > > >> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> > > >> ingestion
> > > >> > > and
> > > >> > > > > > >> > > > transformation
> > > >> > > > > > >> > > > > >> had
> > > >> > > > > > >> > > > > >> > >>> pretty different needs and coupling
> the
> > > two
> > > >> > made
> > > >> > > > > things
> > > >> > > > > > >> > hard.
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
> > > (KIP-26)
> > > >> > > > actually
> > > >> > > > > > will
> > > >> > > > > > >> > do
> > > >> > > > > > >> > > > what
> > > >> > > > > > >> > > > > >> you
> > > >> > > > > > >> > > > > >> > >> are
> > > >> > > > > > >> > > > > >> > >>> looking for.
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> With regard to your point about
> slider,
> > I
> > > >> don't
> > > >> > > > > > >> necessarily
> > > >> > > > > > >> > > > > >> disagree.
> > > >> > > > > > >> > > > > >> > >> But I
> > > >> > > > > > >> > > > > >> > >>> think getting good YARN support is
> quite
> > > >> doable
> > > >> > > > and I
> > > >> > > > > > >> think
> > > >> > > > > > >> > we
> > > >> > > > > > >> > > > can
> > > >> > > > > > >> > > > > >> make
> > > >> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> > > >> proposal
> > > >> > > > > solves
> > > >> > > > > > is
> > > >> > > > > > >> > that
> > > >> > > > > > >> > > > > >> > >> technically
> > > >> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
> > > >> cluster
> > > >> > > > > > management
> > > >> > > > > > >> > > systems
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > way
> > > >> > > > > > >> > > > > >> > >>> things are now, you need to write an
> > "app
> > > >> > master"
> > > >> > > > or
> > > >> > > > > > >> > > "framework"
> > > >> > > > > > >> > > > > for
> > > >> > > > > > >> > > > > >> > each
> > > >> > > > > > >> > > > > >> > >>> and they are all a little different so
> > > >> testing
> > > >> > is
> > > >> > > > > > really
> > > >> > > > > > >> > hard.
> > > >> > > > > > >> > > > In
> > > >> > > > > > >> > > > > >> the
> > > >> > > > > > >> > > > > >> > >>> absence of this we have been stuck
> with
> > > just
> > > >> > YARN
> > > >> > > > > which
> > > >> > > > > > >> has
> > > >> > > > > > >> > > > > >> fantastic
> > > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the
> > > org,
> > > >> but
> > > >> > > > zero
> > > >> > > > > > >> > > penetration
> > > >> > > > > > >> > > > > >> > >> elsewhere.
> > > >> > > > > > >> > > > > >> > >>> Given the huge amount of work being
> put
> > in
> > > >> to
> > > >> > > > slider,
> > > >> > > > > > >> > > marathon,
> > > >> > > > > > >> > > > > aws
> > > >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
> > > related
> > > >> > > > packaging
> > > >> > > > > > >> > > > technologies
> > > >> > > > > > >> > > > > >> > people
> > > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes,
> various
> > > >> > > > > cloud-specific
> > > >> > > > > > >> > deploy
> > > >> > > > > > >> > > > > >> tools,
> > > >> > > > > > >> > > > > >> > >> etc)
> > > >> > > > > > >> > > > > >> > >>> I really think it is important to get
> > this
> > > >> > right.
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> -Jay
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> > > >> > Turkington
> > > >> > > <
> > > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com>
> wrote:
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>>> Hi all,
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> I think the question below re does
> > Samza
> > > >> > become
> > > >> > > a
> > > >> > > > > > >> > sub-project
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > >>>> highlights the broader point around
> > > >> migration.
> > > >> > > > Chris
> > > >> > > > > > >> > mentions
> > > >> > > > > > >> > > > > >> Samza's
> > > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1
> > release
> > > >> but
> > > >> > I'm
> > > >> > > > not
> > > >> > > > > > sure
> > > >> > > > > > >> > it
> > > >> > > > > > >> > > > > feels
> > > >> > > > > > >> > > > > >> > >> right to
> > > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> > > >> deprecate
> > > >> > > > most
> > > >> > > > > of
> > > >> > > > > > >> it.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have
> some
> > > guys
> > > >> > who
> > > >> > > > have
> > > >> > > > > > >> > started
> > > >> > > > > > >> > > > > >> working
> > > >> > > > > > >> > > > > >> > >> with
> > > >> > > > > > >> > > > > >> > >>>> Samza and building some new
> > > >> > consumers/producers
> > > >> > > > was
> > > >> > > > > > next
> > > >> > > > > > >> > up.
> > > >> > > > > > >> > > > > Sounds
> > > >> > > > > > >> > > > > >> > like
> > > >> > > > > > >> > > > > >> > >>>> that is absolutely not the direction
> to
> > > >> go. I
> > > >> > > need
> > > >> > > > > to
> > > >> > > > > > >> look
> > > >> > > > > > >> > > into
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > KIP
> > > >> > > > > > >> > > > > >> > >> in
> > > >> > > > > > >> > > > > >> > >>>> more detail but for me the
> > attractiveness
> > > >> of
> > > >> > > > adding
> > > >> > > > > > new
> > > >> > > > > > >> > Samza
> > > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all
> > > they
> > > >> > were
> > > >> > > > > doing
> > > >> > > > > > was
> > > >> > > > > > >> > > > really
> > > >> > > > > > >> > > > > >> > getting
> > > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to
> > > avoid
> > > >> > > > having
> > > >> > > > > to
> > > >> > > > > > >> > worry
> > > >> > > > > > >> > > > > about
> > > >> > > > > > >> > > > > >> the
> > > >> > > > > > >> > > > > >> > >>>> lifecycle management of external
> > clients.
> > > >> If
> > > >> > > there
> > > >> > > > > is
> > > >> > > > > > a
> > > >> > > > > > >> > > generic
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug
> a
> > > new
> > > >> > > > connector
> > > >> > > > > > into
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > have
> > > >> > > > > > >> > > > > >> a
> > > >> > > > > > >> > > > > >> > >> lot of
> > > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> > > reliability
> > > >> > done
> > > >> > > > for
> > > >> > > > > me
> > > >> > > > > > >> then
> > > >> > > > > > >> > > it
> > > >> > > > > > >> > > > > >> gives
> > > >> > > > > > >> > > > > >> > me
> > > >> > > > > > >> > > > > >> > >> all
> > > >> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers
> > > would.
> > > >> If
> > > >> > > not
> > > >> > > > > > then it
> > > >> > > > > > >> > > > > >> complicates
> > > >> > > > > > >> > > > > >> > my
> > > >> > > > > > >> > > > > >> > >>>> operational deployments.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Which is similar to my other question
> > > with
> > > >> the
> > > >> > > > > > proposal
> > > >> > > > > > >> --
> > > >> > > > > > >> > if
> > > >> > > > > > >> > > > we
> > > >> > > > > > >> > > > > >> > build a
> > > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza
> plus
> > > the
> > > >> > > > requisite
> > > >> > > > > > >> shims
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> > integrate
> > > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former
> > may
> > > >> be a
> > > >> > > lot
> > > >> > > > > more
> > > >> > > > > > >> work
> > > >> > > > > > >> > > > than
> > > >> > > > > > >> > > > > we
> > > >> > > > > > >> > > > > >> > >> think.
> > > >> > > > > > >> > > > > >> > >>>> We may make it much easier for a
> > newcomer
> > > >> to
> > > >> > get
> > > >> > > > > > >> something
> > > >> > > > > > >> > > > > running
> > > >> > > > > > >> > > > > >> but
> > > >> > > > > > >> > > > > >> > >>>> having them step up and get a
> reliable
> > > >> > > production
> > > >> > > > > > >> > deployment
> > > >> > > > > > >> > > > may
> > > >> > > > > > >> > > > > >> still
> > > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if
> for
> > > >> > different
> > > >> > > > > > reasons
> > > >> > > > > > >> > than
> > > >> > > > > > >> > > > > >> today.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable
> > > with
> > > >> > > making
> > > >> > > > > the
> > > >> > > > > > >> Samza
> > > >> > > > > > >> > > > > >> dependency
> > > >> > > > > > >> > > > > >> > >> on
> > > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
> > absolutely
> > > >> see
> > > >> > > the
> > > >> > > > > > >> benefits
> > > >> > > > > > >> > > in
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > > >> > > > > > >> > > > terminologies/abstractions
> > > >> > > > > > >> > > > > >> that
> > > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a
> library
> > > >> would
> > > >> > > > likely
> > > >> > > > > > be a
> > > >> > > > > > >> > very
> > > >> > > > > > >> > > > > nice
> > > >> > > > > > >> > > > > >> > tool
> > > >> > > > > > >> > > > > >> > >> to
> > > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just
> have
> > > the
> > > >> > > > concerns
> > > >> > > > > > >> above
> > > >> > > > > > >> > re
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > >>>> operational side.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Garry
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> > > >> [mailto:
> > > >> > > > > > >> > gdfm@apache.org
> > > >> > > > > > >> > > ]
> > > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and
> obesrvations
> > on
> > > >> > Samza
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > > >> > > > > > >> > > > > >> > >>>> From outside, I have always perceived
> > > Samza
> > > >> > as a
> > > >> > > > > > >> computing
> > > >> > > > > > >> > > > layer
> > > >> > > > > > >> > > > > >> over
> > > >> > > > > > >> > > > > >> > >>>> Kafka.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit
> provocative,
> > is
> > > >> > > "should
> > > >> > > > > > Samza
> > > >> > > > > > >> be
> > > >> > > > > > >> > a
> > > >> > > > > > >> > > > > >> > sub-project
> > > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> > > >> separate
> > > >> > > > > project
> > > >> > > > > > >> > with a
> > > >> > > > > > >> > > > > >> separate
> > > >> > > > > > >> > > > > >> > >>>> governance?
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Cheers,
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> --
> > > >> > > > > > >> > > > > >> > >>>> Gianmarco
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > > >> > > > > > yanfang724@gmail.com>
> > > >> > > > > > >> > > > wrote:
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with
> Kafka
> > > more
> > > >> > > > tightly.
> > > >> > > > > > >> > Because
> > > >> > > > > > >> > > > > Samza
> > > >> > > > > > >> > > > > >> de
> > > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it
> should
> > > >> > leverage
> > > >> > > > > what
> > > >> > > > > > >> Kafka
> > > >> > > > > > >> > > > has.
> > > >> > > > > > >> > > > > At
> > > >> > > > > > >> > > > > >> > the
> > > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
> > > reinvent
> > > >> > what
> > > >> > > > > Samza
> > > >> > > > > > >> > > already
> > > >> > > > > > >> > > > > >> has. I
> > > >> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> > > >> > ingestion
> > > >> > > > and
> > > >> > > > > > >> > > > > transformation.
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me
> to
> > > >> image
> > > >> > > how
> > > >> > > > > the
> > > >> > > > > > >> Samza
> > > >> > > > > > >> > > > will
> > > >> > > > > > >> > > > > >> look
> > > >> > > > > > >> > > > > >> > >>>> like.
> > > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a
> little
> > > >> > > difference
> > > >> > > > > in
> > > >> > > > > > >> terms
> > > >> > > > > > >> > > of
> > > >> > > > > > >> > > > > how
> > > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's
> code
> > > >> shows
> > > >> > (A
> > > >> > > > > > client of
> > > >> > > > > > >> > > > Kakfa)
> > > >> > > > > > >> > > > > ?
> > > >> > > > > > >> > > > > >> And
> > > >> > > > > > >> > > > > >> > >>>>> user's application code calls this
> > > client?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of
> > > Kafka
> > > >> > (like
> > > >> > > > > what
> > > >> > > > > > the
> > > >> > > > > > >> > > code
> > > >> > > > > > >> > > > > >> shows),
> > > >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > > >> > > > > fault-tolerance?
> > > >> > > > > > >> Are
> > > >> > > > > > >> > > they
> > > >> > > > > > >> > > > > >> taken
> > > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> > > >> mechanism,
> > > >> > > such
> > > >> > > > > as
> > > >> > > > > > >> > "Samza
> > > >> > > > > > >> > > > > >> worker"
> > > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 2. What about other features, such
> as
> > > >> > > > auto-scaling,
> > > >> > > > > > >> shared
> > > >> > > > > > >> > > > > state,
> > > >> > > > > > >> > > > > >> > >>>>> monitoring?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is
> > > this
> > > >> > what
> > > >> > > > > Chris
> > > >> > > > > > >> > > > suggests?)
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from
> > > Kakfa
> > > >> > and
> > > >> > > > > > produce
> > > >> > > > > > >> to
> > > >> > > > > > >> > > it.
> > > >> > > > > > >> > > > > >> Then it
> > > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks
> > > like
> > > >> > now,
> > > >> > > > > > except it
> > > >> > > > > > >> > > does
> > > >> > > > > > >> > > > > not
> > > >> > > > > > >> > > > > >> > rely
> > > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> > > >> leverage
> > > >> > > > Kafka's
> > > >> > > > > > >> > metrics,
> > > >> > > > > > >> > > > > logs,
> > > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the
> dependency?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> Thanks,
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM,
> > Guozhang
> > > >> > Wang <
> > > >> > > > > > >> > > > > wangguoz@gmail.com
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> Read through the code example and
> it
> > > >> looks
> > > >> > > good
> > > >> > > > to
> > > >> > > > > > me.
> > > >> > > > > > >> A
> > > >> > > > > > >> > > few
> > > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> > > >> runnable
> > > >> > > like:
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > > >> > > --config-factory=...
> > > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> > > deploying
> > > >> > Samza
> > > >> > > > > more
> > > >> > > > > > as
> > > >> > > > > > >> > > > embedded
> > > >> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> > > >> (ignoring
> > > >> > > the
> > > >> > > > > > >> > > terminology
> > > >> > > > > > >> > > > > >> since
> > > >> > > > > > >> > > > > >> > >>>>>> it is not the
> > > >> > > > > > >> > > > > >> > >>>>> same
> > > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> > > >> MyStreamTask(configs);
> > > >> > > > > Thread
> > > >> > > > > > >> > thread
> > > >> > > > > > >> > > =
> > > >> > > > > > >> > > > > new
> > > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment
> > modes
> > > >> are
> > > >> > > > > important
> > > >> > > > > > >> for
> > > >> > > > > > >> > > > > >> different
> > > >> > > > > > >> > > > > >> > >>>>>> types
> > > >> > > > > > >> > > > > >> > >>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making
> > Samza
> > > >> > purely
> > > >> > > > > > >> standalone
> > > >> > > > > > >> > is
> > > >> > > > > > >> > > > > still
> > > >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or
> > > library
> > > >> > > modes.
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> Guozhang
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM,
> Jay
> > > >> Kreps
> > > >> > <
> > > >> > > > > > >> > > > jay@confluent.io>
> > > >> > > > > > >> > > > > >> > wrote:
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> > > >> example,
> > > >> > it
> > > >> > > > was
> > > >> > > > > > >> > supposed
> > > >> > > > > > >> > > > to
> > > >> > > > > > >> > > > > >> look
> > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > >> > > > > > >> > > > > >> > >>>>>>> this:
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new
> Properties();
> > > >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > > >> > > > "localhost:4242");
> > > >> > > > > > >> > > > > >> StreamingConfig
> > > >> > > > > > >> > > > > >> > >>>>>>> config = new
> StreamingConfig(props);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > >> > > > "test-topic-2");
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > config.processor(ExampleStreamProcessor.class);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > >> > StringSerializer(),
> > > >> > > > new
> > > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
> > KafkaStreaming
> > > >> > > > container =
> > > >> > > > > > new
> > > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> > > container.run();
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> -Jay
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM,
> > Jay
> > > >> > Kreps <
> > > >> > > > > > >> > > > jay@confluent.io
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some
> conversations
> > > >> Chris
> > > >> > > and
> > > >> > > > I
> > > >> > > > > > were
> > > >> > > > > > >> > > having
> > > >> > > > > > >> > > > > >> > >>>>>>>> around
> > > >> > > > > > >> > > > > >> > >>>>>>> whether
> > > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza
> > as a
> > > >> kind
> > > >> > > of
> > > >> > > > > data
> > > >> > > > > > >> > > > ingestion
> > > >> > > > > > >> > > > > >> > >>>>> framework
> > > >> > > > > > >> > > > > >> > >>>>>>> for
> > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to
> > > KIP-26
> > > >> > > > > "copycat").
> > > >> > > > > > >> This
> > > >> > > > > > >> > > > kind
> > > >> > > > > > >> > > > > of
> > > >> > > > > > >> > > > > >> > >>>>>> combined
> > > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and
> > > YARN
> > > >> and
> > > >> > > the
> > > >> > > > > > >> > discussion
> > > >> > > > > > >> > > > > >> around
> > > >> > > > > > >> > > > > >> > >>>>>>>> how
> > > >> > > > > > >> > > > > >> > >>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was,
> > given
> > > >> that
> > > >> > > > Samza
> > > >> > > > > > was
> > > >> > > > > > >> > > > basically
> > > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific,
> > what
> > > if
> > > >> > you
> > > >> > > > just
> > > >> > > > > > >> > embraced
> > > >> > > > > > >> > > > > that
> > > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > > >> > > > > > >> > > > > >> > >>>>>> into
> > > >> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> > > >> > framework
> > > >> > > > and
> > > >> > > > > > more
> > > >> > > > > > >> > > like a
> > > >> > > > > > >> > > > > >> > >>>>>>>> third
> > > >> > > > > > >> > > > > >> > >>>>> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> > > consumer"
> > > >> > with
> > > >> > > > > state
> > > >> > > > > > >> > > > management
> > > >> > > > > > >> > > > > >> > >>>>>> facilities.
> > > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> > > >> complex
> > > >> > > > stream
> > > >> > > > > > >> > > processing
> > > >> > > > > > >> > > > > >> > >>>>>>>> framework
> > > >> > > > > > >> > > > > >> > >>>>>>> this
> > > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple
> > > thing,
> > > >> not
> > > >> > > > much
> > > >> > > > > > more
> > > >> > > > > > >> > > > > >> complicated
> > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>> use
> > > >> > > > > > >> > > > > >> > >>>>>>> or
> > > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As
> > > Chris
> > > >> > said
> > > >> > > > we
> > > >> > > > > > >> thought
> > > >> > > > > > >> > > > about
> > > >> > > > > > >> > > > > >> it
> > > >> > > > > > >> > > > > >> > >>>>>>>> a
> > > >> > > > > > >> > > > > >> > >>>>> lot
> > > >> > > > > > >> > > > > >> > >>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> > > >> > processing
> > > >> > > > > > systems
> > > >> > > > > > >> > were
> > > >> > > > > > >> > > > > doing)
> > > >> > > > > > >> > > > > >> > >>>>> seemed
> > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from
> MapReduce.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to
> ingest/output
> > > >> data
> > > >> > to
> > > >> > > > and
> > > >> > > > > > from
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > > stream
> > > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually
> > > looked
> > > >> > into
> > > >> > > > how
> > > >> > > > > > that
> > > >> > > > > > >> > > would
> > > >> > > > > > >> > > > > >> > >>>>>>>> work,
> > > >> > > > > > >> > > > > >> > >>>>> Samza
> > > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
> > ingestion
> > > >> > > framework
> > > >> > > > > > for a
> > > >> > > > > > >> > > bunch
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > >>>>> reasons.
> > > >> > > > > > >> > > > > >> > >>>>>> To
> > > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a
> > > pretty
> > > >> > > > different
> > > >> > > > > > >> > internal
> > > >> > > > > > >> > > > > data
> > > >> > > > > > >> > > > > >> > >>>>>>>> model
> > > >> > > > > > >> > > > > >> > >>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split
> > > them
> > > >> and
> > > >> > > had
> > > >> > > > > an
> > > >> > > > > > api
> > > >> > > > > > >> > for
> > > >> > > > > > >> > > > > Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA
> KIP-26)
> > > >> and a
> > > >> > > > > separate
> > > >> > > > > > >> api
> > > >> > > > > > >> > > for
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> > > embracing
> > > >> the
> > > >> > > > same
> > > >> > > > > > >> > > terminology
> > > >> > > > > > >> > > > > and
> > > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about
> > the
> > > >> > current
> > > >> > > > > > state is
> > > >> > > > > > >> > > that
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > >>>>>>>> two
> > > >> > > > > > >> > > > > >> > >>>>>>> systems
> > > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on.
> Terminology
> > > >> like
> > > >> > > > > "stream"
> > > >> > > > > > vs
> > > >> > > > > > >> > > > "topic"
> > > >> > > > > > >> > > > > >> and
> > > >> > > > > > >> > > > > >> > >>>>>>> different
> > > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems
> means
> > > you
> > > >> > kind
> > > >> > > > of
> > > >> > > > > > have
> > > >> > > > > > >> to
> > > >> > > > > > >> > > > learn
> > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > > >> > > > > > >> > > > > >> > >>>>>>> way,
> > > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
> > different
> > > >> way,
> > > >> > > > then
> > > >> > > > > > kind
> > > >> > > > > > >> of
> > > >> > > > > > >> > > > > >> > >>>>>>>> understand
> > > >> > > > > > >> > > > > >> > >>>>> how
> > > >> > > > > > >> > > > > >> > >>>>>>> they
> > > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having
> > > walked
> > > >> a
> > > >> > few
> > > >> > > > > > people
> > > >> > > > > > >> > > through
> > > >> > > > > > >> > > > > >> this
> > > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks
> to
> > > >> get.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot
> of
> > > >> time
> > > >> > on
> > > >> > > > > > >> airplanes I
> > > >> > > > > > >> > > > > hacked
> > > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> > > >> incomplete
> > > >> > > > > > prototype
> > > >> > > > > > >> of
> > > >> > > > > > >> > > > what
> > > >> > > > > > >> > > > > >> > >>>>>>>> this would
> > > >> > > > > > >> > > > > >> > >>>>> look
> > > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just
> unceremoniously
> > > >> dumped
> > > >> > > into
> > > >> > > > > > Kafka
> > > >> > > > > > >> as
> > > >> > > > > > >> > > it
> > > >> > > > > > >> > > > > >> > >>>>>>>> required a
> > > >> > > > > > >> > > > > >> > >>>>>> few
> > > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here
> > is
> > > >> the
> > > >> > > code:
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> >
> > > >> > > > > >
> > > >> > >
> > > >>
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype
> I
> > > just
> > > >> > > > > liberally
> > > >> > > > > > >> > renamed
> > > >> > > > > > >> > > > > >> > >>>>>>>> everything
> > > >> > > > > > >> > > > > >> > >>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with
> no
> > > >> regard
> > > >> > > for
> > > >> > > > > > >> > > > compatibility.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something
> like
> > > >> this:
> > > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
> > Properties();
> > > >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > > >> > > > > "localhost:4242");
> > > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > > >> > > > > > >> > > > > >>
> > config.processor(ExampleStreamProcessor.class);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > > >> > > StringDeserializer());
> > > >> > > > > > >> > > > KafkaStreaming
> > > >> > > > > > >> > > > > >> > >>>>>> container =
> > > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > > >> > container.run();
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > > >> > > > SamzaContainer;
> > > >> > > > > > >> > > > > StreamProcessor
> > > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the
> > class
> > > >> names
> > > >> > > in
> > > >> > > > a
> > > >> > > > > > file
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > then
> > > >> > > > > > >> > > > > >> > >>>>>>>> having
> > > >> > > > > > >> > > > > >> > >>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you
> > just
> > > >> > > > > instantiate
> > > >> > > > > > the
> > > >> > > > > > >> > > > > container
> > > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is
> balanced
> > > over
> > > >> > > > however
> > > >> > > > > > many
> > > >> > > > > > >> > > > > instances
> > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>> this
> > > >> > > > > > >> > > > > >> > >>>>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> > > instance
> > > >> > dies,
> > > >> > > > new
> > > >> > > > > > >> tasks
> > > >> > > > > > >> > > are
> > > >> > > > > > >> > > > > >> added
> > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
> > shutting
> > > >> them
> > > >> > > > down).
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
> > > running
> > > >> > this
> > > >> > > > > stuff
> > > >> > > > > > in
> > > >> > > > > > >> > YARN
> > > >> > > > > > >> > > > via
> > > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and
> AWS
> > > >> using
> > > >> > > some
> > > >> > > > > of
> > > >> > > > > > >> their
> > > >> > > > > > >> > > > tools
> > > >> > > > > > >> > > > > >> > >>>>>>>> but from the
> > > >> > > > > > >> > > > > >> > >>>>>> point
> > > >> > > > > > >> > > > > >> > >>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these
> > stream
> > > >> > > > processing
> > > >> > > > > > jobs
> > > >> > > > > > >> > are
> > > >> > > > > > >> > > > > just
> > > >> > > > > > >> > > > > >> > >>>>>> stateless
> > > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
> > > >> expand
> > > >> > and
> > > >> > > > > > contract
> > > >> > > > > > >> > at
> > > >> > > > > > >> > > > > will.
> > > >> > > > > > >> > > > > >> > >>>>>>>> There
> > > >> > > > > > >> > > > > >> > >>>>> is
> > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of
> code,
> > > it
> > > >> > would
> > > >> > > > get
> > > >> > > > > > >> larger
> > > >> > > > > > >> > > if
> > > >> > > > > > >> > > > we
> > > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
> > > larger.
> > > >> We
> > > >> > > > really
> > > >> > > > > > do
> > > >> > > > > > >> > get a
> > > >> > > > > > >> > > > ton
> > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>> leverage
> > > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> > > >> > delegated
> > > >> > > to
> > > >> > > > > the
> > > >> > > > > > >> new
> > > >> > > > > > >> > > > > >> consumer.
> > > >> > > > > > >> > > > > >> > >>>>> This
> > > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> > > >> > management
> > > >> > > > > > strategy
> > > >> > > > > > >> > > > > available
> > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to
> > Samza
> > > >> (and
> > > >> > > vice
> > > >> > > > > > versa)
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > with
> > > >> > > > > > >> > > > > >> > >>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>> exact
> > > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as
> > > state
> > > >> > reuse
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it
> is
> > > >> > thought
> > > >> > > > > > >> provoking.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM,
> > > Chris
> > > >> > > > > Riccomini <
> > > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with
> > > Samza
> > > >> > > > > engineers
> > > >> > > > > > at
> > > >> > > > > > >> > > > LinkedIn
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>> Confluent
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> > > observations
> > > >> > and
> > > >> > > > > would
> > > >> > > > > > >> like
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> > >>>>>>>>> propose
> > > >> > > > > > >> > > > > >> > >>>>> some
> > > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that
> I
> > > >> want to
> > > >> > > > call
> > > >> > > > > > out
> > > >> > > > > > >> > about
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > > >> > > > > > >> > > > > >> > >>>>>> design,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> > > changes.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a
> > dynamic
> > > >> > > > deployment
> > > >> > > > > > >> system.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> > > >> SystemConsumer/SystemProducer
> > > >> > and
> > > >> > > > > > Kafka's
> > > >> > > > > > >> > > > consumer
> > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > >> > > > > > >> > > > > >> > >>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the
> same
> > > >> > problems.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
> > > related,
> > > >> > but
> > > >> > > > I'll
> > > >> > > > > > >> > address
> > > >> > > > > > >> > > > them
> > > >> > > > > > >> > > > > >> in
> > > >> > > > > > >> > > > > >> > >>>>> order.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the
> use
> > > of a
> > > >> > > > dynamic
> > > >> > > > > > >> > > deployment
> > > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > > >> > > > > > >> > > > > >> > >>>>>> such
> > > >> > > > > > >> > > > > >> > >>>>>>>>> as
> > > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
> > initially
> > > >> built
> > > >> > > > > Samza,
> > > >> > > > > > we
> > > >> > > > > > >> > bet
> > > >> > > > > > >> > > > that
> > > >> > > > > > >> > > > > >> > >>>>>>>>> there
> > > >> > > > > > >> > > > > >> > >>>>>> would
> > > >> > > > > > >> > > > > >> > >>>>>>>>> be
> > > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area,
> > and
> > > >> we
> > > >> > > could
> > > >> > > > > > >> support
> > > >> > > > > > >> > > > them,
> > > >> > > > > > >> > > > > >> and
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>> rest
> > > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there
> > are
> > > >> many
> > > >> > > > > > >> variations.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > > >> > > > > > >> > > > > >> > >>>>>> many
> > > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just
> start
> > > >> their
> > > >> > > > > > processors
> > > >> > > > > > >> > like
> > > >> > > > > > >> > > > > normal
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
> > traditional
> > > >> > > > deployment
> > > >> > > > > > >> scripts
> > > >> > > > > > >> > > > such
> > > >> > > > > > >> > > > > as
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > > >> > > > > > >> > > > > >> > >>>>>> Chef,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a
> deployment
> > > >> system
> > > >> > > on
> > > >> > > > > > users
> > > >> > > > > > >> > makes
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
> > > painful
> > > >> for
> > > >> > > > first
> > > >> > > > > > time
> > > >> > > > > > >> > > > users.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
> > requirement
> > > >> was
> > > >> > > also
> > > >> > > > a
> > > >> > > > > > bit
> > > >> > > > > > >> of
> > > >> > > > > > >> > a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > > >> > > > > > >> > > > > >> > >>>>>> because
> > > >> > > > > > >> > > > > >> > >>>>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding
> > > between
> > > >> > the
> > > >> > > > > > nature of
> > > >> > > > > > >> > > batch
> > > >> > > > > > >> > > > > >> jobs
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>> stream
> > > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we
> made
> > > >> > > conscious
> > > >> > > > > > effort
> > > >> > > > > > >> to
> > > >> > > > > > >> > > > favor
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>> Hadoop
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing
> things,
> > > >> since
> > > >> > it
> > > >> > > > > worked
> > > >> > > > > > >> and
> > > >> > > > > > >> > > was
> > > >> > > > > > >> > > > > well
> > > >> > > > > > >> > > > > >> > >>>>>>> understood.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was
> that
> > > >> batch
> > > >> > > jobs
> > > >> > > > > > have a
> > > >> > > > > > >> > > > definite
> > > >> > > > > > >> > > > > >> > >>>>>> beginning,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs
> > > don't
> > > >> > > > > (usually).
> > > >> > > > > > >> This
> > > >> > > > > > >> > > > leads
> > > >> > > > > > >> > > > > to
> > > >> > > > > > >> > > > > >> > >>>>>>>>> a
> > > >> > > > > > >> > > > > >> > >>>>> much
> > > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for
> > > stream
> > > >> > > > > processors.
> > > >> > > > > > >> You
> > > >> > > > > > >> > > > > >> basically
> > > >> > > > > > >> > > > > >> > >>>>>>>>> just
> > > >> > > > > > >> > > > > >> > >>>>>>> need
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> > > >> processor,
> > > >> > and
> > > >> > > > > start
> > > >> > > > > > >> it.
> > > >> > > > > > >> > > The
> > > >> > > > > > >> > > > > way
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn,
> there's
> > > no
> > > >> > > concept
> > > >> > > > > of
> > > >> > > > > > a
> > > >> > > > > > >> > > cluster
> > > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > > >> > > > > > >> > > > > >> > >>>>>> add
> > > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> > > >> coupling
> > > >> > > > Samza
> > > >> > > > > > with
> > > >> > > > > > >> a
> > > >> > > > > > >> > > > > >> scheduler
> > > >> > > > > > >> > > > > >> > >>>>>>>>> is
> > > >> > > > > > >> > > > > >> > >>>>>> that
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has
> to
> > > >> handle
> > > >> > > > > > deployment.
> > > >> > > > > > >> > > This
> > > >> > > > > > >> > > > > >> pulls
> > > >> > > > > > >> > > > > >> > >>>>>>>>> in a
> > > >> > > > > > >> > > > > >> > >>>>>>> bunch
> > > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> > > >> > > distribution
> > > >> > > > > > (config
> > > >> > > > > > >> > > > > stream),
> > > >> > > > > > >> > > > > >> > >>>>>>>>> shell
> > > >> > > > > > >> > > > > >> > >>>>>>> scrips
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> > > packaging
> > > >> > (all
> > > >> > > > the
> > > >> > > > > > .tgz
> > > >> > > > > > >> > > > stuff),
> > > >> > > > > > >> > > > > >> etc.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring
> > dynamic
> > > >> > > > deployment
> > > >> > > > > > was
> > > >> > > > > > >> to
> > > >> > > > > > >> > > > > support
> > > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to
> have
> > > >> > > locality,
> > > >> > > > > you
> > > >> > > > > > >> need
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > put
> > > >> > > > > > >> > > > > >> > >>>>>>>>> your
> > > >> > > > > > >> > > > > >> > >>>>>> processors
> > > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> > > processing.
> > > >> > Upon
> > > >> > > > > > further
> > > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > > >> > > > > > >> > > > > >> > >>>>>>> though,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
> > beneficial.
> > > >> > There
> > > >> > > is
> > > >> > > > > > some
> > > >> > > > > > >> > good
> > > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > > >> > > > > > >> > > > > >> > >>>>>> about
> > > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
> > SAMZA-335.
> > > >> > Again,
> > > >> > > we
> > > >> > > > > > took
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > > >> > > > > > >> > > > > >> > >>>>>> path,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> > > differences
> > > >> > > > between
> > > >> > > > > > HDFS
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > > >> > > > > > >> > > > > >> > >>>>>> has
> > > >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has
> > partitions.
> > > >> This
> > > >> > > > leads
> > > >> > > > > to
> > > >> > > > > > >> less
> > > >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with
> stream
> > > >> > > processors
> > > >> > > > > on
> > > >> > > > > > top
> > > >> > > > > > >> > of
> > > >> > > > > > >> > > > > Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a
> > > crutch.
> > > >> > > Samza
> > > >> > > > > > doesn't
> > > >> > > > > > >> > > have
> > > >> > > > > > >> > > > > any
> > > >> > > > > > >> > > > > >> > >>>>>>>>> built
> > > >> > > > > > >> > > > > >> > >>>>> in
> > > >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead,
> it
> > > >> > depends
> > > >> > > on
> > > >> > > > > the
> > > >> > > > > > >> > > dynamic
> > > >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to
> > > handle
> > > >> > > > restarts
> > > >> > > > > > >> when a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > > >> > > > > > >> > > > > >> > >>>>>>> made
> > > >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> > > >> standalone
> > > >> > > Samza
> > > >> > > > > > >> > container
> > > >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is
> > good,
> > > >> but I
> > > >> > > > think
> > > >> > > > > > that
> > > >> > > > > > >> > > we've
> > > >> > > > > > >> > > > > >> gone
> > > >> > > > > > >> > > > > >> > >>>>>>>>> too
> > > >> > > > > > >> > > > > >> > >>>>>> far
> > > >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > > >> > > > (SystemConsumer,
> > > >> > > > > > >> > > > > SystemProducer,
> > > >> > > > > > >> > > > > >> > >>>> etc).
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just
> > > about
> > > >> > every
> > > >> > > > > > >> component
> > > >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> > > >> > > ConfigRewriter,
> > > >> > > > > > etc).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> > > >> > forgotten,
> > > >> > > as
> > > >> > > > > > well.
> > > >> > > > > > >> > Some
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > >>>>>>>>> these
> > > >> > > > > > >> > > > > >> > >>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not
> > to
> > > >> be.
> > > >> > > This
> > > >> > > > > all
> > > >> > > > > > >> comes
> > > >> > > > > > >> > > at
> > > >> > > > > > >> > > > a
> > > >> > > > > > >> > > > > >> cost:
> > > >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is
> > > making
> > > >> it
> > > >> > > > harder
> > > >> > > > > > for
> > > >> > > > > > >> > our
> > > >> > > > > > >> > > > > users
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>> pick
> > > >> > > > > > >> > > > > >> > >>>>>> up
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It
> > > also
> > > >> > makes
> > > >> > > > it
> > > >> > > > > > >> > difficult
> > > >> > > > > > >> > > > for
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about
> > > what
> > > >> the
> > > >> > > > > > >> > > characteristics
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> > > >> characteristics
> > > >> > > > change
> > > >> > > > > > >> > > depending
> > > >> > > > > > >> > > > on
> > > >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are
> > > most
> > > >> > > visible
> > > >> > > > > in
> > > >> > > > > > the
> > > >> > > > > > >> > > > System
> > > >> > > > > > >> > > > > >> APIs.
> > > >> > > > > > >> > > > > >> > >>>>> What
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
> > > >> functional is
> > > >> > > > Kafka
> > > >> > > > > > as
> > > >> > > > > > >> its
> > > >> > > > > > >> > > > > >> > >>>>>>>>> transport
> > > >> > > > > > >> > > > > >> > >>>>>> layer.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> But
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated
> use
> > > >> cases
> > > >> > > into
> > > >> > > > > one
> > > >> > > > > > >> API:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports
> > both
> > > >> of
> > > >> > > these
> > > >> > > > > use
> > > >> > > > > > >> > cases.
> > > >> > > > > > >> > > > The
> > > >> > > > > > >> > > > > >> > >>>>>>>>> problem
> > > >> > > > > > >> > > > > >> > >>>>>> is,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we
> > > >> > > > > > >> > > > > >> > >>>>>>>>> actually want different features
> > for
> > > >> each
> > > >> > > use
> > > >> > > > > > case.
> > > >> > > > > > >> By
> > > >> > > > > > >> > > > > >> papering
> > > >> > > > > > >> > > > > >> > >>>>>>>>> over
> > > >> > > > > > >> > > > > >> > >>>>>>> these
> > > >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a
> > > single
> > > >> > API,
> > > >> > > > > we've
> > > >> > > > > > >> > > > introduced
> > > >> > > > > > >> > > > > a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > >> > > > > > >> > > > > >> > >>>>>>> leaky
> > > >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really
> like
> > > in
> > > >> (2)
> > > >> > > is
> > > >> > > > to
> > > >> > > > > > have
> > > >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs
> for
> > > >> > offsets
> > > >> > > > > (like
> > > >> > > > > > >> > Kafka).
> > > >> > > > > > >> > > > > This
> > > >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > > >> > > > > > >> > > > > >> > >>>>> with
> > > >> > > > > > >> > > > > >> > >>>>>>> (1),
> > > >> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems
> > have
> > > >> > > > different
> > > >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> > > >> mailing
> > > >> > > list
> > > >> > > > > and
> > > >> > > > > > >> the
> > > >> > > > > > >> > > SQL
> > > >> > > > > > >> > > > > >> JIRAs
> > > >> > > > > > >> > > > > >> > >>>>> about
> > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> > > >> > > replayability.
> > > >> > > > > > Kafka
> > > >> > > > > > >> > > allows
> > > >> > > > > > >> > > > us
> > > >> > > > > > >> > > > > >> to
> > > >> > > > > > >> > > > > >> > >>>>> rewind
> > > >> > > > > > >> > > > > >> > >>>>>>>>> when
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other
> > > systems
> > > >> > > don't.
> > > >> > > > In
> > > >> > > > > > some
> > > >> > > > > > >> > > > cases,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > >> > > > > > >> > > > > >> > >>>>>>> return
> > > >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > > >> > > > > > >> WikipediaSystemConsumer)
> > > >> > > > > > >> > > > > because
> > > >> > > > > > >> > > > > >> > >>>>>>>>> they
> > > >> > > > > > >> > > > > >> > >>>>>> have
> > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example.
> > > Kafka
> > > >> > > > supports
> > > >> > > > > > >> > > > > partitioning,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > >> > > > > > >> > > > > >> > >>>>> many
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
> > > >> having a
> > > >> > > > single
> > > >> > > > > > >> > > partition
> > > >> > > > > > >> > > > > for
> > > >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other
> > systems
> > > >> model
> > > >> > > > > > >> partitioning
> > > >> > > > > > >> > > > > >> > >>>> differently (e.g.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is
> also
> > a
> > > >> mess.
> > > >> > > > > > Creating
> > > >> > > > > > >> > > streams
> > > >> > > > > > >> > > > > in
> > > >> > > > > > >> > > > > >> a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> > > >> impossible.
> > > >> > > As
> > > >> > > > is
> > > >> > > > > > >> > modeling
> > > >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > > >> > > > > > >> > > > > >> > >>>>> for
> > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> > > >> partitions,
> > > >> > > > > location,
> > > >> > > > > > >> > etc).
> > > >> > > > > > >> > > > The
> > > >> > > > > > >> > > > > >> > >>>>>>>>> list
> > > >> > > > > > >> > > > > >> > >>>>> goes
> > > >> > > > > > >> > > > > >> > >>>>>>> on.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began
> writing
> > > >> Samza,
> > > >> > > > > Kafka's
> > > >> > > > > > >> > > consumer
> > > >> > > > > > >> > > > > and
> > > >> > > > > > >> > > > > >> > >>>>> producer
> > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature
> set.
> > > On
> > > >> the
> > > >> > > > > > >> > consumer-side,
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> > >>>>>>>>> had two
> > > >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
> > > consumer,
> > > >> or
> > > >> > > the
> > > >> > > > > > simple
> > > >> > > > > > >> > > > > consumer.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The
> > > >> > > > > > >> > > > > >> > >>>>>>> problem
> > > >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was
> > > that
> > > >> it
> > > >> > > > > > controlled
> > > >> > > > > > >> > your
> > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments,
> > and
> > > >> the
> > > >> > > order
> > > >> > > > > in
> > > >> > > > > > >> which
> > > >> > > > > > >> > > you
> > > >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > > >> > > > > > >> > > > > >> > >>>>> problem
> > > >> > > > > > >> > > > > >> > >>>>>>>>> with
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's
> > not
> > > >> > > simple.
> > > >> > > > > It's
> > > >> > > > > > >> > basic.
> > > >> > > > > > >> > > > You
> > > >> > > > > > >> > > > > >> > >>>>>>>>> end up
> > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really
> > low-level
> > > >> stuff
> > > >> > > > that
> > > >> > > > > > you
> > > >> > > > > > >> > > > > shouldn't.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> We
> > > >> > > > > > >> > > > > >> > >>>>>> spent a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > > >> > > > KafkaSystemConsumer
> > > >> > > > > > very
> > > >> > > > > > >> > > > robust.
> > > >> > > > > > >> > > > > >> It
> > > >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some
> > cool
> > > >> > > features:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering
> > and
> > > >> > > > > > prioritization.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> > > >> assignment
> > > >> > > to
> > > >> > > > > > support
> > > >> > > > > > >> > > > joins,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> global
> > > >> > > > > > >> > > > > >> > >>>>>> state
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)),
> > > etc.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> > > >> > checkpointing.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the
> time
> > > is
> > > >> > that
> > > >> > > > > these
> > > >> > > > > > >> > > features
> > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > >> > > > > > >> > > > > >> > >>>>>>> actually
> > > >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
> > > consumers
> > > >> > (not
> > > >> > > > just
> > > >> > > > > > >> Samza
> > > >> > > > > > >> > > > stream
> > > >> > > > > > >> > > > > >> > >>>>>> processors)
> > > >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like
> > > joins
> > > >> > and
> > > >> > > > > > partition
> > > >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> > > >> > conclusion.
> > > >> > > > > > They're
> > > >> > > > > > >> > > adding
> > > >> > > > > > >> > > > a
> > > >> > > > > > >> > > > > >> ton
> > > >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> > > >> consumer
> > > >> > > > > > >> > > implementation.
> > > >> > > > > > >> > > > > To a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > > >> > > > > > >> > > > > >> > >>>>> it's
> > > >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've
> > already
> > > >> done
> > > >> > > in
> > > >> > > > > > Samza.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up
> > > taking
> > > >> a
> > > >> > > very
> > > >> > > > > > similar
> > > >> > > > > > >> > > > > approach
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>> Samza's
> > > >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> > > implementation
> > > >> for
> > > >> > > > > > handling
> > > >> > > > > > >> > > offset
> > > >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> > > >> management
> > > >> > > > > feature
> > > >> > > > > > >> > stores
> > > >> > > > > > >> > > > > >> offset
> > > >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and
> allows
> > > >> you to
> > > >> > > > fetch
> > > >> > > > > > them
> > > >> > > > > > >> > > from
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a
> waste,
> > > >> since
> > > >> > we
> > > >> > > > > could
> > > >> > > > > > >> have
> > > >> > > > > > >> > > > shared
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>> work
> > > >> > > > > > >> > > > > >> > >>>>>> if
> > > >> > > > > > >> > > > > >> > >>>>>>>>> it
> > > >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
> > > >> get-go.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> > > >> radical
> > > >> > > > > > proposal.
> > > >> > > > > > >> > Samza
> > > >> > > > > > >> > > > is
> > > >> > > > > > >> > > > > >> > >>>>> relatively
> > > >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd
> venture
> > to
> > > >> say
> > > >> > > that
> > > >> > > > > > we're
> > > >> > > > > > >> > > near a
> > > >> > > > > > >> > > > > 1.0
> > > >> > > > > > >> > > > > >> > >>>>>> release.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > > >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take
> what
> > > >> we've
> > > >> > > > > learned,
> > > >> > > > > > and
> > > >> > > > > > >> > > begin
> > > >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > > >> > > > > > >> > > > > >> > >>>>>>> about
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we
> > > >> change if
> > > >> > > we
> > > >> > > > > were
> > > >> > > > > > >> > > starting
> > > >> > > > > > >> > > > > >> from
> > > >> > > > > > >> > > > > >> > >>>>>> scratch?
> > > >> > > > > > >> > > > > >> > >>>>>>>>> My
> > > >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the
> > *only*
> > > >> way
> > > >> > to
> > > >> > > > run
> > > >> > > > > > Samza
> > > >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all
> > direct
> > > >> > > > > dependences
> > > >> > > > > > on
> > > >> > > > > > >> > > YARN,
> > > >> > > > > > >> > > > > >> Mesos,
> > > >> > > > > > >> > > > > >> > >>>> etc.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to
> > support
> > > >> only
> > > >> > > > Kafka
> > > >> > > > > > as
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > > stream
> > > >> > > > > > >> > > > > >> > >>>>>> processing
> > > >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics,
> > > logging,
> > > >> > > > > > >> serialization,
> > > >> > > > > > >> > > and
> > > >> > > > > > >> > > > > >> > >>>>>>>>> config
> > > >> > > > > > >> > > > > >> > >>>>>>> systems,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues
> > > that
> > > >> I
> > > >> > > > > outlined
> > > >> > > > > > >> > above.
> > > >> > > > > > >> > > It
> > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > >> > > > > > >> > > > > >> > >>>>> also
> > > >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base
> pretty
> > > >> > > > dramatically.
> > > >> > > > > > >> > > Supporting
> > > >> > > > > > >> > > > > >> only
> > > >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will
> allow
> > > >> Samza
> > > >> > to
> > > >> > > be
> > > >> > > > > > >> executed
> > > >> > > > > > >> > > on
> > > >> > > > > > >> > > > > YARN
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > > >> > > > Marathon/Aurora),
> > > >> > > > > or
> > > >> > > > > > >> most
> > > >> > > > > > >> > > > other
> > > >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > > >> > > > > > >> > > > > >> > >>>>>>> deployment
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a
> > lot
> > > >> > easier
> > > >> > > > for
> > > >> > > > > > new
> > > >> > > > > > >> > > users.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without
> > > YARN.
> > > >> > The
> > > >> > > > drop
> > > >> > > > > > in
> > > >> > > > > > >> > > mailing
> > > >> > > > > > >> > > > > >> list
> > > >> > > > > > >> > > > > >> > >>>>>> traffic
> > > >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
> > > >> overdue to
> > > >> > > me.
> > > >> > > > > The
> > > >> > > > > > >> > > reality
> > > >> > > > > > >> > > > > is,
> > > >> > > > > > >> > > > > >> > >>>>> everyone
> > > >> > > > > > >> > > > > >> > >>>>>>>>> that
> > > >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
> > > >> Kafka.
> > > >> > We
> > > >> > > > > > basically
> > > >> > > > > > >> > > > require
> > > >> > > > > > >> > > > > >> it
> > > >> > > > > > >> > > > > >> > >>>>>> already
> > > >> > > > > > >> > > > > >> > >>>>>>> in
> > > >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work.
> > > Those
> > > >> > that
> > > >> > > > are
> > > >> > > > > > >> using
> > > >> > > > > > >> > > > other
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > >> > > > > > >> > > > > >> > >>>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest
> into
> > > >> Kafka
> > > >> > > (1),
> > > >> > > > > and
> > > >> > > > > > >> then
> > > >> > > > > > >> > > > they
> > > >> > > > > > >> > > > > do
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
> > > >> already
> > > >> > > > > > discussion (
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> >
> > > >> > > > > >
> > > >> > >
> > > >>
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > >> > > > > > >> > > > > >> > >>>>> 767
> > > >> > > > > > >> > > > > >> > >>>>>>>>> )
> > > >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into
> > > Kafka
> > > >> > > > extremely
> > > >> > > > > > >> easy.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple
> > with
> > > >> > Kafka,
> > > >> > > > we
> > > >> > > > > > can
> > > >> > > > > > >> > > > leverage
> > > >> > > > > > >> > > > > a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > >> > > > > > >> > > > > >> > >>>>>>> their
> > > >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> > > >> maintain
> > > >> > > our
> > > >> > > > > own
> > > >> > > > > > >> > config,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > > >> > > > > > >> > > > > >> > >>>>> etc.
> > > >> > > > > > >> > > > > >> > >>>>>>> We
> > > >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same
> libraries,
> > > and
> > > >> > make
> > > >> > > > them
> > > >> > > > > > >> > better.
> > > >> > > > > > >> > > > This
> > > >> > > > > > >> > > > > >> > >>>>>>>>> will
> > > >> > > > > > >> > > > > >> >
> > > >> ...
> > > >>
> > > >> [Message clipped]
> > > >
> > > >
> > > >
> > >
> >
> >
> >
> > --
> > Jordan Shaw
> > Full Stack Software Engineer
> > PubNub Inc
> > 1045 17th St
> > San Francisco, CA 94107
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
Hi, Garry,

Just want to chime in to state our experience in LinkedIn. In LinkedIn, we
have a lot of aggregation/transformation stream processing jobs that falls
into the "transformation" category. That's also the motivation for us to
develop the SQL layer on top of streams to allow easy programming model for
data transformation on streams. Ingestion from wide-range of sources and
egress to some serving tier are important, but I would argue that w/o the
"transformation" in between, there is not much value added by stream
processing.

Just my 2-cents.

On Mon, Jul 13, 2015 at 12:56 PM, Garry Turkington <
g.turkington@improvedigital.com> wrote:

> Hi,
>
> I'm also supportive of Jay's option 5. There is a risk the "transformer
> API" -- I'd have preferred Metamorphosis but it's too hard to type! --
> takes on a life of its own and we end up with two very different things but
> given how good the Kafka community has been at introducing new producer and
> consumer clients and giving very clear guidance on when they are production
> ready this is a danger I believe can be managed. It'd also be excellent to
> get some working code to beat around the notions of stream processing atop
> a system with transacdtional messages.
>
> On the question of whether to keep or deprecate SystemConsumer/Producer I
> believe we need get a better understanding over the next while of just what
> the Samza community is looking for in such connectivity. For my own use
> cases I have been looking to add additional implementations primarily to
> use Samza as the data ingress and egress component around Kafka. Writing
> external clients that require their own reliability and scalability
> management gets old real fast and pushing this into a simple Samza job that
> reads from system X and pushes into Kafka (or vice versa) was the obvious
> choice for me in the current model. For this type of usage though copycat
> is likely much superior (obviously needs proven) and the question then is
> if most Samza users look to the system implementations to also act as a
> front-end into Kafka or if significant usage is indeed intended to have the
> alternative systems as the primary message source. That understanding will
> I think give much clarity in just what value the abstraction overhead of
> the current model brings.
>
> Garry
>
> -----Original Message-----
> From: Yan Fang [mailto:yanfang724@gmail.com]
> Sent: 13 July 2015 19:58
> To: dev@samza.apache.org
> Subject: Re: Thoughts and obesrvations on Samza
>
> I am leaning to Jay's fifth approach. It is not radical and gives us some
> time to see the outcome.
>
> In addition, I would suggest:
>
> 1) Keep the SystemConsumer/SystemProducer API. Because current
> SystemConsumer/SystemProducer API satisfies the usage (From Joardan, and
> even Garry's feedback) and is not so broken that we want to deprecate it.
> Though there are some issues in implemnting the Kinesis, they are not
> unfixable. Nothing should prevent Samza, as a stream processing system, to
> support other systems. In addition, there already are some systems
> exiting besides Kafka: ElasticSearch (committed to the master), HDFS
> (patch-available), S3( from the mailing list), Kinesis (developing in
> another repository), ActiveMQ (in two months). We may want to see how those
> go before we "kill" them.
>
> 2) Can have some Samza devs involved in Kafka's "transformer" client API.
> This can not only help the future integration (if any) much easier, because
> they have knowledge about both systems, but also good for Kafka's
> community, because Samza devs have the streaming process experience that
> Kafka devs may miss.
>
> 3) Samza's partition management system may still support other systems.
> Though the partition management logic in samza-kafka will be moved to
> Kafka, its still useful for other systems that do not have the partition
> management layer.
>
> 4) Start sharing the docs/websites and using the same terminology (though
> do not know how to do this exactly. :). This will reduce the future
> confusion and does not hurt Samza's independency.
>
> In my opinion, Samza, as a standalone project, still can (and already)
> heavily replying on Kafka, and even more tuned for Kafka-specific usage.
> Kafka, also can embed Samza in the document, I do not see anything prevent
> doing this.
>
> Thanks,
>
> Fang, Yan
> yanfang724@gmail.com
>
> On Mon, Jul 13, 2015 at 11:25 AM, Jordan Shaw <jo...@pubnub.com> wrote:
>
> > Jay,
> > I think doing this iteratively in smaller chunks is a better way to go as
> > new issues arise. As Navina said Kafka is a "stream system" and Samza is
> a
> > "stream processor" and those two ideas should be mutually exclusive.
> >
> > -Jordan
> >
> > On Mon, Jul 13, 2015 at 10:06 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hmm, thought about this more. Maybe this is just too much too quick.
> > > Overall I think there is some enthusiasm for the proposal but it's not
> > > really unanimous enough to make any kind of change this big cleanly.
> The
> > > board doesn't really like the merging stuff, user's are concerned about
> > > compatibility, I didn't feel there was unanimous agreement on dropping
> > > SystemConsumer, etc. Even if this is the right end state to get to,
> > > probably trying to push all this through at once isn't the right way to
> > do
> > > it.
> > >
> > > So let me propose a kind of fifth (?) option which I think is less
> > dramatic
> > > and let's things happen gradually. I think this is kind of like
> combining
> > > the first part of Yi's proposal and Jakob's third option, leaving the
> > rest
> > > to be figured out incrementally:
> > >
> > > Option 5: We continue the prototype I shared and propose that as a kind
> > of
> > > "transformer" client API in Kafka. This isn't really a full-fledged
> > stream
> > > processing layer, more like a supped up consumer api for munging
> topics.
> > > This would let us figure out some of the technical bits, how to do this
> > on
> > > Kafka's group management features, how to integrate the txn feature to
> do
> > > the exactly-once stuff in these transformations, and get all this stuff
> > > solid. This api would have valid uses in it's own right, especially
> when
> > > your transformation will be embedded inside an existing service or
> > > application which isn't possible with Samza (or other existing systems
> > that
> > > I know of).
> > >
> > > Independently we can iterate on some of the ideas of the original
> > proposal
> > > individually and figure out how (if at all) to make use of this
> > > functionality. This can be done bit-by-bit:
> > > - Could be that the existing StreamTask API ends up wrapping this
> > > - Could end up exposed directly in Samza as Yi proposed
> > > - Could be that just the lower-level group-management stuff get's used,
> > and
> > > in this case it could be either just for standalone mode, or always
> > > - Could be that it stays as-is
> > >
> > > The advantage of this is it is lower risk...we basically don't have to
> > make
> > > 12 major decisions all at once that kind of hinge on what amounts to a
> > > pretty aggressive rewrite. The disadvantage of this is it is a bit more
> > > confusing as all this is getting figured out.
> > >
> > > As with some of the other stuff, this would require a further
> discussion
> > in
> > > the Kafka community if people do like this approach.
> > >
> > > Thoughts?
> > >
> > > -Jay
> > >
> > >
> > >
> > >
> > > On Sun, Jul 12, 2015 at 10:52 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hey Chris,
> > > >
> > > > Yeah, I'm obviously in favor of this.
> > > >
> > > > The sub-project approach seems the ideal way to take a graceful step
> in
> > > > this direction, so I will ping the board folks and see why they are
> > > > discouraged, it would be good to understand that. If we go that route
> > we
> > > > would need to do a similar discussion in the Kafka list (but makes
> > sense
> > > to
> > > > figure out first if it is what Samza wants).
> > > >
> > > > Irrespective of how it's implemented, though, to me the important
> > things
> > > > are the following:
> > > > 1. Unify the website, config, naming, docs, metrics, etc--basically
> fix
> > > > the product experience so the "stream" and the "processing" feel
> like a
> > > > single user experience and brand. This seems minor but I think is a
> > > really
> > > > big deal.
> > > > 2. Make "standalone" mode a first class citizen and have a real
> > technical
> > > > plan to be able to support cluster managers other than YARN.
> > > > 3. Make the config and out-of-the-box experience more usable
> > > >
> > > > I think that prototype gives a practical example of how 1-3 could be
> > done
> > > > and we should pursue it. This is a pretty radical change, so I
> wouldn't
> > > be
> > > > shocked if people didn't want to take a step like that.
> > > >
> > > > Maybe it would make sense to see if people are on board with that
> > general
> > > > idea, and then try to get some advice on sub-projects in parallel and
> > > nail
> > > > down those details?
> > > >
> > > > -Jay
> > > >
> > > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
> > criccomini@apache.org>
> > > > wrote:
> > > >
> > > >> Hey all,
> > > >>
> > > >> I want to start by saying that I'm absolutely thrilled to be a part
> of
> > > >> this
> > > >> community. The amount of level-headed, thoughtful, educated
> discussion
> > > >> that's gone on over the past ~10 days is overwhelming. Wonderful.
> > > >>
> > > >> It seems like discussion is waning a bit, and we've reached some
> > > >> conclusions. There are several key emails in this threat, which I
> want
> > > to
> > > >> call out:
> > > >>
> > > >> 1. Jakob's summary of the three potential ways forward.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > > >> 2. Julian's call out that we should be focusing on community over
> > code.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > > >> 3. Martin's summary about the benefits of merging communities.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > > >> 4. Jakob's comments about the distinction between community and code
> > > >> paths.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> > > >>
> > > >> I agree with the comments on all of these emails. I think Martin's
> > > summary
> > > >> of his position aligns very closely with my own. To that end, I
> think
> > we
> > > >> should get concrete about what the proposal is, and call a vote on
> it.
> > > >> Given that Jay, Martin, and I seem to be aligning fairly closely, I
> > > think
> > > >> we should start with:
> > > >>
> > > >> 1. [community] Make Samza a subproject of Kafka.
> > > >> 2. [community] Make all Samza PMC/committers committers of the
> > > subproject.
> > > >> 3. [community] Migrate Samza's website/documentation into Kafka's.
> > > >> 4. [code] Have the Samza community and the Kafka community start a
> > > >> from-scratch reboot together in the new Kafka subproject. We can
> > > >> borrow/copy &  paste significant chunks of code from Samza's code
> > base.
> > > >> 5. [code] The subproject would intentionally eliminate support for
> > both
> > > >> other streaming systems and all deployment systems.
> > > >> 6. [code] Attempt to provide a bridge from our SystemConsumer to
> > KIP-26
> > > >> (copy cat)
> > > >> 7. [code] Attempt to provide a bridge from the new subproject's
> > > processor
> > > >> interface to our legacy StreamTask interface.
> > > >> 8. [code/community] Sunset Samza as a TLP when we have a working
> Kafka
> > > >> subproject that has a fault-tolerant container with state
> management.
> > > >>
> > > >> It's likely that (6) and (7) won't be fully drop-in. Still, the
> closer
> > > we
> > > >> can get, the better it's going to be for our existing community.
> > > >>
> > > >> One thing that I didn't touch on with (2) is whether any Samza PMC
> > > members
> > > >> should be rolled into Kafka PMC membership as well (though, Jay and
> > > Jakob
> > > >> are already PMC members on both). I think that Samza's community
> > > deserves
> > > >> a
> > > >> voice on the PMC, so I'd propose that we roll at least a few PMC
> > members
> > > >> into the Kafka PMC, but I don't have a strong framework for which
> > people
> > > >> to
> > > >> pick.
> > > >>
> > > >> Before (8), I think that Samza's TLP can continue to commit bug
> fixes
> > > and
> > > >> patches as it sees fit, provided that we openly communicate that we
> > > won't
> > > >> necessarily migrate new features to the new subproject, and that the
> > TLP
> > > >> will be shut down after the migration to the Kafka subproject
> occurs.
> > > >>
> > > >> Jakob, I could use your guidance here about about how to achieve
> this
> > > from
> > > >> an Apache process perspective (sorry).
> > > >>
> > > >> * Should I just call a vote on this proposal?
> > > >> * Should it happen on dev or private?
> > > >> * Do committers have binding votes, or just PMC?
> > > >>
> > > >> Having trouble finding much detail on the Apache wikis. :(
> > > >>
> > > >> Cheers,
> > > >> Chris
> > > >>
> > > >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com>
> > wrote:
> > > >>
> > > >> > Thanks, Jay. This argument persuaded me actually. :)
> > > >> >
> > > >> > Fang, Yan
> > > >> > yanfang724@gmail.com
> > > >> >
> > > >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io>
> > wrote:
> > > >> >
> > > >> > > Hey Yan,
> > > >> > >
> > > >> > > Yeah philosophically I think the argument is that you should
> > capture
> > > >> the
> > > >> > > stream in Kafka independent of the transformation. This is
> > > obviously a
> > > >> > > Kafka-centric view point.
> > > >> > >
> > > >> > > Advantages of this:
> > > >> > > - In practice I think this is what e.g. Storm people often end
> up
> > > >> doing
> > > >> > > anyway. You usually need to throttle any access to a live
> serving
> > > >> > database.
> > > >> > > - Can have multiple subscribers and they get the same thing
> > without
> > > >> > > additional load on the source system.
> > > >> > > - Applications can tap into the stream if need be by
> subscribing.
> > > >> > > - You can debug your transformation by tailing the Kafka topic
> > with
> > > >> the
> > > >> > > console consumer
> > > >> > > - Can tee off the same data stream for batch analysis or Lambda
> > arch
> > > >> > style
> > > >> > > re-processing
> > > >> > >
> > > >> > > The disadvantage is that it will use Kafka resources. But the
> idea
> > > is
> > > >> > > eventually you will have multiple subscribers to any data source
> > (at
> > > >> > least
> > > >> > > for monitoring) so you will end up there soon enough anyway.
> > > >> > >
> > > >> > > Down the road the technical benefit is that I think it gives us
> a
> > > good
> > > >> > path
> > > >> > > towards end-to-end exactly once semantics from source to
> > > destination.
> > > >> > > Basically the connectors need to support idempotence when
> talking
> > to
> > > >> > Kafka
> > > >> > > and we need the transactional write feature in Kafka to make the
> > > >> > > transformation atomic. This is actually pretty doable if you
> > > separate
> > > >> > > connector=>kafka problem from the generic transformations which
> > are
> > > >> > always
> > > >> > > kafka=>kafka. However I think it is quite impossible to do in a
> > > >> > all_things
> > > >> > > => all_things environment. Today you can say "well the semantics
> > of
> > > >> the
> > > >> > > Samza APIs depend on the connectors you use" but it is actually
> > > worse
> > > >> > then
> > > >> > > that because the semantics actually depend on the pairing of
> > > >> > connectors--so
> > > >> > > not only can you probably not get a usable "exactly once"
> > guarantee
> > > >> > > end-to-end it can actually be quite hard to reverse engineer
> what
> > > >> > property
> > > >> > > (if any) your end-to-end flow has if you have heterogenous
> > systems.
> > > >> > >
> > > >> > > -Jay
> > > >> > >
> > > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <yanfang724@gmail.com
> >
> > > >> wrote:
> > > >> > >
> > > >> > > > {quote}
> > > >> > > > maintained in a separate repository and retaining the existing
> > > >> > > > committership but sharing as much else as possible (website,
> > etc)
> > > >> > > > {quote}
> > > >> > > >
> > > >> > > > Overall, I agree on this idea. Now the question is more about
> > "how
> > > >> to
> > > >> > do
> > > >> > > > it".
> > > >> > > >
> > > >> > > > On the other hand, one thing I want to point out is that, if
> we
> > > >> decide
> > > >> > to
> > > >> > > > go this way, how do we want to support
> > > >> > > > otherSystem-transformation-otherSystem use case?
> > > >> > > >
> > > >> > > > Basically, there are four user groups here:
> > > >> > > >
> > > >> > > > 1. Kafka-transformation-Kafka
> > > >> > > > 2. Kafka-transformation-otherSystem
> > > >> > > > 3. otherSystem-transformation-Kafka
> > > >> > > > 4. otherSystem-transformation-otherSystem
> > > >> > > >
> > > >> > > > For group 1, they can easily use the new Samza library to
> > achieve.
> > > >> For
> > > >> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka
> > or
> > > >> > Kafka->
> > > >> > > > transformation -> copyCat.
> > > >> > > >
> > > >> > > > The problem is for group 4. Do we want to abandon this or
> still
> > > >> support
> > > >> > > it?
> > > >> > > > Of course, this use case can be achieved by using copyCat ->
> > > >> > > transformation
> > > >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> > > persuade
> > > >> > them
> > > >> > > to
> > > >> > > > do this long chain. If yes, it will also be a win for Kafka
> too.
> > > Or
> > > >> if
> > > >> > > > there is no one in this community actually doing this so far,
> > > maybe
> > > >> ok
> > > >> > to
> > > >> > > > not support the group 4 directly.
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > >
> > > >> > > > Fang, Yan
> > > >> > > > yanfang724@gmail.com
> > > >> > > >
> > > >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <jay@confluent.io
> >
> > > >> wrote:
> > > >> > > >
> > > >> > > > > Yeah I agree with this summary. I think there are kind of
> two
> > > >> > questions
> > > >> > > > > here:
> > > >> > > > > 1. Technically does alignment/reliance on Kafka make sense
> > > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
> > alignment
> > > >> with
> > > >> > > > Kafka
> > > >> > > > > make sense
> > > >> > > > >
> > > >> > > > > Personally I do think both of these things would be really
> > > >> valuable,
> > > >> > > and
> > > >> > > > > would dramatically alter the trajectory of the project.
> > > >> > > > >
> > > >> > > > > My preference would be to see if people can mostly agree on
> a
> > > >> > direction
> > > >> > > > > rather than splintering things off. From my point of view
> the
> > > >> ideal
> > > >> > > > outcome
> > > >> > > > > of all the options discussed would be to make Samza a
> closely
> > > >> aligned
> > > >> > > > > subproject, maintained in a separate repository and
> retaining
> > > the
> > > >> > > > existing
> > > >> > > > > committership but sharing as much else as possible (website,
> > > >> etc). No
> > > >> > > > idea
> > > >> > > > > about how these things work, Jacob, you probably know more.
> > > >> > > > >
> > > >> > > > > No discussion amongst the Kafka folks has happened on this,
> > but
> > > >> > likely
> > > >> > > we
> > > >> > > > > should figure out what the Samza community actually wants
> > first.
> > > >> > > > >
> > > >> > > > > I admit that this is a fairly radical departure from how
> > things
> > > >> are.
> > > >> > > > >
> > > >> > > > > If that doesn't fly, I think, yeah we could leave Samza as
> it
> > is
> > > >> and
> > > >> > do
> > > >> > > > the
> > > >> > > > > more radical reboot inside Kafka. From my point of view that
> > > does
> > > >> > leave
> > > >> > > > > things in a somewhat confusing state since now there are two
> > > >> stream
> > > >> > > > > processing systems more or less coupled to Kafka in large
> part
> > > >> made
> > > >> > by
> > > >> > > > the
> > > >> > > > > same people. But, arguably that might be a cleaner way to
> make
> > > the
> > > >> > > > cut-over
> > > >> > > > > and perhaps less risky for Samza community since if it works
> > > >> people
> > > >> > can
> > > >> > > > > switch and if it doesn't nothing will have changed. Dunno,
> how
> > > do
> > > >> > > people
> > > >> > > > > feel about this?
> > > >> > > > >
> > > >> > > > > -Jay
> > > >> > > > >
> > > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> > > jghoman@gmail.com>
> > > >> > > wrote:
> > > >> > > > >
> > > >> > > > > > >  This leads me to thinking that merging projects and
> > > >> communities
> > > >> > > > might
> > > >> > > > > > be a good idea: with the union of experience from both
> > > >> communities,
> > > >> > > we
> > > >> > > > > will
> > > >> > > > > > probably build a better system that is better for users.
> > > >> > > > > > Is this what's being proposed though? Merging the projects
> > > seems
> > > >> > like
> > > >> > > > > > a consequence of at most one of the three directions under
> > > >> > > discussion:
> > > >> > > > > > 1) Samza 2.0: The Samza community relies more heavily on
> > Kafka
> > > >> for
> > > >> > > > > > configuration, etc. (to a greater or lesser extent to be
> > > >> > determined)
> > > >> > > > > > but the Samza community would not automatically merge
> withe
> > > >> Kafka
> > > >> > > > > > community (the Phoenix/HBase example is a good one here).
> > > >> > > > > > 2) Samza Reboot: The Samza community continues to exist
> > with a
> > > >> > > limited
> > > >> > > > > > project scope, but similarly would not need to be part of
> > the
> > > >> Kafka
> > > >> > > > > > community (ie given committership) to progress.  Here,
> maybe
> > > the
> > > >> > > Samza
> > > >> > > > > > team would become a subproject of Kafka (the Board frowns
> on
> > > >> > > > > > subprojects at the moment, so I'm not sure if that's even
> > > >> > feasible),
> > > >> > > > > > but that would not be required.
> > > >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this
> option
> > > the
> > > >> > Kafka
> > > >> > > > > > team builds its own streaming library, possibly off of
> Jay's
> > > >> > > > > > prototype, which has not direct lineage to the Samza team.
> > > >> There's
> > > >> > > no
> > > >> > > > > > reason for the Kafka team to bring in the Samza team.
> > > >> > > > > >
> > > >> > > > > > Is the Kafka community on board with this?
> > > >> > > > > >
> > > >> > > > > > To be clear, all three options under discussion are
> > > interesting,
> > > >> > > > > > technically valid and likely healthy directions for the
> > > project.
> > > >> > > > > > Also, they are not mutually exclusive.  The Samza
> community
> > > >> could
> > > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
> > community
> > > >> went
> > > >> > > > > > forward with 'Hey Samza!'  My points above are directed
> > > >> entirely at
> > > >> > > > > > the community aspect of these choices.
> > > >> > > > > > -Jakob
> > > >> > > > > >
> > > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> > > roger.hoover@gmail.com>
> > > >> > > wrote:
> > > >> > > > > > > That's great.  Thanks, Jay.
> > > >> > > > > > >
> > > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> > > jay@confluent.io>
> > > >> > > wrote:
> > > >> > > > > > >
> > > >> > > > > > >> Yeah totally agree. I think you have this issue even
> > today,
> > > >> > right?
> > > >> > > > > I.e.
> > > >> > > > > > if
> > > >> > > > > > >> you need to make a simple config change and you're
> > running
> > > in
> > > >> > YARN
> > > >> > > > > today
> > > >> > > > > > >> you end up bouncing the job which then rebuilds state.
> I
> > > >> think
> > > >> > the
> > > >> > > > fix
> > > >> > > > > > is
> > > >> > > > > > >> exactly what you described which is to have a long
> > timeout
> > > on
> > > >> > > > > partition
> > > >> > > > > > >> movement for stateful jobs so that if a job is just
> > getting
> > > >> > > bounced,
> > > >> > > > > and
> > > >> > > > > > >> the cluster manager (or admin) is smart enough to
> restart
> > > it
> > > >> on
> > > >> > > the
> > > >> > > > > same
> > > >> > > > > > >> host when possible, it can optimistically reuse any
> > > existing
> > > >> > state
> > > >> > > > it
> > > >> > > > > > finds
> > > >> > > > > > >> on disk (if it is valid).
> > > >> > > > > > >>
> > > >> > > > > > >> So in this model the charter of the CM is to place
> > > processes
> > > >> as
> > > >> > > > > > stickily as
> > > >> > > > > > >> possible and to restart or re-place failed processes.
> The
> > > >> > charter
> > > >> > > of
> > > >> > > > > the
> > > >> > > > > > >> partition management system is to control the
> assignment
> > of
> > > >> work
> > > >> > > to
> > > >> > > > > > these
> > > >> > > > > > >> processes. The nice thing about this is that the work
> > > >> > assignment,
> > > >> > > > > > timeouts,
> > > >> > > > > > >> behavior, configs, and code will all be the same across
> > all
> > > >> > > cluster
> > > >> > > > > > >> managers.
> > > >> > > > > > >>
> > > >> > > > > > >> So I think that prototype would actually give you
> exactly
> > > >> what
> > > >> > you
> > > >> > > > > want
> > > >> > > > > > >> today for any cluster manager (or manual placement +
> > > restart
> > > >> > > script)
> > > >> > > > > > that
> > > >> > > > > > >> was sticky in terms of host placement since there is
> > > already
> > > >> a
> > > >> > > > > > configurable
> > > >> > > > > > >> partition movement timeout and task-by-task state reuse
> > > with
> > > >> a
> > > >> > > check
> > > >> > > > > on
> > > >> > > > > > >> state validity.
> > > >> > > > > > >>
> > > >> > > > > > >> -Jay
> > > >> > > > > > >>
> > > >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > > >> > > > roger.hoover@gmail.com
> > > >> > > > > >
> > > >> > > > > > >> wrote:
> > > >> > > > > > >>
> > > >> > > > > > >> > That would be great to let Kafka do as much heavy
> > lifting
> > > >> as
> > > >> > > > > possible
> > > >> > > > > > and
> > > >> > > > > > >> > make it easier for other languages to implement Samza
> > > apis.
> > > >> > > > > > >> >
> > > >> > > > > > >> > One thing to watch out for is the interplay between
> > > Kafka's
> > > >> > > group
> > > >> > > > > > >> > management and the external scheduler/process
> manager's
> > > >> fault
> > > >> > > > > > tolerance.
> > > >> > > > > > >> > If a container dies, the Kafka group membership
> > protocol
> > > >> will
> > > >> > > try
> > > >> > > > to
> > > >> > > > > > >> assign
> > > >> > > > > > >> > it's tasks to other containers while at the same time
> > the
> > > >> > > process
> > > >> > > > > > manager
> > > >> > > > > > >> > is trying to relaunch the container.  Without some
> > > >> > consideration
> > > >> > > > for
> > > >> > > > > > this
> > > >> > > > > > >> > (like a configurable amount of time to wait before
> > Kafka
> > > >> > alters
> > > >> > > > the
> > > >> > > > > > group
> > > >> > > > > > >> > membership), there may be thrashing going on which is
> > > >> > especially
> > > >> > > > bad
> > > >> > > > > > for
> > > >> > > > > > >> > containers with large amounts of local state.
> > > >> > > > > > >> >
> > > >> > > > > > >> > Someone else pointed this out already but I thought
> it
> > > >> might
> > > >> > be
> > > >> > > > > worth
> > > >> > > > > > >> > calling out again.
> > > >> > > > > > >> >
> > > >> > > > > > >> > Cheers,
> > > >> > > > > > >> >
> > > >> > > > > > >> > Roger
> > > >> > > > > > >> >
> > > >> > > > > > >> >
> > > >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> > > >> jay@confluent.io>
> > > >> > > > > wrote:
> > > >> > > > > > >> >
> > > >> > > > > > >> > > Hey Roger,
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > I couldn't agree more. We spent a bunch of time
> > talking
> > > >> to
> > > >> > > > people
> > > >> > > > > > and
> > > >> > > > > > >> > that
> > > >> > > > > > >> > > is exactly the stuff we heard time and again. What
> > > makes
> > > >> it
> > > >> > > > hard,
> > > >> > > > > of
> > > >> > > > > > >> > > course, is that there is some tension between
> > > >> compatibility
> > > >> > > with
> > > >> > > > > > what's
> > > >> > > > > > >> > > there now and making things better for new users.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > I also strongly agree with the importance of
> > > >> multi-language
> > > >> > > > > > support. We
> > > >> > > > > > >> > are
> > > >> > > > > > >> > > talking now about Java, but for application
> > development
> > > >> use
> > > >> > > > cases
> > > >> > > > > > >> people
> > > >> > > > > > >> > > want to work in whatever language they are using
> > > >> elsewhere.
> > > >> > I
> > > >> > > > > think
> > > >> > > > > > >> > moving
> > > >> > > > > > >> > > to a model where Kafka itself does the group
> > > membership,
> > > >> > > > lifecycle
> > > >> > > > > > >> > control,
> > > >> > > > > > >> > > and partition assignment has the advantage of
> putting
> > > all
> > > >> > that
> > > >> > > > > > complex
> > > >> > > > > > >> > > stuff behind a clean api that the clients are
> already
> > > >> going
> > > >> > to
> > > >> > > > be
> > > >> > > > > > >> > > implementing for their consumer, so the added
> > > >> functionality
> > > >> > > for
> > > >> > > > > > stream
> > > >> > > > > > >> > > processing beyond a consumer becomes very minor.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > -Jay
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > > >> > > > > > roger.hoover@gmail.com>
> > > >> > > > > > >> > > wrote:
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > > Metamorphosis...nice. :)
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > This has been a great discussion.  As a user of
> > Samza
> > > >> > who's
> > > >> > > > > > recently
> > > >> > > > > > >> > > > integrated it into a relatively large
> > organization, I
> > > >> just
> > > >> > > > want
> > > >> > > > > to
> > > >> > > > > > >> add
> > > >> > > > > > >> > > > support to a few points already made.
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
> > > >> currently
> > > >> > > > exists
> > > >> > > > > > that
> > > >> > > > > > >> > I've
> > > >> > > > > > >> > > > experienced are:
> > > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
> > environments
> > > >> > where
> > > >> > > > > Puppet
> > > >> > > > > > >> > would
> > > >> > > > > > >> > > do
> > > >> > > > > > >> > > > just fine but it was the only mechanism to get
> > fault
> > > >> > > > tolerance.
> > > >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> > > >> configuring
> > > >> > > most
> > > >> > > > > of
> > > >> > > > > > the
> > > >> > > > > > >> > job
> > > >> > > > > > >> > > > in code rather than config files.  In general, I
> > > think
> > > >> the
> > > >> > > > goal
> > > >> > > > > > >> should
> > > >> > > > > > >> > be
> > > >> > > > > > >> > > > to make it harder to make mistakes, especially of
> > the
> > > >> kind
> > > >> > > > where
> > > >> > > > > > the
> > > >> > > > > > >> > code
> > > >> > > > > > >> > > > expects something and the config doesn't match.
> > The
> > > >> > current
> > > >> > > > > > config
> > > >> > > > > > >> is
> > > >> > > > > > >> > > > quite intricate and error-prone.  For example,
> the
> > > >> > > application
> > > >> > > > > > logic
> > > >> > > > > > >> > may
> > > >> > > > > > >> > > > depend on bootstrapping a topic but rather than
> > > >> asserting
> > > >> > > that
> > > >> > > > > in
> > > >> > > > > > the
> > > >> > > > > > >> > > code,
> > > >> > > > > > >> > > > you have to rely on getting the config right.
> > > Likewise
> > > >> > with
> > > >> > > > > > serdes,
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > Java representations produced by various serdes
> > > (JSON,
> > > >> > Avro,
> > > >> > > > > etc.)
> > > >> > > > > > >> are
> > > >> > > > > > >> > > not
> > > >> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> > > >> without
> > > >> > > > > changing
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > code.   It would be nice for jobs to be able to
> > > assert
> > > >> > what
> > > >> > > > they
> > > >> > > > > > >> expect
> > > >> > > > > > >> > > > from their input topics in terms of partitioning.
> > > >> This is
> > > >> > > > > > getting a
> > > >> > > > > > >> > > little
> > > >> > > > > > >> > > > off topic but I was even thinking about creating
> a
> > > >> "Samza
> > > >> > > > config
> > > >> > > > > > >> > linter"
> > > >> > > > > > >> > > > that would sanity check a set of configs.
> > Especially
> > > >> in
> > > >> > > > > > >> organizations
> > > >> > > > > > >> > > > where config is managed by a different team than
> > the
> > > >> > > > application
> > > >> > > > > > >> > > developer,
> > > >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > > >> > > > > > >> > > > 3) Java/Scala centric - for many teams
> (especially
> > > >> > > DevOps-type
> > > >> > > > > > >> folks),
> > > >> > > > > > >> > > the
> > > >> > > > > > >> > > > pain of the Java toolchain (maven, slow builds,
> > weak
> > > >> > command
> > > >> > > > > line
> > > >> > > > > > >> > > support,
> > > >> > > > > > >> > > > configuration over convention) really inhibits
> > > >> > productivity.
> > > >> > > > As
> > > >> > > > > > more
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > more high-quality clients become available for
> > > Kafka, I
> > > >> > hope
> > > >> > > > > > they'll
> > > >> > > > > > >> > > follow
> > > >> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> > > >> proposals
> > > >> > > in
> > > >> > > > > this
> > > >> > > > > > >> > thread
> > > >> > > > > > >> > > > but please consider other languages in the
> > ecosystem
> > > as
> > > >> > > well.
> > > >> > > > > > From
> > > >> > > > > > >> > what
> > > >> > > > > > >> > > > I've heard, Spark has more Python users than
> > > >> Java/Scala.
> > > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > >
> > > >> > > > > > >> >
> > > >> > > > > > >>
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > >> > > > > > >> > > > and are working on a Yeoman generator
> > > >> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> > > >> > > Jython/Samza
> > > >> > > > > > >> projects
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > alleviate some of the pain)
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > I also want to underscore Jay's point about
> > improving
> > > >> the
> > > >> > > user
> > > >> > > > > > >> > > experience.
> > > >> > > > > > >> > > > That's a very important factor for adoption.  I
> > think
> > > >> the
> > > >> > > goal
> > > >> > > > > > should
> > > >> > > > > > >> > be
> > > >> > > > > > >> > > to
> > > >> > > > > > >> > > > make Samza as easy to get started with as
> something
> > > >> like
> > > >> > > > > Logstash.
> > > >> > > > > > >> > > > Logstash is vastly inferior in terms of
> > capabilities
> > > to
> > > >> > > Samza
> > > >> > > > > but
> > > >> > > > > > >> it's
> > > >> > > > > > >> > > easy
> > > >> > > > > > >> > > > to get started and that makes a big difference.
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > Cheers,
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > Roger
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> > > Francisci
> > > >> > > > Morales <
> > > >> > > > > > >> > > > gdfm@apache.org> wrote:
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> > > >> Metamorphosis
> > > >> > > is
> > > >> > > > a
> > > >> > > > > > clear
> > > >> > > > > > >> > > > winner
> > > >> > > > > > >> > > > > :)
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > --
> > > >> > > > > > >> > > > > Gianmarco
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> > > >> Morales
> > > >> > <
> > > >> > > > > > >> > > gdfm@apache.org
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > wrote:
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > > Hi,
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > @Martin, thanks for you comments.
> > > >> > > > > > >> > > > > > Maybe I'm missing some important point, but I
> > > think
> > > >> > > > coupling
> > > >> > > > > > the
> > > >> > > > > > >> > > > releases
> > > >> > > > > > >> > > > > > is actually a *good* thing.
> > > >> > > > > > >> > > > > > To make an example, would it be better if the
> > MR
> > > >> and
> > > >> > > HDFS
> > > >> > > > > > >> > components
> > > >> > > > > > >> > > of
> > > >> > > > > > >> > > > > > Hadoop had different release schedules?
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > Actually, keeping the discussion in a single
> > > place
> > > >> > would
> > > >> > > > > make
> > > >> > > > > > >> > > agreeing
> > > >> > > > > > >> > > > on
> > > >> > > > > > >> > > > > > releases (and backwards compatibility) much
> > > >> easier, as
> > > >> > > > > > everybody
> > > >> > > > > > >> > > would
> > > >> > > > > > >> > > > be
> > > >> > > > > > >> > > > > > responsible for the whole codebase.
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > That said, I like the idea of absorbing
> > > samza-core
> > > >> as
> > > >> > a
> > > >> > > > > > >> > sub-project,
> > > >> > > > > > >> > > > and
> > > >> > > > > > >> > > > > > leave the fancy stuff separate.
> > > >> > > > > > >> > > > > > It probably gives 90% of the benefits we have
> > > been
> > > >> > > > > discussing
> > > >> > > > > > >> here.
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > Cheers,
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > --
> > > >> > > > > > >> > > > > > Gianmarco
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > > >> > jay.kreps@gmail.com
> > > >> > > >
> > > >> > > > > > wrote:
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > >> Hey Martin,
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> I agree coupling release schedules is a
> > > downside.
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> Definitely we can try to solve some of the
> > > >> > integration
> > > >> > > > > > problems
> > > >> > > > > > >> in
> > > >> > > > > > >> > > > > >> Confluent Platform or in other
> distributions.
> > > But
> > > >> I
> > > >> > > think
> > > >> > > > > > this
> > > >> > > > > > >> > ends
> > > >> > > > > > >> > > up
> > > >> > > > > > >> > > > > >> being really shallow. I guess I feel to
> really
> > > >> get a
> > > >> > > good
> > > >> > > > > > user
> > > >> > > > > > >> > > > > experience
> > > >> > > > > > >> > > > > >> the two systems have to kind of feel like
> part
> > > of
> > > >> the
> > > >> > > > same
> > > >> > > > > > thing
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> can't really add that in later--you can put
> > both
> > > >> in
> > > >> > the
> > > >> > > > > same
> > > >> > > > > > >> > > > > downloadable
> > > >> > > > > > >> > > > > >> tar file but it doesn't really give a very
> > > >> cohesive
> > > >> > > > > feeling.
> > > >> > > > > > I
> > > >> > > > > > >> > agree
> > > >> > > > > > >> > > > > that
> > > >> > > > > > >> > > > > >> ultimately any of the project stuff is as
> much
> > > >> social
> > > >> > > and
> > > >> > > > > > naming
> > > >> > > > > > >> > as
> > > >> > > > > > >> > > > > >> anything else--theoretically two totally
> > > >> independent
> > > >> > > > > projects
> > > >> > > > > > >> > could
> > > >> > > > > > >> > > > work
> > > >> > > > > > >> > > > > >> to
> > > >> > > > > > >> > > > > >> tightly align. In practice this seems to be
> > > quite
> > > >> > > > difficult
> > > >> > > > > > >> > though.
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> For the frameworks--totally agree it would
> be
> > > >> good to
> > > >> > > > > > maintain
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > > >> framework support with the project. In some
> > > cases
> > > >> > there
> > > >> > > > may
> > > >> > > > > > not
> > > >> > > > > > >> be
> > > >> > > > > > >> > > too
> > > >> > > > > > >> > > > > >> much
> > > >> > > > > > >> > > > > >> there since the integration gets lighter
> but I
> > > >> think
> > > >> > > > > whatever
> > > >> > > > > > >> > stubs
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> need should be included. So no I definitely
> > > wasn't
> > > >> > > trying
> > > >> > > > > to
> > > >> > > > > > >> imply
> > > >> > > > > > >> > > > > >> dropping
> > > >> > > > > > >> > > > > >> support for these frameworks, just making
> the
> > > >> > > integration
> > > >> > > > > > >> lighter
> > > >> > > > > > >> > by
> > > >> > > > > > >> > > > > >> separating process management from partition
> > > >> > > management.
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> You raise two good points we would have to
> > > figure
> > > >> out
> > > >> > > if
> > > >> > > > we
> > > >> > > > > > went
> > > >> > > > > > >> > > down
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> alignment path:
> > > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think
> the
> > > >> first
> > > >> > > > > question
> > > >> > > > > > is
> > > >> > > > > > >> > > > whether
> > > >> > > > > > >> > > > > >> some "re-branding" would be worth it. If so
> > > then I
> > > >> > > think
> > > >> > > > we
> > > >> > > > > > can
> > > >> > > > > > >> > > have a
> > > >> > > > > > >> > > > > big
> > > >> > > > > > >> > > > > >> thread on the name. I'm definitely not set
> on
> > > >> Kafka
> > > >> > > > > > Streaming or
> > > >> > > > > > >> > > Kafka
> > > >> > > > > > >> > > > > >> Streams I was just using them to be kind of
> > > >> > > > illustrative. I
> > > >> > > > > > >> agree
> > > >> > > > > > >> > > with
> > > >> > > > > > >> > > > > >> your
> > > >> > > > > > >> > > > > >> critique of these names, though I think
> people
> > > >> would
> > > >> > > get
> > > >> > > > > the
> > > >> > > > > > >> idea.
> > > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about
> how
> > to
> > > >> > > "factor"
> > > >> > > > > it.
> > > >> > > > > > >> Here
> > > >> > > > > > >> > > are
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> options I see (I could get enthusiastic
> about
> > > any
> > > >> of
> > > >> > > > them):
> > > >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> > > seperation
> > > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api
> > and
> > > >> > > > samza-core
> > > >> > > > > > is
> > > >> > > > > > >> > > > absorbed
> > > >> > > > > > >> > > > > >> almost like a third client
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> Cheers,
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> -Jay
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> > > Kleppmann <
> > > >> > > > > > >> > > > martin@kleppmann.com>
> > > >> > > > > > >> > > > > >> wrote:
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a
> > few
> > > >> > > follow-up
> > > >> > > > > > >> > comments.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka
> or
> > > >> > becoming
> > > >> > > a
> > > >> > > > > > >> > subproject:
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > reasons you mention are good. The risk I
> see
> > > is
> > > >> > that
> > > >> > > > > > release
> > > >> > > > > > >> > > > schedules
> > > >> > > > > > >> > > > > >> > become coupled to each other, which can
> slow
> > > >> > everyone
> > > >> > > > > down,
> > > >> > > > > > >> and
> > > >> > > > > > >> > > > large
> > > >> > > > > > >> > > > > >> > projects with many contributors are harder
> > to
> > > >> > manage.
> > > >> > > > > > (Jakob,
> > > >> > > > > > >> > can
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> speak
> > > >> > > > > > >> > > > > >> > from experience, having seen a wider range
> > of
> > > >> > Hadoop
> > > >> > > > > > ecosystem
> > > >> > > > > > >> > > > > >> projects?)
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > Some of the goals of a better unified
> > > developer
> > > >> > > > > experience
> > > >> > > > > > >> could
> > > >> > > > > > >> > > > also
> > > >> > > > > > >> > > > > be
> > > >> > > > > > >> > > > > >> > solved by integrating Samza nicely into a
> > > Kafka
> > > >> > > > > > distribution
> > > >> > > > > > >> > (such
> > > >> > > > > > >> > > > as
> > > >> > > > > > >> > > > > >> > Confluent's). I'm not against merging
> > projects
> > > >> if
> > > >> > we
> > > >> > > > > decide
> > > >> > > > > > >> > that's
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> way
> > > >> > > > > > >> > > > > >> > to go, just pointing out the same goals
> can
> > > >> perhaps
> > > >> > > > also
> > > >> > > > > be
> > > >> > > > > > >> > > achieved
> > > >> > > > > > >> > > > > in
> > > >> > > > > > >> > > > > >> > other ways.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
> > dependency:
> > > >> are
> > > >> > > you
> > > >> > > > > > >> proposing
> > > >> > > > > > >> > > > that
> > > >> > > > > > >> > > > > >> > Samza doesn't give any help to people
> > wanting
> > > to
> > > >> > run
> > > >> > > on
> > > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > > >> > > > > > >> > > > > >> > So the docs would basically have a link to
> > > >> Slider
> > > >> > and
> > > >> > > > > > nothing
> > > >> > > > > > >> > > else?
> > > >> > > > > > >> > > > Or
> > > >> > > > > > >> > > > > >> > would we maintain integrations with a
> bunch
> > of
> > > >> > > popular
> > > >> > > > > > >> > deployment
> > > >> > > > > > >> > > > > >> methods
> > > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts
> > to
> > > >> make
> > > >> > > > Samza
> > > >> > > > > > work
> > > >> > > > > > >> > with
> > > >> > > > > > >> > > > > >> Slider)?
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > I absolutely think it's a good idea to
> have
> > > the
> > > >> > "as a
> > > >> > > > > > library"
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > "as a
> > > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
> > > >> people
> > > >> > who
> > > >> > > > > want
> > > >> > > > > > >> them,
> > > >> > > > > > >> > > > but I
> > > >> > > > > > >> > > > > >> > think there should also be a low-friction
> > path
> > > >> for
> > > >> > > > common
> > > >> > > > > > "as
> > > >> > > > > > >> a
> > > >> > > > > > >> > > > > service"
> > > >> > > > > > >> > > > > >> > deployment methods, for which we probably
> > need
> > > >> to
> > > >> > > > > maintain
> > > >> > > > > > >> > > > > integrations.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems
> odd
> > to
> > > >> me,
> > > >> > > > > because
> > > >> > > > > > >> Kafka
> > > >> > > > > > >> > > is
> > > >> > > > > > >> > > > > all
> > > >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> > > >> Transformers"
> > > >> > > or
> > > >> > > > > > "Kafka
> > > >> > > > > > >> > > > Filters"
> > > >> > > > > > >> > > > > >> > would be more apt?
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
> > > >> (stream
> > > >> > > > > > >> transformation
> > > >> > > > > > >> > > > with
> > > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> > > >> library"
> > > >> > > bit)
> > > >> > > > > > could
> > > >> > > > > > >> > > become
> > > >> > > > > > >> > > > > >> part of
> > > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
> > > >> streaming
> > > >> > SQL
> > > >> > > > and
> > > >> > > > > > >> > > > integrations
> > > >> > > > > > >> > > > > >> with
> > > >> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> > > >> project?
> > > >> > > In
> > > >> > > > > > other
> > > >> > > > > > >> > > words,
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > would absorb the proven, stable core of
> > Samza,
> > > >> > which
> > > >> > > > > would
> > > >> > > > > > >> > become
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in
> this
> > > >> > thread.
> > > >> > > > The
> > > >> > > > > > Samza
> > > >> > > > > > >> > > > project
> > > >> > > > > > >> > > > > >> > would then target that third Kafka client
> as
> > > its
> > > >> > base
> > > >> > > > > API,
> > > >> > > > > > and
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > > >> project
> > > >> > > > > > >> > > > > >> > would be freed up to explore more
> > experimental
> > > >> new
> > > >> > > > > > horizons.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > Martin
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > > >> > > > jay.kreps@gmail.com>
> > > >> > > > > > >> wrote:
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > > Hey Martin,
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I
> > actually
> > > >> > don't
> > > >> > > > > think
> > > >> > > > > > it
> > > >> > > > > > >> > ties
> > > >> > > > > > >> > > > our
> > > >> > > > > > >> > > > > >> > hands
> > > >> > > > > > >> > > > > >> > > at all, all it does is refactor things.
> > The
> > > >> > > division
> > > >> > > > of
> > > >> > > > > > >> > > > > >> responsibility is
> > > >> > > > > > >> > > > > >> > > that Samza core is responsible for task
> > > >> > lifecycle,
> > > >> > > > > state,
> > > >> > > > > > >> and
> > > >> > > > > > >> > > > > >> partition
> > > >> > > > > > >> > > > > >> > > management (using the Kafka
> co-ordinator)
> > > but
> > > >> it
> > > >> > is
> > > >> > > > NOT
> > > >> > > > > > >> > > > responsible
> > > >> > > > > > >> > > > > >> for
> > > >> > > > > > >> > > > > >> > > packaging, configuration deployment or
> > > >> execution
> > > >> > of
> > > >> > > > > > >> processes.
> > > >> > > > > > >> > > The
> > > >> > > > > > >> > > > > >> > problem
> > > >> > > > > > >> > > > > >> > > of packaging and starting these
> processes
> > is
> > > >> > > > > > >> > > > > >> > > framework/environment-specific. This
> > leaves
> > > >> > > > individual
> > > >> > > > > > >> > > frameworks
> > > >> > > > > > >> > > > to
> > > >> > > > > > >> > > > > >> be
> > > >> > > > > > >> > > > > >> > as
> > > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you
> can
> > > get
> > > >> > > simple
> > > >> > > > > > >> stateless
> > > >> > > > > > >> > > > > >> support in
> > > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their
> off-the-shelf
> > > app
> > > >> > > > > framework
> > > >> > > > > > >> > > (Slider,
> > > >> > > > > > >> > > > > >> > Marathon,
> > > >> > > > > > >> > > > > >> > > etc). These are well known by people and
> > > have
> > > >> > nice
> > > >> > > > UIs
> > > >> > > > > > and a
> > > >> > > > > > >> > lot
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > > flexibility. I don't think they have
> node
> > > >> > affinity
> > > >> > > > as a
> > > >> > > > > > >> built
> > > >> > > > > > >> > in
> > > >> > > > > > >> > > > > >> option
> > > >> > > > > > >> > > > > >> > > (though I could be wrong). So if we want
> > > that
> > > >> we
> > > >> > > can
> > > >> > > > > > either
> > > >> > > > > > >> > wait
> > > >> > > > > > >> > > > for
> > > >> > > > > > >> > > > > >> them
> > > >> > > > > > >> > > > > >> > > to add it or do a custom framework to
> add
> > > that
> > > >> > > > feature
> > > >> > > > > > (as
> > > >> > > > > > >> > now).
> > > >> > > > > > >> > > > > >> > Obviously
> > > >> > > > > > >> > > > > >> > > if you manage things with old-school ops
> > > tools
> > > >> > > > > > >> > (puppet/chef/etc)
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> get
> > > >> > > > > > >> > > > > >> > > locality easily. The nice thing, though,
> > is
> > > >> that
> > > >> > > all
> > > >> > > > > the
> > > >> > > > > > >> samza
> > > >> > > > > > >> > > > > >> "business
> > > >> > > > > > >> > > > > >> > > logic" around partition management and
> > fault
> > > >> > > > tolerance
> > > >> > > > > > is in
> > > >> > > > > > >> > > Samza
> > > >> > > > > > >> > > > > >> core
> > > >> > > > > > >> > > > > >> > so
> > > >> > > > > > >> > > > > >> > > it is shared across frameworks and the
> > > >> framework
> > > >> > > > > specific
> > > >> > > > > > >> bit
> > > >> > > > > > >> > is
> > > >> > > > > > >> > > > > just
> > > >> > > > > > >> > > > > >> > > whether it is smart enough to try to get
> > the
> > > >> same
> > > >> > > > host
> > > >> > > > > > when
> > > >> > > > > > >> a
> > > >> > > > > > >> > > job
> > > >> > > > > > >> > > > is
> > > >> > > > > > >> > > > > >> > > restarted.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment,
> yeah
> > I
> > > >> think
> > > >> > > the
> > > >> > > > > > goal
> > > >> > > > > > >> > would
> > > >> > > > > > >> > > > be
> > > >> > > > > > >> > > > > >> (a)
> > > >> > > > > > >> > > > > >> > > actually get better alignment in user
> > > >> experience,
> > > >> > > and
> > > >> > > > > (b)
> > > >> > > > > > >> > > express
> > > >> > > > > > >> > > > > >> this in
> > > >> > > > > > >> > > > > >> > > the naming and project branding.
> > > Specifically:
> > > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for
> the
> > > >> > > > > > "transformation"
> > > >> > > > > > >> api
> > > >> > > > > > >> > > to
> > > >> > > > > > >> > > > be
> > > >> > > > > > >> > > > > >> > > discoverable in the main Kafka
> docs--i.e.
> > be
> > > >> able
> > > >> > > to
> > > >> > > > > > explain
> > > >> > > > > > >> > > when
> > > >> > > > > > >> > > > to
> > > >> > > > > > >> > > > > >> use
> > > >> > > > > > >> > > > > >> > > the consumer and when to use the stream
> > > >> > processing
> > > >> > > > > > >> > functionality
> > > >> > > > > > >> > > > and
> > > >> > > > > > >> > > > > >> lead
> > > >> > > > > > >> > > > > >> > > people into that experience.
> > > >> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza
> > 1.4.2
> > > >> (or
> > > >> > > > > > whatever)
> > > >> > > > > > >> > that
> > > >> > > > > > >> > > > has
> > > >> > > > > > >> > > > > >> both
> > > >> > > > > > >> > > > > >> > > Kafka and the stream processing part and
> > > they
> > > >> > > > actually
> > > >> > > > > > work
> > > >> > > > > > >> > > > > together.
> > > >> > > > > > >> > > > > >> > > 3. Unify the programming experience so
> the
> > > >> client
> > > >> > > and
> > > >> > > > > > Samza
> > > >> > > > > > >> > api
> > > >> > > > > > >> > > > > share
> > > >> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> > > committers
> > > >> and
> > > >> > > can
> > > >> > > > > > have a
> > > >> > > > > > >> > > > > separate
> > > >> > > > > > >> > > > > >> > repo,
> > > >> > > > > > >> > > > > >> > > but I'm actually not really sure (I
> can't
> > > >> find a
> > > >> > > > > > definition
> > > >> > > > > > >> > of a
> > > >> > > > > > >> > > > > >> > subproject
> > > >> > > > > > >> > > > > >> > > in Apache).
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > Basically at a high-level you want the
> > > >> experience
> > > >> > > to
> > > >> > > > > > "feel"
> > > >> > > > > > >> > > like a
> > > >> > > > > > >> > > > > >> single
> > > >> > > > > > >> > > > > >> > > system, not to relatively independent
> > things
> > > >> that
> > > >> > > are
> > > >> > > > > > kind
> > > >> > > > > > >> of
> > > >> > > > > > >> > > > > >> awkwardly
> > > >> > > > > > >> > > > > >> > > glued together.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > I think if we did that they having
> naming
> > or
> > > >> > > branding
> > > >> > > > > > like
> > > >> > > > > > >> > > "kafka
> > > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or
> something
> > > >> like
> > > >> > > that
> > > >> > > > > > would
> > > >> > > > > > >> > > > actually
> > > >> > > > > > >> > > > > >> do a
> > > >> > > > > > >> > > > > >> > > good job of conveying what it is. I do
> > that
> > > >> this
> > > >> > > > would
> > > >> > > > > > help
> > > >> > > > > > >> > > > adoption
> > > >> > > > > > >> > > > > >> > quite
> > > >> > > > > > >> > > > > >> > > a lot as it would correctly convey that
> > > using
> > > >> > Kafka
> > > >> > > > > > >> Streaming
> > > >> > > > > > >> > > with
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > is
> > > >> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka
> is
> > > >> pretty
> > > >> > > > > heavily
> > > >> > > > > > >> > adopted
> > > >> > > > > > >> > > > at
> > > >> > > > > > >> > > > > >> this
> > > >> > > > > > >> > > > > >> > > point.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> > > >> originally
> > > >> > > > when
> > > >> > > > > > open
> > > >> > > > > > >> > > > sourcing
> > > >> > > > > > >> > > > > >> > Samza,
> > > >> > > > > > >> > > > > >> > > however at that time Kafka was
> relatively
> > > >> unknown
> > > >> > > and
> > > >> > > > > we
> > > >> > > > > > >> > decided
> > > >> > > > > > >> > > > not
> > > >> > > > > > >> > > > > >> to
> > > >> > > > > > >> > > > > >> > do
> > > >> > > > > > >> > > > > >> > > it since we felt it would be limiting.
> > From
> > > my
> > > >> > > point
> > > >> > > > of
> > > >> > > > > > view
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > > three
> > > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now
> > really
> > > >> > heavily
> > > >> > > > > used
> > > >> > > > > > for
> > > >> > > > > > >> > > > stream
> > > >> > > > > > >> > > > > >> > > processing, (2) we learned that
> > abstracting
> > > >> out
> > > >> > the
> > > >> > > > > > stream
> > > >> > > > > > >> > well
> > > >> > > > > > >> > > is
> > > >> > > > > > >> > > > > >> > > basically impossible, (3) we learned it
> is
> > > >> really
> > > >> > > > hard
> > > >> > > > > to
> > > >> > > > > > >> keep
> > > >> > > > > > >> > > the
> > > >> > > > > > >> > > > > two
> > > >> > > > > > >> > > > > >> > > things feeling like a single product.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > -Jay
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> > > >> Kleppmann
> > > >> > <
> > > >> > > > > > >> > > > > >> martin@kleppmann.com>
> > > >> > > > > > >> > > > > >> > > wrote:
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > >> Hi all,
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> I agree with the general philosophy of
> > > tying
> > > >> > Samza
> > > >> > > > > more
> > > >> > > > > > >> > firmly
> > > >> > > > > > >> > > to
> > > >> > > > > > >> > > > > >> Kafka.
> > > >> > > > > > >> > > > > >> > >> After I spent a while looking at
> > > integrating
> > > >> > other
> > > >> > > > > > message
> > > >> > > > > > >> > > > brokers
> > > >> > > > > > >> > > > > >> (e.g.
> > > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to
> > the
> > > >> > > > conclusion
> > > >> > > > > > that
> > > >> > > > > > >> > > > > >> > SystemConsumer
> > > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
> > > Kafka's
> > > >> > that
> > > >> > > > > pretty
> > > >> > > > > > >> much
> > > >> > > > > > >> > > > > nobody
> > > >> > > > > > >> > > > > >> but
> > > >> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus
> is
> > > >> > perhaps
> > > >> > > an
> > > >> > > > > > >> > exception,
> > > >> > > > > > >> > > > but
> > > >> > > > > > >> > > > > >> it
> > > >> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.)
> > > Thus,
> > > >> > > making
> > > >> > > > > > Samza
> > > >> > > > > > >> > > fully
> > > >> > > > > > >> > > > > >> > dependent
> > > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > > >> > system-independence
> > > >> > > > was
> > > >> > > > > > >> never
> > > >> > > > > > >> > as
> > > >> > > > > > >> > > > > real
> > > >> > > > > > >> > > > > >> as
> > > >> > > > > > >> > > > > >> > we
> > > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of
> > > code
> > > >> > reuse
> > > >> > > > are
> > > >> > > > > > >> real.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN
> > has
> > > >> also
> > > >> > > > always
> > > >> > > > > > been
> > > >> > > > > > >> > > > > >> appealing to
> > > >> > > > > > >> > > > > >> > >> me, for various reasons already
> mentioned
> > > in
> > > >> > this
> > > >> > > > > > thread.
> > > >> > > > > > >> > > > Although
> > > >> > > > > > >> > > > > >> > making
> > > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > > >> > > > (YARN/Mesos/AWS/etc)
> > > >> > > > > > >> seems
> > > >> > > > > > >> > > > > >> laudable,
> > > >> > > > > > >> > > > > >> > I am
> > > >> > > > > > >> > > > > >> > >> a little concerned that it will
> restrict
> > us
> > > >> to a
> > > >> > > > > lowest
> > > >> > > > > > >> > common
> > > >> > > > > > >> > > > > >> > denominator.
> > > >> > > > > > >> > > > > >> > >> For example, would host affinity
> > > (SAMZA-617)
> > > >> > still
> > > >> > > > be
> > > >> > > > > > >> > possible?
> > > >> > > > > > >> > > > For
> > > >> > > > > > >> > > > > >> jobs
> > > >> > > > > > >> > > > > >> > >> with large amounts of state, I think
> > > >> SAMZA-617
> > > >> > > would
> > > >> > > > > be
> > > >> > > > > > a
> > > >> > > > > > >> big
> > > >> > > > > > >> > > > boon,
> > > >> > > > > > >> > > > > >> > since
> > > >> > > > > > >> > > > > >> > >> restoring state off the changelog on
> > every
> > > >> > single
> > > >> > > > > > restart
> > > >> > > > > > >> is
> > > >> > > > > > >> > > > > painful,
> > > >> > > > > > >> > > > > >> > due
> > > >> > > > > > >> > > > > >> > >> to long recovery times. It would be a
> > shame
> > > >> if
> > > >> > the
> > > >> > > > > > >> decoupling
> > > >> > > > > > >> > > > from
> > > >> > > > > > >> > > > > >> YARN
> > > >> > > > > > >> > > > > >> > >> made host affinity impossible.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Jay, a question about the proposed API
> > for
> > > >> > > > > > instantiating a
> > > >> > > > > > >> > job
> > > >> > > > > > >> > > in
> > > >> > > > > > >> > > > > >> code
> > > >> > > > > > >> > > > > >> > >> (rather than a properties file): when
> > > >> > submitting a
> > > >> > > > job
> > > >> > > > > > to a
> > > >> > > > > > >> > > > > cluster,
> > > >> > > > > > >> > > > > >> is
> > > >> > > > > > >> > > > > >> > the
> > > >> > > > > > >> > > > > >> > >> idea that the instantiation code runs
> on
> > a
> > > >> > client
> > > >> > > > > > >> somewhere,
> > > >> > > > > > >> > > > which
> > > >> > > > > > >> > > > > >> then
> > > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > > >> > > YARN/Mesos/AWS/etc?
> > > >> > > > > Or
> > > >> > > > > > >> does
> > > >> > > > > > >> > > that
> > > >> > > > > > >> > > > > >> code
> > > >> > > > > > >> > > > > >> > run
> > > >> > > > > > >> > > > > >> > >> on each container that is part of the
> job
> > > (in
> > > >> > > which
> > > >> > > > > > case,
> > > >> > > > > > >> how
> > > >> > > > > > >> > > > does
> > > >> > > > > > >> > > > > >> the
> > > >> > > > > > >> > > > > >> > job
> > > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
> > > >> right to
> > > >> > > > make
> > > >> > > > > a
> > > >> > > > > > 1.0
> > > >> > > > > > >> > > > release
> > > >> > > > > > >> > > > > >> > with a
> > > >> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete.
> > So
> > > if
> > > >> > this
> > > >> > > > is
> > > >> > > > > > going
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> happen, I
> > > >> > > > > > >> > > > > >> > >> think it would be more honest to stick
> > with
> > > >> 0.*
> > > >> > > > > version
> > > >> > > > > > >> > numbers
> > > >> > > > > > >> > > > > until
> > > >> > > > > > >> > > > > >> > the
> > > >> > > > > > >> > > > > >> > >> library-ified Samza has been
> implemented,
> > > is
> > > >> > > stable
> > > >> > > > > and
> > > >> > > > > > >> > widely
> > > >> > > > > > >> > > > > used.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
> > > >> Kafka?
> > > >> > > There
> > > >> > > > > is
> > > >> > > > > > >> > > precedent
> > > >> > > > > > >> > > > > for
> > > >> > > > > > >> > > > > >> > >> tight coupling between different Apache
> > > >> projects
> > > >> > > > (e.g.
> > > >> > > > > > >> > Curator
> > > >> > > > > > >> > > > and
> > > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I
> > think
> > > >> > > remaining
> > > >> > > > > > >> separate
> > > >> > > > > > >> > > > would
> > > >> > > > > > >> > > > > >> be
> > > >> > > > > > >> > > > > >> > ok.
> > > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on
> > Kafka,
> > > >> there
> > > >> > > is
> > > >> > > > > > enough
> > > >> > > > > > >> > > > > substance
> > > >> > > > > > >> > > > > >> in
> > > >> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> > > >> project.
> > > >> > > An
> > > >> > > > > > >> argument
> > > >> > > > > > >> > in
> > > >> > > > > > >> > > > > >> favour
> > > >> > > > > > >> > > > > >> > of
> > > >> > > > > > >> > > > > >> > >> merging would be if we think Kafka has
> a
> > > much
> > > >> > > > stronger
> > > >> > > > > > >> "brand
> > > >> > > > > > >> > > > > >> presence"
> > > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one.
> > If
> > > >> the
> > > >> > > Kafka
> > > >> > > > > > >> project
> > > >> > > > > > >> > is
> > > >> > > > > > >> > > > > >> willing
> > > >> > > > > > >> > > > > >> > to
> > > >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of
> > > doing
> > > >> > > > stateful
> > > >> > > > > > >> stream
> > > >> > > > > > >> > > > > >> > >> transformations, that would probably
> have
> > > >> much
> > > >> > the
> > > >> > > > > same
> > > >> > > > > > >> > effect
> > > >> > > > > > >> > > as
> > > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> > > >> Processors"
> > > >> > or
> > > >> > > > > > suchlike.
> > > >> > > > > > >> > > Close
> > > >> > > > > > >> > > > > >> > >> collaboration between the two projects
> > will
> > > >> be
> > > >> > > > needed
> > > >> > > > > in
> > > >> > > > > > >> any
> > > >> > > > > > >> > > > case.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> From a project management perspective,
> I
> > > >> guess
> > > >> > the
> > > >> > > > > "new
> > > >> > > > > > >> > Samza"
> > > >> > > > > > >> > > > > would
> > > >> > > > > > >> > > > > >> > have
> > > >> > > > > > >> > > > > >> > >> to be developed on a branch alongside
> > > ongoing
> > > >> > > > > > maintenance
> > > >> > > > > > >> of
> > > >> > > > > > >> > > the
> > > >> > > > > > >> > > > > >> current
> > > >> > > > > > >> > > > > >> > >> line of development? I think it would
> be
> > > >> > important
> > > >> > > > to
> > > >> > > > > > >> > continue
> > > >> > > > > > >> > > > > >> > supporting
> > > >> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> > > >> migration
> > > >> > > > path
> > > >> > > > > to
> > > >> > > > > > >> the
> > > >> > > > > > >> > > new
> > > >> > > > > > >> > > > > >> > version.
> > > >> > > > > > >> > > > > >> > >> Leaving the current versions
> unsupported
> > > and
> > > >> > > forcing
> > > >> > > > > > people
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> rewrite
> > > >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Best,
> > > >> > > > > > >> > > > > >> > >> Martin
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > > >> > > > jay@confluent.io>
> > > >> > > > > > >> wrote:
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >>> Hey Garry,
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be
> > > happy
> > > >> to
> > > >> > > chat
> > > >> > > > > > more
> > > >> > > > > > >> > about
> > > >> > > > > > >> > > > > this
> > > >> > > > > > >> > > > > >> if
> > > >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris
> and I
> > > >> > started
> > > >> > > > with
> > > >> > > > > > the
> > > >> > > > > > >> > idea
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> "what
> > > >> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> > > >> > ingestion
> > > >> > > > > tool"
> > > >> > > > > > but
> > > >> > > > > > >> > > > > >> ultimately
> > > >> > > > > > >> > > > > >> > we
> > > >> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> > > >> ingestion
> > > >> > > and
> > > >> > > > > > >> > > > transformation
> > > >> > > > > > >> > > > > >> had
> > > >> > > > > > >> > > > > >> > >>> pretty different needs and coupling
> the
> > > two
> > > >> > made
> > > >> > > > > things
> > > >> > > > > > >> > hard.
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
> > > (KIP-26)
> > > >> > > > actually
> > > >> > > > > > will
> > > >> > > > > > >> > do
> > > >> > > > > > >> > > > what
> > > >> > > > > > >> > > > > >> you
> > > >> > > > > > >> > > > > >> > >> are
> > > >> > > > > > >> > > > > >> > >>> looking for.
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> With regard to your point about
> slider,
> > I
> > > >> don't
> > > >> > > > > > >> necessarily
> > > >> > > > > > >> > > > > >> disagree.
> > > >> > > > > > >> > > > > >> > >> But I
> > > >> > > > > > >> > > > > >> > >>> think getting good YARN support is
> quite
> > > >> doable
> > > >> > > > and I
> > > >> > > > > > >> think
> > > >> > > > > > >> > we
> > > >> > > > > > >> > > > can
> > > >> > > > > > >> > > > > >> make
> > > >> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> > > >> proposal
> > > >> > > > > solves
> > > >> > > > > > is
> > > >> > > > > > >> > that
> > > >> > > > > > >> > > > > >> > >> technically
> > > >> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
> > > >> cluster
> > > >> > > > > > management
> > > >> > > > > > >> > > systems
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > way
> > > >> > > > > > >> > > > > >> > >>> things are now, you need to write an
> > "app
> > > >> > master"
> > > >> > > > or
> > > >> > > > > > >> > > "framework"
> > > >> > > > > > >> > > > > for
> > > >> > > > > > >> > > > > >> > each
> > > >> > > > > > >> > > > > >> > >>> and they are all a little different so
> > > >> testing
> > > >> > is
> > > >> > > > > > really
> > > >> > > > > > >> > hard.
> > > >> > > > > > >> > > > In
> > > >> > > > > > >> > > > > >> the
> > > >> > > > > > >> > > > > >> > >>> absence of this we have been stuck
> with
> > > just
> > > >> > YARN
> > > >> > > > > which
> > > >> > > > > > >> has
> > > >> > > > > > >> > > > > >> fantastic
> > > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the
> > > org,
> > > >> but
> > > >> > > > zero
> > > >> > > > > > >> > > penetration
> > > >> > > > > > >> > > > > >> > >> elsewhere.
> > > >> > > > > > >> > > > > >> > >>> Given the huge amount of work being
> put
> > in
> > > >> to
> > > >> > > > slider,
> > > >> > > > > > >> > > marathon,
> > > >> > > > > > >> > > > > aws
> > > >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
> > > related
> > > >> > > > packaging
> > > >> > > > > > >> > > > technologies
> > > >> > > > > > >> > > > > >> > people
> > > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes,
> various
> > > >> > > > > cloud-specific
> > > >> > > > > > >> > deploy
> > > >> > > > > > >> > > > > >> tools,
> > > >> > > > > > >> > > > > >> > >> etc)
> > > >> > > > > > >> > > > > >> > >>> I really think it is important to get
> > this
> > > >> > right.
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> -Jay
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> > > >> > Turkington
> > > >> > > <
> > > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com>
> wrote:
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>>> Hi all,
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> I think the question below re does
> > Samza
> > > >> > become
> > > >> > > a
> > > >> > > > > > >> > sub-project
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > >>>> highlights the broader point around
> > > >> migration.
> > > >> > > > Chris
> > > >> > > > > > >> > mentions
> > > >> > > > > > >> > > > > >> Samza's
> > > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1
> > release
> > > >> but
> > > >> > I'm
> > > >> > > > not
> > > >> > > > > > sure
> > > >> > > > > > >> > it
> > > >> > > > > > >> > > > > feels
> > > >> > > > > > >> > > > > >> > >> right to
> > > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> > > >> deprecate
> > > >> > > > most
> > > >> > > > > of
> > > >> > > > > > >> it.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have
> some
> > > guys
> > > >> > who
> > > >> > > > have
> > > >> > > > > > >> > started
> > > >> > > > > > >> > > > > >> working
> > > >> > > > > > >> > > > > >> > >> with
> > > >> > > > > > >> > > > > >> > >>>> Samza and building some new
> > > >> > consumers/producers
> > > >> > > > was
> > > >> > > > > > next
> > > >> > > > > > >> > up.
> > > >> > > > > > >> > > > > Sounds
> > > >> > > > > > >> > > > > >> > like
> > > >> > > > > > >> > > > > >> > >>>> that is absolutely not the direction
> to
> > > >> go. I
> > > >> > > need
> > > >> > > > > to
> > > >> > > > > > >> look
> > > >> > > > > > >> > > into
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > KIP
> > > >> > > > > > >> > > > > >> > >> in
> > > >> > > > > > >> > > > > >> > >>>> more detail but for me the
> > attractiveness
> > > >> of
> > > >> > > > adding
> > > >> > > > > > new
> > > >> > > > > > >> > Samza
> > > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all
> > > they
> > > >> > were
> > > >> > > > > doing
> > > >> > > > > > was
> > > >> > > > > > >> > > > really
> > > >> > > > > > >> > > > > >> > getting
> > > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to
> > > avoid
> > > >> > > > having
> > > >> > > > > to
> > > >> > > > > > >> > worry
> > > >> > > > > > >> > > > > about
> > > >> > > > > > >> > > > > >> the
> > > >> > > > > > >> > > > > >> > >>>> lifecycle management of external
> > clients.
> > > >> If
> > > >> > > there
> > > >> > > > > is
> > > >> > > > > > a
> > > >> > > > > > >> > > generic
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug
> a
> > > new
> > > >> > > > connector
> > > >> > > > > > into
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > have
> > > >> > > > > > >> > > > > >> a
> > > >> > > > > > >> > > > > >> > >> lot of
> > > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> > > reliability
> > > >> > done
> > > >> > > > for
> > > >> > > > > me
> > > >> > > > > > >> then
> > > >> > > > > > >> > > it
> > > >> > > > > > >> > > > > >> gives
> > > >> > > > > > >> > > > > >> > me
> > > >> > > > > > >> > > > > >> > >> all
> > > >> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers
> > > would.
> > > >> If
> > > >> > > not
> > > >> > > > > > then it
> > > >> > > > > > >> > > > > >> complicates
> > > >> > > > > > >> > > > > >> > my
> > > >> > > > > > >> > > > > >> > >>>> operational deployments.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Which is similar to my other question
> > > with
> > > >> the
> > > >> > > > > > proposal
> > > >> > > > > > >> --
> > > >> > > > > > >> > if
> > > >> > > > > > >> > > > we
> > > >> > > > > > >> > > > > >> > build a
> > > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza
> plus
> > > the
> > > >> > > > requisite
> > > >> > > > > > >> shims
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> > integrate
> > > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former
> > may
> > > >> be a
> > > >> > > lot
> > > >> > > > > more
> > > >> > > > > > >> work
> > > >> > > > > > >> > > > than
> > > >> > > > > > >> > > > > we
> > > >> > > > > > >> > > > > >> > >> think.
> > > >> > > > > > >> > > > > >> > >>>> We may make it much easier for a
> > newcomer
> > > >> to
> > > >> > get
> > > >> > > > > > >> something
> > > >> > > > > > >> > > > > running
> > > >> > > > > > >> > > > > >> but
> > > >> > > > > > >> > > > > >> > >>>> having them step up and get a
> reliable
> > > >> > > production
> > > >> > > > > > >> > deployment
> > > >> > > > > > >> > > > may
> > > >> > > > > > >> > > > > >> still
> > > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if
> for
> > > >> > different
> > > >> > > > > > reasons
> > > >> > > > > > >> > than
> > > >> > > > > > >> > > > > >> today.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable
> > > with
> > > >> > > making
> > > >> > > > > the
> > > >> > > > > > >> Samza
> > > >> > > > > > >> > > > > >> dependency
> > > >> > > > > > >> > > > > >> > >> on
> > > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
> > absolutely
> > > >> see
> > > >> > > the
> > > >> > > > > > >> benefits
> > > >> > > > > > >> > > in
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > > >> > > > > > >> > > > terminologies/abstractions
> > > >> > > > > > >> > > > > >> that
> > > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a
> library
> > > >> would
> > > >> > > > likely
> > > >> > > > > > be a
> > > >> > > > > > >> > very
> > > >> > > > > > >> > > > > nice
> > > >> > > > > > >> > > > > >> > tool
> > > >> > > > > > >> > > > > >> > >> to
> > > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just
> have
> > > the
> > > >> > > > concerns
> > > >> > > > > > >> above
> > > >> > > > > > >> > re
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > >>>> operational side.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Garry
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> > > >> [mailto:
> > > >> > > > > > >> > gdfm@apache.org
> > > >> > > > > > >> > > ]
> > > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and
> obesrvations
> > on
> > > >> > Samza
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > > >> > > > > > >> > > > > >> > >>>> From outside, I have always perceived
> > > Samza
> > > >> > as a
> > > >> > > > > > >> computing
> > > >> > > > > > >> > > > layer
> > > >> > > > > > >> > > > > >> over
> > > >> > > > > > >> > > > > >> > >>>> Kafka.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit
> provocative,
> > is
> > > >> > > "should
> > > >> > > > > > Samza
> > > >> > > > > > >> be
> > > >> > > > > > >> > a
> > > >> > > > > > >> > > > > >> > sub-project
> > > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> > > >> separate
> > > >> > > > > project
> > > >> > > > > > >> > with a
> > > >> > > > > > >> > > > > >> separate
> > > >> > > > > > >> > > > > >> > >>>> governance?
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Cheers,
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> --
> > > >> > > > > > >> > > > > >> > >>>> Gianmarco
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > > >> > > > > > yanfang724@gmail.com>
> > > >> > > > > > >> > > > wrote:
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with
> Kafka
> > > more
> > > >> > > > tightly.
> > > >> > > > > > >> > Because
> > > >> > > > > > >> > > > > Samza
> > > >> > > > > > >> > > > > >> de
> > > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it
> should
> > > >> > leverage
> > > >> > > > > what
> > > >> > > > > > >> Kafka
> > > >> > > > > > >> > > > has.
> > > >> > > > > > >> > > > > At
> > > >> > > > > > >> > > > > >> > the
> > > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
> > > reinvent
> > > >> > what
> > > >> > > > > Samza
> > > >> > > > > > >> > > already
> > > >> > > > > > >> > > > > >> has. I
> > > >> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> > > >> > ingestion
> > > >> > > > and
> > > >> > > > > > >> > > > > transformation.
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me
> to
> > > >> image
> > > >> > > how
> > > >> > > > > the
> > > >> > > > > > >> Samza
> > > >> > > > > > >> > > > will
> > > >> > > > > > >> > > > > >> look
> > > >> > > > > > >> > > > > >> > >>>> like.
> > > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a
> little
> > > >> > > difference
> > > >> > > > > in
> > > >> > > > > > >> terms
> > > >> > > > > > >> > > of
> > > >> > > > > > >> > > > > how
> > > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's
> code
> > > >> shows
> > > >> > (A
> > > >> > > > > > client of
> > > >> > > > > > >> > > > Kakfa)
> > > >> > > > > > >> > > > > ?
> > > >> > > > > > >> > > > > >> And
> > > >> > > > > > >> > > > > >> > >>>>> user's application code calls this
> > > client?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of
> > > Kafka
> > > >> > (like
> > > >> > > > > what
> > > >> > > > > > the
> > > >> > > > > > >> > > code
> > > >> > > > > > >> > > > > >> shows),
> > > >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > > >> > > > > fault-tolerance?
> > > >> > > > > > >> Are
> > > >> > > > > > >> > > they
> > > >> > > > > > >> > > > > >> taken
> > > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> > > >> mechanism,
> > > >> > > such
> > > >> > > > > as
> > > >> > > > > > >> > "Samza
> > > >> > > > > > >> > > > > >> worker"
> > > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 2. What about other features, such
> as
> > > >> > > > auto-scaling,
> > > >> > > > > > >> shared
> > > >> > > > > > >> > > > > state,
> > > >> > > > > > >> > > > > >> > >>>>> monitoring?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is
> > > this
> > > >> > what
> > > >> > > > > Chris
> > > >> > > > > > >> > > > suggests?)
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from
> > > Kakfa
> > > >> > and
> > > >> > > > > > produce
> > > >> > > > > > >> to
> > > >> > > > > > >> > > it.
> > > >> > > > > > >> > > > > >> Then it
> > > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks
> > > like
> > > >> > now,
> > > >> > > > > > except it
> > > >> > > > > > >> > > does
> > > >> > > > > > >> > > > > not
> > > >> > > > > > >> > > > > >> > rely
> > > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> > > >> leverage
> > > >> > > > Kafka's
> > > >> > > > > > >> > metrics,
> > > >> > > > > > >> > > > > logs,
> > > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the
> dependency?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> Thanks,
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM,
> > Guozhang
> > > >> > Wang <
> > > >> > > > > > >> > > > > wangguoz@gmail.com
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> Read through the code example and
> it
> > > >> looks
> > > >> > > good
> > > >> > > > to
> > > >> > > > > > me.
> > > >> > > > > > >> A
> > > >> > > > > > >> > > few
> > > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> > > >> runnable
> > > >> > > like:
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > > >> > > --config-factory=...
> > > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> > > deploying
> > > >> > Samza
> > > >> > > > > more
> > > >> > > > > > as
> > > >> > > > > > >> > > > embedded
> > > >> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> > > >> (ignoring
> > > >> > > the
> > > >> > > > > > >> > > terminology
> > > >> > > > > > >> > > > > >> since
> > > >> > > > > > >> > > > > >> > >>>>>> it is not the
> > > >> > > > > > >> > > > > >> > >>>>> same
> > > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> > > >> MyStreamTask(configs);
> > > >> > > > > Thread
> > > >> > > > > > >> > thread
> > > >> > > > > > >> > > =
> > > >> > > > > > >> > > > > new
> > > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment
> > modes
> > > >> are
> > > >> > > > > important
> > > >> > > > > > >> for
> > > >> > > > > > >> > > > > >> different
> > > >> > > > > > >> > > > > >> > >>>>>> types
> > > >> > > > > > >> > > > > >> > >>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making
> > Samza
> > > >> > purely
> > > >> > > > > > >> standalone
> > > >> > > > > > >> > is
> > > >> > > > > > >> > > > > still
> > > >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or
> > > library
> > > >> > > modes.
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> Guozhang
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM,
> Jay
> > > >> Kreps
> > > >> > <
> > > >> > > > > > >> > > > jay@confluent.io>
> > > >> > > > > > >> > > > > >> > wrote:
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> > > >> example,
> > > >> > it
> > > >> > > > was
> > > >> > > > > > >> > supposed
> > > >> > > > > > >> > > > to
> > > >> > > > > > >> > > > > >> look
> > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > >> > > > > > >> > > > > >> > >>>>>>> this:
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new
> Properties();
> > > >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > > >> > > > "localhost:4242");
> > > >> > > > > > >> > > > > >> StreamingConfig
> > > >> > > > > > >> > > > > >> > >>>>>>> config = new
> StreamingConfig(props);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > >> > > > "test-topic-2");
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > config.processor(ExampleStreamProcessor.class);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > >> > StringSerializer(),
> > > >> > > > new
> > > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
> > KafkaStreaming
> > > >> > > > container =
> > > >> > > > > > new
> > > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> > > container.run();
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> -Jay
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM,
> > Jay
> > > >> > Kreps <
> > > >> > > > > > >> > > > jay@confluent.io
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some
> conversations
> > > >> Chris
> > > >> > > and
> > > >> > > > I
> > > >> > > > > > were
> > > >> > > > > > >> > > having
> > > >> > > > > > >> > > > > >> > >>>>>>>> around
> > > >> > > > > > >> > > > > >> > >>>>>>> whether
> > > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza
> > as a
> > > >> kind
> > > >> > > of
> > > >> > > > > data
> > > >> > > > > > >> > > > ingestion
> > > >> > > > > > >> > > > > >> > >>>>> framework
> > > >> > > > > > >> > > > > >> > >>>>>>> for
> > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to
> > > KIP-26
> > > >> > > > > "copycat").
> > > >> > > > > > >> This
> > > >> > > > > > >> > > > kind
> > > >> > > > > > >> > > > > of
> > > >> > > > > > >> > > > > >> > >>>>>> combined
> > > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and
> > > YARN
> > > >> and
> > > >> > > the
> > > >> > > > > > >> > discussion
> > > >> > > > > > >> > > > > >> around
> > > >> > > > > > >> > > > > >> > >>>>>>>> how
> > > >> > > > > > >> > > > > >> > >>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was,
> > given
> > > >> that
> > > >> > > > Samza
> > > >> > > > > > was
> > > >> > > > > > >> > > > basically
> > > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific,
> > what
> > > if
> > > >> > you
> > > >> > > > just
> > > >> > > > > > >> > embraced
> > > >> > > > > > >> > > > > that
> > > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > > >> > > > > > >> > > > > >> > >>>>>> into
> > > >> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> > > >> > framework
> > > >> > > > and
> > > >> > > > > > more
> > > >> > > > > > >> > > like a
> > > >> > > > > > >> > > > > >> > >>>>>>>> third
> > > >> > > > > > >> > > > > >> > >>>>> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> > > consumer"
> > > >> > with
> > > >> > > > > state
> > > >> > > > > > >> > > > management
> > > >> > > > > > >> > > > > >> > >>>>>> facilities.
> > > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> > > >> complex
> > > >> > > > stream
> > > >> > > > > > >> > > processing
> > > >> > > > > > >> > > > > >> > >>>>>>>> framework
> > > >> > > > > > >> > > > > >> > >>>>>>> this
> > > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple
> > > thing,
> > > >> not
> > > >> > > > much
> > > >> > > > > > more
> > > >> > > > > > >> > > > > >> complicated
> > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>> use
> > > >> > > > > > >> > > > > >> > >>>>>>> or
> > > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As
> > > Chris
> > > >> > said
> > > >> > > > we
> > > >> > > > > > >> thought
> > > >> > > > > > >> > > > about
> > > >> > > > > > >> > > > > >> it
> > > >> > > > > > >> > > > > >> > >>>>>>>> a
> > > >> > > > > > >> > > > > >> > >>>>> lot
> > > >> > > > > > >> > > > > >> > >>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> > > >> > processing
> > > >> > > > > > systems
> > > >> > > > > > >> > were
> > > >> > > > > > >> > > > > doing)
> > > >> > > > > > >> > > > > >> > >>>>> seemed
> > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from
> MapReduce.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to
> ingest/output
> > > >> data
> > > >> > to
> > > >> > > > and
> > > >> > > > > > from
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > > stream
> > > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually
> > > looked
> > > >> > into
> > > >> > > > how
> > > >> > > > > > that
> > > >> > > > > > >> > > would
> > > >> > > > > > >> > > > > >> > >>>>>>>> work,
> > > >> > > > > > >> > > > > >> > >>>>> Samza
> > > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
> > ingestion
> > > >> > > framework
> > > >> > > > > > for a
> > > >> > > > > > >> > > bunch
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > >>>>> reasons.
> > > >> > > > > > >> > > > > >> > >>>>>> To
> > > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a
> > > pretty
> > > >> > > > different
> > > >> > > > > > >> > internal
> > > >> > > > > > >> > > > > data
> > > >> > > > > > >> > > > > >> > >>>>>>>> model
> > > >> > > > > > >> > > > > >> > >>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split
> > > them
> > > >> and
> > > >> > > had
> > > >> > > > > an
> > > >> > > > > > api
> > > >> > > > > > >> > for
> > > >> > > > > > >> > > > > Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA
> KIP-26)
> > > >> and a
> > > >> > > > > separate
> > > >> > > > > > >> api
> > > >> > > > > > >> > > for
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> > > embracing
> > > >> the
> > > >> > > > same
> > > >> > > > > > >> > > terminology
> > > >> > > > > > >> > > > > and
> > > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about
> > the
> > > >> > current
> > > >> > > > > > state is
> > > >> > > > > > >> > > that
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > >>>>>>>> two
> > > >> > > > > > >> > > > > >> > >>>>>>> systems
> > > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on.
> Terminology
> > > >> like
> > > >> > > > > "stream"
> > > >> > > > > > vs
> > > >> > > > > > >> > > > "topic"
> > > >> > > > > > >> > > > > >> and
> > > >> > > > > > >> > > > > >> > >>>>>>> different
> > > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems
> means
> > > you
> > > >> > kind
> > > >> > > > of
> > > >> > > > > > have
> > > >> > > > > > >> to
> > > >> > > > > > >> > > > learn
> > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > > >> > > > > > >> > > > > >> > >>>>>>> way,
> > > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
> > different
> > > >> way,
> > > >> > > > then
> > > >> > > > > > kind
> > > >> > > > > > >> of
> > > >> > > > > > >> > > > > >> > >>>>>>>> understand
> > > >> > > > > > >> > > > > >> > >>>>> how
> > > >> > > > > > >> > > > > >> > >>>>>>> they
> > > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having
> > > walked
> > > >> a
> > > >> > few
> > > >> > > > > > people
> > > >> > > > > > >> > > through
> > > >> > > > > > >> > > > > >> this
> > > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks
> to
> > > >> get.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot
> of
> > > >> time
> > > >> > on
> > > >> > > > > > >> airplanes I
> > > >> > > > > > >> > > > > hacked
> > > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> > > >> incomplete
> > > >> > > > > > prototype
> > > >> > > > > > >> of
> > > >> > > > > > >> > > > what
> > > >> > > > > > >> > > > > >> > >>>>>>>> this would
> > > >> > > > > > >> > > > > >> > >>>>> look
> > > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just
> unceremoniously
> > > >> dumped
> > > >> > > into
> > > >> > > > > > Kafka
> > > >> > > > > > >> as
> > > >> > > > > > >> > > it
> > > >> > > > > > >> > > > > >> > >>>>>>>> required a
> > > >> > > > > > >> > > > > >> > >>>>>> few
> > > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here
> > is
> > > >> the
> > > >> > > code:
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> >
> > > >> > > > > >
> > > >> > >
> > > >>
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype
> I
> > > just
> > > >> > > > > liberally
> > > >> > > > > > >> > renamed
> > > >> > > > > > >> > > > > >> > >>>>>>>> everything
> > > >> > > > > > >> > > > > >> > >>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with
> no
> > > >> regard
> > > >> > > for
> > > >> > > > > > >> > > > compatibility.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something
> like
> > > >> this:
> > > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
> > Properties();
> > > >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > > >> > > > > "localhost:4242");
> > > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > > >> > > > > > >> > > > > >>
> > config.processor(ExampleStreamProcessor.class);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > > >> > > StringDeserializer());
> > > >> > > > > > >> > > > KafkaStreaming
> > > >> > > > > > >> > > > > >> > >>>>>> container =
> > > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > > >> > container.run();
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > > >> > > > SamzaContainer;
> > > >> > > > > > >> > > > > StreamProcessor
> > > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the
> > class
> > > >> names
> > > >> > > in
> > > >> > > > a
> > > >> > > > > > file
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > then
> > > >> > > > > > >> > > > > >> > >>>>>>>> having
> > > >> > > > > > >> > > > > >> > >>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you
> > just
> > > >> > > > > instantiate
> > > >> > > > > > the
> > > >> > > > > > >> > > > > container
> > > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is
> balanced
> > > over
> > > >> > > > however
> > > >> > > > > > many
> > > >> > > > > > >> > > > > instances
> > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>> this
> > > >> > > > > > >> > > > > >> > >>>>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> > > instance
> > > >> > dies,
> > > >> > > > new
> > > >> > > > > > >> tasks
> > > >> > > > > > >> > > are
> > > >> > > > > > >> > > > > >> added
> > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
> > shutting
> > > >> them
> > > >> > > > down).
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
> > > running
> > > >> > this
> > > >> > > > > stuff
> > > >> > > > > > in
> > > >> > > > > > >> > YARN
> > > >> > > > > > >> > > > via
> > > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and
> AWS
> > > >> using
> > > >> > > some
> > > >> > > > > of
> > > >> > > > > > >> their
> > > >> > > > > > >> > > > tools
> > > >> > > > > > >> > > > > >> > >>>>>>>> but from the
> > > >> > > > > > >> > > > > >> > >>>>>> point
> > > >> > > > > > >> > > > > >> > >>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these
> > stream
> > > >> > > > processing
> > > >> > > > > > jobs
> > > >> > > > > > >> > are
> > > >> > > > > > >> > > > > just
> > > >> > > > > > >> > > > > >> > >>>>>> stateless
> > > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
> > > >> expand
> > > >> > and
> > > >> > > > > > contract
> > > >> > > > > > >> > at
> > > >> > > > > > >> > > > > will.
> > > >> > > > > > >> > > > > >> > >>>>>>>> There
> > > >> > > > > > >> > > > > >> > >>>>> is
> > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of
> code,
> > > it
> > > >> > would
> > > >> > > > get
> > > >> > > > > > >> larger
> > > >> > > > > > >> > > if
> > > >> > > > > > >> > > > we
> > > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
> > > larger.
> > > >> We
> > > >> > > > really
> > > >> > > > > > do
> > > >> > > > > > >> > get a
> > > >> > > > > > >> > > > ton
> > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>> leverage
> > > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> > > >> > delegated
> > > >> > > to
> > > >> > > > > the
> > > >> > > > > > >> new
> > > >> > > > > > >> > > > > >> consumer.
> > > >> > > > > > >> > > > > >> > >>>>> This
> > > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> > > >> > management
> > > >> > > > > > strategy
> > > >> > > > > > >> > > > > available
> > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to
> > Samza
> > > >> (and
> > > >> > > vice
> > > >> > > > > > versa)
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > with
> > > >> > > > > > >> > > > > >> > >>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>> exact
> > > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as
> > > state
> > > >> > reuse
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it
> is
> > > >> > thought
> > > >> > > > > > >> provoking.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM,
> > > Chris
> > > >> > > > > Riccomini <
> > > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with
> > > Samza
> > > >> > > > > engineers
> > > >> > > > > > at
> > > >> > > > > > >> > > > LinkedIn
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>> Confluent
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> > > observations
> > > >> > and
> > > >> > > > > would
> > > >> > > > > > >> like
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> > >>>>>>>>> propose
> > > >> > > > > > >> > > > > >> > >>>>> some
> > > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that
> I
> > > >> want to
> > > >> > > > call
> > > >> > > > > > out
> > > >> > > > > > >> > about
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > > >> > > > > > >> > > > > >> > >>>>>> design,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> > > changes.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a
> > dynamic
> > > >> > > > deployment
> > > >> > > > > > >> system.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> > > >> SystemConsumer/SystemProducer
> > > >> > and
> > > >> > > > > > Kafka's
> > > >> > > > > > >> > > > consumer
> > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > >> > > > > > >> > > > > >> > >>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the
> same
> > > >> > problems.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
> > > related,
> > > >> > but
> > > >> > > > I'll
> > > >> > > > > > >> > address
> > > >> > > > > > >> > > > them
> > > >> > > > > > >> > > > > >> in
> > > >> > > > > > >> > > > > >> > >>>>> order.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the
> use
> > > of a
> > > >> > > > dynamic
> > > >> > > > > > >> > > deployment
> > > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > > >> > > > > > >> > > > > >> > >>>>>> such
> > > >> > > > > > >> > > > > >> > >>>>>>>>> as
> > > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
> > initially
> > > >> built
> > > >> > > > > Samza,
> > > >> > > > > > we
> > > >> > > > > > >> > bet
> > > >> > > > > > >> > > > that
> > > >> > > > > > >> > > > > >> > >>>>>>>>> there
> > > >> > > > > > >> > > > > >> > >>>>>> would
> > > >> > > > > > >> > > > > >> > >>>>>>>>> be
> > > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area,
> > and
> > > >> we
> > > >> > > could
> > > >> > > > > > >> support
> > > >> > > > > > >> > > > them,
> > > >> > > > > > >> > > > > >> and
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>> rest
> > > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there
> > are
> > > >> many
> > > >> > > > > > >> variations.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > > >> > > > > > >> > > > > >> > >>>>>> many
> > > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just
> start
> > > >> their
> > > >> > > > > > processors
> > > >> > > > > > >> > like
> > > >> > > > > > >> > > > > normal
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
> > traditional
> > > >> > > > deployment
> > > >> > > > > > >> scripts
> > > >> > > > > > >> > > > such
> > > >> > > > > > >> > > > > as
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > > >> > > > > > >> > > > > >> > >>>>>> Chef,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a
> deployment
> > > >> system
> > > >> > > on
> > > >> > > > > > users
> > > >> > > > > > >> > makes
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
> > > painful
> > > >> for
> > > >> > > > first
> > > >> > > > > > time
> > > >> > > > > > >> > > > users.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
> > requirement
> > > >> was
> > > >> > > also
> > > >> > > > a
> > > >> > > > > > bit
> > > >> > > > > > >> of
> > > >> > > > > > >> > a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > > >> > > > > > >> > > > > >> > >>>>>> because
> > > >> > > > > > >> > > > > >> > >>>>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding
> > > between
> > > >> > the
> > > >> > > > > > nature of
> > > >> > > > > > >> > > batch
> > > >> > > > > > >> > > > > >> jobs
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>> stream
> > > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we
> made
> > > >> > > conscious
> > > >> > > > > > effort
> > > >> > > > > > >> to
> > > >> > > > > > >> > > > favor
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>> Hadoop
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing
> things,
> > > >> since
> > > >> > it
> > > >> > > > > worked
> > > >> > > > > > >> and
> > > >> > > > > > >> > > was
> > > >> > > > > > >> > > > > well
> > > >> > > > > > >> > > > > >> > >>>>>>> understood.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was
> that
> > > >> batch
> > > >> > > jobs
> > > >> > > > > > have a
> > > >> > > > > > >> > > > definite
> > > >> > > > > > >> > > > > >> > >>>>>> beginning,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs
> > > don't
> > > >> > > > > (usually).
> > > >> > > > > > >> This
> > > >> > > > > > >> > > > leads
> > > >> > > > > > >> > > > > to
> > > >> > > > > > >> > > > > >> > >>>>>>>>> a
> > > >> > > > > > >> > > > > >> > >>>>> much
> > > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for
> > > stream
> > > >> > > > > processors.
> > > >> > > > > > >> You
> > > >> > > > > > >> > > > > >> basically
> > > >> > > > > > >> > > > > >> > >>>>>>>>> just
> > > >> > > > > > >> > > > > >> > >>>>>>> need
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> > > >> processor,
> > > >> > and
> > > >> > > > > start
> > > >> > > > > > >> it.
> > > >> > > > > > >> > > The
> > > >> > > > > > >> > > > > way
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn,
> there's
> > > no
> > > >> > > concept
> > > >> > > > > of
> > > >> > > > > > a
> > > >> > > > > > >> > > cluster
> > > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > > >> > > > > > >> > > > > >> > >>>>>> add
> > > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> > > >> coupling
> > > >> > > > Samza
> > > >> > > > > > with
> > > >> > > > > > >> a
> > > >> > > > > > >> > > > > >> scheduler
> > > >> > > > > > >> > > > > >> > >>>>>>>>> is
> > > >> > > > > > >> > > > > >> > >>>>>> that
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has
> to
> > > >> handle
> > > >> > > > > > deployment.
> > > >> > > > > > >> > > This
> > > >> > > > > > >> > > > > >> pulls
> > > >> > > > > > >> > > > > >> > >>>>>>>>> in a
> > > >> > > > > > >> > > > > >> > >>>>>>> bunch
> > > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> > > >> > > distribution
> > > >> > > > > > (config
> > > >> > > > > > >> > > > > stream),
> > > >> > > > > > >> > > > > >> > >>>>>>>>> shell
> > > >> > > > > > >> > > > > >> > >>>>>>> scrips
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> > > packaging
> > > >> > (all
> > > >> > > > the
> > > >> > > > > > .tgz
> > > >> > > > > > >> > > > stuff),
> > > >> > > > > > >> > > > > >> etc.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring
> > dynamic
> > > >> > > > deployment
> > > >> > > > > > was
> > > >> > > > > > >> to
> > > >> > > > > > >> > > > > support
> > > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to
> have
> > > >> > > locality,
> > > >> > > > > you
> > > >> > > > > > >> need
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > put
> > > >> > > > > > >> > > > > >> > >>>>>>>>> your
> > > >> > > > > > >> > > > > >> > >>>>>> processors
> > > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> > > processing.
> > > >> > Upon
> > > >> > > > > > further
> > > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > > >> > > > > > >> > > > > >> > >>>>>>> though,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
> > beneficial.
> > > >> > There
> > > >> > > is
> > > >> > > > > > some
> > > >> > > > > > >> > good
> > > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > > >> > > > > > >> > > > > >> > >>>>>> about
> > > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
> > SAMZA-335.
> > > >> > Again,
> > > >> > > we
> > > >> > > > > > took
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > > >> > > > > > >> > > > > >> > >>>>>> path,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> > > differences
> > > >> > > > between
> > > >> > > > > > HDFS
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > > >> > > > > > >> > > > > >> > >>>>>> has
> > > >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has
> > partitions.
> > > >> This
> > > >> > > > leads
> > > >> > > > > to
> > > >> > > > > > >> less
> > > >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with
> stream
> > > >> > > processors
> > > >> > > > > on
> > > >> > > > > > top
> > > >> > > > > > >> > of
> > > >> > > > > > >> > > > > Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a
> > > crutch.
> > > >> > > Samza
> > > >> > > > > > doesn't
> > > >> > > > > > >> > > have
> > > >> > > > > > >> > > > > any
> > > >> > > > > > >> > > > > >> > >>>>>>>>> built
> > > >> > > > > > >> > > > > >> > >>>>> in
> > > >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead,
> it
> > > >> > depends
> > > >> > > on
> > > >> > > > > the
> > > >> > > > > > >> > > dynamic
> > > >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to
> > > handle
> > > >> > > > restarts
> > > >> > > > > > >> when a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > > >> > > > > > >> > > > > >> > >>>>>>> made
> > > >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> > > >> standalone
> > > >> > > Samza
> > > >> > > > > > >> > container
> > > >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is
> > good,
> > > >> but I
> > > >> > > > think
> > > >> > > > > > that
> > > >> > > > > > >> > > we've
> > > >> > > > > > >> > > > > >> gone
> > > >> > > > > > >> > > > > >> > >>>>>>>>> too
> > > >> > > > > > >> > > > > >> > >>>>>> far
> > > >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > > >> > > > (SystemConsumer,
> > > >> > > > > > >> > > > > SystemProducer,
> > > >> > > > > > >> > > > > >> > >>>> etc).
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just
> > > about
> > > >> > every
> > > >> > > > > > >> component
> > > >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> > > >> > > ConfigRewriter,
> > > >> > > > > > etc).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> > > >> > forgotten,
> > > >> > > as
> > > >> > > > > > well.
> > > >> > > > > > >> > Some
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > >>>>>>>>> these
> > > >> > > > > > >> > > > > >> > >>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not
> > to
> > > >> be.
> > > >> > > This
> > > >> > > > > all
> > > >> > > > > > >> comes
> > > >> > > > > > >> > > at
> > > >> > > > > > >> > > > a
> > > >> > > > > > >> > > > > >> cost:
> > > >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is
> > > making
> > > >> it
> > > >> > > > harder
> > > >> > > > > > for
> > > >> > > > > > >> > our
> > > >> > > > > > >> > > > > users
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>> pick
> > > >> > > > > > >> > > > > >> > >>>>>> up
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It
> > > also
> > > >> > makes
> > > >> > > > it
> > > >> > > > > > >> > difficult
> > > >> > > > > > >> > > > for
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about
> > > what
> > > >> the
> > > >> > > > > > >> > > characteristics
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> > > >> characteristics
> > > >> > > > change
> > > >> > > > > > >> > > depending
> > > >> > > > > > >> > > > on
> > > >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are
> > > most
> > > >> > > visible
> > > >> > > > > in
> > > >> > > > > > the
> > > >> > > > > > >> > > > System
> > > >> > > > > > >> > > > > >> APIs.
> > > >> > > > > > >> > > > > >> > >>>>> What
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
> > > >> functional is
> > > >> > > > Kafka
> > > >> > > > > > as
> > > >> > > > > > >> its
> > > >> > > > > > >> > > > > >> > >>>>>>>>> transport
> > > >> > > > > > >> > > > > >> > >>>>>> layer.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> But
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated
> use
> > > >> cases
> > > >> > > into
> > > >> > > > > one
> > > >> > > > > > >> API:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports
> > both
> > > >> of
> > > >> > > these
> > > >> > > > > use
> > > >> > > > > > >> > cases.
> > > >> > > > > > >> > > > The
> > > >> > > > > > >> > > > > >> > >>>>>>>>> problem
> > > >> > > > > > >> > > > > >> > >>>>>> is,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we
> > > >> > > > > > >> > > > > >> > >>>>>>>>> actually want different features
> > for
> > > >> each
> > > >> > > use
> > > >> > > > > > case.
> > > >> > > > > > >> By
> > > >> > > > > > >> > > > > >> papering
> > > >> > > > > > >> > > > > >> > >>>>>>>>> over
> > > >> > > > > > >> > > > > >> > >>>>>>> these
> > > >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a
> > > single
> > > >> > API,
> > > >> > > > > we've
> > > >> > > > > > >> > > > introduced
> > > >> > > > > > >> > > > > a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > >> > > > > > >> > > > > >> > >>>>>>> leaky
> > > >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really
> like
> > > in
> > > >> (2)
> > > >> > > is
> > > >> > > > to
> > > >> > > > > > have
> > > >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs
> for
> > > >> > offsets
> > > >> > > > > (like
> > > >> > > > > > >> > Kafka).
> > > >> > > > > > >> > > > > This
> > > >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > > >> > > > > > >> > > > > >> > >>>>> with
> > > >> > > > > > >> > > > > >> > >>>>>>> (1),
> > > >> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems
> > have
> > > >> > > > different
> > > >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> > > >> mailing
> > > >> > > list
> > > >> > > > > and
> > > >> > > > > > >> the
> > > >> > > > > > >> > > SQL
> > > >> > > > > > >> > > > > >> JIRAs
> > > >> > > > > > >> > > > > >> > >>>>> about
> > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> > > >> > > replayability.
> > > >> > > > > > Kafka
> > > >> > > > > > >> > > allows
> > > >> > > > > > >> > > > us
> > > >> > > > > > >> > > > > >> to
> > > >> > > > > > >> > > > > >> > >>>>> rewind
> > > >> > > > > > >> > > > > >> > >>>>>>>>> when
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other
> > > systems
> > > >> > > don't.
> > > >> > > > In
> > > >> > > > > > some
> > > >> > > > > > >> > > > cases,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > >> > > > > > >> > > > > >> > >>>>>>> return
> > > >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > > >> > > > > > >> WikipediaSystemConsumer)
> > > >> > > > > > >> > > > > because
> > > >> > > > > > >> > > > > >> > >>>>>>>>> they
> > > >> > > > > > >> > > > > >> > >>>>>> have
> > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example.
> > > Kafka
> > > >> > > > supports
> > > >> > > > > > >> > > > > partitioning,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > >> > > > > > >> > > > > >> > >>>>> many
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
> > > >> having a
> > > >> > > > single
> > > >> > > > > > >> > > partition
> > > >> > > > > > >> > > > > for
> > > >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other
> > systems
> > > >> model
> > > >> > > > > > >> partitioning
> > > >> > > > > > >> > > > > >> > >>>> differently (e.g.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is
> also
> > a
> > > >> mess.
> > > >> > > > > > Creating
> > > >> > > > > > >> > > streams
> > > >> > > > > > >> > > > > in
> > > >> > > > > > >> > > > > >> a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> > > >> impossible.
> > > >> > > As
> > > >> > > > is
> > > >> > > > > > >> > modeling
> > > >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > > >> > > > > > >> > > > > >> > >>>>> for
> > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> > > >> partitions,
> > > >> > > > > location,
> > > >> > > > > > >> > etc).
> > > >> > > > > > >> > > > The
> > > >> > > > > > >> > > > > >> > >>>>>>>>> list
> > > >> > > > > > >> > > > > >> > >>>>> goes
> > > >> > > > > > >> > > > > >> > >>>>>>> on.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began
> writing
> > > >> Samza,
> > > >> > > > > Kafka's
> > > >> > > > > > >> > > consumer
> > > >> > > > > > >> > > > > and
> > > >> > > > > > >> > > > > >> > >>>>> producer
> > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature
> set.
> > > On
> > > >> the
> > > >> > > > > > >> > consumer-side,
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> > >>>>>>>>> had two
> > > >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
> > > consumer,
> > > >> or
> > > >> > > the
> > > >> > > > > > simple
> > > >> > > > > > >> > > > > consumer.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The
> > > >> > > > > > >> > > > > >> > >>>>>>> problem
> > > >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was
> > > that
> > > >> it
> > > >> > > > > > controlled
> > > >> > > > > > >> > your
> > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments,
> > and
> > > >> the
> > > >> > > order
> > > >> > > > > in
> > > >> > > > > > >> which
> > > >> > > > > > >> > > you
> > > >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > > >> > > > > > >> > > > > >> > >>>>> problem
> > > >> > > > > > >> > > > > >> > >>>>>>>>> with
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's
> > not
> > > >> > > simple.
> > > >> > > > > It's
> > > >> > > > > > >> > basic.
> > > >> > > > > > >> > > > You
> > > >> > > > > > >> > > > > >> > >>>>>>>>> end up
> > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really
> > low-level
> > > >> stuff
> > > >> > > > that
> > > >> > > > > > you
> > > >> > > > > > >> > > > > shouldn't.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> We
> > > >> > > > > > >> > > > > >> > >>>>>> spent a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > > >> > > > KafkaSystemConsumer
> > > >> > > > > > very
> > > >> > > > > > >> > > > robust.
> > > >> > > > > > >> > > > > >> It
> > > >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some
> > cool
> > > >> > > features:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering
> > and
> > > >> > > > > > prioritization.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> > > >> assignment
> > > >> > > to
> > > >> > > > > > support
> > > >> > > > > > >> > > > joins,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> global
> > > >> > > > > > >> > > > > >> > >>>>>> state
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)),
> > > etc.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> > > >> > checkpointing.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the
> time
> > > is
> > > >> > that
> > > >> > > > > these
> > > >> > > > > > >> > > features
> > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > >> > > > > > >> > > > > >> > >>>>>>> actually
> > > >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
> > > consumers
> > > >> > (not
> > > >> > > > just
> > > >> > > > > > >> Samza
> > > >> > > > > > >> > > > stream
> > > >> > > > > > >> > > > > >> > >>>>>> processors)
> > > >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like
> > > joins
> > > >> > and
> > > >> > > > > > partition
> > > >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> > > >> > conclusion.
> > > >> > > > > > They're
> > > >> > > > > > >> > > adding
> > > >> > > > > > >> > > > a
> > > >> > > > > > >> > > > > >> ton
> > > >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> > > >> consumer
> > > >> > > > > > >> > > implementation.
> > > >> > > > > > >> > > > > To a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > > >> > > > > > >> > > > > >> > >>>>> it's
> > > >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've
> > already
> > > >> done
> > > >> > > in
> > > >> > > > > > Samza.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up
> > > taking
> > > >> a
> > > >> > > very
> > > >> > > > > > similar
> > > >> > > > > > >> > > > > approach
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>> Samza's
> > > >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> > > implementation
> > > >> for
> > > >> > > > > > handling
> > > >> > > > > > >> > > offset
> > > >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> > > >> management
> > > >> > > > > feature
> > > >> > > > > > >> > stores
> > > >> > > > > > >> > > > > >> offset
> > > >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and
> allows
> > > >> you to
> > > >> > > > fetch
> > > >> > > > > > them
> > > >> > > > > > >> > > from
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a
> waste,
> > > >> since
> > > >> > we
> > > >> > > > > could
> > > >> > > > > > >> have
> > > >> > > > > > >> > > > shared
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>> work
> > > >> > > > > > >> > > > > >> > >>>>>> if
> > > >> > > > > > >> > > > > >> > >>>>>>>>> it
> > > >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
> > > >> get-go.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> > > >> radical
> > > >> > > > > > proposal.
> > > >> > > > > > >> > Samza
> > > >> > > > > > >> > > > is
> > > >> > > > > > >> > > > > >> > >>>>> relatively
> > > >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd
> venture
> > to
> > > >> say
> > > >> > > that
> > > >> > > > > > we're
> > > >> > > > > > >> > > near a
> > > >> > > > > > >> > > > > 1.0
> > > >> > > > > > >> > > > > >> > >>>>>> release.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > > >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take
> what
> > > >> we've
> > > >> > > > > learned,
> > > >> > > > > > and
> > > >> > > > > > >> > > begin
> > > >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > > >> > > > > > >> > > > > >> > >>>>>>> about
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we
> > > >> change if
> > > >> > > we
> > > >> > > > > were
> > > >> > > > > > >> > > starting
> > > >> > > > > > >> > > > > >> from
> > > >> > > > > > >> > > > > >> > >>>>>> scratch?
> > > >> > > > > > >> > > > > >> > >>>>>>>>> My
> > > >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the
> > *only*
> > > >> way
> > > >> > to
> > > >> > > > run
> > > >> > > > > > Samza
> > > >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all
> > direct
> > > >> > > > > dependences
> > > >> > > > > > on
> > > >> > > > > > >> > > YARN,
> > > >> > > > > > >> > > > > >> Mesos,
> > > >> > > > > > >> > > > > >> > >>>> etc.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to
> > support
> > > >> only
> > > >> > > > Kafka
> > > >> > > > > > as
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > > stream
> > > >> > > > > > >> > > > > >> > >>>>>> processing
> > > >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics,
> > > logging,
> > > >> > > > > > >> serialization,
> > > >> > > > > > >> > > and
> > > >> > > > > > >> > > > > >> > >>>>>>>>> config
> > > >> > > > > > >> > > > > >> > >>>>>>> systems,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues
> > > that
> > > >> I
> > > >> > > > > outlined
> > > >> > > > > > >> > above.
> > > >> > > > > > >> > > It
> > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > >> > > > > > >> > > > > >> > >>>>> also
> > > >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base
> pretty
> > > >> > > > dramatically.
> > > >> > > > > > >> > > Supporting
> > > >> > > > > > >> > > > > >> only
> > > >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will
> allow
> > > >> Samza
> > > >> > to
> > > >> > > be
> > > >> > > > > > >> executed
> > > >> > > > > > >> > > on
> > > >> > > > > > >> > > > > YARN
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > > >> > > > Marathon/Aurora),
> > > >> > > > > or
> > > >> > > > > > >> most
> > > >> > > > > > >> > > > other
> > > >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > > >> > > > > > >> > > > > >> > >>>>>>> deployment
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a
> > lot
> > > >> > easier
> > > >> > > > for
> > > >> > > > > > new
> > > >> > > > > > >> > > users.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without
> > > YARN.
> > > >> > The
> > > >> > > > drop
> > > >> > > > > > in
> > > >> > > > > > >> > > mailing
> > > >> > > > > > >> > > > > >> list
> > > >> > > > > > >> > > > > >> > >>>>>> traffic
> > > >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
> > > >> overdue to
> > > >> > > me.
> > > >> > > > > The
> > > >> > > > > > >> > > reality
> > > >> > > > > > >> > > > > is,
> > > >> > > > > > >> > > > > >> > >>>>> everyone
> > > >> > > > > > >> > > > > >> > >>>>>>>>> that
> > > >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
> > > >> Kafka.
> > > >> > We
> > > >> > > > > > basically
> > > >> > > > > > >> > > > require
> > > >> > > > > > >> > > > > >> it
> > > >> > > > > > >> > > > > >> > >>>>>> already
> > > >> > > > > > >> > > > > >> > >>>>>>> in
> > > >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work.
> > > Those
> > > >> > that
> > > >> > > > are
> > > >> > > > > > >> using
> > > >> > > > > > >> > > > other
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > >> > > > > > >> > > > > >> > >>>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest
> into
> > > >> Kafka
> > > >> > > (1),
> > > >> > > > > and
> > > >> > > > > > >> then
> > > >> > > > > > >> > > > they
> > > >> > > > > > >> > > > > do
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
> > > >> already
> > > >> > > > > > discussion (
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> >
> > > >> > > > > >
> > > >> > >
> > > >>
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > >> > > > > > >> > > > > >> > >>>>> 767
> > > >> > > > > > >> > > > > >> > >>>>>>>>> )
> > > >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into
> > > Kafka
> > > >> > > > extremely
> > > >> > > > > > >> easy.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple
> > with
> > > >> > Kafka,
> > > >> > > > we
> > > >> > > > > > can
> > > >> > > > > > >> > > > leverage
> > > >> > > > > > >> > > > > a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > >> > > > > > >> > > > > >> > >>>>>>> their
> > > >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> > > >> maintain
> > > >> > > our
> > > >> > > > > own
> > > >> > > > > > >> > config,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > > >> > > > > > >> > > > > >> > >>>>> etc.
> > > >> > > > > > >> > > > > >> > >>>>>>> We
> > > >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same
> libraries,
> > > and
> > > >> > make
> > > >> > > > them
> > > >> > > > > > >> > better.
> > > >> > > > > > >> > > > This
> > > >> > > > > > >> > > > > >> > >>>>>>>>> will
> > > >> > > > > > >> > > > > >> >
> > > >> ...
> > > >>
> > > >> [Message clipped]
> > > >
> > > >
> > > >
> > >
> >
> >
> >
> > --
> > Jordan Shaw
> > Full Stack Software Engineer
> > PubNub Inc
> > 1045 17th St
> > San Francisco, CA 94107
> >
>

RE: Thoughts and obesrvations on Samza

Posted by Garry Turkington <g....@improvedigital.com>.
Hi,

I'm also supportive of Jay's option 5. There is a risk the "transformer API" -- I'd have preferred Metamorphosis but it's too hard to type! -- takes on a life of its own and we end up with two very different things but given how good the Kafka community has been at introducing new producer and consumer clients and giving very clear guidance on when they are production ready this is a danger I believe can be managed. It'd also be excellent to get some working code to beat around the notions of stream processing atop a system with transacdtional messages.

On the question of whether to keep or deprecate SystemConsumer/Producer I believe we need get a better understanding over the next while of just what the Samza community is looking for in such connectivity. For my own use cases I have been looking to add additional implementations primarily to use Samza as the data ingress and egress component around Kafka. Writing external clients that require their own reliability and scalability management gets old real fast and pushing this into a simple Samza job that reads from system X and pushes into Kafka (or vice versa) was the obvious choice for me in the current model. For this type of usage though copycat is likely much superior (obviously needs proven) and the question then is if most Samza users look to the system implementations to also act as a front-end into Kafka or if significant usage is indeed intended to have the alternative systems as the primary message source. That understanding will I think give much clarity in just what value the abstraction overhead of the current model brings.

Garry

-----Original Message-----
From: Yan Fang [mailto:yanfang724@gmail.com] 
Sent: 13 July 2015 19:58
To: dev@samza.apache.org
Subject: Re: Thoughts and obesrvations on Samza

I am leaning to Jay's fifth approach. It is not radical and gives us some
time to see the outcome.

In addition, I would suggest:

1) Keep the SystemConsumer/SystemProducer API. Because current
SystemConsumer/SystemProducer API satisfies the usage (From Joardan, and
even Garry's feedback) and is not so broken that we want to deprecate it.
Though there are some issues in implemnting the Kinesis, they are not
unfixable. Nothing should prevent Samza, as a stream processing system, to
support other systems. In addition, there already are some systems
exiting besides Kafka: ElasticSearch (committed to the master), HDFS
(patch-available), S3( from the mailing list), Kinesis (developing in
another repository), ActiveMQ (in two months). We may want to see how those
go before we "kill" them.

2) Can have some Samza devs involved in Kafka's "transformer" client API.
This can not only help the future integration (if any) much easier, because
they have knowledge about both systems, but also good for Kafka's
community, because Samza devs have the streaming process experience that
Kafka devs may miss.

3) Samza's partition management system may still support other systems.
Though the partition management logic in samza-kafka will be moved to
Kafka, its still useful for other systems that do not have the partition
management layer.

4) Start sharing the docs/websites and using the same terminology (though
do not know how to do this exactly. :). This will reduce the future
confusion and does not hurt Samza's independency.

In my opinion, Samza, as a standalone project, still can (and already)
heavily replying on Kafka, and even more tuned for Kafka-specific usage.
Kafka, also can embed Samza in the document, I do not see anything prevent
doing this.

Thanks,

Fang, Yan
yanfang724@gmail.com

On Mon, Jul 13, 2015 at 11:25 AM, Jordan Shaw <jo...@pubnub.com> wrote:

> Jay,
> I think doing this iteratively in smaller chunks is a better way to go as
> new issues arise. As Navina said Kafka is a "stream system" and Samza is a
> "stream processor" and those two ideas should be mutually exclusive.
>
> -Jordan
>
> On Mon, Jul 13, 2015 at 10:06 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hmm, thought about this more. Maybe this is just too much too quick.
> > Overall I think there is some enthusiasm for the proposal but it's not
> > really unanimous enough to make any kind of change this big cleanly. The
> > board doesn't really like the merging stuff, user's are concerned about
> > compatibility, I didn't feel there was unanimous agreement on dropping
> > SystemConsumer, etc. Even if this is the right end state to get to,
> > probably trying to push all this through at once isn't the right way to
> do
> > it.
> >
> > So let me propose a kind of fifth (?) option which I think is less
> dramatic
> > and let's things happen gradually. I think this is kind of like combining
> > the first part of Yi's proposal and Jakob's third option, leaving the
> rest
> > to be figured out incrementally:
> >
> > Option 5: We continue the prototype I shared and propose that as a kind
> of
> > "transformer" client API in Kafka. This isn't really a full-fledged
> stream
> > processing layer, more like a supped up consumer api for munging topics.
> > This would let us figure out some of the technical bits, how to do this
> on
> > Kafka's group management features, how to integrate the txn feature to do
> > the exactly-once stuff in these transformations, and get all this stuff
> > solid. This api would have valid uses in it's own right, especially when
> > your transformation will be embedded inside an existing service or
> > application which isn't possible with Samza (or other existing systems
> that
> > I know of).
> >
> > Independently we can iterate on some of the ideas of the original
> proposal
> > individually and figure out how (if at all) to make use of this
> > functionality. This can be done bit-by-bit:
> > - Could be that the existing StreamTask API ends up wrapping this
> > - Could end up exposed directly in Samza as Yi proposed
> > - Could be that just the lower-level group-management stuff get's used,
> and
> > in this case it could be either just for standalone mode, or always
> > - Could be that it stays as-is
> >
> > The advantage of this is it is lower risk...we basically don't have to
> make
> > 12 major decisions all at once that kind of hinge on what amounts to a
> > pretty aggressive rewrite. The disadvantage of this is it is a bit more
> > confusing as all this is getting figured out.
> >
> > As with some of the other stuff, this would require a further discussion
> in
> > the Kafka community if people do like this approach.
> >
> > Thoughts?
> >
> > -Jay
> >
> >
> >
> >
> > On Sun, Jul 12, 2015 at 10:52 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Chris,
> > >
> > > Yeah, I'm obviously in favor of this.
> > >
> > > The sub-project approach seems the ideal way to take a graceful step in
> > > this direction, so I will ping the board folks and see why they are
> > > discouraged, it would be good to understand that. If we go that route
> we
> > > would need to do a similar discussion in the Kafka list (but makes
> sense
> > to
> > > figure out first if it is what Samza wants).
> > >
> > > Irrespective of how it's implemented, though, to me the important
> things
> > > are the following:
> > > 1. Unify the website, config, naming, docs, metrics, etc--basically fix
> > > the product experience so the "stream" and the "processing" feel like a
> > > single user experience and brand. This seems minor but I think is a
> > really
> > > big deal.
> > > 2. Make "standalone" mode a first class citizen and have a real
> technical
> > > plan to be able to support cluster managers other than YARN.
> > > 3. Make the config and out-of-the-box experience more usable
> > >
> > > I think that prototype gives a practical example of how 1-3 could be
> done
> > > and we should pursue it. This is a pretty radical change, so I wouldn't
> > be
> > > shocked if people didn't want to take a step like that.
> > >
> > > Maybe it would make sense to see if people are on board with that
> general
> > > idea, and then try to get some advice on sub-projects in parallel and
> > nail
> > > down those details?
> > >
> > > -Jay
> > >
> > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
> criccomini@apache.org>
> > > wrote:
> > >
> > >> Hey all,
> > >>
> > >> I want to start by saying that I'm absolutely thrilled to be a part of
> > >> this
> > >> community. The amount of level-headed, thoughtful, educated discussion
> > >> that's gone on over the past ~10 days is overwhelming. Wonderful.
> > >>
> > >> It seems like discussion is waning a bit, and we've reached some
> > >> conclusions. There are several key emails in this threat, which I want
> > to
> > >> call out:
> > >>
> > >> 1. Jakob's summary of the three potential ways forward.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > >> 2. Julian's call out that we should be focusing on community over
> code.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > >> 3. Martin's summary about the benefits of merging communities.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > >> 4. Jakob's comments about the distinction between community and code
> > >> paths.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> > >>
> > >> I agree with the comments on all of these emails. I think Martin's
> > summary
> > >> of his position aligns very closely with my own. To that end, I think
> we
> > >> should get concrete about what the proposal is, and call a vote on it.
> > >> Given that Jay, Martin, and I seem to be aligning fairly closely, I
> > think
> > >> we should start with:
> > >>
> > >> 1. [community] Make Samza a subproject of Kafka.
> > >> 2. [community] Make all Samza PMC/committers committers of the
> > subproject.
> > >> 3. [community] Migrate Samza's website/documentation into Kafka's.
> > >> 4. [code] Have the Samza community and the Kafka community start a
> > >> from-scratch reboot together in the new Kafka subproject. We can
> > >> borrow/copy &  paste significant chunks of code from Samza's code
> base.
> > >> 5. [code] The subproject would intentionally eliminate support for
> both
> > >> other streaming systems and all deployment systems.
> > >> 6. [code] Attempt to provide a bridge from our SystemConsumer to
> KIP-26
> > >> (copy cat)
> > >> 7. [code] Attempt to provide a bridge from the new subproject's
> > processor
> > >> interface to our legacy StreamTask interface.
> > >> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> > >> subproject that has a fault-tolerant container with state management.
> > >>
> > >> It's likely that (6) and (7) won't be fully drop-in. Still, the closer
> > we
> > >> can get, the better it's going to be for our existing community.
> > >>
> > >> One thing that I didn't touch on with (2) is whether any Samza PMC
> > members
> > >> should be rolled into Kafka PMC membership as well (though, Jay and
> > Jakob
> > >> are already PMC members on both). I think that Samza's community
> > deserves
> > >> a
> > >> voice on the PMC, so I'd propose that we roll at least a few PMC
> members
> > >> into the Kafka PMC, but I don't have a strong framework for which
> people
> > >> to
> > >> pick.
> > >>
> > >> Before (8), I think that Samza's TLP can continue to commit bug fixes
> > and
> > >> patches as it sees fit, provided that we openly communicate that we
> > won't
> > >> necessarily migrate new features to the new subproject, and that the
> TLP
> > >> will be shut down after the migration to the Kafka subproject occurs.
> > >>
> > >> Jakob, I could use your guidance here about about how to achieve this
> > from
> > >> an Apache process perspective (sorry).
> > >>
> > >> * Should I just call a vote on this proposal?
> > >> * Should it happen on dev or private?
> > >> * Do committers have binding votes, or just PMC?
> > >>
> > >> Having trouble finding much detail on the Apache wikis. :(
> > >>
> > >> Cheers,
> > >> Chris
> > >>
> > >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com>
> wrote:
> > >>
> > >> > Thanks, Jay. This argument persuaded me actually. :)
> > >> >
> > >> > Fang, Yan
> > >> > yanfang724@gmail.com
> > >> >
> > >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > >> >
> > >> > > Hey Yan,
> > >> > >
> > >> > > Yeah philosophically I think the argument is that you should
> capture
> > >> the
> > >> > > stream in Kafka independent of the transformation. This is
> > obviously a
> > >> > > Kafka-centric view point.
> > >> > >
> > >> > > Advantages of this:
> > >> > > - In practice I think this is what e.g. Storm people often end up
> > >> doing
> > >> > > anyway. You usually need to throttle any access to a live serving
> > >> > database.
> > >> > > - Can have multiple subscribers and they get the same thing
> without
> > >> > > additional load on the source system.
> > >> > > - Applications can tap into the stream if need be by subscribing.
> > >> > > - You can debug your transformation by tailing the Kafka topic
> with
> > >> the
> > >> > > console consumer
> > >> > > - Can tee off the same data stream for batch analysis or Lambda
> arch
> > >> > style
> > >> > > re-processing
> > >> > >
> > >> > > The disadvantage is that it will use Kafka resources. But the idea
> > is
> > >> > > eventually you will have multiple subscribers to any data source
> (at
> > >> > least
> > >> > > for monitoring) so you will end up there soon enough anyway.
> > >> > >
> > >> > > Down the road the technical benefit is that I think it gives us a
> > good
> > >> > path
> > >> > > towards end-to-end exactly once semantics from source to
> > destination.
> > >> > > Basically the connectors need to support idempotence when talking
> to
> > >> > Kafka
> > >> > > and we need the transactional write feature in Kafka to make the
> > >> > > transformation atomic. This is actually pretty doable if you
> > separate
> > >> > > connector=>kafka problem from the generic transformations which
> are
> > >> > always
> > >> > > kafka=>kafka. However I think it is quite impossible to do in a
> > >> > all_things
> > >> > > => all_things environment. Today you can say "well the semantics
> of
> > >> the
> > >> > > Samza APIs depend on the connectors you use" but it is actually
> > worse
> > >> > then
> > >> > > that because the semantics actually depend on the pairing of
> > >> > connectors--so
> > >> > > not only can you probably not get a usable "exactly once"
> guarantee
> > >> > > end-to-end it can actually be quite hard to reverse engineer what
> > >> > property
> > >> > > (if any) your end-to-end flow has if you have heterogenous
> systems.
> > >> > >
> > >> > > -Jay
> > >> > >
> > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > > > {quote}
> > >> > > > maintained in a separate repository and retaining the existing
> > >> > > > committership but sharing as much else as possible (website,
> etc)
> > >> > > > {quote}
> > >> > > >
> > >> > > > Overall, I agree on this idea. Now the question is more about
> "how
> > >> to
> > >> > do
> > >> > > > it".
> > >> > > >
> > >> > > > On the other hand, one thing I want to point out is that, if we
> > >> decide
> > >> > to
> > >> > > > go this way, how do we want to support
> > >> > > > otherSystem-transformation-otherSystem use case?
> > >> > > >
> > >> > > > Basically, there are four user groups here:
> > >> > > >
> > >> > > > 1. Kafka-transformation-Kafka
> > >> > > > 2. Kafka-transformation-otherSystem
> > >> > > > 3. otherSystem-transformation-Kafka
> > >> > > > 4. otherSystem-transformation-otherSystem
> > >> > > >
> > >> > > > For group 1, they can easily use the new Samza library to
> achieve.
> > >> For
> > >> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka
> or
> > >> > Kafka->
> > >> > > > transformation -> copyCat.
> > >> > > >
> > >> > > > The problem is for group 4. Do we want to abandon this or still
> > >> support
> > >> > > it?
> > >> > > > Of course, this use case can be achieved by using copyCat ->
> > >> > > transformation
> > >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> > persuade
> > >> > them
> > >> > > to
> > >> > > > do this long chain. If yes, it will also be a win for Kafka too.
> > Or
> > >> if
> > >> > > > there is no one in this community actually doing this so far,
> > maybe
> > >> ok
> > >> > to
> > >> > > > not support the group 4 directly.
> > >> > > >
> > >> > > > Thanks,
> > >> > > >
> > >> > > > Fang, Yan
> > >> > > > yanfang724@gmail.com
> > >> > > >
> > >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
> > >> wrote:
> > >> > > >
> > >> > > > > Yeah I agree with this summary. I think there are kind of two
> > >> > questions
> > >> > > > > here:
> > >> > > > > 1. Technically does alignment/reliance on Kafka make sense
> > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
> alignment
> > >> with
> > >> > > > Kafka
> > >> > > > > make sense
> > >> > > > >
> > >> > > > > Personally I do think both of these things would be really
> > >> valuable,
> > >> > > and
> > >> > > > > would dramatically alter the trajectory of the project.
> > >> > > > >
> > >> > > > > My preference would be to see if people can mostly agree on a
> > >> > direction
> > >> > > > > rather than splintering things off. From my point of view the
> > >> ideal
> > >> > > > outcome
> > >> > > > > of all the options discussed would be to make Samza a closely
> > >> aligned
> > >> > > > > subproject, maintained in a separate repository and retaining
> > the
> > >> > > > existing
> > >> > > > > committership but sharing as much else as possible (website,
> > >> etc). No
> > >> > > > idea
> > >> > > > > about how these things work, Jacob, you probably know more.
> > >> > > > >
> > >> > > > > No discussion amongst the Kafka folks has happened on this,
> but
> > >> > likely
> > >> > > we
> > >> > > > > should figure out what the Samza community actually wants
> first.
> > >> > > > >
> > >> > > > > I admit that this is a fairly radical departure from how
> things
> > >> are.
> > >> > > > >
> > >> > > > > If that doesn't fly, I think, yeah we could leave Samza as it
> is
> > >> and
> > >> > do
> > >> > > > the
> > >> > > > > more radical reboot inside Kafka. From my point of view that
> > does
> > >> > leave
> > >> > > > > things in a somewhat confusing state since now there are two
> > >> stream
> > >> > > > > processing systems more or less coupled to Kafka in large part
> > >> made
> > >> > by
> > >> > > > the
> > >> > > > > same people. But, arguably that might be a cleaner way to make
> > the
> > >> > > > cut-over
> > >> > > > > and perhaps less risky for Samza community since if it works
> > >> people
> > >> > can
> > >> > > > > switch and if it doesn't nothing will have changed. Dunno, how
> > do
> > >> > > people
> > >> > > > > feel about this?
> > >> > > > >
> > >> > > > > -Jay
> > >> > > > >
> > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> > jghoman@gmail.com>
> > >> > > wrote:
> > >> > > > >
> > >> > > > > > >  This leads me to thinking that merging projects and
> > >> communities
> > >> > > > might
> > >> > > > > > be a good idea: with the union of experience from both
> > >> communities,
> > >> > > we
> > >> > > > > will
> > >> > > > > > probably build a better system that is better for users.
> > >> > > > > > Is this what's being proposed though? Merging the projects
> > seems
> > >> > like
> > >> > > > > > a consequence of at most one of the three directions under
> > >> > > discussion:
> > >> > > > > > 1) Samza 2.0: The Samza community relies more heavily on
> Kafka
> > >> for
> > >> > > > > > configuration, etc. (to a greater or lesser extent to be
> > >> > determined)
> > >> > > > > > but the Samza community would not automatically merge withe
> > >> Kafka
> > >> > > > > > community (the Phoenix/HBase example is a good one here).
> > >> > > > > > 2) Samza Reboot: The Samza community continues to exist
> with a
> > >> > > limited
> > >> > > > > > project scope, but similarly would not need to be part of
> the
> > >> Kafka
> > >> > > > > > community (ie given committership) to progress.  Here, maybe
> > the
> > >> > > Samza
> > >> > > > > > team would become a subproject of Kafka (the Board frowns on
> > >> > > > > > subprojects at the moment, so I'm not sure if that's even
> > >> > feasible),
> > >> > > > > > but that would not be required.
> > >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option
> > the
> > >> > Kafka
> > >> > > > > > team builds its own streaming library, possibly off of Jay's
> > >> > > > > > prototype, which has not direct lineage to the Samza team.
> > >> There's
> > >> > > no
> > >> > > > > > reason for the Kafka team to bring in the Samza team.
> > >> > > > > >
> > >> > > > > > Is the Kafka community on board with this?
> > >> > > > > >
> > >> > > > > > To be clear, all three options under discussion are
> > interesting,
> > >> > > > > > technically valid and likely healthy directions for the
> > project.
> > >> > > > > > Also, they are not mutually exclusive.  The Samza community
> > >> could
> > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
> community
> > >> went
> > >> > > > > > forward with 'Hey Samza!'  My points above are directed
> > >> entirely at
> > >> > > > > > the community aspect of these choices.
> > >> > > > > > -Jakob
> > >> > > > > >
> > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> > roger.hoover@gmail.com>
> > >> > > wrote:
> > >> > > > > > > That's great.  Thanks, Jay.
> > >> > > > > > >
> > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> > jay@confluent.io>
> > >> > > wrote:
> > >> > > > > > >
> > >> > > > > > >> Yeah totally agree. I think you have this issue even
> today,
> > >> > right?
> > >> > > > > I.e.
> > >> > > > > > if
> > >> > > > > > >> you need to make a simple config change and you're
> running
> > in
> > >> > YARN
> > >> > > > > today
> > >> > > > > > >> you end up bouncing the job which then rebuilds state. I
> > >> think
> > >> > the
> > >> > > > fix
> > >> > > > > > is
> > >> > > > > > >> exactly what you described which is to have a long
> timeout
> > on
> > >> > > > > partition
> > >> > > > > > >> movement for stateful jobs so that if a job is just
> getting
> > >> > > bounced,
> > >> > > > > and
> > >> > > > > > >> the cluster manager (or admin) is smart enough to restart
> > it
> > >> on
> > >> > > the
> > >> > > > > same
> > >> > > > > > >> host when possible, it can optimistically reuse any
> > existing
> > >> > state
> > >> > > > it
> > >> > > > > > finds
> > >> > > > > > >> on disk (if it is valid).
> > >> > > > > > >>
> > >> > > > > > >> So in this model the charter of the CM is to place
> > processes
> > >> as
> > >> > > > > > stickily as
> > >> > > > > > >> possible and to restart or re-place failed processes. The
> > >> > charter
> > >> > > of
> > >> > > > > the
> > >> > > > > > >> partition management system is to control the assignment
> of
> > >> work
> > >> > > to
> > >> > > > > > these
> > >> > > > > > >> processes. The nice thing about this is that the work
> > >> > assignment,
> > >> > > > > > timeouts,
> > >> > > > > > >> behavior, configs, and code will all be the same across
> all
> > >> > > cluster
> > >> > > > > > >> managers.
> > >> > > > > > >>
> > >> > > > > > >> So I think that prototype would actually give you exactly
> > >> what
> > >> > you
> > >> > > > > want
> > >> > > > > > >> today for any cluster manager (or manual placement +
> > restart
> > >> > > script)
> > >> > > > > > that
> > >> > > > > > >> was sticky in terms of host placement since there is
> > already
> > >> a
> > >> > > > > > configurable
> > >> > > > > > >> partition movement timeout and task-by-task state reuse
> > with
> > >> a
> > >> > > check
> > >> > > > > on
> > >> > > > > > >> state validity.
> > >> > > > > > >>
> > >> > > > > > >> -Jay
> > >> > > > > > >>
> > >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > >> > > > roger.hoover@gmail.com
> > >> > > > > >
> > >> > > > > > >> wrote:
> > >> > > > > > >>
> > >> > > > > > >> > That would be great to let Kafka do as much heavy
> lifting
> > >> as
> > >> > > > > possible
> > >> > > > > > and
> > >> > > > > > >> > make it easier for other languages to implement Samza
> > apis.
> > >> > > > > > >> >
> > >> > > > > > >> > One thing to watch out for is the interplay between
> > Kafka's
> > >> > > group
> > >> > > > > > >> > management and the external scheduler/process manager's
> > >> fault
> > >> > > > > > tolerance.
> > >> > > > > > >> > If a container dies, the Kafka group membership
> protocol
> > >> will
> > >> > > try
> > >> > > > to
> > >> > > > > > >> assign
> > >> > > > > > >> > it's tasks to other containers while at the same time
> the
> > >> > > process
> > >> > > > > > manager
> > >> > > > > > >> > is trying to relaunch the container.  Without some
> > >> > consideration
> > >> > > > for
> > >> > > > > > this
> > >> > > > > > >> > (like a configurable amount of time to wait before
> Kafka
> > >> > alters
> > >> > > > the
> > >> > > > > > group
> > >> > > > > > >> > membership), there may be thrashing going on which is
> > >> > especially
> > >> > > > bad
> > >> > > > > > for
> > >> > > > > > >> > containers with large amounts of local state.
> > >> > > > > > >> >
> > >> > > > > > >> > Someone else pointed this out already but I thought it
> > >> might
> > >> > be
> > >> > > > > worth
> > >> > > > > > >> > calling out again.
> > >> > > > > > >> >
> > >> > > > > > >> > Cheers,
> > >> > > > > > >> >
> > >> > > > > > >> > Roger
> > >> > > > > > >> >
> > >> > > > > > >> >
> > >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> > >> jay@confluent.io>
> > >> > > > > wrote:
> > >> > > > > > >> >
> > >> > > > > > >> > > Hey Roger,
> > >> > > > > > >> > >
> > >> > > > > > >> > > I couldn't agree more. We spent a bunch of time
> talking
> > >> to
> > >> > > > people
> > >> > > > > > and
> > >> > > > > > >> > that
> > >> > > > > > >> > > is exactly the stuff we heard time and again. What
> > makes
> > >> it
> > >> > > > hard,
> > >> > > > > of
> > >> > > > > > >> > > course, is that there is some tension between
> > >> compatibility
> > >> > > with
> > >> > > > > > what's
> > >> > > > > > >> > > there now and making things better for new users.
> > >> > > > > > >> > >
> > >> > > > > > >> > > I also strongly agree with the importance of
> > >> multi-language
> > >> > > > > > support. We
> > >> > > > > > >> > are
> > >> > > > > > >> > > talking now about Java, but for application
> development
> > >> use
> > >> > > > cases
> > >> > > > > > >> people
> > >> > > > > > >> > > want to work in whatever language they are using
> > >> elsewhere.
> > >> > I
> > >> > > > > think
> > >> > > > > > >> > moving
> > >> > > > > > >> > > to a model where Kafka itself does the group
> > membership,
> > >> > > > lifecycle
> > >> > > > > > >> > control,
> > >> > > > > > >> > > and partition assignment has the advantage of putting
> > all
> > >> > that
> > >> > > > > > complex
> > >> > > > > > >> > > stuff behind a clean api that the clients are already
> > >> going
> > >> > to
> > >> > > > be
> > >> > > > > > >> > > implementing for their consumer, so the added
> > >> functionality
> > >> > > for
> > >> > > > > > stream
> > >> > > > > > >> > > processing beyond a consumer becomes very minor.
> > >> > > > > > >> > >
> > >> > > > > > >> > > -Jay
> > >> > > > > > >> > >
> > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > >> > > > > > roger.hoover@gmail.com>
> > >> > > > > > >> > > wrote:
> > >> > > > > > >> > >
> > >> > > > > > >> > > > Metamorphosis...nice. :)
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > This has been a great discussion.  As a user of
> Samza
> > >> > who's
> > >> > > > > > recently
> > >> > > > > > >> > > > integrated it into a relatively large
> organization, I
> > >> just
> > >> > > > want
> > >> > > > > to
> > >> > > > > > >> add
> > >> > > > > > >> > > > support to a few points already made.
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
> > >> currently
> > >> > > > exists
> > >> > > > > > that
> > >> > > > > > >> > I've
> > >> > > > > > >> > > > experienced are:
> > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
> environments
> > >> > where
> > >> > > > > Puppet
> > >> > > > > > >> > would
> > >> > > > > > >> > > do
> > >> > > > > > >> > > > just fine but it was the only mechanism to get
> fault
> > >> > > > tolerance.
> > >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> > >> configuring
> > >> > > most
> > >> > > > > of
> > >> > > > > > the
> > >> > > > > > >> > job
> > >> > > > > > >> > > > in code rather than config files.  In general, I
> > think
> > >> the
> > >> > > > goal
> > >> > > > > > >> should
> > >> > > > > > >> > be
> > >> > > > > > >> > > > to make it harder to make mistakes, especially of
> the
> > >> kind
> > >> > > > where
> > >> > > > > > the
> > >> > > > > > >> > code
> > >> > > > > > >> > > > expects something and the config doesn't match.
> The
> > >> > current
> > >> > > > > > config
> > >> > > > > > >> is
> > >> > > > > > >> > > > quite intricate and error-prone.  For example, the
> > >> > > application
> > >> > > > > > logic
> > >> > > > > > >> > may
> > >> > > > > > >> > > > depend on bootstrapping a topic but rather than
> > >> asserting
> > >> > > that
> > >> > > > > in
> > >> > > > > > the
> > >> > > > > > >> > > code,
> > >> > > > > > >> > > > you have to rely on getting the config right.
> > Likewise
> > >> > with
> > >> > > > > > serdes,
> > >> > > > > > >> > the
> > >> > > > > > >> > > > Java representations produced by various serdes
> > (JSON,
> > >> > Avro,
> > >> > > > > etc.)
> > >> > > > > > >> are
> > >> > > > > > >> > > not
> > >> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> > >> without
> > >> > > > > changing
> > >> > > > > > >> the
> > >> > > > > > >> > > > code.   It would be nice for jobs to be able to
> > assert
> > >> > what
> > >> > > > they
> > >> > > > > > >> expect
> > >> > > > > > >> > > > from their input topics in terms of partitioning.
> > >> This is
> > >> > > > > > getting a
> > >> > > > > > >> > > little
> > >> > > > > > >> > > > off topic but I was even thinking about creating a
> > >> "Samza
> > >> > > > config
> > >> > > > > > >> > linter"
> > >> > > > > > >> > > > that would sanity check a set of configs.
> Especially
> > >> in
> > >> > > > > > >> organizations
> > >> > > > > > >> > > > where config is managed by a different team than
> the
> > >> > > > application
> > >> > > > > > >> > > developer,
> > >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > >> > > > > > >> > > > 3) Java/Scala centric - for many teams (especially
> > >> > > DevOps-type
> > >> > > > > > >> folks),
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > pain of the Java toolchain (maven, slow builds,
> weak
> > >> > command
> > >> > > > > line
> > >> > > > > > >> > > support,
> > >> > > > > > >> > > > configuration over convention) really inhibits
> > >> > productivity.
> > >> > > > As
> > >> > > > > > more
> > >> > > > > > >> > and
> > >> > > > > > >> > > > more high-quality clients become available for
> > Kafka, I
> > >> > hope
> > >> > > > > > they'll
> > >> > > > > > >> > > follow
> > >> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> > >> proposals
> > >> > > in
> > >> > > > > this
> > >> > > > > > >> > thread
> > >> > > > > > >> > > > but please consider other languages in the
> ecosystem
> > as
> > >> > > well.
> > >> > > > > > From
> > >> > > > > > >> > what
> > >> > > > > > >> > > > I've heard, Spark has more Python users than
> > >> Java/Scala.
> > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > >
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > >> > > > > > >> > > > and are working on a Yeoman generator
> > >> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> > >> > > Jython/Samza
> > >> > > > > > >> projects
> > >> > > > > > >> > to
> > >> > > > > > >> > > > alleviate some of the pain)
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > I also want to underscore Jay's point about
> improving
> > >> the
> > >> > > user
> > >> > > > > > >> > > experience.
> > >> > > > > > >> > > > That's a very important factor for adoption.  I
> think
> > >> the
> > >> > > goal
> > >> > > > > > should
> > >> > > > > > >> > be
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > make Samza as easy to get started with as something
> > >> like
> > >> > > > > Logstash.
> > >> > > > > > >> > > > Logstash is vastly inferior in terms of
> capabilities
> > to
> > >> > > Samza
> > >> > > > > but
> > >> > > > > > >> it's
> > >> > > > > > >> > > easy
> > >> > > > > > >> > > > to get started and that makes a big difference.
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > Cheers,
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > Roger
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> > Francisci
> > >> > > > Morales <
> > >> > > > > > >> > > > gdfm@apache.org> wrote:
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> > >> Metamorphosis
> > >> > > is
> > >> > > > a
> > >> > > > > > clear
> > >> > > > > > >> > > > winner
> > >> > > > > > >> > > > > :)
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > --
> > >> > > > > > >> > > > > Gianmarco
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> > >> Morales
> > >> > <
> > >> > > > > > >> > > gdfm@apache.org
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > wrote:
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > > Hi,
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > @Martin, thanks for you comments.
> > >> > > > > > >> > > > > > Maybe I'm missing some important point, but I
> > think
> > >> > > > coupling
> > >> > > > > > the
> > >> > > > > > >> > > > releases
> > >> > > > > > >> > > > > > is actually a *good* thing.
> > >> > > > > > >> > > > > > To make an example, would it be better if the
> MR
> > >> and
> > >> > > HDFS
> > >> > > > > > >> > components
> > >> > > > > > >> > > of
> > >> > > > > > >> > > > > > Hadoop had different release schedules?
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > Actually, keeping the discussion in a single
> > place
> > >> > would
> > >> > > > > make
> > >> > > > > > >> > > agreeing
> > >> > > > > > >> > > > on
> > >> > > > > > >> > > > > > releases (and backwards compatibility) much
> > >> easier, as
> > >> > > > > > everybody
> > >> > > > > > >> > > would
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > > responsible for the whole codebase.
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > That said, I like the idea of absorbing
> > samza-core
> > >> as
> > >> > a
> > >> > > > > > >> > sub-project,
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > > leave the fancy stuff separate.
> > >> > > > > > >> > > > > > It probably gives 90% of the benefits we have
> > been
> > >> > > > > discussing
> > >> > > > > > >> here.
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > Cheers,
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > --
> > >> > > > > > >> > > > > > Gianmarco
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > >> > jay.kreps@gmail.com
> > >> > > >
> > >> > > > > > wrote:
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >> Hey Martin,
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> I agree coupling release schedules is a
> > downside.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> Definitely we can try to solve some of the
> > >> > integration
> > >> > > > > > problems
> > >> > > > > > >> in
> > >> > > > > > >> > > > > >> Confluent Platform or in other distributions.
> > But
> > >> I
> > >> > > think
> > >> > > > > > this
> > >> > > > > > >> > ends
> > >> > > > > > >> > > up
> > >> > > > > > >> > > > > >> being really shallow. I guess I feel to really
> > >> get a
> > >> > > good
> > >> > > > > > user
> > >> > > > > > >> > > > > experience
> > >> > > > > > >> > > > > >> the two systems have to kind of feel like part
> > of
> > >> the
> > >> > > > same
> > >> > > > > > thing
> > >> > > > > > >> > and
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> can't really add that in later--you can put
> both
> > >> in
> > >> > the
> > >> > > > > same
> > >> > > > > > >> > > > > downloadable
> > >> > > > > > >> > > > > >> tar file but it doesn't really give a very
> > >> cohesive
> > >> > > > > feeling.
> > >> > > > > > I
> > >> > > > > > >> > agree
> > >> > > > > > >> > > > > that
> > >> > > > > > >> > > > > >> ultimately any of the project stuff is as much
> > >> social
> > >> > > and
> > >> > > > > > naming
> > >> > > > > > >> > as
> > >> > > > > > >> > > > > >> anything else--theoretically two totally
> > >> independent
> > >> > > > > projects
> > >> > > > > > >> > could
> > >> > > > > > >> > > > work
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> tightly align. In practice this seems to be
> > quite
> > >> > > > difficult
> > >> > > > > > >> > though.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> For the frameworks--totally agree it would be
> > >> good to
> > >> > > > > > maintain
> > >> > > > > > >> the
> > >> > > > > > >> > > > > >> framework support with the project. In some
> > cases
> > >> > there
> > >> > > > may
> > >> > > > > > not
> > >> > > > > > >> be
> > >> > > > > > >> > > too
> > >> > > > > > >> > > > > >> much
> > >> > > > > > >> > > > > >> there since the integration gets lighter but I
> > >> think
> > >> > > > > whatever
> > >> > > > > > >> > stubs
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> need should be included. So no I definitely
> > wasn't
> > >> > > trying
> > >> > > > > to
> > >> > > > > > >> imply
> > >> > > > > > >> > > > > >> dropping
> > >> > > > > > >> > > > > >> support for these frameworks, just making the
> > >> > > integration
> > >> > > > > > >> lighter
> > >> > > > > > >> > by
> > >> > > > > > >> > > > > >> separating process management from partition
> > >> > > management.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> You raise two good points we would have to
> > figure
> > >> out
> > >> > > if
> > >> > > > we
> > >> > > > > > went
> > >> > > > > > >> > > down
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> alignment path:
> > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think the
> > >> first
> > >> > > > > question
> > >> > > > > > is
> > >> > > > > > >> > > > whether
> > >> > > > > > >> > > > > >> some "re-branding" would be worth it. If so
> > then I
> > >> > > think
> > >> > > > we
> > >> > > > > > can
> > >> > > > > > >> > > have a
> > >> > > > > > >> > > > > big
> > >> > > > > > >> > > > > >> thread on the name. I'm definitely not set on
> > >> Kafka
> > >> > > > > > Streaming or
> > >> > > > > > >> > > Kafka
> > >> > > > > > >> > > > > >> Streams I was just using them to be kind of
> > >> > > > illustrative. I
> > >> > > > > > >> agree
> > >> > > > > > >> > > with
> > >> > > > > > >> > > > > >> your
> > >> > > > > > >> > > > > >> critique of these names, though I think people
> > >> would
> > >> > > get
> > >> > > > > the
> > >> > > > > > >> idea.
> > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about how
> to
> > >> > > "factor"
> > >> > > > > it.
> > >> > > > > > >> Here
> > >> > > > > > >> > > are
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> options I see (I could get enthusiastic about
> > any
> > >> of
> > >> > > > them):
> > >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> > seperation
> > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api
> and
> > >> > > > samza-core
> > >> > > > > > is
> > >> > > > > > >> > > > absorbed
> > >> > > > > > >> > > > > >> almost like a third client
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> Cheers,
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> -Jay
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> > Kleppmann <
> > >> > > > > > >> > > > martin@kleppmann.com>
> > >> > > > > > >> > > > > >> wrote:
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a
> few
> > >> > > follow-up
> > >> > > > > > >> > comments.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
> > >> > becoming
> > >> > > a
> > >> > > > > > >> > subproject:
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > reasons you mention are good. The risk I see
> > is
> > >> > that
> > >> > > > > > release
> > >> > > > > > >> > > > schedules
> > >> > > > > > >> > > > > >> > become coupled to each other, which can slow
> > >> > everyone
> > >> > > > > down,
> > >> > > > > > >> and
> > >> > > > > > >> > > > large
> > >> > > > > > >> > > > > >> > projects with many contributors are harder
> to
> > >> > manage.
> > >> > > > > > (Jakob,
> > >> > > > > > >> > can
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> speak
> > >> > > > > > >> > > > > >> > from experience, having seen a wider range
> of
> > >> > Hadoop
> > >> > > > > > ecosystem
> > >> > > > > > >> > > > > >> projects?)
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > Some of the goals of a better unified
> > developer
> > >> > > > > experience
> > >> > > > > > >> could
> > >> > > > > > >> > > > also
> > >> > > > > > >> > > > > be
> > >> > > > > > >> > > > > >> > solved by integrating Samza nicely into a
> > Kafka
> > >> > > > > > distribution
> > >> > > > > > >> > (such
> > >> > > > > > >> > > > as
> > >> > > > > > >> > > > > >> > Confluent's). I'm not against merging
> projects
> > >> if
> > >> > we
> > >> > > > > decide
> > >> > > > > > >> > that's
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> way
> > >> > > > > > >> > > > > >> > to go, just pointing out the same goals can
> > >> perhaps
> > >> > > > also
> > >> > > > > be
> > >> > > > > > >> > > achieved
> > >> > > > > > >> > > > > in
> > >> > > > > > >> > > > > >> > other ways.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
> dependency:
> > >> are
> > >> > > you
> > >> > > > > > >> proposing
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > >> > Samza doesn't give any help to people
> wanting
> > to
> > >> > run
> > >> > > on
> > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > >> > > > > > >> > > > > >> > So the docs would basically have a link to
> > >> Slider
> > >> > and
> > >> > > > > > nothing
> > >> > > > > > >> > > else?
> > >> > > > > > >> > > > Or
> > >> > > > > > >> > > > > >> > would we maintain integrations with a bunch
> of
> > >> > > popular
> > >> > > > > > >> > deployment
> > >> > > > > > >> > > > > >> methods
> > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts
> to
> > >> make
> > >> > > > Samza
> > >> > > > > > work
> > >> > > > > > >> > with
> > >> > > > > > >> > > > > >> Slider)?
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > I absolutely think it's a good idea to have
> > the
> > >> > "as a
> > >> > > > > > library"
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > "as a
> > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
> > >> people
> > >> > who
> > >> > > > > want
> > >> > > > > > >> them,
> > >> > > > > > >> > > > but I
> > >> > > > > > >> > > > > >> > think there should also be a low-friction
> path
> > >> for
> > >> > > > common
> > >> > > > > > "as
> > >> > > > > > >> a
> > >> > > > > > >> > > > > service"
> > >> > > > > > >> > > > > >> > deployment methods, for which we probably
> need
> > >> to
> > >> > > > > maintain
> > >> > > > > > >> > > > > integrations.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd
> to
> > >> me,
> > >> > > > > because
> > >> > > > > > >> Kafka
> > >> > > > > > >> > > is
> > >> > > > > > >> > > > > all
> > >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> > >> Transformers"
> > >> > > or
> > >> > > > > > "Kafka
> > >> > > > > > >> > > > Filters"
> > >> > > > > > >> > > > > >> > would be more apt?
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
> > >> (stream
> > >> > > > > > >> transformation
> > >> > > > > > >> > > > with
> > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> > >> library"
> > >> > > bit)
> > >> > > > > > could
> > >> > > > > > >> > > become
> > >> > > > > > >> > > > > >> part of
> > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
> > >> streaming
> > >> > SQL
> > >> > > > and
> > >> > > > > > >> > > > integrations
> > >> > > > > > >> > > > > >> with
> > >> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> > >> project?
> > >> > > In
> > >> > > > > > other
> > >> > > > > > >> > > words,
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > would absorb the proven, stable core of
> Samza,
> > >> > which
> > >> > > > > would
> > >> > > > > > >> > become
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in this
> > >> > thread.
> > >> > > > The
> > >> > > > > > Samza
> > >> > > > > > >> > > > project
> > >> > > > > > >> > > > > >> > would then target that third Kafka client as
> > its
> > >> > base
> > >> > > > > API,
> > >> > > > > > and
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > >> project
> > >> > > > > > >> > > > > >> > would be freed up to explore more
> experimental
> > >> new
> > >> > > > > > horizons.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > Martin
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > >> > > > jay.kreps@gmail.com>
> > >> > > > > > >> wrote:
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > > Hey Martin,
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I
> actually
> > >> > don't
> > >> > > > > think
> > >> > > > > > it
> > >> > > > > > >> > ties
> > >> > > > > > >> > > > our
> > >> > > > > > >> > > > > >> > hands
> > >> > > > > > >> > > > > >> > > at all, all it does is refactor things.
> The
> > >> > > division
> > >> > > > of
> > >> > > > > > >> > > > > >> responsibility is
> > >> > > > > > >> > > > > >> > > that Samza core is responsible for task
> > >> > lifecycle,
> > >> > > > > state,
> > >> > > > > > >> and
> > >> > > > > > >> > > > > >> partition
> > >> > > > > > >> > > > > >> > > management (using the Kafka co-ordinator)
> > but
> > >> it
> > >> > is
> > >> > > > NOT
> > >> > > > > > >> > > > responsible
> > >> > > > > > >> > > > > >> for
> > >> > > > > > >> > > > > >> > > packaging, configuration deployment or
> > >> execution
> > >> > of
> > >> > > > > > >> processes.
> > >> > > > > > >> > > The
> > >> > > > > > >> > > > > >> > problem
> > >> > > > > > >> > > > > >> > > of packaging and starting these processes
> is
> > >> > > > > > >> > > > > >> > > framework/environment-specific. This
> leaves
> > >> > > > individual
> > >> > > > > > >> > > frameworks
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > as
> > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you can
> > get
> > >> > > simple
> > >> > > > > > >> stateless
> > >> > > > > > >> > > > > >> support in
> > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf
> > app
> > >> > > > > framework
> > >> > > > > > >> > > (Slider,
> > >> > > > > > >> > > > > >> > Marathon,
> > >> > > > > > >> > > > > >> > > etc). These are well known by people and
> > have
> > >> > nice
> > >> > > > UIs
> > >> > > > > > and a
> > >> > > > > > >> > lot
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > > flexibility. I don't think they have node
> > >> > affinity
> > >> > > > as a
> > >> > > > > > >> built
> > >> > > > > > >> > in
> > >> > > > > > >> > > > > >> option
> > >> > > > > > >> > > > > >> > > (though I could be wrong). So if we want
> > that
> > >> we
> > >> > > can
> > >> > > > > > either
> > >> > > > > > >> > wait
> > >> > > > > > >> > > > for
> > >> > > > > > >> > > > > >> them
> > >> > > > > > >> > > > > >> > > to add it or do a custom framework to add
> > that
> > >> > > > feature
> > >> > > > > > (as
> > >> > > > > > >> > now).
> > >> > > > > > >> > > > > >> > Obviously
> > >> > > > > > >> > > > > >> > > if you manage things with old-school ops
> > tools
> > >> > > > > > >> > (puppet/chef/etc)
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> get
> > >> > > > > > >> > > > > >> > > locality easily. The nice thing, though,
> is
> > >> that
> > >> > > all
> > >> > > > > the
> > >> > > > > > >> samza
> > >> > > > > > >> > > > > >> "business
> > >> > > > > > >> > > > > >> > > logic" around partition management and
> fault
> > >> > > > tolerance
> > >> > > > > > is in
> > >> > > > > > >> > > Samza
> > >> > > > > > >> > > > > >> core
> > >> > > > > > >> > > > > >> > so
> > >> > > > > > >> > > > > >> > > it is shared across frameworks and the
> > >> framework
> > >> > > > > specific
> > >> > > > > > >> bit
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > just
> > >> > > > > > >> > > > > >> > > whether it is smart enough to try to get
> the
> > >> same
> > >> > > > host
> > >> > > > > > when
> > >> > > > > > >> a
> > >> > > > > > >> > > job
> > >> > > > > > >> > > > is
> > >> > > > > > >> > > > > >> > > restarted.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah
> I
> > >> think
> > >> > > the
> > >> > > > > > goal
> > >> > > > > > >> > would
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > >> (a)
> > >> > > > > > >> > > > > >> > > actually get better alignment in user
> > >> experience,
> > >> > > and
> > >> > > > > (b)
> > >> > > > > > >> > > express
> > >> > > > > > >> > > > > >> this in
> > >> > > > > > >> > > > > >> > > the naming and project branding.
> > Specifically:
> > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> > >> > > > > > "transformation"
> > >> > > > > > >> api
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e.
> be
> > >> able
> > >> > > to
> > >> > > > > > explain
> > >> > > > > > >> > > when
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> use
> > >> > > > > > >> > > > > >> > > the consumer and when to use the stream
> > >> > processing
> > >> > > > > > >> > functionality
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> lead
> > >> > > > > > >> > > > > >> > > people into that experience.
> > >> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza
> 1.4.2
> > >> (or
> > >> > > > > > whatever)
> > >> > > > > > >> > that
> > >> > > > > > >> > > > has
> > >> > > > > > >> > > > > >> both
> > >> > > > > > >> > > > > >> > > Kafka and the stream processing part and
> > they
> > >> > > > actually
> > >> > > > > > work
> > >> > > > > > >> > > > > together.
> > >> > > > > > >> > > > > >> > > 3. Unify the programming experience so the
> > >> client
> > >> > > and
> > >> > > > > > Samza
> > >> > > > > > >> > api
> > >> > > > > > >> > > > > share
> > >> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> > committers
> > >> and
> > >> > > can
> > >> > > > > > have a
> > >> > > > > > >> > > > > separate
> > >> > > > > > >> > > > > >> > repo,
> > >> > > > > > >> > > > > >> > > but I'm actually not really sure (I can't
> > >> find a
> > >> > > > > > definition
> > >> > > > > > >> > of a
> > >> > > > > > >> > > > > >> > subproject
> > >> > > > > > >> > > > > >> > > in Apache).
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > Basically at a high-level you want the
> > >> experience
> > >> > > to
> > >> > > > > > "feel"
> > >> > > > > > >> > > like a
> > >> > > > > > >> > > > > >> single
> > >> > > > > > >> > > > > >> > > system, not to relatively independent
> things
> > >> that
> > >> > > are
> > >> > > > > > kind
> > >> > > > > > >> of
> > >> > > > > > >> > > > > >> awkwardly
> > >> > > > > > >> > > > > >> > > glued together.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > I think if we did that they having naming
> or
> > >> > > branding
> > >> > > > > > like
> > >> > > > > > >> > > "kafka
> > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or something
> > >> like
> > >> > > that
> > >> > > > > > would
> > >> > > > > > >> > > > actually
> > >> > > > > > >> > > > > >> do a
> > >> > > > > > >> > > > > >> > > good job of conveying what it is. I do
> that
> > >> this
> > >> > > > would
> > >> > > > > > help
> > >> > > > > > >> > > > adoption
> > >> > > > > > >> > > > > >> > quite
> > >> > > > > > >> > > > > >> > > a lot as it would correctly convey that
> > using
> > >> > Kafka
> > >> > > > > > >> Streaming
> > >> > > > > > >> > > with
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > is
> > >> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
> > >> pretty
> > >> > > > > heavily
> > >> > > > > > >> > adopted
> > >> > > > > > >> > > > at
> > >> > > > > > >> > > > > >> this
> > >> > > > > > >> > > > > >> > > point.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> > >> originally
> > >> > > > when
> > >> > > > > > open
> > >> > > > > > >> > > > sourcing
> > >> > > > > > >> > > > > >> > Samza,
> > >> > > > > > >> > > > > >> > > however at that time Kafka was relatively
> > >> unknown
> > >> > > and
> > >> > > > > we
> > >> > > > > > >> > decided
> > >> > > > > > >> > > > not
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> > do
> > >> > > > > > >> > > > > >> > > it since we felt it would be limiting.
> From
> > my
> > >> > > point
> > >> > > > of
> > >> > > > > > view
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > three
> > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now
> really
> > >> > heavily
> > >> > > > > used
> > >> > > > > > for
> > >> > > > > > >> > > > stream
> > >> > > > > > >> > > > > >> > > processing, (2) we learned that
> abstracting
> > >> out
> > >> > the
> > >> > > > > > stream
> > >> > > > > > >> > well
> > >> > > > > > >> > > is
> > >> > > > > > >> > > > > >> > > basically impossible, (3) we learned it is
> > >> really
> > >> > > > hard
> > >> > > > > to
> > >> > > > > > >> keep
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > > two
> > >> > > > > > >> > > > > >> > > things feeling like a single product.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > -Jay
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> > >> Kleppmann
> > >> > <
> > >> > > > > > >> > > > > >> martin@kleppmann.com>
> > >> > > > > > >> > > > > >> > > wrote:
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > >> Hi all,
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> I agree with the general philosophy of
> > tying
> > >> > Samza
> > >> > > > > more
> > >> > > > > > >> > firmly
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > > >> Kafka.
> > >> > > > > > >> > > > > >> > >> After I spent a while looking at
> > integrating
> > >> > other
> > >> > > > > > message
> > >> > > > > > >> > > > brokers
> > >> > > > > > >> > > > > >> (e.g.
> > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to
> the
> > >> > > > conclusion
> > >> > > > > > that
> > >> > > > > > >> > > > > >> > SystemConsumer
> > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
> > Kafka's
> > >> > that
> > >> > > > > pretty
> > >> > > > > > >> much
> > >> > > > > > >> > > > > nobody
> > >> > > > > > >> > > > > >> but
> > >> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
> > >> > perhaps
> > >> > > an
> > >> > > > > > >> > exception,
> > >> > > > > > >> > > > but
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.)
> > Thus,
> > >> > > making
> > >> > > > > > Samza
> > >> > > > > > >> > > fully
> > >> > > > > > >> > > > > >> > dependent
> > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > >> > system-independence
> > >> > > > was
> > >> > > > > > >> never
> > >> > > > > > >> > as
> > >> > > > > > >> > > > > real
> > >> > > > > > >> > > > > >> as
> > >> > > > > > >> > > > > >> > we
> > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of
> > code
> > >> > reuse
> > >> > > > are
> > >> > > > > > >> real.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN
> has
> > >> also
> > >> > > > always
> > >> > > > > > been
> > >> > > > > > >> > > > > >> appealing to
> > >> > > > > > >> > > > > >> > >> me, for various reasons already mentioned
> > in
> > >> > this
> > >> > > > > > thread.
> > >> > > > > > >> > > > Although
> > >> > > > > > >> > > > > >> > making
> > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > >> > > > (YARN/Mesos/AWS/etc)
> > >> > > > > > >> seems
> > >> > > > > > >> > > > > >> laudable,
> > >> > > > > > >> > > > > >> > I am
> > >> > > > > > >> > > > > >> > >> a little concerned that it will restrict
> us
> > >> to a
> > >> > > > > lowest
> > >> > > > > > >> > common
> > >> > > > > > >> > > > > >> > denominator.
> > >> > > > > > >> > > > > >> > >> For example, would host affinity
> > (SAMZA-617)
> > >> > still
> > >> > > > be
> > >> > > > > > >> > possible?
> > >> > > > > > >> > > > For
> > >> > > > > > >> > > > > >> jobs
> > >> > > > > > >> > > > > >> > >> with large amounts of state, I think
> > >> SAMZA-617
> > >> > > would
> > >> > > > > be
> > >> > > > > > a
> > >> > > > > > >> big
> > >> > > > > > >> > > > boon,
> > >> > > > > > >> > > > > >> > since
> > >> > > > > > >> > > > > >> > >> restoring state off the changelog on
> every
> > >> > single
> > >> > > > > > restart
> > >> > > > > > >> is
> > >> > > > > > >> > > > > painful,
> > >> > > > > > >> > > > > >> > due
> > >> > > > > > >> > > > > >> > >> to long recovery times. It would be a
> shame
> > >> if
> > >> > the
> > >> > > > > > >> decoupling
> > >> > > > > > >> > > > from
> > >> > > > > > >> > > > > >> YARN
> > >> > > > > > >> > > > > >> > >> made host affinity impossible.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Jay, a question about the proposed API
> for
> > >> > > > > > instantiating a
> > >> > > > > > >> > job
> > >> > > > > > >> > > in
> > >> > > > > > >> > > > > >> code
> > >> > > > > > >> > > > > >> > >> (rather than a properties file): when
> > >> > submitting a
> > >> > > > job
> > >> > > > > > to a
> > >> > > > > > >> > > > > cluster,
> > >> > > > > > >> > > > > >> is
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >> idea that the instantiation code runs on
> a
> > >> > client
> > >> > > > > > >> somewhere,
> > >> > > > > > >> > > > which
> > >> > > > > > >> > > > > >> then
> > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > >> > > YARN/Mesos/AWS/etc?
> > >> > > > > Or
> > >> > > > > > >> does
> > >> > > > > > >> > > that
> > >> > > > > > >> > > > > >> code
> > >> > > > > > >> > > > > >> > run
> > >> > > > > > >> > > > > >> > >> on each container that is part of the job
> > (in
> > >> > > which
> > >> > > > > > case,
> > >> > > > > > >> how
> > >> > > > > > >> > > > does
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > job
> > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
> > >> right to
> > >> > > > make
> > >> > > > > a
> > >> > > > > > 1.0
> > >> > > > > > >> > > > release
> > >> > > > > > >> > > > > >> > with a
> > >> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete.
> So
> > if
> > >> > this
> > >> > > > is
> > >> > > > > > going
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> happen, I
> > >> > > > > > >> > > > > >> > >> think it would be more honest to stick
> with
> > >> 0.*
> > >> > > > > version
> > >> > > > > > >> > numbers
> > >> > > > > > >> > > > > until
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >> library-ified Samza has been implemented,
> > is
> > >> > > stable
> > >> > > > > and
> > >> > > > > > >> > widely
> > >> > > > > > >> > > > > used.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
> > >> Kafka?
> > >> > > There
> > >> > > > > is
> > >> > > > > > >> > > precedent
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > >> tight coupling between different Apache
> > >> projects
> > >> > > > (e.g.
> > >> > > > > > >> > Curator
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I
> think
> > >> > > remaining
> > >> > > > > > >> separate
> > >> > > > > > >> > > > would
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > ok.
> > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on
> Kafka,
> > >> there
> > >> > > is
> > >> > > > > > enough
> > >> > > > > > >> > > > > substance
> > >> > > > > > >> > > > > >> in
> > >> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> > >> project.
> > >> > > An
> > >> > > > > > >> argument
> > >> > > > > > >> > in
> > >> > > > > > >> > > > > >> favour
> > >> > > > > > >> > > > > >> > of
> > >> > > > > > >> > > > > >> > >> merging would be if we think Kafka has a
> > much
> > >> > > > stronger
> > >> > > > > > >> "brand
> > >> > > > > > >> > > > > >> presence"
> > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one.
> If
> > >> the
> > >> > > Kafka
> > >> > > > > > >> project
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > >> willing
> > >> > > > > > >> > > > > >> > to
> > >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of
> > doing
> > >> > > > stateful
> > >> > > > > > >> stream
> > >> > > > > > >> > > > > >> > >> transformations, that would probably have
> > >> much
> > >> > the
> > >> > > > > same
> > >> > > > > > >> > effect
> > >> > > > > > >> > > as
> > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> > >> Processors"
> > >> > or
> > >> > > > > > suchlike.
> > >> > > > > > >> > > Close
> > >> > > > > > >> > > > > >> > >> collaboration between the two projects
> will
> > >> be
> > >> > > > needed
> > >> > > > > in
> > >> > > > > > >> any
> > >> > > > > > >> > > > case.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> From a project management perspective, I
> > >> guess
> > >> > the
> > >> > > > > "new
> > >> > > > > > >> > Samza"
> > >> > > > > > >> > > > > would
> > >> > > > > > >> > > > > >> > have
> > >> > > > > > >> > > > > >> > >> to be developed on a branch alongside
> > ongoing
> > >> > > > > > maintenance
> > >> > > > > > >> of
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > > >> current
> > >> > > > > > >> > > > > >> > >> line of development? I think it would be
> > >> > important
> > >> > > > to
> > >> > > > > > >> > continue
> > >> > > > > > >> > > > > >> > supporting
> > >> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> > >> migration
> > >> > > > path
> > >> > > > > to
> > >> > > > > > >> the
> > >> > > > > > >> > > new
> > >> > > > > > >> > > > > >> > version.
> > >> > > > > > >> > > > > >> > >> Leaving the current versions unsupported
> > and
> > >> > > forcing
> > >> > > > > > people
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> rewrite
> > >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Best,
> > >> > > > > > >> > > > > >> > >> Martin
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > >> > > > jay@confluent.io>
> > >> > > > > > >> wrote:
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >>> Hey Garry,
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be
> > happy
> > >> to
> > >> > > chat
> > >> > > > > > more
> > >> > > > > > >> > about
> > >> > > > > > >> > > > > this
> > >> > > > > > >> > > > > >> if
> > >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
> > >> > started
> > >> > > > with
> > >> > > > > > the
> > >> > > > > > >> > idea
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> "what
> > >> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> > >> > ingestion
> > >> > > > > tool"
> > >> > > > > > but
> > >> > > > > > >> > > > > >> ultimately
> > >> > > > > > >> > > > > >> > we
> > >> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> > >> ingestion
> > >> > > and
> > >> > > > > > >> > > > transformation
> > >> > > > > > >> > > > > >> had
> > >> > > > > > >> > > > > >> > >>> pretty different needs and coupling the
> > two
> > >> > made
> > >> > > > > things
> > >> > > > > > >> > hard.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
> > (KIP-26)
> > >> > > > actually
> > >> > > > > > will
> > >> > > > > > >> > do
> > >> > > > > > >> > > > what
> > >> > > > > > >> > > > > >> you
> > >> > > > > > >> > > > > >> > >> are
> > >> > > > > > >> > > > > >> > >>> looking for.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> With regard to your point about slider,
> I
> > >> don't
> > >> > > > > > >> necessarily
> > >> > > > > > >> > > > > >> disagree.
> > >> > > > > > >> > > > > >> > >> But I
> > >> > > > > > >> > > > > >> > >>> think getting good YARN support is quite
> > >> doable
> > >> > > > and I
> > >> > > > > > >> think
> > >> > > > > > >> > we
> > >> > > > > > >> > > > can
> > >> > > > > > >> > > > > >> make
> > >> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> > >> proposal
> > >> > > > > solves
> > >> > > > > > is
> > >> > > > > > >> > that
> > >> > > > > > >> > > > > >> > >> technically
> > >> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
> > >> cluster
> > >> > > > > > management
> > >> > > > > > >> > > systems
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > way
> > >> > > > > > >> > > > > >> > >>> things are now, you need to write an
> "app
> > >> > master"
> > >> > > > or
> > >> > > > > > >> > > "framework"
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > each
> > >> > > > > > >> > > > > >> > >>> and they are all a little different so
> > >> testing
> > >> > is
> > >> > > > > > really
> > >> > > > > > >> > hard.
> > >> > > > > > >> > > > In
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > >>> absence of this we have been stuck with
> > just
> > >> > YARN
> > >> > > > > which
> > >> > > > > > >> has
> > >> > > > > > >> > > > > >> fantastic
> > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the
> > org,
> > >> but
> > >> > > > zero
> > >> > > > > > >> > > penetration
> > >> > > > > > >> > > > > >> > >> elsewhere.
> > >> > > > > > >> > > > > >> > >>> Given the huge amount of work being put
> in
> > >> to
> > >> > > > slider,
> > >> > > > > > >> > > marathon,
> > >> > > > > > >> > > > > aws
> > >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
> > related
> > >> > > > packaging
> > >> > > > > > >> > > > technologies
> > >> > > > > > >> > > > > >> > people
> > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> > >> > > > > cloud-specific
> > >> > > > > > >> > deploy
> > >> > > > > > >> > > > > >> tools,
> > >> > > > > > >> > > > > >> > >> etc)
> > >> > > > > > >> > > > > >> > >>> I really think it is important to get
> this
> > >> > right.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> -Jay
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> > >> > Turkington
> > >> > > <
> > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>>> Hi all,
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> I think the question below re does
> Samza
> > >> > become
> > >> > > a
> > >> > > > > > >> > sub-project
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>> highlights the broader point around
> > >> migration.
> > >> > > > Chris
> > >> > > > > > >> > mentions
> > >> > > > > > >> > > > > >> Samza's
> > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1
> release
> > >> but
> > >> > I'm
> > >> > > > not
> > >> > > > > > sure
> > >> > > > > > >> > it
> > >> > > > > > >> > > > > feels
> > >> > > > > > >> > > > > >> > >> right to
> > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> > >> deprecate
> > >> > > > most
> > >> > > > > of
> > >> > > > > > >> it.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have some
> > guys
> > >> > who
> > >> > > > have
> > >> > > > > > >> > started
> > >> > > > > > >> > > > > >> working
> > >> > > > > > >> > > > > >> > >> with
> > >> > > > > > >> > > > > >> > >>>> Samza and building some new
> > >> > consumers/producers
> > >> > > > was
> > >> > > > > > next
> > >> > > > > > >> > up.
> > >> > > > > > >> > > > > Sounds
> > >> > > > > > >> > > > > >> > like
> > >> > > > > > >> > > > > >> > >>>> that is absolutely not the direction to
> > >> go. I
> > >> > > need
> > >> > > > > to
> > >> > > > > > >> look
> > >> > > > > > >> > > into
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > KIP
> > >> > > > > > >> > > > > >> > >> in
> > >> > > > > > >> > > > > >> > >>>> more detail but for me the
> attractiveness
> > >> of
> > >> > > > adding
> > >> > > > > > new
> > >> > > > > > >> > Samza
> > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all
> > they
> > >> > were
> > >> > > > > doing
> > >> > > > > > was
> > >> > > > > > >> > > > really
> > >> > > > > > >> > > > > >> > getting
> > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to
> > avoid
> > >> > > > having
> > >> > > > > to
> > >> > > > > > >> > worry
> > >> > > > > > >> > > > > about
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > >>>> lifecycle management of external
> clients.
> > >> If
> > >> > > there
> > >> > > > > is
> > >> > > > > > a
> > >> > > > > > >> > > generic
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a
> > new
> > >> > > > connector
> > >> > > > > > into
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > have
> > >> > > > > > >> > > > > >> a
> > >> > > > > > >> > > > > >> > >> lot of
> > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> > reliability
> > >> > done
> > >> > > > for
> > >> > > > > me
> > >> > > > > > >> then
> > >> > > > > > >> > > it
> > >> > > > > > >> > > > > >> gives
> > >> > > > > > >> > > > > >> > me
> > >> > > > > > >> > > > > >> > >> all
> > >> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers
> > would.
> > >> If
> > >> > > not
> > >> > > > > > then it
> > >> > > > > > >> > > > > >> complicates
> > >> > > > > > >> > > > > >> > my
> > >> > > > > > >> > > > > >> > >>>> operational deployments.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Which is similar to my other question
> > with
> > >> the
> > >> > > > > > proposal
> > >> > > > > > >> --
> > >> > > > > > >> > if
> > >> > > > > > >> > > > we
> > >> > > > > > >> > > > > >> > build a
> > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus
> > the
> > >> > > > requisite
> > >> > > > > > >> shims
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> > integrate
> > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former
> may
> > >> be a
> > >> > > lot
> > >> > > > > more
> > >> > > > > > >> work
> > >> > > > > > >> > > > than
> > >> > > > > > >> > > > > we
> > >> > > > > > >> > > > > >> > >> think.
> > >> > > > > > >> > > > > >> > >>>> We may make it much easier for a
> newcomer
> > >> to
> > >> > get
> > >> > > > > > >> something
> > >> > > > > > >> > > > > running
> > >> > > > > > >> > > > > >> but
> > >> > > > > > >> > > > > >> > >>>> having them step up and get a reliable
> > >> > > production
> > >> > > > > > >> > deployment
> > >> > > > > > >> > > > may
> > >> > > > > > >> > > > > >> still
> > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
> > >> > different
> > >> > > > > > reasons
> > >> > > > > > >> > than
> > >> > > > > > >> > > > > >> today.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable
> > with
> > >> > > making
> > >> > > > > the
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > > >> dependency
> > >> > > > > > >> > > > > >> > >> on
> > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
> absolutely
> > >> see
> > >> > > the
> > >> > > > > > >> benefits
> > >> > > > > > >> > > in
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > >> > > > > > >> > > > terminologies/abstractions
> > >> > > > > > >> > > > > >> that
> > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library
> > >> would
> > >> > > > likely
> > >> > > > > > be a
> > >> > > > > > >> > very
> > >> > > > > > >> > > > > nice
> > >> > > > > > >> > > > > >> > tool
> > >> > > > > > >> > > > > >> > >> to
> > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have
> > the
> > >> > > > concerns
> > >> > > > > > >> above
> > >> > > > > > >> > re
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > >>>> operational side.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Garry
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> > >> [mailto:
> > >> > > > > > >> > gdfm@apache.org
> > >> > > > > > >> > > ]
> > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations
> on
> > >> > Samza
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > >> > > > > > >> > > > > >> > >>>> From outside, I have always perceived
> > Samza
> > >> > as a
> > >> > > > > > >> computing
> > >> > > > > > >> > > > layer
> > >> > > > > > >> > > > > >> over
> > >> > > > > > >> > > > > >> > >>>> Kafka.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit provocative,
> is
> > >> > > "should
> > >> > > > > > Samza
> > >> > > > > > >> be
> > >> > > > > > >> > a
> > >> > > > > > >> > > > > >> > sub-project
> > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> > >> separate
> > >> > > > > project
> > >> > > > > > >> > with a
> > >> > > > > > >> > > > > >> separate
> > >> > > > > > >> > > > > >> > >>>> governance?
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Cheers,
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> --
> > >> > > > > > >> > > > > >> > >>>> Gianmarco
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > >> > > > > > yanfang724@gmail.com>
> > >> > > > > > >> > > > wrote:
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka
> > more
> > >> > > > tightly.
> > >> > > > > > >> > Because
> > >> > > > > > >> > > > > Samza
> > >> > > > > > >> > > > > >> de
> > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
> > >> > leverage
> > >> > > > > what
> > >> > > > > > >> Kafka
> > >> > > > > > >> > > > has.
> > >> > > > > > >> > > > > At
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
> > reinvent
> > >> > what
> > >> > > > > Samza
> > >> > > > > > >> > > already
> > >> > > > > > >> > > > > >> has. I
> > >> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> > >> > ingestion
> > >> > > > and
> > >> > > > > > >> > > > > transformation.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
> > >> image
> > >> > > how
> > >> > > > > the
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > will
> > >> > > > > > >> > > > > >> look
> > >> > > > > > >> > > > > >> > >>>> like.
> > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> > >> > > difference
> > >> > > > > in
> > >> > > > > > >> terms
> > >> > > > > > >> > > of
> > >> > > > > > >> > > > > how
> > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code
> > >> shows
> > >> > (A
> > >> > > > > > client of
> > >> > > > > > >> > > > Kakfa)
> > >> > > > > > >> > > > > ?
> > >> > > > > > >> > > > > >> And
> > >> > > > > > >> > > > > >> > >>>>> user's application code calls this
> > client?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of
> > Kafka
> > >> > (like
> > >> > > > > what
> > >> > > > > > the
> > >> > > > > > >> > > code
> > >> > > > > > >> > > > > >> shows),
> > >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > >> > > > > fault-tolerance?
> > >> > > > > > >> Are
> > >> > > > > > >> > > they
> > >> > > > > > >> > > > > >> taken
> > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> > >> mechanism,
> > >> > > such
> > >> > > > > as
> > >> > > > > > >> > "Samza
> > >> > > > > > >> > > > > >> worker"
> > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 2. What about other features, such as
> > >> > > > auto-scaling,
> > >> > > > > > >> shared
> > >> > > > > > >> > > > > state,
> > >> > > > > > >> > > > > >> > >>>>> monitoring?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is
> > this
> > >> > what
> > >> > > > > Chris
> > >> > > > > > >> > > > suggests?)
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from
> > Kakfa
> > >> > and
> > >> > > > > > produce
> > >> > > > > > >> to
> > >> > > > > > >> > > it.
> > >> > > > > > >> > > > > >> Then it
> > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks
> > like
> > >> > now,
> > >> > > > > > except it
> > >> > > > > > >> > > does
> > >> > > > > > >> > > > > not
> > >> > > > > > >> > > > > >> > rely
> > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> > >> leverage
> > >> > > > Kafka's
> > >> > > > > > >> > metrics,
> > >> > > > > > >> > > > > logs,
> > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> Thanks,
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM,
> Guozhang
> > >> > Wang <
> > >> > > > > > >> > > > > wangguoz@gmail.com
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > >>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Read through the code example and it
> > >> looks
> > >> > > good
> > >> > > > to
> > >> > > > > > me.
> > >> > > > > > >> A
> > >> > > > > > >> > > few
> > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> > >> runnable
> > >> > > like:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > >> > > --config-factory=...
> > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> > deploying
> > >> > Samza
> > >> > > > > more
> > >> > > > > > as
> > >> > > > > > >> > > > embedded
> > >> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> > >> (ignoring
> > >> > > the
> > >> > > > > > >> > > terminology
> > >> > > > > > >> > > > > >> since
> > >> > > > > > >> > > > > >> > >>>>>> it is not the
> > >> > > > > > >> > > > > >> > >>>>> same
> > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> > >> MyStreamTask(configs);
> > >> > > > > Thread
> > >> > > > > > >> > thread
> > >> > > > > > >> > > =
> > >> > > > > > >> > > > > new
> > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment
> modes
> > >> are
> > >> > > > > important
> > >> > > > > > >> for
> > >> > > > > > >> > > > > >> different
> > >> > > > > > >> > > > > >> > >>>>>> types
> > >> > > > > > >> > > > > >> > >>>>> of
> > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making
> Samza
> > >> > purely
> > >> > > > > > >> standalone
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > still
> > >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or
> > library
> > >> > > modes.
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Guozhang
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
> > >> Kreps
> > >> > <
> > >> > > > > > >> > > > jay@confluent.io>
> > >> > > > > > >> > > > > >> > wrote:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> > >> example,
> > >> > it
> > >> > > > was
> > >> > > > > > >> > supposed
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> look
> > >> > > > > > >> > > > > >> > >>>>>>> like
> > >> > > > > > >> > > > > >> > >>>>>>> this:
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> > >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > >> > > > "localhost:4242");
> > >> > > > > > >> > > > > >> StreamingConfig
> > >> > > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > > > "test-topic-2");
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > config.processor(ExampleStreamProcessor.class);
> > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > >> > StringSerializer(),
> > >> > > > new
> > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
> KafkaStreaming
> > >> > > > container =
> > >> > > > > > new
> > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> > container.run();
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> -Jay
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM,
> Jay
> > >> > Kreps <
> > >> > > > > > >> > > > jay@confluent.io
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >> > >>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
> > >> Chris
> > >> > > and
> > >> > > > I
> > >> > > > > > were
> > >> > > > > > >> > > having
> > >> > > > > > >> > > > > >> > >>>>>>>> around
> > >> > > > > > >> > > > > >> > >>>>>>> whether
> > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza
> as a
> > >> kind
> > >> > > of
> > >> > > > > data
> > >> > > > > > >> > > > ingestion
> > >> > > > > > >> > > > > >> > >>>>> framework
> > >> > > > > > >> > > > > >> > >>>>>>> for
> > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to
> > KIP-26
> > >> > > > > "copycat").
> > >> > > > > > >> This
> > >> > > > > > >> > > > kind
> > >> > > > > > >> > > > > of
> > >> > > > > > >> > > > > >> > >>>>>> combined
> > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and
> > YARN
> > >> and
> > >> > > the
> > >> > > > > > >> > discussion
> > >> > > > > > >> > > > > >> around
> > >> > > > > > >> > > > > >> > >>>>>>>> how
> > >> > > > > > >> > > > > >> > >>>>> to
> > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was,
> given
> > >> that
> > >> > > > Samza
> > >> > > > > > was
> > >> > > > > > >> > > > basically
> > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific,
> what
> > if
> > >> > you
> > >> > > > just
> > >> > > > > > >> > embraced
> > >> > > > > > >> > > > > that
> > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > >> > > > > > >> > > > > >> > >>>>>> into
> > >> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> > >> > framework
> > >> > > > and
> > >> > > > > > more
> > >> > > > > > >> > > like a
> > >> > > > > > >> > > > > >> > >>>>>>>> third
> > >> > > > > > >> > > > > >> > >>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> > consumer"
> > >> > with
> > >> > > > > state
> > >> > > > > > >> > > > management
> > >> > > > > > >> > > > > >> > >>>>>> facilities.
> > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> > >> complex
> > >> > > > stream
> > >> > > > > > >> > > processing
> > >> > > > > > >> > > > > >> > >>>>>>>> framework
> > >> > > > > > >> > > > > >> > >>>>>>> this
> > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple
> > thing,
> > >> not
> > >> > > > much
> > >> > > > > > more
> > >> > > > > > >> > > > > >> complicated
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> use
> > >> > > > > > >> > > > > >> > >>>>>>> or
> > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As
> > Chris
> > >> > said
> > >> > > > we
> > >> > > > > > >> thought
> > >> > > > > > >> > > > about
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >>>>>>>> a
> > >> > > > > > >> > > > > >> > >>>>> lot
> > >> > > > > > >> > > > > >> > >>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> > >> > processing
> > >> > > > > > systems
> > >> > > > > > >> > were
> > >> > > > > > >> > > > > doing)
> > >> > > > > > >> > > > > >> > >>>>> seemed
> > >> > > > > > >> > > > > >> > >>>>>>> like
> > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output
> > >> data
> > >> > to
> > >> > > > and
> > >> > > > > > from
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > stream
> > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually
> > looked
> > >> > into
> > >> > > > how
> > >> > > > > > that
> > >> > > > > > >> > > would
> > >> > > > > > >> > > > > >> > >>>>>>>> work,
> > >> > > > > > >> > > > > >> > >>>>> Samza
> > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
> ingestion
> > >> > > framework
> > >> > > > > > for a
> > >> > > > > > >> > > bunch
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>> reasons.
> > >> > > > > > >> > > > > >> > >>>>>> To
> > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a
> > pretty
> > >> > > > different
> > >> > > > > > >> > internal
> > >> > > > > > >> > > > > data
> > >> > > > > > >> > > > > >> > >>>>>>>> model
> > >> > > > > > >> > > > > >> > >>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split
> > them
> > >> and
> > >> > > had
> > >> > > > > an
> > >> > > > > > api
> > >> > > > > > >> > for
> > >> > > > > > >> > > > > Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26)
> > >> and a
> > >> > > > > separate
> > >> > > > > > >> api
> > >> > > > > > >> > > for
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> > embracing
> > >> the
> > >> > > > same
> > >> > > > > > >> > > terminology
> > >> > > > > > >> > > > > and
> > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about
> the
> > >> > current
> > >> > > > > > state is
> > >> > > > > > >> > > that
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>> two
> > >> > > > > > >> > > > > >> > >>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology
> > >> like
> > >> > > > > "stream"
> > >> > > > > > vs
> > >> > > > > > >> > > > "topic"
> > >> > > > > > >> > > > > >> and
> > >> > > > > > >> > > > > >> > >>>>>>> different
> > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means
> > you
> > >> > kind
> > >> > > > of
> > >> > > > > > have
> > >> > > > > > >> to
> > >> > > > > > >> > > > learn
> > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > >> > > > > > >> > > > > >> > >>>>>>> way,
> > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
> different
> > >> way,
> > >> > > > then
> > >> > > > > > kind
> > >> > > > > > >> of
> > >> > > > > > >> > > > > >> > >>>>>>>> understand
> > >> > > > > > >> > > > > >> > >>>>> how
> > >> > > > > > >> > > > > >> > >>>>>>> they
> > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having
> > walked
> > >> a
> > >> > few
> > >> > > > > > people
> > >> > > > > > >> > > through
> > >> > > > > > >> > > > > >> this
> > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to
> > >> get.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of
> > >> time
> > >> > on
> > >> > > > > > >> airplanes I
> > >> > > > > > >> > > > > hacked
> > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> > >> incomplete
> > >> > > > > > prototype
> > >> > > > > > >> of
> > >> > > > > > >> > > > what
> > >> > > > > > >> > > > > >> > >>>>>>>> this would
> > >> > > > > > >> > > > > >> > >>>>> look
> > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
> > >> dumped
> > >> > > into
> > >> > > > > > Kafka
> > >> > > > > > >> as
> > >> > > > > > >> > > it
> > >> > > > > > >> > > > > >> > >>>>>>>> required a
> > >> > > > > > >> > > > > >> > >>>>>> few
> > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here
> is
> > >> the
> > >> > > code:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > >
> > >> > > > > > >> >
> > >> > > > > >
> > >> > >
> > >>
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I
> > just
> > >> > > > > liberally
> > >> > > > > > >> > renamed
> > >> > > > > > >> > > > > >> > >>>>>>>> everything
> > >> > > > > > >> > > > > >> > >>>>> to
> > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
> > >> regard
> > >> > > for
> > >> > > > > > >> > > > compatibility.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something like
> > >> this:
> > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
> Properties();
> > >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > >> > > > > "localhost:4242");
> > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > >> > > > > > >> > > > > >>
> config.processor(ExampleStreamProcessor.class);
> > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > >> > > StringDeserializer());
> > >> > > > > > >> > > > KafkaStreaming
> > >> > > > > > >> > > > > >> > >>>>>> container =
> > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > >> > container.run();
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > >> > > > SamzaContainer;
> > >> > > > > > >> > > > > StreamProcessor
> > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the
> class
> > >> names
> > >> > > in
> > >> > > > a
> > >> > > > > > file
> > >> > > > > > >> > and
> > >> > > > > > >> > > > then
> > >> > > > > > >> > > > > >> > >>>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you
> just
> > >> > > > > instantiate
> > >> > > > > > the
> > >> > > > > > >> > > > > container
> > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced
> > over
> > >> > > > however
> > >> > > > > > many
> > >> > > > > > >> > > > > instances
> > >> > > > > > >> > > > > >> > >>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>> this
> > >> > > > > > >> > > > > >> > >>>>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> > instance
> > >> > dies,
> > >> > > > new
> > >> > > > > > >> tasks
> > >> > > > > > >> > > are
> > >> > > > > > >> > > > > >> added
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
> shutting
> > >> them
> > >> > > > down).
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
> > running
> > >> > this
> > >> > > > > stuff
> > >> > > > > > in
> > >> > > > > > >> > YARN
> > >> > > > > > >> > > > via
> > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
> > >> using
> > >> > > some
> > >> > > > > of
> > >> > > > > > >> their
> > >> > > > > > >> > > > tools
> > >> > > > > > >> > > > > >> > >>>>>>>> but from the
> > >> > > > > > >> > > > > >> > >>>>>> point
> > >> > > > > > >> > > > > >> > >>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these
> stream
> > >> > > > processing
> > >> > > > > > jobs
> > >> > > > > > >> > are
> > >> > > > > > >> > > > > just
> > >> > > > > > >> > > > > >> > >>>>>> stateless
> > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
> > >> expand
> > >> > and
> > >> > > > > > contract
> > >> > > > > > >> > at
> > >> > > > > > >> > > > > will.
> > >> > > > > > >> > > > > >> > >>>>>>>> There
> > >> > > > > > >> > > > > >> > >>>>> is
> > >> > > > > > >> > > > > >> > >>>>>>> no
> > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code,
> > it
> > >> > would
> > >> > > > get
> > >> > > > > > >> larger
> > >> > > > > > >> > > if
> > >> > > > > > >> > > > we
> > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
> > larger.
> > >> We
> > >> > > > really
> > >> > > > > > do
> > >> > > > > > >> > get a
> > >> > > > > > >> > > > ton
> > >> > > > > > >> > > > > >> > >>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>> leverage
> > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> > >> > delegated
> > >> > > to
> > >> > > > > the
> > >> > > > > > >> new
> > >> > > > > > >> > > > > >> consumer.
> > >> > > > > > >> > > > > >> > >>>>> This
> > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> > >> > management
> > >> > > > > > strategy
> > >> > > > > > >> > > > > available
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to
> Samza
> > >> (and
> > >> > > vice
> > >> > > > > > versa)
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > with
> > >> > > > > > >> > > > > >> > >>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>> exact
> > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as
> > state
> > >> > reuse
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
> > >> > thought
> > >> > > > > > >> provoking.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM,
> > Chris
> > >> > > > > Riccomini <
> > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with
> > Samza
> > >> > > > > engineers
> > >> > > > > > at
> > >> > > > > > >> > > > LinkedIn
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>> Confluent
> > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> > observations
> > >> > and
> > >> > > > > would
> > >> > > > > > >> like
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> > >>>>>>>>> propose
> > >> > > > > > >> > > > > >> > >>>>> some
> > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I
> > >> want to
> > >> > > > call
> > >> > > > > > out
> > >> > > > > > >> > about
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > >> > > > > > >> > > > > >> > >>>>>> design,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> > changes.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a
> dynamic
> > >> > > > deployment
> > >> > > > > > >> system.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> > >> SystemConsumer/SystemProducer
> > >> > and
> > >> > > > > > Kafka's
> > >> > > > > > >> > > > consumer
> > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > >> > > > > > >> > > > > >> > >>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
> > >> > problems.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
> > related,
> > >> > but
> > >> > > > I'll
> > >> > > > > > >> > address
> > >> > > > > > >> > > > them
> > >> > > > > > >> > > > > >> in
> > >> > > > > > >> > > > > >> > >>>>> order.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use
> > of a
> > >> > > > dynamic
> > >> > > > > > >> > > deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > >> > > > > > >> > > > > >> > >>>>>> such
> > >> > > > > > >> > > > > >> > >>>>>>>>> as
> > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
> initially
> > >> built
> > >> > > > > Samza,
> > >> > > > > > we
> > >> > > > > > >> > bet
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > >> > >>>>>>>>> there
> > >> > > > > > >> > > > > >> > >>>>>> would
> > >> > > > > > >> > > > > >> > >>>>>>>>> be
> > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area,
> and
> > >> we
> > >> > > could
> > >> > > > > > >> support
> > >> > > > > > >> > > > them,
> > >> > > > > > >> > > > > >> and
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>> rest
> > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there
> are
> > >> many
> > >> > > > > > >> variations.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > >> > > > > > >> > > > > >> > >>>>>> many
> > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start
> > >> their
> > >> > > > > > processors
> > >> > > > > > >> > like
> > >> > > > > > >> > > > > normal
> > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
> traditional
> > >> > > > deployment
> > >> > > > > > >> scripts
> > >> > > > > > >> > > > such
> > >> > > > > > >> > > > > as
> > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > >> > > > > > >> > > > > >> > >>>>>> Chef,
> > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
> > >> system
> > >> > > on
> > >> > > > > > users
> > >> > > > > > >> > makes
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
> > painful
> > >> for
> > >> > > > first
> > >> > > > > > time
> > >> > > > > > >> > > > users.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
> requirement
> > >> was
> > >> > > also
> > >> > > > a
> > >> > > > > > bit
> > >> > > > > > >> of
> > >> > > > > > >> > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > >> > > > > > >> > > > > >> > >>>>>> because
> > >> > > > > > >> > > > > >> > >>>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding
> > between
> > >> > the
> > >> > > > > > nature of
> > >> > > > > > >> > > batch
> > >> > > > > > >> > > > > >> jobs
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>> stream
> > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> > >> > > conscious
> > >> > > > > > effort
> > >> > > > > > >> to
> > >> > > > > > >> > > > favor
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>> Hadoop
> > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things,
> > >> since
> > >> > it
> > >> > > > > worked
> > >> > > > > > >> and
> > >> > > > > > >> > > was
> > >> > > > > > >> > > > > well
> > >> > > > > > >> > > > > >> > >>>>>>> understood.
> > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that
> > >> batch
> > >> > > jobs
> > >> > > > > > have a
> > >> > > > > > >> > > > definite
> > >> > > > > > >> > > > > >> > >>>>>> beginning,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs
> > don't
> > >> > > > > (usually).
> > >> > > > > > >> This
> > >> > > > > > >> > > > leads
> > >> > > > > > >> > > > > to
> > >> > > > > > >> > > > > >> > >>>>>>>>> a
> > >> > > > > > >> > > > > >> > >>>>> much
> > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for
> > stream
> > >> > > > > processors.
> > >> > > > > > >> You
> > >> > > > > > >> > > > > >> basically
> > >> > > > > > >> > > > > >> > >>>>>>>>> just
> > >> > > > > > >> > > > > >> > >>>>>>> need
> > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> > >> processor,
> > >> > and
> > >> > > > > start
> > >> > > > > > >> it.
> > >> > > > > > >> > > The
> > >> > > > > > >> > > > > way
> > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's
> > no
> > >> > > concept
> > >> > > > > of
> > >> > > > > > a
> > >> > > > > > >> > > cluster
> > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > >> > > > > > >> > > > > >> > >>>>>> add
> > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> > >> coupling
> > >> > > > Samza
> > >> > > > > > with
> > >> > > > > > >> a
> > >> > > > > > >> > > > > >> scheduler
> > >> > > > > > >> > > > > >> > >>>>>>>>> is
> > >> > > > > > >> > > > > >> > >>>>>> that
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
> > >> handle
> > >> > > > > > deployment.
> > >> > > > > > >> > > This
> > >> > > > > > >> > > > > >> pulls
> > >> > > > > > >> > > > > >> > >>>>>>>>> in a
> > >> > > > > > >> > > > > >> > >>>>>>> bunch
> > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> > >> > > distribution
> > >> > > > > > (config
> > >> > > > > > >> > > > > stream),
> > >> > > > > > >> > > > > >> > >>>>>>>>> shell
> > >> > > > > > >> > > > > >> > >>>>>>> scrips
> > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> > packaging
> > >> > (all
> > >> > > > the
> > >> > > > > > .tgz
> > >> > > > > > >> > > > stuff),
> > >> > > > > > >> > > > > >> etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring
> dynamic
> > >> > > > deployment
> > >> > > > > > was
> > >> > > > > > >> to
> > >> > > > > > >> > > > > support
> > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> > >> > > locality,
> > >> > > > > you
> > >> > > > > > >> need
> > >> > > > > > >> > to
> > >> > > > > > >> > > > put
> > >> > > > > > >> > > > > >> > >>>>>>>>> your
> > >> > > > > > >> > > > > >> > >>>>>> processors
> > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> > processing.
> > >> > Upon
> > >> > > > > > further
> > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > >> > > > > > >> > > > > >> > >>>>>>> though,
> > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
> beneficial.
> > >> > There
> > >> > > is
> > >> > > > > > some
> > >> > > > > > >> > good
> > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > >> > > > > > >> > > > > >> > >>>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
> SAMZA-335.
> > >> > Again,
> > >> > > we
> > >> > > > > > took
> > >> > > > > > >> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > >> > > > > > >> > > > > >> > >>>>>> path,
> > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> > differences
> > >> > > > between
> > >> > > > > > HDFS
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > >> > > > > > >> > > > > >> > >>>>>> has
> > >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has
> partitions.
> > >> This
> > >> > > > leads
> > >> > > > > to
> > >> > > > > > >> less
> > >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
> > >> > > processors
> > >> > > > > on
> > >> > > > > > top
> > >> > > > > > >> > of
> > >> > > > > > >> > > > > Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a
> > crutch.
> > >> > > Samza
> > >> > > > > > doesn't
> > >> > > > > > >> > > have
> > >> > > > > > >> > > > > any
> > >> > > > > > >> > > > > >> > >>>>>>>>> built
> > >> > > > > > >> > > > > >> > >>>>> in
> > >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
> > >> > depends
> > >> > > on
> > >> > > > > the
> > >> > > > > > >> > > dynamic
> > >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to
> > handle
> > >> > > > restarts
> > >> > > > > > >> when a
> > >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > >> > > > > > >> > > > > >> > >>>>>>> made
> > >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> > >> standalone
> > >> > > Samza
> > >> > > > > > >> > container
> > >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is
> good,
> > >> but I
> > >> > > > think
> > >> > > > > > that
> > >> > > > > > >> > > we've
> > >> > > > > > >> > > > > >> gone
> > >> > > > > > >> > > > > >> > >>>>>>>>> too
> > >> > > > > > >> > > > > >> > >>>>>> far
> > >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > >> > > > (SystemConsumer,
> > >> > > > > > >> > > > > SystemProducer,
> > >> > > > > > >> > > > > >> > >>>> etc).
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just
> > about
> > >> > every
> > >> > > > > > >> component
> > >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > >> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> > >> > > ConfigRewriter,
> > >> > > > > > etc).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> > >> > forgotten,
> > >> > > as
> > >> > > > > > well.
> > >> > > > > > >> > Some
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>>>>>> these
> > >> > > > > > >> > > > > >> > >>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not
> to
> > >> be.
> > >> > > This
> > >> > > > > all
> > >> > > > > > >> comes
> > >> > > > > > >> > > at
> > >> > > > > > >> > > > a
> > >> > > > > > >> > > > > >> cost:
> > >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is
> > making
> > >> it
> > >> > > > harder
> > >> > > > > > for
> > >> > > > > > >> > our
> > >> > > > > > >> > > > > users
> > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> pick
> > >> > > > > > >> > > > > >> > >>>>>> up
> > >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It
> > also
> > >> > makes
> > >> > > > it
> > >> > > > > > >> > difficult
> > >> > > > > > >> > > > for
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about
> > what
> > >> the
> > >> > > > > > >> > > characteristics
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> > >> characteristics
> > >> > > > change
> > >> > > > > > >> > > depending
> > >> > > > > > >> > > > on
> > >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are
> > most
> > >> > > visible
> > >> > > > > in
> > >> > > > > > the
> > >> > > > > > >> > > > System
> > >> > > > > > >> > > > > >> APIs.
> > >> > > > > > >> > > > > >> > >>>>> What
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
> > >> functional is
> > >> > > > Kafka
> > >> > > > > > as
> > >> > > > > > >> its
> > >> > > > > > >> > > > > >> > >>>>>>>>> transport
> > >> > > > > > >> > > > > >> > >>>>>> layer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> But
> > >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use
> > >> cases
> > >> > > into
> > >> > > > > one
> > >> > > > > > >> API:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports
> both
> > >> of
> > >> > > these
> > >> > > > > use
> > >> > > > > > >> > cases.
> > >> > > > > > >> > > > The
> > >> > > > > > >> > > > > >> > >>>>>>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>> is,
> > >> > > > > > >> > > > > >> > >>>>>>>>> we
> > >> > > > > > >> > > > > >> > >>>>>>>>> actually want different features
> for
> > >> each
> > >> > > use
> > >> > > > > > case.
> > >> > > > > > >> By
> > >> > > > > > >> > > > > >> papering
> > >> > > > > > >> > > > > >> > >>>>>>>>> over
> > >> > > > > > >> > > > > >> > >>>>>>> these
> > >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a
> > single
> > >> > API,
> > >> > > > > we've
> > >> > > > > > >> > > > introduced
> > >> > > > > > >> > > > > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > >> > > > > > >> > > > > >> > >>>>>>> leaky
> > >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like
> > in
> > >> (2)
> > >> > > is
> > >> > > > to
> > >> > > > > > have
> > >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
> > >> > offsets
> > >> > > > > (like
> > >> > > > > > >> > Kafka).
> > >> > > > > > >> > > > > This
> > >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > >> > > > > > >> > > > > >> > >>>>> with
> > >> > > > > > >> > > > > >> > >>>>>>> (1),
> > >> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems
> have
> > >> > > > different
> > >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> > >> mailing
> > >> > > list
> > >> > > > > and
> > >> > > > > > >> the
> > >> > > > > > >> > > SQL
> > >> > > > > > >> > > > > >> JIRAs
> > >> > > > > > >> > > > > >> > >>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> > >> > > replayability.
> > >> > > > > > Kafka
> > >> > > > > > >> > > allows
> > >> > > > > > >> > > > us
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> > >>>>> rewind
> > >> > > > > > >> > > > > >> > >>>>>>>>> when
> > >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other
> > systems
> > >> > > don't.
> > >> > > > In
> > >> > > > > > some
> > >> > > > > > >> > > > cases,
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>>> return
> > >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > >> > > > > > >> WikipediaSystemConsumer)
> > >> > > > > > >> > > > > because
> > >> > > > > > >> > > > > >> > >>>>>>>>> they
> > >> > > > > > >> > > > > >> > >>>>>> have
> > >> > > > > > >> > > > > >> > >>>>>>> no
> > >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example.
> > Kafka
> > >> > > > supports
> > >> > > > > > >> > > > > partitioning,
> > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > >> > > > > > >> > > > > >> > >>>>> many
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
> > >> having a
> > >> > > > single
> > >> > > > > > >> > > partition
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other
> systems
> > >> model
> > >> > > > > > >> partitioning
> > >> > > > > > >> > > > > >> > >>>> differently (e.g.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also
> a
> > >> mess.
> > >> > > > > > Creating
> > >> > > > > > >> > > streams
> > >> > > > > > >> > > > > in
> > >> > > > > > >> > > > > >> a
> > >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> > >> impossible.
> > >> > > As
> > >> > > > is
> > >> > > > > > >> > modeling
> > >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > >> > > > > > >> > > > > >> > >>>>> for
> > >> > > > > > >> > > > > >> > >>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> > >> partitions,
> > >> > > > > location,
> > >> > > > > > >> > etc).
> > >> > > > > > >> > > > The
> > >> > > > > > >> > > > > >> > >>>>>>>>> list
> > >> > > > > > >> > > > > >> > >>>>> goes
> > >> > > > > > >> > > > > >> > >>>>>>> on.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing
> > >> Samza,
> > >> > > > > Kafka's
> > >> > > > > > >> > > consumer
> > >> > > > > > >> > > > > and
> > >> > > > > > >> > > > > >> > >>>>> producer
> > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set.
> > On
> > >> the
> > >> > > > > > >> > consumer-side,
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> > >>>>>>>>> had two
> > >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
> > consumer,
> > >> or
> > >> > > the
> > >> > > > > > simple
> > >> > > > > > >> > > > > consumer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> The
> > >> > > > > > >> > > > > >> > >>>>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was
> > that
> > >> it
> > >> > > > > > controlled
> > >> > > > > > >> > your
> > >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments,
> and
> > >> the
> > >> > > order
> > >> > > > > in
> > >> > > > > > >> which
> > >> > > > > > >> > > you
> > >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > >> > > > > > >> > > > > >> > >>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>>>>> with
> > >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's
> not
> > >> > > simple.
> > >> > > > > It's
> > >> > > > > > >> > basic.
> > >> > > > > > >> > > > You
> > >> > > > > > >> > > > > >> > >>>>>>>>> end up
> > >> > > > > > >> > > > > >> > >>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really
> low-level
> > >> stuff
> > >> > > > that
> > >> > > > > > you
> > >> > > > > > >> > > > > shouldn't.
> > >> > > > > > >> > > > > >> > >>>>>>>>> We
> > >> > > > > > >> > > > > >> > >>>>>> spent a
> > >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > >> > > > KafkaSystemConsumer
> > >> > > > > > very
> > >> > > > > > >> > > > robust.
> > >> > > > > > >> > > > > >> It
> > >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some
> cool
> > >> > > features:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering
> and
> > >> > > > > > prioritization.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> > >> assignment
> > >> > > to
> > >> > > > > > support
> > >> > > > > > >> > > > joins,
> > >> > > > > > >> > > > > >> > >>>>>>>>> global
> > >> > > > > > >> > > > > >> > >>>>>> state
> > >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)),
> > etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> > >> > checkpointing.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time
> > is
> > >> > that
> > >> > > > > these
> > >> > > > > > >> > > features
> > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > >> > > > > > >> > > > > >> > >>>>>>> actually
> > >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
> > consumers
> > >> > (not
> > >> > > > just
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > stream
> > >> > > > > > >> > > > > >> > >>>>>> processors)
> > >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like
> > joins
> > >> > and
> > >> > > > > > partition
> > >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> > >> > conclusion.
> > >> > > > > > They're
> > >> > > > > > >> > > adding
> > >> > > > > > >> > > > a
> > >> > > > > > >> > > > > >> ton
> > >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> > >> consumer
> > >> > > > > > >> > > implementation.
> > >> > > > > > >> > > > > To a
> > >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > >> > > > > > >> > > > > >> > >>>>> it's
> > >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've
> already
> > >> done
> > >> > > in
> > >> > > > > > Samza.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up
> > taking
> > >> a
> > >> > > very
> > >> > > > > > similar
> > >> > > > > > >> > > > > approach
> > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>>> Samza's
> > >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> > implementation
> > >> for
> > >> > > > > > handling
> > >> > > > > > >> > > offset
> > >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> > >> management
> > >> > > > > feature
> > >> > > > > > >> > stores
> > >> > > > > > >> > > > > >> offset
> > >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows
> > >> you to
> > >> > > > fetch
> > >> > > > > > them
> > >> > > > > > >> > > from
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste,
> > >> since
> > >> > we
> > >> > > > > could
> > >> > > > > > >> have
> > >> > > > > > >> > > > shared
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>> work
> > >> > > > > > >> > > > > >> > >>>>>> if
> > >> > > > > > >> > > > > >> > >>>>>>>>> it
> > >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
> > >> get-go.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> > >> radical
> > >> > > > > > proposal.
> > >> > > > > > >> > Samza
> > >> > > > > > >> > > > is
> > >> > > > > > >> > > > > >> > >>>>> relatively
> > >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture
> to
> > >> say
> > >> > > that
> > >> > > > > > we're
> > >> > > > > > >> > > near a
> > >> > > > > > >> > > > > 1.0
> > >> > > > > > >> > > > > >> > >>>>>> release.
> > >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what
> > >> we've
> > >> > > > > learned,
> > >> > > > > > and
> > >> > > > > > >> > > begin
> > >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > >> > > > > > >> > > > > >> > >>>>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we
> > >> change if
> > >> > > we
> > >> > > > > were
> > >> > > > > > >> > > starting
> > >> > > > > > >> > > > > >> from
> > >> > > > > > >> > > > > >> > >>>>>> scratch?
> > >> > > > > > >> > > > > >> > >>>>>>>>> My
> > >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the
> *only*
> > >> way
> > >> > to
> > >> > > > run
> > >> > > > > > Samza
> > >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all
> direct
> > >> > > > > dependences
> > >> > > > > > on
> > >> > > > > > >> > > YARN,
> > >> > > > > > >> > > > > >> Mesos,
> > >> > > > > > >> > > > > >> > >>>> etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to
> support
> > >> only
> > >> > > > Kafka
> > >> > > > > > as
> > >> > > > > > >> the
> > >> > > > > > >> > > > > stream
> > >> > > > > > >> > > > > >> > >>>>>> processing
> > >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics,
> > logging,
> > >> > > > > > >> serialization,
> > >> > > > > > >> > > and
> > >> > > > > > >> > > > > >> > >>>>>>>>> config
> > >> > > > > > >> > > > > >> > >>>>>>> systems,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues
> > that
> > >> I
> > >> > > > > outlined
> > >> > > > > > >> > above.
> > >> > > > > > >> > > It
> > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > >> > > > > > >> > > > > >> > >>>>> also
> > >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> > >> > > > dramatically.
> > >> > > > > > >> > > Supporting
> > >> > > > > > >> > > > > >> only
> > >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow
> > >> Samza
> > >> > to
> > >> > > be
> > >> > > > > > >> executed
> > >> > > > > > >> > > on
> > >> > > > > > >> > > > > YARN
> > >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > >> > > > Marathon/Aurora),
> > >> > > > > or
> > >> > > > > > >> most
> > >> > > > > > >> > > > other
> > >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > >> > > > > > >> > > > > >> > >>>>>>> deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a
> lot
> > >> > easier
> > >> > > > for
> > >> > > > > > new
> > >> > > > > > >> > > users.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > >> > > > > > >> > > > > >> > >>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without
> > YARN.
> > >> > The
> > >> > > > drop
> > >> > > > > > in
> > >> > > > > > >> > > mailing
> > >> > > > > > >> > > > > >> list
> > >> > > > > > >> > > > > >> > >>>>>> traffic
> > >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
> > >> overdue to
> > >> > > me.
> > >> > > > > The
> > >> > > > > > >> > > reality
> > >> > > > > > >> > > > > is,
> > >> > > > > > >> > > > > >> > >>>>> everyone
> > >> > > > > > >> > > > > >> > >>>>>>>>> that
> > >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
> > >> Kafka.
> > >> > We
> > >> > > > > > basically
> > >> > > > > > >> > > > require
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >>>>>> already
> > >> > > > > > >> > > > > >> > >>>>>>> in
> > >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work.
> > Those
> > >> > that
> > >> > > > are
> > >> > > > > > >> using
> > >> > > > > > >> > > > other
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into
> > >> Kafka
> > >> > > (1),
> > >> > > > > and
> > >> > > > > > >> then
> > >> > > > > > >> > > > they
> > >> > > > > > >> > > > > do
> > >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
> > >> already
> > >> > > > > > discussion (
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > >
> > >> > > > > > >> >
> > >> > > > > >
> > >> > >
> > >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >> > > > > > >> > > > > >> > >>>>> 767
> > >> > > > > > >> > > > > >> > >>>>>>>>> )
> > >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into
> > Kafka
> > >> > > > extremely
> > >> > > > > > >> easy.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple
> with
> > >> > Kafka,
> > >> > > > we
> > >> > > > > > can
> > >> > > > > > >> > > > leverage
> > >> > > > > > >> > > > > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > >> > > > > > >> > > > > >> > >>>>>>> their
> > >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> > >> maintain
> > >> > > our
> > >> > > > > own
> > >> > > > > > >> > config,
> > >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > >> > > > > > >> > > > > >> > >>>>> etc.
> > >> > > > > > >> > > > > >> > >>>>>>> We
> > >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries,
> > and
> > >> > make
> > >> > > > them
> > >> > > > > > >> > better.
> > >> > > > > > >> > > > This
> > >> > > > > > >> > > > > >> > >>>>>>>>> will
> > >> > > > > > >> > > > > >> >
> > >> ...
> > >>
> > >> [Message clipped]
> > >
> > >
> > >
> >
>
>
>
> --
> Jordan Shaw
> Full Stack Software Engineer
> PubNub Inc
> 1045 17th St
> San Francisco, CA 94107
>

Re: Thoughts and obesrvations on Samza

Posted by Yan Fang <ya...@gmail.com>.
I am leaning to Jay's fifth approach. It is not radical and gives us some
time to see the outcome.

In addition, I would suggest:

1) Keep the SystemConsumer/SystemProducer API. Because current
SystemConsumer/SystemProducer API satisfies the usage (From Joardan, and
even Garry's feedback) and is not so broken that we want to deprecate it.
Though there are some issues in implemnting the Kinesis, they are not
unfixable. Nothing should prevent Samza, as a stream processing system, to
support other systems. In addition, there already are some systems
exiting besides Kafka: ElasticSearch (committed to the master), HDFS
(patch-available), S3( from the mailing list), Kinesis (developing in
another repository), ActiveMQ (in two months). We may want to see how those
go before we "kill" them.

2) Can have some Samza devs involved in Kafka's "transformer" client API.
This can not only help the future integration (if any) much easier, because
they have knowledge about both systems, but also good for Kafka's
community, because Samza devs have the streaming process experience that
Kafka devs may miss.

3) Samza's partition management system may still support other systems.
Though the partition management logic in samza-kafka will be moved to
Kafka, its still useful for other systems that do not have the partition
management layer.

4) Start sharing the docs/websites and using the same terminology (though
do not know how to do this exactly. :). This will reduce the future
confusion and does not hurt Samza's independency.

In my opinion, Samza, as a standalone project, still can (and already)
heavily replying on Kafka, and even more tuned for Kafka-specific usage.
Kafka, also can embed Samza in the document, I do not see anything prevent
doing this.

Thanks,

Fang, Yan
yanfang724@gmail.com

On Mon, Jul 13, 2015 at 11:25 AM, Jordan Shaw <jo...@pubnub.com> wrote:

> Jay,
> I think doing this iteratively in smaller chunks is a better way to go as
> new issues arise. As Navina said Kafka is a "stream system" and Samza is a
> "stream processor" and those two ideas should be mutually exclusive.
>
> -Jordan
>
> On Mon, Jul 13, 2015 at 10:06 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hmm, thought about this more. Maybe this is just too much too quick.
> > Overall I think there is some enthusiasm for the proposal but it's not
> > really unanimous enough to make any kind of change this big cleanly. The
> > board doesn't really like the merging stuff, user's are concerned about
> > compatibility, I didn't feel there was unanimous agreement on dropping
> > SystemConsumer, etc. Even if this is the right end state to get to,
> > probably trying to push all this through at once isn't the right way to
> do
> > it.
> >
> > So let me propose a kind of fifth (?) option which I think is less
> dramatic
> > and let's things happen gradually. I think this is kind of like combining
> > the first part of Yi's proposal and Jakob's third option, leaving the
> rest
> > to be figured out incrementally:
> >
> > Option 5: We continue the prototype I shared and propose that as a kind
> of
> > "transformer" client API in Kafka. This isn't really a full-fledged
> stream
> > processing layer, more like a supped up consumer api for munging topics.
> > This would let us figure out some of the technical bits, how to do this
> on
> > Kafka's group management features, how to integrate the txn feature to do
> > the exactly-once stuff in these transformations, and get all this stuff
> > solid. This api would have valid uses in it's own right, especially when
> > your transformation will be embedded inside an existing service or
> > application which isn't possible with Samza (or other existing systems
> that
> > I know of).
> >
> > Independently we can iterate on some of the ideas of the original
> proposal
> > individually and figure out how (if at all) to make use of this
> > functionality. This can be done bit-by-bit:
> > - Could be that the existing StreamTask API ends up wrapping this
> > - Could end up exposed directly in Samza as Yi proposed
> > - Could be that just the lower-level group-management stuff get's used,
> and
> > in this case it could be either just for standalone mode, or always
> > - Could be that it stays as-is
> >
> > The advantage of this is it is lower risk...we basically don't have to
> make
> > 12 major decisions all at once that kind of hinge on what amounts to a
> > pretty aggressive rewrite. The disadvantage of this is it is a bit more
> > confusing as all this is getting figured out.
> >
> > As with some of the other stuff, this would require a further discussion
> in
> > the Kafka community if people do like this approach.
> >
> > Thoughts?
> >
> > -Jay
> >
> >
> >
> >
> > On Sun, Jul 12, 2015 at 10:52 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Chris,
> > >
> > > Yeah, I'm obviously in favor of this.
> > >
> > > The sub-project approach seems the ideal way to take a graceful step in
> > > this direction, so I will ping the board folks and see why they are
> > > discouraged, it would be good to understand that. If we go that route
> we
> > > would need to do a similar discussion in the Kafka list (but makes
> sense
> > to
> > > figure out first if it is what Samza wants).
> > >
> > > Irrespective of how it's implemented, though, to me the important
> things
> > > are the following:
> > > 1. Unify the website, config, naming, docs, metrics, etc--basically fix
> > > the product experience so the "stream" and the "processing" feel like a
> > > single user experience and brand. This seems minor but I think is a
> > really
> > > big deal.
> > > 2. Make "standalone" mode a first class citizen and have a real
> technical
> > > plan to be able to support cluster managers other than YARN.
> > > 3. Make the config and out-of-the-box experience more usable
> > >
> > > I think that prototype gives a practical example of how 1-3 could be
> done
> > > and we should pursue it. This is a pretty radical change, so I wouldn't
> > be
> > > shocked if people didn't want to take a step like that.
> > >
> > > Maybe it would make sense to see if people are on board with that
> general
> > > idea, and then try to get some advice on sub-projects in parallel and
> > nail
> > > down those details?
> > >
> > > -Jay
> > >
> > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
> criccomini@apache.org>
> > > wrote:
> > >
> > >> Hey all,
> > >>
> > >> I want to start by saying that I'm absolutely thrilled to be a part of
> > >> this
> > >> community. The amount of level-headed, thoughtful, educated discussion
> > >> that's gone on over the past ~10 days is overwhelming. Wonderful.
> > >>
> > >> It seems like discussion is waning a bit, and we've reached some
> > >> conclusions. There are several key emails in this threat, which I want
> > to
> > >> call out:
> > >>
> > >> 1. Jakob's summary of the three potential ways forward.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > >> 2. Julian's call out that we should be focusing on community over
> code.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > >> 3. Martin's summary about the benefits of merging communities.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > >> 4. Jakob's comments about the distinction between community and code
> > >> paths.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> > >>
> > >> I agree with the comments on all of these emails. I think Martin's
> > summary
> > >> of his position aligns very closely with my own. To that end, I think
> we
> > >> should get concrete about what the proposal is, and call a vote on it.
> > >> Given that Jay, Martin, and I seem to be aligning fairly closely, I
> > think
> > >> we should start with:
> > >>
> > >> 1. [community] Make Samza a subproject of Kafka.
> > >> 2. [community] Make all Samza PMC/committers committers of the
> > subproject.
> > >> 3. [community] Migrate Samza's website/documentation into Kafka's.
> > >> 4. [code] Have the Samza community and the Kafka community start a
> > >> from-scratch reboot together in the new Kafka subproject. We can
> > >> borrow/copy &  paste significant chunks of code from Samza's code
> base.
> > >> 5. [code] The subproject would intentionally eliminate support for
> both
> > >> other streaming systems and all deployment systems.
> > >> 6. [code] Attempt to provide a bridge from our SystemConsumer to
> KIP-26
> > >> (copy cat)
> > >> 7. [code] Attempt to provide a bridge from the new subproject's
> > processor
> > >> interface to our legacy StreamTask interface.
> > >> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> > >> subproject that has a fault-tolerant container with state management.
> > >>
> > >> It's likely that (6) and (7) won't be fully drop-in. Still, the closer
> > we
> > >> can get, the better it's going to be for our existing community.
> > >>
> > >> One thing that I didn't touch on with (2) is whether any Samza PMC
> > members
> > >> should be rolled into Kafka PMC membership as well (though, Jay and
> > Jakob
> > >> are already PMC members on both). I think that Samza's community
> > deserves
> > >> a
> > >> voice on the PMC, so I'd propose that we roll at least a few PMC
> members
> > >> into the Kafka PMC, but I don't have a strong framework for which
> people
> > >> to
> > >> pick.
> > >>
> > >> Before (8), I think that Samza's TLP can continue to commit bug fixes
> > and
> > >> patches as it sees fit, provided that we openly communicate that we
> > won't
> > >> necessarily migrate new features to the new subproject, and that the
> TLP
> > >> will be shut down after the migration to the Kafka subproject occurs.
> > >>
> > >> Jakob, I could use your guidance here about about how to achieve this
> > from
> > >> an Apache process perspective (sorry).
> > >>
> > >> * Should I just call a vote on this proposal?
> > >> * Should it happen on dev or private?
> > >> * Do committers have binding votes, or just PMC?
> > >>
> > >> Having trouble finding much detail on the Apache wikis. :(
> > >>
> > >> Cheers,
> > >> Chris
> > >>
> > >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com>
> wrote:
> > >>
> > >> > Thanks, Jay. This argument persuaded me actually. :)
> > >> >
> > >> > Fang, Yan
> > >> > yanfang724@gmail.com
> > >> >
> > >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > >> >
> > >> > > Hey Yan,
> > >> > >
> > >> > > Yeah philosophically I think the argument is that you should
> capture
> > >> the
> > >> > > stream in Kafka independent of the transformation. This is
> > obviously a
> > >> > > Kafka-centric view point.
> > >> > >
> > >> > > Advantages of this:
> > >> > > - In practice I think this is what e.g. Storm people often end up
> > >> doing
> > >> > > anyway. You usually need to throttle any access to a live serving
> > >> > database.
> > >> > > - Can have multiple subscribers and they get the same thing
> without
> > >> > > additional load on the source system.
> > >> > > - Applications can tap into the stream if need be by subscribing.
> > >> > > - You can debug your transformation by tailing the Kafka topic
> with
> > >> the
> > >> > > console consumer
> > >> > > - Can tee off the same data stream for batch analysis or Lambda
> arch
> > >> > style
> > >> > > re-processing
> > >> > >
> > >> > > The disadvantage is that it will use Kafka resources. But the idea
> > is
> > >> > > eventually you will have multiple subscribers to any data source
> (at
> > >> > least
> > >> > > for monitoring) so you will end up there soon enough anyway.
> > >> > >
> > >> > > Down the road the technical benefit is that I think it gives us a
> > good
> > >> > path
> > >> > > towards end-to-end exactly once semantics from source to
> > destination.
> > >> > > Basically the connectors need to support idempotence when talking
> to
> > >> > Kafka
> > >> > > and we need the transactional write feature in Kafka to make the
> > >> > > transformation atomic. This is actually pretty doable if you
> > separate
> > >> > > connector=>kafka problem from the generic transformations which
> are
> > >> > always
> > >> > > kafka=>kafka. However I think it is quite impossible to do in a
> > >> > all_things
> > >> > > => all_things environment. Today you can say "well the semantics
> of
> > >> the
> > >> > > Samza APIs depend on the connectors you use" but it is actually
> > worse
> > >> > then
> > >> > > that because the semantics actually depend on the pairing of
> > >> > connectors--so
> > >> > > not only can you probably not get a usable "exactly once"
> guarantee
> > >> > > end-to-end it can actually be quite hard to reverse engineer what
> > >> > property
> > >> > > (if any) your end-to-end flow has if you have heterogenous
> systems.
> > >> > >
> > >> > > -Jay
> > >> > >
> > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > > > {quote}
> > >> > > > maintained in a separate repository and retaining the existing
> > >> > > > committership but sharing as much else as possible (website,
> etc)
> > >> > > > {quote}
> > >> > > >
> > >> > > > Overall, I agree on this idea. Now the question is more about
> "how
> > >> to
> > >> > do
> > >> > > > it".
> > >> > > >
> > >> > > > On the other hand, one thing I want to point out is that, if we
> > >> decide
> > >> > to
> > >> > > > go this way, how do we want to support
> > >> > > > otherSystem-transformation-otherSystem use case?
> > >> > > >
> > >> > > > Basically, there are four user groups here:
> > >> > > >
> > >> > > > 1. Kafka-transformation-Kafka
> > >> > > > 2. Kafka-transformation-otherSystem
> > >> > > > 3. otherSystem-transformation-Kafka
> > >> > > > 4. otherSystem-transformation-otherSystem
> > >> > > >
> > >> > > > For group 1, they can easily use the new Samza library to
> achieve.
> > >> For
> > >> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka
> or
> > >> > Kafka->
> > >> > > > transformation -> copyCat.
> > >> > > >
> > >> > > > The problem is for group 4. Do we want to abandon this or still
> > >> support
> > >> > > it?
> > >> > > > Of course, this use case can be achieved by using copyCat ->
> > >> > > transformation
> > >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> > persuade
> > >> > them
> > >> > > to
> > >> > > > do this long chain. If yes, it will also be a win for Kafka too.
> > Or
> > >> if
> > >> > > > there is no one in this community actually doing this so far,
> > maybe
> > >> ok
> > >> > to
> > >> > > > not support the group 4 directly.
> > >> > > >
> > >> > > > Thanks,
> > >> > > >
> > >> > > > Fang, Yan
> > >> > > > yanfang724@gmail.com
> > >> > > >
> > >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
> > >> wrote:
> > >> > > >
> > >> > > > > Yeah I agree with this summary. I think there are kind of two
> > >> > questions
> > >> > > > > here:
> > >> > > > > 1. Technically does alignment/reliance on Kafka make sense
> > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
> alignment
> > >> with
> > >> > > > Kafka
> > >> > > > > make sense
> > >> > > > >
> > >> > > > > Personally I do think both of these things would be really
> > >> valuable,
> > >> > > and
> > >> > > > > would dramatically alter the trajectory of the project.
> > >> > > > >
> > >> > > > > My preference would be to see if people can mostly agree on a
> > >> > direction
> > >> > > > > rather than splintering things off. From my point of view the
> > >> ideal
> > >> > > > outcome
> > >> > > > > of all the options discussed would be to make Samza a closely
> > >> aligned
> > >> > > > > subproject, maintained in a separate repository and retaining
> > the
> > >> > > > existing
> > >> > > > > committership but sharing as much else as possible (website,
> > >> etc). No
> > >> > > > idea
> > >> > > > > about how these things work, Jacob, you probably know more.
> > >> > > > >
> > >> > > > > No discussion amongst the Kafka folks has happened on this,
> but
> > >> > likely
> > >> > > we
> > >> > > > > should figure out what the Samza community actually wants
> first.
> > >> > > > >
> > >> > > > > I admit that this is a fairly radical departure from how
> things
> > >> are.
> > >> > > > >
> > >> > > > > If that doesn't fly, I think, yeah we could leave Samza as it
> is
> > >> and
> > >> > do
> > >> > > > the
> > >> > > > > more radical reboot inside Kafka. From my point of view that
> > does
> > >> > leave
> > >> > > > > things in a somewhat confusing state since now there are two
> > >> stream
> > >> > > > > processing systems more or less coupled to Kafka in large part
> > >> made
> > >> > by
> > >> > > > the
> > >> > > > > same people. But, arguably that might be a cleaner way to make
> > the
> > >> > > > cut-over
> > >> > > > > and perhaps less risky for Samza community since if it works
> > >> people
> > >> > can
> > >> > > > > switch and if it doesn't nothing will have changed. Dunno, how
> > do
> > >> > > people
> > >> > > > > feel about this?
> > >> > > > >
> > >> > > > > -Jay
> > >> > > > >
> > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> > jghoman@gmail.com>
> > >> > > wrote:
> > >> > > > >
> > >> > > > > > >  This leads me to thinking that merging projects and
> > >> communities
> > >> > > > might
> > >> > > > > > be a good idea: with the union of experience from both
> > >> communities,
> > >> > > we
> > >> > > > > will
> > >> > > > > > probably build a better system that is better for users.
> > >> > > > > > Is this what's being proposed though? Merging the projects
> > seems
> > >> > like
> > >> > > > > > a consequence of at most one of the three directions under
> > >> > > discussion:
> > >> > > > > > 1) Samza 2.0: The Samza community relies more heavily on
> Kafka
> > >> for
> > >> > > > > > configuration, etc. (to a greater or lesser extent to be
> > >> > determined)
> > >> > > > > > but the Samza community would not automatically merge withe
> > >> Kafka
> > >> > > > > > community (the Phoenix/HBase example is a good one here).
> > >> > > > > > 2) Samza Reboot: The Samza community continues to exist
> with a
> > >> > > limited
> > >> > > > > > project scope, but similarly would not need to be part of
> the
> > >> Kafka
> > >> > > > > > community (ie given committership) to progress.  Here, maybe
> > the
> > >> > > Samza
> > >> > > > > > team would become a subproject of Kafka (the Board frowns on
> > >> > > > > > subprojects at the moment, so I'm not sure if that's even
> > >> > feasible),
> > >> > > > > > but that would not be required.
> > >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option
> > the
> > >> > Kafka
> > >> > > > > > team builds its own streaming library, possibly off of Jay's
> > >> > > > > > prototype, which has not direct lineage to the Samza team.
> > >> There's
> > >> > > no
> > >> > > > > > reason for the Kafka team to bring in the Samza team.
> > >> > > > > >
> > >> > > > > > Is the Kafka community on board with this?
> > >> > > > > >
> > >> > > > > > To be clear, all three options under discussion are
> > interesting,
> > >> > > > > > technically valid and likely healthy directions for the
> > project.
> > >> > > > > > Also, they are not mutually exclusive.  The Samza community
> > >> could
> > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
> community
> > >> went
> > >> > > > > > forward with 'Hey Samza!'  My points above are directed
> > >> entirely at
> > >> > > > > > the community aspect of these choices.
> > >> > > > > > -Jakob
> > >> > > > > >
> > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> > roger.hoover@gmail.com>
> > >> > > wrote:
> > >> > > > > > > That's great.  Thanks, Jay.
> > >> > > > > > >
> > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> > jay@confluent.io>
> > >> > > wrote:
> > >> > > > > > >
> > >> > > > > > >> Yeah totally agree. I think you have this issue even
> today,
> > >> > right?
> > >> > > > > I.e.
> > >> > > > > > if
> > >> > > > > > >> you need to make a simple config change and you're
> running
> > in
> > >> > YARN
> > >> > > > > today
> > >> > > > > > >> you end up bouncing the job which then rebuilds state. I
> > >> think
> > >> > the
> > >> > > > fix
> > >> > > > > > is
> > >> > > > > > >> exactly what you described which is to have a long
> timeout
> > on
> > >> > > > > partition
> > >> > > > > > >> movement for stateful jobs so that if a job is just
> getting
> > >> > > bounced,
> > >> > > > > and
> > >> > > > > > >> the cluster manager (or admin) is smart enough to restart
> > it
> > >> on
> > >> > > the
> > >> > > > > same
> > >> > > > > > >> host when possible, it can optimistically reuse any
> > existing
> > >> > state
> > >> > > > it
> > >> > > > > > finds
> > >> > > > > > >> on disk (if it is valid).
> > >> > > > > > >>
> > >> > > > > > >> So in this model the charter of the CM is to place
> > processes
> > >> as
> > >> > > > > > stickily as
> > >> > > > > > >> possible and to restart or re-place failed processes. The
> > >> > charter
> > >> > > of
> > >> > > > > the
> > >> > > > > > >> partition management system is to control the assignment
> of
> > >> work
> > >> > > to
> > >> > > > > > these
> > >> > > > > > >> processes. The nice thing about this is that the work
> > >> > assignment,
> > >> > > > > > timeouts,
> > >> > > > > > >> behavior, configs, and code will all be the same across
> all
> > >> > > cluster
> > >> > > > > > >> managers.
> > >> > > > > > >>
> > >> > > > > > >> So I think that prototype would actually give you exactly
> > >> what
> > >> > you
> > >> > > > > want
> > >> > > > > > >> today for any cluster manager (or manual placement +
> > restart
> > >> > > script)
> > >> > > > > > that
> > >> > > > > > >> was sticky in terms of host placement since there is
> > already
> > >> a
> > >> > > > > > configurable
> > >> > > > > > >> partition movement timeout and task-by-task state reuse
> > with
> > >> a
> > >> > > check
> > >> > > > > on
> > >> > > > > > >> state validity.
> > >> > > > > > >>
> > >> > > > > > >> -Jay
> > >> > > > > > >>
> > >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > >> > > > roger.hoover@gmail.com
> > >> > > > > >
> > >> > > > > > >> wrote:
> > >> > > > > > >>
> > >> > > > > > >> > That would be great to let Kafka do as much heavy
> lifting
> > >> as
> > >> > > > > possible
> > >> > > > > > and
> > >> > > > > > >> > make it easier for other languages to implement Samza
> > apis.
> > >> > > > > > >> >
> > >> > > > > > >> > One thing to watch out for is the interplay between
> > Kafka's
> > >> > > group
> > >> > > > > > >> > management and the external scheduler/process manager's
> > >> fault
> > >> > > > > > tolerance.
> > >> > > > > > >> > If a container dies, the Kafka group membership
> protocol
> > >> will
> > >> > > try
> > >> > > > to
> > >> > > > > > >> assign
> > >> > > > > > >> > it's tasks to other containers while at the same time
> the
> > >> > > process
> > >> > > > > > manager
> > >> > > > > > >> > is trying to relaunch the container.  Without some
> > >> > consideration
> > >> > > > for
> > >> > > > > > this
> > >> > > > > > >> > (like a configurable amount of time to wait before
> Kafka
> > >> > alters
> > >> > > > the
> > >> > > > > > group
> > >> > > > > > >> > membership), there may be thrashing going on which is
> > >> > especially
> > >> > > > bad
> > >> > > > > > for
> > >> > > > > > >> > containers with large amounts of local state.
> > >> > > > > > >> >
> > >> > > > > > >> > Someone else pointed this out already but I thought it
> > >> might
> > >> > be
> > >> > > > > worth
> > >> > > > > > >> > calling out again.
> > >> > > > > > >> >
> > >> > > > > > >> > Cheers,
> > >> > > > > > >> >
> > >> > > > > > >> > Roger
> > >> > > > > > >> >
> > >> > > > > > >> >
> > >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> > >> jay@confluent.io>
> > >> > > > > wrote:
> > >> > > > > > >> >
> > >> > > > > > >> > > Hey Roger,
> > >> > > > > > >> > >
> > >> > > > > > >> > > I couldn't agree more. We spent a bunch of time
> talking
> > >> to
> > >> > > > people
> > >> > > > > > and
> > >> > > > > > >> > that
> > >> > > > > > >> > > is exactly the stuff we heard time and again. What
> > makes
> > >> it
> > >> > > > hard,
> > >> > > > > of
> > >> > > > > > >> > > course, is that there is some tension between
> > >> compatibility
> > >> > > with
> > >> > > > > > what's
> > >> > > > > > >> > > there now and making things better for new users.
> > >> > > > > > >> > >
> > >> > > > > > >> > > I also strongly agree with the importance of
> > >> multi-language
> > >> > > > > > support. We
> > >> > > > > > >> > are
> > >> > > > > > >> > > talking now about Java, but for application
> development
> > >> use
> > >> > > > cases
> > >> > > > > > >> people
> > >> > > > > > >> > > want to work in whatever language they are using
> > >> elsewhere.
> > >> > I
> > >> > > > > think
> > >> > > > > > >> > moving
> > >> > > > > > >> > > to a model where Kafka itself does the group
> > membership,
> > >> > > > lifecycle
> > >> > > > > > >> > control,
> > >> > > > > > >> > > and partition assignment has the advantage of putting
> > all
> > >> > that
> > >> > > > > > complex
> > >> > > > > > >> > > stuff behind a clean api that the clients are already
> > >> going
> > >> > to
> > >> > > > be
> > >> > > > > > >> > > implementing for their consumer, so the added
> > >> functionality
> > >> > > for
> > >> > > > > > stream
> > >> > > > > > >> > > processing beyond a consumer becomes very minor.
> > >> > > > > > >> > >
> > >> > > > > > >> > > -Jay
> > >> > > > > > >> > >
> > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > >> > > > > > roger.hoover@gmail.com>
> > >> > > > > > >> > > wrote:
> > >> > > > > > >> > >
> > >> > > > > > >> > > > Metamorphosis...nice. :)
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > This has been a great discussion.  As a user of
> Samza
> > >> > who's
> > >> > > > > > recently
> > >> > > > > > >> > > > integrated it into a relatively large
> organization, I
> > >> just
> > >> > > > want
> > >> > > > > to
> > >> > > > > > >> add
> > >> > > > > > >> > > > support to a few points already made.
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
> > >> currently
> > >> > > > exists
> > >> > > > > > that
> > >> > > > > > >> > I've
> > >> > > > > > >> > > > experienced are:
> > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
> environments
> > >> > where
> > >> > > > > Puppet
> > >> > > > > > >> > would
> > >> > > > > > >> > > do
> > >> > > > > > >> > > > just fine but it was the only mechanism to get
> fault
> > >> > > > tolerance.
> > >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> > >> configuring
> > >> > > most
> > >> > > > > of
> > >> > > > > > the
> > >> > > > > > >> > job
> > >> > > > > > >> > > > in code rather than config files.  In general, I
> > think
> > >> the
> > >> > > > goal
> > >> > > > > > >> should
> > >> > > > > > >> > be
> > >> > > > > > >> > > > to make it harder to make mistakes, especially of
> the
> > >> kind
> > >> > > > where
> > >> > > > > > the
> > >> > > > > > >> > code
> > >> > > > > > >> > > > expects something and the config doesn't match.
> The
> > >> > current
> > >> > > > > > config
> > >> > > > > > >> is
> > >> > > > > > >> > > > quite intricate and error-prone.  For example, the
> > >> > > application
> > >> > > > > > logic
> > >> > > > > > >> > may
> > >> > > > > > >> > > > depend on bootstrapping a topic but rather than
> > >> asserting
> > >> > > that
> > >> > > > > in
> > >> > > > > > the
> > >> > > > > > >> > > code,
> > >> > > > > > >> > > > you have to rely on getting the config right.
> > Likewise
> > >> > with
> > >> > > > > > serdes,
> > >> > > > > > >> > the
> > >> > > > > > >> > > > Java representations produced by various serdes
> > (JSON,
> > >> > Avro,
> > >> > > > > etc.)
> > >> > > > > > >> are
> > >> > > > > > >> > > not
> > >> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> > >> without
> > >> > > > > changing
> > >> > > > > > >> the
> > >> > > > > > >> > > > code.   It would be nice for jobs to be able to
> > assert
> > >> > what
> > >> > > > they
> > >> > > > > > >> expect
> > >> > > > > > >> > > > from their input topics in terms of partitioning.
> > >> This is
> > >> > > > > > getting a
> > >> > > > > > >> > > little
> > >> > > > > > >> > > > off topic but I was even thinking about creating a
> > >> "Samza
> > >> > > > config
> > >> > > > > > >> > linter"
> > >> > > > > > >> > > > that would sanity check a set of configs.
> Especially
> > >> in
> > >> > > > > > >> organizations
> > >> > > > > > >> > > > where config is managed by a different team than
> the
> > >> > > > application
> > >> > > > > > >> > > developer,
> > >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > >> > > > > > >> > > > 3) Java/Scala centric - for many teams (especially
> > >> > > DevOps-type
> > >> > > > > > >> folks),
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > pain of the Java toolchain (maven, slow builds,
> weak
> > >> > command
> > >> > > > > line
> > >> > > > > > >> > > support,
> > >> > > > > > >> > > > configuration over convention) really inhibits
> > >> > productivity.
> > >> > > > As
> > >> > > > > > more
> > >> > > > > > >> > and
> > >> > > > > > >> > > > more high-quality clients become available for
> > Kafka, I
> > >> > hope
> > >> > > > > > they'll
> > >> > > > > > >> > > follow
> > >> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> > >> proposals
> > >> > > in
> > >> > > > > this
> > >> > > > > > >> > thread
> > >> > > > > > >> > > > but please consider other languages in the
> ecosystem
> > as
> > >> > > well.
> > >> > > > > > From
> > >> > > > > > >> > what
> > >> > > > > > >> > > > I've heard, Spark has more Python users than
> > >> Java/Scala.
> > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > >
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > >> > > > > > >> > > > and are working on a Yeoman generator
> > >> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> > >> > > Jython/Samza
> > >> > > > > > >> projects
> > >> > > > > > >> > to
> > >> > > > > > >> > > > alleviate some of the pain)
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > I also want to underscore Jay's point about
> improving
> > >> the
> > >> > > user
> > >> > > > > > >> > > experience.
> > >> > > > > > >> > > > That's a very important factor for adoption.  I
> think
> > >> the
> > >> > > goal
> > >> > > > > > should
> > >> > > > > > >> > be
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > make Samza as easy to get started with as something
> > >> like
> > >> > > > > Logstash.
> > >> > > > > > >> > > > Logstash is vastly inferior in terms of
> capabilities
> > to
> > >> > > Samza
> > >> > > > > but
> > >> > > > > > >> it's
> > >> > > > > > >> > > easy
> > >> > > > > > >> > > > to get started and that makes a big difference.
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > Cheers,
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > Roger
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> > Francisci
> > >> > > > Morales <
> > >> > > > > > >> > > > gdfm@apache.org> wrote:
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> > >> Metamorphosis
> > >> > > is
> > >> > > > a
> > >> > > > > > clear
> > >> > > > > > >> > > > winner
> > >> > > > > > >> > > > > :)
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > --
> > >> > > > > > >> > > > > Gianmarco
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> > >> Morales
> > >> > <
> > >> > > > > > >> > > gdfm@apache.org
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > wrote:
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > > Hi,
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > @Martin, thanks for you comments.
> > >> > > > > > >> > > > > > Maybe I'm missing some important point, but I
> > think
> > >> > > > coupling
> > >> > > > > > the
> > >> > > > > > >> > > > releases
> > >> > > > > > >> > > > > > is actually a *good* thing.
> > >> > > > > > >> > > > > > To make an example, would it be better if the
> MR
> > >> and
> > >> > > HDFS
> > >> > > > > > >> > components
> > >> > > > > > >> > > of
> > >> > > > > > >> > > > > > Hadoop had different release schedules?
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > Actually, keeping the discussion in a single
> > place
> > >> > would
> > >> > > > > make
> > >> > > > > > >> > > agreeing
> > >> > > > > > >> > > > on
> > >> > > > > > >> > > > > > releases (and backwards compatibility) much
> > >> easier, as
> > >> > > > > > everybody
> > >> > > > > > >> > > would
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > > responsible for the whole codebase.
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > That said, I like the idea of absorbing
> > samza-core
> > >> as
> > >> > a
> > >> > > > > > >> > sub-project,
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > > leave the fancy stuff separate.
> > >> > > > > > >> > > > > > It probably gives 90% of the benefits we have
> > been
> > >> > > > > discussing
> > >> > > > > > >> here.
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > Cheers,
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > --
> > >> > > > > > >> > > > > > Gianmarco
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > >> > jay.kreps@gmail.com
> > >> > > >
> > >> > > > > > wrote:
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >> Hey Martin,
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> I agree coupling release schedules is a
> > downside.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> Definitely we can try to solve some of the
> > >> > integration
> > >> > > > > > problems
> > >> > > > > > >> in
> > >> > > > > > >> > > > > >> Confluent Platform or in other distributions.
> > But
> > >> I
> > >> > > think
> > >> > > > > > this
> > >> > > > > > >> > ends
> > >> > > > > > >> > > up
> > >> > > > > > >> > > > > >> being really shallow. I guess I feel to really
> > >> get a
> > >> > > good
> > >> > > > > > user
> > >> > > > > > >> > > > > experience
> > >> > > > > > >> > > > > >> the two systems have to kind of feel like part
> > of
> > >> the
> > >> > > > same
> > >> > > > > > thing
> > >> > > > > > >> > and
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> can't really add that in later--you can put
> both
> > >> in
> > >> > the
> > >> > > > > same
> > >> > > > > > >> > > > > downloadable
> > >> > > > > > >> > > > > >> tar file but it doesn't really give a very
> > >> cohesive
> > >> > > > > feeling.
> > >> > > > > > I
> > >> > > > > > >> > agree
> > >> > > > > > >> > > > > that
> > >> > > > > > >> > > > > >> ultimately any of the project stuff is as much
> > >> social
> > >> > > and
> > >> > > > > > naming
> > >> > > > > > >> > as
> > >> > > > > > >> > > > > >> anything else--theoretically two totally
> > >> independent
> > >> > > > > projects
> > >> > > > > > >> > could
> > >> > > > > > >> > > > work
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> tightly align. In practice this seems to be
> > quite
> > >> > > > difficult
> > >> > > > > > >> > though.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> For the frameworks--totally agree it would be
> > >> good to
> > >> > > > > > maintain
> > >> > > > > > >> the
> > >> > > > > > >> > > > > >> framework support with the project. In some
> > cases
> > >> > there
> > >> > > > may
> > >> > > > > > not
> > >> > > > > > >> be
> > >> > > > > > >> > > too
> > >> > > > > > >> > > > > >> much
> > >> > > > > > >> > > > > >> there since the integration gets lighter but I
> > >> think
> > >> > > > > whatever
> > >> > > > > > >> > stubs
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> need should be included. So no I definitely
> > wasn't
> > >> > > trying
> > >> > > > > to
> > >> > > > > > >> imply
> > >> > > > > > >> > > > > >> dropping
> > >> > > > > > >> > > > > >> support for these frameworks, just making the
> > >> > > integration
> > >> > > > > > >> lighter
> > >> > > > > > >> > by
> > >> > > > > > >> > > > > >> separating process management from partition
> > >> > > management.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> You raise two good points we would have to
> > figure
> > >> out
> > >> > > if
> > >> > > > we
> > >> > > > > > went
> > >> > > > > > >> > > down
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> alignment path:
> > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think the
> > >> first
> > >> > > > > question
> > >> > > > > > is
> > >> > > > > > >> > > > whether
> > >> > > > > > >> > > > > >> some "re-branding" would be worth it. If so
> > then I
> > >> > > think
> > >> > > > we
> > >> > > > > > can
> > >> > > > > > >> > > have a
> > >> > > > > > >> > > > > big
> > >> > > > > > >> > > > > >> thread on the name. I'm definitely not set on
> > >> Kafka
> > >> > > > > > Streaming or
> > >> > > > > > >> > > Kafka
> > >> > > > > > >> > > > > >> Streams I was just using them to be kind of
> > >> > > > illustrative. I
> > >> > > > > > >> agree
> > >> > > > > > >> > > with
> > >> > > > > > >> > > > > >> your
> > >> > > > > > >> > > > > >> critique of these names, though I think people
> > >> would
> > >> > > get
> > >> > > > > the
> > >> > > > > > >> idea.
> > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about how
> to
> > >> > > "factor"
> > >> > > > > it.
> > >> > > > > > >> Here
> > >> > > > > > >> > > are
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> options I see (I could get enthusiastic about
> > any
> > >> of
> > >> > > > them):
> > >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> > seperation
> > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api
> and
> > >> > > > samza-core
> > >> > > > > > is
> > >> > > > > > >> > > > absorbed
> > >> > > > > > >> > > > > >> almost like a third client
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> Cheers,
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> -Jay
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> > Kleppmann <
> > >> > > > > > >> > > > martin@kleppmann.com>
> > >> > > > > > >> > > > > >> wrote:
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a
> few
> > >> > > follow-up
> > >> > > > > > >> > comments.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
> > >> > becoming
> > >> > > a
> > >> > > > > > >> > subproject:
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > reasons you mention are good. The risk I see
> > is
> > >> > that
> > >> > > > > > release
> > >> > > > > > >> > > > schedules
> > >> > > > > > >> > > > > >> > become coupled to each other, which can slow
> > >> > everyone
> > >> > > > > down,
> > >> > > > > > >> and
> > >> > > > > > >> > > > large
> > >> > > > > > >> > > > > >> > projects with many contributors are harder
> to
> > >> > manage.
> > >> > > > > > (Jakob,
> > >> > > > > > >> > can
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> speak
> > >> > > > > > >> > > > > >> > from experience, having seen a wider range
> of
> > >> > Hadoop
> > >> > > > > > ecosystem
> > >> > > > > > >> > > > > >> projects?)
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > Some of the goals of a better unified
> > developer
> > >> > > > > experience
> > >> > > > > > >> could
> > >> > > > > > >> > > > also
> > >> > > > > > >> > > > > be
> > >> > > > > > >> > > > > >> > solved by integrating Samza nicely into a
> > Kafka
> > >> > > > > > distribution
> > >> > > > > > >> > (such
> > >> > > > > > >> > > > as
> > >> > > > > > >> > > > > >> > Confluent's). I'm not against merging
> projects
> > >> if
> > >> > we
> > >> > > > > decide
> > >> > > > > > >> > that's
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> way
> > >> > > > > > >> > > > > >> > to go, just pointing out the same goals can
> > >> perhaps
> > >> > > > also
> > >> > > > > be
> > >> > > > > > >> > > achieved
> > >> > > > > > >> > > > > in
> > >> > > > > > >> > > > > >> > other ways.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
> dependency:
> > >> are
> > >> > > you
> > >> > > > > > >> proposing
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > >> > Samza doesn't give any help to people
> wanting
> > to
> > >> > run
> > >> > > on
> > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > >> > > > > > >> > > > > >> > So the docs would basically have a link to
> > >> Slider
> > >> > and
> > >> > > > > > nothing
> > >> > > > > > >> > > else?
> > >> > > > > > >> > > > Or
> > >> > > > > > >> > > > > >> > would we maintain integrations with a bunch
> of
> > >> > > popular
> > >> > > > > > >> > deployment
> > >> > > > > > >> > > > > >> methods
> > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts
> to
> > >> make
> > >> > > > Samza
> > >> > > > > > work
> > >> > > > > > >> > with
> > >> > > > > > >> > > > > >> Slider)?
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > I absolutely think it's a good idea to have
> > the
> > >> > "as a
> > >> > > > > > library"
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > "as a
> > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
> > >> people
> > >> > who
> > >> > > > > want
> > >> > > > > > >> them,
> > >> > > > > > >> > > > but I
> > >> > > > > > >> > > > > >> > think there should also be a low-friction
> path
> > >> for
> > >> > > > common
> > >> > > > > > "as
> > >> > > > > > >> a
> > >> > > > > > >> > > > > service"
> > >> > > > > > >> > > > > >> > deployment methods, for which we probably
> need
> > >> to
> > >> > > > > maintain
> > >> > > > > > >> > > > > integrations.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd
> to
> > >> me,
> > >> > > > > because
> > >> > > > > > >> Kafka
> > >> > > > > > >> > > is
> > >> > > > > > >> > > > > all
> > >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> > >> Transformers"
> > >> > > or
> > >> > > > > > "Kafka
> > >> > > > > > >> > > > Filters"
> > >> > > > > > >> > > > > >> > would be more apt?
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
> > >> (stream
> > >> > > > > > >> transformation
> > >> > > > > > >> > > > with
> > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> > >> library"
> > >> > > bit)
> > >> > > > > > could
> > >> > > > > > >> > > become
> > >> > > > > > >> > > > > >> part of
> > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
> > >> streaming
> > >> > SQL
> > >> > > > and
> > >> > > > > > >> > > > integrations
> > >> > > > > > >> > > > > >> with
> > >> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> > >> project?
> > >> > > In
> > >> > > > > > other
> > >> > > > > > >> > > words,
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > would absorb the proven, stable core of
> Samza,
> > >> > which
> > >> > > > > would
> > >> > > > > > >> > become
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in this
> > >> > thread.
> > >> > > > The
> > >> > > > > > Samza
> > >> > > > > > >> > > > project
> > >> > > > > > >> > > > > >> > would then target that third Kafka client as
> > its
> > >> > base
> > >> > > > > API,
> > >> > > > > > and
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > >> project
> > >> > > > > > >> > > > > >> > would be freed up to explore more
> experimental
> > >> new
> > >> > > > > > horizons.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > Martin
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > >> > > > jay.kreps@gmail.com>
> > >> > > > > > >> wrote:
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > > Hey Martin,
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I
> actually
> > >> > don't
> > >> > > > > think
> > >> > > > > > it
> > >> > > > > > >> > ties
> > >> > > > > > >> > > > our
> > >> > > > > > >> > > > > >> > hands
> > >> > > > > > >> > > > > >> > > at all, all it does is refactor things.
> The
> > >> > > division
> > >> > > > of
> > >> > > > > > >> > > > > >> responsibility is
> > >> > > > > > >> > > > > >> > > that Samza core is responsible for task
> > >> > lifecycle,
> > >> > > > > state,
> > >> > > > > > >> and
> > >> > > > > > >> > > > > >> partition
> > >> > > > > > >> > > > > >> > > management (using the Kafka co-ordinator)
> > but
> > >> it
> > >> > is
> > >> > > > NOT
> > >> > > > > > >> > > > responsible
> > >> > > > > > >> > > > > >> for
> > >> > > > > > >> > > > > >> > > packaging, configuration deployment or
> > >> execution
> > >> > of
> > >> > > > > > >> processes.
> > >> > > > > > >> > > The
> > >> > > > > > >> > > > > >> > problem
> > >> > > > > > >> > > > > >> > > of packaging and starting these processes
> is
> > >> > > > > > >> > > > > >> > > framework/environment-specific. This
> leaves
> > >> > > > individual
> > >> > > > > > >> > > frameworks
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > as
> > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you can
> > get
> > >> > > simple
> > >> > > > > > >> stateless
> > >> > > > > > >> > > > > >> support in
> > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf
> > app
> > >> > > > > framework
> > >> > > > > > >> > > (Slider,
> > >> > > > > > >> > > > > >> > Marathon,
> > >> > > > > > >> > > > > >> > > etc). These are well known by people and
> > have
> > >> > nice
> > >> > > > UIs
> > >> > > > > > and a
> > >> > > > > > >> > lot
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > > flexibility. I don't think they have node
> > >> > affinity
> > >> > > > as a
> > >> > > > > > >> built
> > >> > > > > > >> > in
> > >> > > > > > >> > > > > >> option
> > >> > > > > > >> > > > > >> > > (though I could be wrong). So if we want
> > that
> > >> we
> > >> > > can
> > >> > > > > > either
> > >> > > > > > >> > wait
> > >> > > > > > >> > > > for
> > >> > > > > > >> > > > > >> them
> > >> > > > > > >> > > > > >> > > to add it or do a custom framework to add
> > that
> > >> > > > feature
> > >> > > > > > (as
> > >> > > > > > >> > now).
> > >> > > > > > >> > > > > >> > Obviously
> > >> > > > > > >> > > > > >> > > if you manage things with old-school ops
> > tools
> > >> > > > > > >> > (puppet/chef/etc)
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> get
> > >> > > > > > >> > > > > >> > > locality easily. The nice thing, though,
> is
> > >> that
> > >> > > all
> > >> > > > > the
> > >> > > > > > >> samza
> > >> > > > > > >> > > > > >> "business
> > >> > > > > > >> > > > > >> > > logic" around partition management and
> fault
> > >> > > > tolerance
> > >> > > > > > is in
> > >> > > > > > >> > > Samza
> > >> > > > > > >> > > > > >> core
> > >> > > > > > >> > > > > >> > so
> > >> > > > > > >> > > > > >> > > it is shared across frameworks and the
> > >> framework
> > >> > > > > specific
> > >> > > > > > >> bit
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > just
> > >> > > > > > >> > > > > >> > > whether it is smart enough to try to get
> the
> > >> same
> > >> > > > host
> > >> > > > > > when
> > >> > > > > > >> a
> > >> > > > > > >> > > job
> > >> > > > > > >> > > > is
> > >> > > > > > >> > > > > >> > > restarted.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah
> I
> > >> think
> > >> > > the
> > >> > > > > > goal
> > >> > > > > > >> > would
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > >> (a)
> > >> > > > > > >> > > > > >> > > actually get better alignment in user
> > >> experience,
> > >> > > and
> > >> > > > > (b)
> > >> > > > > > >> > > express
> > >> > > > > > >> > > > > >> this in
> > >> > > > > > >> > > > > >> > > the naming and project branding.
> > Specifically:
> > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> > >> > > > > > "transformation"
> > >> > > > > > >> api
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e.
> be
> > >> able
> > >> > > to
> > >> > > > > > explain
> > >> > > > > > >> > > when
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> use
> > >> > > > > > >> > > > > >> > > the consumer and when to use the stream
> > >> > processing
> > >> > > > > > >> > functionality
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> lead
> > >> > > > > > >> > > > > >> > > people into that experience.
> > >> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza
> 1.4.2
> > >> (or
> > >> > > > > > whatever)
> > >> > > > > > >> > that
> > >> > > > > > >> > > > has
> > >> > > > > > >> > > > > >> both
> > >> > > > > > >> > > > > >> > > Kafka and the stream processing part and
> > they
> > >> > > > actually
> > >> > > > > > work
> > >> > > > > > >> > > > > together.
> > >> > > > > > >> > > > > >> > > 3. Unify the programming experience so the
> > >> client
> > >> > > and
> > >> > > > > > Samza
> > >> > > > > > >> > api
> > >> > > > > > >> > > > > share
> > >> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> > committers
> > >> and
> > >> > > can
> > >> > > > > > have a
> > >> > > > > > >> > > > > separate
> > >> > > > > > >> > > > > >> > repo,
> > >> > > > > > >> > > > > >> > > but I'm actually not really sure (I can't
> > >> find a
> > >> > > > > > definition
> > >> > > > > > >> > of a
> > >> > > > > > >> > > > > >> > subproject
> > >> > > > > > >> > > > > >> > > in Apache).
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > Basically at a high-level you want the
> > >> experience
> > >> > > to
> > >> > > > > > "feel"
> > >> > > > > > >> > > like a
> > >> > > > > > >> > > > > >> single
> > >> > > > > > >> > > > > >> > > system, not to relatively independent
> things
> > >> that
> > >> > > are
> > >> > > > > > kind
> > >> > > > > > >> of
> > >> > > > > > >> > > > > >> awkwardly
> > >> > > > > > >> > > > > >> > > glued together.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > I think if we did that they having naming
> or
> > >> > > branding
> > >> > > > > > like
> > >> > > > > > >> > > "kafka
> > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or something
> > >> like
> > >> > > that
> > >> > > > > > would
> > >> > > > > > >> > > > actually
> > >> > > > > > >> > > > > >> do a
> > >> > > > > > >> > > > > >> > > good job of conveying what it is. I do
> that
> > >> this
> > >> > > > would
> > >> > > > > > help
> > >> > > > > > >> > > > adoption
> > >> > > > > > >> > > > > >> > quite
> > >> > > > > > >> > > > > >> > > a lot as it would correctly convey that
> > using
> > >> > Kafka
> > >> > > > > > >> Streaming
> > >> > > > > > >> > > with
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > is
> > >> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
> > >> pretty
> > >> > > > > heavily
> > >> > > > > > >> > adopted
> > >> > > > > > >> > > > at
> > >> > > > > > >> > > > > >> this
> > >> > > > > > >> > > > > >> > > point.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> > >> originally
> > >> > > > when
> > >> > > > > > open
> > >> > > > > > >> > > > sourcing
> > >> > > > > > >> > > > > >> > Samza,
> > >> > > > > > >> > > > > >> > > however at that time Kafka was relatively
> > >> unknown
> > >> > > and
> > >> > > > > we
> > >> > > > > > >> > decided
> > >> > > > > > >> > > > not
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> > do
> > >> > > > > > >> > > > > >> > > it since we felt it would be limiting.
> From
> > my
> > >> > > point
> > >> > > > of
> > >> > > > > > view
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > three
> > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now
> really
> > >> > heavily
> > >> > > > > used
> > >> > > > > > for
> > >> > > > > > >> > > > stream
> > >> > > > > > >> > > > > >> > > processing, (2) we learned that
> abstracting
> > >> out
> > >> > the
> > >> > > > > > stream
> > >> > > > > > >> > well
> > >> > > > > > >> > > is
> > >> > > > > > >> > > > > >> > > basically impossible, (3) we learned it is
> > >> really
> > >> > > > hard
> > >> > > > > to
> > >> > > > > > >> keep
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > > two
> > >> > > > > > >> > > > > >> > > things feeling like a single product.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > -Jay
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> > >> Kleppmann
> > >> > <
> > >> > > > > > >> > > > > >> martin@kleppmann.com>
> > >> > > > > > >> > > > > >> > > wrote:
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > >> Hi all,
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> I agree with the general philosophy of
> > tying
> > >> > Samza
> > >> > > > > more
> > >> > > > > > >> > firmly
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > > >> Kafka.
> > >> > > > > > >> > > > > >> > >> After I spent a while looking at
> > integrating
> > >> > other
> > >> > > > > > message
> > >> > > > > > >> > > > brokers
> > >> > > > > > >> > > > > >> (e.g.
> > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to
> the
> > >> > > > conclusion
> > >> > > > > > that
> > >> > > > > > >> > > > > >> > SystemConsumer
> > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
> > Kafka's
> > >> > that
> > >> > > > > pretty
> > >> > > > > > >> much
> > >> > > > > > >> > > > > nobody
> > >> > > > > > >> > > > > >> but
> > >> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
> > >> > perhaps
> > >> > > an
> > >> > > > > > >> > exception,
> > >> > > > > > >> > > > but
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.)
> > Thus,
> > >> > > making
> > >> > > > > > Samza
> > >> > > > > > >> > > fully
> > >> > > > > > >> > > > > >> > dependent
> > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > >> > system-independence
> > >> > > > was
> > >> > > > > > >> never
> > >> > > > > > >> > as
> > >> > > > > > >> > > > > real
> > >> > > > > > >> > > > > >> as
> > >> > > > > > >> > > > > >> > we
> > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of
> > code
> > >> > reuse
> > >> > > > are
> > >> > > > > > >> real.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN
> has
> > >> also
> > >> > > > always
> > >> > > > > > been
> > >> > > > > > >> > > > > >> appealing to
> > >> > > > > > >> > > > > >> > >> me, for various reasons already mentioned
> > in
> > >> > this
> > >> > > > > > thread.
> > >> > > > > > >> > > > Although
> > >> > > > > > >> > > > > >> > making
> > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > >> > > > (YARN/Mesos/AWS/etc)
> > >> > > > > > >> seems
> > >> > > > > > >> > > > > >> laudable,
> > >> > > > > > >> > > > > >> > I am
> > >> > > > > > >> > > > > >> > >> a little concerned that it will restrict
> us
> > >> to a
> > >> > > > > lowest
> > >> > > > > > >> > common
> > >> > > > > > >> > > > > >> > denominator.
> > >> > > > > > >> > > > > >> > >> For example, would host affinity
> > (SAMZA-617)
> > >> > still
> > >> > > > be
> > >> > > > > > >> > possible?
> > >> > > > > > >> > > > For
> > >> > > > > > >> > > > > >> jobs
> > >> > > > > > >> > > > > >> > >> with large amounts of state, I think
> > >> SAMZA-617
> > >> > > would
> > >> > > > > be
> > >> > > > > > a
> > >> > > > > > >> big
> > >> > > > > > >> > > > boon,
> > >> > > > > > >> > > > > >> > since
> > >> > > > > > >> > > > > >> > >> restoring state off the changelog on
> every
> > >> > single
> > >> > > > > > restart
> > >> > > > > > >> is
> > >> > > > > > >> > > > > painful,
> > >> > > > > > >> > > > > >> > due
> > >> > > > > > >> > > > > >> > >> to long recovery times. It would be a
> shame
> > >> if
> > >> > the
> > >> > > > > > >> decoupling
> > >> > > > > > >> > > > from
> > >> > > > > > >> > > > > >> YARN
> > >> > > > > > >> > > > > >> > >> made host affinity impossible.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Jay, a question about the proposed API
> for
> > >> > > > > > instantiating a
> > >> > > > > > >> > job
> > >> > > > > > >> > > in
> > >> > > > > > >> > > > > >> code
> > >> > > > > > >> > > > > >> > >> (rather than a properties file): when
> > >> > submitting a
> > >> > > > job
> > >> > > > > > to a
> > >> > > > > > >> > > > > cluster,
> > >> > > > > > >> > > > > >> is
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >> idea that the instantiation code runs on
> a
> > >> > client
> > >> > > > > > >> somewhere,
> > >> > > > > > >> > > > which
> > >> > > > > > >> > > > > >> then
> > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > >> > > YARN/Mesos/AWS/etc?
> > >> > > > > Or
> > >> > > > > > >> does
> > >> > > > > > >> > > that
> > >> > > > > > >> > > > > >> code
> > >> > > > > > >> > > > > >> > run
> > >> > > > > > >> > > > > >> > >> on each container that is part of the job
> > (in
> > >> > > which
> > >> > > > > > case,
> > >> > > > > > >> how
> > >> > > > > > >> > > > does
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > job
> > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
> > >> right to
> > >> > > > make
> > >> > > > > a
> > >> > > > > > 1.0
> > >> > > > > > >> > > > release
> > >> > > > > > >> > > > > >> > with a
> > >> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete.
> So
> > if
> > >> > this
> > >> > > > is
> > >> > > > > > going
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> happen, I
> > >> > > > > > >> > > > > >> > >> think it would be more honest to stick
> with
> > >> 0.*
> > >> > > > > version
> > >> > > > > > >> > numbers
> > >> > > > > > >> > > > > until
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >> library-ified Samza has been implemented,
> > is
> > >> > > stable
> > >> > > > > and
> > >> > > > > > >> > widely
> > >> > > > > > >> > > > > used.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
> > >> Kafka?
> > >> > > There
> > >> > > > > is
> > >> > > > > > >> > > precedent
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > >> tight coupling between different Apache
> > >> projects
> > >> > > > (e.g.
> > >> > > > > > >> > Curator
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I
> think
> > >> > > remaining
> > >> > > > > > >> separate
> > >> > > > > > >> > > > would
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > ok.
> > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on
> Kafka,
> > >> there
> > >> > > is
> > >> > > > > > enough
> > >> > > > > > >> > > > > substance
> > >> > > > > > >> > > > > >> in
> > >> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> > >> project.
> > >> > > An
> > >> > > > > > >> argument
> > >> > > > > > >> > in
> > >> > > > > > >> > > > > >> favour
> > >> > > > > > >> > > > > >> > of
> > >> > > > > > >> > > > > >> > >> merging would be if we think Kafka has a
> > much
> > >> > > > stronger
> > >> > > > > > >> "brand
> > >> > > > > > >> > > > > >> presence"
> > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one.
> If
> > >> the
> > >> > > Kafka
> > >> > > > > > >> project
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > >> willing
> > >> > > > > > >> > > > > >> > to
> > >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of
> > doing
> > >> > > > stateful
> > >> > > > > > >> stream
> > >> > > > > > >> > > > > >> > >> transformations, that would probably have
> > >> much
> > >> > the
> > >> > > > > same
> > >> > > > > > >> > effect
> > >> > > > > > >> > > as
> > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> > >> Processors"
> > >> > or
> > >> > > > > > suchlike.
> > >> > > > > > >> > > Close
> > >> > > > > > >> > > > > >> > >> collaboration between the two projects
> will
> > >> be
> > >> > > > needed
> > >> > > > > in
> > >> > > > > > >> any
> > >> > > > > > >> > > > case.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> From a project management perspective, I
> > >> guess
> > >> > the
> > >> > > > > "new
> > >> > > > > > >> > Samza"
> > >> > > > > > >> > > > > would
> > >> > > > > > >> > > > > >> > have
> > >> > > > > > >> > > > > >> > >> to be developed on a branch alongside
> > ongoing
> > >> > > > > > maintenance
> > >> > > > > > >> of
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > > >> current
> > >> > > > > > >> > > > > >> > >> line of development? I think it would be
> > >> > important
> > >> > > > to
> > >> > > > > > >> > continue
> > >> > > > > > >> > > > > >> > supporting
> > >> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> > >> migration
> > >> > > > path
> > >> > > > > to
> > >> > > > > > >> the
> > >> > > > > > >> > > new
> > >> > > > > > >> > > > > >> > version.
> > >> > > > > > >> > > > > >> > >> Leaving the current versions unsupported
> > and
> > >> > > forcing
> > >> > > > > > people
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> rewrite
> > >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Best,
> > >> > > > > > >> > > > > >> > >> Martin
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > >> > > > jay@confluent.io>
> > >> > > > > > >> wrote:
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >>> Hey Garry,
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be
> > happy
> > >> to
> > >> > > chat
> > >> > > > > > more
> > >> > > > > > >> > about
> > >> > > > > > >> > > > > this
> > >> > > > > > >> > > > > >> if
> > >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
> > >> > started
> > >> > > > with
> > >> > > > > > the
> > >> > > > > > >> > idea
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> "what
> > >> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> > >> > ingestion
> > >> > > > > tool"
> > >> > > > > > but
> > >> > > > > > >> > > > > >> ultimately
> > >> > > > > > >> > > > > >> > we
> > >> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> > >> ingestion
> > >> > > and
> > >> > > > > > >> > > > transformation
> > >> > > > > > >> > > > > >> had
> > >> > > > > > >> > > > > >> > >>> pretty different needs and coupling the
> > two
> > >> > made
> > >> > > > > things
> > >> > > > > > >> > hard.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
> > (KIP-26)
> > >> > > > actually
> > >> > > > > > will
> > >> > > > > > >> > do
> > >> > > > > > >> > > > what
> > >> > > > > > >> > > > > >> you
> > >> > > > > > >> > > > > >> > >> are
> > >> > > > > > >> > > > > >> > >>> looking for.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> With regard to your point about slider,
> I
> > >> don't
> > >> > > > > > >> necessarily
> > >> > > > > > >> > > > > >> disagree.
> > >> > > > > > >> > > > > >> > >> But I
> > >> > > > > > >> > > > > >> > >>> think getting good YARN support is quite
> > >> doable
> > >> > > > and I
> > >> > > > > > >> think
> > >> > > > > > >> > we
> > >> > > > > > >> > > > can
> > >> > > > > > >> > > > > >> make
> > >> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> > >> proposal
> > >> > > > > solves
> > >> > > > > > is
> > >> > > > > > >> > that
> > >> > > > > > >> > > > > >> > >> technically
> > >> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
> > >> cluster
> > >> > > > > > management
> > >> > > > > > >> > > systems
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > way
> > >> > > > > > >> > > > > >> > >>> things are now, you need to write an
> "app
> > >> > master"
> > >> > > > or
> > >> > > > > > >> > > "framework"
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > each
> > >> > > > > > >> > > > > >> > >>> and they are all a little different so
> > >> testing
> > >> > is
> > >> > > > > > really
> > >> > > > > > >> > hard.
> > >> > > > > > >> > > > In
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > >>> absence of this we have been stuck with
> > just
> > >> > YARN
> > >> > > > > which
> > >> > > > > > >> has
> > >> > > > > > >> > > > > >> fantastic
> > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the
> > org,
> > >> but
> > >> > > > zero
> > >> > > > > > >> > > penetration
> > >> > > > > > >> > > > > >> > >> elsewhere.
> > >> > > > > > >> > > > > >> > >>> Given the huge amount of work being put
> in
> > >> to
> > >> > > > slider,
> > >> > > > > > >> > > marathon,
> > >> > > > > > >> > > > > aws
> > >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
> > related
> > >> > > > packaging
> > >> > > > > > >> > > > technologies
> > >> > > > > > >> > > > > >> > people
> > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> > >> > > > > cloud-specific
> > >> > > > > > >> > deploy
> > >> > > > > > >> > > > > >> tools,
> > >> > > > > > >> > > > > >> > >> etc)
> > >> > > > > > >> > > > > >> > >>> I really think it is important to get
> this
> > >> > right.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> -Jay
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> > >> > Turkington
> > >> > > <
> > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>>> Hi all,
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> I think the question below re does
> Samza
> > >> > become
> > >> > > a
> > >> > > > > > >> > sub-project
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>> highlights the broader point around
> > >> migration.
> > >> > > > Chris
> > >> > > > > > >> > mentions
> > >> > > > > > >> > > > > >> Samza's
> > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1
> release
> > >> but
> > >> > I'm
> > >> > > > not
> > >> > > > > > sure
> > >> > > > > > >> > it
> > >> > > > > > >> > > > > feels
> > >> > > > > > >> > > > > >> > >> right to
> > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> > >> deprecate
> > >> > > > most
> > >> > > > > of
> > >> > > > > > >> it.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have some
> > guys
> > >> > who
> > >> > > > have
> > >> > > > > > >> > started
> > >> > > > > > >> > > > > >> working
> > >> > > > > > >> > > > > >> > >> with
> > >> > > > > > >> > > > > >> > >>>> Samza and building some new
> > >> > consumers/producers
> > >> > > > was
> > >> > > > > > next
> > >> > > > > > >> > up.
> > >> > > > > > >> > > > > Sounds
> > >> > > > > > >> > > > > >> > like
> > >> > > > > > >> > > > > >> > >>>> that is absolutely not the direction to
> > >> go. I
> > >> > > need
> > >> > > > > to
> > >> > > > > > >> look
> > >> > > > > > >> > > into
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > KIP
> > >> > > > > > >> > > > > >> > >> in
> > >> > > > > > >> > > > > >> > >>>> more detail but for me the
> attractiveness
> > >> of
> > >> > > > adding
> > >> > > > > > new
> > >> > > > > > >> > Samza
> > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all
> > they
> > >> > were
> > >> > > > > doing
> > >> > > > > > was
> > >> > > > > > >> > > > really
> > >> > > > > > >> > > > > >> > getting
> > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to
> > avoid
> > >> > > > having
> > >> > > > > to
> > >> > > > > > >> > worry
> > >> > > > > > >> > > > > about
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > >>>> lifecycle management of external
> clients.
> > >> If
> > >> > > there
> > >> > > > > is
> > >> > > > > > a
> > >> > > > > > >> > > generic
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a
> > new
> > >> > > > connector
> > >> > > > > > into
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > have
> > >> > > > > > >> > > > > >> a
> > >> > > > > > >> > > > > >> > >> lot of
> > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> > reliability
> > >> > done
> > >> > > > for
> > >> > > > > me
> > >> > > > > > >> then
> > >> > > > > > >> > > it
> > >> > > > > > >> > > > > >> gives
> > >> > > > > > >> > > > > >> > me
> > >> > > > > > >> > > > > >> > >> all
> > >> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers
> > would.
> > >> If
> > >> > > not
> > >> > > > > > then it
> > >> > > > > > >> > > > > >> complicates
> > >> > > > > > >> > > > > >> > my
> > >> > > > > > >> > > > > >> > >>>> operational deployments.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Which is similar to my other question
> > with
> > >> the
> > >> > > > > > proposal
> > >> > > > > > >> --
> > >> > > > > > >> > if
> > >> > > > > > >> > > > we
> > >> > > > > > >> > > > > >> > build a
> > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus
> > the
> > >> > > > requisite
> > >> > > > > > >> shims
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> > integrate
> > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former
> may
> > >> be a
> > >> > > lot
> > >> > > > > more
> > >> > > > > > >> work
> > >> > > > > > >> > > > than
> > >> > > > > > >> > > > > we
> > >> > > > > > >> > > > > >> > >> think.
> > >> > > > > > >> > > > > >> > >>>> We may make it much easier for a
> newcomer
> > >> to
> > >> > get
> > >> > > > > > >> something
> > >> > > > > > >> > > > > running
> > >> > > > > > >> > > > > >> but
> > >> > > > > > >> > > > > >> > >>>> having them step up and get a reliable
> > >> > > production
> > >> > > > > > >> > deployment
> > >> > > > > > >> > > > may
> > >> > > > > > >> > > > > >> still
> > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
> > >> > different
> > >> > > > > > reasons
> > >> > > > > > >> > than
> > >> > > > > > >> > > > > >> today.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable
> > with
> > >> > > making
> > >> > > > > the
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > > >> dependency
> > >> > > > > > >> > > > > >> > >> on
> > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
> absolutely
> > >> see
> > >> > > the
> > >> > > > > > >> benefits
> > >> > > > > > >> > > in
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > >> > > > > > >> > > > terminologies/abstractions
> > >> > > > > > >> > > > > >> that
> > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library
> > >> would
> > >> > > > likely
> > >> > > > > > be a
> > >> > > > > > >> > very
> > >> > > > > > >> > > > > nice
> > >> > > > > > >> > > > > >> > tool
> > >> > > > > > >> > > > > >> > >> to
> > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have
> > the
> > >> > > > concerns
> > >> > > > > > >> above
> > >> > > > > > >> > re
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > >>>> operational side.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Garry
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> > >> [mailto:
> > >> > > > > > >> > gdfm@apache.org
> > >> > > > > > >> > > ]
> > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations
> on
> > >> > Samza
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > >> > > > > > >> > > > > >> > >>>> From outside, I have always perceived
> > Samza
> > >> > as a
> > >> > > > > > >> computing
> > >> > > > > > >> > > > layer
> > >> > > > > > >> > > > > >> over
> > >> > > > > > >> > > > > >> > >>>> Kafka.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit provocative,
> is
> > >> > > "should
> > >> > > > > > Samza
> > >> > > > > > >> be
> > >> > > > > > >> > a
> > >> > > > > > >> > > > > >> > sub-project
> > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> > >> separate
> > >> > > > > project
> > >> > > > > > >> > with a
> > >> > > > > > >> > > > > >> separate
> > >> > > > > > >> > > > > >> > >>>> governance?
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Cheers,
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> --
> > >> > > > > > >> > > > > >> > >>>> Gianmarco
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > >> > > > > > yanfang724@gmail.com>
> > >> > > > > > >> > > > wrote:
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka
> > more
> > >> > > > tightly.
> > >> > > > > > >> > Because
> > >> > > > > > >> > > > > Samza
> > >> > > > > > >> > > > > >> de
> > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
> > >> > leverage
> > >> > > > > what
> > >> > > > > > >> Kafka
> > >> > > > > > >> > > > has.
> > >> > > > > > >> > > > > At
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
> > reinvent
> > >> > what
> > >> > > > > Samza
> > >> > > > > > >> > > already
> > >> > > > > > >> > > > > >> has. I
> > >> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> > >> > ingestion
> > >> > > > and
> > >> > > > > > >> > > > > transformation.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
> > >> image
> > >> > > how
> > >> > > > > the
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > will
> > >> > > > > > >> > > > > >> look
> > >> > > > > > >> > > > > >> > >>>> like.
> > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> > >> > > difference
> > >> > > > > in
> > >> > > > > > >> terms
> > >> > > > > > >> > > of
> > >> > > > > > >> > > > > how
> > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code
> > >> shows
> > >> > (A
> > >> > > > > > client of
> > >> > > > > > >> > > > Kakfa)
> > >> > > > > > >> > > > > ?
> > >> > > > > > >> > > > > >> And
> > >> > > > > > >> > > > > >> > >>>>> user's application code calls this
> > client?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of
> > Kafka
> > >> > (like
> > >> > > > > what
> > >> > > > > > the
> > >> > > > > > >> > > code
> > >> > > > > > >> > > > > >> shows),
> > >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > >> > > > > fault-tolerance?
> > >> > > > > > >> Are
> > >> > > > > > >> > > they
> > >> > > > > > >> > > > > >> taken
> > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> > >> mechanism,
> > >> > > such
> > >> > > > > as
> > >> > > > > > >> > "Samza
> > >> > > > > > >> > > > > >> worker"
> > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 2. What about other features, such as
> > >> > > > auto-scaling,
> > >> > > > > > >> shared
> > >> > > > > > >> > > > > state,
> > >> > > > > > >> > > > > >> > >>>>> monitoring?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is
> > this
> > >> > what
> > >> > > > > Chris
> > >> > > > > > >> > > > suggests?)
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from
> > Kakfa
> > >> > and
> > >> > > > > > produce
> > >> > > > > > >> to
> > >> > > > > > >> > > it.
> > >> > > > > > >> > > > > >> Then it
> > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks
> > like
> > >> > now,
> > >> > > > > > except it
> > >> > > > > > >> > > does
> > >> > > > > > >> > > > > not
> > >> > > > > > >> > > > > >> > rely
> > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> > >> leverage
> > >> > > > Kafka's
> > >> > > > > > >> > metrics,
> > >> > > > > > >> > > > > logs,
> > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> Thanks,
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM,
> Guozhang
> > >> > Wang <
> > >> > > > > > >> > > > > wangguoz@gmail.com
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > >>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Read through the code example and it
> > >> looks
> > >> > > good
> > >> > > > to
> > >> > > > > > me.
> > >> > > > > > >> A
> > >> > > > > > >> > > few
> > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> > >> runnable
> > >> > > like:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > >> > > --config-factory=...
> > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> > deploying
> > >> > Samza
> > >> > > > > more
> > >> > > > > > as
> > >> > > > > > >> > > > embedded
> > >> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> > >> (ignoring
> > >> > > the
> > >> > > > > > >> > > terminology
> > >> > > > > > >> > > > > >> since
> > >> > > > > > >> > > > > >> > >>>>>> it is not the
> > >> > > > > > >> > > > > >> > >>>>> same
> > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> > >> MyStreamTask(configs);
> > >> > > > > Thread
> > >> > > > > > >> > thread
> > >> > > > > > >> > > =
> > >> > > > > > >> > > > > new
> > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment
> modes
> > >> are
> > >> > > > > important
> > >> > > > > > >> for
> > >> > > > > > >> > > > > >> different
> > >> > > > > > >> > > > > >> > >>>>>> types
> > >> > > > > > >> > > > > >> > >>>>> of
> > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making
> Samza
> > >> > purely
> > >> > > > > > >> standalone
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > still
> > >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or
> > library
> > >> > > modes.
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Guozhang
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
> > >> Kreps
> > >> > <
> > >> > > > > > >> > > > jay@confluent.io>
> > >> > > > > > >> > > > > >> > wrote:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> > >> example,
> > >> > it
> > >> > > > was
> > >> > > > > > >> > supposed
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> look
> > >> > > > > > >> > > > > >> > >>>>>>> like
> > >> > > > > > >> > > > > >> > >>>>>>> this:
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> > >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > >> > > > "localhost:4242");
> > >> > > > > > >> > > > > >> StreamingConfig
> > >> > > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > > > "test-topic-2");
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > config.processor(ExampleStreamProcessor.class);
> > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > >> > StringSerializer(),
> > >> > > > new
> > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
> KafkaStreaming
> > >> > > > container =
> > >> > > > > > new
> > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> > container.run();
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> -Jay
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM,
> Jay
> > >> > Kreps <
> > >> > > > > > >> > > > jay@confluent.io
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >> > >>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
> > >> Chris
> > >> > > and
> > >> > > > I
> > >> > > > > > were
> > >> > > > > > >> > > having
> > >> > > > > > >> > > > > >> > >>>>>>>> around
> > >> > > > > > >> > > > > >> > >>>>>>> whether
> > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza
> as a
> > >> kind
> > >> > > of
> > >> > > > > data
> > >> > > > > > >> > > > ingestion
> > >> > > > > > >> > > > > >> > >>>>> framework
> > >> > > > > > >> > > > > >> > >>>>>>> for
> > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to
> > KIP-26
> > >> > > > > "copycat").
> > >> > > > > > >> This
> > >> > > > > > >> > > > kind
> > >> > > > > > >> > > > > of
> > >> > > > > > >> > > > > >> > >>>>>> combined
> > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and
> > YARN
> > >> and
> > >> > > the
> > >> > > > > > >> > discussion
> > >> > > > > > >> > > > > >> around
> > >> > > > > > >> > > > > >> > >>>>>>>> how
> > >> > > > > > >> > > > > >> > >>>>> to
> > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was,
> given
> > >> that
> > >> > > > Samza
> > >> > > > > > was
> > >> > > > > > >> > > > basically
> > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific,
> what
> > if
> > >> > you
> > >> > > > just
> > >> > > > > > >> > embraced
> > >> > > > > > >> > > > > that
> > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > >> > > > > > >> > > > > >> > >>>>>> into
> > >> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> > >> > framework
> > >> > > > and
> > >> > > > > > more
> > >> > > > > > >> > > like a
> > >> > > > > > >> > > > > >> > >>>>>>>> third
> > >> > > > > > >> > > > > >> > >>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> > consumer"
> > >> > with
> > >> > > > > state
> > >> > > > > > >> > > > management
> > >> > > > > > >> > > > > >> > >>>>>> facilities.
> > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> > >> complex
> > >> > > > stream
> > >> > > > > > >> > > processing
> > >> > > > > > >> > > > > >> > >>>>>>>> framework
> > >> > > > > > >> > > > > >> > >>>>>>> this
> > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple
> > thing,
> > >> not
> > >> > > > much
> > >> > > > > > more
> > >> > > > > > >> > > > > >> complicated
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> use
> > >> > > > > > >> > > > > >> > >>>>>>> or
> > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As
> > Chris
> > >> > said
> > >> > > > we
> > >> > > > > > >> thought
> > >> > > > > > >> > > > about
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >>>>>>>> a
> > >> > > > > > >> > > > > >> > >>>>> lot
> > >> > > > > > >> > > > > >> > >>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> > >> > processing
> > >> > > > > > systems
> > >> > > > > > >> > were
> > >> > > > > > >> > > > > doing)
> > >> > > > > > >> > > > > >> > >>>>> seemed
> > >> > > > > > >> > > > > >> > >>>>>>> like
> > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output
> > >> data
> > >> > to
> > >> > > > and
> > >> > > > > > from
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > stream
> > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually
> > looked
> > >> > into
> > >> > > > how
> > >> > > > > > that
> > >> > > > > > >> > > would
> > >> > > > > > >> > > > > >> > >>>>>>>> work,
> > >> > > > > > >> > > > > >> > >>>>> Samza
> > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
> ingestion
> > >> > > framework
> > >> > > > > > for a
> > >> > > > > > >> > > bunch
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>> reasons.
> > >> > > > > > >> > > > > >> > >>>>>> To
> > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a
> > pretty
> > >> > > > different
> > >> > > > > > >> > internal
> > >> > > > > > >> > > > > data
> > >> > > > > > >> > > > > >> > >>>>>>>> model
> > >> > > > > > >> > > > > >> > >>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split
> > them
> > >> and
> > >> > > had
> > >> > > > > an
> > >> > > > > > api
> > >> > > > > > >> > for
> > >> > > > > > >> > > > > Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26)
> > >> and a
> > >> > > > > separate
> > >> > > > > > >> api
> > >> > > > > > >> > > for
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> > embracing
> > >> the
> > >> > > > same
> > >> > > > > > >> > > terminology
> > >> > > > > > >> > > > > and
> > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about
> the
> > >> > current
> > >> > > > > > state is
> > >> > > > > > >> > > that
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>> two
> > >> > > > > > >> > > > > >> > >>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology
> > >> like
> > >> > > > > "stream"
> > >> > > > > > vs
> > >> > > > > > >> > > > "topic"
> > >> > > > > > >> > > > > >> and
> > >> > > > > > >> > > > > >> > >>>>>>> different
> > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means
> > you
> > >> > kind
> > >> > > > of
> > >> > > > > > have
> > >> > > > > > >> to
> > >> > > > > > >> > > > learn
> > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > >> > > > > > >> > > > > >> > >>>>>>> way,
> > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
> different
> > >> way,
> > >> > > > then
> > >> > > > > > kind
> > >> > > > > > >> of
> > >> > > > > > >> > > > > >> > >>>>>>>> understand
> > >> > > > > > >> > > > > >> > >>>>> how
> > >> > > > > > >> > > > > >> > >>>>>>> they
> > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having
> > walked
> > >> a
> > >> > few
> > >> > > > > > people
> > >> > > > > > >> > > through
> > >> > > > > > >> > > > > >> this
> > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to
> > >> get.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of
> > >> time
> > >> > on
> > >> > > > > > >> airplanes I
> > >> > > > > > >> > > > > hacked
> > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> > >> incomplete
> > >> > > > > > prototype
> > >> > > > > > >> of
> > >> > > > > > >> > > > what
> > >> > > > > > >> > > > > >> > >>>>>>>> this would
> > >> > > > > > >> > > > > >> > >>>>> look
> > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
> > >> dumped
> > >> > > into
> > >> > > > > > Kafka
> > >> > > > > > >> as
> > >> > > > > > >> > > it
> > >> > > > > > >> > > > > >> > >>>>>>>> required a
> > >> > > > > > >> > > > > >> > >>>>>> few
> > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here
> is
> > >> the
> > >> > > code:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > >
> > >> > > > > > >> >
> > >> > > > > >
> > >> > >
> > >>
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I
> > just
> > >> > > > > liberally
> > >> > > > > > >> > renamed
> > >> > > > > > >> > > > > >> > >>>>>>>> everything
> > >> > > > > > >> > > > > >> > >>>>> to
> > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
> > >> regard
> > >> > > for
> > >> > > > > > >> > > > compatibility.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something like
> > >> this:
> > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
> Properties();
> > >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > >> > > > > "localhost:4242");
> > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > >> > > > > > >> > > > > >>
> config.processor(ExampleStreamProcessor.class);
> > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > >> > > StringDeserializer());
> > >> > > > > > >> > > > KafkaStreaming
> > >> > > > > > >> > > > > >> > >>>>>> container =
> > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > >> > container.run();
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > >> > > > SamzaContainer;
> > >> > > > > > >> > > > > StreamProcessor
> > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the
> class
> > >> names
> > >> > > in
> > >> > > > a
> > >> > > > > > file
> > >> > > > > > >> > and
> > >> > > > > > >> > > > then
> > >> > > > > > >> > > > > >> > >>>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you
> just
> > >> > > > > instantiate
> > >> > > > > > the
> > >> > > > > > >> > > > > container
> > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced
> > over
> > >> > > > however
> > >> > > > > > many
> > >> > > > > > >> > > > > instances
> > >> > > > > > >> > > > > >> > >>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>> this
> > >> > > > > > >> > > > > >> > >>>>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> > instance
> > >> > dies,
> > >> > > > new
> > >> > > > > > >> tasks
> > >> > > > > > >> > > are
> > >> > > > > > >> > > > > >> added
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
> shutting
> > >> them
> > >> > > > down).
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
> > running
> > >> > this
> > >> > > > > stuff
> > >> > > > > > in
> > >> > > > > > >> > YARN
> > >> > > > > > >> > > > via
> > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
> > >> using
> > >> > > some
> > >> > > > > of
> > >> > > > > > >> their
> > >> > > > > > >> > > > tools
> > >> > > > > > >> > > > > >> > >>>>>>>> but from the
> > >> > > > > > >> > > > > >> > >>>>>> point
> > >> > > > > > >> > > > > >> > >>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these
> stream
> > >> > > > processing
> > >> > > > > > jobs
> > >> > > > > > >> > are
> > >> > > > > > >> > > > > just
> > >> > > > > > >> > > > > >> > >>>>>> stateless
> > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
> > >> expand
> > >> > and
> > >> > > > > > contract
> > >> > > > > > >> > at
> > >> > > > > > >> > > > > will.
> > >> > > > > > >> > > > > >> > >>>>>>>> There
> > >> > > > > > >> > > > > >> > >>>>> is
> > >> > > > > > >> > > > > >> > >>>>>>> no
> > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code,
> > it
> > >> > would
> > >> > > > get
> > >> > > > > > >> larger
> > >> > > > > > >> > > if
> > >> > > > > > >> > > > we
> > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
> > larger.
> > >> We
> > >> > > > really
> > >> > > > > > do
> > >> > > > > > >> > get a
> > >> > > > > > >> > > > ton
> > >> > > > > > >> > > > > >> > >>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>> leverage
> > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> > >> > delegated
> > >> > > to
> > >> > > > > the
> > >> > > > > > >> new
> > >> > > > > > >> > > > > >> consumer.
> > >> > > > > > >> > > > > >> > >>>>> This
> > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> > >> > management
> > >> > > > > > strategy
> > >> > > > > > >> > > > > available
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to
> Samza
> > >> (and
> > >> > > vice
> > >> > > > > > versa)
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > with
> > >> > > > > > >> > > > > >> > >>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>> exact
> > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as
> > state
> > >> > reuse
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
> > >> > thought
> > >> > > > > > >> provoking.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM,
> > Chris
> > >> > > > > Riccomini <
> > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with
> > Samza
> > >> > > > > engineers
> > >> > > > > > at
> > >> > > > > > >> > > > LinkedIn
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>> Confluent
> > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> > observations
> > >> > and
> > >> > > > > would
> > >> > > > > > >> like
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> > >>>>>>>>> propose
> > >> > > > > > >> > > > > >> > >>>>> some
> > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I
> > >> want to
> > >> > > > call
> > >> > > > > > out
> > >> > > > > > >> > about
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > >> > > > > > >> > > > > >> > >>>>>> design,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> > changes.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a
> dynamic
> > >> > > > deployment
> > >> > > > > > >> system.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> > >> SystemConsumer/SystemProducer
> > >> > and
> > >> > > > > > Kafka's
> > >> > > > > > >> > > > consumer
> > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > >> > > > > > >> > > > > >> > >>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
> > >> > problems.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
> > related,
> > >> > but
> > >> > > > I'll
> > >> > > > > > >> > address
> > >> > > > > > >> > > > them
> > >> > > > > > >> > > > > >> in
> > >> > > > > > >> > > > > >> > >>>>> order.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use
> > of a
> > >> > > > dynamic
> > >> > > > > > >> > > deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > >> > > > > > >> > > > > >> > >>>>>> such
> > >> > > > > > >> > > > > >> > >>>>>>>>> as
> > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
> initially
> > >> built
> > >> > > > > Samza,
> > >> > > > > > we
> > >> > > > > > >> > bet
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > >> > >>>>>>>>> there
> > >> > > > > > >> > > > > >> > >>>>>> would
> > >> > > > > > >> > > > > >> > >>>>>>>>> be
> > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area,
> and
> > >> we
> > >> > > could
> > >> > > > > > >> support
> > >> > > > > > >> > > > them,
> > >> > > > > > >> > > > > >> and
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>> rest
> > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there
> are
> > >> many
> > >> > > > > > >> variations.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > >> > > > > > >> > > > > >> > >>>>>> many
> > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start
> > >> their
> > >> > > > > > processors
> > >> > > > > > >> > like
> > >> > > > > > >> > > > > normal
> > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
> traditional
> > >> > > > deployment
> > >> > > > > > >> scripts
> > >> > > > > > >> > > > such
> > >> > > > > > >> > > > > as
> > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > >> > > > > > >> > > > > >> > >>>>>> Chef,
> > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
> > >> system
> > >> > > on
> > >> > > > > > users
> > >> > > > > > >> > makes
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
> > painful
> > >> for
> > >> > > > first
> > >> > > > > > time
> > >> > > > > > >> > > > users.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
> requirement
> > >> was
> > >> > > also
> > >> > > > a
> > >> > > > > > bit
> > >> > > > > > >> of
> > >> > > > > > >> > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > >> > > > > > >> > > > > >> > >>>>>> because
> > >> > > > > > >> > > > > >> > >>>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding
> > between
> > >> > the
> > >> > > > > > nature of
> > >> > > > > > >> > > batch
> > >> > > > > > >> > > > > >> jobs
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>> stream
> > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> > >> > > conscious
> > >> > > > > > effort
> > >> > > > > > >> to
> > >> > > > > > >> > > > favor
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>> Hadoop
> > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things,
> > >> since
> > >> > it
> > >> > > > > worked
> > >> > > > > > >> and
> > >> > > > > > >> > > was
> > >> > > > > > >> > > > > well
> > >> > > > > > >> > > > > >> > >>>>>>> understood.
> > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that
> > >> batch
> > >> > > jobs
> > >> > > > > > have a
> > >> > > > > > >> > > > definite
> > >> > > > > > >> > > > > >> > >>>>>> beginning,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs
> > don't
> > >> > > > > (usually).
> > >> > > > > > >> This
> > >> > > > > > >> > > > leads
> > >> > > > > > >> > > > > to
> > >> > > > > > >> > > > > >> > >>>>>>>>> a
> > >> > > > > > >> > > > > >> > >>>>> much
> > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for
> > stream
> > >> > > > > processors.
> > >> > > > > > >> You
> > >> > > > > > >> > > > > >> basically
> > >> > > > > > >> > > > > >> > >>>>>>>>> just
> > >> > > > > > >> > > > > >> > >>>>>>> need
> > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> > >> processor,
> > >> > and
> > >> > > > > start
> > >> > > > > > >> it.
> > >> > > > > > >> > > The
> > >> > > > > > >> > > > > way
> > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's
> > no
> > >> > > concept
> > >> > > > > of
> > >> > > > > > a
> > >> > > > > > >> > > cluster
> > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > >> > > > > > >> > > > > >> > >>>>>> add
> > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> > >> coupling
> > >> > > > Samza
> > >> > > > > > with
> > >> > > > > > >> a
> > >> > > > > > >> > > > > >> scheduler
> > >> > > > > > >> > > > > >> > >>>>>>>>> is
> > >> > > > > > >> > > > > >> > >>>>>> that
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
> > >> handle
> > >> > > > > > deployment.
> > >> > > > > > >> > > This
> > >> > > > > > >> > > > > >> pulls
> > >> > > > > > >> > > > > >> > >>>>>>>>> in a
> > >> > > > > > >> > > > > >> > >>>>>>> bunch
> > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> > >> > > distribution
> > >> > > > > > (config
> > >> > > > > > >> > > > > stream),
> > >> > > > > > >> > > > > >> > >>>>>>>>> shell
> > >> > > > > > >> > > > > >> > >>>>>>> scrips
> > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> > packaging
> > >> > (all
> > >> > > > the
> > >> > > > > > .tgz
> > >> > > > > > >> > > > stuff),
> > >> > > > > > >> > > > > >> etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring
> dynamic
> > >> > > > deployment
> > >> > > > > > was
> > >> > > > > > >> to
> > >> > > > > > >> > > > > support
> > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> > >> > > locality,
> > >> > > > > you
> > >> > > > > > >> need
> > >> > > > > > >> > to
> > >> > > > > > >> > > > put
> > >> > > > > > >> > > > > >> > >>>>>>>>> your
> > >> > > > > > >> > > > > >> > >>>>>> processors
> > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> > processing.
> > >> > Upon
> > >> > > > > > further
> > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > >> > > > > > >> > > > > >> > >>>>>>> though,
> > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
> beneficial.
> > >> > There
> > >> > > is
> > >> > > > > > some
> > >> > > > > > >> > good
> > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > >> > > > > > >> > > > > >> > >>>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
> SAMZA-335.
> > >> > Again,
> > >> > > we
> > >> > > > > > took
> > >> > > > > > >> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > >> > > > > > >> > > > > >> > >>>>>> path,
> > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> > differences
> > >> > > > between
> > >> > > > > > HDFS
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > >> > > > > > >> > > > > >> > >>>>>> has
> > >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has
> partitions.
> > >> This
> > >> > > > leads
> > >> > > > > to
> > >> > > > > > >> less
> > >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
> > >> > > processors
> > >> > > > > on
> > >> > > > > > top
> > >> > > > > > >> > of
> > >> > > > > > >> > > > > Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a
> > crutch.
> > >> > > Samza
> > >> > > > > > doesn't
> > >> > > > > > >> > > have
> > >> > > > > > >> > > > > any
> > >> > > > > > >> > > > > >> > >>>>>>>>> built
> > >> > > > > > >> > > > > >> > >>>>> in
> > >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
> > >> > depends
> > >> > > on
> > >> > > > > the
> > >> > > > > > >> > > dynamic
> > >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to
> > handle
> > >> > > > restarts
> > >> > > > > > >> when a
> > >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > >> > > > > > >> > > > > >> > >>>>>>> made
> > >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> > >> standalone
> > >> > > Samza
> > >> > > > > > >> > container
> > >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is
> good,
> > >> but I
> > >> > > > think
> > >> > > > > > that
> > >> > > > > > >> > > we've
> > >> > > > > > >> > > > > >> gone
> > >> > > > > > >> > > > > >> > >>>>>>>>> too
> > >> > > > > > >> > > > > >> > >>>>>> far
> > >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > >> > > > (SystemConsumer,
> > >> > > > > > >> > > > > SystemProducer,
> > >> > > > > > >> > > > > >> > >>>> etc).
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just
> > about
> > >> > every
> > >> > > > > > >> component
> > >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > >> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> > >> > > ConfigRewriter,
> > >> > > > > > etc).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> > >> > forgotten,
> > >> > > as
> > >> > > > > > well.
> > >> > > > > > >> > Some
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>>>>>> these
> > >> > > > > > >> > > > > >> > >>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not
> to
> > >> be.
> > >> > > This
> > >> > > > > all
> > >> > > > > > >> comes
> > >> > > > > > >> > > at
> > >> > > > > > >> > > > a
> > >> > > > > > >> > > > > >> cost:
> > >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is
> > making
> > >> it
> > >> > > > harder
> > >> > > > > > for
> > >> > > > > > >> > our
> > >> > > > > > >> > > > > users
> > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> pick
> > >> > > > > > >> > > > > >> > >>>>>> up
> > >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It
> > also
> > >> > makes
> > >> > > > it
> > >> > > > > > >> > difficult
> > >> > > > > > >> > > > for
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about
> > what
> > >> the
> > >> > > > > > >> > > characteristics
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> > >> characteristics
> > >> > > > change
> > >> > > > > > >> > > depending
> > >> > > > > > >> > > > on
> > >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are
> > most
> > >> > > visible
> > >> > > > > in
> > >> > > > > > the
> > >> > > > > > >> > > > System
> > >> > > > > > >> > > > > >> APIs.
> > >> > > > > > >> > > > > >> > >>>>> What
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
> > >> functional is
> > >> > > > Kafka
> > >> > > > > > as
> > >> > > > > > >> its
> > >> > > > > > >> > > > > >> > >>>>>>>>> transport
> > >> > > > > > >> > > > > >> > >>>>>> layer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> But
> > >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use
> > >> cases
> > >> > > into
> > >> > > > > one
> > >> > > > > > >> API:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports
> both
> > >> of
> > >> > > these
> > >> > > > > use
> > >> > > > > > >> > cases.
> > >> > > > > > >> > > > The
> > >> > > > > > >> > > > > >> > >>>>>>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>> is,
> > >> > > > > > >> > > > > >> > >>>>>>>>> we
> > >> > > > > > >> > > > > >> > >>>>>>>>> actually want different features
> for
> > >> each
> > >> > > use
> > >> > > > > > case.
> > >> > > > > > >> By
> > >> > > > > > >> > > > > >> papering
> > >> > > > > > >> > > > > >> > >>>>>>>>> over
> > >> > > > > > >> > > > > >> > >>>>>>> these
> > >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a
> > single
> > >> > API,
> > >> > > > > we've
> > >> > > > > > >> > > > introduced
> > >> > > > > > >> > > > > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > >> > > > > > >> > > > > >> > >>>>>>> leaky
> > >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like
> > in
> > >> (2)
> > >> > > is
> > >> > > > to
> > >> > > > > > have
> > >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
> > >> > offsets
> > >> > > > > (like
> > >> > > > > > >> > Kafka).
> > >> > > > > > >> > > > > This
> > >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > >> > > > > > >> > > > > >> > >>>>> with
> > >> > > > > > >> > > > > >> > >>>>>>> (1),
> > >> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems
> have
> > >> > > > different
> > >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> > >> mailing
> > >> > > list
> > >> > > > > and
> > >> > > > > > >> the
> > >> > > > > > >> > > SQL
> > >> > > > > > >> > > > > >> JIRAs
> > >> > > > > > >> > > > > >> > >>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> > >> > > replayability.
> > >> > > > > > Kafka
> > >> > > > > > >> > > allows
> > >> > > > > > >> > > > us
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> > >>>>> rewind
> > >> > > > > > >> > > > > >> > >>>>>>>>> when
> > >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other
> > systems
> > >> > > don't.
> > >> > > > In
> > >> > > > > > some
> > >> > > > > > >> > > > cases,
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>>> return
> > >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > >> > > > > > >> WikipediaSystemConsumer)
> > >> > > > > > >> > > > > because
> > >> > > > > > >> > > > > >> > >>>>>>>>> they
> > >> > > > > > >> > > > > >> > >>>>>> have
> > >> > > > > > >> > > > > >> > >>>>>>> no
> > >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example.
> > Kafka
> > >> > > > supports
> > >> > > > > > >> > > > > partitioning,
> > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > >> > > > > > >> > > > > >> > >>>>> many
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
> > >> having a
> > >> > > > single
> > >> > > > > > >> > > partition
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other
> systems
> > >> model
> > >> > > > > > >> partitioning
> > >> > > > > > >> > > > > >> > >>>> differently (e.g.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also
> a
> > >> mess.
> > >> > > > > > Creating
> > >> > > > > > >> > > streams
> > >> > > > > > >> > > > > in
> > >> > > > > > >> > > > > >> a
> > >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> > >> impossible.
> > >> > > As
> > >> > > > is
> > >> > > > > > >> > modeling
> > >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > >> > > > > > >> > > > > >> > >>>>> for
> > >> > > > > > >> > > > > >> > >>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> > >> partitions,
> > >> > > > > location,
> > >> > > > > > >> > etc).
> > >> > > > > > >> > > > The
> > >> > > > > > >> > > > > >> > >>>>>>>>> list
> > >> > > > > > >> > > > > >> > >>>>> goes
> > >> > > > > > >> > > > > >> > >>>>>>> on.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing
> > >> Samza,
> > >> > > > > Kafka's
> > >> > > > > > >> > > consumer
> > >> > > > > > >> > > > > and
> > >> > > > > > >> > > > > >> > >>>>> producer
> > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set.
> > On
> > >> the
> > >> > > > > > >> > consumer-side,
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> > >>>>>>>>> had two
> > >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
> > consumer,
> > >> or
> > >> > > the
> > >> > > > > > simple
> > >> > > > > > >> > > > > consumer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> The
> > >> > > > > > >> > > > > >> > >>>>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was
> > that
> > >> it
> > >> > > > > > controlled
> > >> > > > > > >> > your
> > >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments,
> and
> > >> the
> > >> > > order
> > >> > > > > in
> > >> > > > > > >> which
> > >> > > > > > >> > > you
> > >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > >> > > > > > >> > > > > >> > >>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>>>>> with
> > >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's
> not
> > >> > > simple.
> > >> > > > > It's
> > >> > > > > > >> > basic.
> > >> > > > > > >> > > > You
> > >> > > > > > >> > > > > >> > >>>>>>>>> end up
> > >> > > > > > >> > > > > >> > >>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really
> low-level
> > >> stuff
> > >> > > > that
> > >> > > > > > you
> > >> > > > > > >> > > > > shouldn't.
> > >> > > > > > >> > > > > >> > >>>>>>>>> We
> > >> > > > > > >> > > > > >> > >>>>>> spent a
> > >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > >> > > > KafkaSystemConsumer
> > >> > > > > > very
> > >> > > > > > >> > > > robust.
> > >> > > > > > >> > > > > >> It
> > >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some
> cool
> > >> > > features:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering
> and
> > >> > > > > > prioritization.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> > >> assignment
> > >> > > to
> > >> > > > > > support
> > >> > > > > > >> > > > joins,
> > >> > > > > > >> > > > > >> > >>>>>>>>> global
> > >> > > > > > >> > > > > >> > >>>>>> state
> > >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)),
> > etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> > >> > checkpointing.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time
> > is
> > >> > that
> > >> > > > > these
> > >> > > > > > >> > > features
> > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > >> > > > > > >> > > > > >> > >>>>>>> actually
> > >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
> > consumers
> > >> > (not
> > >> > > > just
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > stream
> > >> > > > > > >> > > > > >> > >>>>>> processors)
> > >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like
> > joins
> > >> > and
> > >> > > > > > partition
> > >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> > >> > conclusion.
> > >> > > > > > They're
> > >> > > > > > >> > > adding
> > >> > > > > > >> > > > a
> > >> > > > > > >> > > > > >> ton
> > >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> > >> consumer
> > >> > > > > > >> > > implementation.
> > >> > > > > > >> > > > > To a
> > >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > >> > > > > > >> > > > > >> > >>>>> it's
> > >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've
> already
> > >> done
> > >> > > in
> > >> > > > > > Samza.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up
> > taking
> > >> a
> > >> > > very
> > >> > > > > > similar
> > >> > > > > > >> > > > > approach
> > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>>> Samza's
> > >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> > implementation
> > >> for
> > >> > > > > > handling
> > >> > > > > > >> > > offset
> > >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> > >> management
> > >> > > > > feature
> > >> > > > > > >> > stores
> > >> > > > > > >> > > > > >> offset
> > >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows
> > >> you to
> > >> > > > fetch
> > >> > > > > > them
> > >> > > > > > >> > > from
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste,
> > >> since
> > >> > we
> > >> > > > > could
> > >> > > > > > >> have
> > >> > > > > > >> > > > shared
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>> work
> > >> > > > > > >> > > > > >> > >>>>>> if
> > >> > > > > > >> > > > > >> > >>>>>>>>> it
> > >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
> > >> get-go.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> > >> radical
> > >> > > > > > proposal.
> > >> > > > > > >> > Samza
> > >> > > > > > >> > > > is
> > >> > > > > > >> > > > > >> > >>>>> relatively
> > >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture
> to
> > >> say
> > >> > > that
> > >> > > > > > we're
> > >> > > > > > >> > > near a
> > >> > > > > > >> > > > > 1.0
> > >> > > > > > >> > > > > >> > >>>>>> release.
> > >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what
> > >> we've
> > >> > > > > learned,
> > >> > > > > > and
> > >> > > > > > >> > > begin
> > >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > >> > > > > > >> > > > > >> > >>>>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we
> > >> change if
> > >> > > we
> > >> > > > > were
> > >> > > > > > >> > > starting
> > >> > > > > > >> > > > > >> from
> > >> > > > > > >> > > > > >> > >>>>>> scratch?
> > >> > > > > > >> > > > > >> > >>>>>>>>> My
> > >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the
> *only*
> > >> way
> > >> > to
> > >> > > > run
> > >> > > > > > Samza
> > >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all
> direct
> > >> > > > > dependences
> > >> > > > > > on
> > >> > > > > > >> > > YARN,
> > >> > > > > > >> > > > > >> Mesos,
> > >> > > > > > >> > > > > >> > >>>> etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to
> support
> > >> only
> > >> > > > Kafka
> > >> > > > > > as
> > >> > > > > > >> the
> > >> > > > > > >> > > > > stream
> > >> > > > > > >> > > > > >> > >>>>>> processing
> > >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics,
> > logging,
> > >> > > > > > >> serialization,
> > >> > > > > > >> > > and
> > >> > > > > > >> > > > > >> > >>>>>>>>> config
> > >> > > > > > >> > > > > >> > >>>>>>> systems,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues
> > that
> > >> I
> > >> > > > > outlined
> > >> > > > > > >> > above.
> > >> > > > > > >> > > It
> > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > >> > > > > > >> > > > > >> > >>>>> also
> > >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> > >> > > > dramatically.
> > >> > > > > > >> > > Supporting
> > >> > > > > > >> > > > > >> only
> > >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow
> > >> Samza
> > >> > to
> > >> > > be
> > >> > > > > > >> executed
> > >> > > > > > >> > > on
> > >> > > > > > >> > > > > YARN
> > >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > >> > > > Marathon/Aurora),
> > >> > > > > or
> > >> > > > > > >> most
> > >> > > > > > >> > > > other
> > >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > >> > > > > > >> > > > > >> > >>>>>>> deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a
> lot
> > >> > easier
> > >> > > > for
> > >> > > > > > new
> > >> > > > > > >> > > users.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > >> > > > > > >> > > > > >> > >>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without
> > YARN.
> > >> > The
> > >> > > > drop
> > >> > > > > > in
> > >> > > > > > >> > > mailing
> > >> > > > > > >> > > > > >> list
> > >> > > > > > >> > > > > >> > >>>>>> traffic
> > >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
> > >> overdue to
> > >> > > me.
> > >> > > > > The
> > >> > > > > > >> > > reality
> > >> > > > > > >> > > > > is,
> > >> > > > > > >> > > > > >> > >>>>> everyone
> > >> > > > > > >> > > > > >> > >>>>>>>>> that
> > >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
> > >> Kafka.
> > >> > We
> > >> > > > > > basically
> > >> > > > > > >> > > > require
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >>>>>> already
> > >> > > > > > >> > > > > >> > >>>>>>> in
> > >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work.
> > Those
> > >> > that
> > >> > > > are
> > >> > > > > > >> using
> > >> > > > > > >> > > > other
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into
> > >> Kafka
> > >> > > (1),
> > >> > > > > and
> > >> > > > > > >> then
> > >> > > > > > >> > > > they
> > >> > > > > > >> > > > > do
> > >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
> > >> already
> > >> > > > > > discussion (
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > >
> > >> > > > > > >> >
> > >> > > > > >
> > >> > >
> > >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >> > > > > > >> > > > > >> > >>>>> 767
> > >> > > > > > >> > > > > >> > >>>>>>>>> )
> > >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into
> > Kafka
> > >> > > > extremely
> > >> > > > > > >> easy.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple
> with
> > >> > Kafka,
> > >> > > > we
> > >> > > > > > can
> > >> > > > > > >> > > > leverage
> > >> > > > > > >> > > > > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > >> > > > > > >> > > > > >> > >>>>>>> their
> > >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> > >> maintain
> > >> > > our
> > >> > > > > own
> > >> > > > > > >> > config,
> > >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > >> > > > > > >> > > > > >> > >>>>> etc.
> > >> > > > > > >> > > > > >> > >>>>>>> We
> > >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries,
> > and
> > >> > make
> > >> > > > them
> > >> > > > > > >> > better.
> > >> > > > > > >> > > > This
> > >> > > > > > >> > > > > >> > >>>>>>>>> will
> > >> > > > > > >> > > > > >> >
> > >> ...
> > >>
> > >> [Message clipped]
> > >
> > >
> > >
> >
>
>
>
> --
> Jordan Shaw
> Full Stack Software Engineer
> PubNub Inc
> 1045 17th St
> San Francisco, CA 94107
>

Re: Thoughts and obesrvations on Samza

Posted by Jordan Shaw <jo...@pubnub.com>.
Jay,
I think doing this iteratively in smaller chunks is a better way to go as
new issues arise. As Navina said Kafka is a "stream system" and Samza is a
"stream processor" and those two ideas should be mutually exclusive.

-Jordan

On Mon, Jul 13, 2015 at 10:06 AM, Jay Kreps <ja...@gmail.com> wrote:

> Hmm, thought about this more. Maybe this is just too much too quick.
> Overall I think there is some enthusiasm for the proposal but it's not
> really unanimous enough to make any kind of change this big cleanly. The
> board doesn't really like the merging stuff, user's are concerned about
> compatibility, I didn't feel there was unanimous agreement on dropping
> SystemConsumer, etc. Even if this is the right end state to get to,
> probably trying to push all this through at once isn't the right way to do
> it.
>
> So let me propose a kind of fifth (?) option which I think is less dramatic
> and let's things happen gradually. I think this is kind of like combining
> the first part of Yi's proposal and Jakob's third option, leaving the rest
> to be figured out incrementally:
>
> Option 5: We continue the prototype I shared and propose that as a kind of
> "transformer" client API in Kafka. This isn't really a full-fledged stream
> processing layer, more like a supped up consumer api for munging topics.
> This would let us figure out some of the technical bits, how to do this on
> Kafka's group management features, how to integrate the txn feature to do
> the exactly-once stuff in these transformations, and get all this stuff
> solid. This api would have valid uses in it's own right, especially when
> your transformation will be embedded inside an existing service or
> application which isn't possible with Samza (or other existing systems that
> I know of).
>
> Independently we can iterate on some of the ideas of the original proposal
> individually and figure out how (if at all) to make use of this
> functionality. This can be done bit-by-bit:
> - Could be that the existing StreamTask API ends up wrapping this
> - Could end up exposed directly in Samza as Yi proposed
> - Could be that just the lower-level group-management stuff get's used, and
> in this case it could be either just for standalone mode, or always
> - Could be that it stays as-is
>
> The advantage of this is it is lower risk...we basically don't have to make
> 12 major decisions all at once that kind of hinge on what amounts to a
> pretty aggressive rewrite. The disadvantage of this is it is a bit more
> confusing as all this is getting figured out.
>
> As with some of the other stuff, this would require a further discussion in
> the Kafka community if people do like this approach.
>
> Thoughts?
>
> -Jay
>
>
>
>
> On Sun, Jul 12, 2015 at 10:52 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Chris,
> >
> > Yeah, I'm obviously in favor of this.
> >
> > The sub-project approach seems the ideal way to take a graceful step in
> > this direction, so I will ping the board folks and see why they are
> > discouraged, it would be good to understand that. If we go that route we
> > would need to do a similar discussion in the Kafka list (but makes sense
> to
> > figure out first if it is what Samza wants).
> >
> > Irrespective of how it's implemented, though, to me the important things
> > are the following:
> > 1. Unify the website, config, naming, docs, metrics, etc--basically fix
> > the product experience so the "stream" and the "processing" feel like a
> > single user experience and brand. This seems minor but I think is a
> really
> > big deal.
> > 2. Make "standalone" mode a first class citizen and have a real technical
> > plan to be able to support cluster managers other than YARN.
> > 3. Make the config and out-of-the-box experience more usable
> >
> > I think that prototype gives a practical example of how 1-3 could be done
> > and we should pursue it. This is a pretty radical change, so I wouldn't
> be
> > shocked if people didn't want to take a step like that.
> >
> > Maybe it would make sense to see if people are on board with that general
> > idea, and then try to get some advice on sub-projects in parallel and
> nail
> > down those details?
> >
> > -Jay
> >
> > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <cr...@apache.org>
> > wrote:
> >
> >> Hey all,
> >>
> >> I want to start by saying that I'm absolutely thrilled to be a part of
> >> this
> >> community. The amount of level-headed, thoughtful, educated discussion
> >> that's gone on over the past ~10 days is overwhelming. Wonderful.
> >>
> >> It seems like discussion is waning a bit, and we've reached some
> >> conclusions. There are several key emails in this threat, which I want
> to
> >> call out:
> >>
> >> 1. Jakob's summary of the three potential ways forward.
> >>
> >>
> >>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> >> 2. Julian's call out that we should be focusing on community over code.
> >>
> >>
> >>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> >> 3. Martin's summary about the benefits of merging communities.
> >>
> >>
> >>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> >> 4. Jakob's comments about the distinction between community and code
> >> paths.
> >>
> >>
> >>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> >>
> >> I agree with the comments on all of these emails. I think Martin's
> summary
> >> of his position aligns very closely with my own. To that end, I think we
> >> should get concrete about what the proposal is, and call a vote on it.
> >> Given that Jay, Martin, and I seem to be aligning fairly closely, I
> think
> >> we should start with:
> >>
> >> 1. [community] Make Samza a subproject of Kafka.
> >> 2. [community] Make all Samza PMC/committers committers of the
> subproject.
> >> 3. [community] Migrate Samza's website/documentation into Kafka's.
> >> 4. [code] Have the Samza community and the Kafka community start a
> >> from-scratch reboot together in the new Kafka subproject. We can
> >> borrow/copy &  paste significant chunks of code from Samza's code base.
> >> 5. [code] The subproject would intentionally eliminate support for both
> >> other streaming systems and all deployment systems.
> >> 6. [code] Attempt to provide a bridge from our SystemConsumer to KIP-26
> >> (copy cat)
> >> 7. [code] Attempt to provide a bridge from the new subproject's
> processor
> >> interface to our legacy StreamTask interface.
> >> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> >> subproject that has a fault-tolerant container with state management.
> >>
> >> It's likely that (6) and (7) won't be fully drop-in. Still, the closer
> we
> >> can get, the better it's going to be for our existing community.
> >>
> >> One thing that I didn't touch on with (2) is whether any Samza PMC
> members
> >> should be rolled into Kafka PMC membership as well (though, Jay and
> Jakob
> >> are already PMC members on both). I think that Samza's community
> deserves
> >> a
> >> voice on the PMC, so I'd propose that we roll at least a few PMC members
> >> into the Kafka PMC, but I don't have a strong framework for which people
> >> to
> >> pick.
> >>
> >> Before (8), I think that Samza's TLP can continue to commit bug fixes
> and
> >> patches as it sees fit, provided that we openly communicate that we
> won't
> >> necessarily migrate new features to the new subproject, and that the TLP
> >> will be shut down after the migration to the Kafka subproject occurs.
> >>
> >> Jakob, I could use your guidance here about about how to achieve this
> from
> >> an Apache process perspective (sorry).
> >>
> >> * Should I just call a vote on this proposal?
> >> * Should it happen on dev or private?
> >> * Do committers have binding votes, or just PMC?
> >>
> >> Having trouble finding much detail on the Apache wikis. :(
> >>
> >> Cheers,
> >> Chris
> >>
> >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com> wrote:
> >>
> >> > Thanks, Jay. This argument persuaded me actually. :)
> >> >
> >> > Fang, Yan
> >> > yanfang724@gmail.com
> >> >
> >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> >> >
> >> > > Hey Yan,
> >> > >
> >> > > Yeah philosophically I think the argument is that you should capture
> >> the
> >> > > stream in Kafka independent of the transformation. This is
> obviously a
> >> > > Kafka-centric view point.
> >> > >
> >> > > Advantages of this:
> >> > > - In practice I think this is what e.g. Storm people often end up
> >> doing
> >> > > anyway. You usually need to throttle any access to a live serving
> >> > database.
> >> > > - Can have multiple subscribers and they get the same thing without
> >> > > additional load on the source system.
> >> > > - Applications can tap into the stream if need be by subscribing.
> >> > > - You can debug your transformation by tailing the Kafka topic with
> >> the
> >> > > console consumer
> >> > > - Can tee off the same data stream for batch analysis or Lambda arch
> >> > style
> >> > > re-processing
> >> > >
> >> > > The disadvantage is that it will use Kafka resources. But the idea
> is
> >> > > eventually you will have multiple subscribers to any data source (at
> >> > least
> >> > > for monitoring) so you will end up there soon enough anyway.
> >> > >
> >> > > Down the road the technical benefit is that I think it gives us a
> good
> >> > path
> >> > > towards end-to-end exactly once semantics from source to
> destination.
> >> > > Basically the connectors need to support idempotence when talking to
> >> > Kafka
> >> > > and we need the transactional write feature in Kafka to make the
> >> > > transformation atomic. This is actually pretty doable if you
> separate
> >> > > connector=>kafka problem from the generic transformations which are
> >> > always
> >> > > kafka=>kafka. However I think it is quite impossible to do in a
> >> > all_things
> >> > > => all_things environment. Today you can say "well the semantics of
> >> the
> >> > > Samza APIs depend on the connectors you use" but it is actually
> worse
> >> > then
> >> > > that because the semantics actually depend on the pairing of
> >> > connectors--so
> >> > > not only can you probably not get a usable "exactly once" guarantee
> >> > > end-to-end it can actually be quite hard to reverse engineer what
> >> > property
> >> > > (if any) your end-to-end flow has if you have heterogenous systems.
> >> > >
> >> > > -Jay
> >> > >
> >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
> >> wrote:
> >> > >
> >> > > > {quote}
> >> > > > maintained in a separate repository and retaining the existing
> >> > > > committership but sharing as much else as possible (website, etc)
> >> > > > {quote}
> >> > > >
> >> > > > Overall, I agree on this idea. Now the question is more about "how
> >> to
> >> > do
> >> > > > it".
> >> > > >
> >> > > > On the other hand, one thing I want to point out is that, if we
> >> decide
> >> > to
> >> > > > go this way, how do we want to support
> >> > > > otherSystem-transformation-otherSystem use case?
> >> > > >
> >> > > > Basically, there are four user groups here:
> >> > > >
> >> > > > 1. Kafka-transformation-Kafka
> >> > > > 2. Kafka-transformation-otherSystem
> >> > > > 3. otherSystem-transformation-Kafka
> >> > > > 4. otherSystem-transformation-otherSystem
> >> > > >
> >> > > > For group 1, they can easily use the new Samza library to achieve.
> >> For
> >> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka or
> >> > Kafka->
> >> > > > transformation -> copyCat.
> >> > > >
> >> > > > The problem is for group 4. Do we want to abandon this or still
> >> support
> >> > > it?
> >> > > > Of course, this use case can be achieved by using copyCat ->
> >> > > transformation
> >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> persuade
> >> > them
> >> > > to
> >> > > > do this long chain. If yes, it will also be a win for Kafka too.
> Or
> >> if
> >> > > > there is no one in this community actually doing this so far,
> maybe
> >> ok
> >> > to
> >> > > > not support the group 4 directly.
> >> > > >
> >> > > > Thanks,
> >> > > >
> >> > > > Fang, Yan
> >> > > > yanfang724@gmail.com
> >> > > >
> >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
> >> wrote:
> >> > > >
> >> > > > > Yeah I agree with this summary. I think there are kind of two
> >> > questions
> >> > > > > here:
> >> > > > > 1. Technically does alignment/reliance on Kafka make sense
> >> > > > > 2. Branding wise (naming, website, concepts, etc) does alignment
> >> with
> >> > > > Kafka
> >> > > > > make sense
> >> > > > >
> >> > > > > Personally I do think both of these things would be really
> >> valuable,
> >> > > and
> >> > > > > would dramatically alter the trajectory of the project.
> >> > > > >
> >> > > > > My preference would be to see if people can mostly agree on a
> >> > direction
> >> > > > > rather than splintering things off. From my point of view the
> >> ideal
> >> > > > outcome
> >> > > > > of all the options discussed would be to make Samza a closely
> >> aligned
> >> > > > > subproject, maintained in a separate repository and retaining
> the
> >> > > > existing
> >> > > > > committership but sharing as much else as possible (website,
> >> etc). No
> >> > > > idea
> >> > > > > about how these things work, Jacob, you probably know more.
> >> > > > >
> >> > > > > No discussion amongst the Kafka folks has happened on this, but
> >> > likely
> >> > > we
> >> > > > > should figure out what the Samza community actually wants first.
> >> > > > >
> >> > > > > I admit that this is a fairly radical departure from how things
> >> are.
> >> > > > >
> >> > > > > If that doesn't fly, I think, yeah we could leave Samza as it is
> >> and
> >> > do
> >> > > > the
> >> > > > > more radical reboot inside Kafka. From my point of view that
> does
> >> > leave
> >> > > > > things in a somewhat confusing state since now there are two
> >> stream
> >> > > > > processing systems more or less coupled to Kafka in large part
> >> made
> >> > by
> >> > > > the
> >> > > > > same people. But, arguably that might be a cleaner way to make
> the
> >> > > > cut-over
> >> > > > > and perhaps less risky for Samza community since if it works
> >> people
> >> > can
> >> > > > > switch and if it doesn't nothing will have changed. Dunno, how
> do
> >> > > people
> >> > > > > feel about this?
> >> > > > >
> >> > > > > -Jay
> >> > > > >
> >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> jghoman@gmail.com>
> >> > > wrote:
> >> > > > >
> >> > > > > > >  This leads me to thinking that merging projects and
> >> communities
> >> > > > might
> >> > > > > > be a good idea: with the union of experience from both
> >> communities,
> >> > > we
> >> > > > > will
> >> > > > > > probably build a better system that is better for users.
> >> > > > > > Is this what's being proposed though? Merging the projects
> seems
> >> > like
> >> > > > > > a consequence of at most one of the three directions under
> >> > > discussion:
> >> > > > > > 1) Samza 2.0: The Samza community relies more heavily on Kafka
> >> for
> >> > > > > > configuration, etc. (to a greater or lesser extent to be
> >> > determined)
> >> > > > > > but the Samza community would not automatically merge withe
> >> Kafka
> >> > > > > > community (the Phoenix/HBase example is a good one here).
> >> > > > > > 2) Samza Reboot: The Samza community continues to exist with a
> >> > > limited
> >> > > > > > project scope, but similarly would not need to be part of the
> >> Kafka
> >> > > > > > community (ie given committership) to progress.  Here, maybe
> the
> >> > > Samza
> >> > > > > > team would become a subproject of Kafka (the Board frowns on
> >> > > > > > subprojects at the moment, so I'm not sure if that's even
> >> > feasible),
> >> > > > > > but that would not be required.
> >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option
> the
> >> > Kafka
> >> > > > > > team builds its own streaming library, possibly off of Jay's
> >> > > > > > prototype, which has not direct lineage to the Samza team.
> >> There's
> >> > > no
> >> > > > > > reason for the Kafka team to bring in the Samza team.
> >> > > > > >
> >> > > > > > Is the Kafka community on board with this?
> >> > > > > >
> >> > > > > > To be clear, all three options under discussion are
> interesting,
> >> > > > > > technically valid and likely healthy directions for the
> project.
> >> > > > > > Also, they are not mutually exclusive.  The Samza community
> >> could
> >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka community
> >> went
> >> > > > > > forward with 'Hey Samza!'  My points above are directed
> >> entirely at
> >> > > > > > the community aspect of these choices.
> >> > > > > > -Jakob
> >> > > > > >
> >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> roger.hoover@gmail.com>
> >> > > wrote:
> >> > > > > > > That's great.  Thanks, Jay.
> >> > > > > > >
> >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> jay@confluent.io>
> >> > > wrote:
> >> > > > > > >
> >> > > > > > >> Yeah totally agree. I think you have this issue even today,
> >> > right?
> >> > > > > I.e.
> >> > > > > > if
> >> > > > > > >> you need to make a simple config change and you're running
> in
> >> > YARN
> >> > > > > today
> >> > > > > > >> you end up bouncing the job which then rebuilds state. I
> >> think
> >> > the
> >> > > > fix
> >> > > > > > is
> >> > > > > > >> exactly what you described which is to have a long timeout
> on
> >> > > > > partition
> >> > > > > > >> movement for stateful jobs so that if a job is just getting
> >> > > bounced,
> >> > > > > and
> >> > > > > > >> the cluster manager (or admin) is smart enough to restart
> it
> >> on
> >> > > the
> >> > > > > same
> >> > > > > > >> host when possible, it can optimistically reuse any
> existing
> >> > state
> >> > > > it
> >> > > > > > finds
> >> > > > > > >> on disk (if it is valid).
> >> > > > > > >>
> >> > > > > > >> So in this model the charter of the CM is to place
> processes
> >> as
> >> > > > > > stickily as
> >> > > > > > >> possible and to restart or re-place failed processes. The
> >> > charter
> >> > > of
> >> > > > > the
> >> > > > > > >> partition management system is to control the assignment of
> >> work
> >> > > to
> >> > > > > > these
> >> > > > > > >> processes. The nice thing about this is that the work
> >> > assignment,
> >> > > > > > timeouts,
> >> > > > > > >> behavior, configs, and code will all be the same across all
> >> > > cluster
> >> > > > > > >> managers.
> >> > > > > > >>
> >> > > > > > >> So I think that prototype would actually give you exactly
> >> what
> >> > you
> >> > > > > want
> >> > > > > > >> today for any cluster manager (or manual placement +
> restart
> >> > > script)
> >> > > > > > that
> >> > > > > > >> was sticky in terms of host placement since there is
> already
> >> a
> >> > > > > > configurable
> >> > > > > > >> partition movement timeout and task-by-task state reuse
> with
> >> a
> >> > > check
> >> > > > > on
> >> > > > > > >> state validity.
> >> > > > > > >>
> >> > > > > > >> -Jay
> >> > > > > > >>
> >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> >> > > > roger.hoover@gmail.com
> >> > > > > >
> >> > > > > > >> wrote:
> >> > > > > > >>
> >> > > > > > >> > That would be great to let Kafka do as much heavy lifting
> >> as
> >> > > > > possible
> >> > > > > > and
> >> > > > > > >> > make it easier for other languages to implement Samza
> apis.
> >> > > > > > >> >
> >> > > > > > >> > One thing to watch out for is the interplay between
> Kafka's
> >> > > group
> >> > > > > > >> > management and the external scheduler/process manager's
> >> fault
> >> > > > > > tolerance.
> >> > > > > > >> > If a container dies, the Kafka group membership protocol
> >> will
> >> > > try
> >> > > > to
> >> > > > > > >> assign
> >> > > > > > >> > it's tasks to other containers while at the same time the
> >> > > process
> >> > > > > > manager
> >> > > > > > >> > is trying to relaunch the container.  Without some
> >> > consideration
> >> > > > for
> >> > > > > > this
> >> > > > > > >> > (like a configurable amount of time to wait before Kafka
> >> > alters
> >> > > > the
> >> > > > > > group
> >> > > > > > >> > membership), there may be thrashing going on which is
> >> > especially
> >> > > > bad
> >> > > > > > for
> >> > > > > > >> > containers with large amounts of local state.
> >> > > > > > >> >
> >> > > > > > >> > Someone else pointed this out already but I thought it
> >> might
> >> > be
> >> > > > > worth
> >> > > > > > >> > calling out again.
> >> > > > > > >> >
> >> > > > > > >> > Cheers,
> >> > > > > > >> >
> >> > > > > > >> > Roger
> >> > > > > > >> >
> >> > > > > > >> >
> >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> >> jay@confluent.io>
> >> > > > > wrote:
> >> > > > > > >> >
> >> > > > > > >> > > Hey Roger,
> >> > > > > > >> > >
> >> > > > > > >> > > I couldn't agree more. We spent a bunch of time talking
> >> to
> >> > > > people
> >> > > > > > and
> >> > > > > > >> > that
> >> > > > > > >> > > is exactly the stuff we heard time and again. What
> makes
> >> it
> >> > > > hard,
> >> > > > > of
> >> > > > > > >> > > course, is that there is some tension between
> >> compatibility
> >> > > with
> >> > > > > > what's
> >> > > > > > >> > > there now and making things better for new users.
> >> > > > > > >> > >
> >> > > > > > >> > > I also strongly agree with the importance of
> >> multi-language
> >> > > > > > support. We
> >> > > > > > >> > are
> >> > > > > > >> > > talking now about Java, but for application development
> >> use
> >> > > > cases
> >> > > > > > >> people
> >> > > > > > >> > > want to work in whatever language they are using
> >> elsewhere.
> >> > I
> >> > > > > think
> >> > > > > > >> > moving
> >> > > > > > >> > > to a model where Kafka itself does the group
> membership,
> >> > > > lifecycle
> >> > > > > > >> > control,
> >> > > > > > >> > > and partition assignment has the advantage of putting
> all
> >> > that
> >> > > > > > complex
> >> > > > > > >> > > stuff behind a clean api that the clients are already
> >> going
> >> > to
> >> > > > be
> >> > > > > > >> > > implementing for their consumer, so the added
> >> functionality
> >> > > for
> >> > > > > > stream
> >> > > > > > >> > > processing beyond a consumer becomes very minor.
> >> > > > > > >> > >
> >> > > > > > >> > > -Jay
> >> > > > > > >> > >
> >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> >> > > > > > roger.hoover@gmail.com>
> >> > > > > > >> > > wrote:
> >> > > > > > >> > >
> >> > > > > > >> > > > Metamorphosis...nice. :)
> >> > > > > > >> > > >
> >> > > > > > >> > > > This has been a great discussion.  As a user of Samza
> >> > who's
> >> > > > > > recently
> >> > > > > > >> > > > integrated it into a relatively large organization, I
> >> just
> >> > > > want
> >> > > > > to
> >> > > > > > >> add
> >> > > > > > >> > > > support to a few points already made.
> >> > > > > > >> > > >
> >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
> >> currently
> >> > > > exists
> >> > > > > > that
> >> > > > > > >> > I've
> >> > > > > > >> > > > experienced are:
> >> > > > > > >> > > > 1) YARN - YARN is overly complex in many environments
> >> > where
> >> > > > > Puppet
> >> > > > > > >> > would
> >> > > > > > >> > > do
> >> > > > > > >> > > > just fine but it was the only mechanism to get fault
> >> > > > tolerance.
> >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> >> configuring
> >> > > most
> >> > > > > of
> >> > > > > > the
> >> > > > > > >> > job
> >> > > > > > >> > > > in code rather than config files.  In general, I
> think
> >> the
> >> > > > goal
> >> > > > > > >> should
> >> > > > > > >> > be
> >> > > > > > >> > > > to make it harder to make mistakes, especially of the
> >> kind
> >> > > > where
> >> > > > > > the
> >> > > > > > >> > code
> >> > > > > > >> > > > expects something and the config doesn't match.  The
> >> > current
> >> > > > > > config
> >> > > > > > >> is
> >> > > > > > >> > > > quite intricate and error-prone.  For example, the
> >> > > application
> >> > > > > > logic
> >> > > > > > >> > may
> >> > > > > > >> > > > depend on bootstrapping a topic but rather than
> >> asserting
> >> > > that
> >> > > > > in
> >> > > > > > the
> >> > > > > > >> > > code,
> >> > > > > > >> > > > you have to rely on getting the config right.
> Likewise
> >> > with
> >> > > > > > serdes,
> >> > > > > > >> > the
> >> > > > > > >> > > > Java representations produced by various serdes
> (JSON,
> >> > Avro,
> >> > > > > etc.)
> >> > > > > > >> are
> >> > > > > > >> > > not
> >> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> >> without
> >> > > > > changing
> >> > > > > > >> the
> >> > > > > > >> > > > code.   It would be nice for jobs to be able to
> assert
> >> > what
> >> > > > they
> >> > > > > > >> expect
> >> > > > > > >> > > > from their input topics in terms of partitioning.
> >> This is
> >> > > > > > getting a
> >> > > > > > >> > > little
> >> > > > > > >> > > > off topic but I was even thinking about creating a
> >> "Samza
> >> > > > config
> >> > > > > > >> > linter"
> >> > > > > > >> > > > that would sanity check a set of configs.  Especially
> >> in
> >> > > > > > >> organizations
> >> > > > > > >> > > > where config is managed by a different team than the
> >> > > > application
> >> > > > > > >> > > developer,
> >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> >> > > > > > >> > > > 3) Java/Scala centric - for many teams (especially
> >> > > DevOps-type
> >> > > > > > >> folks),
> >> > > > > > >> > > the
> >> > > > > > >> > > > pain of the Java toolchain (maven, slow builds, weak
> >> > command
> >> > > > > line
> >> > > > > > >> > > support,
> >> > > > > > >> > > > configuration over convention) really inhibits
> >> > productivity.
> >> > > > As
> >> > > > > > more
> >> > > > > > >> > and
> >> > > > > > >> > > > more high-quality clients become available for
> Kafka, I
> >> > hope
> >> > > > > > they'll
> >> > > > > > >> > > follow
> >> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> >> proposals
> >> > > in
> >> > > > > this
> >> > > > > > >> > thread
> >> > > > > > >> > > > but please consider other languages in the ecosystem
> as
> >> > > well.
> >> > > > > > From
> >> > > > > > >> > what
> >> > > > > > >> > > > I've heard, Spark has more Python users than
> >> Java/Scala.
> >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> >> > > > > > >> > > >
> >> > > > > > >> > > >
> >> > > > > > >> > >
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> >> > > > > > >> > > > and are working on a Yeoman generator
> >> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> >> > > Jython/Samza
> >> > > > > > >> projects
> >> > > > > > >> > to
> >> > > > > > >> > > > alleviate some of the pain)
> >> > > > > > >> > > >
> >> > > > > > >> > > > I also want to underscore Jay's point about improving
> >> the
> >> > > user
> >> > > > > > >> > > experience.
> >> > > > > > >> > > > That's a very important factor for adoption.  I think
> >> the
> >> > > goal
> >> > > > > > should
> >> > > > > > >> > be
> >> > > > > > >> > > to
> >> > > > > > >> > > > make Samza as easy to get started with as something
> >> like
> >> > > > > Logstash.
> >> > > > > > >> > > > Logstash is vastly inferior in terms of capabilities
> to
> >> > > Samza
> >> > > > > but
> >> > > > > > >> it's
> >> > > > > > >> > > easy
> >> > > > > > >> > > > to get started and that makes a big difference.
> >> > > > > > >> > > >
> >> > > > > > >> > > > Cheers,
> >> > > > > > >> > > >
> >> > > > > > >> > > > Roger
> >> > > > > > >> > > >
> >> > > > > > >> > > >
> >> > > > > > >> > > >
> >> > > > > > >> > > >
> >> > > > > > >> > > >
> >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> Francisci
> >> > > > Morales <
> >> > > > > > >> > > > gdfm@apache.org> wrote:
> >> > > > > > >> > > >
> >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> >> Metamorphosis
> >> > > is
> >> > > > a
> >> > > > > > clear
> >> > > > > > >> > > > winner
> >> > > > > > >> > > > > :)
> >> > > > > > >> > > > >
> >> > > > > > >> > > > > --
> >> > > > > > >> > > > > Gianmarco
> >> > > > > > >> > > > >
> >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> >> Morales
> >> > <
> >> > > > > > >> > > gdfm@apache.org
> >> > > > > > >> > > > >
> >> > > > > > >> > > > > wrote:
> >> > > > > > >> > > > >
> >> > > > > > >> > > > > > Hi,
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > @Martin, thanks for you comments.
> >> > > > > > >> > > > > > Maybe I'm missing some important point, but I
> think
> >> > > > coupling
> >> > > > > > the
> >> > > > > > >> > > > releases
> >> > > > > > >> > > > > > is actually a *good* thing.
> >> > > > > > >> > > > > > To make an example, would it be better if the MR
> >> and
> >> > > HDFS
> >> > > > > > >> > components
> >> > > > > > >> > > of
> >> > > > > > >> > > > > > Hadoop had different release schedules?
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > Actually, keeping the discussion in a single
> place
> >> > would
> >> > > > > make
> >> > > > > > >> > > agreeing
> >> > > > > > >> > > > on
> >> > > > > > >> > > > > > releases (and backwards compatibility) much
> >> easier, as
> >> > > > > > everybody
> >> > > > > > >> > > would
> >> > > > > > >> > > > be
> >> > > > > > >> > > > > > responsible for the whole codebase.
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > That said, I like the idea of absorbing
> samza-core
> >> as
> >> > a
> >> > > > > > >> > sub-project,
> >> > > > > > >> > > > and
> >> > > > > > >> > > > > > leave the fancy stuff separate.
> >> > > > > > >> > > > > > It probably gives 90% of the benefits we have
> been
> >> > > > > discussing
> >> > > > > > >> here.
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > Cheers,
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > --
> >> > > > > > >> > > > > > Gianmarco
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> >> > jay.kreps@gmail.com
> >> > > >
> >> > > > > > wrote:
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > >> Hey Martin,
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> I agree coupling release schedules is a
> downside.
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> Definitely we can try to solve some of the
> >> > integration
> >> > > > > > problems
> >> > > > > > >> in
> >> > > > > > >> > > > > >> Confluent Platform or in other distributions.
> But
> >> I
> >> > > think
> >> > > > > > this
> >> > > > > > >> > ends
> >> > > > > > >> > > up
> >> > > > > > >> > > > > >> being really shallow. I guess I feel to really
> >> get a
> >> > > good
> >> > > > > > user
> >> > > > > > >> > > > > experience
> >> > > > > > >> > > > > >> the two systems have to kind of feel like part
> of
> >> the
> >> > > > same
> >> > > > > > thing
> >> > > > > > >> > and
> >> > > > > > >> > > > you
> >> > > > > > >> > > > > >> can't really add that in later--you can put both
> >> in
> >> > the
> >> > > > > same
> >> > > > > > >> > > > > downloadable
> >> > > > > > >> > > > > >> tar file but it doesn't really give a very
> >> cohesive
> >> > > > > feeling.
> >> > > > > > I
> >> > > > > > >> > agree
> >> > > > > > >> > > > > that
> >> > > > > > >> > > > > >> ultimately any of the project stuff is as much
> >> social
> >> > > and
> >> > > > > > naming
> >> > > > > > >> > as
> >> > > > > > >> > > > > >> anything else--theoretically two totally
> >> independent
> >> > > > > projects
> >> > > > > > >> > could
> >> > > > > > >> > > > work
> >> > > > > > >> > > > > >> to
> >> > > > > > >> > > > > >> tightly align. In practice this seems to be
> quite
> >> > > > difficult
> >> > > > > > >> > though.
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> For the frameworks--totally agree it would be
> >> good to
> >> > > > > > maintain
> >> > > > > > >> the
> >> > > > > > >> > > > > >> framework support with the project. In some
> cases
> >> > there
> >> > > > may
> >> > > > > > not
> >> > > > > > >> be
> >> > > > > > >> > > too
> >> > > > > > >> > > > > >> much
> >> > > > > > >> > > > > >> there since the integration gets lighter but I
> >> think
> >> > > > > whatever
> >> > > > > > >> > stubs
> >> > > > > > >> > > > you
> >> > > > > > >> > > > > >> need should be included. So no I definitely
> wasn't
> >> > > trying
> >> > > > > to
> >> > > > > > >> imply
> >> > > > > > >> > > > > >> dropping
> >> > > > > > >> > > > > >> support for these frameworks, just making the
> >> > > integration
> >> > > > > > >> lighter
> >> > > > > > >> > by
> >> > > > > > >> > > > > >> separating process management from partition
> >> > > management.
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> You raise two good points we would have to
> figure
> >> out
> >> > > if
> >> > > > we
> >> > > > > > went
> >> > > > > > >> > > down
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> alignment path:
> >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think the
> >> first
> >> > > > > question
> >> > > > > > is
> >> > > > > > >> > > > whether
> >> > > > > > >> > > > > >> some "re-branding" would be worth it. If so
> then I
> >> > > think
> >> > > > we
> >> > > > > > can
> >> > > > > > >> > > have a
> >> > > > > > >> > > > > big
> >> > > > > > >> > > > > >> thread on the name. I'm definitely not set on
> >> Kafka
> >> > > > > > Streaming or
> >> > > > > > >> > > Kafka
> >> > > > > > >> > > > > >> Streams I was just using them to be kind of
> >> > > > illustrative. I
> >> > > > > > >> agree
> >> > > > > > >> > > with
> >> > > > > > >> > > > > >> your
> >> > > > > > >> > > > > >> critique of these names, though I think people
> >> would
> >> > > get
> >> > > > > the
> >> > > > > > >> idea.
> >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about how to
> >> > > "factor"
> >> > > > > it.
> >> > > > > > >> Here
> >> > > > > > >> > > are
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> options I see (I could get enthusiastic about
> any
> >> of
> >> > > > them):
> >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> seperation
> >> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api and
> >> > > > samza-core
> >> > > > > > is
> >> > > > > > >> > > > absorbed
> >> > > > > > >> > > > > >> almost like a third client
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> Cheers,
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> -Jay
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> Kleppmann <
> >> > > > > > >> > > > martin@kleppmann.com>
> >> > > > > > >> > > > > >> wrote:
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a few
> >> > > follow-up
> >> > > > > > >> > comments.
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
> >> > becoming
> >> > > a
> >> > > > > > >> > subproject:
> >> > > > > > >> > > > the
> >> > > > > > >> > > > > >> > reasons you mention are good. The risk I see
> is
> >> > that
> >> > > > > > release
> >> > > > > > >> > > > schedules
> >> > > > > > >> > > > > >> > become coupled to each other, which can slow
> >> > everyone
> >> > > > > down,
> >> > > > > > >> and
> >> > > > > > >> > > > large
> >> > > > > > >> > > > > >> > projects with many contributors are harder to
> >> > manage.
> >> > > > > > (Jakob,
> >> > > > > > >> > can
> >> > > > > > >> > > > you
> >> > > > > > >> > > > > >> speak
> >> > > > > > >> > > > > >> > from experience, having seen a wider range of
> >> > Hadoop
> >> > > > > > ecosystem
> >> > > > > > >> > > > > >> projects?)
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > Some of the goals of a better unified
> developer
> >> > > > > experience
> >> > > > > > >> could
> >> > > > > > >> > > > also
> >> > > > > > >> > > > > be
> >> > > > > > >> > > > > >> > solved by integrating Samza nicely into a
> Kafka
> >> > > > > > distribution
> >> > > > > > >> > (such
> >> > > > > > >> > > > as
> >> > > > > > >> > > > > >> > Confluent's). I'm not against merging projects
> >> if
> >> > we
> >> > > > > decide
> >> > > > > > >> > that's
> >> > > > > > >> > > > the
> >> > > > > > >> > > > > >> way
> >> > > > > > >> > > > > >> > to go, just pointing out the same goals can
> >> perhaps
> >> > > > also
> >> > > > > be
> >> > > > > > >> > > achieved
> >> > > > > > >> > > > > in
> >> > > > > > >> > > > > >> > other ways.
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > - With regard to dropping the YARN dependency:
> >> are
> >> > > you
> >> > > > > > >> proposing
> >> > > > > > >> > > > that
> >> > > > > > >> > > > > >> > Samza doesn't give any help to people wanting
> to
> >> > run
> >> > > on
> >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> >> > > > > > >> > > > > >> > So the docs would basically have a link to
> >> Slider
> >> > and
> >> > > > > > nothing
> >> > > > > > >> > > else?
> >> > > > > > >> > > > Or
> >> > > > > > >> > > > > >> > would we maintain integrations with a bunch of
> >> > > popular
> >> > > > > > >> > deployment
> >> > > > > > >> > > > > >> methods
> >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts to
> >> make
> >> > > > Samza
> >> > > > > > work
> >> > > > > > >> > with
> >> > > > > > >> > > > > >> Slider)?
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > I absolutely think it's a good idea to have
> the
> >> > "as a
> >> > > > > > library"
> >> > > > > > >> > and
> >> > > > > > >> > > > > "as a
> >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
> >> people
> >> > who
> >> > > > > want
> >> > > > > > >> them,
> >> > > > > > >> > > > but I
> >> > > > > > >> > > > > >> > think there should also be a low-friction path
> >> for
> >> > > > common
> >> > > > > > "as
> >> > > > > > >> a
> >> > > > > > >> > > > > service"
> >> > > > > > >> > > > > >> > deployment methods, for which we probably need
> >> to
> >> > > > > maintain
> >> > > > > > >> > > > > integrations.
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to
> >> me,
> >> > > > > because
> >> > > > > > >> Kafka
> >> > > > > > >> > > is
> >> > > > > > >> > > > > all
> >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> >> Transformers"
> >> > > or
> >> > > > > > "Kafka
> >> > > > > > >> > > > Filters"
> >> > > > > > >> > > > > >> > would be more apt?
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
> >> (stream
> >> > > > > > >> transformation
> >> > > > > > >> > > > with
> >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> >> library"
> >> > > bit)
> >> > > > > > could
> >> > > > > > >> > > become
> >> > > > > > >> > > > > >> part of
> >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
> >> streaming
> >> > SQL
> >> > > > and
> >> > > > > > >> > > > integrations
> >> > > > > > >> > > > > >> with
> >> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> >> project?
> >> > > In
> >> > > > > > other
> >> > > > > > >> > > words,
> >> > > > > > >> > > > > >> Kafka
> >> > > > > > >> > > > > >> > would absorb the proven, stable core of Samza,
> >> > which
> >> > > > > would
> >> > > > > > >> > become
> >> > > > > > >> > > > the
> >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in this
> >> > thread.
> >> > > > The
> >> > > > > > Samza
> >> > > > > > >> > > > project
> >> > > > > > >> > > > > >> > would then target that third Kafka client as
> its
> >> > base
> >> > > > > API,
> >> > > > > > and
> >> > > > > > >> > the
> >> > > > > > >> > > > > >> project
> >> > > > > > >> > > > > >> > would be freed up to explore more experimental
> >> new
> >> > > > > > horizons.
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > Martin
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> >> > > > jay.kreps@gmail.com>
> >> > > > > > >> wrote:
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > > Hey Martin,
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually
> >> > don't
> >> > > > > think
> >> > > > > > it
> >> > > > > > >> > ties
> >> > > > > > >> > > > our
> >> > > > > > >> > > > > >> > hands
> >> > > > > > >> > > > > >> > > at all, all it does is refactor things. The
> >> > > division
> >> > > > of
> >> > > > > > >> > > > > >> responsibility is
> >> > > > > > >> > > > > >> > > that Samza core is responsible for task
> >> > lifecycle,
> >> > > > > state,
> >> > > > > > >> and
> >> > > > > > >> > > > > >> partition
> >> > > > > > >> > > > > >> > > management (using the Kafka co-ordinator)
> but
> >> it
> >> > is
> >> > > > NOT
> >> > > > > > >> > > > responsible
> >> > > > > > >> > > > > >> for
> >> > > > > > >> > > > > >> > > packaging, configuration deployment or
> >> execution
> >> > of
> >> > > > > > >> processes.
> >> > > > > > >> > > The
> >> > > > > > >> > > > > >> > problem
> >> > > > > > >> > > > > >> > > of packaging and starting these processes is
> >> > > > > > >> > > > > >> > > framework/environment-specific. This leaves
> >> > > > individual
> >> > > > > > >> > > frameworks
> >> > > > > > >> > > > to
> >> > > > > > >> > > > > >> be
> >> > > > > > >> > > > > >> > as
> >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you can
> get
> >> > > simple
> >> > > > > > >> stateless
> >> > > > > > >> > > > > >> support in
> >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf
> app
> >> > > > > framework
> >> > > > > > >> > > (Slider,
> >> > > > > > >> > > > > >> > Marathon,
> >> > > > > > >> > > > > >> > > etc). These are well known by people and
> have
> >> > nice
> >> > > > UIs
> >> > > > > > and a
> >> > > > > > >> > lot
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> > > flexibility. I don't think they have node
> >> > affinity
> >> > > > as a
> >> > > > > > >> built
> >> > > > > > >> > in
> >> > > > > > >> > > > > >> option
> >> > > > > > >> > > > > >> > > (though I could be wrong). So if we want
> that
> >> we
> >> > > can
> >> > > > > > either
> >> > > > > > >> > wait
> >> > > > > > >> > > > for
> >> > > > > > >> > > > > >> them
> >> > > > > > >> > > > > >> > > to add it or do a custom framework to add
> that
> >> > > > feature
> >> > > > > > (as
> >> > > > > > >> > now).
> >> > > > > > >> > > > > >> > Obviously
> >> > > > > > >> > > > > >> > > if you manage things with old-school ops
> tools
> >> > > > > > >> > (puppet/chef/etc)
> >> > > > > > >> > > > you
> >> > > > > > >> > > > > >> get
> >> > > > > > >> > > > > >> > > locality easily. The nice thing, though, is
> >> that
> >> > > all
> >> > > > > the
> >> > > > > > >> samza
> >> > > > > > >> > > > > >> "business
> >> > > > > > >> > > > > >> > > logic" around partition management and fault
> >> > > > tolerance
> >> > > > > > is in
> >> > > > > > >> > > Samza
> >> > > > > > >> > > > > >> core
> >> > > > > > >> > > > > >> > so
> >> > > > > > >> > > > > >> > > it is shared across frameworks and the
> >> framework
> >> > > > > specific
> >> > > > > > >> bit
> >> > > > > > >> > is
> >> > > > > > >> > > > > just
> >> > > > > > >> > > > > >> > > whether it is smart enough to try to get the
> >> same
> >> > > > host
> >> > > > > > when
> >> > > > > > >> a
> >> > > > > > >> > > job
> >> > > > > > >> > > > is
> >> > > > > > >> > > > > >> > > restarted.
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah I
> >> think
> >> > > the
> >> > > > > > goal
> >> > > > > > >> > would
> >> > > > > > >> > > > be
> >> > > > > > >> > > > > >> (a)
> >> > > > > > >> > > > > >> > > actually get better alignment in user
> >> experience,
> >> > > and
> >> > > > > (b)
> >> > > > > > >> > > express
> >> > > > > > >> > > > > >> this in
> >> > > > > > >> > > > > >> > > the naming and project branding.
> Specifically:
> >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> >> > > > > > "transformation"
> >> > > > > > >> api
> >> > > > > > >> > > to
> >> > > > > > >> > > > be
> >> > > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be
> >> able
> >> > > to
> >> > > > > > explain
> >> > > > > > >> > > when
> >> > > > > > >> > > > to
> >> > > > > > >> > > > > >> use
> >> > > > > > >> > > > > >> > > the consumer and when to use the stream
> >> > processing
> >> > > > > > >> > functionality
> >> > > > > > >> > > > and
> >> > > > > > >> > > > > >> lead
> >> > > > > > >> > > > > >> > > people into that experience.
> >> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2
> >> (or
> >> > > > > > whatever)
> >> > > > > > >> > that
> >> > > > > > >> > > > has
> >> > > > > > >> > > > > >> both
> >> > > > > > >> > > > > >> > > Kafka and the stream processing part and
> they
> >> > > > actually
> >> > > > > > work
> >> > > > > > >> > > > > together.
> >> > > > > > >> > > > > >> > > 3. Unify the programming experience so the
> >> client
> >> > > and
> >> > > > > > Samza
> >> > > > > > >> > api
> >> > > > > > >> > > > > share
> >> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> committers
> >> and
> >> > > can
> >> > > > > > have a
> >> > > > > > >> > > > > separate
> >> > > > > > >> > > > > >> > repo,
> >> > > > > > >> > > > > >> > > but I'm actually not really sure (I can't
> >> find a
> >> > > > > > definition
> >> > > > > > >> > of a
> >> > > > > > >> > > > > >> > subproject
> >> > > > > > >> > > > > >> > > in Apache).
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > Basically at a high-level you want the
> >> experience
> >> > > to
> >> > > > > > "feel"
> >> > > > > > >> > > like a
> >> > > > > > >> > > > > >> single
> >> > > > > > >> > > > > >> > > system, not to relatively independent things
> >> that
> >> > > are
> >> > > > > > kind
> >> > > > > > >> of
> >> > > > > > >> > > > > >> awkwardly
> >> > > > > > >> > > > > >> > > glued together.
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > I think if we did that they having naming or
> >> > > branding
> >> > > > > > like
> >> > > > > > >> > > "kafka
> >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or something
> >> like
> >> > > that
> >> > > > > > would
> >> > > > > > >> > > > actually
> >> > > > > > >> > > > > >> do a
> >> > > > > > >> > > > > >> > > good job of conveying what it is. I do that
> >> this
> >> > > > would
> >> > > > > > help
> >> > > > > > >> > > > adoption
> >> > > > > > >> > > > > >> > quite
> >> > > > > > >> > > > > >> > > a lot as it would correctly convey that
> using
> >> > Kafka
> >> > > > > > >> Streaming
> >> > > > > > >> > > with
> >> > > > > > >> > > > > >> Kafka
> >> > > > > > >> > > > > >> > is
> >> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
> >> pretty
> >> > > > > heavily
> >> > > > > > >> > adopted
> >> > > > > > >> > > > at
> >> > > > > > >> > > > > >> this
> >> > > > > > >> > > > > >> > > point.
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> >> originally
> >> > > > when
> >> > > > > > open
> >> > > > > > >> > > > sourcing
> >> > > > > > >> > > > > >> > Samza,
> >> > > > > > >> > > > > >> > > however at that time Kafka was relatively
> >> unknown
> >> > > and
> >> > > > > we
> >> > > > > > >> > decided
> >> > > > > > >> > > > not
> >> > > > > > >> > > > > >> to
> >> > > > > > >> > > > > >> > do
> >> > > > > > >> > > > > >> > > it since we felt it would be limiting. From
> my
> >> > > point
> >> > > > of
> >> > > > > > view
> >> > > > > > >> > the
> >> > > > > > >> > > > > three
> >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now really
> >> > heavily
> >> > > > > used
> >> > > > > > for
> >> > > > > > >> > > > stream
> >> > > > > > >> > > > > >> > > processing, (2) we learned that abstracting
> >> out
> >> > the
> >> > > > > > stream
> >> > > > > > >> > well
> >> > > > > > >> > > is
> >> > > > > > >> > > > > >> > > basically impossible, (3) we learned it is
> >> really
> >> > > > hard
> >> > > > > to
> >> > > > > > >> keep
> >> > > > > > >> > > the
> >> > > > > > >> > > > > two
> >> > > > > > >> > > > > >> > > things feeling like a single product.
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > -Jay
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> >> Kleppmann
> >> > <
> >> > > > > > >> > > > > >> martin@kleppmann.com>
> >> > > > > > >> > > > > >> > > wrote:
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > >> Hi all,
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> I agree with the general philosophy of
> tying
> >> > Samza
> >> > > > > more
> >> > > > > > >> > firmly
> >> > > > > > >> > > to
> >> > > > > > >> > > > > >> Kafka.
> >> > > > > > >> > > > > >> > >> After I spent a while looking at
> integrating
> >> > other
> >> > > > > > message
> >> > > > > > >> > > > brokers
> >> > > > > > >> > > > > >> (e.g.
> >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
> >> > > > conclusion
> >> > > > > > that
> >> > > > > > >> > > > > >> > SystemConsumer
> >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
> Kafka's
> >> > that
> >> > > > > pretty
> >> > > > > > >> much
> >> > > > > > >> > > > > nobody
> >> > > > > > >> > > > > >> but
> >> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
> >> > perhaps
> >> > > an
> >> > > > > > >> > exception,
> >> > > > > > >> > > > but
> >> > > > > > >> > > > > >> it
> >> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.)
> Thus,
> >> > > making
> >> > > > > > Samza
> >> > > > > > >> > > fully
> >> > > > > > >> > > > > >> > dependent
> >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> >> > system-independence
> >> > > > was
> >> > > > > > >> never
> >> > > > > > >> > as
> >> > > > > > >> > > > > real
> >> > > > > > >> > > > > >> as
> >> > > > > > >> > > > > >> > we
> >> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of
> code
> >> > reuse
> >> > > > are
> >> > > > > > >> real.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN has
> >> also
> >> > > > always
> >> > > > > > been
> >> > > > > > >> > > > > >> appealing to
> >> > > > > > >> > > > > >> > >> me, for various reasons already mentioned
> in
> >> > this
> >> > > > > > thread.
> >> > > > > > >> > > > Although
> >> > > > > > >> > > > > >> > making
> >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> >> > > > (YARN/Mesos/AWS/etc)
> >> > > > > > >> seems
> >> > > > > > >> > > > > >> laudable,
> >> > > > > > >> > > > > >> > I am
> >> > > > > > >> > > > > >> > >> a little concerned that it will restrict us
> >> to a
> >> > > > > lowest
> >> > > > > > >> > common
> >> > > > > > >> > > > > >> > denominator.
> >> > > > > > >> > > > > >> > >> For example, would host affinity
> (SAMZA-617)
> >> > still
> >> > > > be
> >> > > > > > >> > possible?
> >> > > > > > >> > > > For
> >> > > > > > >> > > > > >> jobs
> >> > > > > > >> > > > > >> > >> with large amounts of state, I think
> >> SAMZA-617
> >> > > would
> >> > > > > be
> >> > > > > > a
> >> > > > > > >> big
> >> > > > > > >> > > > boon,
> >> > > > > > >> > > > > >> > since
> >> > > > > > >> > > > > >> > >> restoring state off the changelog on every
> >> > single
> >> > > > > > restart
> >> > > > > > >> is
> >> > > > > > >> > > > > painful,
> >> > > > > > >> > > > > >> > due
> >> > > > > > >> > > > > >> > >> to long recovery times. It would be a shame
> >> if
> >> > the
> >> > > > > > >> decoupling
> >> > > > > > >> > > > from
> >> > > > > > >> > > > > >> YARN
> >> > > > > > >> > > > > >> > >> made host affinity impossible.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> Jay, a question about the proposed API for
> >> > > > > > instantiating a
> >> > > > > > >> > job
> >> > > > > > >> > > in
> >> > > > > > >> > > > > >> code
> >> > > > > > >> > > > > >> > >> (rather than a properties file): when
> >> > submitting a
> >> > > > job
> >> > > > > > to a
> >> > > > > > >> > > > > cluster,
> >> > > > > > >> > > > > >> is
> >> > > > > > >> > > > > >> > the
> >> > > > > > >> > > > > >> > >> idea that the instantiation code runs on a
> >> > client
> >> > > > > > >> somewhere,
> >> > > > > > >> > > > which
> >> > > > > > >> > > > > >> then
> >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> >> > > YARN/Mesos/AWS/etc?
> >> > > > > Or
> >> > > > > > >> does
> >> > > > > > >> > > that
> >> > > > > > >> > > > > >> code
> >> > > > > > >> > > > > >> > run
> >> > > > > > >> > > > > >> > >> on each container that is part of the job
> (in
> >> > > which
> >> > > > > > case,
> >> > > > > > >> how
> >> > > > > > >> > > > does
> >> > > > > > >> > > > > >> the
> >> > > > > > >> > > > > >> > job
> >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
> >> right to
> >> > > > make
> >> > > > > a
> >> > > > > > 1.0
> >> > > > > > >> > > > release
> >> > > > > > >> > > > > >> > with a
> >> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete. So
> if
> >> > this
> >> > > > is
> >> > > > > > going
> >> > > > > > >> > to
> >> > > > > > >> > > > > >> happen, I
> >> > > > > > >> > > > > >> > >> think it would be more honest to stick with
> >> 0.*
> >> > > > > version
> >> > > > > > >> > numbers
> >> > > > > > >> > > > > until
> >> > > > > > >> > > > > >> > the
> >> > > > > > >> > > > > >> > >> library-ified Samza has been implemented,
> is
> >> > > stable
> >> > > > > and
> >> > > > > > >> > widely
> >> > > > > > >> > > > > used.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
> >> Kafka?
> >> > > There
> >> > > > > is
> >> > > > > > >> > > precedent
> >> > > > > > >> > > > > for
> >> > > > > > >> > > > > >> > >> tight coupling between different Apache
> >> projects
> >> > > > (e.g.
> >> > > > > > >> > Curator
> >> > > > > > >> > > > and
> >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think
> >> > > remaining
> >> > > > > > >> separate
> >> > > > > > >> > > > would
> >> > > > > > >> > > > > >> be
> >> > > > > > >> > > > > >> > ok.
> >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on Kafka,
> >> there
> >> > > is
> >> > > > > > enough
> >> > > > > > >> > > > > substance
> >> > > > > > >> > > > > >> in
> >> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> >> project.
> >> > > An
> >> > > > > > >> argument
> >> > > > > > >> > in
> >> > > > > > >> > > > > >> favour
> >> > > > > > >> > > > > >> > of
> >> > > > > > >> > > > > >> > >> merging would be if we think Kafka has a
> much
> >> > > > stronger
> >> > > > > > >> "brand
> >> > > > > > >> > > > > >> presence"
> >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If
> >> the
> >> > > Kafka
> >> > > > > > >> project
> >> > > > > > >> > is
> >> > > > > > >> > > > > >> willing
> >> > > > > > >> > > > > >> > to
> >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of
> doing
> >> > > > stateful
> >> > > > > > >> stream
> >> > > > > > >> > > > > >> > >> transformations, that would probably have
> >> much
> >> > the
> >> > > > > same
> >> > > > > > >> > effect
> >> > > > > > >> > > as
> >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> >> Processors"
> >> > or
> >> > > > > > suchlike.
> >> > > > > > >> > > Close
> >> > > > > > >> > > > > >> > >> collaboration between the two projects will
> >> be
> >> > > > needed
> >> > > > > in
> >> > > > > > >> any
> >> > > > > > >> > > > case.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> From a project management perspective, I
> >> guess
> >> > the
> >> > > > > "new
> >> > > > > > >> > Samza"
> >> > > > > > >> > > > > would
> >> > > > > > >> > > > > >> > have
> >> > > > > > >> > > > > >> > >> to be developed on a branch alongside
> ongoing
> >> > > > > > maintenance
> >> > > > > > >> of
> >> > > > > > >> > > the
> >> > > > > > >> > > > > >> current
> >> > > > > > >> > > > > >> > >> line of development? I think it would be
> >> > important
> >> > > > to
> >> > > > > > >> > continue
> >> > > > > > >> > > > > >> > supporting
> >> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> >> migration
> >> > > > path
> >> > > > > to
> >> > > > > > >> the
> >> > > > > > >> > > new
> >> > > > > > >> > > > > >> > version.
> >> > > > > > >> > > > > >> > >> Leaving the current versions unsupported
> and
> >> > > forcing
> >> > > > > > people
> >> > > > > > >> > to
> >> > > > > > >> > > > > >> rewrite
> >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> Best,
> >> > > > > > >> > > > > >> > >> Martin
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> >> > > > jay@confluent.io>
> >> > > > > > >> wrote:
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >>> Hey Garry,
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be
> happy
> >> to
> >> > > chat
> >> > > > > > more
> >> > > > > > >> > about
> >> > > > > > >> > > > > this
> >> > > > > > >> > > > > >> if
> >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
> >> > started
> >> > > > with
> >> > > > > > the
> >> > > > > > >> > idea
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> "what
> >> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> >> > ingestion
> >> > > > > tool"
> >> > > > > > but
> >> > > > > > >> > > > > >> ultimately
> >> > > > > > >> > > > > >> > we
> >> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> >> ingestion
> >> > > and
> >> > > > > > >> > > > transformation
> >> > > > > > >> > > > > >> had
> >> > > > > > >> > > > > >> > >>> pretty different needs and coupling the
> two
> >> > made
> >> > > > > things
> >> > > > > > >> > hard.
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
> (KIP-26)
> >> > > > actually
> >> > > > > > will
> >> > > > > > >> > do
> >> > > > > > >> > > > what
> >> > > > > > >> > > > > >> you
> >> > > > > > >> > > > > >> > >> are
> >> > > > > > >> > > > > >> > >>> looking for.
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>> With regard to your point about slider, I
> >> don't
> >> > > > > > >> necessarily
> >> > > > > > >> > > > > >> disagree.
> >> > > > > > >> > > > > >> > >> But I
> >> > > > > > >> > > > > >> > >>> think getting good YARN support is quite
> >> doable
> >> > > > and I
> >> > > > > > >> think
> >> > > > > > >> > we
> >> > > > > > >> > > > can
> >> > > > > > >> > > > > >> make
> >> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> >> proposal
> >> > > > > solves
> >> > > > > > is
> >> > > > > > >> > that
> >> > > > > > >> > > > > >> > >> technically
> >> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
> >> cluster
> >> > > > > > management
> >> > > > > > >> > > systems
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> > way
> >> > > > > > >> > > > > >> > >>> things are now, you need to write an "app
> >> > master"
> >> > > > or
> >> > > > > > >> > > "framework"
> >> > > > > > >> > > > > for
> >> > > > > > >> > > > > >> > each
> >> > > > > > >> > > > > >> > >>> and they are all a little different so
> >> testing
> >> > is
> >> > > > > > really
> >> > > > > > >> > hard.
> >> > > > > > >> > > > In
> >> > > > > > >> > > > > >> the
> >> > > > > > >> > > > > >> > >>> absence of this we have been stuck with
> just
> >> > YARN
> >> > > > > which
> >> > > > > > >> has
> >> > > > > > >> > > > > >> fantastic
> >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the
> org,
> >> but
> >> > > > zero
> >> > > > > > >> > > penetration
> >> > > > > > >> > > > > >> > >> elsewhere.
> >> > > > > > >> > > > > >> > >>> Given the huge amount of work being put in
> >> to
> >> > > > slider,
> >> > > > > > >> > > marathon,
> >> > > > > > >> > > > > aws
> >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
> related
> >> > > > packaging
> >> > > > > > >> > > > technologies
> >> > > > > > >> > > > > >> > people
> >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> >> > > > > cloud-specific
> >> > > > > > >> > deploy
> >> > > > > > >> > > > > >> tools,
> >> > > > > > >> > > > > >> > >> etc)
> >> > > > > > >> > > > > >> > >>> I really think it is important to get this
> >> > right.
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>> -Jay
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> >> > Turkington
> >> > > <
> >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>>> Hi all,
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> I think the question below re does Samza
> >> > become
> >> > > a
> >> > > > > > >> > sub-project
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> Kafka
> >> > > > > > >> > > > > >> > >>>> highlights the broader point around
> >> migration.
> >> > > > Chris
> >> > > > > > >> > mentions
> >> > > > > > >> > > > > >> Samza's
> >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1 release
> >> but
> >> > I'm
> >> > > > not
> >> > > > > > sure
> >> > > > > > >> > it
> >> > > > > > >> > > > > feels
> >> > > > > > >> > > > > >> > >> right to
> >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> >> deprecate
> >> > > > most
> >> > > > > of
> >> > > > > > >> it.
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have some
> guys
> >> > who
> >> > > > have
> >> > > > > > >> > started
> >> > > > > > >> > > > > >> working
> >> > > > > > >> > > > > >> > >> with
> >> > > > > > >> > > > > >> > >>>> Samza and building some new
> >> > consumers/producers
> >> > > > was
> >> > > > > > next
> >> > > > > > >> > up.
> >> > > > > > >> > > > > Sounds
> >> > > > > > >> > > > > >> > like
> >> > > > > > >> > > > > >> > >>>> that is absolutely not the direction to
> >> go. I
> >> > > need
> >> > > > > to
> >> > > > > > >> look
> >> > > > > > >> > > into
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> > KIP
> >> > > > > > >> > > > > >> > >> in
> >> > > > > > >> > > > > >> > >>>> more detail but for me the attractiveness
> >> of
> >> > > > adding
> >> > > > > > new
> >> > > > > > >> > Samza
> >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all
> they
> >> > were
> >> > > > > doing
> >> > > > > > was
> >> > > > > > >> > > > really
> >> > > > > > >> > > > > >> > getting
> >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to
> avoid
> >> > > > having
> >> > > > > to
> >> > > > > > >> > worry
> >> > > > > > >> > > > > about
> >> > > > > > >> > > > > >> the
> >> > > > > > >> > > > > >> > >>>> lifecycle management of external clients.
> >> If
> >> > > there
> >> > > > > is
> >> > > > > > a
> >> > > > > > >> > > generic
> >> > > > > > >> > > > > >> Kafka
> >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a
> new
> >> > > > connector
> >> > > > > > into
> >> > > > > > >> > and
> >> > > > > > >> > > > > have
> >> > > > > > >> > > > > >> a
> >> > > > > > >> > > > > >> > >> lot of
> >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> reliability
> >> > done
> >> > > > for
> >> > > > > me
> >> > > > > > >> then
> >> > > > > > >> > > it
> >> > > > > > >> > > > > >> gives
> >> > > > > > >> > > > > >> > me
> >> > > > > > >> > > > > >> > >> all
> >> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers
> would.
> >> If
> >> > > not
> >> > > > > > then it
> >> > > > > > >> > > > > >> complicates
> >> > > > > > >> > > > > >> > my
> >> > > > > > >> > > > > >> > >>>> operational deployments.
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> Which is similar to my other question
> with
> >> the
> >> > > > > > proposal
> >> > > > > > >> --
> >> > > > > > >> > if
> >> > > > > > >> > > > we
> >> > > > > > >> > > > > >> > build a
> >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus
> the
> >> > > > requisite
> >> > > > > > >> shims
> >> > > > > > >> > to
> >> > > > > > >> > > > > >> > integrate
> >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former may
> >> be a
> >> > > lot
> >> > > > > more
> >> > > > > > >> work
> >> > > > > > >> > > > than
> >> > > > > > >> > > > > we
> >> > > > > > >> > > > > >> > >> think.
> >> > > > > > >> > > > > >> > >>>> We may make it much easier for a newcomer
> >> to
> >> > get
> >> > > > > > >> something
> >> > > > > > >> > > > > running
> >> > > > > > >> > > > > >> but
> >> > > > > > >> > > > > >> > >>>> having them step up and get a reliable
> >> > > production
> >> > > > > > >> > deployment
> >> > > > > > >> > > > may
> >> > > > > > >> > > > > >> still
> >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
> >> > different
> >> > > > > > reasons
> >> > > > > > >> > than
> >> > > > > > >> > > > > >> today.
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable
> with
> >> > > making
> >> > > > > the
> >> > > > > > >> Samza
> >> > > > > > >> > > > > >> dependency
> >> > > > > > >> > > > > >> > >> on
> >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I absolutely
> >> see
> >> > > the
> >> > > > > > >> benefits
> >> > > > > > >> > > in
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> >> > > > > > >> > > > terminologies/abstractions
> >> > > > > > >> > > > > >> that
> >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library
> >> would
> >> > > > likely
> >> > > > > > be a
> >> > > > > > >> > very
> >> > > > > > >> > > > > nice
> >> > > > > > >> > > > > >> > tool
> >> > > > > > >> > > > > >> > >> to
> >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have
> the
> >> > > > concerns
> >> > > > > > >> above
> >> > > > > > >> > re
> >> > > > > > >> > > > the
> >> > > > > > >> > > > > >> > >>>> operational side.
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> Garry
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> >> [mailto:
> >> > > > > > >> > gdfm@apache.org
> >> > > > > > >> > > ]
> >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on
> >> > Samza
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> >> > > > > > >> > > > > >> > >>>> From outside, I have always perceived
> Samza
> >> > as a
> >> > > > > > >> computing
> >> > > > > > >> > > > layer
> >> > > > > > >> > > > > >> over
> >> > > > > > >> > > > > >> > >>>> Kafka.
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> The question, maybe a bit provocative, is
> >> > > "should
> >> > > > > > Samza
> >> > > > > > >> be
> >> > > > > > >> > a
> >> > > > > > >> > > > > >> > sub-project
> >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> >> separate
> >> > > > > project
> >> > > > > > >> > with a
> >> > > > > > >> > > > > >> separate
> >> > > > > > >> > > > > >> > >>>> governance?
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> Cheers,
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> --
> >> > > > > > >> > > > > >> > >>>> Gianmarco
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> >> > > > > > yanfang724@gmail.com>
> >> > > > > > >> > > > wrote:
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka
> more
> >> > > > tightly.
> >> > > > > > >> > Because
> >> > > > > > >> > > > > Samza
> >> > > > > > >> > > > > >> de
> >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
> >> > leverage
> >> > > > > what
> >> > > > > > >> Kafka
> >> > > > > > >> > > > has.
> >> > > > > > >> > > > > At
> >> > > > > > >> > > > > >> > the
> >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
> reinvent
> >> > what
> >> > > > > Samza
> >> > > > > > >> > > already
> >> > > > > > >> > > > > >> has. I
> >> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> >> > ingestion
> >> > > > and
> >> > > > > > >> > > > > transformation.
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
> >> image
> >> > > how
> >> > > > > the
> >> > > > > > >> Samza
> >> > > > > > >> > > > will
> >> > > > > > >> > > > > >> look
> >> > > > > > >> > > > > >> > >>>> like.
> >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> >> > > difference
> >> > > > > in
> >> > > > > > >> terms
> >> > > > > > >> > > of
> >> > > > > > >> > > > > how
> >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code
> >> shows
> >> > (A
> >> > > > > > client of
> >> > > > > > >> > > > Kakfa)
> >> > > > > > >> > > > > ?
> >> > > > > > >> > > > > >> And
> >> > > > > > >> > > > > >> > >>>>> user's application code calls this
> client?
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of
> Kafka
> >> > (like
> >> > > > > what
> >> > > > > > the
> >> > > > > > >> > > code
> >> > > > > > >> > > > > >> shows),
> >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> >> > > > > fault-tolerance?
> >> > > > > > >> Are
> >> > > > > > >> > > they
> >> > > > > > >> > > > > >> taken
> >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> >> mechanism,
> >> > > such
> >> > > > > as
> >> > > > > > >> > "Samza
> >> > > > > > >> > > > > >> worker"
> >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> 2. What about other features, such as
> >> > > > auto-scaling,
> >> > > > > > >> shared
> >> > > > > > >> > > > > state,
> >> > > > > > >> > > > > >> > >>>>> monitoring?
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is
> this
> >> > what
> >> > > > > Chris
> >> > > > > > >> > > > suggests?)
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from
> Kakfa
> >> > and
> >> > > > > > produce
> >> > > > > > >> to
> >> > > > > > >> > > it.
> >> > > > > > >> > > > > >> Then it
> >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks
> like
> >> > now,
> >> > > > > > except it
> >> > > > > > >> > > does
> >> > > > > > >> > > > > not
> >> > > > > > >> > > > > >> > rely
> >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> >> leverage
> >> > > > Kafka's
> >> > > > > > >> > metrics,
> >> > > > > > >> > > > > logs,
> >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> Thanks,
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang
> >> > Wang <
> >> > > > > > >> > > > > wangguoz@gmail.com
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > >>>> wrote:
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>>> Read through the code example and it
> >> looks
> >> > > good
> >> > > > to
> >> > > > > > me.
> >> > > > > > >> A
> >> > > > > > >> > > few
> >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> >> runnable
> >> > > like:
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> >> > > --config-factory=...
> >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> deploying
> >> > Samza
> >> > > > > more
> >> > > > > > as
> >> > > > > > >> > > > embedded
> >> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> >> (ignoring
> >> > > the
> >> > > > > > >> > > terminology
> >> > > > > > >> > > > > >> since
> >> > > > > > >> > > > > >> > >>>>>> it is not the
> >> > > > > > >> > > > > >> > >>>>> same
> >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> >> MyStreamTask(configs);
> >> > > > > Thread
> >> > > > > > >> > thread
> >> > > > > > >> > > =
> >> > > > > > >> > > > > new
> >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment modes
> >> are
> >> > > > > important
> >> > > > > > >> for
> >> > > > > > >> > > > > >> different
> >> > > > > > >> > > > > >> > >>>>>> types
> >> > > > > > >> > > > > >> > >>>>> of
> >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making Samza
> >> > purely
> >> > > > > > >> standalone
> >> > > > > > >> > is
> >> > > > > > >> > > > > still
> >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or
> library
> >> > > modes.
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> Guozhang
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
> >> Kreps
> >> > <
> >> > > > > > >> > > > jay@confluent.io>
> >> > > > > > >> > > > > >> > wrote:
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> >> example,
> >> > it
> >> > > > was
> >> > > > > > >> > supposed
> >> > > > > > >> > > > to
> >> > > > > > >> > > > > >> look
> >> > > > > > >> > > > > >> > >>>>>>> like
> >> > > > > > >> > > > > >> > >>>>>>> this:
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> >> > > > "localhost:4242");
> >> > > > > > >> > > > > >> StreamingConfig
> >> > > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> >> > > > "test-topic-2");
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > config.processor(ExampleStreamProcessor.class);
> >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> >> > StringSerializer(),
> >> > > > new
> >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming
> >> > > > container =
> >> > > > > > new
> >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> container.run();
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>> -Jay
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay
> >> > Kreps <
> >> > > > > > >> > > > jay@confluent.io
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > >> > >>>> wrote:
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
> >> Chris
> >> > > and
> >> > > > I
> >> > > > > > were
> >> > > > > > >> > > having
> >> > > > > > >> > > > > >> > >>>>>>>> around
> >> > > > > > >> > > > > >> > >>>>>>> whether
> >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a
> >> kind
> >> > > of
> >> > > > > data
> >> > > > > > >> > > > ingestion
> >> > > > > > >> > > > > >> > >>>>> framework
> >> > > > > > >> > > > > >> > >>>>>>> for
> >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to
> KIP-26
> >> > > > > "copycat").
> >> > > > > > >> This
> >> > > > > > >> > > > kind
> >> > > > > > >> > > > > of
> >> > > > > > >> > > > > >> > >>>>>> combined
> >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and
> YARN
> >> and
> >> > > the
> >> > > > > > >> > discussion
> >> > > > > > >> > > > > >> around
> >> > > > > > >> > > > > >> > >>>>>>>> how
> >> > > > > > >> > > > > >> > >>>>> to
> >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was, given
> >> that
> >> > > > Samza
> >> > > > > > was
> >> > > > > > >> > > > basically
> >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific, what
> if
> >> > you
> >> > > > just
> >> > > > > > >> > embraced
> >> > > > > > >> > > > > that
> >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> >> > > > > > >> > > > > >> > >>>>>> into
> >> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> >> > framework
> >> > > > and
> >> > > > > > more
> >> > > > > > >> > > like a
> >> > > > > > >> > > > > >> > >>>>>>>> third
> >> > > > > > >> > > > > >> > >>>>> Kafka
> >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> consumer"
> >> > with
> >> > > > > state
> >> > > > > > >> > > > management
> >> > > > > > >> > > > > >> > >>>>>> facilities.
> >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> >> complex
> >> > > > stream
> >> > > > > > >> > > processing
> >> > > > > > >> > > > > >> > >>>>>>>> framework
> >> > > > > > >> > > > > >> > >>>>>>> this
> >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple
> thing,
> >> not
> >> > > > much
> >> > > > > > more
> >> > > > > > >> > > > > >> complicated
> >> > > > > > >> > > > > >> > >>>>>>>> to
> >> > > > > > >> > > > > >> > >>>>> use
> >> > > > > > >> > > > > >> > >>>>>>> or
> >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As
> Chris
> >> > said
> >> > > > we
> >> > > > > > >> thought
> >> > > > > > >> > > > about
> >> > > > > > >> > > > > >> it
> >> > > > > > >> > > > > >> > >>>>>>>> a
> >> > > > > > >> > > > > >> > >>>>> lot
> >> > > > > > >> > > > > >> > >>>>>> of
> >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> >> > processing
> >> > > > > > systems
> >> > > > > > >> > were
> >> > > > > > >> > > > > doing)
> >> > > > > > >> > > > > >> > >>>>> seemed
> >> > > > > > >> > > > > >> > >>>>>>> like
> >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output
> >> data
> >> > to
> >> > > > and
> >> > > > > > from
> >> > > > > > >> > the
> >> > > > > > >> > > > > stream
> >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually
> looked
> >> > into
> >> > > > how
> >> > > > > > that
> >> > > > > > >> > > would
> >> > > > > > >> > > > > >> > >>>>>>>> work,
> >> > > > > > >> > > > > >> > >>>>> Samza
> >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion
> >> > > framework
> >> > > > > > for a
> >> > > > > > >> > > bunch
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> > >>>>> reasons.
> >> > > > > > >> > > > > >> > >>>>>> To
> >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a
> pretty
> >> > > > different
> >> > > > > > >> > internal
> >> > > > > > >> > > > > data
> >> > > > > > >> > > > > >> > >>>>>>>> model
> >> > > > > > >> > > > > >> > >>>>>> and
> >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split
> them
> >> and
> >> > > had
> >> > > > > an
> >> > > > > > api
> >> > > > > > >> > for
> >> > > > > > >> > > > > Kafka
> >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26)
> >> and a
> >> > > > > separate
> >> > > > > > >> api
> >> > > > > > >> > > for
> >> > > > > > >> > > > > >> Kafka
> >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> embracing
> >> the
> >> > > > same
> >> > > > > > >> > > terminology
> >> > > > > > >> > > > > and
> >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about the
> >> > current
> >> > > > > > state is
> >> > > > > > >> > > that
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> > >>>>>>>> two
> >> > > > > > >> > > > > >> > >>>>>>> systems
> >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology
> >> like
> >> > > > > "stream"
> >> > > > > > vs
> >> > > > > > >> > > > "topic"
> >> > > > > > >> > > > > >> and
> >> > > > > > >> > > > > >> > >>>>>>> different
> >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means
> you
> >> > kind
> >> > > > of
> >> > > > > > have
> >> > > > > > >> to
> >> > > > > > >> > > > learn
> >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> >> > > > > > >> > > > > >> > >>>>>>> way,
> >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly different
> >> way,
> >> > > > then
> >> > > > > > kind
> >> > > > > > >> of
> >> > > > > > >> > > > > >> > >>>>>>>> understand
> >> > > > > > >> > > > > >> > >>>>> how
> >> > > > > > >> > > > > >> > >>>>>>> they
> >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having
> walked
> >> a
> >> > few
> >> > > > > > people
> >> > > > > > >> > > through
> >> > > > > > >> > > > > >> this
> >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to
> >> get.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of
> >> time
> >> > on
> >> > > > > > >> airplanes I
> >> > > > > > >> > > > > hacked
> >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> >> incomplete
> >> > > > > > prototype
> >> > > > > > >> of
> >> > > > > > >> > > > what
> >> > > > > > >> > > > > >> > >>>>>>>> this would
> >> > > > > > >> > > > > >> > >>>>> look
> >> > > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
> >> dumped
> >> > > into
> >> > > > > > Kafka
> >> > > > > > >> as
> >> > > > > > >> > > it
> >> > > > > > >> > > > > >> > >>>>>>>> required a
> >> > > > > > >> > > > > >> > >>>>>> few
> >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is
> >> the
> >> > > code:
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > >
> >> > > > > > >> >
> >> > > > > >
> >> > >
> >> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I
> just
> >> > > > > liberally
> >> > > > > > >> > renamed
> >> > > > > > >> > > > > >> > >>>>>>>> everything
> >> > > > > > >> > > > > >> > >>>>> to
> >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
> >> regard
> >> > > for
> >> > > > > > >> > > > compatibility.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something like
> >> this:
> >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new Properties();
> >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> >> > > > > "localhost:4242");
> >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> >> > > > > > >> > > > > >> config.processor(ExampleStreamProcessor.class);
> >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> >> > > StringDeserializer());
> >> > > > > > >> > > > KafkaStreaming
> >> > > > > > >> > > > > >> > >>>>>> container =
> >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> >> > container.run();
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> >> > > > SamzaContainer;
> >> > > > > > >> > > > > StreamProcessor
> >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the class
> >> names
> >> > > in
> >> > > > a
> >> > > > > > file
> >> > > > > > >> > and
> >> > > > > > >> > > > then
> >> > > > > > >> > > > > >> > >>>>>>>> having
> >> > > > > > >> > > > > >> > >>>>>> the
> >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
> >> > > > > instantiate
> >> > > > > > the
> >> > > > > > >> > > > > container
> >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced
> over
> >> > > > however
> >> > > > > > many
> >> > > > > > >> > > > > instances
> >> > > > > > >> > > > > >> > >>>>>>>> of
> >> > > > > > >> > > > > >> > >>>>> this
> >> > > > > > >> > > > > >> > >>>>>>> are
> >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> instance
> >> > dies,
> >> > > > new
> >> > > > > > >> tasks
> >> > > > > > >> > > are
> >> > > > > > >> > > > > >> added
> >> > > > > > >> > > > > >> > >>>>>>>> to
> >> > > > > > >> > > > > >> > >>>>> the
> >> > > > > > >> > > > > >> > >>>>>>>> existing containers without shutting
> >> them
> >> > > > down).
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
> running
> >> > this
> >> > > > > stuff
> >> > > > > > in
> >> > > > > > >> > YARN
> >> > > > > > >> > > > via
> >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
> >> using
> >> > > some
> >> > > > > of
> >> > > > > > >> their
> >> > > > > > >> > > > tools
> >> > > > > > >> > > > > >> > >>>>>>>> but from the
> >> > > > > > >> > > > > >> > >>>>>> point
> >> > > > > > >> > > > > >> > >>>>>>> of
> >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these stream
> >> > > > processing
> >> > > > > > jobs
> >> > > > > > >> > are
> >> > > > > > >> > > > > just
> >> > > > > > >> > > > > >> > >>>>>> stateless
> >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
> >> expand
> >> > and
> >> > > > > > contract
> >> > > > > > >> > at
> >> > > > > > >> > > > > will.
> >> > > > > > >> > > > > >> > >>>>>>>> There
> >> > > > > > >> > > > > >> > >>>>> is
> >> > > > > > >> > > > > >> > >>>>>>> no
> >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code,
> it
> >> > would
> >> > > > get
> >> > > > > > >> larger
> >> > > > > > >> > > if
> >> > > > > > >> > > > we
> >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
> larger.
> >> We
> >> > > > really
> >> > > > > > do
> >> > > > > > >> > get a
> >> > > > > > >> > > > ton
> >> > > > > > >> > > > > >> > >>>>>>>> of
> >> > > > > > >> > > > > >> > >>>>>>> leverage
> >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> >> > delegated
> >> > > to
> >> > > > > the
> >> > > > > > >> new
> >> > > > > > >> > > > > >> consumer.
> >> > > > > > >> > > > > >> > >>>>> This
> >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> >> > management
> >> > > > > > strategy
> >> > > > > > >> > > > > available
> >> > > > > > >> > > > > >> > >>>>>>>> to
> >> > > > > > >> > > > > >> > >>>>>> Kafka
> >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to Samza
> >> (and
> >> > > vice
> >> > > > > > versa)
> >> > > > > > >> > and
> >> > > > > > >> > > > > with
> >> > > > > > >> > > > > >> > >>>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>>> exact
> >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as
> state
> >> > reuse
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
> >> > thought
> >> > > > > > >> provoking.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM,
> Chris
> >> > > > > Riccomini <
> >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with
> Samza
> >> > > > > engineers
> >> > > > > > at
> >> > > > > > >> > > > LinkedIn
> >> > > > > > >> > > > > >> > >>>>>>>>> and
> >> > > > > > >> > > > > >> > >>>>>>> Confluent
> >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> observations
> >> > and
> >> > > > > would
> >> > > > > > >> like
> >> > > > > > >> > to
> >> > > > > > >> > > > > >> > >>>>>>>>> propose
> >> > > > > > >> > > > > >> > >>>>> some
> >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I
> >> want to
> >> > > > call
> >> > > > > > out
> >> > > > > > >> > about
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> >> > > > > > >> > > > > >> > >>>>>> design,
> >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> changes.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic
> >> > > > deployment
> >> > > > > > >> system.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> >> SystemConsumer/SystemProducer
> >> > and
> >> > > > > > Kafka's
> >> > > > > > >> > > > consumer
> >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> >> > > > > > >> > > > > >> > >>>>> are
> >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
> >> > problems.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
> related,
> >> > but
> >> > > > I'll
> >> > > > > > >> > address
> >> > > > > > >> > > > them
> >> > > > > > >> > > > > >> in
> >> > > > > > >> > > > > >> > >>>>> order.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use
> of a
> >> > > > dynamic
> >> > > > > > >> > > deployment
> >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> >> > > > > > >> > > > > >> > >>>>>> such
> >> > > > > > >> > > > > >> > >>>>>>>>> as
> >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially
> >> built
> >> > > > > Samza,
> >> > > > > > we
> >> > > > > > >> > bet
> >> > > > > > >> > > > that
> >> > > > > > >> > > > > >> > >>>>>>>>> there
> >> > > > > > >> > > > > >> > >>>>>> would
> >> > > > > > >> > > > > >> > >>>>>>>>> be
> >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area, and
> >> we
> >> > > could
> >> > > > > > >> support
> >> > > > > > >> > > > them,
> >> > > > > > >> > > > > >> and
> >> > > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>> rest
> >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there are
> >> many
> >> > > > > > >> variations.
> >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> >> > > > > > >> > > > > >> > >>>>>> many
> >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start
> >> their
> >> > > > > > processors
> >> > > > > > >> > like
> >> > > > > > >> > > > > normal
> >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use traditional
> >> > > > deployment
> >> > > > > > >> scripts
> >> > > > > > >> > > > such
> >> > > > > > >> > > > > as
> >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> >> > > > > > >> > > > > >> > >>>>>> Chef,
> >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
> >> system
> >> > > on
> >> > > > > > users
> >> > > > > > >> > makes
> >> > > > > > >> > > > the
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
> painful
> >> for
> >> > > > first
> >> > > > > > time
> >> > > > > > >> > > > users.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement
> >> was
> >> > > also
> >> > > > a
> >> > > > > > bit
> >> > > > > > >> of
> >> > > > > > >> > a
> >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> >> > > > > > >> > > > > >> > >>>>>> because
> >> > > > > > >> > > > > >> > >>>>>>>>> of
> >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding
> between
> >> > the
> >> > > > > > nature of
> >> > > > > > >> > > batch
> >> > > > > > >> > > > > >> jobs
> >> > > > > > >> > > > > >> > >>>>>>>>> and
> >> > > > > > >> > > > > >> > >>>>>>> stream
> >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> >> > > conscious
> >> > > > > > effort
> >> > > > > > >> to
> >> > > > > > >> > > > favor
> >> > > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>> Hadoop
> >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things,
> >> since
> >> > it
> >> > > > > worked
> >> > > > > > >> and
> >> > > > > > >> > > was
> >> > > > > > >> > > > > well
> >> > > > > > >> > > > > >> > >>>>>>> understood.
> >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that
> >> batch
> >> > > jobs
> >> > > > > > have a
> >> > > > > > >> > > > definite
> >> > > > > > >> > > > > >> > >>>>>> beginning,
> >> > > > > > >> > > > > >> > >>>>>>>>> and
> >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs
> don't
> >> > > > > (usually).
> >> > > > > > >> This
> >> > > > > > >> > > > leads
> >> > > > > > >> > > > > to
> >> > > > > > >> > > > > >> > >>>>>>>>> a
> >> > > > > > >> > > > > >> > >>>>> much
> >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for
> stream
> >> > > > > processors.
> >> > > > > > >> You
> >> > > > > > >> > > > > >> basically
> >> > > > > > >> > > > > >> > >>>>>>>>> just
> >> > > > > > >> > > > > >> > >>>>>>> need
> >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> >> processor,
> >> > and
> >> > > > > start
> >> > > > > > >> it.
> >> > > > > > >> > > The
> >> > > > > > >> > > > > way
> >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's
> no
> >> > > concept
> >> > > > > of
> >> > > > > > a
> >> > > > > > >> > > cluster
> >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> >> > > > > > >> > > > > >> > >>>>>> add
> >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> >> coupling
> >> > > > Samza
> >> > > > > > with
> >> > > > > > >> a
> >> > > > > > >> > > > > >> scheduler
> >> > > > > > >> > > > > >> > >>>>>>>>> is
> >> > > > > > >> > > > > >> > >>>>>> that
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
> >> handle
> >> > > > > > deployment.
> >> > > > > > >> > > This
> >> > > > > > >> > > > > >> pulls
> >> > > > > > >> > > > > >> > >>>>>>>>> in a
> >> > > > > > >> > > > > >> > >>>>>>> bunch
> >> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> >> > > distribution
> >> > > > > > (config
> >> > > > > > >> > > > > stream),
> >> > > > > > >> > > > > >> > >>>>>>>>> shell
> >> > > > > > >> > > > > >> > >>>>>>> scrips
> >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> packaging
> >> > (all
> >> > > > the
> >> > > > > > .tgz
> >> > > > > > >> > > > stuff),
> >> > > > > > >> > > > > >> etc.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
> >> > > > deployment
> >> > > > > > was
> >> > > > > > >> to
> >> > > > > > >> > > > > support
> >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> >> > > locality,
> >> > > > > you
> >> > > > > > >> need
> >> > > > > > >> > to
> >> > > > > > >> > > > put
> >> > > > > > >> > > > > >> > >>>>>>>>> your
> >> > > > > > >> > > > > >> > >>>>>> processors
> >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> processing.
> >> > Upon
> >> > > > > > further
> >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> >> > > > > > >> > > > > >> > >>>>>>> though,
> >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that beneficial.
> >> > There
> >> > > is
> >> > > > > > some
> >> > > > > > >> > good
> >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> >> > > > > > >> > > > > >> > >>>>>> about
> >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335.
> >> > Again,
> >> > > we
> >> > > > > > took
> >> > > > > > >> the
> >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> >> > > > > > >> > > > > >> > >>>>>> path,
> >> > > > > > >> > > > > >> > >>>>>>>>> but
> >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> differences
> >> > > > between
> >> > > > > > HDFS
> >> > > > > > >> > and
> >> > > > > > >> > > > > Kafka.
> >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> >> > > > > > >> > > > > >> > >>>>>> has
> >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions.
> >> This
> >> > > > leads
> >> > > > > to
> >> > > > > > >> less
> >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
> >> > > processors
> >> > > > > on
> >> > > > > > top
> >> > > > > > >> > of
> >> > > > > > >> > > > > Kafka.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a
> crutch.
> >> > > Samza
> >> > > > > > doesn't
> >> > > > > > >> > > have
> >> > > > > > >> > > > > any
> >> > > > > > >> > > > > >> > >>>>>>>>> built
> >> > > > > > >> > > > > >> > >>>>> in
> >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
> >> > depends
> >> > > on
> >> > > > > the
> >> > > > > > >> > > dynamic
> >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to
> handle
> >> > > > restarts
> >> > > > > > >> when a
> >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> >> > > > > > >> > > > > >> > >>>>>>> made
> >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> >> standalone
> >> > > Samza
> >> > > > > > >> > container
> >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is good,
> >> but I
> >> > > > think
> >> > > > > > that
> >> > > > > > >> > > we've
> >> > > > > > >> > > > > >> gone
> >> > > > > > >> > > > > >> > >>>>>>>>> too
> >> > > > > > >> > > > > >> > >>>>>> far
> >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> >> > > > (SystemConsumer,
> >> > > > > > >> > > > > SystemProducer,
> >> > > > > > >> > > > > >> > >>>> etc).
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just
> about
> >> > every
> >> > > > > > >> component
> >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> >> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> >> > > ConfigRewriter,
> >> > > > > > etc).
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> >> > forgotten,
> >> > > as
> >> > > > > > well.
> >> > > > > > >> > Some
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> > >>>>>>>>> these
> >> > > > > > >> > > > > >> > >>>>> are
> >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not to
> >> be.
> >> > > This
> >> > > > > all
> >> > > > > > >> comes
> >> > > > > > >> > > at
> >> > > > > > >> > > > a
> >> > > > > > >> > > > > >> cost:
> >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is
> making
> >> it
> >> > > > harder
> >> > > > > > for
> >> > > > > > >> > our
> >> > > > > > >> > > > > users
> >> > > > > > >> > > > > >> > >>>>>>>>> to
> >> > > > > > >> > > > > >> > >>>>> pick
> >> > > > > > >> > > > > >> > >>>>>> up
> >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It
> also
> >> > makes
> >> > > > it
> >> > > > > > >> > difficult
> >> > > > > > >> > > > for
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about
> what
> >> the
> >> > > > > > >> > > characteristics
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> >> characteristics
> >> > > > change
> >> > > > > > >> > > depending
> >> > > > > > >> > > > on
> >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are
> most
> >> > > visible
> >> > > > > in
> >> > > > > > the
> >> > > > > > >> > > > System
> >> > > > > > >> > > > > >> APIs.
> >> > > > > > >> > > > > >> > >>>>> What
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
> >> functional is
> >> > > > Kafka
> >> > > > > > as
> >> > > > > > >> its
> >> > > > > > >> > > > > >> > >>>>>>>>> transport
> >> > > > > > >> > > > > >> > >>>>>> layer.
> >> > > > > > >> > > > > >> > >>>>>>>>> But
> >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use
> >> cases
> >> > > into
> >> > > > > one
> >> > > > > > >> API:
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports both
> >> of
> >> > > these
> >> > > > > use
> >> > > > > > >> > cases.
> >> > > > > > >> > > > The
> >> > > > > > >> > > > > >> > >>>>>>>>> problem
> >> > > > > > >> > > > > >> > >>>>>> is,
> >> > > > > > >> > > > > >> > >>>>>>>>> we
> >> > > > > > >> > > > > >> > >>>>>>>>> actually want different features for
> >> each
> >> > > use
> >> > > > > > case.
> >> > > > > > >> By
> >> > > > > > >> > > > > >> papering
> >> > > > > > >> > > > > >> > >>>>>>>>> over
> >> > > > > > >> > > > > >> > >>>>>>> these
> >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a
> single
> >> > API,
> >> > > > > we've
> >> > > > > > >> > > > introduced
> >> > > > > > >> > > > > a
> >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> >> > > > > > >> > > > > >> > >>>>>>> leaky
> >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like
> in
> >> (2)
> >> > > is
> >> > > > to
> >> > > > > > have
> >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
> >> > offsets
> >> > > > > (like
> >> > > > > > >> > Kafka).
> >> > > > > > >> > > > > This
> >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> >> > > > > > >> > > > > >> > >>>>> with
> >> > > > > > >> > > > > >> > >>>>>>> (1),
> >> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems have
> >> > > > different
> >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> >> mailing
> >> > > list
> >> > > > > and
> >> > > > > > >> the
> >> > > > > > >> > > SQL
> >> > > > > > >> > > > > >> JIRAs
> >> > > > > > >> > > > > >> > >>>>> about
> >> > > > > > >> > > > > >> > >>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> >> > > replayability.
> >> > > > > > Kafka
> >> > > > > > >> > > allows
> >> > > > > > >> > > > us
> >> > > > > > >> > > > > >> to
> >> > > > > > >> > > > > >> > >>>>> rewind
> >> > > > > > >> > > > > >> > >>>>>>>>> when
> >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other
> systems
> >> > > don't.
> >> > > > In
> >> > > > > > some
> >> > > > > > >> > > > cases,
> >> > > > > > >> > > > > >> > >>>>>>>>> systems
> >> > > > > > >> > > > > >> > >>>>>>> return
> >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> >> > > > > > >> WikipediaSystemConsumer)
> >> > > > > > >> > > > > because
> >> > > > > > >> > > > > >> > >>>>>>>>> they
> >> > > > > > >> > > > > >> > >>>>>> have
> >> > > > > > >> > > > > >> > >>>>>>> no
> >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example.
> Kafka
> >> > > > supports
> >> > > > > > >> > > > > partitioning,
> >> > > > > > >> > > > > >> > >>>>>>>>> but
> >> > > > > > >> > > > > >> > >>>>> many
> >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
> >> having a
> >> > > > single
> >> > > > > > >> > > partition
> >> > > > > > >> > > > > for
> >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other systems
> >> model
> >> > > > > > >> partitioning
> >> > > > > > >> > > > > >> > >>>> differently (e.g.
> >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a
> >> mess.
> >> > > > > > Creating
> >> > > > > > >> > > streams
> >> > > > > > >> > > > > in
> >> > > > > > >> > > > > >> a
> >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> >> impossible.
> >> > > As
> >> > > > is
> >> > > > > > >> > modeling
> >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> >> > > > > > >> > > > > >> > >>>>> for
> >> > > > > > >> > > > > >> > >>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> >> partitions,
> >> > > > > location,
> >> > > > > > >> > etc).
> >> > > > > > >> > > > The
> >> > > > > > >> > > > > >> > >>>>>>>>> list
> >> > > > > > >> > > > > >> > >>>>> goes
> >> > > > > > >> > > > > >> > >>>>>>> on.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing
> >> Samza,
> >> > > > > Kafka's
> >> > > > > > >> > > consumer
> >> > > > > > >> > > > > and
> >> > > > > > >> > > > > >> > >>>>> producer
> >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set.
> On
> >> the
> >> > > > > > >> > consumer-side,
> >> > > > > > >> > > > you
> >> > > > > > >> > > > > >> > >>>>>>>>> had two
> >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
> consumer,
> >> or
> >> > > the
> >> > > > > > simple
> >> > > > > > >> > > > > consumer.
> >> > > > > > >> > > > > >> > >>>>>>>>> The
> >> > > > > > >> > > > > >> > >>>>>>> problem
> >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was
> that
> >> it
> >> > > > > > controlled
> >> > > > > > >> > your
> >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and
> >> the
> >> > > order
> >> > > > > in
> >> > > > > > >> which
> >> > > > > > >> > > you
> >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> >> > > > > > >> > > > > >> > >>>>> problem
> >> > > > > > >> > > > > >> > >>>>>>>>> with
> >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not
> >> > > simple.
> >> > > > > It's
> >> > > > > > >> > basic.
> >> > > > > > >> > > > You
> >> > > > > > >> > > > > >> > >>>>>>>>> end up
> >> > > > > > >> > > > > >> > >>>>>>> having
> >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level
> >> stuff
> >> > > > that
> >> > > > > > you
> >> > > > > > >> > > > > shouldn't.
> >> > > > > > >> > > > > >> > >>>>>>>>> We
> >> > > > > > >> > > > > >> > >>>>>> spent a
> >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> >> > > > KafkaSystemConsumer
> >> > > > > > very
> >> > > > > > >> > > > robust.
> >> > > > > > >> > > > > >> It
> >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some cool
> >> > > features:
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
> >> > > > > > prioritization.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> >> assignment
> >> > > to
> >> > > > > > support
> >> > > > > > >> > > > joins,
> >> > > > > > >> > > > > >> > >>>>>>>>> global
> >> > > > > > >> > > > > >> > >>>>>> state
> >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)),
> etc.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> >> > checkpointing.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time
> is
> >> > that
> >> > > > > these
> >> > > > > > >> > > features
> >> > > > > > >> > > > > >> > >>>>>>>>> should
> >> > > > > > >> > > > > >> > >>>>>>> actually
> >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
> consumers
> >> > (not
> >> > > > just
> >> > > > > > >> Samza
> >> > > > > > >> > > > stream
> >> > > > > > >> > > > > >> > >>>>>> processors)
> >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like
> joins
> >> > and
> >> > > > > > partition
> >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> >> > > > > > >> > > > > >> > >>>>>>> Kafka
> >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> >> > conclusion.
> >> > > > > > They're
> >> > > > > > >> > > adding
> >> > > > > > >> > > > a
> >> > > > > > >> > > > > >> ton
> >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> >> consumer
> >> > > > > > >> > > implementation.
> >> > > > > > >> > > > > To a
> >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> >> > > > > > >> > > > > >> > >>>>> it's
> >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've already
> >> done
> >> > > in
> >> > > > > > Samza.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up
> taking
> >> a
> >> > > very
> >> > > > > > similar
> >> > > > > > >> > > > > approach
> >> > > > > > >> > > > > >> > >>>>>>>>> to
> >> > > > > > >> > > > > >> > >>>>>> Samza's
> >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> implementation
> >> for
> >> > > > > > handling
> >> > > > > > >> > > offset
> >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> >> management
> >> > > > > feature
> >> > > > > > >> > stores
> >> > > > > > >> > > > > >> offset
> >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows
> >> you to
> >> > > > fetch
> >> > > > > > them
> >> > > > > > >> > > from
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste,
> >> since
> >> > we
> >> > > > > could
> >> > > > > > >> have
> >> > > > > > >> > > > shared
> >> > > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > > >> > > > > >> > >>>>> work
> >> > > > > > >> > > > > >> > >>>>>> if
> >> > > > > > >> > > > > >> > >>>>>>>>> it
> >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
> >> get-go.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> >> radical
> >> > > > > > proposal.
> >> > > > > > >> > Samza
> >> > > > > > >> > > > is
> >> > > > > > >> > > > > >> > >>>>> relatively
> >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to
> >> say
> >> > > that
> >> > > > > > we're
> >> > > > > > >> > > near a
> >> > > > > > >> > > > > 1.0
> >> > > > > > >> > > > > >> > >>>>>> release.
> >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what
> >> we've
> >> > > > > learned,
> >> > > > > > and
> >> > > > > > >> > > begin
> >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> >> > > > > > >> > > > > >> > >>>>>>> about
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we
> >> change if
> >> > > we
> >> > > > > were
> >> > > > > > >> > > starting
> >> > > > > > >> > > > > >> from
> >> > > > > > >> > > > > >> > >>>>>> scratch?
> >> > > > > > >> > > > > >> > >>>>>>>>> My
> >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only*
> >> way
> >> > to
> >> > > > run
> >> > > > > > Samza
> >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
> >> > > > > dependences
> >> > > > > > on
> >> > > > > > >> > > YARN,
> >> > > > > > >> > > > > >> Mesos,
> >> > > > > > >> > > > > >> > >>>> etc.
> >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support
> >> only
> >> > > > Kafka
> >> > > > > > as
> >> > > > > > >> the
> >> > > > > > >> > > > > stream
> >> > > > > > >> > > > > >> > >>>>>> processing
> >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics,
> logging,
> >> > > > > > >> serialization,
> >> > > > > > >> > > and
> >> > > > > > >> > > > > >> > >>>>>>>>> config
> >> > > > > > >> > > > > >> > >>>>>>> systems,
> >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues
> that
> >> I
> >> > > > > outlined
> >> > > > > > >> > above.
> >> > > > > > >> > > It
> >> > > > > > >> > > > > >> > >>>>>>>>> should
> >> > > > > > >> > > > > >> > >>>>> also
> >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> >> > > > dramatically.
> >> > > > > > >> > > Supporting
> >> > > > > > >> > > > > >> only
> >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow
> >> Samza
> >> > to
> >> > > be
> >> > > > > > >> executed
> >> > > > > > >> > > on
> >> > > > > > >> > > > > YARN
> >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> >> > > > Marathon/Aurora),
> >> > > > > or
> >> > > > > > >> most
> >> > > > > > >> > > > other
> >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> >> > > > > > >> > > > > >> > >>>>>>> deployment
> >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a lot
> >> > easier
> >> > > > for
> >> > > > > > new
> >> > > > > > >> > > users.
> >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> >> > > > > > >> > > > > >> > >>>>>>> having
> >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without
> YARN.
> >> > The
> >> > > > drop
> >> > > > > > in
> >> > > > > > >> > > mailing
> >> > > > > > >> > > > > >> list
> >> > > > > > >> > > > > >> > >>>>>> traffic
> >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
> >> overdue to
> >> > > me.
> >> > > > > The
> >> > > > > > >> > > reality
> >> > > > > > >> > > > > is,
> >> > > > > > >> > > > > >> > >>>>> everyone
> >> > > > > > >> > > > > >> > >>>>>>>>> that
> >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
> >> Kafka.
> >> > We
> >> > > > > > basically
> >> > > > > > >> > > > require
> >> > > > > > >> > > > > >> it
> >> > > > > > >> > > > > >> > >>>>>> already
> >> > > > > > >> > > > > >> > >>>>>>> in
> >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work.
> Those
> >> > that
> >> > > > are
> >> > > > > > >> using
> >> > > > > > >> > > > other
> >> > > > > > >> > > > > >> > >>>>>>>>> systems
> >> > > > > > >> > > > > >> > >>>>>> are
> >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into
> >> Kafka
> >> > > (1),
> >> > > > > and
> >> > > > > > >> then
> >> > > > > > >> > > > they
> >> > > > > > >> > > > > do
> >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
> >> already
> >> > > > > > discussion (
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > >
> >> > > > > > >> >
> >> > > > > >
> >> > >
> >> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >> > > > > > >> > > > > >> > >>>>> 767
> >> > > > > > >> > > > > >> > >>>>>>>>> )
> >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into
> Kafka
> >> > > > extremely
> >> > > > > > >> easy.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple with
> >> > Kafka,
> >> > > > we
> >> > > > > > can
> >> > > > > > >> > > > leverage
> >> > > > > > >> > > > > a
> >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> >> > > > > > >> > > > > >> > >>>>>>> their
> >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> >> maintain
> >> > > our
> >> > > > > own
> >> > > > > > >> > config,
> >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> >> > > > > > >> > > > > >> > >>>>> etc.
> >> > > > > > >> > > > > >> > >>>>>>> We
> >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries,
> and
> >> > make
> >> > > > them
> >> > > > > > >> > better.
> >> > > > > > >> > > > This
> >> > > > > > >> > > > > >> > >>>>>>>>> will
> >> > > > > > >> > > > > >> >
> >> ...
> >>
> >> [Message clipped]
> >
> >
> >
>



-- 
Jordan Shaw
Full Stack Software Engineer
PubNub Inc
1045 17th St
San Francisco, CA 94107

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@gmail.com>.
Hmm, thought about this more. Maybe this is just too much too quick.
Overall I think there is some enthusiasm for the proposal but it's not
really unanimous enough to make any kind of change this big cleanly. The
board doesn't really like the merging stuff, user's are concerned about
compatibility, I didn't feel there was unanimous agreement on dropping
SystemConsumer, etc. Even if this is the right end state to get to,
probably trying to push all this through at once isn't the right way to do
it.

So let me propose a kind of fifth (?) option which I think is less dramatic
and let's things happen gradually. I think this is kind of like combining
the first part of Yi's proposal and Jakob's third option, leaving the rest
to be figured out incrementally:

Option 5: We continue the prototype I shared and propose that as a kind of
"transformer" client API in Kafka. This isn't really a full-fledged stream
processing layer, more like a supped up consumer api for munging topics.
This would let us figure out some of the technical bits, how to do this on
Kafka's group management features, how to integrate the txn feature to do
the exactly-once stuff in these transformations, and get all this stuff
solid. This api would have valid uses in it's own right, especially when
your transformation will be embedded inside an existing service or
application which isn't possible with Samza (or other existing systems that
I know of).

Independently we can iterate on some of the ideas of the original proposal
individually and figure out how (if at all) to make use of this
functionality. This can be done bit-by-bit:
- Could be that the existing StreamTask API ends up wrapping this
- Could end up exposed directly in Samza as Yi proposed
- Could be that just the lower-level group-management stuff get's used, and
in this case it could be either just for standalone mode, or always
- Could be that it stays as-is

The advantage of this is it is lower risk...we basically don't have to make
12 major decisions all at once that kind of hinge on what amounts to a
pretty aggressive rewrite. The disadvantage of this is it is a bit more
confusing as all this is getting figured out.

As with some of the other stuff, this would require a further discussion in
the Kafka community if people do like this approach.

Thoughts?

-Jay




On Sun, Jul 12, 2015 at 10:52 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Chris,
>
> Yeah, I'm obviously in favor of this.
>
> The sub-project approach seems the ideal way to take a graceful step in
> this direction, so I will ping the board folks and see why they are
> discouraged, it would be good to understand that. If we go that route we
> would need to do a similar discussion in the Kafka list (but makes sense to
> figure out first if it is what Samza wants).
>
> Irrespective of how it's implemented, though, to me the important things
> are the following:
> 1. Unify the website, config, naming, docs, metrics, etc--basically fix
> the product experience so the "stream" and the "processing" feel like a
> single user experience and brand. This seems minor but I think is a really
> big deal.
> 2. Make "standalone" mode a first class citizen and have a real technical
> plan to be able to support cluster managers other than YARN.
> 3. Make the config and out-of-the-box experience more usable
>
> I think that prototype gives a practical example of how 1-3 could be done
> and we should pursue it. This is a pretty radical change, so I wouldn't be
> shocked if people didn't want to take a step like that.
>
> Maybe it would make sense to see if people are on board with that general
> idea, and then try to get some advice on sub-projects in parallel and nail
> down those details?
>
> -Jay
>
> On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
>> Hey all,
>>
>> I want to start by saying that I'm absolutely thrilled to be a part of
>> this
>> community. The amount of level-headed, thoughtful, educated discussion
>> that's gone on over the past ~10 days is overwhelming. Wonderful.
>>
>> It seems like discussion is waning a bit, and we've reached some
>> conclusions. There are several key emails in this threat, which I want to
>> call out:
>>
>> 1. Jakob's summary of the three potential ways forward.
>>
>>
>> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
>> 2. Julian's call out that we should be focusing on community over code.
>>
>>
>> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
>> 3. Martin's summary about the benefits of merging communities.
>>
>>
>> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
>> 4. Jakob's comments about the distinction between community and code
>> paths.
>>
>>
>> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
>>
>> I agree with the comments on all of these emails. I think Martin's summary
>> of his position aligns very closely with my own. To that end, I think we
>> should get concrete about what the proposal is, and call a vote on it.
>> Given that Jay, Martin, and I seem to be aligning fairly closely, I think
>> we should start with:
>>
>> 1. [community] Make Samza a subproject of Kafka.
>> 2. [community] Make all Samza PMC/committers committers of the subproject.
>> 3. [community] Migrate Samza's website/documentation into Kafka's.
>> 4. [code] Have the Samza community and the Kafka community start a
>> from-scratch reboot together in the new Kafka subproject. We can
>> borrow/copy &  paste significant chunks of code from Samza's code base.
>> 5. [code] The subproject would intentionally eliminate support for both
>> other streaming systems and all deployment systems.
>> 6. [code] Attempt to provide a bridge from our SystemConsumer to KIP-26
>> (copy cat)
>> 7. [code] Attempt to provide a bridge from the new subproject's processor
>> interface to our legacy StreamTask interface.
>> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
>> subproject that has a fault-tolerant container with state management.
>>
>> It's likely that (6) and (7) won't be fully drop-in. Still, the closer we
>> can get, the better it's going to be for our existing community.
>>
>> One thing that I didn't touch on with (2) is whether any Samza PMC members
>> should be rolled into Kafka PMC membership as well (though, Jay and Jakob
>> are already PMC members on both). I think that Samza's community deserves
>> a
>> voice on the PMC, so I'd propose that we roll at least a few PMC members
>> into the Kafka PMC, but I don't have a strong framework for which people
>> to
>> pick.
>>
>> Before (8), I think that Samza's TLP can continue to commit bug fixes and
>> patches as it sees fit, provided that we openly communicate that we won't
>> necessarily migrate new features to the new subproject, and that the TLP
>> will be shut down after the migration to the Kafka subproject occurs.
>>
>> Jakob, I could use your guidance here about about how to achieve this from
>> an Apache process perspective (sorry).
>>
>> * Should I just call a vote on this proposal?
>> * Should it happen on dev or private?
>> * Do committers have binding votes, or just PMC?
>>
>> Having trouble finding much detail on the Apache wikis. :(
>>
>> Cheers,
>> Chris
>>
>> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com> wrote:
>>
>> > Thanks, Jay. This argument persuaded me actually. :)
>> >
>> > Fang, Yan
>> > yanfang724@gmail.com
>> >
>> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io> wrote:
>> >
>> > > Hey Yan,
>> > >
>> > > Yeah philosophically I think the argument is that you should capture
>> the
>> > > stream in Kafka independent of the transformation. This is obviously a
>> > > Kafka-centric view point.
>> > >
>> > > Advantages of this:
>> > > - In practice I think this is what e.g. Storm people often end up
>> doing
>> > > anyway. You usually need to throttle any access to a live serving
>> > database.
>> > > - Can have multiple subscribers and they get the same thing without
>> > > additional load on the source system.
>> > > - Applications can tap into the stream if need be by subscribing.
>> > > - You can debug your transformation by tailing the Kafka topic with
>> the
>> > > console consumer
>> > > - Can tee off the same data stream for batch analysis or Lambda arch
>> > style
>> > > re-processing
>> > >
>> > > The disadvantage is that it will use Kafka resources. But the idea is
>> > > eventually you will have multiple subscribers to any data source (at
>> > least
>> > > for monitoring) so you will end up there soon enough anyway.
>> > >
>> > > Down the road the technical benefit is that I think it gives us a good
>> > path
>> > > towards end-to-end exactly once semantics from source to destination.
>> > > Basically the connectors need to support idempotence when talking to
>> > Kafka
>> > > and we need the transactional write feature in Kafka to make the
>> > > transformation atomic. This is actually pretty doable if you separate
>> > > connector=>kafka problem from the generic transformations which are
>> > always
>> > > kafka=>kafka. However I think it is quite impossible to do in a
>> > all_things
>> > > => all_things environment. Today you can say "well the semantics of
>> the
>> > > Samza APIs depend on the connectors you use" but it is actually worse
>> > then
>> > > that because the semantics actually depend on the pairing of
>> > connectors--so
>> > > not only can you probably not get a usable "exactly once" guarantee
>> > > end-to-end it can actually be quite hard to reverse engineer what
>> > property
>> > > (if any) your end-to-end flow has if you have heterogenous systems.
>> > >
>> > > -Jay
>> > >
>> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
>> wrote:
>> > >
>> > > > {quote}
>> > > > maintained in a separate repository and retaining the existing
>> > > > committership but sharing as much else as possible (website, etc)
>> > > > {quote}
>> > > >
>> > > > Overall, I agree on this idea. Now the question is more about "how
>> to
>> > do
>> > > > it".
>> > > >
>> > > > On the other hand, one thing I want to point out is that, if we
>> decide
>> > to
>> > > > go this way, how do we want to support
>> > > > otherSystem-transformation-otherSystem use case?
>> > > >
>> > > > Basically, there are four user groups here:
>> > > >
>> > > > 1. Kafka-transformation-Kafka
>> > > > 2. Kafka-transformation-otherSystem
>> > > > 3. otherSystem-transformation-Kafka
>> > > > 4. otherSystem-transformation-otherSystem
>> > > >
>> > > > For group 1, they can easily use the new Samza library to achieve.
>> For
>> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka or
>> > Kafka->
>> > > > transformation -> copyCat.
>> > > >
>> > > > The problem is for group 4. Do we want to abandon this or still
>> support
>> > > it?
>> > > > Of course, this use case can be achieved by using copyCat ->
>> > > transformation
>> > > > -> Kafka -> transformation -> copyCat, the thing is how we persuade
>> > them
>> > > to
>> > > > do this long chain. If yes, it will also be a win for Kafka too. Or
>> if
>> > > > there is no one in this community actually doing this so far, maybe
>> ok
>> > to
>> > > > not support the group 4 directly.
>> > > >
>> > > > Thanks,
>> > > >
>> > > > Fang, Yan
>> > > > yanfang724@gmail.com
>> > > >
>> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
>> wrote:
>> > > >
>> > > > > Yeah I agree with this summary. I think there are kind of two
>> > questions
>> > > > > here:
>> > > > > 1. Technically does alignment/reliance on Kafka make sense
>> > > > > 2. Branding wise (naming, website, concepts, etc) does alignment
>> with
>> > > > Kafka
>> > > > > make sense
>> > > > >
>> > > > > Personally I do think both of these things would be really
>> valuable,
>> > > and
>> > > > > would dramatically alter the trajectory of the project.
>> > > > >
>> > > > > My preference would be to see if people can mostly agree on a
>> > direction
>> > > > > rather than splintering things off. From my point of view the
>> ideal
>> > > > outcome
>> > > > > of all the options discussed would be to make Samza a closely
>> aligned
>> > > > > subproject, maintained in a separate repository and retaining the
>> > > > existing
>> > > > > committership but sharing as much else as possible (website,
>> etc). No
>> > > > idea
>> > > > > about how these things work, Jacob, you probably know more.
>> > > > >
>> > > > > No discussion amongst the Kafka folks has happened on this, but
>> > likely
>> > > we
>> > > > > should figure out what the Samza community actually wants first.
>> > > > >
>> > > > > I admit that this is a fairly radical departure from how things
>> are.
>> > > > >
>> > > > > If that doesn't fly, I think, yeah we could leave Samza as it is
>> and
>> > do
>> > > > the
>> > > > > more radical reboot inside Kafka. From my point of view that does
>> > leave
>> > > > > things in a somewhat confusing state since now there are two
>> stream
>> > > > > processing systems more or less coupled to Kafka in large part
>> made
>> > by
>> > > > the
>> > > > > same people. But, arguably that might be a cleaner way to make the
>> > > > cut-over
>> > > > > and perhaps less risky for Samza community since if it works
>> people
>> > can
>> > > > > switch and if it doesn't nothing will have changed. Dunno, how do
>> > > people
>> > > > > feel about this?
>> > > > >
>> > > > > -Jay
>> > > > >
>> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com>
>> > > wrote:
>> > > > >
>> > > > > > >  This leads me to thinking that merging projects and
>> communities
>> > > > might
>> > > > > > be a good idea: with the union of experience from both
>> communities,
>> > > we
>> > > > > will
>> > > > > > probably build a better system that is better for users.
>> > > > > > Is this what's being proposed though? Merging the projects seems
>> > like
>> > > > > > a consequence of at most one of the three directions under
>> > > discussion:
>> > > > > > 1) Samza 2.0: The Samza community relies more heavily on Kafka
>> for
>> > > > > > configuration, etc. (to a greater or lesser extent to be
>> > determined)
>> > > > > > but the Samza community would not automatically merge withe
>> Kafka
>> > > > > > community (the Phoenix/HBase example is a good one here).
>> > > > > > 2) Samza Reboot: The Samza community continues to exist with a
>> > > limited
>> > > > > > project scope, but similarly would not need to be part of the
>> Kafka
>> > > > > > community (ie given committership) to progress.  Here, maybe the
>> > > Samza
>> > > > > > team would become a subproject of Kafka (the Board frowns on
>> > > > > > subprojects at the moment, so I'm not sure if that's even
>> > feasible),
>> > > > > > but that would not be required.
>> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option the
>> > Kafka
>> > > > > > team builds its own streaming library, possibly off of Jay's
>> > > > > > prototype, which has not direct lineage to the Samza team.
>> There's
>> > > no
>> > > > > > reason for the Kafka team to bring in the Samza team.
>> > > > > >
>> > > > > > Is the Kafka community on board with this?
>> > > > > >
>> > > > > > To be clear, all three options under discussion are interesting,
>> > > > > > technically valid and likely healthy directions for the project.
>> > > > > > Also, they are not mutually exclusive.  The Samza community
>> could
>> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka community
>> went
>> > > > > > forward with 'Hey Samza!'  My points above are directed
>> entirely at
>> > > > > > the community aspect of these choices.
>> > > > > > -Jakob
>> > > > > >
>> > > > > > On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com>
>> > > wrote:
>> > > > > > > That's great.  Thanks, Jay.
>> > > > > > >
>> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io>
>> > > wrote:
>> > > > > > >
>> > > > > > >> Yeah totally agree. I think you have this issue even today,
>> > right?
>> > > > > I.e.
>> > > > > > if
>> > > > > > >> you need to make a simple config change and you're running in
>> > YARN
>> > > > > today
>> > > > > > >> you end up bouncing the job which then rebuilds state. I
>> think
>> > the
>> > > > fix
>> > > > > > is
>> > > > > > >> exactly what you described which is to have a long timeout on
>> > > > > partition
>> > > > > > >> movement for stateful jobs so that if a job is just getting
>> > > bounced,
>> > > > > and
>> > > > > > >> the cluster manager (or admin) is smart enough to restart it
>> on
>> > > the
>> > > > > same
>> > > > > > >> host when possible, it can optimistically reuse any existing
>> > state
>> > > > it
>> > > > > > finds
>> > > > > > >> on disk (if it is valid).
>> > > > > > >>
>> > > > > > >> So in this model the charter of the CM is to place processes
>> as
>> > > > > > stickily as
>> > > > > > >> possible and to restart or re-place failed processes. The
>> > charter
>> > > of
>> > > > > the
>> > > > > > >> partition management system is to control the assignment of
>> work
>> > > to
>> > > > > > these
>> > > > > > >> processes. The nice thing about this is that the work
>> > assignment,
>> > > > > > timeouts,
>> > > > > > >> behavior, configs, and code will all be the same across all
>> > > cluster
>> > > > > > >> managers.
>> > > > > > >>
>> > > > > > >> So I think that prototype would actually give you exactly
>> what
>> > you
>> > > > > want
>> > > > > > >> today for any cluster manager (or manual placement + restart
>> > > script)
>> > > > > > that
>> > > > > > >> was sticky in terms of host placement since there is already
>> a
>> > > > > > configurable
>> > > > > > >> partition movement timeout and task-by-task state reuse with
>> a
>> > > check
>> > > > > on
>> > > > > > >> state validity.
>> > > > > > >>
>> > > > > > >> -Jay
>> > > > > > >>
>> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
>> > > > roger.hoover@gmail.com
>> > > > > >
>> > > > > > >> wrote:
>> > > > > > >>
>> > > > > > >> > That would be great to let Kafka do as much heavy lifting
>> as
>> > > > > possible
>> > > > > > and
>> > > > > > >> > make it easier for other languages to implement Samza apis.
>> > > > > > >> >
>> > > > > > >> > One thing to watch out for is the interplay between Kafka's
>> > > group
>> > > > > > >> > management and the external scheduler/process manager's
>> fault
>> > > > > > tolerance.
>> > > > > > >> > If a container dies, the Kafka group membership protocol
>> will
>> > > try
>> > > > to
>> > > > > > >> assign
>> > > > > > >> > it's tasks to other containers while at the same time the
>> > > process
>> > > > > > manager
>> > > > > > >> > is trying to relaunch the container.  Without some
>> > consideration
>> > > > for
>> > > > > > this
>> > > > > > >> > (like a configurable amount of time to wait before Kafka
>> > alters
>> > > > the
>> > > > > > group
>> > > > > > >> > membership), there may be thrashing going on which is
>> > especially
>> > > > bad
>> > > > > > for
>> > > > > > >> > containers with large amounts of local state.
>> > > > > > >> >
>> > > > > > >> > Someone else pointed this out already but I thought it
>> might
>> > be
>> > > > > worth
>> > > > > > >> > calling out again.
>> > > > > > >> >
>> > > > > > >> > Cheers,
>> > > > > > >> >
>> > > > > > >> > Roger
>> > > > > > >> >
>> > > > > > >> >
>> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
>> jay@confluent.io>
>> > > > > wrote:
>> > > > > > >> >
>> > > > > > >> > > Hey Roger,
>> > > > > > >> > >
>> > > > > > >> > > I couldn't agree more. We spent a bunch of time talking
>> to
>> > > > people
>> > > > > > and
>> > > > > > >> > that
>> > > > > > >> > > is exactly the stuff we heard time and again. What makes
>> it
>> > > > hard,
>> > > > > of
>> > > > > > >> > > course, is that there is some tension between
>> compatibility
>> > > with
>> > > > > > what's
>> > > > > > >> > > there now and making things better for new users.
>> > > > > > >> > >
>> > > > > > >> > > I also strongly agree with the importance of
>> multi-language
>> > > > > > support. We
>> > > > > > >> > are
>> > > > > > >> > > talking now about Java, but for application development
>> use
>> > > > cases
>> > > > > > >> people
>> > > > > > >> > > want to work in whatever language they are using
>> elsewhere.
>> > I
>> > > > > think
>> > > > > > >> > moving
>> > > > > > >> > > to a model where Kafka itself does the group membership,
>> > > > lifecycle
>> > > > > > >> > control,
>> > > > > > >> > > and partition assignment has the advantage of putting all
>> > that
>> > > > > > complex
>> > > > > > >> > > stuff behind a clean api that the clients are already
>> going
>> > to
>> > > > be
>> > > > > > >> > > implementing for their consumer, so the added
>> functionality
>> > > for
>> > > > > > stream
>> > > > > > >> > > processing beyond a consumer becomes very minor.
>> > > > > > >> > >
>> > > > > > >> > > -Jay
>> > > > > > >> > >
>> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
>> > > > > > roger.hoover@gmail.com>
>> > > > > > >> > > wrote:
>> > > > > > >> > >
>> > > > > > >> > > > Metamorphosis...nice. :)
>> > > > > > >> > > >
>> > > > > > >> > > > This has been a great discussion.  As a user of Samza
>> > who's
>> > > > > > recently
>> > > > > > >> > > > integrated it into a relatively large organization, I
>> just
>> > > > want
>> > > > > to
>> > > > > > >> add
>> > > > > > >> > > > support to a few points already made.
>> > > > > > >> > > >
>> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
>> currently
>> > > > exists
>> > > > > > that
>> > > > > > >> > I've
>> > > > > > >> > > > experienced are:
>> > > > > > >> > > > 1) YARN - YARN is overly complex in many environments
>> > where
>> > > > > Puppet
>> > > > > > >> > would
>> > > > > > >> > > do
>> > > > > > >> > > > just fine but it was the only mechanism to get fault
>> > > > tolerance.
>> > > > > > >> > > > 2) Configuration - I think I like the idea of
>> configuring
>> > > most
>> > > > > of
>> > > > > > the
>> > > > > > >> > job
>> > > > > > >> > > > in code rather than config files.  In general, I think
>> the
>> > > > goal
>> > > > > > >> should
>> > > > > > >> > be
>> > > > > > >> > > > to make it harder to make mistakes, especially of the
>> kind
>> > > > where
>> > > > > > the
>> > > > > > >> > code
>> > > > > > >> > > > expects something and the config doesn't match.  The
>> > current
>> > > > > > config
>> > > > > > >> is
>> > > > > > >> > > > quite intricate and error-prone.  For example, the
>> > > application
>> > > > > > logic
>> > > > > > >> > may
>> > > > > > >> > > > depend on bootstrapping a topic but rather than
>> asserting
>> > > that
>> > > > > in
>> > > > > > the
>> > > > > > >> > > code,
>> > > > > > >> > > > you have to rely on getting the config right.  Likewise
>> > with
>> > > > > > serdes,
>> > > > > > >> > the
>> > > > > > >> > > > Java representations produced by various serdes (JSON,
>> > Avro,
>> > > > > etc.)
>> > > > > > >> are
>> > > > > > >> > > not
>> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
>> without
>> > > > > changing
>> > > > > > >> the
>> > > > > > >> > > > code.   It would be nice for jobs to be able to assert
>> > what
>> > > > they
>> > > > > > >> expect
>> > > > > > >> > > > from their input topics in terms of partitioning.
>> This is
>> > > > > > getting a
>> > > > > > >> > > little
>> > > > > > >> > > > off topic but I was even thinking about creating a
>> "Samza
>> > > > config
>> > > > > > >> > linter"
>> > > > > > >> > > > that would sanity check a set of configs.  Especially
>> in
>> > > > > > >> organizations
>> > > > > > >> > > > where config is managed by a different team than the
>> > > > application
>> > > > > > >> > > developer,
>> > > > > > >> > > > it's very hard to get avoid config mistakes.
>> > > > > > >> > > > 3) Java/Scala centric - for many teams (especially
>> > > DevOps-type
>> > > > > > >> folks),
>> > > > > > >> > > the
>> > > > > > >> > > > pain of the Java toolchain (maven, slow builds, weak
>> > command
>> > > > > line
>> > > > > > >> > > support,
>> > > > > > >> > > > configuration over convention) really inhibits
>> > productivity.
>> > > > As
>> > > > > > more
>> > > > > > >> > and
>> > > > > > >> > > > more high-quality clients become available for Kafka, I
>> > hope
>> > > > > > they'll
>> > > > > > >> > > follow
>> > > > > > >> > > > Samza's model.  Not sure how much it affects the
>> proposals
>> > > in
>> > > > > this
>> > > > > > >> > thread
>> > > > > > >> > > > but please consider other languages in the ecosystem as
>> > > well.
>> > > > > > From
>> > > > > > >> > what
>> > > > > > >> > > > I've heard, Spark has more Python users than
>> Java/Scala.
>> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
>> > > > > > >> > > >
>> > > > > > >> > > >
>> > > > > > >> > >
>> > > > > > >> >
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
>> > > > > > >> > > > and are working on a Yeoman generator
>> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
>> > > Jython/Samza
>> > > > > > >> projects
>> > > > > > >> > to
>> > > > > > >> > > > alleviate some of the pain)
>> > > > > > >> > > >
>> > > > > > >> > > > I also want to underscore Jay's point about improving
>> the
>> > > user
>> > > > > > >> > > experience.
>> > > > > > >> > > > That's a very important factor for adoption.  I think
>> the
>> > > goal
>> > > > > > should
>> > > > > > >> > be
>> > > > > > >> > > to
>> > > > > > >> > > > make Samza as easy to get started with as something
>> like
>> > > > > Logstash.
>> > > > > > >> > > > Logstash is vastly inferior in terms of capabilities to
>> > > Samza
>> > > > > but
>> > > > > > >> it's
>> > > > > > >> > > easy
>> > > > > > >> > > > to get started and that makes a big difference.
>> > > > > > >> > > >
>> > > > > > >> > > > Cheers,
>> > > > > > >> > > >
>> > > > > > >> > > > Roger
>> > > > > > >> > > >
>> > > > > > >> > > >
>> > > > > > >> > > >
>> > > > > > >> > > >
>> > > > > > >> > > >
>> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci
>> > > > Morales <
>> > > > > > >> > > > gdfm@apache.org> wrote:
>> > > > > > >> > > >
>> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
>> Metamorphosis
>> > > is
>> > > > a
>> > > > > > clear
>> > > > > > >> > > > winner
>> > > > > > >> > > > > :)
>> > > > > > >> > > > >
>> > > > > > >> > > > > --
>> > > > > > >> > > > > Gianmarco
>> > > > > > >> > > > >
>> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
>> Morales
>> > <
>> > > > > > >> > > gdfm@apache.org
>> > > > > > >> > > > >
>> > > > > > >> > > > > wrote:
>> > > > > > >> > > > >
>> > > > > > >> > > > > > Hi,
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > @Martin, thanks for you comments.
>> > > > > > >> > > > > > Maybe I'm missing some important point, but I think
>> > > > coupling
>> > > > > > the
>> > > > > > >> > > > releases
>> > > > > > >> > > > > > is actually a *good* thing.
>> > > > > > >> > > > > > To make an example, would it be better if the MR
>> and
>> > > HDFS
>> > > > > > >> > components
>> > > > > > >> > > of
>> > > > > > >> > > > > > Hadoop had different release schedules?
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > Actually, keeping the discussion in a single place
>> > would
>> > > > > make
>> > > > > > >> > > agreeing
>> > > > > > >> > > > on
>> > > > > > >> > > > > > releases (and backwards compatibility) much
>> easier, as
>> > > > > > everybody
>> > > > > > >> > > would
>> > > > > > >> > > > be
>> > > > > > >> > > > > > responsible for the whole codebase.
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > That said, I like the idea of absorbing samza-core
>> as
>> > a
>> > > > > > >> > sub-project,
>> > > > > > >> > > > and
>> > > > > > >> > > > > > leave the fancy stuff separate.
>> > > > > > >> > > > > > It probably gives 90% of the benefits we have been
>> > > > > discussing
>> > > > > > >> here.
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > Cheers,
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > --
>> > > > > > >> > > > > > Gianmarco
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
>> > jay.kreps@gmail.com
>> > > >
>> > > > > > wrote:
>> > > > > > >> > > > > >
>> > > > > > >> > > > > >> Hey Martin,
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> I agree coupling release schedules is a downside.
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> Definitely we can try to solve some of the
>> > integration
>> > > > > > problems
>> > > > > > >> in
>> > > > > > >> > > > > >> Confluent Platform or in other distributions. But
>> I
>> > > think
>> > > > > > this
>> > > > > > >> > ends
>> > > > > > >> > > up
>> > > > > > >> > > > > >> being really shallow. I guess I feel to really
>> get a
>> > > good
>> > > > > > user
>> > > > > > >> > > > > experience
>> > > > > > >> > > > > >> the two systems have to kind of feel like part of
>> the
>> > > > same
>> > > > > > thing
>> > > > > > >> > and
>> > > > > > >> > > > you
>> > > > > > >> > > > > >> can't really add that in later--you can put both
>> in
>> > the
>> > > > > same
>> > > > > > >> > > > > downloadable
>> > > > > > >> > > > > >> tar file but it doesn't really give a very
>> cohesive
>> > > > > feeling.
>> > > > > > I
>> > > > > > >> > agree
>> > > > > > >> > > > > that
>> > > > > > >> > > > > >> ultimately any of the project stuff is as much
>> social
>> > > and
>> > > > > > naming
>> > > > > > >> > as
>> > > > > > >> > > > > >> anything else--theoretically two totally
>> independent
>> > > > > projects
>> > > > > > >> > could
>> > > > > > >> > > > work
>> > > > > > >> > > > > >> to
>> > > > > > >> > > > > >> tightly align. In practice this seems to be quite
>> > > > difficult
>> > > > > > >> > though.
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> For the frameworks--totally agree it would be
>> good to
>> > > > > > maintain
>> > > > > > >> the
>> > > > > > >> > > > > >> framework support with the project. In some cases
>> > there
>> > > > may
>> > > > > > not
>> > > > > > >> be
>> > > > > > >> > > too
>> > > > > > >> > > > > >> much
>> > > > > > >> > > > > >> there since the integration gets lighter but I
>> think
>> > > > > whatever
>> > > > > > >> > stubs
>> > > > > > >> > > > you
>> > > > > > >> > > > > >> need should be included. So no I definitely wasn't
>> > > trying
>> > > > > to
>> > > > > > >> imply
>> > > > > > >> > > > > >> dropping
>> > > > > > >> > > > > >> support for these frameworks, just making the
>> > > integration
>> > > > > > >> lighter
>> > > > > > >> > by
>> > > > > > >> > > > > >> separating process management from partition
>> > > management.
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> You raise two good points we would have to figure
>> out
>> > > if
>> > > > we
>> > > > > > went
>> > > > > > >> > > down
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> alignment path:
>> > > > > > >> > > > > >> 1. With respect to the name, yeah I think the
>> first
>> > > > > question
>> > > > > > is
>> > > > > > >> > > > whether
>> > > > > > >> > > > > >> some "re-branding" would be worth it. If so then I
>> > > think
>> > > > we
>> > > > > > can
>> > > > > > >> > > have a
>> > > > > > >> > > > > big
>> > > > > > >> > > > > >> thread on the name. I'm definitely not set on
>> Kafka
>> > > > > > Streaming or
>> > > > > > >> > > Kafka
>> > > > > > >> > > > > >> Streams I was just using them to be kind of
>> > > > illustrative. I
>> > > > > > >> agree
>> > > > > > >> > > with
>> > > > > > >> > > > > >> your
>> > > > > > >> > > > > >> critique of these names, though I think people
>> would
>> > > get
>> > > > > the
>> > > > > > >> idea.
>> > > > > > >> > > > > >> 2. Yeah you also raise a good point about how to
>> > > "factor"
>> > > > > it.
>> > > > > > >> Here
>> > > > > > >> > > are
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> options I see (I could get enthusiastic about any
>> of
>> > > > them):
>> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
>> > > > > > >> > > > > >>    b. Two repos, retaining the current seperation
>> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api and
>> > > > samza-core
>> > > > > > is
>> > > > > > >> > > > absorbed
>> > > > > > >> > > > > >> almost like a third client
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> Cheers,
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> -Jay
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
>> > > > > > >> > > > martin@kleppmann.com>
>> > > > > > >> > > > > >> wrote:
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a few
>> > > follow-up
>> > > > > > >> > comments.
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
>> > becoming
>> > > a
>> > > > > > >> > subproject:
>> > > > > > >> > > > the
>> > > > > > >> > > > > >> > reasons you mention are good. The risk I see is
>> > that
>> > > > > > release
>> > > > > > >> > > > schedules
>> > > > > > >> > > > > >> > become coupled to each other, which can slow
>> > everyone
>> > > > > down,
>> > > > > > >> and
>> > > > > > >> > > > large
>> > > > > > >> > > > > >> > projects with many contributors are harder to
>> > manage.
>> > > > > > (Jakob,
>> > > > > > >> > can
>> > > > > > >> > > > you
>> > > > > > >> > > > > >> speak
>> > > > > > >> > > > > >> > from experience, having seen a wider range of
>> > Hadoop
>> > > > > > ecosystem
>> > > > > > >> > > > > >> projects?)
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > Some of the goals of a better unified developer
>> > > > > experience
>> > > > > > >> could
>> > > > > > >> > > > also
>> > > > > > >> > > > > be
>> > > > > > >> > > > > >> > solved by integrating Samza nicely into a Kafka
>> > > > > > distribution
>> > > > > > >> > (such
>> > > > > > >> > > > as
>> > > > > > >> > > > > >> > Confluent's). I'm not against merging projects
>> if
>> > we
>> > > > > decide
>> > > > > > >> > that's
>> > > > > > >> > > > the
>> > > > > > >> > > > > >> way
>> > > > > > >> > > > > >> > to go, just pointing out the same goals can
>> perhaps
>> > > > also
>> > > > > be
>> > > > > > >> > > achieved
>> > > > > > >> > > > > in
>> > > > > > >> > > > > >> > other ways.
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > - With regard to dropping the YARN dependency:
>> are
>> > > you
>> > > > > > >> proposing
>> > > > > > >> > > > that
>> > > > > > >> > > > > >> > Samza doesn't give any help to people wanting to
>> > run
>> > > on
>> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
>> > > > > > >> > > > > >> > So the docs would basically have a link to
>> Slider
>> > and
>> > > > > > nothing
>> > > > > > >> > > else?
>> > > > > > >> > > > Or
>> > > > > > >> > > > > >> > would we maintain integrations with a bunch of
>> > > popular
>> > > > > > >> > deployment
>> > > > > > >> > > > > >> methods
>> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts to
>> make
>> > > > Samza
>> > > > > > work
>> > > > > > >> > with
>> > > > > > >> > > > > >> Slider)?
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > I absolutely think it's a good idea to have the
>> > "as a
>> > > > > > library"
>> > > > > > >> > and
>> > > > > > >> > > > > "as a
>> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
>> people
>> > who
>> > > > > want
>> > > > > > >> them,
>> > > > > > >> > > > but I
>> > > > > > >> > > > > >> > think there should also be a low-friction path
>> for
>> > > > common
>> > > > > > "as
>> > > > > > >> a
>> > > > > > >> > > > > service"
>> > > > > > >> > > > > >> > deployment methods, for which we probably need
>> to
>> > > > > maintain
>> > > > > > >> > > > > integrations.
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to
>> me,
>> > > > > because
>> > > > > > >> Kafka
>> > > > > > >> > > is
>> > > > > > >> > > > > all
>> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
>> Transformers"
>> > > or
>> > > > > > "Kafka
>> > > > > > >> > > > Filters"
>> > > > > > >> > > > > >> > would be more apt?
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
>> (stream
>> > > > > > >> transformation
>> > > > > > >> > > > with
>> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
>> library"
>> > > bit)
>> > > > > > could
>> > > > > > >> > > become
>> > > > > > >> > > > > >> part of
>> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
>> streaming
>> > SQL
>> > > > and
>> > > > > > >> > > > integrations
>> > > > > > >> > > > > >> with
>> > > > > > >> > > > > >> > deployment frameworks remain in a separate
>> project?
>> > > In
>> > > > > > other
>> > > > > > >> > > words,
>> > > > > > >> > > > > >> Kafka
>> > > > > > >> > > > > >> > would absorb the proven, stable core of Samza,
>> > which
>> > > > > would
>> > > > > > >> > become
>> > > > > > >> > > > the
>> > > > > > >> > > > > >> > "third Kafka client" mentioned early in this
>> > thread.
>> > > > The
>> > > > > > Samza
>> > > > > > >> > > > project
>> > > > > > >> > > > > >> > would then target that third Kafka client as its
>> > base
>> > > > > API,
>> > > > > > and
>> > > > > > >> > the
>> > > > > > >> > > > > >> project
>> > > > > > >> > > > > >> > would be freed up to explore more experimental
>> new
>> > > > > > horizons.
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > Martin
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
>> > > > jay.kreps@gmail.com>
>> > > > > > >> wrote:
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > > Hey Martin,
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually
>> > don't
>> > > > > think
>> > > > > > it
>> > > > > > >> > ties
>> > > > > > >> > > > our
>> > > > > > >> > > > > >> > hands
>> > > > > > >> > > > > >> > > at all, all it does is refactor things. The
>> > > division
>> > > > of
>> > > > > > >> > > > > >> responsibility is
>> > > > > > >> > > > > >> > > that Samza core is responsible for task
>> > lifecycle,
>> > > > > state,
>> > > > > > >> and
>> > > > > > >> > > > > >> partition
>> > > > > > >> > > > > >> > > management (using the Kafka co-ordinator) but
>> it
>> > is
>> > > > NOT
>> > > > > > >> > > > responsible
>> > > > > > >> > > > > >> for
>> > > > > > >> > > > > >> > > packaging, configuration deployment or
>> execution
>> > of
>> > > > > > >> processes.
>> > > > > > >> > > The
>> > > > > > >> > > > > >> > problem
>> > > > > > >> > > > > >> > > of packaging and starting these processes is
>> > > > > > >> > > > > >> > > framework/environment-specific. This leaves
>> > > > individual
>> > > > > > >> > > frameworks
>> > > > > > >> > > > to
>> > > > > > >> > > > > >> be
>> > > > > > >> > > > > >> > as
>> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you can get
>> > > simple
>> > > > > > >> stateless
>> > > > > > >> > > > > >> support in
>> > > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app
>> > > > > framework
>> > > > > > >> > > (Slider,
>> > > > > > >> > > > > >> > Marathon,
>> > > > > > >> > > > > >> > > etc). These are well known by people and have
>> > nice
>> > > > UIs
>> > > > > > and a
>> > > > > > >> > lot
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> > > flexibility. I don't think they have node
>> > affinity
>> > > > as a
>> > > > > > >> built
>> > > > > > >> > in
>> > > > > > >> > > > > >> option
>> > > > > > >> > > > > >> > > (though I could be wrong). So if we want that
>> we
>> > > can
>> > > > > > either
>> > > > > > >> > wait
>> > > > > > >> > > > for
>> > > > > > >> > > > > >> them
>> > > > > > >> > > > > >> > > to add it or do a custom framework to add that
>> > > > feature
>> > > > > > (as
>> > > > > > >> > now).
>> > > > > > >> > > > > >> > Obviously
>> > > > > > >> > > > > >> > > if you manage things with old-school ops tools
>> > > > > > >> > (puppet/chef/etc)
>> > > > > > >> > > > you
>> > > > > > >> > > > > >> get
>> > > > > > >> > > > > >> > > locality easily. The nice thing, though, is
>> that
>> > > all
>> > > > > the
>> > > > > > >> samza
>> > > > > > >> > > > > >> "business
>> > > > > > >> > > > > >> > > logic" around partition management and fault
>> > > > tolerance
>> > > > > > is in
>> > > > > > >> > > Samza
>> > > > > > >> > > > > >> core
>> > > > > > >> > > > > >> > so
>> > > > > > >> > > > > >> > > it is shared across frameworks and the
>> framework
>> > > > > specific
>> > > > > > >> bit
>> > > > > > >> > is
>> > > > > > >> > > > > just
>> > > > > > >> > > > > >> > > whether it is smart enough to try to get the
>> same
>> > > > host
>> > > > > > when
>> > > > > > >> a
>> > > > > > >> > > job
>> > > > > > >> > > > is
>> > > > > > >> > > > > >> > > restarted.
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah I
>> think
>> > > the
>> > > > > > goal
>> > > > > > >> > would
>> > > > > > >> > > > be
>> > > > > > >> > > > > >> (a)
>> > > > > > >> > > > > >> > > actually get better alignment in user
>> experience,
>> > > and
>> > > > > (b)
>> > > > > > >> > > express
>> > > > > > >> > > > > >> this in
>> > > > > > >> > > > > >> > > the naming and project branding. Specifically:
>> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
>> > > > > > "transformation"
>> > > > > > >> api
>> > > > > > >> > > to
>> > > > > > >> > > > be
>> > > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be
>> able
>> > > to
>> > > > > > explain
>> > > > > > >> > > when
>> > > > > > >> > > > to
>> > > > > > >> > > > > >> use
>> > > > > > >> > > > > >> > > the consumer and when to use the stream
>> > processing
>> > > > > > >> > functionality
>> > > > > > >> > > > and
>> > > > > > >> > > > > >> lead
>> > > > > > >> > > > > >> > > people into that experience.
>> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2
>> (or
>> > > > > > whatever)
>> > > > > > >> > that
>> > > > > > >> > > > has
>> > > > > > >> > > > > >> both
>> > > > > > >> > > > > >> > > Kafka and the stream processing part and they
>> > > > actually
>> > > > > > work
>> > > > > > >> > > > > together.
>> > > > > > >> > > > > >> > > 3. Unify the programming experience so the
>> client
>> > > and
>> > > > > > Samza
>> > > > > > >> > api
>> > > > > > >> > > > > share
>> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > I think sub-projects keep separate committers
>> and
>> > > can
>> > > > > > have a
>> > > > > > >> > > > > separate
>> > > > > > >> > > > > >> > repo,
>> > > > > > >> > > > > >> > > but I'm actually not really sure (I can't
>> find a
>> > > > > > definition
>> > > > > > >> > of a
>> > > > > > >> > > > > >> > subproject
>> > > > > > >> > > > > >> > > in Apache).
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > Basically at a high-level you want the
>> experience
>> > > to
>> > > > > > "feel"
>> > > > > > >> > > like a
>> > > > > > >> > > > > >> single
>> > > > > > >> > > > > >> > > system, not to relatively independent things
>> that
>> > > are
>> > > > > > kind
>> > > > > > >> of
>> > > > > > >> > > > > >> awkwardly
>> > > > > > >> > > > > >> > > glued together.
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > I think if we did that they having naming or
>> > > branding
>> > > > > > like
>> > > > > > >> > > "kafka
>> > > > > > >> > > > > >> > > streaming" or "kafka streams" or something
>> like
>> > > that
>> > > > > > would
>> > > > > > >> > > > actually
>> > > > > > >> > > > > >> do a
>> > > > > > >> > > > > >> > > good job of conveying what it is. I do that
>> this
>> > > > would
>> > > > > > help
>> > > > > > >> > > > adoption
>> > > > > > >> > > > > >> > quite
>> > > > > > >> > > > > >> > > a lot as it would correctly convey that using
>> > Kafka
>> > > > > > >> Streaming
>> > > > > > >> > > with
>> > > > > > >> > > > > >> Kafka
>> > > > > > >> > > > > >> > is
>> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
>> pretty
>> > > > > heavily
>> > > > > > >> > adopted
>> > > > > > >> > > > at
>> > > > > > >> > > > > >> this
>> > > > > > >> > > > > >> > > point.
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > Fwiw we actually considered this model
>> originally
>> > > > when
>> > > > > > open
>> > > > > > >> > > > sourcing
>> > > > > > >> > > > > >> > Samza,
>> > > > > > >> > > > > >> > > however at that time Kafka was relatively
>> unknown
>> > > and
>> > > > > we
>> > > > > > >> > decided
>> > > > > > >> > > > not
>> > > > > > >> > > > > >> to
>> > > > > > >> > > > > >> > do
>> > > > > > >> > > > > >> > > it since we felt it would be limiting. From my
>> > > point
>> > > > of
>> > > > > > view
>> > > > > > >> > the
>> > > > > > >> > > > > three
>> > > > > > >> > > > > >> > > things have changed (1) Kafka is now really
>> > heavily
>> > > > > used
>> > > > > > for
>> > > > > > >> > > > stream
>> > > > > > >> > > > > >> > > processing, (2) we learned that abstracting
>> out
>> > the
>> > > > > > stream
>> > > > > > >> > well
>> > > > > > >> > > is
>> > > > > > >> > > > > >> > > basically impossible, (3) we learned it is
>> really
>> > > > hard
>> > > > > to
>> > > > > > >> keep
>> > > > > > >> > > the
>> > > > > > >> > > > > two
>> > > > > > >> > > > > >> > > things feeling like a single product.
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > -Jay
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
>> Kleppmann
>> > <
>> > > > > > >> > > > > >> martin@kleppmann.com>
>> > > > > > >> > > > > >> > > wrote:
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > >> Hi all,
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> Lots of good thoughts here.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> I agree with the general philosophy of tying
>> > Samza
>> > > > > more
>> > > > > > >> > firmly
>> > > > > > >> > > to
>> > > > > > >> > > > > >> Kafka.
>> > > > > > >> > > > > >> > >> After I spent a while looking at integrating
>> > other
>> > > > > > message
>> > > > > > >> > > > brokers
>> > > > > > >> > > > > >> (e.g.
>> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
>> > > > conclusion
>> > > > > > that
>> > > > > > >> > > > > >> > SystemConsumer
>> > > > > > >> > > > > >> > >> tacitly assumes a model so much like Kafka's
>> > that
>> > > > > pretty
>> > > > > > >> much
>> > > > > > >> > > > > nobody
>> > > > > > >> > > > > >> but
>> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
>> > perhaps
>> > > an
>> > > > > > >> > exception,
>> > > > > > >> > > > but
>> > > > > > >> > > > > >> it
>> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus,
>> > > making
>> > > > > > Samza
>> > > > > > >> > > fully
>> > > > > > >> > > > > >> > dependent
>> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
>> > system-independence
>> > > > was
>> > > > > > >> never
>> > > > > > >> > as
>> > > > > > >> > > > > real
>> > > > > > >> > > > > >> as
>> > > > > > >> > > > > >> > we
>> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of code
>> > reuse
>> > > > are
>> > > > > > >> real.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN has
>> also
>> > > > always
>> > > > > > been
>> > > > > > >> > > > > >> appealing to
>> > > > > > >> > > > > >> > >> me, for various reasons already mentioned in
>> > this
>> > > > > > thread.
>> > > > > > >> > > > Although
>> > > > > > >> > > > > >> > making
>> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
>> > > > (YARN/Mesos/AWS/etc)
>> > > > > > >> seems
>> > > > > > >> > > > > >> laudable,
>> > > > > > >> > > > > >> > I am
>> > > > > > >> > > > > >> > >> a little concerned that it will restrict us
>> to a
>> > > > > lowest
>> > > > > > >> > common
>> > > > > > >> > > > > >> > denominator.
>> > > > > > >> > > > > >> > >> For example, would host affinity (SAMZA-617)
>> > still
>> > > > be
>> > > > > > >> > possible?
>> > > > > > >> > > > For
>> > > > > > >> > > > > >> jobs
>> > > > > > >> > > > > >> > >> with large amounts of state, I think
>> SAMZA-617
>> > > would
>> > > > > be
>> > > > > > a
>> > > > > > >> big
>> > > > > > >> > > > boon,
>> > > > > > >> > > > > >> > since
>> > > > > > >> > > > > >> > >> restoring state off the changelog on every
>> > single
>> > > > > > restart
>> > > > > > >> is
>> > > > > > >> > > > > painful,
>> > > > > > >> > > > > >> > due
>> > > > > > >> > > > > >> > >> to long recovery times. It would be a shame
>> if
>> > the
>> > > > > > >> decoupling
>> > > > > > >> > > > from
>> > > > > > >> > > > > >> YARN
>> > > > > > >> > > > > >> > >> made host affinity impossible.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> Jay, a question about the proposed API for
>> > > > > > instantiating a
>> > > > > > >> > job
>> > > > > > >> > > in
>> > > > > > >> > > > > >> code
>> > > > > > >> > > > > >> > >> (rather than a properties file): when
>> > submitting a
>> > > > job
>> > > > > > to a
>> > > > > > >> > > > > cluster,
>> > > > > > >> > > > > >> is
>> > > > > > >> > > > > >> > the
>> > > > > > >> > > > > >> > >> idea that the instantiation code runs on a
>> > client
>> > > > > > >> somewhere,
>> > > > > > >> > > > which
>> > > > > > >> > > > > >> then
>> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
>> > > YARN/Mesos/AWS/etc?
>> > > > > Or
>> > > > > > >> does
>> > > > > > >> > > that
>> > > > > > >> > > > > >> code
>> > > > > > >> > > > > >> > run
>> > > > > > >> > > > > >> > >> on each container that is part of the job (in
>> > > which
>> > > > > > case,
>> > > > > > >> how
>> > > > > > >> > > > does
>> > > > > > >> > > > > >> the
>> > > > > > >> > > > > >> > job
>> > > > > > >> > > > > >> > >> submission to the cluster work)?
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
>> right to
>> > > > make
>> > > > > a
>> > > > > > 1.0
>> > > > > > >> > > > release
>> > > > > > >> > > > > >> > with a
>> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete. So if
>> > this
>> > > > is
>> > > > > > going
>> > > > > > >> > to
>> > > > > > >> > > > > >> happen, I
>> > > > > > >> > > > > >> > >> think it would be more honest to stick with
>> 0.*
>> > > > > version
>> > > > > > >> > numbers
>> > > > > > >> > > > > until
>> > > > > > >> > > > > >> > the
>> > > > > > >> > > > > >> > >> library-ified Samza has been implemented, is
>> > > stable
>> > > > > and
>> > > > > > >> > widely
>> > > > > > >> > > > > used.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
>> Kafka?
>> > > There
>> > > > > is
>> > > > > > >> > > precedent
>> > > > > > >> > > > > for
>> > > > > > >> > > > > >> > >> tight coupling between different Apache
>> projects
>> > > > (e.g.
>> > > > > > >> > Curator
>> > > > > > >> > > > and
>> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think
>> > > remaining
>> > > > > > >> separate
>> > > > > > >> > > > would
>> > > > > > >> > > > > >> be
>> > > > > > >> > > > > >> > ok.
>> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on Kafka,
>> there
>> > > is
>> > > > > > enough
>> > > > > > >> > > > > substance
>> > > > > > >> > > > > >> in
>> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
>> project.
>> > > An
>> > > > > > >> argument
>> > > > > > >> > in
>> > > > > > >> > > > > >> favour
>> > > > > > >> > > > > >> > of
>> > > > > > >> > > > > >> > >> merging would be if we think Kafka has a much
>> > > > stronger
>> > > > > > >> "brand
>> > > > > > >> > > > > >> presence"
>> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If
>> the
>> > > Kafka
>> > > > > > >> project
>> > > > > > >> > is
>> > > > > > >> > > > > >> willing
>> > > > > > >> > > > > >> > to
>> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of doing
>> > > > stateful
>> > > > > > >> stream
>> > > > > > >> > > > > >> > >> transformations, that would probably have
>> much
>> > the
>> > > > > same
>> > > > > > >> > effect
>> > > > > > >> > > as
>> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
>> Processors"
>> > or
>> > > > > > suchlike.
>> > > > > > >> > > Close
>> > > > > > >> > > > > >> > >> collaboration between the two projects will
>> be
>> > > > needed
>> > > > > in
>> > > > > > >> any
>> > > > > > >> > > > case.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> From a project management perspective, I
>> guess
>> > the
>> > > > > "new
>> > > > > > >> > Samza"
>> > > > > > >> > > > > would
>> > > > > > >> > > > > >> > have
>> > > > > > >> > > > > >> > >> to be developed on a branch alongside ongoing
>> > > > > > maintenance
>> > > > > > >> of
>> > > > > > >> > > the
>> > > > > > >> > > > > >> current
>> > > > > > >> > > > > >> > >> line of development? I think it would be
>> > important
>> > > > to
>> > > > > > >> > continue
>> > > > > > >> > > > > >> > supporting
>> > > > > > >> > > > > >> > >> existing users, and provide a graceful
>> migration
>> > > > path
>> > > > > to
>> > > > > > >> the
>> > > > > > >> > > new
>> > > > > > >> > > > > >> > version.
>> > > > > > >> > > > > >> > >> Leaving the current versions unsupported and
>> > > forcing
>> > > > > > people
>> > > > > > >> > to
>> > > > > > >> > > > > >> rewrite
>> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> Best,
>> > > > > > >> > > > > >> > >> Martin
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
>> > > > jay@confluent.io>
>> > > > > > >> wrote:
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >>> Hey Garry,
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy
>> to
>> > > chat
>> > > > > > more
>> > > > > > >> > about
>> > > > > > >> > > > > this
>> > > > > > >> > > > > >> if
>> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
>> > started
>> > > > with
>> > > > > > the
>> > > > > > >> > idea
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> "what
>> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
>> > ingestion
>> > > > > tool"
>> > > > > > but
>> > > > > > >> > > > > >> ultimately
>> > > > > > >> > > > > >> > we
>> > > > > > >> > > > > >> > >>> kind of came around to the idea that
>> ingestion
>> > > and
>> > > > > > >> > > > transformation
>> > > > > > >> > > > > >> had
>> > > > > > >> > > > > >> > >>> pretty different needs and coupling the two
>> > made
>> > > > > things
>> > > > > > >> > hard.
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>> For what it's worth I think copycat (KIP-26)
>> > > > actually
>> > > > > > will
>> > > > > > >> > do
>> > > > > > >> > > > what
>> > > > > > >> > > > > >> you
>> > > > > > >> > > > > >> > >> are
>> > > > > > >> > > > > >> > >>> looking for.
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>> With regard to your point about slider, I
>> don't
>> > > > > > >> necessarily
>> > > > > > >> > > > > >> disagree.
>> > > > > > >> > > > > >> > >> But I
>> > > > > > >> > > > > >> > >>> think getting good YARN support is quite
>> doable
>> > > > and I
>> > > > > > >> think
>> > > > > > >> > we
>> > > > > > >> > > > can
>> > > > > > >> > > > > >> make
>> > > > > > >> > > > > >> > >>> that work well. I think the issue this
>> proposal
>> > > > > solves
>> > > > > > is
>> > > > > > >> > that
>> > > > > > >> > > > > >> > >> technically
>> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
>> cluster
>> > > > > > management
>> > > > > > >> > > systems
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> > way
>> > > > > > >> > > > > >> > >>> things are now, you need to write an "app
>> > master"
>> > > > or
>> > > > > > >> > > "framework"
>> > > > > > >> > > > > for
>> > > > > > >> > > > > >> > each
>> > > > > > >> > > > > >> > >>> and they are all a little different so
>> testing
>> > is
>> > > > > > really
>> > > > > > >> > hard.
>> > > > > > >> > > > In
>> > > > > > >> > > > > >> the
>> > > > > > >> > > > > >> > >>> absence of this we have been stuck with just
>> > YARN
>> > > > > which
>> > > > > > >> has
>> > > > > > >> > > > > >> fantastic
>> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the org,
>> but
>> > > > zero
>> > > > > > >> > > penetration
>> > > > > > >> > > > > >> > >> elsewhere.
>> > > > > > >> > > > > >> > >>> Given the huge amount of work being put in
>> to
>> > > > slider,
>> > > > > > >> > > marathon,
>> > > > > > >> > > > > aws
>> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen related
>> > > > packaging
>> > > > > > >> > > > technologies
>> > > > > > >> > > > > >> > people
>> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
>> > > > > cloud-specific
>> > > > > > >> > deploy
>> > > > > > >> > > > > >> tools,
>> > > > > > >> > > > > >> > >> etc)
>> > > > > > >> > > > > >> > >>> I really think it is important to get this
>> > right.
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>> -Jay
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
>> > Turkington
>> > > <
>> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>>> Hi all,
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> I think the question below re does Samza
>> > become
>> > > a
>> > > > > > >> > sub-project
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> Kafka
>> > > > > > >> > > > > >> > >>>> highlights the broader point around
>> migration.
>> > > > Chris
>> > > > > > >> > mentions
>> > > > > > >> > > > > >> Samza's
>> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1 release
>> but
>> > I'm
>> > > > not
>> > > > > > sure
>> > > > > > >> > it
>> > > > > > >> > > > > feels
>> > > > > > >> > > > > >> > >> right to
>> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
>> deprecate
>> > > > most
>> > > > > of
>> > > > > > >> it.
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> From a selfish perspective I have some guys
>> > who
>> > > > have
>> > > > > > >> > started
>> > > > > > >> > > > > >> working
>> > > > > > >> > > > > >> > >> with
>> > > > > > >> > > > > >> > >>>> Samza and building some new
>> > consumers/producers
>> > > > was
>> > > > > > next
>> > > > > > >> > up.
>> > > > > > >> > > > > Sounds
>> > > > > > >> > > > > >> > like
>> > > > > > >> > > > > >> > >>>> that is absolutely not the direction to
>> go. I
>> > > need
>> > > > > to
>> > > > > > >> look
>> > > > > > >> > > into
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> > KIP
>> > > > > > >> > > > > >> > >> in
>> > > > > > >> > > > > >> > >>>> more detail but for me the attractiveness
>> of
>> > > > adding
>> > > > > > new
>> > > > > > >> > Samza
>> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all they
>> > were
>> > > > > doing
>> > > > > > was
>> > > > > > >> > > > really
>> > > > > > >> > > > > >> > getting
>> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to avoid
>> > > > having
>> > > > > to
>> > > > > > >> > worry
>> > > > > > >> > > > > about
>> > > > > > >> > > > > >> the
>> > > > > > >> > > > > >> > >>>> lifecycle management of external clients.
>> If
>> > > there
>> > > > > is
>> > > > > > a
>> > > > > > >> > > generic
>> > > > > > >> > > > > >> Kafka
>> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a new
>> > > > connector
>> > > > > > into
>> > > > > > >> > and
>> > > > > > >> > > > > have
>> > > > > > >> > > > > >> a
>> > > > > > >> > > > > >> > >> lot of
>> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and reliability
>> > done
>> > > > for
>> > > > > me
>> > > > > > >> then
>> > > > > > >> > > it
>> > > > > > >> > > > > >> gives
>> > > > > > >> > > > > >> > me
>> > > > > > >> > > > > >> > >> all
>> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers would.
>> If
>> > > not
>> > > > > > then it
>> > > > > > >> > > > > >> complicates
>> > > > > > >> > > > > >> > my
>> > > > > > >> > > > > >> > >>>> operational deployments.
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> Which is similar to my other question with
>> the
>> > > > > > proposal
>> > > > > > >> --
>> > > > > > >> > if
>> > > > > > >> > > > we
>> > > > > > >> > > > > >> > build a
>> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus the
>> > > > requisite
>> > > > > > >> shims
>> > > > > > >> > to
>> > > > > > >> > > > > >> > integrate
>> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former may
>> be a
>> > > lot
>> > > > > more
>> > > > > > >> work
>> > > > > > >> > > > than
>> > > > > > >> > > > > we
>> > > > > > >> > > > > >> > >> think.
>> > > > > > >> > > > > >> > >>>> We may make it much easier for a newcomer
>> to
>> > get
>> > > > > > >> something
>> > > > > > >> > > > > running
>> > > > > > >> > > > > >> but
>> > > > > > >> > > > > >> > >>>> having them step up and get a reliable
>> > > production
>> > > > > > >> > deployment
>> > > > > > >> > > > may
>> > > > > > >> > > > > >> still
>> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
>> > different
>> > > > > > reasons
>> > > > > > >> > than
>> > > > > > >> > > > > >> today.
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with
>> > > making
>> > > > > the
>> > > > > > >> Samza
>> > > > > > >> > > > > >> dependency
>> > > > > > >> > > > > >> > >> on
>> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I absolutely
>> see
>> > > the
>> > > > > > >> benefits
>> > > > > > >> > > in
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
>> > > > > > >> > > > terminologies/abstractions
>> > > > > > >> > > > > >> that
>> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library
>> would
>> > > > likely
>> > > > > > be a
>> > > > > > >> > very
>> > > > > > >> > > > > nice
>> > > > > > >> > > > > >> > tool
>> > > > > > >> > > > > >> > >> to
>> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the
>> > > > concerns
>> > > > > > >> above
>> > > > > > >> > re
>> > > > > > >> > > > the
>> > > > > > >> > > > > >> > >>>> operational side.
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> Garry
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> -----Original Message-----
>> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
>> [mailto:
>> > > > > > >> > gdfm@apache.org
>> > > > > > >> > > ]
>> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
>> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
>> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on
>> > Samza
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
>> > > > > > >> > > > > >> > >>>> From outside, I have always perceived Samza
>> > as a
>> > > > > > >> computing
>> > > > > > >> > > > layer
>> > > > > > >> > > > > >> over
>> > > > > > >> > > > > >> > >>>> Kafka.
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> The question, maybe a bit provocative, is
>> > > "should
>> > > > > > Samza
>> > > > > > >> be
>> > > > > > >> > a
>> > > > > > >> > > > > >> > sub-project
>> > > > > > >> > > > > >> > >>>> of Kafka then?"
>> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
>> separate
>> > > > > project
>> > > > > > >> > with a
>> > > > > > >> > > > > >> separate
>> > > > > > >> > > > > >> > >>>> governance?
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> Cheers,
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> --
>> > > > > > >> > > > > >> > >>>> Gianmarco
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
>> > > > > > yanfang724@gmail.com>
>> > > > > > >> > > > wrote:
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more
>> > > > tightly.
>> > > > > > >> > Because
>> > > > > > >> > > > > Samza
>> > > > > > >> > > > > >> de
>> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
>> > leverage
>> > > > > what
>> > > > > > >> Kafka
>> > > > > > >> > > > has.
>> > > > > > >> > > > > At
>> > > > > > >> > > > > >> > the
>> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to reinvent
>> > what
>> > > > > Samza
>> > > > > > >> > > already
>> > > > > > >> > > > > >> has. I
>> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
>> > ingestion
>> > > > and
>> > > > > > >> > > > > transformation.
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
>> image
>> > > how
>> > > > > the
>> > > > > > >> Samza
>> > > > > > >> > > > will
>> > > > > > >> > > > > >> look
>> > > > > > >> > > > > >> > >>>> like.
>> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
>> > > difference
>> > > > > in
>> > > > > > >> terms
>> > > > > > >> > > of
>> > > > > > >> > > > > how
>> > > > > > >> > > > > >> > >>>>> Samza should look like.
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code
>> shows
>> > (A
>> > > > > > client of
>> > > > > > >> > > > Kakfa)
>> > > > > > >> > > > > ?
>> > > > > > >> > > > > >> And
>> > > > > > >> > > > > >> > >>>>> user's application code calls this client?
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka
>> > (like
>> > > > > what
>> > > > > > the
>> > > > > > >> > > code
>> > > > > > >> > > > > >> shows),
>> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
>> > > > > fault-tolerance?
>> > > > > > >> Are
>> > > > > > >> > > they
>> > > > > > >> > > > > >> taken
>> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
>> mechanism,
>> > > such
>> > > > > as
>> > > > > > >> > "Samza
>> > > > > > >> > > > > >> worker"
>> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> 2. What about other features, such as
>> > > > auto-scaling,
>> > > > > > >> shared
>> > > > > > >> > > > > state,
>> > > > > > >> > > > > >> > >>>>> monitoring?
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is this
>> > what
>> > > > > Chris
>> > > > > > >> > > > suggests?)
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa
>> > and
>> > > > > > produce
>> > > > > > >> to
>> > > > > > >> > > it.
>> > > > > > >> > > > > >> Then it
>> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks like
>> > now,
>> > > > > > except it
>> > > > > > >> > > does
>> > > > > > >> > > > > not
>> > > > > > >> > > > > >> > rely
>> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
>> leverage
>> > > > Kafka's
>> > > > > > >> > metrics,
>> > > > > > >> > > > > logs,
>> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> Thanks,
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> Fang, Yan
>> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang
>> > Wang <
>> > > > > > >> > > > > wangguoz@gmail.com
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > >>>> wrote:
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>>> Read through the code example and it
>> looks
>> > > good
>> > > > to
>> > > > > > me.
>> > > > > > >> A
>> > > > > > >> > > few
>> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
>> runnable
>> > > like:
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
>> > > --config-factory=...
>> > > > > > >> > > > > >> > >>>> --config-path=file://...
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for deploying
>> > Samza
>> > > > > more
>> > > > > > as
>> > > > > > >> > > > embedded
>> > > > > > >> > > > > >> > >>>>>> libraries in user application code
>> (ignoring
>> > > the
>> > > > > > >> > > terminology
>> > > > > > >> > > > > >> since
>> > > > > > >> > > > > >> > >>>>>> it is not the
>> > > > > > >> > > > > >> > >>>>> same
>> > > > > > >> > > > > >> > >>>>>> as the prototype code):
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
>> MyStreamTask(configs);
>> > > > > Thread
>> > > > > > >> > thread
>> > > > > > >> > > =
>> > > > > > >> > > > > new
>> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> I think both of these deployment modes
>> are
>> > > > > important
>> > > > > > >> for
>> > > > > > >> > > > > >> different
>> > > > > > >> > > > > >> > >>>>>> types
>> > > > > > >> > > > > >> > >>>>> of
>> > > > > > >> > > > > >> > >>>>>> users. That said, I think making Samza
>> > purely
>> > > > > > >> standalone
>> > > > > > >> > is
>> > > > > > >> > > > > still
>> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or library
>> > > modes.
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> Guozhang
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
>> Kreps
>> > <
>> > > > > > >> > > > jay@confluent.io>
>> > > > > > >> > > > > >> > wrote:
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
>> example,
>> > it
>> > > > was
>> > > > > > >> > supposed
>> > > > > > >> > > > to
>> > > > > > >> > > > > >> look
>> > > > > > >> > > > > >> > >>>>>>> like
>> > > > > > >> > > > > >> > >>>>>>> this:
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
>> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
>> > > > "localhost:4242");
>> > > > > > >> > > > > >> StreamingConfig
>> > > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
>> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
>> > > > "test-topic-2");
>> > > > > > >> > > > > >> > >>>>>>>
>> > > config.processor(ExampleStreamProcessor.class);
>> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
>> > StringSerializer(),
>> > > > new
>> > > > > > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming
>> > > > container =
>> > > > > > new
>> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>> -Jay
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay
>> > Kreps <
>> > > > > > >> > > > jay@confluent.io
>> > > > > > >> > > > > >
>> > > > > > >> > > > > >> > >>>> wrote:
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
>> Chris
>> > > and
>> > > > I
>> > > > > > were
>> > > > > > >> > > having
>> > > > > > >> > > > > >> > >>>>>>>> around
>> > > > > > >> > > > > >> > >>>>>>> whether
>> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a
>> kind
>> > > of
>> > > > > data
>> > > > > > >> > > > ingestion
>> > > > > > >> > > > > >> > >>>>> framework
>> > > > > > >> > > > > >> > >>>>>>> for
>> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26
>> > > > > "copycat").
>> > > > > > >> This
>> > > > > > >> > > > kind
>> > > > > > >> > > > > of
>> > > > > > >> > > > > >> > >>>>>> combined
>> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and YARN
>> and
>> > > the
>> > > > > > >> > discussion
>> > > > > > >> > > > > >> around
>> > > > > > >> > > > > >> > >>>>>>>> how
>> > > > > > >> > > > > >> > >>>>> to
>> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was, given
>> that
>> > > > Samza
>> > > > > > was
>> > > > > > >> > > > basically
>> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific, what if
>> > you
>> > > > just
>> > > > > > >> > embraced
>> > > > > > >> > > > > that
>> > > > > > >> > > > > >> > >>>>>>>> and turned it
>> > > > > > >> > > > > >> > >>>>>> into
>> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
>> > framework
>> > > > and
>> > > > > > more
>> > > > > > >> > > like a
>> > > > > > >> > > > > >> > >>>>>>>> third
>> > > > > > >> > > > > >> > >>>>> Kafka
>> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing consumer"
>> > with
>> > > > > state
>> > > > > > >> > > > management
>> > > > > > >> > > > > >> > >>>>>> facilities.
>> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
>> complex
>> > > > stream
>> > > > > > >> > > processing
>> > > > > > >> > > > > >> > >>>>>>>> framework
>> > > > > > >> > > > > >> > >>>>>>> this
>> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple thing,
>> not
>> > > > much
>> > > > > > more
>> > > > > > >> > > > > >> complicated
>> > > > > > >> > > > > >> > >>>>>>>> to
>> > > > > > >> > > > > >> > >>>>> use
>> > > > > > >> > > > > >> > >>>>>>> or
>> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris
>> > said
>> > > > we
>> > > > > > >> thought
>> > > > > > >> > > > about
>> > > > > > >> > > > > >> it
>> > > > > > >> > > > > >> > >>>>>>>> a
>> > > > > > >> > > > > >> > >>>>> lot
>> > > > > > >> > > > > >> > >>>>>> of
>> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
>> > processing
>> > > > > > systems
>> > > > > > >> > were
>> > > > > > >> > > > > doing)
>> > > > > > >> > > > > >> > >>>>> seemed
>> > > > > > >> > > > > >> > >>>>>>> like
>> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output
>> data
>> > to
>> > > > and
>> > > > > > from
>> > > > > > >> > the
>> > > > > > >> > > > > stream
>> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually looked
>> > into
>> > > > how
>> > > > > > that
>> > > > > > >> > > would
>> > > > > > >> > > > > >> > >>>>>>>> work,
>> > > > > > >> > > > > >> > >>>>> Samza
>> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion
>> > > framework
>> > > > > > for a
>> > > > > > >> > > bunch
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> > >>>>> reasons.
>> > > > > > >> > > > > >> > >>>>>> To
>> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a pretty
>> > > > different
>> > > > > > >> > internal
>> > > > > > >> > > > > data
>> > > > > > >> > > > > >> > >>>>>>>> model
>> > > > > > >> > > > > >> > >>>>>> and
>> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split them
>> and
>> > > had
>> > > > > an
>> > > > > > api
>> > > > > > >> > for
>> > > > > > >> > > > > Kafka
>> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26)
>> and a
>> > > > > separate
>> > > > > > >> api
>> > > > > > >> > > for
>> > > > > > >> > > > > >> Kafka
>> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> This would also allow really embracing
>> the
>> > > > same
>> > > > > > >> > > terminology
>> > > > > > >> > > > > and
>> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about the
>> > current
>> > > > > > state is
>> > > > > > >> > > that
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> > >>>>>>>> two
>> > > > > > >> > > > > >> > >>>>>>> systems
>> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology
>> like
>> > > > > "stream"
>> > > > > > vs
>> > > > > > >> > > > "topic"
>> > > > > > >> > > > > >> and
>> > > > > > >> > > > > >> > >>>>>>> different
>> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means you
>> > kind
>> > > > of
>> > > > > > have
>> > > > > > >> to
>> > > > > > >> > > > learn
>> > > > > > >> > > > > >> > >>>>>>>> Kafka's
>> > > > > > >> > > > > >> > >>>>>>> way,
>> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly different
>> way,
>> > > > then
>> > > > > > kind
>> > > > > > >> of
>> > > > > > >> > > > > >> > >>>>>>>> understand
>> > > > > > >> > > > > >> > >>>>> how
>> > > > > > >> > > > > >> > >>>>>>> they
>> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having walked
>> a
>> > few
>> > > > > > people
>> > > > > > >> > > through
>> > > > > > >> > > > > >> this
>> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to
>> get.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of
>> time
>> > on
>> > > > > > >> airplanes I
>> > > > > > >> > > > > hacked
>> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
>> incomplete
>> > > > > > prototype
>> > > > > > >> of
>> > > > > > >> > > > what
>> > > > > > >> > > > > >> > >>>>>>>> this would
>> > > > > > >> > > > > >> > >>>>> look
>> > > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
>> dumped
>> > > into
>> > > > > > Kafka
>> > > > > > >> as
>> > > > > > >> > > it
>> > > > > > >> > > > > >> > >>>>>>>> required a
>> > > > > > >> > > > > >> > >>>>>> few
>> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is
>> the
>> > > code:
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > >
>> > > > > > >> >
>> > > > > >
>> > >
>> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I just
>> > > > > liberally
>> > > > > > >> > renamed
>> > > > > > >> > > > > >> > >>>>>>>> everything
>> > > > > > >> > > > > >> > >>>>> to
>> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
>> regard
>> > > for
>> > > > > > >> > > > compatibility.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> To use this would be something like
>> this:
>> > > > > > >> > > > > >> > >>>>>>>> Properties props = new Properties();
>> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
>> > > > > "localhost:4242");
>> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
>> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
>> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
>> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
>> > > > > > >> > > > > >> config.processor(ExampleStreamProcessor.class);
>> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
>> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
>> > > StringDeserializer());
>> > > > > > >> > > > KafkaStreaming
>> > > > > > >> > > > > >> > >>>>>> container =
>> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
>> > container.run();
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
>> > > > SamzaContainer;
>> > > > > > >> > > > > StreamProcessor
>> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the class
>> names
>> > > in
>> > > > a
>> > > > > > file
>> > > > > > >> > and
>> > > > > > >> > > > then
>> > > > > > >> > > > > >> > >>>>>>>> having
>> > > > > > >> > > > > >> > >>>>>> the
>> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
>> > > > > instantiate
>> > > > > > the
>> > > > > > >> > > > > container
>> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced over
>> > > > however
>> > > > > > many
>> > > > > > >> > > > > instances
>> > > > > > >> > > > > >> > >>>>>>>> of
>> > > > > > >> > > > > >> > >>>>> this
>> > > > > > >> > > > > >> > >>>>>>> are
>> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance
>> > dies,
>> > > > new
>> > > > > > >> tasks
>> > > > > > >> > > are
>> > > > > > >> > > > > >> added
>> > > > > > >> > > > > >> > >>>>>>>> to
>> > > > > > >> > > > > >> > >>>>> the
>> > > > > > >> > > > > >> > >>>>>>>> existing containers without shutting
>> them
>> > > > down).
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for running
>> > this
>> > > > > stuff
>> > > > > > in
>> > > > > > >> > YARN
>> > > > > > >> > > > via
>> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
>> using
>> > > some
>> > > > > of
>> > > > > > >> their
>> > > > > > >> > > > tools
>> > > > > > >> > > > > >> > >>>>>>>> but from the
>> > > > > > >> > > > > >> > >>>>>> point
>> > > > > > >> > > > > >> > >>>>>>> of
>> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these stream
>> > > > processing
>> > > > > > jobs
>> > > > > > >> > are
>> > > > > > >> > > > > just
>> > > > > > >> > > > > >> > >>>>>> stateless
>> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
>> expand
>> > and
>> > > > > > contract
>> > > > > > >> > at
>> > > > > > >> > > > > will.
>> > > > > > >> > > > > >> > >>>>>>>> There
>> > > > > > >> > > > > >> > >>>>> is
>> > > > > > >> > > > > >> > >>>>>>> no
>> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it
>> > would
>> > > > get
>> > > > > > >> larger
>> > > > > > >> > > if
>> > > > > > >> > > > we
>> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly larger.
>> We
>> > > > really
>> > > > > > do
>> > > > > > >> > get a
>> > > > > > >> > > > ton
>> > > > > > >> > > > > >> > >>>>>>>> of
>> > > > > > >> > > > > >> > >>>>>>> leverage
>> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
>> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
>> > delegated
>> > > to
>> > > > > the
>> > > > > > >> new
>> > > > > > >> > > > > >> consumer.
>> > > > > > >> > > > > >> > >>>>> This
>> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
>> > management
>> > > > > > strategy
>> > > > > > >> > > > > available
>> > > > > > >> > > > > >> > >>>>>>>> to
>> > > > > > >> > > > > >> > >>>>>> Kafka
>> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to Samza
>> (and
>> > > vice
>> > > > > > versa)
>> > > > > > >> > and
>> > > > > > >> > > > > with
>> > > > > > >> > > > > >> > >>>>>>>> the
>> > > > > > >> > > > > >> > >>>>>>> exact
>> > > > > > >> > > > > >> > >>>>>>>>  same configs.
>> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as state
>> > reuse
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
>> > thought
>> > > > > > >> provoking.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> -Jay
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris
>> > > > > Riccomini <
>> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
>> > > > > > >> > > > > >> > >>>>>>>> wrote:
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with Samza
>> > > > > engineers
>> > > > > > at
>> > > > > > >> > > > LinkedIn
>> > > > > > >> > > > > >> > >>>>>>>>> and
>> > > > > > >> > > > > >> > >>>>>>> Confluent
>> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few observations
>> > and
>> > > > > would
>> > > > > > >> like
>> > > > > > >> > to
>> > > > > > >> > > > > >> > >>>>>>>>> propose
>> > > > > > >> > > > > >> > >>>>> some
>> > > > > > >> > > > > >> > >>>>>>>>> changes.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I
>> want to
>> > > > call
>> > > > > > out
>> > > > > > >> > about
>> > > > > > >> > > > > >> > >>>>>>>>> Samza's
>> > > > > > >> > > > > >> > >>>>>> design,
>> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic
>> > > > deployment
>> > > > > > >> system.
>> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
>> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
>> SystemConsumer/SystemProducer
>> > and
>> > > > > > Kafka's
>> > > > > > >> > > > consumer
>> > > > > > >> > > > > >> > >>>>>>>>> APIs
>> > > > > > >> > > > > >> > >>>>> are
>> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
>> > problems.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are related,
>> > but
>> > > > I'll
>> > > > > > >> > address
>> > > > > > >> > > > them
>> > > > > > >> > > > > >> in
>> > > > > > >> > > > > >> > >>>>> order.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Deployment
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a
>> > > > dynamic
>> > > > > > >> > > deployment
>> > > > > > >> > > > > >> > >>>>>>>>> scheduler
>> > > > > > >> > > > > >> > >>>>>> such
>> > > > > > >> > > > > >> > >>>>>>>>> as
>> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially
>> built
>> > > > > Samza,
>> > > > > > we
>> > > > > > >> > bet
>> > > > > > >> > > > that
>> > > > > > >> > > > > >> > >>>>>>>>> there
>> > > > > > >> > > > > >> > >>>>>> would
>> > > > > > >> > > > > >> > >>>>>>>>> be
>> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area, and
>> we
>> > > could
>> > > > > > >> support
>> > > > > > >> > > > them,
>> > > > > > >> > > > > >> and
>> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > > >> > > > > >> > >>>>>> rest
>> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there are
>> many
>> > > > > > >> variations.
>> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
>> > > > > > >> > > > > >> > >>>>>> many
>> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start
>> their
>> > > > > > processors
>> > > > > > >> > like
>> > > > > > >> > > > > normal
>> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use traditional
>> > > > deployment
>> > > > > > >> scripts
>> > > > > > >> > > > such
>> > > > > > >> > > > > as
>> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
>> > > > > > >> > > > > >> > >>>>>> Chef,
>> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
>> system
>> > > on
>> > > > > > users
>> > > > > > >> > makes
>> > > > > > >> > > > the
>> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really painful
>> for
>> > > > first
>> > > > > > time
>> > > > > > >> > > > users.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement
>> was
>> > > also
>> > > > a
>> > > > > > bit
>> > > > > > >> of
>> > > > > > >> > a
>> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
>> > > > > > >> > > > > >> > >>>>>> because
>> > > > > > >> > > > > >> > >>>>>>>>> of
>> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between
>> > the
>> > > > > > nature of
>> > > > > > >> > > batch
>> > > > > > >> > > > > >> jobs
>> > > > > > >> > > > > >> > >>>>>>>>> and
>> > > > > > >> > > > > >> > >>>>>>> stream
>> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
>> > > conscious
>> > > > > > effort
>> > > > > > >> to
>> > > > > > >> > > > favor
>> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > > >> > > > > >> > >>>>>> Hadoop
>> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things,
>> since
>> > it
>> > > > > worked
>> > > > > > >> and
>> > > > > > >> > > was
>> > > > > > >> > > > > well
>> > > > > > >> > > > > >> > >>>>>>> understood.
>> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that
>> batch
>> > > jobs
>> > > > > > have a
>> > > > > > >> > > > definite
>> > > > > > >> > > > > >> > >>>>>> beginning,
>> > > > > > >> > > > > >> > >>>>>>>>> and
>> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't
>> > > > > (usually).
>> > > > > > >> This
>> > > > > > >> > > > leads
>> > > > > > >> > > > > to
>> > > > > > >> > > > > >> > >>>>>>>>> a
>> > > > > > >> > > > > >> > >>>>> much
>> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream
>> > > > > processors.
>> > > > > > >> You
>> > > > > > >> > > > > >> basically
>> > > > > > >> > > > > >> > >>>>>>>>> just
>> > > > > > >> > > > > >> > >>>>>>> need
>> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
>> processor,
>> > and
>> > > > > start
>> > > > > > >> it.
>> > > > > > >> > > The
>> > > > > > >> > > > > way
>> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no
>> > > concept
>> > > > > of
>> > > > > > a
>> > > > > > >> > > cluster
>> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
>> > > > > > >> > > > > >> > >>>>>> add
>> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
>> coupling
>> > > > Samza
>> > > > > > with
>> > > > > > >> a
>> > > > > > >> > > > > >> scheduler
>> > > > > > >> > > > > >> > >>>>>>>>> is
>> > > > > > >> > > > > >> > >>>>>> that
>> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
>> handle
>> > > > > > deployment.
>> > > > > > >> > > This
>> > > > > > >> > > > > >> pulls
>> > > > > > >> > > > > >> > >>>>>>>>> in a
>> > > > > > >> > > > > >> > >>>>>>> bunch
>> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
>> > > distribution
>> > > > > > (config
>> > > > > > >> > > > > stream),
>> > > > > > >> > > > > >> > >>>>>>>>> shell
>> > > > > > >> > > > > >> > >>>>>>> scrips
>> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging
>> > (all
>> > > > the
>> > > > > > .tgz
>> > > > > > >> > > > stuff),
>> > > > > > >> > > > > >> etc.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
>> > > > deployment
>> > > > > > was
>> > > > > > >> to
>> > > > > > >> > > > > support
>> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
>> > > locality,
>> > > > > you
>> > > > > > >> need
>> > > > > > >> > to
>> > > > > > >> > > > put
>> > > > > > >> > > > > >> > >>>>>>>>> your
>> > > > > > >> > > > > >> > >>>>>> processors
>> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're processing.
>> > Upon
>> > > > > > further
>> > > > > > >> > > > > >> > >>>>>>>>> investigation,
>> > > > > > >> > > > > >> > >>>>>>> though,
>> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that beneficial.
>> > There
>> > > is
>> > > > > > some
>> > > > > > >> > good
>> > > > > > >> > > > > >> > >>>>>>>>> discussion
>> > > > > > >> > > > > >> > >>>>>> about
>> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335.
>> > Again,
>> > > we
>> > > > > > took
>> > > > > > >> the
>> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
>> > > > > > >> > > > > >> > >>>>>> path,
>> > > > > > >> > > > > >> > >>>>>>>>> but
>> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental differences
>> > > > between
>> > > > > > HDFS
>> > > > > > >> > and
>> > > > > > >> > > > > Kafka.
>> > > > > > >> > > > > >> > >>>>>>>>> HDFS
>> > > > > > >> > > > > >> > >>>>>> has
>> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions.
>> This
>> > > > leads
>> > > > > to
>> > > > > > >> less
>> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
>> > > processors
>> > > > > on
>> > > > > > top
>> > > > > > >> > of
>> > > > > > >> > > > > Kafka.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a crutch.
>> > > Samza
>> > > > > > doesn't
>> > > > > > >> > > have
>> > > > > > >> > > > > any
>> > > > > > >> > > > > >> > >>>>>>>>> built
>> > > > > > >> > > > > >> > >>>>> in
>> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
>> > depends
>> > > on
>> > > > > the
>> > > > > > >> > > dynamic
>> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to handle
>> > > > restarts
>> > > > > > >> when a
>> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
>> > > > > > >> > > > > >> > >>>>>>> made
>> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
>> standalone
>> > > Samza
>> > > > > > >> > container
>> > > > > > >> > > > > >> > >>>> (SAMZA-516).
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is good,
>> but I
>> > > > think
>> > > > > > that
>> > > > > > >> > > we've
>> > > > > > >> > > > > >> gone
>> > > > > > >> > > > > >> > >>>>>>>>> too
>> > > > > > >> > > > > >> > >>>>>> far
>> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
>> > > > (SystemConsumer,
>> > > > > > >> > > > > SystemProducer,
>> > > > > > >> > > > > >> > >>>> etc).
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about
>> > every
>> > > > > > >> component
>> > > > > > >> > > > > >> > >>>>> (MessageChooser,
>> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
>> > > ConfigRewriter,
>> > > > > > etc).
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
>> > forgotten,
>> > > as
>> > > > > > well.
>> > > > > > >> > Some
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> > >>>>>>>>> these
>> > > > > > >> > > > > >> > >>>>> are
>> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not to
>> be.
>> > > This
>> > > > > all
>> > > > > > >> comes
>> > > > > > >> > > at
>> > > > > > >> > > > a
>> > > > > > >> > > > > >> cost:
>> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is making
>> it
>> > > > harder
>> > > > > > for
>> > > > > > >> > our
>> > > > > > >> > > > > users
>> > > > > > >> > > > > >> > >>>>>>>>> to
>> > > > > > >> > > > > >> > >>>>> pick
>> > > > > > >> > > > > >> > >>>>>> up
>> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also
>> > makes
>> > > > it
>> > > > > > >> > difficult
>> > > > > > >> > > > for
>> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about what
>> the
>> > > > > > >> > > characteristics
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
>> characteristics
>> > > > change
>> > > > > > >> > > depending
>> > > > > > >> > > > on
>> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are most
>> > > visible
>> > > > > in
>> > > > > > the
>> > > > > > >> > > > System
>> > > > > > >> > > > > >> APIs.
>> > > > > > >> > > > > >> > >>>>> What
>> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
>> functional is
>> > > > Kafka
>> > > > > > as
>> > > > > > >> its
>> > > > > > >> > > > > >> > >>>>>>>>> transport
>> > > > > > >> > > > > >> > >>>>>> layer.
>> > > > > > >> > > > > >> > >>>>>>>>> But
>> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use
>> cases
>> > > into
>> > > > > one
>> > > > > > >> API:
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
>> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports both
>> of
>> > > these
>> > > > > use
>> > > > > > >> > cases.
>> > > > > > >> > > > The
>> > > > > > >> > > > > >> > >>>>>>>>> problem
>> > > > > > >> > > > > >> > >>>>>> is,
>> > > > > > >> > > > > >> > >>>>>>>>> we
>> > > > > > >> > > > > >> > >>>>>>>>> actually want different features for
>> each
>> > > use
>> > > > > > case.
>> > > > > > >> By
>> > > > > > >> > > > > >> papering
>> > > > > > >> > > > > >> > >>>>>>>>> over
>> > > > > > >> > > > > >> > >>>>>>> these
>> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a single
>> > API,
>> > > > > we've
>> > > > > > >> > > > introduced
>> > > > > > >> > > > > a
>> > > > > > >> > > > > >> > >>>>>>>>> ton of
>> > > > > > >> > > > > >> > >>>>>>> leaky
>> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like in
>> (2)
>> > > is
>> > > > to
>> > > > > > have
>> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
>> > offsets
>> > > > > (like
>> > > > > > >> > Kafka).
>> > > > > > >> > > > > This
>> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
>> > > > > > >> > > > > >> > >>>>> with
>> > > > > > >> > > > > >> > >>>>>>> (1),
>> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems have
>> > > > different
>> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
>> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
>> mailing
>> > > list
>> > > > > and
>> > > > > > >> the
>> > > > > > >> > > SQL
>> > > > > > >> > > > > >> JIRAs
>> > > > > > >> > > > > >> > >>>>> about
>> > > > > > >> > > > > >> > >>>>>>> the
>> > > > > > >> > > > > >> > >>>>>>>>> need for this.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
>> > > replayability.
>> > > > > > Kafka
>> > > > > > >> > > allows
>> > > > > > >> > > > us
>> > > > > > >> > > > > >> to
>> > > > > > >> > > > > >> > >>>>> rewind
>> > > > > > >> > > > > >> > >>>>>>>>> when
>> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other systems
>> > > don't.
>> > > > In
>> > > > > > some
>> > > > > > >> > > > cases,
>> > > > > > >> > > > > >> > >>>>>>>>> systems
>> > > > > > >> > > > > >> > >>>>>>> return
>> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
>> > > > > > >> WikipediaSystemConsumer)
>> > > > > > >> > > > > because
>> > > > > > >> > > > > >> > >>>>>>>>> they
>> > > > > > >> > > > > >> > >>>>>> have
>> > > > > > >> > > > > >> > >>>>>>> no
>> > > > > > >> > > > > >> > >>>>>>>>> offsets.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka
>> > > > supports
>> > > > > > >> > > > > partitioning,
>> > > > > > >> > > > > >> > >>>>>>>>> but
>> > > > > > >> > > > > >> > >>>>> many
>> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
>> having a
>> > > > single
>> > > > > > >> > > partition
>> > > > > > >> > > > > for
>> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other systems
>> model
>> > > > > > >> partitioning
>> > > > > > >> > > > > >> > >>>> differently (e.g.
>> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a
>> mess.
>> > > > > > Creating
>> > > > > > >> > > streams
>> > > > > > >> > > > > in
>> > > > > > >> > > > > >> a
>> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
>> impossible.
>> > > As
>> > > > is
>> > > > > > >> > modeling
>> > > > > > >> > > > > >> > >>>>>>>>> metadata
>> > > > > > >> > > > > >> > >>>>> for
>> > > > > > >> > > > > >> > >>>>>>> the
>> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
>> partitions,
>> > > > > location,
>> > > > > > >> > etc).
>> > > > > > >> > > > The
>> > > > > > >> > > > > >> > >>>>>>>>> list
>> > > > > > >> > > > > >> > >>>>> goes
>> > > > > > >> > > > > >> > >>>>>>> on.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing
>> Samza,
>> > > > > Kafka's
>> > > > > > >> > > consumer
>> > > > > > >> > > > > and
>> > > > > > >> > > > > >> > >>>>> producer
>> > > > > > >> > > > > >> > >>>>>>>>> APIs
>> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On
>> the
>> > > > > > >> > consumer-side,
>> > > > > > >> > > > you
>> > > > > > >> > > > > >> > >>>>>>>>> had two
>> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level consumer,
>> or
>> > > the
>> > > > > > simple
>> > > > > > >> > > > > consumer.
>> > > > > > >> > > > > >> > >>>>>>>>> The
>> > > > > > >> > > > > >> > >>>>>>> problem
>> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was that
>> it
>> > > > > > controlled
>> > > > > > >> > your
>> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and
>> the
>> > > order
>> > > > > in
>> > > > > > >> which
>> > > > > > >> > > you
>> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
>> > > > > > >> > > > > >> > >>>>> problem
>> > > > > > >> > > > > >> > >>>>>>>>> with
>> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not
>> > > simple.
>> > > > > It's
>> > > > > > >> > basic.
>> > > > > > >> > > > You
>> > > > > > >> > > > > >> > >>>>>>>>> end up
>> > > > > > >> > > > > >> > >>>>>>> having
>> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level
>> stuff
>> > > > that
>> > > > > > you
>> > > > > > >> > > > > shouldn't.
>> > > > > > >> > > > > >> > >>>>>>>>> We
>> > > > > > >> > > > > >> > >>>>>> spent a
>> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
>> > > > KafkaSystemConsumer
>> > > > > > very
>> > > > > > >> > > > robust.
>> > > > > > >> > > > > >> It
>> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some cool
>> > > features:
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
>> > > > > > prioritization.
>> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
>> assignment
>> > > to
>> > > > > > support
>> > > > > > >> > > > joins,
>> > > > > > >> > > > > >> > >>>>>>>>> global
>> > > > > > >> > > > > >> > >>>>>> state
>> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
>> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
>> > checkpointing.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time is
>> > that
>> > > > > these
>> > > > > > >> > > features
>> > > > > > >> > > > > >> > >>>>>>>>> should
>> > > > > > >> > > > > >> > >>>>>>> actually
>> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers
>> > (not
>> > > > just
>> > > > > > >> Samza
>> > > > > > >> > > > stream
>> > > > > > >> > > > > >> > >>>>>> processors)
>> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like joins
>> > and
>> > > > > > partition
>> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
>> > > > > > >> > > > > >> > >>>>>>> Kafka
>> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
>> > conclusion.
>> > > > > > They're
>> > > > > > >> > > adding
>> > > > > > >> > > > a
>> > > > > > >> > > > > >> ton
>> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
>> consumer
>> > > > > > >> > > implementation.
>> > > > > > >> > > > > To a
>> > > > > > >> > > > > >> > >>>>>>>>> large extent,
>> > > > > > >> > > > > >> > >>>>> it's
>> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've already
>> done
>> > > in
>> > > > > > Samza.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking
>> a
>> > > very
>> > > > > > similar
>> > > > > > >> > > > > approach
>> > > > > > >> > > > > >> > >>>>>>>>> to
>> > > > > > >> > > > > >> > >>>>>> Samza's
>> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation
>> for
>> > > > > > handling
>> > > > > > >> > > offset
>> > > > > > >> > > > > >> > >>>>>> checkpointing.
>> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
>> management
>> > > > > feature
>> > > > > > >> > stores
>> > > > > > >> > > > > >> offset
>> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows
>> you to
>> > > > fetch
>> > > > > > them
>> > > > > > >> > > from
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> > >>>>>>>>> broker.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste,
>> since
>> > we
>> > > > > could
>> > > > > > >> have
>> > > > > > >> > > > shared
>> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > > >> > > > > >> > >>>>> work
>> > > > > > >> > > > > >> > >>>>>> if
>> > > > > > >> > > > > >> > >>>>>>>>> it
>> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
>> get-go.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Vision
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
>> radical
>> > > > > > proposal.
>> > > > > > >> > Samza
>> > > > > > >> > > > is
>> > > > > > >> > > > > >> > >>>>> relatively
>> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to
>> say
>> > > that
>> > > > > > we're
>> > > > > > >> > > near a
>> > > > > > >> > > > > 1.0
>> > > > > > >> > > > > >> > >>>>>> release.
>> > > > > > >> > > > > >> > >>>>>>>>> I'd
>> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what
>> we've
>> > > > > learned,
>> > > > > > and
>> > > > > > >> > > begin
>> > > > > > >> > > > > >> > >>>>>>>>> thinking
>> > > > > > >> > > > > >> > >>>>>>> about
>> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we
>> change if
>> > > we
>> > > > > were
>> > > > > > >> > > starting
>> > > > > > >> > > > > >> from
>> > > > > > >> > > > > >> > >>>>>> scratch?
>> > > > > > >> > > > > >> > >>>>>>>>> My
>> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only*
>> way
>> > to
>> > > > run
>> > > > > > Samza
>> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
>> > > > > dependences
>> > > > > > on
>> > > > > > >> > > YARN,
>> > > > > > >> > > > > >> Mesos,
>> > > > > > >> > > > > >> > >>>> etc.
>> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support
>> only
>> > > > Kafka
>> > > > > > as
>> > > > > > >> the
>> > > > > > >> > > > > stream
>> > > > > > >> > > > > >> > >>>>>> processing
>> > > > > > >> > > > > >> > >>>>>>>>> layer.
>> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
>> > > > > > >> serialization,
>> > > > > > >> > > and
>> > > > > > >> > > > > >> > >>>>>>>>> config
>> > > > > > >> > > > > >> > >>>>>>> systems,
>> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues that
>> I
>> > > > > outlined
>> > > > > > >> > above.
>> > > > > > >> > > It
>> > > > > > >> > > > > >> > >>>>>>>>> should
>> > > > > > >> > > > > >> > >>>>> also
>> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
>> > > > dramatically.
>> > > > > > >> > > Supporting
>> > > > > > >> > > > > >> only
>> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow
>> Samza
>> > to
>> > > be
>> > > > > > >> executed
>> > > > > > >> > > on
>> > > > > > >> > > > > YARN
>> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
>> > > > Marathon/Aurora),
>> > > > > or
>> > > > > > >> most
>> > > > > > >> > > > other
>> > > > > > >> > > > > >> > >>>>>>>>> in-house
>> > > > > > >> > > > > >> > >>>>>>> deployment
>> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a lot
>> > easier
>> > > > for
>> > > > > > new
>> > > > > > >> > > users.
>> > > > > > >> > > > > >> > >>>>>>>>> Imagine
>> > > > > > >> > > > > >> > >>>>>>> having
>> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN.
>> > The
>> > > > drop
>> > > > > > in
>> > > > > > >> > > mailing
>> > > > > > >> > > > > >> list
>> > > > > > >> > > > > >> > >>>>>> traffic
>> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
>> overdue to
>> > > me.
>> > > > > The
>> > > > > > >> > > reality
>> > > > > > >> > > > > is,
>> > > > > > >> > > > > >> > >>>>> everyone
>> > > > > > >> > > > > >> > >>>>>>>>> that
>> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
>> Kafka.
>> > We
>> > > > > > basically
>> > > > > > >> > > > require
>> > > > > > >> > > > > >> it
>> > > > > > >> > > > > >> > >>>>>> already
>> > > > > > >> > > > > >> > >>>>>>> in
>> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work. Those
>> > that
>> > > > are
>> > > > > > >> using
>> > > > > > >> > > > other
>> > > > > > >> > > > > >> > >>>>>>>>> systems
>> > > > > > >> > > > > >> > >>>>>> are
>> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into
>> Kafka
>> > > (1),
>> > > > > and
>> > > > > > >> then
>> > > > > > >> > > > they
>> > > > > > >> > > > > do
>> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
>> already
>> > > > > > discussion (
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > >
>> > > > > > >> >
>> > > > > >
>> > >
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> > > > > > >> > > > > >> > >>>>> 767
>> > > > > > >> > > > > >> > >>>>>>>>> )
>> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka
>> > > > extremely
>> > > > > > >> easy.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple with
>> > Kafka,
>> > > > we
>> > > > > > can
>> > > > > > >> > > > leverage
>> > > > > > >> > > > > a
>> > > > > > >> > > > > >> > >>>>>>>>> ton of
>> > > > > > >> > > > > >> > >>>>>>> their
>> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
>> maintain
>> > > our
>> > > > > own
>> > > > > > >> > config,
>> > > > > > >> > > > > >> > >>>>>>>>> metrics,
>> > > > > > >> > > > > >> > >>>>> etc.
>> > > > > > >> > > > > >> > >>>>>>> We
>> > > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries, and
>> > make
>> > > > them
>> > > > > > >> > better.
>> > > > > > >> > > > This
>> > > > > > >> > > > > >> > >>>>>>>>> will
>> > > > > > >> > > > > >> >
>> ...
>>
>> [Message clipped]
>
>
>

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@gmail.com>.
Hey Chris,

Yeah, I'm obviously in favor of this.

The sub-project approach seems the ideal way to take a graceful step in
this direction, so I will ping the board folks and see why they are
discouraged, it would be good to understand that. If we go that route we
would need to do a similar discussion in the Kafka list (but makes sense to
figure out first if it is what Samza wants).

Irrespective of how it's implemented, though, to me the important things
are the following:
1. Unify the website, config, naming, docs, metrics, etc--basically fix the
product experience so the "stream" and the "processing" feel like a single
user experience and brand. This seems minor but I think is a really big
deal.
2. Make "standalone" mode a first class citizen and have a real technical
plan to be able to support cluster managers other than YARN.
3. Make the config and out-of-the-box experience more usable

I think that prototype gives a practical example of how 1-3 could be done
and we should pursue it. This is a pretty radical change, so I wouldn't be
shocked if people didn't want to take a step like that.

Maybe it would make sense to see if people are on board with that general
idea, and then try to get some advice on sub-projects in parallel and nail
down those details?

-Jay

On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <cr...@apache.org>
wrote:

> Hey all,
>
> I want to start by saying that I'm absolutely thrilled to be a part of this
> community. The amount of level-headed, thoughtful, educated discussion
> that's gone on over the past ~10 days is overwhelming. Wonderful.
>
> It seems like discussion is waning a bit, and we've reached some
> conclusions. There are several key emails in this threat, which I want to
> call out:
>
> 1. Jakob's summary of the three potential ways forward.
>
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> 2. Julian's call out that we should be focusing on community over code.
>
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> 3. Martin's summary about the benefits of merging communities.
>
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> 4. Jakob's comments about the distinction between community and code paths.
>
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
>
> I agree with the comments on all of these emails. I think Martin's summary
> of his position aligns very closely with my own. To that end, I think we
> should get concrete about what the proposal is, and call a vote on it.
> Given that Jay, Martin, and I seem to be aligning fairly closely, I think
> we should start with:
>
> 1. [community] Make Samza a subproject of Kafka.
> 2. [community] Make all Samza PMC/committers committers of the subproject.
> 3. [community] Migrate Samza's website/documentation into Kafka's.
> 4. [code] Have the Samza community and the Kafka community start a
> from-scratch reboot together in the new Kafka subproject. We can
> borrow/copy &  paste significant chunks of code from Samza's code base.
> 5. [code] The subproject would intentionally eliminate support for both
> other streaming systems and all deployment systems.
> 6. [code] Attempt to provide a bridge from our SystemConsumer to KIP-26
> (copy cat)
> 7. [code] Attempt to provide a bridge from the new subproject's processor
> interface to our legacy StreamTask interface.
> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> subproject that has a fault-tolerant container with state management.
>
> It's likely that (6) and (7) won't be fully drop-in. Still, the closer we
> can get, the better it's going to be for our existing community.
>
> One thing that I didn't touch on with (2) is whether any Samza PMC members
> should be rolled into Kafka PMC membership as well (though, Jay and Jakob
> are already PMC members on both). I think that Samza's community deserves a
> voice on the PMC, so I'd propose that we roll at least a few PMC members
> into the Kafka PMC, but I don't have a strong framework for which people to
> pick.
>
> Before (8), I think that Samza's TLP can continue to commit bug fixes and
> patches as it sees fit, provided that we openly communicate that we won't
> necessarily migrate new features to the new subproject, and that the TLP
> will be shut down after the migration to the Kafka subproject occurs.
>
> Jakob, I could use your guidance here about about how to achieve this from
> an Apache process perspective (sorry).
>
> * Should I just call a vote on this proposal?
> * Should it happen on dev or private?
> * Do committers have binding votes, or just PMC?
>
> Having trouble finding much detail on the Apache wikis. :(
>
> Cheers,
> Chris
>
> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com> wrote:
>
> > Thanks, Jay. This argument persuaded me actually. :)
> >
> > Fang, Yan
> > yanfang724@gmail.com
> >
> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Hey Yan,
> > >
> > > Yeah philosophically I think the argument is that you should capture
> the
> > > stream in Kafka independent of the transformation. This is obviously a
> > > Kafka-centric view point.
> > >
> > > Advantages of this:
> > > - In practice I think this is what e.g. Storm people often end up doing
> > > anyway. You usually need to throttle any access to a live serving
> > database.
> > > - Can have multiple subscribers and they get the same thing without
> > > additional load on the source system.
> > > - Applications can tap into the stream if need be by subscribing.
> > > - You can debug your transformation by tailing the Kafka topic with the
> > > console consumer
> > > - Can tee off the same data stream for batch analysis or Lambda arch
> > style
> > > re-processing
> > >
> > > The disadvantage is that it will use Kafka resources. But the idea is
> > > eventually you will have multiple subscribers to any data source (at
> > least
> > > for monitoring) so you will end up there soon enough anyway.
> > >
> > > Down the road the technical benefit is that I think it gives us a good
> > path
> > > towards end-to-end exactly once semantics from source to destination.
> > > Basically the connectors need to support idempotence when talking to
> > Kafka
> > > and we need the transactional write feature in Kafka to make the
> > > transformation atomic. This is actually pretty doable if you separate
> > > connector=>kafka problem from the generic transformations which are
> > always
> > > kafka=>kafka. However I think it is quite impossible to do in a
> > all_things
> > > => all_things environment. Today you can say "well the semantics of the
> > > Samza APIs depend on the connectors you use" but it is actually worse
> > then
> > > that because the semantics actually depend on the pairing of
> > connectors--so
> > > not only can you probably not get a usable "exactly once" guarantee
> > > end-to-end it can actually be quite hard to reverse engineer what
> > property
> > > (if any) your end-to-end flow has if you have heterogenous systems.
> > >
> > > -Jay
> > >
> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
> wrote:
> > >
> > > > {quote}
> > > > maintained in a separate repository and retaining the existing
> > > > committership but sharing as much else as possible (website, etc)
> > > > {quote}
> > > >
> > > > Overall, I agree on this idea. Now the question is more about "how to
> > do
> > > > it".
> > > >
> > > > On the other hand, one thing I want to point out is that, if we
> decide
> > to
> > > > go this way, how do we want to support
> > > > otherSystem-transformation-otherSystem use case?
> > > >
> > > > Basically, there are four user groups here:
> > > >
> > > > 1. Kafka-transformation-Kafka
> > > > 2. Kafka-transformation-otherSystem
> > > > 3. otherSystem-transformation-Kafka
> > > > 4. otherSystem-transformation-otherSystem
> > > >
> > > > For group 1, they can easily use the new Samza library to achieve.
> For
> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka or
> > Kafka->
> > > > transformation -> copyCat.
> > > >
> > > > The problem is for group 4. Do we want to abandon this or still
> support
> > > it?
> > > > Of course, this use case can be achieved by using copyCat ->
> > > transformation
> > > > -> Kafka -> transformation -> copyCat, the thing is how we persuade
> > them
> > > to
> > > > do this long chain. If yes, it will also be a win for Kafka too. Or
> if
> > > > there is no one in this community actually doing this so far, maybe
> ok
> > to
> > > > not support the group 4 directly.
> > > >
> > > > Thanks,
> > > >
> > > > Fang, Yan
> > > > yanfang724@gmail.com
> > > >
> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > > >
> > > > > Yeah I agree with this summary. I think there are kind of two
> > questions
> > > > > here:
> > > > > 1. Technically does alignment/reliance on Kafka make sense
> > > > > 2. Branding wise (naming, website, concepts, etc) does alignment
> with
> > > > Kafka
> > > > > make sense
> > > > >
> > > > > Personally I do think both of these things would be really
> valuable,
> > > and
> > > > > would dramatically alter the trajectory of the project.
> > > > >
> > > > > My preference would be to see if people can mostly agree on a
> > direction
> > > > > rather than splintering things off. From my point of view the ideal
> > > > outcome
> > > > > of all the options discussed would be to make Samza a closely
> aligned
> > > > > subproject, maintained in a separate repository and retaining the
> > > > existing
> > > > > committership but sharing as much else as possible (website, etc).
> No
> > > > idea
> > > > > about how these things work, Jacob, you probably know more.
> > > > >
> > > > > No discussion amongst the Kafka folks has happened on this, but
> > likely
> > > we
> > > > > should figure out what the Samza community actually wants first.
> > > > >
> > > > > I admit that this is a fairly radical departure from how things
> are.
> > > > >
> > > > > If that doesn't fly, I think, yeah we could leave Samza as it is
> and
> > do
> > > > the
> > > > > more radical reboot inside Kafka. From my point of view that does
> > leave
> > > > > things in a somewhat confusing state since now there are two stream
> > > > > processing systems more or less coupled to Kafka in large part made
> > by
> > > > the
> > > > > same people. But, arguably that might be a cleaner way to make the
> > > > cut-over
> > > > > and perhaps less risky for Samza community since if it works people
> > can
> > > > > switch and if it doesn't nothing will have changed. Dunno, how do
> > > people
> > > > > feel about this?
> > > > >
> > > > > -Jay
> > > > >
> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com>
> > > wrote:
> > > > >
> > > > > > >  This leads me to thinking that merging projects and
> communities
> > > > might
> > > > > > be a good idea: with the union of experience from both
> communities,
> > > we
> > > > > will
> > > > > > probably build a better system that is better for users.
> > > > > > Is this what's being proposed though? Merging the projects seems
> > like
> > > > > > a consequence of at most one of the three directions under
> > > discussion:
> > > > > > 1) Samza 2.0: The Samza community relies more heavily on Kafka
> for
> > > > > > configuration, etc. (to a greater or lesser extent to be
> > determined)
> > > > > > but the Samza community would not automatically merge withe Kafka
> > > > > > community (the Phoenix/HBase example is a good one here).
> > > > > > 2) Samza Reboot: The Samza community continues to exist with a
> > > limited
> > > > > > project scope, but similarly would not need to be part of the
> Kafka
> > > > > > community (ie given committership) to progress.  Here, maybe the
> > > Samza
> > > > > > team would become a subproject of Kafka (the Board frowns on
> > > > > > subprojects at the moment, so I'm not sure if that's even
> > feasible),
> > > > > > but that would not be required.
> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option the
> > Kafka
> > > > > > team builds its own streaming library, possibly off of Jay's
> > > > > > prototype, which has not direct lineage to the Samza team.
> There's
> > > no
> > > > > > reason for the Kafka team to bring in the Samza team.
> > > > > >
> > > > > > Is the Kafka community on board with this?
> > > > > >
> > > > > > To be clear, all three options under discussion are interesting,
> > > > > > technically valid and likely healthy directions for the project.
> > > > > > Also, they are not mutually exclusive.  The Samza community could
> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka community
> went
> > > > > > forward with 'Hey Samza!'  My points above are directed entirely
> at
> > > > > > the community aspect of these choices.
> > > > > > -Jakob
> > > > > >
> > > > > > On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com>
> > > wrote:
> > > > > > > That's great.  Thanks, Jay.
> > > > > > >
> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io>
> > > wrote:
> > > > > > >
> > > > > > >> Yeah totally agree. I think you have this issue even today,
> > right?
> > > > > I.e.
> > > > > > if
> > > > > > >> you need to make a simple config change and you're running in
> > YARN
> > > > > today
> > > > > > >> you end up bouncing the job which then rebuilds state. I think
> > the
> > > > fix
> > > > > > is
> > > > > > >> exactly what you described which is to have a long timeout on
> > > > > partition
> > > > > > >> movement for stateful jobs so that if a job is just getting
> > > bounced,
> > > > > and
> > > > > > >> the cluster manager (or admin) is smart enough to restart it
> on
> > > the
> > > > > same
> > > > > > >> host when possible, it can optimistically reuse any existing
> > state
> > > > it
> > > > > > finds
> > > > > > >> on disk (if it is valid).
> > > > > > >>
> > > > > > >> So in this model the charter of the CM is to place processes
> as
> > > > > > stickily as
> > > > > > >> possible and to restart or re-place failed processes. The
> > charter
> > > of
> > > > > the
> > > > > > >> partition management system is to control the assignment of
> work
> > > to
> > > > > > these
> > > > > > >> processes. The nice thing about this is that the work
> > assignment,
> > > > > > timeouts,
> > > > > > >> behavior, configs, and code will all be the same across all
> > > cluster
> > > > > > >> managers.
> > > > > > >>
> > > > > > >> So I think that prototype would actually give you exactly what
> > you
> > > > > want
> > > > > > >> today for any cluster manager (or manual placement + restart
> > > script)
> > > > > > that
> > > > > > >> was sticky in terms of host placement since there is already a
> > > > > > configurable
> > > > > > >> partition movement timeout and task-by-task state reuse with a
> > > check
> > > > > on
> > > > > > >> state validity.
> > > > > > >>
> > > > > > >> -Jay
> > > > > > >>
> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > > > roger.hoover@gmail.com
> > > > > >
> > > > > > >> wrote:
> > > > > > >>
> > > > > > >> > That would be great to let Kafka do as much heavy lifting as
> > > > > possible
> > > > > > and
> > > > > > >> > make it easier for other languages to implement Samza apis.
> > > > > > >> >
> > > > > > >> > One thing to watch out for is the interplay between Kafka's
> > > group
> > > > > > >> > management and the external scheduler/process manager's
> fault
> > > > > > tolerance.
> > > > > > >> > If a container dies, the Kafka group membership protocol
> will
> > > try
> > > > to
> > > > > > >> assign
> > > > > > >> > it's tasks to other containers while at the same time the
> > > process
> > > > > > manager
> > > > > > >> > is trying to relaunch the container.  Without some
> > consideration
> > > > for
> > > > > > this
> > > > > > >> > (like a configurable amount of time to wait before Kafka
> > alters
> > > > the
> > > > > > group
> > > > > > >> > membership), there may be thrashing going on which is
> > especially
> > > > bad
> > > > > > for
> > > > > > >> > containers with large amounts of local state.
> > > > > > >> >
> > > > > > >> > Someone else pointed this out already but I thought it might
> > be
> > > > > worth
> > > > > > >> > calling out again.
> > > > > > >> >
> > > > > > >> > Cheers,
> > > > > > >> >
> > > > > > >> > Roger
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> jay@confluent.io>
> > > > > wrote:
> > > > > > >> >
> > > > > > >> > > Hey Roger,
> > > > > > >> > >
> > > > > > >> > > I couldn't agree more. We spent a bunch of time talking to
> > > > people
> > > > > > and
> > > > > > >> > that
> > > > > > >> > > is exactly the stuff we heard time and again. What makes
> it
> > > > hard,
> > > > > of
> > > > > > >> > > course, is that there is some tension between
> compatibility
> > > with
> > > > > > what's
> > > > > > >> > > there now and making things better for new users.
> > > > > > >> > >
> > > > > > >> > > I also strongly agree with the importance of
> multi-language
> > > > > > support. We
> > > > > > >> > are
> > > > > > >> > > talking now about Java, but for application development
> use
> > > > cases
> > > > > > >> people
> > > > > > >> > > want to work in whatever language they are using
> elsewhere.
> > I
> > > > > think
> > > > > > >> > moving
> > > > > > >> > > to a model where Kafka itself does the group membership,
> > > > lifecycle
> > > > > > >> > control,
> > > > > > >> > > and partition assignment has the advantage of putting all
> > that
> > > > > > complex
> > > > > > >> > > stuff behind a clean api that the clients are already
> going
> > to
> > > > be
> > > > > > >> > > implementing for their consumer, so the added
> functionality
> > > for
> > > > > > stream
> > > > > > >> > > processing beyond a consumer becomes very minor.
> > > > > > >> > >
> > > > > > >> > > -Jay
> > > > > > >> > >
> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > > > > > roger.hoover@gmail.com>
> > > > > > >> > > wrote:
> > > > > > >> > >
> > > > > > >> > > > Metamorphosis...nice. :)
> > > > > > >> > > >
> > > > > > >> > > > This has been a great discussion.  As a user of Samza
> > who's
> > > > > > recently
> > > > > > >> > > > integrated it into a relatively large organization, I
> just
> > > > want
> > > > > to
> > > > > > >> add
> > > > > > >> > > > support to a few points already made.
> > > > > > >> > > >
> > > > > > >> > > > The biggest hurdles to adoption of Samza as it currently
> > > > exists
> > > > > > that
> > > > > > >> > I've
> > > > > > >> > > > experienced are:
> > > > > > >> > > > 1) YARN - YARN is overly complex in many environments
> > where
> > > > > Puppet
> > > > > > >> > would
> > > > > > >> > > do
> > > > > > >> > > > just fine but it was the only mechanism to get fault
> > > > tolerance.
> > > > > > >> > > > 2) Configuration - I think I like the idea of
> configuring
> > > most
> > > > > of
> > > > > > the
> > > > > > >> > job
> > > > > > >> > > > in code rather than config files.  In general, I think
> the
> > > > goal
> > > > > > >> should
> > > > > > >> > be
> > > > > > >> > > > to make it harder to make mistakes, especially of the
> kind
> > > > where
> > > > > > the
> > > > > > >> > code
> > > > > > >> > > > expects something and the config doesn't match.  The
> > current
> > > > > > config
> > > > > > >> is
> > > > > > >> > > > quite intricate and error-prone.  For example, the
> > > application
> > > > > > logic
> > > > > > >> > may
> > > > > > >> > > > depend on bootstrapping a topic but rather than
> asserting
> > > that
> > > > > in
> > > > > > the
> > > > > > >> > > code,
> > > > > > >> > > > you have to rely on getting the config right.  Likewise
> > with
> > > > > > serdes,
> > > > > > >> > the
> > > > > > >> > > > Java representations produced by various serdes (JSON,
> > Avro,
> > > > > etc.)
> > > > > > >> are
> > > > > > >> > > not
> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> without
> > > > > changing
> > > > > > >> the
> > > > > > >> > > > code.   It would be nice for jobs to be able to assert
> > what
> > > > they
> > > > > > >> expect
> > > > > > >> > > > from their input topics in terms of partitioning.  This
> is
> > > > > > getting a
> > > > > > >> > > little
> > > > > > >> > > > off topic but I was even thinking about creating a
> "Samza
> > > > config
> > > > > > >> > linter"
> > > > > > >> > > > that would sanity check a set of configs.  Especially in
> > > > > > >> organizations
> > > > > > >> > > > where config is managed by a different team than the
> > > > application
> > > > > > >> > > developer,
> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > > > > > >> > > > 3) Java/Scala centric - for many teams (especially
> > > DevOps-type
> > > > > > >> folks),
> > > > > > >> > > the
> > > > > > >> > > > pain of the Java toolchain (maven, slow builds, weak
> > command
> > > > > line
> > > > > > >> > > support,
> > > > > > >> > > > configuration over convention) really inhibits
> > productivity.
> > > > As
> > > > > > more
> > > > > > >> > and
> > > > > > >> > > > more high-quality clients become available for Kafka, I
> > hope
> > > > > > they'll
> > > > > > >> > > follow
> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> proposals
> > > in
> > > > > this
> > > > > > >> > thread
> > > > > > >> > > > but please consider other languages in the ecosystem as
> > > well.
> > > > > > From
> > > > > > >> > what
> > > > > > >> > > > I've heard, Spark has more Python users than Java/Scala.
> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > > > > >> > > > and are working on a Yeoman generator
> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> > > Jython/Samza
> > > > > > >> projects
> > > > > > >> > to
> > > > > > >> > > > alleviate some of the pain)
> > > > > > >> > > >
> > > > > > >> > > > I also want to underscore Jay's point about improving
> the
> > > user
> > > > > > >> > > experience.
> > > > > > >> > > > That's a very important factor for adoption.  I think
> the
> > > goal
> > > > > > should
> > > > > > >> > be
> > > > > > >> > > to
> > > > > > >> > > > make Samza as easy to get started with as something like
> > > > > Logstash.
> > > > > > >> > > > Logstash is vastly inferior in terms of capabilities to
> > > Samza
> > > > > but
> > > > > > >> it's
> > > > > > >> > > easy
> > > > > > >> > > > to get started and that makes a big difference.
> > > > > > >> > > >
> > > > > > >> > > > Cheers,
> > > > > > >> > > >
> > > > > > >> > > > Roger
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci
> > > > Morales <
> > > > > > >> > > > gdfm@apache.org> wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> Metamorphosis
> > > is
> > > > a
> > > > > > clear
> > > > > > >> > > > winner
> > > > > > >> > > > > :)
> > > > > > >> > > > >
> > > > > > >> > > > > --
> > > > > > >> > > > > Gianmarco
> > > > > > >> > > > >
> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> Morales
> > <
> > > > > > >> > > gdfm@apache.org
> > > > > > >> > > > >
> > > > > > >> > > > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > Hi,
> > > > > > >> > > > > >
> > > > > > >> > > > > > @Martin, thanks for you comments.
> > > > > > >> > > > > > Maybe I'm missing some important point, but I think
> > > > coupling
> > > > > > the
> > > > > > >> > > > releases
> > > > > > >> > > > > > is actually a *good* thing.
> > > > > > >> > > > > > To make an example, would it be better if the MR and
> > > HDFS
> > > > > > >> > components
> > > > > > >> > > of
> > > > > > >> > > > > > Hadoop had different release schedules?
> > > > > > >> > > > > >
> > > > > > >> > > > > > Actually, keeping the discussion in a single place
> > would
> > > > > make
> > > > > > >> > > agreeing
> > > > > > >> > > > on
> > > > > > >> > > > > > releases (and backwards compatibility) much easier,
> as
> > > > > > everybody
> > > > > > >> > > would
> > > > > > >> > > > be
> > > > > > >> > > > > > responsible for the whole codebase.
> > > > > > >> > > > > >
> > > > > > >> > > > > > That said, I like the idea of absorbing samza-core
> as
> > a
> > > > > > >> > sub-project,
> > > > > > >> > > > and
> > > > > > >> > > > > > leave the fancy stuff separate.
> > > > > > >> > > > > > It probably gives 90% of the benefits we have been
> > > > > discussing
> > > > > > >> here.
> > > > > > >> > > > > >
> > > > > > >> > > > > > Cheers,
> > > > > > >> > > > > >
> > > > > > >> > > > > > --
> > > > > > >> > > > > > Gianmarco
> > > > > > >> > > > > >
> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > jay.kreps@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >> > > > > >
> > > > > > >> > > > > >> Hey Martin,
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> I agree coupling release schedules is a downside.
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> Definitely we can try to solve some of the
> > integration
> > > > > > problems
> > > > > > >> in
> > > > > > >> > > > > >> Confluent Platform or in other distributions. But I
> > > think
> > > > > > this
> > > > > > >> > ends
> > > > > > >> > > up
> > > > > > >> > > > > >> being really shallow. I guess I feel to really get
> a
> > > good
> > > > > > user
> > > > > > >> > > > > experience
> > > > > > >> > > > > >> the two systems have to kind of feel like part of
> the
> > > > same
> > > > > > thing
> > > > > > >> > and
> > > > > > >> > > > you
> > > > > > >> > > > > >> can't really add that in later--you can put both in
> > the
> > > > > same
> > > > > > >> > > > > downloadable
> > > > > > >> > > > > >> tar file but it doesn't really give a very cohesive
> > > > > feeling.
> > > > > > I
> > > > > > >> > agree
> > > > > > >> > > > > that
> > > > > > >> > > > > >> ultimately any of the project stuff is as much
> social
> > > and
> > > > > > naming
> > > > > > >> > as
> > > > > > >> > > > > >> anything else--theoretically two totally
> independent
> > > > > projects
> > > > > > >> > could
> > > > > > >> > > > work
> > > > > > >> > > > > >> to
> > > > > > >> > > > > >> tightly align. In practice this seems to be quite
> > > > difficult
> > > > > > >> > though.
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> For the frameworks--totally agree it would be good
> to
> > > > > > maintain
> > > > > > >> the
> > > > > > >> > > > > >> framework support with the project. In some cases
> > there
> > > > may
> > > > > > not
> > > > > > >> be
> > > > > > >> > > too
> > > > > > >> > > > > >> much
> > > > > > >> > > > > >> there since the integration gets lighter but I
> think
> > > > > whatever
> > > > > > >> > stubs
> > > > > > >> > > > you
> > > > > > >> > > > > >> need should be included. So no I definitely wasn't
> > > trying
> > > > > to
> > > > > > >> imply
> > > > > > >> > > > > >> dropping
> > > > > > >> > > > > >> support for these frameworks, just making the
> > > integration
> > > > > > >> lighter
> > > > > > >> > by
> > > > > > >> > > > > >> separating process management from partition
> > > management.
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> You raise two good points we would have to figure
> out
> > > if
> > > > we
> > > > > > went
> > > > > > >> > > down
> > > > > > >> > > > > the
> > > > > > >> > > > > >> alignment path:
> > > > > > >> > > > > >> 1. With respect to the name, yeah I think the first
> > > > > question
> > > > > > is
> > > > > > >> > > > whether
> > > > > > >> > > > > >> some "re-branding" would be worth it. If so then I
> > > think
> > > > we
> > > > > > can
> > > > > > >> > > have a
> > > > > > >> > > > > big
> > > > > > >> > > > > >> thread on the name. I'm definitely not set on Kafka
> > > > > > Streaming or
> > > > > > >> > > Kafka
> > > > > > >> > > > > >> Streams I was just using them to be kind of
> > > > illustrative. I
> > > > > > >> agree
> > > > > > >> > > with
> > > > > > >> > > > > >> your
> > > > > > >> > > > > >> critique of these names, though I think people
> would
> > > get
> > > > > the
> > > > > > >> idea.
> > > > > > >> > > > > >> 2. Yeah you also raise a good point about how to
> > > "factor"
> > > > > it.
> > > > > > >> Here
> > > > > > >> > > are
> > > > > > >> > > > > the
> > > > > > >> > > > > >> options I see (I could get enthusiastic about any
> of
> > > > them):
> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > > > > > >> > > > > >>    b. Two repos, retaining the current seperation
> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api and
> > > > samza-core
> > > > > > is
> > > > > > >> > > > absorbed
> > > > > > >> > > > > >> almost like a third client
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> Cheers,
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> -Jay
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> > > > > > >> > > > martin@kleppmann.com>
> > > > > > >> > > > > >> wrote:
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a few
> > > follow-up
> > > > > > >> > comments.
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
> > becoming
> > > a
> > > > > > >> > subproject:
> > > > > > >> > > > the
> > > > > > >> > > > > >> > reasons you mention are good. The risk I see is
> > that
> > > > > > release
> > > > > > >> > > > schedules
> > > > > > >> > > > > >> > become coupled to each other, which can slow
> > everyone
> > > > > down,
> > > > > > >> and
> > > > > > >> > > > large
> > > > > > >> > > > > >> > projects with many contributors are harder to
> > manage.
> > > > > > (Jakob,
> > > > > > >> > can
> > > > > > >> > > > you
> > > > > > >> > > > > >> speak
> > > > > > >> > > > > >> > from experience, having seen a wider range of
> > Hadoop
> > > > > > ecosystem
> > > > > > >> > > > > >> projects?)
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > Some of the goals of a better unified developer
> > > > > experience
> > > > > > >> could
> > > > > > >> > > > also
> > > > > > >> > > > > be
> > > > > > >> > > > > >> > solved by integrating Samza nicely into a Kafka
> > > > > > distribution
> > > > > > >> > (such
> > > > > > >> > > > as
> > > > > > >> > > > > >> > Confluent's). I'm not against merging projects if
> > we
> > > > > decide
> > > > > > >> > that's
> > > > > > >> > > > the
> > > > > > >> > > > > >> way
> > > > > > >> > > > > >> > to go, just pointing out the same goals can
> perhaps
> > > > also
> > > > > be
> > > > > > >> > > achieved
> > > > > > >> > > > > in
> > > > > > >> > > > > >> > other ways.
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > - With regard to dropping the YARN dependency:
> are
> > > you
> > > > > > >> proposing
> > > > > > >> > > > that
> > > > > > >> > > > > >> > Samza doesn't give any help to people wanting to
> > run
> > > on
> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > > > > > >> > > > > >> > So the docs would basically have a link to Slider
> > and
> > > > > > nothing
> > > > > > >> > > else?
> > > > > > >> > > > Or
> > > > > > >> > > > > >> > would we maintain integrations with a bunch of
> > > popular
> > > > > > >> > deployment
> > > > > > >> > > > > >> methods
> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts to
> make
> > > > Samza
> > > > > > work
> > > > > > >> > with
> > > > > > >> > > > > >> Slider)?
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > I absolutely think it's a good idea to have the
> > "as a
> > > > > > library"
> > > > > > >> > and
> > > > > > >> > > > > "as a
> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for people
> > who
> > > > > want
> > > > > > >> them,
> > > > > > >> > > > but I
> > > > > > >> > > > > >> > think there should also be a low-friction path
> for
> > > > common
> > > > > > "as
> > > > > > >> a
> > > > > > >> > > > > service"
> > > > > > >> > > > > >> > deployment methods, for which we probably need to
> > > > > maintain
> > > > > > >> > > > > integrations.
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to
> me,
> > > > > because
> > > > > > >> Kafka
> > > > > > >> > > is
> > > > > > >> > > > > all
> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> Transformers"
> > > or
> > > > > > "Kafka
> > > > > > >> > > > Filters"
> > > > > > >> > > > > >> > would be more apt?
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza (stream
> > > > > > >> transformation
> > > > > > >> > > > with
> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a library"
> > > bit)
> > > > > > could
> > > > > > >> > > become
> > > > > > >> > > > > >> part of
> > > > > > >> > > > > >> > Kafka, while higher-level tools such as streaming
> > SQL
> > > > and
> > > > > > >> > > > integrations
> > > > > > >> > > > > >> with
> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> project?
> > > In
> > > > > > other
> > > > > > >> > > words,
> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > >> > would absorb the proven, stable core of Samza,
> > which
> > > > > would
> > > > > > >> > become
> > > > > > >> > > > the
> > > > > > >> > > > > >> > "third Kafka client" mentioned early in this
> > thread.
> > > > The
> > > > > > Samza
> > > > > > >> > > > project
> > > > > > >> > > > > >> > would then target that third Kafka client as its
> > base
> > > > > API,
> > > > > > and
> > > > > > >> > the
> > > > > > >> > > > > >> project
> > > > > > >> > > > > >> > would be freed up to explore more experimental
> new
> > > > > > horizons.
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > Martin
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > > > jay.kreps@gmail.com>
> > > > > > >> wrote:
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > > Hey Martin,
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually
> > don't
> > > > > think
> > > > > > it
> > > > > > >> > ties
> > > > > > >> > > > our
> > > > > > >> > > > > >> > hands
> > > > > > >> > > > > >> > > at all, all it does is refactor things. The
> > > division
> > > > of
> > > > > > >> > > > > >> responsibility is
> > > > > > >> > > > > >> > > that Samza core is responsible for task
> > lifecycle,
> > > > > state,
> > > > > > >> and
> > > > > > >> > > > > >> partition
> > > > > > >> > > > > >> > > management (using the Kafka co-ordinator) but
> it
> > is
> > > > NOT
> > > > > > >> > > > responsible
> > > > > > >> > > > > >> for
> > > > > > >> > > > > >> > > packaging, configuration deployment or
> execution
> > of
> > > > > > >> processes.
> > > > > > >> > > The
> > > > > > >> > > > > >> > problem
> > > > > > >> > > > > >> > > of packaging and starting these processes is
> > > > > > >> > > > > >> > > framework/environment-specific. This leaves
> > > > individual
> > > > > > >> > > frameworks
> > > > > > >> > > > to
> > > > > > >> > > > > >> be
> > > > > > >> > > > > >> > as
> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you can get
> > > simple
> > > > > > >> stateless
> > > > > > >> > > > > >> support in
> > > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app
> > > > > framework
> > > > > > >> > > (Slider,
> > > > > > >> > > > > >> > Marathon,
> > > > > > >> > > > > >> > > etc). These are well known by people and have
> > nice
> > > > UIs
> > > > > > and a
> > > > > > >> > lot
> > > > > > >> > > > of
> > > > > > >> > > > > >> > > flexibility. I don't think they have node
> > affinity
> > > > as a
> > > > > > >> built
> > > > > > >> > in
> > > > > > >> > > > > >> option
> > > > > > >> > > > > >> > > (though I could be wrong). So if we want that
> we
> > > can
> > > > > > either
> > > > > > >> > wait
> > > > > > >> > > > for
> > > > > > >> > > > > >> them
> > > > > > >> > > > > >> > > to add it or do a custom framework to add that
> > > > feature
> > > > > > (as
> > > > > > >> > now).
> > > > > > >> > > > > >> > Obviously
> > > > > > >> > > > > >> > > if you manage things with old-school ops tools
> > > > > > >> > (puppet/chef/etc)
> > > > > > >> > > > you
> > > > > > >> > > > > >> get
> > > > > > >> > > > > >> > > locality easily. The nice thing, though, is
> that
> > > all
> > > > > the
> > > > > > >> samza
> > > > > > >> > > > > >> "business
> > > > > > >> > > > > >> > > logic" around partition management and fault
> > > > tolerance
> > > > > > is in
> > > > > > >> > > Samza
> > > > > > >> > > > > >> core
> > > > > > >> > > > > >> > so
> > > > > > >> > > > > >> > > it is shared across frameworks and the
> framework
> > > > > specific
> > > > > > >> bit
> > > > > > >> > is
> > > > > > >> > > > > just
> > > > > > >> > > > > >> > > whether it is smart enough to try to get the
> same
> > > > host
> > > > > > when
> > > > > > >> a
> > > > > > >> > > job
> > > > > > >> > > > is
> > > > > > >> > > > > >> > > restarted.
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah I
> think
> > > the
> > > > > > goal
> > > > > > >> > would
> > > > > > >> > > > be
> > > > > > >> > > > > >> (a)
> > > > > > >> > > > > >> > > actually get better alignment in user
> experience,
> > > and
> > > > > (b)
> > > > > > >> > > express
> > > > > > >> > > > > >> this in
> > > > > > >> > > > > >> > > the naming and project branding. Specifically:
> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> > > > > > "transformation"
> > > > > > >> api
> > > > > > >> > > to
> > > > > > >> > > > be
> > > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be
> able
> > > to
> > > > > > explain
> > > > > > >> > > when
> > > > > > >> > > > to
> > > > > > >> > > > > >> use
> > > > > > >> > > > > >> > > the consumer and when to use the stream
> > processing
> > > > > > >> > functionality
> > > > > > >> > > > and
> > > > > > >> > > > > >> lead
> > > > > > >> > > > > >> > > people into that experience.
> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2
> (or
> > > > > > whatever)
> > > > > > >> > that
> > > > > > >> > > > has
> > > > > > >> > > > > >> both
> > > > > > >> > > > > >> > > Kafka and the stream processing part and they
> > > > actually
> > > > > > work
> > > > > > >> > > > > together.
> > > > > > >> > > > > >> > > 3. Unify the programming experience so the
> client
> > > and
> > > > > > Samza
> > > > > > >> > api
> > > > > > >> > > > > share
> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > I think sub-projects keep separate committers
> and
> > > can
> > > > > > have a
> > > > > > >> > > > > separate
> > > > > > >> > > > > >> > repo,
> > > > > > >> > > > > >> > > but I'm actually not really sure (I can't find
> a
> > > > > > definition
> > > > > > >> > of a
> > > > > > >> > > > > >> > subproject
> > > > > > >> > > > > >> > > in Apache).
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > Basically at a high-level you want the
> experience
> > > to
> > > > > > "feel"
> > > > > > >> > > like a
> > > > > > >> > > > > >> single
> > > > > > >> > > > > >> > > system, not to relatively independent things
> that
> > > are
> > > > > > kind
> > > > > > >> of
> > > > > > >> > > > > >> awkwardly
> > > > > > >> > > > > >> > > glued together.
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > I think if we did that they having naming or
> > > branding
> > > > > > like
> > > > > > >> > > "kafka
> > > > > > >> > > > > >> > > streaming" or "kafka streams" or something like
> > > that
> > > > > > would
> > > > > > >> > > > actually
> > > > > > >> > > > > >> do a
> > > > > > >> > > > > >> > > good job of conveying what it is. I do that
> this
> > > > would
> > > > > > help
> > > > > > >> > > > adoption
> > > > > > >> > > > > >> > quite
> > > > > > >> > > > > >> > > a lot as it would correctly convey that using
> > Kafka
> > > > > > >> Streaming
> > > > > > >> > > with
> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > >> > is
> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
> pretty
> > > > > heavily
> > > > > > >> > adopted
> > > > > > >> > > > at
> > > > > > >> > > > > >> this
> > > > > > >> > > > > >> > > point.
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> originally
> > > > when
> > > > > > open
> > > > > > >> > > > sourcing
> > > > > > >> > > > > >> > Samza,
> > > > > > >> > > > > >> > > however at that time Kafka was relatively
> unknown
> > > and
> > > > > we
> > > > > > >> > decided
> > > > > > >> > > > not
> > > > > > >> > > > > >> to
> > > > > > >> > > > > >> > do
> > > > > > >> > > > > >> > > it since we felt it would be limiting. From my
> > > point
> > > > of
> > > > > > view
> > > > > > >> > the
> > > > > > >> > > > > three
> > > > > > >> > > > > >> > > things have changed (1) Kafka is now really
> > heavily
> > > > > used
> > > > > > for
> > > > > > >> > > > stream
> > > > > > >> > > > > >> > > processing, (2) we learned that abstracting out
> > the
> > > > > > stream
> > > > > > >> > well
> > > > > > >> > > is
> > > > > > >> > > > > >> > > basically impossible, (3) we learned it is
> really
> > > > hard
> > > > > to
> > > > > > >> keep
> > > > > > >> > > the
> > > > > > >> > > > > two
> > > > > > >> > > > > >> > > things feeling like a single product.
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > -Jay
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> Kleppmann
> > <
> > > > > > >> > > > > >> martin@kleppmann.com>
> > > > > > >> > > > > >> > > wrote:
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > >> Hi all,
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> I agree with the general philosophy of tying
> > Samza
> > > > > more
> > > > > > >> > firmly
> > > > > > >> > > to
> > > > > > >> > > > > >> Kafka.
> > > > > > >> > > > > >> > >> After I spent a while looking at integrating
> > other
> > > > > > message
> > > > > > >> > > > brokers
> > > > > > >> > > > > >> (e.g.
> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
> > > > conclusion
> > > > > > that
> > > > > > >> > > > > >> > SystemConsumer
> > > > > > >> > > > > >> > >> tacitly assumes a model so much like Kafka's
> > that
> > > > > pretty
> > > > > > >> much
> > > > > > >> > > > > nobody
> > > > > > >> > > > > >> but
> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
> > perhaps
> > > an
> > > > > > >> > exception,
> > > > > > >> > > > but
> > > > > > >> > > > > >> it
> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus,
> > > making
> > > > > > Samza
> > > > > > >> > > fully
> > > > > > >> > > > > >> > dependent
> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > system-independence
> > > > was
> > > > > > >> never
> > > > > > >> > as
> > > > > > >> > > > > real
> > > > > > >> > > > > >> as
> > > > > > >> > > > > >> > we
> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of code
> > reuse
> > > > are
> > > > > > >> real.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN has
> also
> > > > always
> > > > > > been
> > > > > > >> > > > > >> appealing to
> > > > > > >> > > > > >> > >> me, for various reasons already mentioned in
> > this
> > > > > > thread.
> > > > > > >> > > > Although
> > > > > > >> > > > > >> > making
> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > > > (YARN/Mesos/AWS/etc)
> > > > > > >> seems
> > > > > > >> > > > > >> laudable,
> > > > > > >> > > > > >> > I am
> > > > > > >> > > > > >> > >> a little concerned that it will restrict us
> to a
> > > > > lowest
> > > > > > >> > common
> > > > > > >> > > > > >> > denominator.
> > > > > > >> > > > > >> > >> For example, would host affinity (SAMZA-617)
> > still
> > > > be
> > > > > > >> > possible?
> > > > > > >> > > > For
> > > > > > >> > > > > >> jobs
> > > > > > >> > > > > >> > >> with large amounts of state, I think SAMZA-617
> > > would
> > > > > be
> > > > > > a
> > > > > > >> big
> > > > > > >> > > > boon,
> > > > > > >> > > > > >> > since
> > > > > > >> > > > > >> > >> restoring state off the changelog on every
> > single
> > > > > > restart
> > > > > > >> is
> > > > > > >> > > > > painful,
> > > > > > >> > > > > >> > due
> > > > > > >> > > > > >> > >> to long recovery times. It would be a shame if
> > the
> > > > > > >> decoupling
> > > > > > >> > > > from
> > > > > > >> > > > > >> YARN
> > > > > > >> > > > > >> > >> made host affinity impossible.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> Jay, a question about the proposed API for
> > > > > > instantiating a
> > > > > > >> > job
> > > > > > >> > > in
> > > > > > >> > > > > >> code
> > > > > > >> > > > > >> > >> (rather than a properties file): when
> > submitting a
> > > > job
> > > > > > to a
> > > > > > >> > > > > cluster,
> > > > > > >> > > > > >> is
> > > > > > >> > > > > >> > the
> > > > > > >> > > > > >> > >> idea that the instantiation code runs on a
> > client
> > > > > > >> somewhere,
> > > > > > >> > > > which
> > > > > > >> > > > > >> then
> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > > YARN/Mesos/AWS/etc?
> > > > > Or
> > > > > > >> does
> > > > > > >> > > that
> > > > > > >> > > > > >> code
> > > > > > >> > > > > >> > run
> > > > > > >> > > > > >> > >> on each container that is part of the job (in
> > > which
> > > > > > case,
> > > > > > >> how
> > > > > > >> > > > does
> > > > > > >> > > > > >> the
> > > > > > >> > > > > >> > job
> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel right
> to
> > > > make
> > > > > a
> > > > > > 1.0
> > > > > > >> > > > release
> > > > > > >> > > > > >> > with a
> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete. So if
> > this
> > > > is
> > > > > > going
> > > > > > >> > to
> > > > > > >> > > > > >> happen, I
> > > > > > >> > > > > >> > >> think it would be more honest to stick with
> 0.*
> > > > > version
> > > > > > >> > numbers
> > > > > > >> > > > > until
> > > > > > >> > > > > >> > the
> > > > > > >> > > > > >> > >> library-ified Samza has been implemented, is
> > > stable
> > > > > and
> > > > > > >> > widely
> > > > > > >> > > > > used.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of Kafka?
> > > There
> > > > > is
> > > > > > >> > > precedent
> > > > > > >> > > > > for
> > > > > > >> > > > > >> > >> tight coupling between different Apache
> projects
> > > > (e.g.
> > > > > > >> > Curator
> > > > > > >> > > > and
> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think
> > > remaining
> > > > > > >> separate
> > > > > > >> > > > would
> > > > > > >> > > > > >> be
> > > > > > >> > > > > >> > ok.
> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on Kafka,
> there
> > > is
> > > > > > enough
> > > > > > >> > > > > substance
> > > > > > >> > > > > >> in
> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> project.
> > > An
> > > > > > >> argument
> > > > > > >> > in
> > > > > > >> > > > > >> favour
> > > > > > >> > > > > >> > of
> > > > > > >> > > > > >> > >> merging would be if we think Kafka has a much
> > > > stronger
> > > > > > >> "brand
> > > > > > >> > > > > >> presence"
> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If the
> > > Kafka
> > > > > > >> project
> > > > > > >> > is
> > > > > > >> > > > > >> willing
> > > > > > >> > > > > >> > to
> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of doing
> > > > stateful
> > > > > > >> stream
> > > > > > >> > > > > >> > >> transformations, that would probably have much
> > the
> > > > > same
> > > > > > >> > effect
> > > > > > >> > > as
> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream Processors"
> > or
> > > > > > suchlike.
> > > > > > >> > > Close
> > > > > > >> > > > > >> > >> collaboration between the two projects will be
> > > > needed
> > > > > in
> > > > > > >> any
> > > > > > >> > > > case.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> From a project management perspective, I guess
> > the
> > > > > "new
> > > > > > >> > Samza"
> > > > > > >> > > > > would
> > > > > > >> > > > > >> > have
> > > > > > >> > > > > >> > >> to be developed on a branch alongside ongoing
> > > > > > maintenance
> > > > > > >> of
> > > > > > >> > > the
> > > > > > >> > > > > >> current
> > > > > > >> > > > > >> > >> line of development? I think it would be
> > important
> > > > to
> > > > > > >> > continue
> > > > > > >> > > > > >> > supporting
> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> migration
> > > > path
> > > > > to
> > > > > > >> the
> > > > > > >> > > new
> > > > > > >> > > > > >> > version.
> > > > > > >> > > > > >> > >> Leaving the current versions unsupported and
> > > forcing
> > > > > > people
> > > > > > >> > to
> > > > > > >> > > > > >> rewrite
> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> Best,
> > > > > > >> > > > > >> > >> Martin
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > > > jay@confluent.io>
> > > > > > >> wrote:
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >>> Hey Garry,
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy
> to
> > > chat
> > > > > > more
> > > > > > >> > about
> > > > > > >> > > > > this
> > > > > > >> > > > > >> if
> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
> > started
> > > > with
> > > > > > the
> > > > > > >> > idea
> > > > > > >> > > > of
> > > > > > >> > > > > >> "what
> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> > ingestion
> > > > > tool"
> > > > > > but
> > > > > > >> > > > > >> ultimately
> > > > > > >> > > > > >> > we
> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> ingestion
> > > and
> > > > > > >> > > > transformation
> > > > > > >> > > > > >> had
> > > > > > >> > > > > >> > >>> pretty different needs and coupling the two
> > made
> > > > > things
> > > > > > >> > hard.
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>> For what it's worth I think copycat (KIP-26)
> > > > actually
> > > > > > will
> > > > > > >> > do
> > > > > > >> > > > what
> > > > > > >> > > > > >> you
> > > > > > >> > > > > >> > >> are
> > > > > > >> > > > > >> > >>> looking for.
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>> With regard to your point about slider, I
> don't
> > > > > > >> necessarily
> > > > > > >> > > > > >> disagree.
> > > > > > >> > > > > >> > >> But I
> > > > > > >> > > > > >> > >>> think getting good YARN support is quite
> doable
> > > > and I
> > > > > > >> think
> > > > > > >> > we
> > > > > > >> > > > can
> > > > > > >> > > > > >> make
> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> proposal
> > > > > solves
> > > > > > is
> > > > > > >> > that
> > > > > > >> > > > > >> > >> technically
> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple cluster
> > > > > > management
> > > > > > >> > > systems
> > > > > > >> > > > > the
> > > > > > >> > > > > >> > way
> > > > > > >> > > > > >> > >>> things are now, you need to write an "app
> > master"
> > > > or
> > > > > > >> > > "framework"
> > > > > > >> > > > > for
> > > > > > >> > > > > >> > each
> > > > > > >> > > > > >> > >>> and they are all a little different so
> testing
> > is
> > > > > > really
> > > > > > >> > hard.
> > > > > > >> > > > In
> > > > > > >> > > > > >> the
> > > > > > >> > > > > >> > >>> absence of this we have been stuck with just
> > YARN
> > > > > which
> > > > > > >> has
> > > > > > >> > > > > >> fantastic
> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the org,
> but
> > > > zero
> > > > > > >> > > penetration
> > > > > > >> > > > > >> > >> elsewhere.
> > > > > > >> > > > > >> > >>> Given the huge amount of work being put in to
> > > > slider,
> > > > > > >> > > marathon,
> > > > > > >> > > > > aws
> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen related
> > > > packaging
> > > > > > >> > > > technologies
> > > > > > >> > > > > >> > people
> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> > > > > cloud-specific
> > > > > > >> > deploy
> > > > > > >> > > > > >> tools,
> > > > > > >> > > > > >> > >> etc)
> > > > > > >> > > > > >> > >>> I really think it is important to get this
> > right.
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>> -Jay
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> > Turkington
> > > <
> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>>> Hi all,
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> I think the question below re does Samza
> > become
> > > a
> > > > > > >> > sub-project
> > > > > > >> > > > of
> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > >> > >>>> highlights the broader point around
> migration.
> > > > Chris
> > > > > > >> > mentions
> > > > > > >> > > > > >> Samza's
> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1 release but
> > I'm
> > > > not
> > > > > > sure
> > > > > > >> > it
> > > > > > >> > > > > feels
> > > > > > >> > > > > >> > >> right to
> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> deprecate
> > > > most
> > > > > of
> > > > > > >> it.
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> From a selfish perspective I have some guys
> > who
> > > > have
> > > > > > >> > started
> > > > > > >> > > > > >> working
> > > > > > >> > > > > >> > >> with
> > > > > > >> > > > > >> > >>>> Samza and building some new
> > consumers/producers
> > > > was
> > > > > > next
> > > > > > >> > up.
> > > > > > >> > > > > Sounds
> > > > > > >> > > > > >> > like
> > > > > > >> > > > > >> > >>>> that is absolutely not the direction to go.
> I
> > > need
> > > > > to
> > > > > > >> look
> > > > > > >> > > into
> > > > > > >> > > > > the
> > > > > > >> > > > > >> > KIP
> > > > > > >> > > > > >> > >> in
> > > > > > >> > > > > >> > >>>> more detail but for me the attractiveness of
> > > > adding
> > > > > > new
> > > > > > >> > Samza
> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all they
> > were
> > > > > doing
> > > > > > was
> > > > > > >> > > > really
> > > > > > >> > > > > >> > getting
> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to avoid
> > > > having
> > > > > to
> > > > > > >> > worry
> > > > > > >> > > > > about
> > > > > > >> > > > > >> the
> > > > > > >> > > > > >> > >>>> lifecycle management of external clients. If
> > > there
> > > > > is
> > > > > > a
> > > > > > >> > > generic
> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a new
> > > > connector
> > > > > > into
> > > > > > >> > and
> > > > > > >> > > > > have
> > > > > > >> > > > > >> a
> > > > > > >> > > > > >> > >> lot of
> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and reliability
> > done
> > > > for
> > > > > me
> > > > > > >> then
> > > > > > >> > > it
> > > > > > >> > > > > >> gives
> > > > > > >> > > > > >> > me
> > > > > > >> > > > > >> > >> all
> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers would.
> If
> > > not
> > > > > > then it
> > > > > > >> > > > > >> complicates
> > > > > > >> > > > > >> > my
> > > > > > >> > > > > >> > >>>> operational deployments.
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> Which is similar to my other question with
> the
> > > > > > proposal
> > > > > > >> --
> > > > > > >> > if
> > > > > > >> > > > we
> > > > > > >> > > > > >> > build a
> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus the
> > > > requisite
> > > > > > >> shims
> > > > > > >> > to
> > > > > > >> > > > > >> > integrate
> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former may be
> a
> > > lot
> > > > > more
> > > > > > >> work
> > > > > > >> > > > than
> > > > > > >> > > > > we
> > > > > > >> > > > > >> > >> think.
> > > > > > >> > > > > >> > >>>> We may make it much easier for a newcomer to
> > get
> > > > > > >> something
> > > > > > >> > > > > running
> > > > > > >> > > > > >> but
> > > > > > >> > > > > >> > >>>> having them step up and get a reliable
> > > production
> > > > > > >> > deployment
> > > > > > >> > > > may
> > > > > > >> > > > > >> still
> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
> > different
> > > > > > reasons
> > > > > > >> > than
> > > > > > >> > > > > >> today.
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with
> > > making
> > > > > the
> > > > > > >> Samza
> > > > > > >> > > > > >> dependency
> > > > > > >> > > > > >> > >> on
> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I absolutely
> see
> > > the
> > > > > > >> benefits
> > > > > > >> > > in
> > > > > > >> > > > > the
> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > > > > > >> > > > terminologies/abstractions
> > > > > > >> > > > > >> that
> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library would
> > > > likely
> > > > > > be a
> > > > > > >> > very
> > > > > > >> > > > > nice
> > > > > > >> > > > > >> > tool
> > > > > > >> > > > > >> > >> to
> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the
> > > > concerns
> > > > > > >> above
> > > > > > >> > re
> > > > > > >> > > > the
> > > > > > >> > > > > >> > >>>> operational side.
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> Garry
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> [mailto:
> > > > > > >> > gdfm@apache.org
> > > > > > >> > > ]
> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on
> > Samza
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > > > > > >> > > > > >> > >>>> From outside, I have always perceived Samza
> > as a
> > > > > > >> computing
> > > > > > >> > > > layer
> > > > > > >> > > > > >> over
> > > > > > >> > > > > >> > >>>> Kafka.
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> The question, maybe a bit provocative, is
> > > "should
> > > > > > Samza
> > > > > > >> be
> > > > > > >> > a
> > > > > > >> > > > > >> > sub-project
> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> separate
> > > > > project
> > > > > > >> > with a
> > > > > > >> > > > > >> separate
> > > > > > >> > > > > >> > >>>> governance?
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> Cheers,
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> --
> > > > > > >> > > > > >> > >>>> Gianmarco
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > > > > > yanfang724@gmail.com>
> > > > > > >> > > > wrote:
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more
> > > > tightly.
> > > > > > >> > Because
> > > > > > >> > > > > Samza
> > > > > > >> > > > > >> de
> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
> > leverage
> > > > > what
> > > > > > >> Kafka
> > > > > > >> > > > has.
> > > > > > >> > > > > At
> > > > > > >> > > > > >> > the
> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to reinvent
> > what
> > > > > Samza
> > > > > > >> > > already
> > > > > > >> > > > > >> has. I
> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> > ingestion
> > > > and
> > > > > > >> > > > > transformation.
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
> image
> > > how
> > > > > the
> > > > > > >> Samza
> > > > > > >> > > > will
> > > > > > >> > > > > >> look
> > > > > > >> > > > > >> > >>>> like.
> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> > > difference
> > > > > in
> > > > > > >> terms
> > > > > > >> > > of
> > > > > > >> > > > > how
> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code shows
> > (A
> > > > > > client of
> > > > > > >> > > > Kakfa)
> > > > > > >> > > > > ?
> > > > > > >> > > > > >> And
> > > > > > >> > > > > >> > >>>>> user's application code calls this client?
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka
> > (like
> > > > > what
> > > > > > the
> > > > > > >> > > code
> > > > > > >> > > > > >> shows),
> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > > > > fault-tolerance?
> > > > > > >> Are
> > > > > > >> > > they
> > > > > > >> > > > > >> taken
> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> mechanism,
> > > such
> > > > > as
> > > > > > >> > "Samza
> > > > > > >> > > > > >> worker"
> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> 2. What about other features, such as
> > > > auto-scaling,
> > > > > > >> shared
> > > > > > >> > > > > state,
> > > > > > >> > > > > >> > >>>>> monitoring?
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is this
> > what
> > > > > Chris
> > > > > > >> > > > suggests?)
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa
> > and
> > > > > > produce
> > > > > > >> to
> > > > > > >> > > it.
> > > > > > >> > > > > >> Then it
> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks like
> > now,
> > > > > > except it
> > > > > > >> > > does
> > > > > > >> > > > > not
> > > > > > >> > > > > >> > rely
> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it leverage
> > > > Kafka's
> > > > > > >> > metrics,
> > > > > > >> > > > > logs,
> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> Thanks,
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang
> > Wang <
> > > > > > >> > > > > wangguoz@gmail.com
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > >>>> wrote:
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>>> Read through the code example and it looks
> > > good
> > > > to
> > > > > > me.
> > > > > > >> A
> > > > > > >> > > few
> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable runnable
> > > like:
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > > --config-factory=...
> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for deploying
> > Samza
> > > > > more
> > > > > > as
> > > > > > >> > > > embedded
> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> (ignoring
> > > the
> > > > > > >> > > terminology
> > > > > > >> > > > > >> since
> > > > > > >> > > > > >> > >>>>>> it is not the
> > > > > > >> > > > > >> > >>>>> same
> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> MyStreamTask(configs);
> > > > > Thread
> > > > > > >> > thread
> > > > > > >> > > =
> > > > > > >> > > > > new
> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> I think both of these deployment modes are
> > > > > important
> > > > > > >> for
> > > > > > >> > > > > >> different
> > > > > > >> > > > > >> > >>>>>> types
> > > > > > >> > > > > >> > >>>>> of
> > > > > > >> > > > > >> > >>>>>> users. That said, I think making Samza
> > purely
> > > > > > >> standalone
> > > > > > >> > is
> > > > > > >> > > > > still
> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or library
> > > modes.
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> Guozhang
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
> Kreps
> > <
> > > > > > >> > > > jay@confluent.io>
> > > > > > >> > > > > >> > wrote:
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> example,
> > it
> > > > was
> > > > > > >> > supposed
> > > > > > >> > > > to
> > > > > > >> > > > > >> look
> > > > > > >> > > > > >> > >>>>>>> like
> > > > > > >> > > > > >> > >>>>>>> this:
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > > > "localhost:4242");
> > > > > > >> > > > > >> StreamingConfig
> > > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > > "test-topic-2");
> > > > > > >> > > > > >> > >>>>>>>
> > > config.processor(ExampleStreamProcessor.class);
> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > StringSerializer(),
> > > > new
> > > > > > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming
> > > > container =
> > > > > > new
> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>> -Jay
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay
> > Kreps <
> > > > > > >> > > > jay@confluent.io
> > > > > > >> > > > > >
> > > > > > >> > > > > >> > >>>> wrote:
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
> Chris
> > > and
> > > > I
> > > > > > were
> > > > > > >> > > having
> > > > > > >> > > > > >> > >>>>>>>> around
> > > > > > >> > > > > >> > >>>>>>> whether
> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a
> kind
> > > of
> > > > > data
> > > > > > >> > > > ingestion
> > > > > > >> > > > > >> > >>>>> framework
> > > > > > >> > > > > >> > >>>>>>> for
> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26
> > > > > "copycat").
> > > > > > >> This
> > > > > > >> > > > kind
> > > > > > >> > > > > of
> > > > > > >> > > > > >> > >>>>>> combined
> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and YARN
> and
> > > the
> > > > > > >> > discussion
> > > > > > >> > > > > >> around
> > > > > > >> > > > > >> > >>>>>>>> how
> > > > > > >> > > > > >> > >>>>> to
> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was, given
> that
> > > > Samza
> > > > > > was
> > > > > > >> > > > basically
> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific, what if
> > you
> > > > just
> > > > > > >> > embraced
> > > > > > >> > > > > that
> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > > > > > >> > > > > >> > >>>>>> into
> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> > framework
> > > > and
> > > > > > more
> > > > > > >> > > like a
> > > > > > >> > > > > >> > >>>>>>>> third
> > > > > > >> > > > > >> > >>>>> Kafka
> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing consumer"
> > with
> > > > > state
> > > > > > >> > > > management
> > > > > > >> > > > > >> > >>>>>> facilities.
> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> complex
> > > > stream
> > > > > > >> > > processing
> > > > > > >> > > > > >> > >>>>>>>> framework
> > > > > > >> > > > > >> > >>>>>>> this
> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple thing,
> not
> > > > much
> > > > > > more
> > > > > > >> > > > > >> complicated
> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > > >> > > > > >> > >>>>> use
> > > > > > >> > > > > >> > >>>>>>> or
> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris
> > said
> > > > we
> > > > > > >> thought
> > > > > > >> > > > about
> > > > > > >> > > > > >> it
> > > > > > >> > > > > >> > >>>>>>>> a
> > > > > > >> > > > > >> > >>>>> lot
> > > > > > >> > > > > >> > >>>>>> of
> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> > processing
> > > > > > systems
> > > > > > >> > were
> > > > > > >> > > > > doing)
> > > > > > >> > > > > >> > >>>>> seemed
> > > > > > >> > > > > >> > >>>>>>> like
> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output data
> > to
> > > > and
> > > > > > from
> > > > > > >> > the
> > > > > > >> > > > > stream
> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually looked
> > into
> > > > how
> > > > > > that
> > > > > > >> > > would
> > > > > > >> > > > > >> > >>>>>>>> work,
> > > > > > >> > > > > >> > >>>>> Samza
> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion
> > > framework
> > > > > > for a
> > > > > > >> > > bunch
> > > > > > >> > > > of
> > > > > > >> > > > > >> > >>>>> reasons.
> > > > > > >> > > > > >> > >>>>>> To
> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a pretty
> > > > different
> > > > > > >> > internal
> > > > > > >> > > > > data
> > > > > > >> > > > > >> > >>>>>>>> model
> > > > > > >> > > > > >> > >>>>>> and
> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split them
> and
> > > had
> > > > > an
> > > > > > api
> > > > > > >> > for
> > > > > > >> > > > > Kafka
> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and
> a
> > > > > separate
> > > > > > >> api
> > > > > > >> > > for
> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> This would also allow really embracing
> the
> > > > same
> > > > > > >> > > terminology
> > > > > > >> > > > > and
> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about the
> > current
> > > > > > state is
> > > > > > >> > > that
> > > > > > >> > > > > the
> > > > > > >> > > > > >> > >>>>>>>> two
> > > > > > >> > > > > >> > >>>>>>> systems
> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like
> > > > > "stream"
> > > > > > vs
> > > > > > >> > > > "topic"
> > > > > > >> > > > > >> and
> > > > > > >> > > > > >> > >>>>>>> different
> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means you
> > kind
> > > > of
> > > > > > have
> > > > > > >> to
> > > > > > >> > > > learn
> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > > > > > >> > > > > >> > >>>>>>> way,
> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly different
> way,
> > > > then
> > > > > > kind
> > > > > > >> of
> > > > > > >> > > > > >> > >>>>>>>> understand
> > > > > > >> > > > > >> > >>>>> how
> > > > > > >> > > > > >> > >>>>>>> they
> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having walked a
> > few
> > > > > > people
> > > > > > >> > > through
> > > > > > >> > > > > >> this
> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of time
> > on
> > > > > > >> airplanes I
> > > > > > >> > > > > hacked
> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> incomplete
> > > > > > prototype
> > > > > > >> of
> > > > > > >> > > > what
> > > > > > >> > > > > >> > >>>>>>>> this would
> > > > > > >> > > > > >> > >>>>> look
> > > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
> dumped
> > > into
> > > > > > Kafka
> > > > > > >> as
> > > > > > >> > > it
> > > > > > >> > > > > >> > >>>>>>>> required a
> > > > > > >> > > > > >> > >>>>>> few
> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is the
> > > code:
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> >
> > > > > > >> > > > >
> > > > > > >> >
> > > > > >
> > > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I just
> > > > > liberally
> > > > > > >> > renamed
> > > > > > >> > > > > >> > >>>>>>>> everything
> > > > > > >> > > > > >> > >>>>> to
> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
> regard
> > > for
> > > > > > >> > > > compatibility.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> To use this would be something like
> this:
> > > > > > >> > > > > >> > >>>>>>>> Properties props = new Properties();
> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > > > > "localhost:4242");
> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > > > > > >> > > > > >> config.processor(ExampleStreamProcessor.class);
> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > > StringDeserializer());
> > > > > > >> > > > KafkaStreaming
> > > > > > >> > > > > >> > >>>>>> container =
> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > container.run();
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > > > SamzaContainer;
> > > > > > >> > > > > StreamProcessor
> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the class
> names
> > > in
> > > > a
> > > > > > file
> > > > > > >> > and
> > > > > > >> > > > then
> > > > > > >> > > > > >> > >>>>>>>> having
> > > > > > >> > > > > >> > >>>>>> the
> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
> > > > > instantiate
> > > > > > the
> > > > > > >> > > > > container
> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced over
> > > > however
> > > > > > many
> > > > > > >> > > > > instances
> > > > > > >> > > > > >> > >>>>>>>> of
> > > > > > >> > > > > >> > >>>>> this
> > > > > > >> > > > > >> > >>>>>>> are
> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance
> > dies,
> > > > new
> > > > > > >> tasks
> > > > > > >> > > are
> > > > > > >> > > > > >> added
> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > > >> > > > > >> > >>>>> the
> > > > > > >> > > > > >> > >>>>>>>> existing containers without shutting
> them
> > > > down).
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for running
> > this
> > > > > stuff
> > > > > > in
> > > > > > >> > YARN
> > > > > > >> > > > via
> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
> using
> > > some
> > > > > of
> > > > > > >> their
> > > > > > >> > > > tools
> > > > > > >> > > > > >> > >>>>>>>> but from the
> > > > > > >> > > > > >> > >>>>>> point
> > > > > > >> > > > > >> > >>>>>>> of
> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these stream
> > > > processing
> > > > > > jobs
> > > > > > >> > are
> > > > > > >> > > > > just
> > > > > > >> > > > > >> > >>>>>> stateless
> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and expand
> > and
> > > > > > contract
> > > > > > >> > at
> > > > > > >> > > > > will.
> > > > > > >> > > > > >> > >>>>>>>> There
> > > > > > >> > > > > >> > >>>>> is
> > > > > > >> > > > > >> > >>>>>>> no
> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it
> > would
> > > > get
> > > > > > >> larger
> > > > > > >> > > if
> > > > > > >> > > > we
> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly larger.
> We
> > > > really
> > > > > > do
> > > > > > >> > get a
> > > > > > >> > > > ton
> > > > > > >> > > > > >> > >>>>>>>> of
> > > > > > >> > > > > >> > >>>>>>> leverage
> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> > delegated
> > > to
> > > > > the
> > > > > > >> new
> > > > > > >> > > > > >> consumer.
> > > > > > >> > > > > >> > >>>>> This
> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> > management
> > > > > > strategy
> > > > > > >> > > > > available
> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > > >> > > > > >> > >>>>>> Kafka
> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to Samza
> (and
> > > vice
> > > > > > versa)
> > > > > > >> > and
> > > > > > >> > > > > with
> > > > > > >> > > > > >> > >>>>>>>> the
> > > > > > >> > > > > >> > >>>>>>> exact
> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as state
> > reuse
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
> > thought
> > > > > > >> provoking.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris
> > > > > Riccomini <
> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with Samza
> > > > > engineers
> > > > > > at
> > > > > > >> > > > LinkedIn
> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > > >> > > > > >> > >>>>>>> Confluent
> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few observations
> > and
> > > > > would
> > > > > > >> like
> > > > > > >> > to
> > > > > > >> > > > > >> > >>>>>>>>> propose
> > > > > > >> > > > > >> > >>>>> some
> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I want
> to
> > > > call
> > > > > > out
> > > > > > >> > about
> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > > > > > >> > > > > >> > >>>>>> design,
> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic
> > > > deployment
> > > > > > >> system.
> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > > > > >> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer
> > and
> > > > > > Kafka's
> > > > > > >> > > > consumer
> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > > > >> > > > > >> > >>>>> are
> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
> > problems.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are related,
> > but
> > > > I'll
> > > > > > >> > address
> > > > > > >> > > > them
> > > > > > >> > > > > >> in
> > > > > > >> > > > > >> > >>>>> order.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a
> > > > dynamic
> > > > > > >> > > deployment
> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > > > > > >> > > > > >> > >>>>>> such
> > > > > > >> > > > > >> > >>>>>>>>> as
> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially
> built
> > > > > Samza,
> > > > > > we
> > > > > > >> > bet
> > > > > > >> > > > that
> > > > > > >> > > > > >> > >>>>>>>>> there
> > > > > > >> > > > > >> > >>>>>> would
> > > > > > >> > > > > >> > >>>>>>>>> be
> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area, and we
> > > could
> > > > > > >> support
> > > > > > >> > > > them,
> > > > > > >> > > > > >> and
> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > > >> > > > > >> > >>>>>> rest
> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there are
> many
> > > > > > >> variations.
> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > > > > > >> > > > > >> > >>>>>> many
> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start their
> > > > > > processors
> > > > > > >> > like
> > > > > > >> > > > > normal
> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use traditional
> > > > deployment
> > > > > > >> scripts
> > > > > > >> > > > such
> > > > > > >> > > > > as
> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > > > > > >> > > > > >> > >>>>>> Chef,
> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
> system
> > > on
> > > > > > users
> > > > > > >> > makes
> > > > > > >> > > > the
> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really painful
> for
> > > > first
> > > > > > time
> > > > > > >> > > > users.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was
> > > also
> > > > a
> > > > > > bit
> > > > > > >> of
> > > > > > >> > a
> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > > > > > >> > > > > >> > >>>>>> because
> > > > > > >> > > > > >> > >>>>>>>>> of
> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between
> > the
> > > > > > nature of
> > > > > > >> > > batch
> > > > > > >> > > > > >> jobs
> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > > >> > > > > >> > >>>>>>> stream
> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> > > conscious
> > > > > > effort
> > > > > > >> to
> > > > > > >> > > > favor
> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > > >> > > > > >> > >>>>>> Hadoop
> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since
> > it
> > > > > worked
> > > > > > >> and
> > > > > > >> > > was
> > > > > > >> > > > > well
> > > > > > >> > > > > >> > >>>>>>> understood.
> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that batch
> > > jobs
> > > > > > have a
> > > > > > >> > > > definite
> > > > > > >> > > > > >> > >>>>>> beginning,
> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't
> > > > > (usually).
> > > > > > >> This
> > > > > > >> > > > leads
> > > > > > >> > > > > to
> > > > > > >> > > > > >> > >>>>>>>>> a
> > > > > > >> > > > > >> > >>>>> much
> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream
> > > > > processors.
> > > > > > >> You
> > > > > > >> > > > > >> basically
> > > > > > >> > > > > >> > >>>>>>>>> just
> > > > > > >> > > > > >> > >>>>>>> need
> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the processor,
> > and
> > > > > start
> > > > > > >> it.
> > > > > > >> > > The
> > > > > > >> > > > > way
> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no
> > > concept
> > > > > of
> > > > > > a
> > > > > > >> > > cluster
> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > > > > > >> > > > > >> > >>>>>> add
> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> coupling
> > > > Samza
> > > > > > with
> > > > > > >> a
> > > > > > >> > > > > >> scheduler
> > > > > > >> > > > > >> > >>>>>>>>> is
> > > > > > >> > > > > >> > >>>>>> that
> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
> handle
> > > > > > deployment.
> > > > > > >> > > This
> > > > > > >> > > > > >> pulls
> > > > > > >> > > > > >> > >>>>>>>>> in a
> > > > > > >> > > > > >> > >>>>>>> bunch
> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> > > distribution
> > > > > > (config
> > > > > > >> > > > > stream),
> > > > > > >> > > > > >> > >>>>>>>>> shell
> > > > > > >> > > > > >> > >>>>>>> scrips
> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging
> > (all
> > > > the
> > > > > > .tgz
> > > > > > >> > > > stuff),
> > > > > > >> > > > > >> etc.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
> > > > deployment
> > > > > > was
> > > > > > >> to
> > > > > > >> > > > > support
> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> > > locality,
> > > > > you
> > > > > > >> need
> > > > > > >> > to
> > > > > > >> > > > put
> > > > > > >> > > > > >> > >>>>>>>>> your
> > > > > > >> > > > > >> > >>>>>> processors
> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're processing.
> > Upon
> > > > > > further
> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > > > > > >> > > > > >> > >>>>>>> though,
> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that beneficial.
> > There
> > > is
> > > > > > some
> > > > > > >> > good
> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > > > > > >> > > > > >> > >>>>>> about
> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335.
> > Again,
> > > we
> > > > > > took
> > > > > > >> the
> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > > > > > >> > > > > >> > >>>>>> path,
> > > > > > >> > > > > >> > >>>>>>>>> but
> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental differences
> > > > between
> > > > > > HDFS
> > > > > > >> > and
> > > > > > >> > > > > Kafka.
> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > > > > > >> > > > > >> > >>>>>> has
> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions.
> This
> > > > leads
> > > > > to
> > > > > > >> less
> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
> > > processors
> > > > > on
> > > > > > top
> > > > > > >> > of
> > > > > > >> > > > > Kafka.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a crutch.
> > > Samza
> > > > > > doesn't
> > > > > > >> > > have
> > > > > > >> > > > > any
> > > > > > >> > > > > >> > >>>>>>>>> built
> > > > > > >> > > > > >> > >>>>> in
> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
> > depends
> > > on
> > > > > the
> > > > > > >> > > dynamic
> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to handle
> > > > restarts
> > > > > > >> when a
> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > > > > > >> > > > > >> > >>>>>>> made
> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a standalone
> > > Samza
> > > > > > >> > container
> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is good,
> but I
> > > > think
> > > > > > that
> > > > > > >> > > we've
> > > > > > >> > > > > >> gone
> > > > > > >> > > > > >> > >>>>>>>>> too
> > > > > > >> > > > > >> > >>>>>> far
> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > > > (SystemConsumer,
> > > > > > >> > > > > SystemProducer,
> > > > > > >> > > > > >> > >>>> etc).
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about
> > every
> > > > > > >> component
> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> > > ConfigRewriter,
> > > > > > etc).
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> > forgotten,
> > > as
> > > > > > well.
> > > > > > >> > Some
> > > > > > >> > > > of
> > > > > > >> > > > > >> > >>>>>>>>> these
> > > > > > >> > > > > >> > >>>>> are
> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not to be.
> > > This
> > > > > all
> > > > > > >> comes
> > > > > > >> > > at
> > > > > > >> > > > a
> > > > > > >> > > > > >> cost:
> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is making
> it
> > > > harder
> > > > > > for
> > > > > > >> > our
> > > > > > >> > > > > users
> > > > > > >> > > > > >> > >>>>>>>>> to
> > > > > > >> > > > > >> > >>>>> pick
> > > > > > >> > > > > >> > >>>>>> up
> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also
> > makes
> > > > it
> > > > > > >> > difficult
> > > > > > >> > > > for
> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about what
> the
> > > > > > >> > > characteristics
> > > > > > >> > > > of
> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> characteristics
> > > > change
> > > > > > >> > > depending
> > > > > > >> > > > on
> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are most
> > > visible
> > > > > in
> > > > > > the
> > > > > > >> > > > System
> > > > > > >> > > > > >> APIs.
> > > > > > >> > > > > >> > >>>>> What
> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be functional
> is
> > > > Kafka
> > > > > > as
> > > > > > >> its
> > > > > > >> > > > > >> > >>>>>>>>> transport
> > > > > > >> > > > > >> > >>>>>> layer.
> > > > > > >> > > > > >> > >>>>>>>>> But
> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases
> > > into
> > > > > one
> > > > > > >> API:
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports both of
> > > these
> > > > > use
> > > > > > >> > cases.
> > > > > > >> > > > The
> > > > > > >> > > > > >> > >>>>>>>>> problem
> > > > > > >> > > > > >> > >>>>>> is,
> > > > > > >> > > > > >> > >>>>>>>>> we
> > > > > > >> > > > > >> > >>>>>>>>> actually want different features for
> each
> > > use
> > > > > > case.
> > > > > > >> By
> > > > > > >> > > > > >> papering
> > > > > > >> > > > > >> > >>>>>>>>> over
> > > > > > >> > > > > >> > >>>>>>> these
> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a single
> > API,
> > > > > we've
> > > > > > >> > > > introduced
> > > > > > >> > > > > a
> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > > > >> > > > > >> > >>>>>>> leaky
> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like in
> (2)
> > > is
> > > > to
> > > > > > have
> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
> > offsets
> > > > > (like
> > > > > > >> > Kafka).
> > > > > > >> > > > > This
> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > > > > > >> > > > > >> > >>>>> with
> > > > > > >> > > > > >> > >>>>>>> (1),
> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems have
> > > > different
> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> mailing
> > > list
> > > > > and
> > > > > > >> the
> > > > > > >> > > SQL
> > > > > > >> > > > > >> JIRAs
> > > > > > >> > > > > >> > >>>>> about
> > > > > > >> > > > > >> > >>>>>>> the
> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> > > replayability.
> > > > > > Kafka
> > > > > > >> > > allows
> > > > > > >> > > > us
> > > > > > >> > > > > >> to
> > > > > > >> > > > > >> > >>>>> rewind
> > > > > > >> > > > > >> > >>>>>>>>> when
> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other systems
> > > don't.
> > > > In
> > > > > > some
> > > > > > >> > > > cases,
> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > > > > >> > > > > >> > >>>>>>> return
> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > > > > > >> WikipediaSystemConsumer)
> > > > > > >> > > > > because
> > > > > > >> > > > > >> > >>>>>>>>> they
> > > > > > >> > > > > >> > >>>>>> have
> > > > > > >> > > > > >> > >>>>>>> no
> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka
> > > > supports
> > > > > > >> > > > > partitioning,
> > > > > > >> > > > > >> > >>>>>>>>> but
> > > > > > >> > > > > >> > >>>>> many
> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by having
> a
> > > > single
> > > > > > >> > > partition
> > > > > > >> > > > > for
> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other systems
> model
> > > > > > >> partitioning
> > > > > > >> > > > > >> > >>>> differently (e.g.
> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a
> mess.
> > > > > > Creating
> > > > > > >> > > streams
> > > > > > >> > > > > in
> > > > > > >> > > > > >> a
> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> impossible.
> > > As
> > > > is
> > > > > > >> > modeling
> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > > > > > >> > > > > >> > >>>>> for
> > > > > > >> > > > > >> > >>>>>>> the
> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor, partitions,
> > > > > location,
> > > > > > >> > etc).
> > > > > > >> > > > The
> > > > > > >> > > > > >> > >>>>>>>>> list
> > > > > > >> > > > > >> > >>>>> goes
> > > > > > >> > > > > >> > >>>>>>> on.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing
> Samza,
> > > > > Kafka's
> > > > > > >> > > consumer
> > > > > > >> > > > > and
> > > > > > >> > > > > >> > >>>>> producer
> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On
> the
> > > > > > >> > consumer-side,
> > > > > > >> > > > you
> > > > > > >> > > > > >> > >>>>>>>>> had two
> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level consumer,
> or
> > > the
> > > > > > simple
> > > > > > >> > > > > consumer.
> > > > > > >> > > > > >> > >>>>>>>>> The
> > > > > > >> > > > > >> > >>>>>>> problem
> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was that
> it
> > > > > > controlled
> > > > > > >> > your
> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and the
> > > order
> > > > > in
> > > > > > >> which
> > > > > > >> > > you
> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > > > > > >> > > > > >> > >>>>> problem
> > > > > > >> > > > > >> > >>>>>>>>> with
> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not
> > > simple.
> > > > > It's
> > > > > > >> > basic.
> > > > > > >> > > > You
> > > > > > >> > > > > >> > >>>>>>>>> end up
> > > > > > >> > > > > >> > >>>>>>> having
> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level
> stuff
> > > > that
> > > > > > you
> > > > > > >> > > > > shouldn't.
> > > > > > >> > > > > >> > >>>>>>>>> We
> > > > > > >> > > > > >> > >>>>>> spent a
> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > > > KafkaSystemConsumer
> > > > > > very
> > > > > > >> > > > robust.
> > > > > > >> > > > > >> It
> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some cool
> > > features:
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
> > > > > > prioritization.
> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> assignment
> > > to
> > > > > > support
> > > > > > >> > > > joins,
> > > > > > >> > > > > >> > >>>>>>>>> global
> > > > > > >> > > > > >> > >>>>>> state
> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> > checkpointing.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time is
> > that
> > > > > these
> > > > > > >> > > features
> > > > > > >> > > > > >> > >>>>>>>>> should
> > > > > > >> > > > > >> > >>>>>>> actually
> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers
> > (not
> > > > just
> > > > > > >> Samza
> > > > > > >> > > > stream
> > > > > > >> > > > > >> > >>>>>> processors)
> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like joins
> > and
> > > > > > partition
> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > > > > > >> > > > > >> > >>>>>>> Kafka
> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> > conclusion.
> > > > > > They're
> > > > > > >> > > adding
> > > > > > >> > > > a
> > > > > > >> > > > > >> ton
> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> consumer
> > > > > > >> > > implementation.
> > > > > > >> > > > > To a
> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > > > > > >> > > > > >> > >>>>> it's
> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've already
> done
> > > in
> > > > > > Samza.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a
> > > very
> > > > > > similar
> > > > > > >> > > > > approach
> > > > > > >> > > > > >> > >>>>>>>>> to
> > > > > > >> > > > > >> > >>>>>> Samza's
> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation
> for
> > > > > > handling
> > > > > > >> > > offset
> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> management
> > > > > feature
> > > > > > >> > stores
> > > > > > >> > > > > >> offset
> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you
> to
> > > > fetch
> > > > > > them
> > > > > > >> > > from
> > > > > > >> > > > > the
> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since
> > we
> > > > > could
> > > > > > >> have
> > > > > > >> > > > shared
> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > > >> > > > > >> > >>>>> work
> > > > > > >> > > > > >> > >>>>>> if
> > > > > > >> > > > > >> > >>>>>>>>> it
> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> radical
> > > > > > proposal.
> > > > > > >> > Samza
> > > > > > >> > > > is
> > > > > > >> > > > > >> > >>>>> relatively
> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to
> say
> > > that
> > > > > > we're
> > > > > > >> > > near a
> > > > > > >> > > > > 1.0
> > > > > > >> > > > > >> > >>>>>> release.
> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what we've
> > > > > learned,
> > > > > > and
> > > > > > >> > > begin
> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > > > > > >> > > > > >> > >>>>>>> about
> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change
> if
> > > we
> > > > > were
> > > > > > >> > > starting
> > > > > > >> > > > > >> from
> > > > > > >> > > > > >> > >>>>>> scratch?
> > > > > > >> > > > > >> > >>>>>>>>> My
> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way
> > to
> > > > run
> > > > > > Samza
> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
> > > > > dependences
> > > > > > on
> > > > > > >> > > YARN,
> > > > > > >> > > > > >> Mesos,
> > > > > > >> > > > > >> > >>>> etc.
> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support
> only
> > > > Kafka
> > > > > > as
> > > > > > >> the
> > > > > > >> > > > > stream
> > > > > > >> > > > > >> > >>>>>> processing
> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
> > > > > > >> serialization,
> > > > > > >> > > and
> > > > > > >> > > > > >> > >>>>>>>>> config
> > > > > > >> > > > > >> > >>>>>>> systems,
> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues that I
> > > > > outlined
> > > > > > >> > above.
> > > > > > >> > > It
> > > > > > >> > > > > >> > >>>>>>>>> should
> > > > > > >> > > > > >> > >>>>> also
> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> > > > dramatically.
> > > > > > >> > > Supporting
> > > > > > >> > > > > >> only
> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow Samza
> > to
> > > be
> > > > > > >> executed
> > > > > > >> > > on
> > > > > > >> > > > > YARN
> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > > > Marathon/Aurora),
> > > > > or
> > > > > > >> most
> > > > > > >> > > > other
> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > > > > > >> > > > > >> > >>>>>>> deployment
> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a lot
> > easier
> > > > for
> > > > > > new
> > > > > > >> > > users.
> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > > > > > >> > > > > >> > >>>>>>> having
> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN.
> > The
> > > > drop
> > > > > > in
> > > > > > >> > > mailing
> > > > > > >> > > > > >> list
> > > > > > >> > > > > >> > >>>>>> traffic
> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue
> to
> > > me.
> > > > > The
> > > > > > >> > > reality
> > > > > > >> > > > > is,
> > > > > > >> > > > > >> > >>>>> everyone
> > > > > > >> > > > > >> > >>>>>>>>> that
> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka.
> > We
> > > > > > basically
> > > > > > >> > > > require
> > > > > > >> > > > > >> it
> > > > > > >> > > > > >> > >>>>>> already
> > > > > > >> > > > > >> > >>>>>>> in
> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work. Those
> > that
> > > > are
> > > > > > >> using
> > > > > > >> > > > other
> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > > > > >> > > > > >> > >>>>>> are
> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into
> Kafka
> > > (1),
> > > > > and
> > > > > > >> then
> > > > > > >> > > > they
> > > > > > >> > > > > do
> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is already
> > > > > > discussion (
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> >
> > > > > > >> > > > >
> > > > > > >> >
> > > > > >
> > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > > > >> > > > > >> > >>>>> 767
> > > > > > >> > > > > >> > >>>>>>>>> )
> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka
> > > > extremely
> > > > > > >> easy.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple with
> > Kafka,
> > > > we
> > > > > > can
> > > > > > >> > > > leverage
> > > > > > >> > > > > a
> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > > > >> > > > > >> > >>>>>>> their
> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> maintain
> > > our
> > > > > own
> > > > > > >> > config,
> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > > > > > >> > > > > >> > >>>>> etc.
> > > > > > >> > > > > >> > >>>>>>> We
> > > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries, and
> > make
> > > > them
> > > > > > >> > better.
> > > > > > >> > > > This
> > > > > > >> > > > > >> > >>>>>>>>> will
> > > > > > >> > > > > >> >
> ...
>
> [Message clipped]

Re: Thoughts and obesrvations on Samza

Posted by Tim Williams <wi...@gmail.com>.
On Sun, Jul 12, 2015 at 8:54 PM, Chris Riccomini <cr...@apache.org> wrote:
> Hey all,
>
> I want to start by saying that I'm absolutely thrilled to be a part of this
> community. The amount of level-headed, thoughtful, educated discussion
> that's gone on over the past ~10 days is overwhelming. Wonderful.
>
> It seems like discussion is waning a bit, and we've reached some
> conclusions. There are several key emails in this threat, which I want to
> call out:
>
> 1. Jakob's summary of the three potential ways forward.
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> 2. Julian's call out that we should be focusing on community over code.
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> 3. Martin's summary about the benefits of merging communities.
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> 4. Jakob's comments about the distinction between community and code paths.
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
>
> I agree with the comments on all of these emails. I think Martin's summary
> of his position aligns very closely with my own. To that end, I think we
> should get concrete about what the proposal is, and call a vote on it.
> Given that Jay, Martin, and I seem to be aligning fairly closely, I think
> we should start with:
>
> 1. [community] Make Samza a subproject of Kafka.

Can you expound on what you mean by "subproject"?  There is such a
thing around here and they are considered an anti-pattern at the ASF
but since the Lucene/Solr merge has been mentioned (which was truly a
merge, not a subproject) I'm curious what exactly the proposal is?  I
imagine the board would want this clearly articulated too, so your
energy likely wouldn't be wasted on a lowly user.

> 2. [community] Make all Samza PMC/committers committers of the subproject.

This hints at truly a subproject, which I'd suggest is a bad idea.
Even if it's technology is closely coupled to Kafka, Samza has proven
it can stand on it's own as a community and technical focus.

> 4. [code] Have the Samza community and the Kafka community start a
> from-scratch reboot together in the new Kafka subproject. We can
> borrow/copy &  paste significant chunks of code from Samza's code base.

Huh? are you suggesting a big "2.0" rewrite?  as a user this kinda
talk is frightening.

> 5. [code] The subproject would intentionally eliminate support for both
> other streaming systems and all deployment systems.

as in eliminate support for Yarn deployments?

> 6. [code] Attempt to provide a bridge from our SystemConsumer to KIP-26
> (copy cat)
> 7. [code] Attempt to provide a bridge from the new subproject's processor
> interface to our legacy StreamTask interface.
> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> subproject that has a fault-tolerant container with state management.
>
> It's likely that (6) and (7) won't be fully drop-in. Still, the closer we
> can get, the better it's going to be for our existing community.
>
> One thing that I didn't touch on with (2) is whether any Samza PMC members
> should be rolled into Kafka PMC membership as well (though, Jay and Jakob
> are already PMC members on both). I think that Samza's community deserves a
> voice on the PMC, so I'd propose that we roll at least a few PMC members
> into the Kafka PMC, but I don't have a strong framework for which people to
> pick.
>
> Before (8), I think that Samza's TLP can continue to commit bug fixes and
> patches as it sees fit, provided that we openly communicate that we won't
> necessarily migrate new features to the new subproject, and that the TLP
> will be shut down after the migration to the Kafka subproject occurs.
>
> Jakob, I could use your guidance here about about how to achieve this from
> an Apache process perspective (sorry).

I'm not Jakob, but can tell you for the technical bits you could call
a vote to gauge consensus on the technical approach and just start
pursuing it.

Depending on your answer to my questions on whether this is truly a
subproject being proposed vs. a truly merged project, it may a board
resolution - but I reckon you'll want to consult board@ for advice
well before it gets to that point.

> * Should I just call a vote on this proposal?

I'll repeat my previous mail and recommend you break up the
'future-of-samza-code' from 'future-of-samza-community' proposals.

> * Should it happen on dev or private?

dev@, *definitely*

> * Do committers have binding votes, or just PMC?

PMC members technically have a binding vote, but if that ends up
mattering when the votes are tallied, you should hope everyone will
think long and hard about it.

> Having trouble finding much detail on the Apache wikis. :(

http://www.apache.org/foundation/voting.html :)

--tim

Re: Thoughts and obesrvations on Samza

Posted by Navina Ramesh <nr...@linkedin.com.INVALID>.
Hi everyone,
I have been following this email thread closely and have discussed with
some of the folks at LinkedIn. I wanted to send out my opinion before it
is too late.

@Yi: Thanks for summarizing your proposal. I think it aligns very well
with our discussions.

I have been a customer of Samza at LinkedIn before I turned into a
develope. So, needless to say, I can relate more closely to the Samza
community. 

Philosophically, ³streaming system² ( or mesaging system ) and ³stream
processing² are solving two different problems.Speaking as a member of
the community, I still don¹t see why Samza has to be a part of the Kafka
community. Clearly, the projects are solving different problems (yes.. we
may have overlapping sub-problems).

A close integration with Kafka is definitely a good option for both the
communities and project. Other than the technical benefits of using Kafka
as the ³transport² i Samza, copycat for ³source/sink² connectors and
perhaps, a shared co-ordinator component, I don¹t see any real benefits
for the community. 
In fact, it only worries me that we will lose focus on stream processing
since we become a part of a much larger community like Kafka. Especially,
since we have made a significant strides towards building a lot of
advanced features in samza like SQL, auto-scaling, broadcast streams etc.
I think we should ot consider sunsetting Samza as a TLP and instead,
re-charter our vision for Samza to be in close alignment with Kafka.

Having used Yarn for quite sometime now, I am also not a fan of 2 projects
solving different problems being part of the same
repository/community/release-cycle.


Having said all this, I think the main pain-points of Samza users are:
1. Tight coupling with Yarn (or rather no standalone version)
2. Too manyconfigurations / pluggable components


Any proposal that addresses these issues for our users without dissolving
the Samza identity and community is acceptable, in my opinion.

Thanks!
Navina

On 7/12/15, 10:02 PM, "Jordan Sha <jo...@pubnub.com> wrote:

>Hi Everyone,
>I still don't completely understand why there's a need to make Samza
>"part-of" or rely on Kafka as a transmission medium? Is it not correct to
>say that given proper API's and proper implementation of those API's
>things
will work just fine? We've had really good results from using the current
>API's adding some SystemProducer/SystemConsumer functionality. I wouldn't
>mind if those API's were refined or even broken in order to make what the
>community thinks is btter but to completely abandon something that really
>isn't broken I don't understand. I also don't understand placing a bet
>that
>Kafka will be the defacto transmission media form now and forever.
>
>s I outlined before switching to Kafka wasn't an obvious decision for us
>and we had to spend significant resources to make that decision. Where as
>the decision on Samza was much easier. Just to reiterate transmission
>mediums like ZeroMQ do streaming just as well as Kafka, or better for some
>use cases but do it in a very different way, with very different scaling
>characteristics so why wouldn't Samza want to have the option to support
>that? (which it already does)
>
>So I agree with Jakob on the Samza 2.0 approach, breaking API's to make
>the
>code better but not abandoning, and to reiterate my questions:
>1) What is so broken with the current API's that can't be fixed by
>refactoring in the current context of Samza?
>2) Why should Kafka be the defacto transmission medium for Samza when
>something like ZeroMQ would work just fine and support different
>infrastructure requirements?
>
>-Jordan
>
>On Sun, Jul 12, 2015 at 8:57 PM, Jay Kreps <ja...@gmail.com> wrote:
>
>> Hey guys,
>>
>> There seems to be some confusion in the last few emails: there is no
>>plan
>> whatsoever to remove YARN support. The change suggested was to move the
>> partition management out of the YARN app master and rely on Kafka's
>> partition management. The advantage of this would be to make the vast
>> majority of Samza totally cluster manager agnostic, and make it
>>possible to
>> implement a high quality "stand-alone" mode and support other
>>frameworks.
>> It is possible this will let us run in YARN without *any* samza-specific
>> code using something generic like Slider, or maybe not.I don't think
>> anyone has tried this so it is probably premature to say how much
>> YARN-specific code could be killed. If that works out, then a bunch of
>>the
>> YARN-specific code will disappear, but this doesn't mean YARN support
>>will
>> disappear, just that we would retain less code to implement the same
>> thig.  Either wy Samza in YARN isn't going away.
>>
>> -Jay
>>
>> On Sun, Jul 12, 2015 at 7:48 PM, Garrett Barton
>><ga...@gmail.com>
>> wrote:
>>
>> >Yi,
>> >
>> >  What you just summarized makes a whole lot more sense to me.>> Shamelessly
>> > I am looking at this shift as a customer with a production workflow
>> riding
>> > on it so I am looking for some kind of consistency into the future of
>> > Samza.  This makes me feel a lot better about it.
>> >
>> >
>> > Thank you!
>> >
>> > OnSun, Jul 12, 2015 at 10:44 PM, Yi Pan <ni...@gmail.com> wrote:
>> >
>> > > Just to make it explicitly clear what I am proposing, here is a
>>version
>> > of
>> > > more detailed description:
>> > >
>> > > The fourth option (in addition to what Jakob summarized) we are
>> proposing
>> > > is:
>> > >
>> > > - Recharter Samza to ³stream processing as a service²
>> > >
>> > > - The current Samza core (the basic transformation API w/ basic
>> partition
>> > > and offset management bild-in) will be moved to Kafka Streams (i.e.
>> part
>> > > of Kafka) and supports ³run-as-a-library²
>> > >
>> > > - Deprecate the SystemConsumers and SystemProducers APIs and move
>>them
>> to
>> > > Copycat
>> > >
>> > > - The current SQL development:
>> > >
>> > >    * physical operators and a Trident-like stream API should stay in
>> > Kafka
>> > > Steams as libraries, enabling any stadalone deployment to use the
>> core
>> > > window/join functions
>> > >
>> > >    * query parser/planner and execution ontop of a distributed
>>service
>> > > should stay in new Samza (i.e. ³stream processing as a service²)
>> > >
>> > >  Advaned features related to job scheduling/state management
>>stays in
>> > new
>> > > Samza (i.e. ³streaming processing as a service²)
>> > >
>> > >    * Any advanced PartitionManager implementation that can be
>>plugged
>> > into
>>> > Kafka Streams
>> > >
>> > >    * Any auto-scaling, dynami configuration via coordinator stream
>> > >
>> > >    * Any advanced state management s.t. host-affinity etc.
>> > >
>> > >
>> > > Pros:
>> > >
>> > > - W/ the current Samza core as Kafka Streams and move the ingestion
>>to
>> > > Copycat, we achieved most of the goals in the initial proposal:
>> > >
>> > >    * Tighter coupling w/ Kafka
>> > >
>> > >    * Reuse Kafka¹s build-in functionalities, such as offset mnager,
>> > basic
>> > > partition distribution
>> > >
>> > >    * Separation of ingestion vs transformation APIs
>> > >
>> > >    * offload a lot of system-specific configuration to Kafka Streams
>> and
>> > > Copycat(i.e. SystemFactory configure, serde configure, etc.)
>> > >
>> > >    * remove YARN dependency and make standalone deployment easy.As
>> > > Guozhang mentioned, it would be really easy to start a process that
>> > > internally run Kafka Streams as library.
>> > >
>> > > - By re-chartering Samza as ³stream processing as a service², we
>> address
>> > > the concern regarding to
>>> >
>> > >    * Pluggable partition management
>> > >
>> > >    * Running in a distributed cluster to manage process lifecycle,
>> > > fault-tolerance, resource-allocation, etc.
>> > >
>> > >    * More advanced features s.t. host-affinty, auto-scaling, and
>> dynamic
>> > > configre changes, etc.
>> > >
>> > >
>> > > Regarding to the code and community organization, I think the
>>following
>> > may
>> > > be the best:
>> > >
>> > > Code:
>> > >
>> > > - A Kafka sub-project Kafka Streams to hold samza-core,
>>samza-kv-store,
>> > and
>>  > the physical operator layer as library in SQL: this would allow
>>better
>> > > alignment w/ Kafka, in code, doc, and branding
>> > >
>> > > - Retain the current Samza prject just to keep
>> >>
>> > >    * A pluggable explicit partition management in Kafka Streams
>>client
>> > >
>> > >    * Integration w/ cluster-management systems for advanced
>>features:
>> > >
>> > >       * host-affinity, auto-scaling,, dynamic configuration, etc.
>> > >
>> > >   * It will fully depend on the Kafka Streams API and remove all
>> support
>> > > for SystemConsumers/SystemProducers in the future
>> > >
>> > > Community: (this is almost the same as what Chris proposed)
>> > >
>> > > - Kafka Sreams: the current Samza community should be supporting
>>this
>> > > effort together with some Kafka members, since most of the code here
>> will
>> > > be from samza-core, samza-kv-store, and samza-sql.
>> > >
>> > > - new Samza: the current Samza community should continue serve the
> course
>> > > to support more advanced features to run Kafka Streams as a service.
>> > > Arguably, the new Samza framework may be used to run Copycat
>>workers as
>> > > well, at least to manage Copycat worker¹s lifecycle in a clustered
>> >  environment. Hence, it would stay as a general stream processing
>> > framework
>> > > that takes in any source and output to any destination, just the
>> > transport
>> > > system is fixed to Kafka.
>> > >
>> > > On Sun, Jul 12, 2015 at 7:29 PM, Yi Pn <ni...@gmail.com> wrote:
>> > >
>> > > > Hi, Chris,
>> > > >
>> > > > Thanks for sending out this concrete set of points hee. I agree
>>w/
>> all
>> > > > but have a slight different point view on 8).
>> > > >
>> > > > My iew on this is: instead of sunset Samza as TLP, can we
>>re-charter
>> > the
>> > > > scope of Samza to be the home for "running streaming proess as a
>> > > service"?
>> > > >
>> > > > My main motivationis from the following points from a long
>>internal
>> > > > discussion in LinkedIn:
>> > > >
>> > > > - There is a clear ask for pluggable partition management, like
>>we do
>> > in
>> > > > LinkedIn, and as Ben Kirwin has mentioned in
>> > > >
>> > >
>> >
>> 
>>http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCACuX-D
>>-yJX++2GnF_1Laf10KYUVyamg7UP_Dt19v0ZNmmhbCPQ@mail.gmail.com%3E
>> > > > - There are concerns on lack of support for running stream
>>processing
>> > in
>> > > a
>> > > > cluster: lifecycle management, resource allocation, fault
>>tolerance,
>> > etc.
>> > > > - There is a question to how to support more advanced features
>>s.t.
>> > > > host-affinity, auto-scaling, and dynamic configuration in Samza
>>jobs,
>> > as
>> > > > raised by Martin here:
>> >> >
>> > >
>> >
>> 
>>http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3C0D66EFD
>>0-B7CD-4E4E-8B2F-2716167C313C@kleppmann.com%3E
>> > > >
>> > > > We have use cases tat need to address all the above three cases
>>and
>> > most
>> > > > of the functions are all in the current Samza project, in some
>> flavor.
>> > We
>> > > > are all supporting to merge the samza-core functionalities into
>>Kafka
>> > > > Streams, but there is a question where we keep these functions in
>>the
>> > > > future. One option is to start a new project that includes these
>> > > functions
>> > > > that are closely related w/ "run stream-processing as-a-service",
>> while
>> > > > anther personally more attractive option is to re-charter Samza
>> > project
>> > > > just do "run stream processing as-a-service". We can avoid the
>> overhead
>> > > of
>> > > > re-starting another community for this project. Personally,I felt
>> that
>> > > > here are the benefits we should be getting:
>> > > >
>> > > > 1. We have already agreed mostly that Kafka Streams API would
>>allow
>> > some
>> > > > pluggable partition management functions. Hence, the advanced
>> partition
>> > > > management can live out-side the new Kafka Streams core w/o
>>affecting
>> > the
>> > > > run-as-a-library model in Kafka Streams.
>> > > > 2. The integration w/ cluster management system and advanced
>>features
>> > > > listed above stays in the sam project and allow existing users
>>enjoy
>> > > > no-impact migration to Kafka Stream as the core. That also
>>addresses
>> > > im's
>> > > > question on "removing the support for YARN".
>> > > > 3. A separate project for stream-processing-as-a-service also
>>allow
>> the
>> > > > new Kafka Streams being independent to any cluster management and
>> just
>> > > > focusing on stream process core functions, while leaving the
>> functions
>> > > that
>> > > > requires cluster-resource and state management to a separate
>>layer.
>> > > >
>> > > > Please feel free to comment. Thanks!
>> > > >
>> > > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
>> > criccomini@apache.org>
>> > > > wrote:
>> > > >
>> > > >> Hey all,
>> > > >>
>> > > >> I want to tart by saying that I'm absolutely thrilled to be a
>>part
>> of
>> > > >> this
>> > > >> community. The amount of level-headed, thoughtful, educated
>> discussion
>> > > >> that's gone on over the past ~10 days is overwhelmng. Wonderful.
>> > > >>
>> > > >> It seems like discussion is waning a bit, and we've reached some
>> > > >> conclusions. There are several key emails in this thret, which I
>> want
>> > > to
>> > > >> call out:
>> > > >>
>> > > >> 1. Jakob'ssummary of the three potential ways forward.
>> > > >>
>> > > >>
>> > > >>
>> > >
>> >
>> 
>>http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvV
>>u-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43%40mail.gmail.com%3E
> > > >> 2. Julian's call out that we should be focusing on community over
>> > code.
>> > > >>
>> > > >>
>> > > >>
>> > >
>> >
>> 
>>http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeE
>>SZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
>> > > >> 3. Martin's summary about the benefits of merging communities.
>> > > >>
>> > > >>
>> >> >>
>> > >
>> >
>> 
>>http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mox/%3CBFB866B
>>6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
>> > > >> 4. Jakob's comments about the distinction between community and
>>code
>> > > >> paths.
>> > > >>
>> > > >>
>> > > >>
>> > >
>> >
>> 
>>http://mail-archives.pache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvV
>>tWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
>> > > >>
>> > > >> I agree with the comments on all of these emails. I think
>>Martin's
>> > > summary
>> > > >> of his position aligns very closely with my own. To that end, I
>> think
>> > w
>> > > >> should get concrete about what the proposal is, and call a vote
>>on
>> it.
>> > > >> Given that Jay, Martin, and I seem to be aligning fairly
>>closely, I
>> > > think
>> > > >> we should start with:
>> > > >>
>> > > >> 1. [community] Make Samza a subproject of Kafka.
>> > > >> 2. [community Make all Samza PMC/committers committers of the
>> > > subproject.
>> > > >> 3. [community] Migrate Samza's website/documentation into
>>Kafka's.
>> > > >> 4. [code] Have the Samza community and the Kafka community start
>>a
>> > >> from-scratch reboot together in the new Kafka subproject. We can
>> > > >> borrow/copy &  paste significant chunks of code from Samza's code
>> > base.
>> > > >> 5. [code] The subproject would intentionally eliminate support
>>for
>> > both
>> > > >> other streaming systems and all deployment systems.
>> > > >> 6. [code] Attempt to provide a bridge from our SystemConsumer to
>> > KIP-26
>> > > >> (copy cat)
>> > > >> 7. [code] Attempt to provide a bridge from the new subproject's
>> > > processor
>> > > >> interface to our legacy StreamTask interface.
> > > >> 8. [code/community] Sunset Samza as a TL when we have a working
>> Kafka
>> > > >> subproject that has a fault-tolerant container with state
>> management.
>> > > >>
>> > > >> It's likely that (6) and (7) won't be fully drop-in. Stil, the
>> closer
>> > > we
>> > > >> can get, the better it's going to be for our existing community.
>> > > >>
>> > > >> One thing that I didn't touch on with (2) is whether any Samza
>>PMC
>> > > members
>> > > >> should be rolled ino Kafka PMC membership as well (though, Jay
>>and
>> > > Jakob
>> > > >> are already PMC member on both). I think that Samza's community
>> > > deserves
>> > > >> a
>> > > >> voice on the PMC, so I'd propose that we roll at least a few PMC
>> > members
>> > > >> into the Kafka PMC, but I don't have a strong framework for which
>> > people
>> > > >> to
>> > > >> pick.
>> > > >>
>> > > >> Before (8), I think that Samza's TLP can continue to commit bug
>> fixes
>> > > and
>> > > >> patches as it sees fit, provided that we openly communicate that
>>we
>> > > won't
>> > > >> necessarily migrate new features to the new subproject, and that
>>the
>> > TLP
>> > > >> will be shut down after the migration tothe Kafka subproject
>> occurs.
>> > > >>
>> > > >> Jakob, I could use your guidance here about about how to achieve
>> this
>> > > from
>> > > >> an Apache process perspective (sorry).
>> > > >>
>> > > >> * Should I just call a vote on this proposal?
>> > > >> * Should it happen on dev or private?
>> > > >> * Do committers have binding votes, or just PMC?
>> > > >>
>> > > >> Having trouble finding much detail on the Apache wikis. :(
>> > > >>
>> > > >> Cheers,
>> > > >> Chris
>> > > >>
>> > > >> On ri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com>
>> > wrote:
>> > > >>
>> > > >> > Thanks, Jay. This argument persuaded me actually. :)
>> > > >> >
>> > > >> > Fang, Yan
>> > > >> > yanfang724@gmail.com
>> > > >> >
>> > >  > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io>
>> > wrote:
>> > > >> >
>> > > >> > > Hey Yan,
>> > > >> > >
>> > > >> > > Yeah philosophically I think the argument is that you should
>> > capture
>> > > >> the
>> > > >> > > stream in Kafka independent of the transformation. This is
>> > > obviously a
>> > > >> > > Kafka-centric view point.
>> > > >> > >
>> > > >> > > Advantages of this:
>> > > >> > > - In practice I think this is what e.g. Storm people ften
>>ed
>> up
>> > > >> doing
>> > > >> > > anyway. You usually need to throttle any access to a live
>> serving
>> > > >> > database.
>> > > >> > > - Can have multiple subscribers and they get the same thing
>> > without
>> > > >> > > additional load on the source system.
>> > > >> > > - Applications can tap into the stream if need be by
>> subscribing.
>> > > >> > > - You can debug your transformation by tailing the Kafka
>>topic
>> > with
>> > > >> the
>> > > >> > > console consumer
>> > > >> > > - Can tee off the same data stream for batch analysis or
>>Lambda
>> > arch
>> > > >> > style
>> > > >> > > e-processing
>> > > >> > >
>> > > >> > > The disadvantage is that it will use Kafka resources. But the
>> idea
>> > > is
>> > > >> > > eventually you will have multiple subscribers to any data
>>source
>> > (at
>> > > >> > least
>> > > >> > > for monitoring) so you will end up there soon enough anyway.
>> > > >> > >
>> > > >> > > Down the road the technicl benefit is that I think it gives
>>us
>> a
>> > > good
>> > > >> > path
>> > > >> > > towards end-to-end exactly once semantics from source to
>> > > destination.
>> > > >> >> Basically the connectors need to support idempotence when
>> talking
>> > to
>> > > >> > Kafka
>> > > >> > > and we need the transactional write feature in Kafka to make
>>the
>> > > >> > > transformation atomic. This is actually pretty doable if you
>> > > separate
>> > > >> > > connector=>kafka problem from he generic transformations
>>which
>> > are
>> > > >> > always
>> > > >> > > kafka=>kafka. However I think it is quite impossible to do
>>in a
>> > > >> > all_things
>> > > >> > > => all_things environment Today you can say "well the
>>semantics
>> > of
>> > > >> the
>> > > >> > > Samza APIs depend on the connectors you use" but it is
>>actually
>> > > worse
>> > > >> > then
>> > > >> > > tha because the semantics actually depend on the pairing of
>> > > >> > connectors--so
>> > > >> > > not only can you probably not get a usable "exactly once"
>> > guarantee
>> > > >> > > end-to-end it can actually be quite hard to reverse engineer
>> what
>> > > >> > property
>> > > >> > > (if any) your end-to-end flow ha if you have heterogenous
>> > systems.
>> > > >> > >
>> > > >> > > -Jay
>> > > >> > >
>> > > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang
>><yanfang724@gmail.com
>> >
>> > > >> wrote:
>> > > >> > >
>> > >>> > > > {quote}
>> > > >> > > > maintainedin a separate repository and retaining the
>>existing
>> > > >> > > > committership but sharing as much else as possible
>>(website,
>> > etc)
>> > > >> > > > {quote}
>> > > >> > > >
>> > > >> > > > Overall, I agree on this dea. Now the question is more
>>about
>> > "how
>> > > >> to
>> > > >> > do
>> > > >> > > > it".
>> > > >> > > >
>> > > >> > > > On the other hand, one thing I want to point out is that,
>>if
>> we
>> > > >> decide
>> > > >> > to
>> > > >> > > > g this way, how do we want to support
>> > > >> >  > otherSystem-transformation-otherSystem use case?
>> > > >> > > >
>> > > >> > > > Basically, there are four user groups here:
>> > > >> > > >
>> > > >> > > > 1. Kafka-transformation-Kafka
>> > > >> > > > 2. Kafa-transformation-otherSystem
>> > > >> > > > 3. otherSystem-transformation-Kafka
>> > > >> > > > 4. otherSystem-transformation-otherSystem
>> > > >> > > >
>> > > >> > > > For group 1, they can easily use the new Samza library to
>> > achieve.
>> > > >> For
>> > > >> > > > group 2 and 3, they can use copyCat -> transformation ->
>>Kafka
>> > or
>> > > >> > Kafka->
>> > > >> > > > transformation -> copyCat.
>> > > >> > > >
>> > > >> > > > The problem is for group 4. Do we want to abandon this or
>> still
>> > > >> support
>> > > >> > > it?
>> > > >> > > > Of course, this use case can be achieved by using copyCat
>>->
>> > > >> > > transformation
>> > > >> > > > -> Kafk -> transformation -> copyCat, the thing is how we
>> > > persuade
>> > > >> > them
>> > > >> > > to
>> > > >> > > > do this long chain. If yes, it will also be a win for Kafka
>> too.
>> > > Or
>> > > >> if
>> > > >> > > > there is no one in this community actually doing this so
>>far,
>> > > maybe
>> > > >> ok
>> > > >> > to
>> > > >> > > > not support the group 4 directly.
>> > > >> > > >
>> > > >> > > > Thanks,
>> > > >> > > >
>> > > >> > > > Fang, Yan
>> > > >> > > > yanfang724@gmail.com
>> > > >> > > >
>> >> >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps
>><jay@confluent.io
>> >
>> > > >> wrote:
>> > > >> > > >
>> > > >> > > > > Yeah I agree with this summary. I think there are kind of
>> two
>> > > >> > questions
>> > > >> > > > > here:
>> > > >> > > > > 1. Technically does alignment/reliance on Kafka make
>>sense
>> > > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
>> > alignment
>> > > >> with
>> > > >> > > > Kafka
>> > > >> > > > > make sense
>> > > >> > > > >
>> > > >> > > > > Personally I do think both of these things would be
>>really
>> > > >> valuable,
>> > > >> > > and>> > > >> > > > > would dramatically alter the trajectory of the project.
>> > > >> > > > >
>> > > >> > > > > My preference would be to see if people can mostly agree
>>on
>> a
>> > > >> > direction
>> > > >> > > > > rather than splintering things off. From my point of view
>> the
>> > > >> ideal
>> > > >> > > > outcome
>> > > >> > > > > of all the options discussed would be to make Samza a
>> closely
>> > > >> aligned
>> > > >> > > > > subproject, maintained in a separate repository and
>> retaining
>> > > the
>> >  >> > > > existing
>> > > >> > > > > committership but sharing as much else as possible
>>(website,
>> > > >> etc). No
>> > > >> > > > idea
>> > > >> > > > > about how these things work, Jacob, you probably know
>>more.
>> > > >> > > > >
>> > > >> > > > > No discussion amongst the Kafka folks has happened on
>>this,
>> > but
>> > > >> > likely
>> > > >> > > we
>> > > >> > > > > should figure out what the Samza community actually wants
>> > first.
>> > > >> > > > >
>> > > >> > > > > I admit that this is a fairly radical departure from how
>> > things
>> > > >> are.
>> > > >> > > > >
>> > > >> > > > > If that doesn't fly, I think, yeah we could leave Samza
>>as
>> it
>> > is
>> > > >> and
>> > > >> > do
>> > > >> > > > the
>> > > >> > > > > more radical reboot inside Kafka. From my point of view
>>that
>> > > does
>> > > >> > leave
>> > > >> > > > > things in a somewhat confusing state sice now there are
>>two
>> > > >> stream
>> > > >> > > > > processing systems more or less coupled to Kafka in large
>> part
>> > > >> made
>> > > >> > by
>> > > >> > > > the
>> > > >> > > > > same people. But, aruably that might be a cleaner way to
>> make
>> > > the
>> > > >> > > > cut-over
>> > > >> > > > > and perhaps less risky for Samza community since if it
>>works
>> > > >> people
>> > > >> > can
>> > > >> > > > > switch and if it doesn't nothing will have changed.
>>Dunno,
>> how
>> > > do
>> > > >> > > people
>> > > >> > > > > feel about this?
>> > > >> > > > >
>> > > >> > > > > -Jay
>> > > >> > > > >
>> > > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
>> > > jghoman@gmail.com>
>> > > >> > > wrote:
>> > > >> > > > >
>> > > >> > > > > > >  This leads me to thinking that merging projects and
>> > > >> communities
>> > > >> > > > might
>> > > >> > > > > > be a good idea: with the union of experience from both
>> > > >> communities,
>> > > >> > > we
>> > > >> > > > > will
>> > > >> > > > > > probably build a better system that is better for
>>users
>> > > >> > > > > > Is this what's being proposed though? Merging the
>>projects
>> > > seems
>> > > >> > like
>> > > >> > > > > > a conseuence of at most one of the three directions
>>under
>> > > >> > > discussion:
>> > > >> > > > > > 1) Samza 2.0: The Samza community relies more heavily
>>on
>> > Kafka
>> > > >> for
>> > > >> > > > > > configuration, etc. (to a greter or lesser extent to
>>be
>> > > >> > determined)
>> > > >> > > > > > but the Samza community would not automatically merge
>> withe
>> > > >> Kafka
>> > > >> > > > > > community (the Phoenix/HBase example is a good one
>>here).
>> > > >> > > > > > 2) Samza Reboot: The Samza community continues to exist
>> > with a
>> > > >> > > limited
>> > > >> > > > > > project scope, but similarly would not need to be part
>>of
>> > the
>> > > >> Kafka
>> > > >> > > > > > community (ie given committership) to progress.  Here,
>> maybe
>>> > the
>> > > >> > > Samza
>> > > >> > > > > > team would become a subproject of Kafka (the Board
>>frowns
>> on
>> > > >> > > > > > subprojects at the moment, so I'm not sure if that's
>>even
>> > > >> > feasible),
>> > > >> > > > > > but that would not be required.
>> > > >> > > > > > 3) Hey Samza! FYI, Kafka does streaing now: In this
>> option
>> > > the
>> > > >> > Kafka
>> > > >> > > > > > team builds its own streaming library, possibly off of
>> Jay's
>> > > >> > > > > > prototype, which has not direct lineage to the Samza
>>eam.
>> > > >> There's
>> > > >> > > no
>> > > >> > > > > > reason for the Kafka team to bring in the Samza team.
>> > > >> > > > > >
>> > > >> > > > > > Is the Kafka community on board with this?
>> > > >> > > > > >
>> > > >> > > > > >To be clear, all three options under discussion are
>> > > interesting,
>> > > >> > > > > > technically valid and likely healthy directions for the
>> > > project.
>> > > >> > > > > > Also, they are not mutually exclusive.  The Samza
>> community
>> > > >> could
>> > > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
>> > community
> > > >> went
>> > > >> > > > > > forward with 'Hey Samza!'  My points above are directed
>> > > >> entirely at
>> > > >> > > > > > the community aspect of these choices.
>> > > >> > > > > > -Jakob
>> > > >> > > > > >
>> > > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
>> > > roger.hoover@gmail.com>
>> > > >> > > wrote:
>> > > >> > > > > > > That's great.  Thanks, Jay.
>> > > >> > > > > > >
>> > > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
>> > > jay@confluent.io>
>> > > >> > > wrote:
>> > > >> > > > > > >
>> > > >> > > > > > >> Yeah totally agree. I think you have this issue even
>> > today,
>> > > >> > right?
>> > > >> > > > > I.e.
>> > > >> > > > > > if
>> > > >> > > > > > >> you need to make a simple config change and you're
>> > running
>> > > in
>> > > >> > YARN
>> > > >> > > > > today
>> > > >> > > > > > >> you end up bouncing the job which then rbuilds
>>state.
>> I
>> > > >> think
>> > > >> > the
>> > > >> > > > fix
>> > > >> > > > > > is
>> > > >> > > > > > >> exactly what you described which is to have a long
>> > timeout
>> > > on
>> > > >> > > > > partition
>> > > >> > > > > > >> movement for stateful jobs so that if a job is just
>> > getting
>> > > >> > > bounce,
>> > > >> > > > > and
>> > > >> > > > > > >> the cluster manager (or admin) is smart enough to
>> restart
>> > > it
>> > > >> on
>> > > >> > > the
>> > > >> > > > > same
>> > > >> > > > > > >> host when possible, it can optimistically reuse any
>> > > existing
>> > > >> > state
>> > > >> > > > it
>> > > >> > > > > > finds
>> > > >> > > > > > >> on disk (if it is valid).
>> > > >> > > > > > >>
>> > > >> > > > > > >> So in this model the charter of the CM is to place
>> > > processes
>> > > >> as
>> > > >> > > > > > stickily as
>> > > >> > > > > > >> possible and to resart or re-place failed
>>processes.
>> The
>> > > >> > charter
>> > > >> > > of
>> > > >> > > > > the
>> > > >> > > > > > >> partition management system is to control the
>> assignment
>> > of
>> > > >> work
>> > > >> > > to
>> > > >> > > > > > these
>> > > >> > > > > > >> processes. The nice thing about this is that the
>>work
>> > > >> > assignment,
>> > > >> > > > > > timeouts,
>> > > >> > > > > > >> behavior, configs, and code will all be the same
>>across
>> > all
>> > > >> > > cluster
>> > > >> > > > > > >> managers.
>> > > >> > > > > > >>
>> > > >> > > > > > >> So I think that prototype would actually give you
>> exactly
>> > > >> what
>> > > >> > you
>> > > >> > > > > want
>> > > >> > > > > > >> today for any cluster manager (or manual placement +
>> > > restart
>> > > >> > > script)
>> > > >> > > > > > that
>> > > >> > > > > > >> was sticky in terms of host placement since there is
>> > > already
>> > > >> a
>> > > >> > > > > > configurable
>> > > >> > > > > > >> partition movement timeout and task-by-task state
>>reuse
>> > > with
>> > > >> a
>> > > >> > > check
>> > > >> > > > > on
>> > > >> > > > > > >> state validity.
>> > > >> > > > > > >>
>> > > >> > > > > > >> -Jay
>> > > >> > > > > > >>
>> > > >> > > > > > >> n Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
>> > > >> > > > roger.hoover@gmail.com
>> > > >> > > > > >
>> > > >> > > > > > >> wrote:
>> > > >> > > > > > >>
>> > > >> > > > > > >> > That would be great to let Kafka do as much heavy
>> > lifting
>> >  >> as
>> > > >> > > > > possible
>> > > >> > > > > > and
>> > > >> > > > > > >> > make it easier for other languages to implement
>>Samza
>> > > apis.
>> > > >> > > > > > >> >
>> > > >> > > > > > >> > One thing to watch out for is the interplay
>>btween
>> > > Kafka's
>> > > >> > > group
>> > > >> > > > > > >> > management and the external scheduler/process
>> manager's
>> > > >> fault
>> > > >> > > > > > tolerance.
>> > > >> > > > > > >> > If a container dies, the Kafka group membership
>> > protocol
>> > > >> will
>> > > >> > > try
>> > > >> > > > to
>> > > >> > > > > > >> assign
>> > > >> > > > > > >> > it's tasks to other containers while at the same
>>time
>> > the
>> > > >> > > process
>> > > >> > > > > > manager
>> > > >> > > > > > >> > is trying to relaunch the container.  Without some
>> > > >> > consideration
>> > > >> > > > for
>> > > >> > > > > > this
>> > > >> > > > > > >> > (like a configurable amount f time to wait before
>> > Kafka
>> > > >> > alters
>> > > >> > > > the
>> > > >> > > > > > group
>> > > >> > > > > > >> > membership), there may be thrashing going on
>>which is
>> > > >> > especially
>> > > >> > > > bad
>> > > >> > > > > > for
>> > > >> > > > > > >> > containers with large amounts of local state.
>> > > >> > > > > > >> >
>> > > >> > > > > > >> > Someone else pointed this out already but I
>>thought
>> it
>> > > >> might
>> > > >> > be
>> > > >> > > > > worth
>> > > >> > > > > > >> > calling out again.
>> > > >> > > > > > >> >
>> > > >> > > > > > >> > Cheers,
>> > > >> > > > > > >> >
>> > > >> > > > > > >> > Roger
>> > > >> > > > > > >> >
>> > > >> > > > > > >> >
>> > > >> > > > > > >> > On Tue, Jul 7, 2015 at11:35 AM, Jay Kreps <
>> > > >> jay@confluent.io>
>> > > >> > > > > wrote:
>> > > >> > > > > > >> >
>> > > >> > > > > > >> > > Hey Roger,
>> > > >> > > > > > >> > >
>> > > >> > > > > > >> > > I couldn't agree more. We spent a bunch of time
>> > talking
>> > > >> to
>> > > >> > > > people
>> > > >> > > > > > and
>> > > >> > > > > > >> > that
>> > > >> > > > > > >> > > is exactly the stuff we heard time and again. 
>>What
>> > > makes
>> > > >> it
>> > > >> > > > hard,
>> > > >> > > > > of
>> > > >> > > > > > >> > > course, is that there is some tension between
>> > > >> compatibility
>> > > >> > > with
>> > > >> > > > > > what's
>> > > >> > > > > > >> > > there now and making things better for new 
>>users.
>> > > >> > > > > > >> > >
>> > > >> > > > > > >> > > I also strongly agree with the importance of
>> > > >> multi-language
>> > > >> > > > > > support. We
>> > > >> > > > > > >> > are
>> > > >> > > > > > >> > > talking now about Java, but for application
>> > development
>> > > >> use
>> > > >> > > > cases
>> > > >> > > > > > >> people
>> > > >> > > > > > >> > > want to work in whatever language they are using
>> > > >> elsewhere.
>> > > >> > I
>> > > >> > > > > think
>> > > >> > > > > > >> > moving
>> > > >> > > > > > >> > > to a model where Kafka itself does the group
>> > > membership,
>> > > >> > > > lifecycle
>> > > >> > > > > > >> > control,
>> > > >> > > > > > >> > > and partition assignment has the advantage of
>> putting
>> > > all
>> > > >> > that
>> > > >> > > > > > complex
>> > > >> > > > > > >> > > stuff behind a clean api that the clients are
>> already
>> > > >> going
>> > > >> > to
>> > > >> > > > be
>> > > >> > > > > > >> > > implementing for their consumer, so the added
>> > > >> functionality
>> > > >> > > for
>> > > >> > > > > > stream
>> > > >> > > > > > >> > > processing beyond a consumer becomes very minor.
>> > > >> > > > > > >> > >
>> > > >> > > > > > >> > > -Jay
>> > > >> > > > > > >> > >
>> > > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
>> > > >> > > > > > roger.hoover@gmail.com>
>> > > >> > > > > > >> > > wrote:
>> > > >> > > > > > >> > >
>> > > >> > > > > > >> > > > Metamorphosis...nice. :)
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > > This has been a great discussion.  As a user 
>>of
>> > Samza
>> > > >> > who's
>> > > >> > > > > > recently
>> > > >> > > > > > >> > > > integrated it into a relatively large
>> > organization, I
>> > > >> just
>> > > >> > > > want
>> > > >> > > > > to
>> > > >> > > > > > >> add
>> > > >> > > > > > >> > > > support to a few points already made.
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
>> > > >> currently
>> > > >> > > > exists
>> > > >> > > > > > that
>> > > >> > > > > > >> > I've
>> > > >> > > > > > >> > > > experienced are:
>> > > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
>> > environments
>> > > >> > where
>> > > >> > > > > Puppet
>> > > >> > > > > > >> > would
>> > > >> > > > > > >> > > do
>> > > >> > > > > > >> > > > just fine but it was the only mechanism to get
>> > fault
>> > > >> > > > tolerance.
>> > > >> > > > > > >> > > > 2) Configuration - I think I like the idea of
>> > > >> configuring
>> > > >> > > most
>> > > >> > > > > of
>> > > >> > > > > > the
>> > > >> > > > > > >> > job
>> > > >> > > > > > >> > > > in code rather than config files.  In 
>>general, I
>> > > think
>> > > >> the
>> > > >> > > > goal
>> > > >> > > > > > >> should
>> > > >> > > > > > >> > be
>> > > >> > > > > > >> > > > to make it harder to make mistakes, 
>>especially of
>> > the
>> > > >> kind
>> > > >> > > > where
>> > > >> > > > > > the
>> > > >> > > > > > >> > code
>> > > >> > > > > > >> > > > expects something and the config doesn't 
>>match.
>> > The
>> > > >> > current
>> > > >> > > > > > config
>> > > >> > > > > > >> is
>> > > >> > > > > > >> > > > quite intricate and error-prone.  For example,
>> the
>> > > >> > > application
>> > > >> > > > > > logic
>> > > >> > > > > > >> > may
>> > > >> > > > > > >> > > > depend on bootstrapping a topic but rather 
>>than
>> > > >> asserting
>> > > >> > > that
>> > > >> > > > > in
>> > > >> > > > > > the
>> > > >> > > > > > >> > > code,
>> > > >> > > > > > >> > > > you have to rely on getting the config right.
>> > > Likewise
>> > > >> > with
>> > > >> > > > > > serdes,
>> > > >> > > > > > >> > the
>> > > >> > > > > > >> > > > Java representations produced by various 
>>serdes
>> > > (JSON,
>> > > >> > Avro,
>> > > >> > > > > etc.)
>> > > >> > > > > > >> are
>> > > >> > > > > > >> > > not
>> > > >> > > > > > >> > > > equivalent so you cannot just reconfigure a 
>>serde
>> > > >> without
>> > > >> > > > > changing
>> > > >> > > > > > >> the
>> > > >> > > > > > >> > > > code.   It would be nice for jobs to be able 
>>to
>> > > assert
>> > > >> > what
>> > > >> > > > they
>> > > >> > > > > > >> expect
>> > > >> > > > > > >> > > > from their input topics in terms of 
>>partitioning.
>> > > >> This is
>> > > >> > > > > > getting a
>> > > >> > > > > > >> > > little
>> > > >> > > > > > >> > > > off topic but I was even thinking about 
>>creating
>> a
>> > > >> "Samza
>> > > >> > > > config
>> > > >> > > > > > >> > linter"
>> > > >> > > > > > >> > > > that would sanity check a set of configs.
>> > Especially
>> > > >> in
>> > > >> > > > > > >> organizations
>> > > >> > > > > > >> > > > where config is managed by a different team 
>>than
>> > the
>> > > >> > > > application
>> > > >> > > > > > >> > > developer,
>> > > >> > > > > > >> > > > it's very hard to get avoid config mistakes.
>> > > >> > > > > > >> > > > 3) Java/Scala centric - for many teams
>> (especially
>> > > >> > > DevOps-type
>> > > >> > > > > > >> folks),
>> > > >> > > > > > >> > > the
>> > > >> > > > > > >> > > > pain of the Java toolchain (maven, slow 
>>builds,
>> > weak
>> > > >> > command
>> > > >> > > > > line
>> > > >> > > > > > >> > > support,
>> > > >> > > > > > >> > > > configuration over convention) really inhibits
>> > > >> > productivity.
>> > > >> > > > As
>> > > >> > > > > > more
>> > > >> > > > > > >> > and
>> > > >> > > > > > >> > > > more high-quality clients become available for
>> > > Kafka, I
>> > > >> > hope
>> > > >> > > > > > they'll
>> > > >> > > > > > >> > > follow
>> > > >> > > > > > >> > > > Samza's model.  Not sure how much it affects 
>>the
>> > > >> proposals
>> > > >> > > in
>> > > >> > > > > this
>> > > >> > > > > > >> > thread
>> > > >> > > > > > >> > > > but please consider other languages in the
>> > ecosystem
>> > > as
>> > > >> > > well.
>> > > >> > > > > > From
>> > > >> > > > > > >> > what
>> > > >> > > > > > >> > > > I've heard, Spark has more Python users than
>> > > >> Java/Scala.
>> > > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza 
>>API
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > >
>> > > >> > > > > > >> >
>> > > >> > > > > > >>
>> > > >> > > > > >
>> > > >> > > > >
>> > > >> > > >
>> > > >> > >
>> > > >> >
>> > > >>
>> > >
>> >
>> 
>>https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/qu
>>antiply/samza
>> > > >> > > > > > >> > > > and are working on a Yeoman generator
>> > > >> > > > > > >> > > > https://github.com/Quantiply/generator-rico 
>>for
>> > > >> > > Jython/Samza
>> > > >> > > > > > >> projects
>> > > >> > > > > > >> > to
>> > > >> > > > > > >> > > > alleviate some of the pain)
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > > I also want to underscore Jay's point about
>> > improving
>> > > >> the
>> > > >> > > user
>> > > >> > > > > > >> > > experience.
>> > > >> > > > > > >> > > > That's a very important factor for adoption.  
>>I
>> > think
>> > > >> the
>> > > >> > > goal
>> > > >> > > > > > should
>> > > >> > > > > > >> > be
>> > > >> > > > > > >> > > to
>> > > >> > > > > > >> > > > make Samza as easy to get started with as
>> something
>> > > >> like
>> > > >> > > > > Logstash.
>> > > >> > > > > > >> > > > Logstash is vastly inferior in terms of
>> > capabilities
>> > > to
>> > > >> > > Samza
>> > > >> > > > > but
>> > > >> > > > > > >> it's
>> > > >> > > > > > >> > > easy
>> > > >> > > > > > >> > > > to get started and that makes a big 
>>difference.
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > > Cheers,
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > > Roger
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
>> > > Francisci
>> > > >> > > > Morales <
>> > > >> > > > > > >> > > > gdfm@apache.org> wrote:
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
>> > > >> Metamorphosis
>> > > >> > > is
>> > > >> > > > a
>> > > >> > > > > > clear
>> > > >> > > > > > >> > > > winner
>> > > >> > > > > > >> > > > > :)
>> > > >> > > > > > >> > > > >
>> > > >> > > > > > >> > > > > --
>> > > >> > > > > > >> > > > > Gianmarco
>> > > >> > > > > > >> > > > >
>> > > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De 
>>Francisci
>> > > >> Morales
>> > > >> > <
>> > > >> > > > > > >> > > gdfm@apache.org
>> > > >> > > > > > >> > > > >
>> > > >> > > > > > >> > > > > wrote:
>> > > >> > > > > > >> > > > >
>> > > >> > > > > > >> > > > > > Hi,
>> > > >> > > > > > >> > > > > >
>> > > >> > > > > > >> > > > > > @Martin, thanks for you comments.
>> > > >> > > > > > >> > > > > > Maybe I'm missing some important point, 
>>but I
>> > > think
>> > > >> > > > coupling
>> > > >> > > > > > the
>> > > >> > > > > > >> > > > releases
>> > > >> > > > > > >> > > > > > is actually a *good* thing.
>> > > >> > > > > > >> > > > > > To make an example, would it be better if 
>>the
>> > MR
>> > > >> and
>> > > >> > > HDFS
>> > > >> > > > > > >> > components
>> > > >> > > > > > >> > > of
>> > > >> > > > > > >> > > > > > Hadoop had different release schedules?
>> > > >> > > > > > >> > > > > >
>> > > >> > > > > > >> > > > > > Actually, keeping the discussion in a 
>>single
>> > > place
>> > > >> > would
>> > > >> > > > > make
>> > > >> > > > > > >> > > agreeing
>> > > >> > > > > > >> > > > on
>> > > >> > > > > > >> > > > > > releases (and backwards compatibility) 
>>much
>> > > >> easier, as
>> > > >> > > > > > everybody
>> > > >> > > > > > >> > > would
>> > > >> > > > > > >> > > > be
>> > > >> > > > > > >> > > > > > responsible for the whole codebase.
>> > > >> > > > > > >> > > > > >
>> > > >> > > > > > >> > > > > > That said, I like the idea of absorbing
>> > > samza-core
>> > > >> as
>> > > >> > a
>> > > >> > > > > > >> > sub-project,
>> > > >> > > > > > >> > > > and
>> > > >> > > > > > >> > > > > > leave the fancy stuff separate.
>> > > >> > > > > > >> > > > > > It probably gives 90% of the benefits we 
>>have
>> > > been
>> > > >> > > > > discussing
>> > > >> > > > > > >> here.
>> > > >> > > > > > >> > > > > >
>> > > >> > > > > > >> > > > > > Cheers,
>> > > >> > > > > > >> > > > > >
>> > > >> > > > > > >> > > > > > --
>> > > >> > > > > > >> > > > > > Gianmarco
>> > > >> > > > > > >> > > > > >
>> > > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
>> > > >> > jay.kreps@gmail.com
>> > > >> > > >
>> > > >> > > > > > wrote:
>> > > >> > > > > > >> > > > > >
>> > > >> > > > > > >> > > > > >> Hey Martin,
>> > > >> > > > > > >> > > > > >>
>> > > >> > > > > > >> > > > > >> I agree coupling release schedules is a
>> > > downside.
>> > > >> > > > > > >> > > > > >>
>> > > >> > > > > > >> > > > > >> Definitely we can try to solve some of 
>>the
>> > > >> > integration
>> > > >> > > > > > problems
>> > > >> > > > > > >> in
>> > > >> > > > > > >> > > > > >> Confluent Platform or in other
>> distributions.
>> > > But
>> > > >> I
>> > > >> > > think
>> > > >> > > > > > this
>> > > >> > > > > > >> > ends
>> > > >> > > > > > >> > > up
>> > > >> > > > > > >> > > > > >> being really shallow. I guess I feel to
>> really
>> > > >> get a
>> > > >> > > good
>> > > >> > > > > > user
>> > > >> > > > > > >> > > > > experience
>> > > >> > > > > > >> > > > > >> the two systems have to kind of feel like
>> part
>> > > of
>> > > >> the
>> > > >> > > > same
>> > > >> > > > > > thing
>> > > >> > > > > > >> > and
>> > > >> > > > > > >> > > > you
>> > > >> > > > > > >> > > > > >> can't really add that in later--you can 
>>put
>> > both
>> > > >> in
>> > > >> > the
>> > > >> > > > > same
>> > > >> > > > > > >> > > > > downloadable
>> > > >> > > > > > >> > > > > >> tar file but it doesn't really give a 
>>very
>> > > >> cohesive
>> > > >> > > > > feeling.
>> > > >> > > > > > I
>> > > >> > > > > > >> > agree
>> > > >> > > > > > >> > > > > that
>> > > >> > > > > > >> > > > > >> ultimately any of the project stuff is as
>> much
>> > > >> social
>> > > >> > > and
>> > > >> > > > > > naming
>> > > >> > > > > > >> > as
>> > > >> > > > > > >> > > > > >> anything else--theoretically two totally
>> > > >> independent
>> > > >> > > > > projects
>> > > >> > > > > > >> > could
>> > > >> > > > > > >> > > > work
>> > > >> > > > > > >> > > > > >> to
>> > > >> > > > > > >> > > > > >> tightly align. In practice this seems to 
>>be
>> > > quite
>> > > >> > > > difficult
>> > > >> > > > > > >> > though.
>> > > >> > > > > > >> > > > > >>
>> > > >> > > > > > >> > > > > >> For the frameworks--totally agree it 
>>would
>> be
>> > > >> good to
>> > > >> > > > > > maintain
>> > > >> > > > > > >> the
>> > > >> > > > > > >> > > > > >> framework support with the project. In 
>>some
>> > > cases
>> > > >> > there
>> > > >> > > > may
>> > > >> > > > > > not
>> > > >> > > > > > >> be
>> > > >> > > > > > >> > > too
>> > > >> > > > > > >> > > > > >> much
>> > > >> > > > > > >> > > > > >> there since the integration gets lighter
>> but I
>> > > >> think
>> > > >> > > > > whatever
>> > > >> > > > > > >> > stubs
>> > > >> > > > > > >> > > > you
>> > > >> > > > > > >> > > > > >> need should be included. So no I 
>>definitely
>> > > wasn't
>> > > >> > > trying
>> > > >> > > > > to
>> > > >> > > > > > >> imply
>> > > >> > > > > > >> > > > > >> dropping
>> > > >> > > > > > >> > > > > >> support for these frameworks, just making
>> the
>> > > >> > > integration
>> > > >> > > > > > >> lighter
>> > > >> > > > > > >> > by
>> > > >> > > > > > >> > > > > >> separating process management from 
>>partition
>> > > >> > > management.
>> > > >> > > > > > >> > > > > >>
>> > > >> > > > > > >> > > > > >> You raise two good points we would have 
>>to
>> > > figure
>> > > >> out
>> > > >> > > if
>> > > >> > > > we
>> > > >> > > > > > went
>> > > >> > > > > > >> > > down
>> > > >> > > > > > >> > > > > the
>> > > >> > > > > > >> > > > > >> alignment path:
>> > > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think
>> the
>> > > >> first
>> > > >> > > > > question
>> > > >> > > > > > is
>> > > >> > > > > > >> > > > whether
>> > > >> > > > > > >> > > > > >> some "re-branding" would be worth it. If 
>>so
>> > > then I
>> > > >> > > think
>> > > >> > > > we
>> > > >> > > > > > can
>> > > >> > > > > > >> > > have a
>> > > >> > > > > > >> > > > > big
>> > > >> > > > > > >> > > > > >> thread on the name. I'm definitely not 
>>set
>> on
>> > > >> Kafka
>> > > >> > > > > > Streaming or
>> > > >> > > > > > >> > > Kafka
>> > > >> > > > > > >> > > > > >> Streams I was just using them to be kind 
>>of
>> > > >> > > > illustrative. I
>> > > >> > > > > > >> agree
>> > > >> > > > > > >> > > with
>> > > >> > > > > > >> > > > > >> your
>> > > >> > > > > > >> > > > > >> critique of these names, though I think
>> people
>> > > >> would
>> > > >> > > get
>> > > >> > > > > the
>> > > >> > > > > > >> idea.
>> > > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about
>> how
>> > to
>> > > >> > > "factor"
>> > > >> > > > > it.
>> > > >> > > > > > >> Here
>> > > >> > > > > > >> > > are
>> > > >> > > > > > >> > > > > the
>> > > >> > > > > > >> > > > > >> options I see (I could get enthusiastic
>> about
>> > > any
>> > > >> of
>> > > >> > > > them):
>> > > >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
>> > > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
>> > > seperation
>> > > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of 
>>samza-api
>> > and
>> > > >> > > > samza-core
>> > > >> > > > > > is
>> > > >> > > > > > >> > > > absorbed
>> > > >> > > > > > >> > > > > >> almost like a third client
>> > > >> > > > > > >> > > > > >>
>> > > >> > > > > > >> > > > > >> Cheers,
>> > > >> > > > > > >> > > > > >>
>> > > >> > > > > > >> > > > > >> -Jay
>> > > >> > > > > > >> > > > > >>
>> > > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
>> > > Kleppmann <
>> > > >> > > > > > >> > > > martin@kleppmann.com>
>> > > >> > > > > > >> > > > > >> wrote:
>> > > >> > > > > > >> > > > > >>
>> > > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. 
>>Just a
>> > few
>> > > >> > > follow-up
>> > > >> > > > > > >> > comments.
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> > - I see the appeal of merging with 
>>Kafka
>> or
>> > > >> > becoming
>> > > >> > > a
>> > > >> > > > > > >> > subproject:
>> > > >> > > > > > >> > > > the
>> > > >> > > > > > >> > > > > >> > reasons you mention are good. The risk 
>>I
>> see
>> > > is
>> > > >> > that
>> > > >> > > > > > release
>> > > >> > > > > > >> > > > schedules
>> > > >> > > > > > >> > > > > >> > become coupled to each other, which can
>> slow
>> > > >> > everyone
>> > > >> > > > > down,
>> > > >> > > > > > >> and
>> > > >> > > > > > >> > > > large
>> > > >> > > > > > >> > > > > >> > projects with many contributors are 
>>harder
>> > to
>> > > >> > manage.
>> > > >> > > > > > (Jakob,
>> > > >> > > > > > >> > can
>> > > >> > > > > > >> > > > you
>> > > >> > > > > > >> > > > > >> speak
>> > > >> > > > > > >> > > > > >> > from experience, having seen a wider 
>>range
>> > of
>> > > >> > Hadoop
>> > > >> > > > > > ecosystem
>> > > >> > > > > > >> > > > > >> projects?)
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> > Some of the goals of a better unified
>> > > developer
>> > > >> > > > > experience
>> > > >> > > > > > >> could
>> > > >> > > > > > >> > > > also
>> > > >> > > > > > >> > > > > be
>> > > >> > > > > > >> > > > > >> > solved by integrating Samza nicely 
>>into a
>> > > Kafka
>> > > >> > > > > > distribution
>> > > >> > > > > > >> > (such
>> > > >> > > > > > >> > > > as
>> > > >> > > > > > >> > > > > >> > Confluent's). I'm not against merging
>> > projects
>> > > >> if
>> > > >> > we
>> > > >> > > > > decide
>> > > >> > > > > > >> > that's
>> > > >> > > > > > >> > > > the
>> > > >> > > > > > >> > > > > >> way
>> > > >> > > > > > >> > > > > >> > to go, just pointing out the same goals
>> can
>> > > >> perhaps
>> > > >> > > > also
>> > > >> > > > > be
>> > > >> > > > > > >> > > achieved
>> > > >> > > > > > >> > > > > in
>> > > >> > > > > > >> > > > > >> > other ways.
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
>> > dependency:
>> > > >> are
>> > > >> > > you
>> > > >> > > > > > >> proposing
>> > > >> > > > > > >> > > > that
>> > > >> > > > > > >> > > > > >> > Samza doesn't give any help to people
>> > wanting
>> > > to
>> > > >> > run
>> > > >> > > on
>> > > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
>> > > >> > > > > > >> > > > > >> > So the docs would basically have a 
>>link to
>> > > >> Slider
>> > > >> > and
>> > > >> > > > > > nothing
>> > > >> > > > > > >> > > else?
>> > > >> > > > > > >> > > > Or
>> > > >> > > > > > >> > > > > >> > would we maintain integrations with a
>> bunch
>> > of
>> > > >> > > popular
>> > > >> > > > > > >> > deployment
>> > > >> > > > > > >> > > > > >> methods
>> > > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell 
>>scripts
>> > to
>> > > >> make
>> > > >> > > > Samza
>> > > >> > > > > > work
>> > > >> > > > > > >> > with
>> > > >> > > > > > >> > > > > >> Slider)?
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> > I absolutely think it's a good idea to
>> have
>> > > the
>> > > >> > "as a
>> > > >> > > > > > library"
>> > > >> > > > > > >> > and
>> > > >> > > > > > >> > > > > "as a
>> > > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options 
>>for
>> > > >> people
>> > > >> > who
>> > > >> > > > > want
>> > > >> > > > > > >> them,
>> > > >> > > > > > >> > > > but I
>> > > >> > > > > > >> > > > > >> > think there should also be a 
>>low-friction
>> > path
>> > > >> for
>> > > >> > > > common
>> > > >> > > > > > "as
>> > > >> > > > > > >> a
>> > > >> > > > > > >> > > > > service"
>> > > >> > > > > > >> > > > > >> > deployment methods, for which we 
>>probably
>> > need
>> > > >> to
>> > > >> > > > > maintain
>> > > >> > > > > > >> > > > > integrations.
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems
>> odd
>> > to
>> > > >> me,
>> > > >> > > > > because
>> > > >> > > > > > >> Kafka
>> > > >> > > > > > >> > > is
>> > > >> > > > > > >> > > > > all
>> > > >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
>> > > >> Transformers"
>> > > >> > > or
>> > > >> > > > > > "Kafka
>> > > >> > > > > > >> > > > Filters"
>> > > >> > > > > > >> > > > > >> > would be more apt?
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of 
>>Samza
>> > > >> (stream
>> > > >> > > > > > >> transformation
>> > > >> > > > > > >> > > > with
>> > > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as 
>>a
>> > > >> library"
>> > > >> > > bit)
>> > > >> > > > > > could
>> > > >> > > > > > >> > > become
>> > > >> > > > > > >> > > > > >> part of
>> > > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
>> > > >> streaming
>> > > >> > SQL
>> > > >> > > > and
>> > > >> > > > > > >> > > > integrations
>> > > >> > > > > > >> > > > > >> with
>> > > >> > > > > > >> > > > > >> > deployment frameworks remain in a 
>>separate
>> > > >> project?
>> > > >> > > In
>> > > >> > > > > > other
>> > > >> > > > > > >> > > words,
>> > > >> > > > > > >> > > > > >> Kafka
>> > > >> > > > > > >> > > > > >> > would absorb the proven, stable core of
>> > Samza,
>> > > >> > which
>> > > >> > > > > would
>> > > >> > > > > > >> > become
>> > > >> > > > > > >> > > > the
>> > > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in
>> this
>> > > >> > thread.
>> > > >> > > > The
>> > > >> > > > > > Samza
>> > > >> > > > > > >> > > > project
>> > > >> > > > > > >> > > > > >> > would then target that third Kafka 
>>client
>> as
>> > > its
>> > > >> > base
>> > > >> > > > > API,
>> > > >> > > > > > and
>> > > >> > > > > > >> > the
>> > > >> > > > > > >> > > > > >> project
>> > > >> > > > > > >> > > > > >> > would be freed up to explore more
>> > experimental
>> > > >> new
>> > > >> > > > > > horizons.
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> > Martin
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
>> > > >> > > > jay.kreps@gmail.com>
>> > > >> > > > > > >> wrote:
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> > > Hey Martin,
>> > > >> > > > > > >> > > > > >> > >
>> > > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I
>> > actually
>> > > >> > don't
>> > > >> > > > > think
>> > > >> > > > > > it
>> > > >> > > > > > >> > ties
>> > > >> > > > > > >> > > > our
>> > > >> > > > > > >> > > > > >> > hands
>> > > >> > > > > > >> > > > > >> > > at all, all it does is refactor 
>>things.
>> > The
>> > > >> > > division
>> > > >> > > > of
>> > > >> > > > > > >> > > > > >> responsibility is
>> > > >> > > > > > >> > > > > >> > > that Samza core is responsible for 
>>task
>> > > >> > lifecycle,
>> > > >> > > > > state,
>> > > >> > > > > > >> and
>> > > >> > > > > > >> > > > > >> partition
>> > > >> > > > > > >> > > > > >> > > management (using the Kafka
>> co-ordinator)
>> > > but
>> > > >> it
>> > > >> > is
>> > > >> > > > NOT
>> > > >> > > > > > >> > > > responsible
>> > > >> > > > > > >> > > > > >> for
>> > > >> > > > > > >> > > > > >> > > packaging, configuration deployment 
>>or
>> > > >> execution
>> > > >> > of
>> > > >> > > > > > >> processes.
>> > > >> > > > > > >> > > The
>> > > >> > > > > > >> > > > > >> > problem
>> > > >> > > > > > >> > > > > >> > > of packaging and starting these
>> processes
>> > is
>> > > >> > > > > > >> > > > > >> > > framework/environment-specific. This
>> > leaves
>> > > >> > > > individual
>> > > >> > > > > > >> > > frameworks
>> > > >> > > > > > >> > > > to
>> > > >> > > > > > >> > > > > >> be
>> > > >> > > > > > >> > > > > >> > as
>> > > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you
>> can
>> > > get
>> > > >> > > simple
>> > > >> > > > > > >> stateless
>> > > >> > > > > > >> > > > > >> support in
>> > > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their
>> off-the-shelf
>> > > app
>> > > >> > > > > framework
>> > > >> > > > > > >> > > (Slider,
>> > > >> > > > > > >> > > > > >> > Marathon,
>> > > >> > > > > > >> > > > > >> > > etc). These are well known by people 
>>and
>> > > have
>> > > >> > nice
>> > > >> > > > UIs
>> > > >> > > > > > and a
>> > > >> > > > > > >> > lot
>> > > >> > > > > > >> > > > of
>> > > >> > > > > > >> > > > > >> > > flexibility. I don't think they have
>> node
>> > > >> > affinity
>> > > >> > > > as a
>> > > >> > > > > > >> built
>> > > >> > > > > > >> > in
>> > > >> > > > > > >> > > > > >> option
>> > > >> > > > > > >> > > > > >> > > (though I could be wrong). So if we 
>>want
>> > > that
>> > > >> we
>> > > >> > > can
>> > > >> > > > > > either
>> > > >> > > > > > >> > wait
>> > > >> > > > > > >> > > > for
>> > > >> > > > > > >> > > > > >> them
>> > > >> > > > > > >> > > > > >> > > to add it or do a custom framework to
>> add
>> > > that
>> > > >> > > > feature
>> > > >> > > > > > (as
>> > > >> > > > > > >> > now).
>> > > >> > > > > > >> > > > > >> > Obviously
>> > > >> > > > > > >> > > > > >> > > if you manage things with old-school 
>>ops
>> > > tools
>> > > >> > > > > > >> > (puppet/chef/etc)
>> > > >> > > > > > >> > > > you
>> > > >> > > > > > >> > > > > >> get
>> > > >> > > > > > >> > > > > >> > > locality easily. The nice thing, 
>>though,
>> > is
>> > > >> that
>> > > >> > > all
>> > > >> > > > > the
>> > > >> > > > > > >> samza
>> > > >> > > > > > >> > > > > >> "business
>> > > >> > > > > > >> > > > > >> > > logic" around partition management 
>>and
>> > fault
>> > > >> > > > tolerance
>> > > >> > > > > > is in
>> > > >> > > > > > >> > > Samza
>> > > >> > > > > > >> > > > > >> core
>> > > >> > > > > > >> > > > > >> > so
>> > > >> > > > > > >> > > > > >> > > it is shared across frameworks and 
>>the
>> > > >> framework
>> > > >> > > > > specific
>> > > >> > > > > > >> bit
>> > > >> > > > > > >> > is
>> > > >> > > > > > >> > > > > just
>> > > >> > > > > > >> > > > > >> > > whether it is smart enough to try to 
>>get
>> > the
>> > > >> same
>> > > >> > > > host
>> > > >> > > > > > when
>> > > >> > > > > > >> a
>> > > >> > > > > > >> > > job
>> > > >> > > > > > >> > > > is
>> > > >> > > > > > >> > > > > >> > > restarted.
>> > > >> > > > > > >> > > > > >> > >
>> > > >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment,
>> yeah
>> > I
>> > > >> think
>> > > >> > > the
>> > > >> > > > > > goal
>> > > >> > > > > > >> > would
>> > > >> > > > > > >> > > > be
>> > > >> > > > > > >> > > > > >> (a)
>> > > >> > > > > > >> > > > > >> > > actually get better alignment in user
>> > > >> experience,
>> > > >> > > and
>> > > >> > > > > (b)
>> > > >> > > > > > >> > > express
>> > > >> > > > > > >> > > > > >> this in
>> > > >> > > > > > >> > > > > >> > > the naming and project branding.
>> > > Specifically:
>> > > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for
>> the
>> > > >> > > > > > "transformation"
>> > > >> > > > > > >> api
>> > > >> > > > > > >> > > to
>> > > >> > > > > > >> > > > be
>> > > >> > > > > > >> > > > > >> > > discoverable in the main Kafka
>> docs--i.e.
>> > be
>> > > >> able
>> > > >> > > to
>> > > >> > > > > > explain
>> > > >> > > > > > >> > > when
>> > > >> > > > > > >> > > > to
>> > > >> > > > > > >> > > > > >> use
>> > > >> > > > > > >> > > > > >> > > the consumer and when to use the 
>>stream
>> > > >> > processing
>> > > >> > > > > > >> > functionality
>> > > >> > > > > > >> > > > and
>> > > >> > > > > > >> > > > > >> lead
>> > > >> > > > > > >> > > > > >> > > people into that experience.
>> > > >> > > > > > >> > > > > >> > > 2. Align releases so if you get 
>>Kafkza
>> > 1.4.2
>> > > >> (or
>> > > >> > > > > > whatever)
>> > > >> > > > > > >> > that
>> > > >> > > > > > >> > > > has
>> > > >> > > > > > >> > > > > >> both
>> > > >> > > > > > >> > > > > >> > > Kafka and the stream processing part 
>>and
>> > > they
>> > > >> > > > actually
>> > > >> > > > > > work
>> > > >> > > > > > >> > > > > together.
>> > > >> > > > > > >> > > > > >> > > 3. Unify the programming experience 
>>so
>> the
>> > > >> client
>> > > >> > > and
>> > > >> > > > > > Samza
>> > > >> > > > > > >> > api
>> > > >> > > > > > >> > > > > share
>> > > >> > > > > > >> > > > > >> > > 
>>config/monitoring/naming/packaging/etc.
>> > > >> > > > > > >> > > > > >> > >
>> > > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
>> > > committers
>> > > >> and
>> > > >> > > can
>> > > >> > > > > > have a
>> > > >> > > > > > >> > > > > separate
>> > > >> > > > > > >> > > > > >> > repo,
>> > > >> > > > > > >> > > > > >> > > but I'm actually not really sure (I
>> can't
>> > > >> find a
>> > > >> > > > > > definition
>> > > >> > > > > > >> > of a
>> > > >> > > > > > >> > > > > >> > subproject
>> > > >> > > > > > >> > > > > >> > > in Apache).
>> > > >> > > > > > >> > > > > >> > >
>> > > >> > > > > > >> > > > > >> > > Basically at a high-level you want 
>>the
>> > > >> experience
>> > > >> > > to
>> > > >> > > > > > "feel"
>> > > >> > > > > > >> > > like a
>> > > >> > > > > > >> > > > > >> single
>> > > >> > > > > > >> > > > > >> > > system, not to relatively independent
>> > things
>> > > >> that
>> > > >> > > are
>> > > >> > > > > > kind
>> > > >> > > > > > >> of
>> > > >> > > > > > >> > > > > >> awkwardly
>> > > >> > > > > > >> > > > > >> > > glued together.
>> > > >> > > > > > >> > > > > >> > >
>> > > >> > > > > > >> > > > > >> > > I think if we did that they having
>> naming
>> > or
>> > > >> > > branding
>> > > >> > > > > > like
>> > > >> > > > > > >> > > "kafka
>> > > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or
>> something
>> > > >> like
>> > > >> > > that
>> > > >> > > > > > would
>> > > >> > > > > > >> > > > actually
>> > > >> > > > > > >> > > > > >> do a
>> > > >> > > > > > >> > > > > >> > > good job of conveying what it is. I 
>>do
>> > that
>> > > >> this
>> > > >> > > > would
>> > > >> > > > > > help
>> > > >> > > > > > >> > > > adoption
>> > > >> > > > > > >> > > > > >> > quite
>> > > >> > > > > > >> > > > > >> > > a lot as it would correctly convey 
>>that
>> > > using
>> > > >> > Kafka
>> > > >> > > > > > >> Streaming
>> > > >> > > > > > >> > > with
>> > > >> > > > > > >> > > > > >> Kafka
>> > > >> > > > > > >> > > > > >> > is
>> > > >> > > > > > >> > > > > >> > > a fairly seamless experience and 
>>Kafka
>> is
>> > > >> pretty
>> > > >> > > > > heavily
>> > > >> > > > > > >> > adopted
>> > > >> > > > > > >> > > > at
>> > > >> > > > > > >> > > > > >> this
>> > > >> > > > > > >> > > > > >> > > point.
>> > > >> > > > > > >> > > > > >> > >
>> > > >> > > > > > >> > > > > >> > > Fwiw we actually considered this 
>>model
>> > > >> originally
>> > > >> > > > when
>> > > >> > > > > > open
>> > > >> > > > > > >> > > > sourcing
>> > > >> > > > > > >> > > > > >> > Samza,
>> > > >> > > > > > >> > > > > >> > > however at that time Kafka was
>> relatively
>> > > >> unknown
>> > > >> > > and
>> > > >> > > > > we
>> > > >> > > > > > >> > decided
>> > > >> > > > > > >> > > > not
>> > > >> > > > > > >> > > > > >> to
>> > > >> > > > > > >> > > > > >> > do
>> > > >> > > > > > >> > > > > >> > > it since we felt it would be 
>>limiting.
>> > From
>> > > my
>> > > >> > > point
>> > > >> > > > of
>> > > >> > > > > > view
>> > > >> > > > > > >> > the
>> > > >> > > > > > >> > > > > three
>> > > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now
>> > really
>> > > >> > heavily
>> > > >> > > > > used
>> > > >> > > > > > for
>> > > >> > > > > > >> > > > stream
>> > > >> > > > > > >> > > > > >> > > processing, (2) we learned that
>> > abstracting
>> > > >> out
>> > > >> > the
>> > > >> > > > > > stream
>> > > >> > > > > > >> > well
>> > > >> > > > > > >> > > is
>> > > >> > > > > > >> > > > > >> > > basically impossible, (3) we learned 
>>it
>> is
>> > > >> really
>> > > >> > > > hard
>> > > >> > > > > to
>> > > >> > > > > > >> keep
>> > > >> > > > > > >> > > the
>> > > >> > > > > > >> > > > > two
>> > > >> > > > > > >> > > > > >> > > things feeling like a single product.
>> > > >> > > > > > >> > > > > >> > >
>> > > >> > > > > > >> > > > > >> > > -Jay
>> > > >> > > > > > >> > > > > >> > >
>> > > >> > > > > > >> > > > > >> > >
>> > > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, 
>>Martin
>> > > >> Kleppmann
>> > > >> > <
>> > > >> > > > > > >> > > > > >> martin@kleppmann.com>
>> > > >> > > > > > >> > > > > >> > > wrote:
>> > > >> > > > > > >> > > > > >> > >
>> > > >> > > > > > >> > > > > >> > >> Hi all,
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> > >> I agree with the general philosophy 
>>of
>> > > tying
>> > > >> > Samza
>> > > >> > > > > more
>> > > >> > > > > > >> > firmly
>> > > >> > > > > > >> > > to
>> > > >> > > > > > >> > > > > >> Kafka.
>> > > >> > > > > > >> > > > > >> > >> After I spent a while looking at
>> > > integrating
>> > > >> > other
>> > > >> > > > > > message
>> > > >> > > > > > >> > > > brokers
>> > > >> > > > > > >> > > > > >> (e.g.
>> > > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I 
>>came to
>> > the
>> > > >> > > > conclusion
>> > > >> > > > > > that
>> > > >> > > > > > >> > > > > >> > SystemConsumer
>> > > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
>> > > Kafka's
>> > > >> > that
>> > > >> > > > > pretty
>> > > >> > > > > > >> much
>> > > >> > > > > > >> > > > > nobody
>> > > >> > > > > > >> > > > > >> but
>> > > >> > > > > > >> > > > > >> > >> Kafka actually implements it. 
>>(Databus
>> is
>> > > >> > perhaps
>> > > >> > > an
>> > > >> > > > > > >> > exception,
>> > > >> > > > > > >> > > > but
>> > > >> > > > > > >> > > > > >> it
>> > > >> > > > > > >> > > > > >> > >> isn't widely used outside of 
>>LinkedIn.)
>> > > Thus,
>> > > >> > > making
>> > > >> > > > > > Samza
>> > > >> > > > > > >> > > fully
>> > > >> > > > > > >> > > > > >> > dependent
>> > > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
>> > > >> > system-independence
>> > > >> > > > was
>> > > >> > > > > > >> never
>> > > >> > > > > > >> > as
>> > > >> > > > > > >> > > > > real
>> > > >> > > > > > >> > > > > >> as
>> > > >> > > > > > >> > > > > >> > we
>> > > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The 
>>gains of
>> > > code
>> > > >> > reuse
>> > > >> > > > are
>> > > >> > > > > > >> real.
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from 
>>YARN
>> > has
>> > > >> also
>> > > >> > > > always
>> > > >> > > > > > been
>> > > >> > > > > > >> > > > > >> appealing to
>> > > >> > > > > > >> > > > > >> > >> me, for various reasons already
>> mentioned
>> > > in
>> > > >> > this
>> > > >> > > > > > thread.
>> > > >> > > > > > >> > > > Although
>> > > >> > > > > > >> > > > > >> > making
>> > > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
>> > > >> > > > (YARN/Mesos/AWS/etc)
>> > > >> > > > > > >> seems
>> > > >> > > > > > >> > > > > >> laudable,
>> > > >> > > > > > >> > > > > >> > I am
>> > > >> > > > > > >> > > > > >> > >> a little concerned that it will
>> restrict
>> > us
>> > > >> to a
>> > > >> > > > > lowest
>> > > >> > > > > > >> > common
>> > > >> > > > > > >> > > > > >> > denominator.
>> > > >> > > > > > >> > > > > >> > >> For example, would host affinity
>> > > (SAMZA-617)
>> > > >> > still
>> > > >> > > > be
>> > > >> > > > > > >> > possible?
>> > > >> > > > > > >> > > > For
>> > > >> > > > > > >> > > > > >> jobs
>> > > >> > > > > > >> > > > > >> > >> with large amounts of state, I think
>> > > >> SAMZA-617
>> > > >> > > would
>> > > >> > > > > be
>> > > >> > > > > > a
>> > > >> > > > > > >> big
>> > > >> > > > > > >> > > > boon,
>> > > >> > > > > > >> > > > > >> > since
>> > > >> > > > > > >> > > > > >> > >> restoring state off the changelog on
>> > every
>> > > >> > single
>> > > >> > > > > > restart
>> > > >> > > > > > >> is
>> > > >> > > > > > >> > > > > painful,
>> > > >> > > > > > >> > > > > >> > due
>> > > >> > > > > > >> > > > > >> > >> to long recovery times. It would be 
>>a
>> > shame
>> > > >> if
>> > > >> > the
>> > > >> > > > > > >> decoupling
>> > > >> > > > > > >> > > > from
>> > > >> > > > > > >> > > > > >> YARN
>> > > >> > > > > > >> > > > > >> > >> made host affinity impossible.
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> > >> Jay, a question about the proposed 
>>API
>> > for
>> > > >> > > > > > instantiating a
>> > > >> > > > > > >> > job
>> > > >> > > > > > >> > > in
>> > > >> > > > > > >> > > > > >> code
>> > > >> > > > > > >> > > > > >> > >> (rather than a properties file): 
>>when
>> > > >> > submitting a
>> > > >> > > > job
>> > > >> > > > > > to a
>> > > >> > > > > > >> > > > > cluster,
>> > > >> > > > > > >> > > > > >> is
>> > > >> > > > > > >> > > > > >> > the
>> > > >> > > > > > >> > > > > >> > >> idea that the instantiation code 
>>runs
>> on
>> > a
>> > > >> > client
>> > > >> > > > > > >> somewhere,
>> > > >> > > > > > >> > > > which
>> > > >> > > > > > >> > > > > >> then
>> > > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
>> > > >> > > YARN/Mesos/AWS/etc?
>> > > >> > > > > Or
>> > > >> > > > > > >> does
>> > > >> > > > > > >> > > that
>> > > >> > > > > > >> > > > > >> code
>> > > >> > > > > > >> > > > > >> > run
>> > > >> > > > > > >> > > > > >> > >> on each container that is part of 
>>the
>> job
>> > > (in
>> > > >> > > which
>> > > >> > > > > > case,
>> > > >> > > > > > >> how
>> > > >> > > > > > >> > > > does
>> > > >> > > > > > >> > > > > >> the
>> > > >> > > > > > >> > > > > >> > job
>> > > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't 
>>feel
>> > > >> right to
>> > > >> > > > make
>> > > >> > > > > a
>> > > >> > > > > > 1.0
>> > > >> > > > > > >> > > > release
>> > > >> > > > > > >> > > > > >> > with a
>> > > >> > > > > > >> > > > > >> > >> plan for it to be immediately 
>>obsolete.
>> > So
>> > > if
>> > > >> > this
>> > > >> > > > is
>> > > >> > > > > > going
>> > > >> > > > > > >> > to
>> > > >> > > > > > >> > > > > >> happen, I
>> > > >> > > > > > >> > > > > >> > >> think it would be more honest to 
>>stick
>> > with
>> > > >> 0.*
>> > > >> > > > > version
>> > > >> > > > > > >> > numbers
>> > > >> > > > > > >> > > > > until
>> > > >> > > > > > >> > > > > >> > the
>> > > >> > > > > > >> > > > > >> > >> library-ified Samza has been
>> implemented,
>> > > is
>> > > >> > > stable
>> > > >> > > > > and
>> > > >> > > > > > >> > widely
>> > > >> > > > > > >> > > > > used.
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> > >> Should the new Samza be a 
>>subproject of
>> > > >> Kafka?
>> > > >> > > There
>> > > >> > > > > is
>> > > >> > > > > > >> > > precedent
>> > > >> > > > > > >> > > > > for
>> > > >> > > > > > >> > > > > >> > >> tight coupling between different 
>>Apache
>> > > >> projects
>> > > >> > > > (e.g.
>> > > >> > > > > > >> > Curator
>> > > >> > > > > > >> > > > and
>> > > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I
>> > think
>> > > >> > > remaining
>> > > >> > > > > > >> separate
>> > > >> > > > > > >> > > > would
>> > > >> > > > > > >> > > > > >> be
>> > > >> > > > > > >> > > > > >> > ok.
>> > > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on
>> > Kafka,
>> > > >> there
>> > > >> > > is
>> > > >> > > > > > enough
>> > > >> > > > > > >> > > > > substance
>> > > >> > > > > > >> > > > > >> in
>> > > >> > > > > > >> > > > > >> > >> Samza that it warrants being a 
>>separate
>> > > >> project.
>> > > >> > > An
>> > > >> > > > > > >> argument
>> > > >> > > > > > >> > in
>> > > >> > > > > > >> > > > > >> favour
>> > > >> > > > > > >> > > > > >> > of
>> > > >> > > > > > >> > > > > >> > >> merging would be if we think Kafka 
>>has
>> a
>> > > much
>> > > >> > > > stronger
>> > > >> > > > > > >> "brand
>> > > >> > > > > > >> > > > > >> presence"
>> > > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that 
>>one.
>> > If
>> > > >> the
>> > > >> > > Kafka
>> > > >> > > > > > >> project
>> > > >> > > > > > >> > is
>> > > >> > > > > > >> > > > > >> willing
>> > > >> > > > > > >> > > > > >> > to
>> > > >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way 
>>of
>> > > doing
>> > > >> > > > stateful
>> > > >> > > > > > >> stream
>> > > >> > > > > > >> > > > > >> > >> transformations, that would probably
>> have
>> > > >> much
>> > > >> > the
>> > > >> > > > > same
>> > > >> > > > > > >> > effect
>> > > >> > > > > > >> > > as
>> > > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
>> > > >> Processors"
>> > > >> > or
>> > > >> > > > > > suchlike.
>> > > >> > > > > > >> > > Close
>> > > >> > > > > > >> > > > > >> > >> collaboration between the two 
>>projects
>> > will
>> > > >> be
>> > > >> > > > needed
>> > > >> > > > > in
>> > > >> > > > > > >> any
>> > > >> > > > > > >> > > > case.
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> > >> From a project management 
>>perspective,
>> I
>> > > >> guess
>> > > >> > the
>> > > >> > > > > "new
>> > > >> > > > > > >> > Samza"
>> > > >> > > > > > >> > > > > would
>> > > >> > > > > > >> > > > > >> > have
>> > > >> > > > > > >> > > > > >> > >> to be developed on a branch 
>>alongside
>> > > ongoing
>> > > >> > > > > > maintenance
>> > > >> > > > > > >> of
>> > > >> > > > > > >> > > the
>> > > >> > > > > > >> > > > > >> current
>> > > >> > > > > > >> > > > > >> > >> line of development? I think it 
>>would
>> be
>> > > >> > important
>> > > >> > > > to
>> > > >> > > > > > >> > continue
>> > > >> > > > > > >> > > > > >> > supporting
>> > > >> > > > > > >> > > > > >> > >> existing users, and provide a 
>>graceful
>> > > >> migration
>> > > >> > > > path
>> > > >> > > > > to
>> > > >> > > > > > >> the
>> > > >> > > > > > >> > > new
>> > > >> > > > > > >> > > > > >> > version.
>> > > >> > > > > > >> > > > > >> > >> Leaving the current versions
>> unsupported
>> > > and
>> > > >> > > forcing
>> > > >> > > > > > people
>> > > >> > > > > > >> > to
>> > > >> > > > > > >> > > > > >> rewrite
>> > > >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> > >> Best,
>> > > >> > > > > > >> > > > > >> > >> Martin
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
>> > > >> > > > jay@confluent.io>
>> > > >> > > > > > >> wrote:
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> > >>> Hey Garry,
>> > > >> > > > > > >> > > > > >> > >>>
>> > > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd 
>>be
>> > > happy
>> > > >> to
>> > > >> > > chat
>> > > >> > > > > > more
>> > > >> > > > > > >> > about
>> > > >> > > > > > >> > > > > this
>> > > >> > > > > > >> > > > > >> if
>> > > >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris
>> and I
>> > > >> > started
>> > > >> > > > with
>> > > >> > > > > > the
>> > > >> > > > > > >> > idea
>> > > >> > > > > > >> > > > of
>> > > >> > > > > > >> > > > > >> "what
>> > > >> > > > > > >> > > > > >> > >>> would it take to make Samza a 
>>kick-ass
>> > > >> > ingestion
>> > > >> > > > > tool"
>> > > >> > > > > > but
>> > > >> > > > > > >> > > > > >> ultimately
>> > > >> > > > > > >> > > > > >> > we
>> > > >> > > > > > >> > > > > >> > >>> kind of came around to the idea 
>>that
>> > > >> ingestion
>> > > >> > > and
>> > > >> > > > > > >> > > > transformation
>> > > >> > > > > > >> > > > > >> had
>> > > >> > > > > > >> > > > > >> > >>> pretty different needs and coupling
>> the
>> > > two
>> > > >> > made
>> > > >> > > > > things
>> > > >> > > > > > >> > hard.
>> > > >> > > > > > >> > > > > >> > >>>
>> > > >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
>> > > (KIP-26)
>> > > >> > > > actually
>> > > >> > > > > > will
>> > > >> > > > > > >> > do
>> > > >> > > > > > >> > > > what
>> > > >> > > > > > >> > > > > >> you
>> > > >> > > > > > >> > > > > >> > >> are
>> > > >> > > > > > >> > > > > >> > >>> looking for.
>> > > >> > > > > > >> > > > > >> > >>>
>> > > >> > > > > > >> > > > > >> > >>> With regard to your point about
>> slider,
>> > I
>> > > >> don't
>> > > >> > > > > > >> necessarily
>> > > >> > > > > > >> > > > > >> disagree.
>> > > >> > > > > > >> > > > > >> > >> But I
>> > > >> > > > > > >> > > > > >> > >>> think getting good YARN support is
>> quite
>> > > >> doable
>> > > >> > > > and I
>> > > >> > > > > > >> think
>> > > >> > > > > > >> > we
>> > > >> > > > > > >> > > > can
>> > > >> > > > > > >> > > > > >> make
>> > > >> > > > > > >> > > > > >> > >>> that work well. I think the issue 
>>this
>> > > >> proposal
>> > > >> > > > > solves
>> > > >> > > > > > is
>> > > >> > > > > > >> > that
>> > > >> > > > > > >> > > > > >> > >> technically
>> > > >> > > > > > >> > > > > >> > >>> it is pretty hard to support 
>>multiple
>> > > >> cluster
>> > > >> > > > > > management
>> > > >> > > > > > >> > > systems
>> > > >> > > > > > >> > > > > the
>> > > >> > > > > > >> > > > > >> > way
>> > > >> > > > > > >> > > > > >> > >>> things are now, you need to write 
>>an
>> > "app
>> > > >> > master"
>> > > >> > > > or
>> > > >> > > > > > >> > > "framework"
>> > > >> > > > > > >> > > > > for
>> > > >> > > > > > >> > > > > >> > each
>> > > >> > > > > > >> > > > > >> > >>> and they are all a little 
>>different so
>> > > >> testing
>> > > >> > is
>> > > >> > > > > > really
>> > > >> > > > > > >> > hard.
>> > > >> > > > > > >> > > > In
>> > > >> > > > > > >> > > > > >> the
>> > > >> > > > > > >> > > > > >> > >>> absence of this we have been stuck
>> with
>> > > just
>> > > >> > YARN
>> > > >> > > > > which
>> > > >> > > > > > >> has
>> > > >> > > > > > >> > > > > >> fantastic
>> > > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of 
>>the
>> > > org,
>> > > >> but
>> > > >> > > > zero
>> > > >> > > > > > >> > > penetration
>> > > >> > > > > > >> > > > > >> > >> elsewhere.
>> > > >> > > > > > >> > > > > >> > >>> Given the huge amount of work being
>> put
>> > in
>> > > >> to
>> > > >> > > > slider,
>> > > >> > > > > > >> > > marathon,
>> > > >> > > > > > >> > > > > aws
>> > > >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
>> > > related
>> > > >> > > > packaging
>> > > >> > > > > > >> > > > technologies
>> > > >> > > > > > >> > > > > >> > people
>> > > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes,
>> various
>> > > >> > > > > cloud-specific
>> > > >> > > > > > >> > deploy
>> > > >> > > > > > >> > > > > >> tools,
>> > > >> > > > > > >> > > > > >> > >> etc)
>> > > >> > > > > > >> > > > > >> > >>> I really think it is important to 
>>get
>> > this
>> > > >> > right.
>> > > >> > > > > > >> > > > > >> > >>>
>> > > >> > > > > > >> > > > > >> > >>> -Jay
>> > > >> > > > > > >> > > > > >> > >>>
>> > > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, 
>>Garry
>> > > >> > Turkington
>> > > >> > > <
>> > > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com>
>> wrote:
>> > > >> > > > > > >> > > > > >> > >>>
>> > > >> > > > > > >> > > > > >> > >>>> Hi all,
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>> I think the question below re does
>> > Samza
>> > > >> > become
>> > > >> > > a
>> > > >> > > > > > >> > sub-project
>> > > >> > > > > > >> > > > of
>> > > >> > > > > > >> > > > > >> Kafka
>> > > >> > > > > > >> > > > > >> > >>>> highlights the broader point 
>>around
>> > > >> migration.
>> > > >> > > > Chris
>> > > >> > > > > > >> > mentions
>> > > >> > > > > > >> > > > > >> Samza's
>> > > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1
>> > release
>> > > >> but
>> > > >> > I'm
>> > > >> > > > not
>> > > >> > > > > > sure
>> > > >> > > > > > >> > it
>> > > >> > > > > > >> > > > > feels
>> > > >> > > > > > >> > > > > >> > >> right to
>> > > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan 
>>to
>> > > >> deprecate
>> > > >> > > > most
>> > > >> > > > > of
>> > > >> > > > > > >> it.
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have
>> some
>> > > guys
>> > > >> > who
>> > > >> > > > have
>> > > >> > > > > > >> > started
>> > > >> > > > > > >> > > > > >> working
>> > > >> > > > > > >> > > > > >> > >> with
>> > > >> > > > > > >> > > > > >> > >>>> Samza and building some new
>> > > >> > consumers/producers
>> > > >> > > > was
>> > > >> > > > > > next
>> > > >> > > > > > >> > up.
>> > > >> > > > > > >> > > > > Sounds
>> > > >> > > > > > >> > > > > >> > like
>> > > >> > > > > > >> > > > > >> > >>>> that is absolutely not the 
>>direction
>> to
>> > > >> go. I
>> > > >> > > need
>> > > >> > > > > to
>> > > >> > > > > > >> look
>> > > >> > > > > > >> > > into
>> > > >> > > > > > >> > > > > the
>> > > >> > > > > > >> > > > > >> > KIP
>> > > >> > > > > > >> > > > > >> > >> in
>> > > >> > > > > > >> > > > > >> > >>>> more detail but for me the
>> > attractiveness
>> > > >> of
>> > > >> > > > adding
>> > > >> > > > > > new
>> > > >> > > > > > >> > Samza
>> > > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes 
>>all
>> > > they
>> > > >> > were
>> > > >> > > > > doing
>> > > >> > > > > > was
>> > > >> > > > > > >> > > > really
>> > > >> > > > > > >> > > > > >> > getting
>> > > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  
>>was to
>> > > avoid
>> > > >> > > > having
>> > > >> > > > > to
>> > > >> > > > > > >> > worry
>> > > >> > > > > > >> > > > > about
>> > > >> > > > > > >> > > > > >> the
>> > > >> > > > > > >> > > > > >> > >>>> lifecycle management of external
>> > clients.
>> > > >> If
>> > > >> > > there
>> > > >> > > > > is
>> > > >> > > > > > a
>> > > >> > > > > > >> > > generic
>> > > >> > > > > > >> > > > > >> Kafka
>> > > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can 
>>plug
>> a
>> > > new
>> > > >> > > > connector
>> > > >> > > > > > into
>> > > >> > > > > > >> > and
>> > > >> > > > > > >> > > > > have
>> > > >> > > > > > >> > > > > >> a
>> > > >> > > > > > >> > > > > >> > >> lot of
>> > > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
>> > > reliability
>> > > >> > done
>> > > >> > > > for
>> > > >> > > > > me
>> > > >> > > > > > >> then
>> > > >> > > > > > >> > > it
>> > > >> > > > > > >> > > > > >> gives
>> > > >> > > > > > >> > > > > >> > me
>> > > >> > > > > > >> > > > > >> > >> all
>> > > >> > > > > > >> > > > > >> > >>>> the pushing new 
>>consumers/producers
>> > > would.
>> > > >> If
>> > > >> > > not
>> > > >> > > > > > then it
>> > > >> > > > > > >> > > > > >> complicates
>> > > >> > > > > > >> > > > > >> > my
>> > > >> > > > > > >> > > > > >> > >>>> operational deployments.
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>> Which is similar to my other 
>>question
>> > > with
>> > > >> the
>> > > >> > > > > > proposal
>> > > >> > > > > > >> --
>> > > >> > > > > > >> > if
>> > > >> > > > > > >> > > > we
>> > > >> > > > > > >> > > > > >> > build a
>> > > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza
>> plus
>> > > the
>> > > >> > > > requisite
>> > > >> > > > > > >> shims
>> > > >> > > > > > >> > to
>> > > >> > > > > > >> > > > > >> > integrate
>> > > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the 
>>former
>> > may
>> > > >> be a
>> > > >> > > lot
>> > > >> > > > > more
>> > > >> > > > > > >> work
>> > > >> > > > > > >> > > > than
>> > > >> > > > > > >> > > > > we
>> > > >> > > > > > >> > > > > >> > >> think.
>> > > >> > > > > > >> > > > > >> > >>>> We may make it much easier for a
>> > newcomer
>> > > >> to
>> > > >> > get
>> > > >> > > > > > >> something
>> > > >> > > > > > >> > > > > running
>> > > >> > > > > > >> > > > > >> but
>> > > >> > > > > > >> > > > > >> > >>>> having them step up and get a
>> reliable
>> > > >> > > production
>> > > >> > > > > > >> > deployment
>> > > >> > > > > > >> > > > may
>> > > >> > > > > > >> > > > > >> still
>> > > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if
>> for
>> > > >> > different
>> > > >> > > > > > reasons
>> > > >> > > > > > >> > than
>> > > >> > > > > > >> > > > > >> today.
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm 
>>comfortable
>> > > with
>> > > >> > > making
>> > > >> > > > > the
>> > > >> > > > > > >> Samza
>> > > >> > > > > > >> > > > > >> dependency
>> > > >> > > > > > >> > > > > >> > >> on
>> > > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
>> > absolutely
>> > > >> see
>> > > >> > > the
>> > > >> > > > > > >> benefits
>> > > >> > > > > > >> > > in
>> > > >> > > > > > >> > > > > the
>> > > >> > > > > > >> > > > > >> > >>>> reduction of duplication and 
>>clashing
>> > > >> > > > > > >> > > > terminologies/abstractions
>> > > >> > > > > > >> > > > > >> that
>> > > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a
>> library
>> > > >> would
>> > > >> > > > likely
>> > > >> > > > > > be a
>> > > >> > > > > > >> > very
>> > > >> > > > > > >> > > > > nice
>> > > >> > > > > > >> > > > > >> > tool
>> > > >> > > > > > >> > > > > >> > >> to
>> > > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just
>> have
>> > > the
>> > > >> > > > concerns
>> > > >> > > > > > >> above
>> > > >> > > > > > >> > re
>> > > >> > > > > > >> > > > the
>> > > >> > > > > > >> > > > > >> > >>>> operational side.
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>> Garry
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
>> > > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci 
>>Morales
>> > > >> [mailto:
>> > > >> > > > > > >> > gdfm@apache.org
>> > > >> > > > > > >> > > ]
>> > > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
>> > > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
>> > > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and
>> obesrvations
>> > on
>> > > >> > Samza
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
>> > > >> > > > > > >> > > > > >> > >>>> From outside, I have always 
>>perceived
>> > > Samza
>> > > >> > as a
>> > > >> > > > > > >> computing
>> > > >> > > > > > >> > > > layer
>> > > >> > > > > > >> > > > > >> over
>> > > >> > > > > > >> > > > > >> > >>>> Kafka.
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit
>> provocative,
>> > is
>> > > >> > > "should
>> > > >> > > > > > Samza
>> > > >> > > > > > >> be
>> > > >> > > > > > >> > a
>> > > >> > > > > > >> > > > > >> > sub-project
>> > > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
>> > > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it 
>>as a
>> > > >> separate
>> > > >> > > > > project
>> > > >> > > > > > >> > with a
>> > > >> > > > > > >> > > > > >> separate
>> > > >> > > > > > >> > > > > >> > >>>> governance?
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>> Cheers,
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>> --
>> > > >> > > > > > >> > > > > >> > >>>> Gianmarco
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang 
>><
>> > > >> > > > > > yanfang724@gmail.com>
>> > > >> > > > > > >> > > > wrote:
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with
>> Kafka
>> > > more
>> > > >> > > > tightly.
>> > > >> > > > > > >> > Because
>> > > >> > > > > > >> > > > > Samza
>> > > >> > > > > > >> > > > > >> de
>> > > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it
>> should
>> > > >> > leverage
>> > > >> > > > > what
>> > > >> > > > > > >> Kafka
>> > > >> > > > > > >> > > > has.
>> > > >> > > > > > >> > > > > At
>> > > >> > > > > > >> > > > > >> > the
>> > > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
>> > > reinvent
>> > > >> > what
>> > > >> > > > > Samza
>> > > >> > > > > > >> > > already
>> > > >> > > > > > >> > > > > >> has. I
>> > > >> > > > > > >> > > > > >> > >>>>> also like the idea of separating 
>>the
>> > > >> > ingestion
>> > > >> > > > and
>> > > >> > > > > > >> > > > > transformation.
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for 
>>me
>> to
>> > > >> image
>> > > >> > > how
>> > > >> > > > > the
>> > > >> > > > > > >> Samza
>> > > >> > > > > > >> > > > will
>> > > >> > > > > > >> > > > > >> look
>> > > >> > > > > > >> > > > > >> > >>>> like.
>> > > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a
>> little
>> > > >> > > difference
>> > > >> > > > > in
>> > > >> > > > > > >> terms
>> > > >> > > > > > >> > > of
>> > > >> > > > > > >> > > > > how
>> > > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's
>> code
>> > > >> shows
>> > > >> > (A
>> > > >> > > > > > client of
>> > > >> > > > > > >> > > > Kakfa)
>> > > >> > > > > > >> > > > > ?
>> > > >> > > > > > >> > > > > >> And
>> > > >> > > > > > >> > > > > >> > >>>>> user's application code calls 
>>this
>> > > client?
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library 
>>of
>> > > Kafka
>> > > >> > (like
>> > > >> > > > > what
>> > > >> > > > > > the
>> > > >> > > > > > >> > > code
>> > > >> > > > > > >> > > > > >> shows),
>> > > >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance 
>>and
>> > > >> > > > > fault-tolerance?
>> > > >> > > > > > >> Are
>> > > >> > > > > > >> > > they
>> > > >> > > > > > >> > > > > >> taken
>> > > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
>> > > >> mechanism,
>> > > >> > > such
>> > > >> > > > > as
>> > > >> > > > > > >> > "Samza
>> > > >> > > > > > >> > > > > >> worker"
>> > > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>> 2. What about other features, 
>>such
>> as
>> > > >> > > > auto-scaling,
>> > > >> > > > > > >> shared
>> > > >> > > > > > >> > > > > state,
>> > > >> > > > > > >> > > > > >> > >>>>> monitoring?
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, 
>>(is
>> > > this
>> > > >> > what
>> > > >> > > > > Chris
>> > > >> > > > > > >> > > > suggests?)
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data 
>>from
>> > > Kakfa
>> > > >> > and
>> > > >> > > > > > produce
>> > > >> > > > > > >> to
>> > > >> > > > > > >> > > it.
>> > > >> > > > > > >> > > > > >> Then it
>> > > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza 
>>looks
>> > > like
>> > > >> > now,
>> > > >> > > > > > except it
>> > > >> > > > > > >> > > does
>> > > >> > > > > > >> > > > > not
>> > > >> > > > > > >> > > > > >> > rely
>> > > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can 
>>it
>> > > >> leverage
>> > > >> > > > Kafka's
>> > > >> > > > > > >> > metrics,
>> > > >> > > > > > >> > > > > logs,
>> > > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the
>> dependency?
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>> Thanks,
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
>> > > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM,
>> > Guozhang
>> > > >> > Wang <
>> > > >> > > > > > >> > > > > wangguoz@gmail.com
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> > >>>> wrote:
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>> Read through the code example 
>>and
>> it
>> > > >> looks
>> > > >> > > good
>> > > >> > > > to
>> > > >> > > > > > me.
>> > > >> > > > > > >> A
>> > > >> > > > > > >> > > few
>> > > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as 
>>executable
>> > > >> runnable
>> > > >> > > like:
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
>> > > >> > > --config-factory=...
>> > > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
>> > > deploying
>> > > >> > Samza
>> > > >> > > > > more
>> > > >> > > > > > as
>> > > >> > > > > > >> > > > embedded
>> > > >> > > > > > >> > > > > >> > >>>>>> libraries in user application 
>>code
>> > > >> (ignoring
>> > > >> > > the
>> > > >> > > > > > >> > > terminology
>> > > >> > > > > > >> > > > > >> since
>> > > >> > > > > > >> > > > > >> > >>>>>> it is not the
>> > > >> > > > > > >> > > > > >> > >>>>> same
>> > > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
>> > > >> MyStreamTask(configs);
>> > > >> > > > > Thread
>> > > >> > > > > > >> > thread
>> > > >> > > > > > >> > > =
>> > > >> > > > > > >> > > > > new
>> > > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment
>> > modes
>> > > >> are
>> > > >> > > > > important
>> > > >> > > > > > >> for
>> > > >> > > > > > >> > > > > >> different
>> > > >> > > > > > >> > > > > >> > >>>>>> types
>> > > >> > > > > > >> > > > > >> > >>>>> of
>> > > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making
>> > Samza
>> > > >> > purely
>> > > >> > > > > > >> standalone
>> > > >> > > > > > >> > is
>> > > >> > > > > > >> > > > > still
>> > > >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable 
>>or
>> > > library
>> > > >> > > modes.
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>> Guozhang
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 
>>PM,
>> Jay
>> > > >> Kreps
>> > > >> > <
>> > > >> > > > > > >> > > > jay@confluent.io>
>> > > >> > > > > > >> > > > > >> > wrote:
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the 
>>code
>> > > >> example,
>> > > >> > it
>> > > >> > > > was
>> > > >> > > > > > >> > supposed
>> > > >> > > > > > >> > > > to
>> > > >> > > > > > >> > > > > >> look
>> > > >> > > > > > >> > > > > >> > >>>>>>> like
>> > > >> > > > > > >> > > > > >> > >>>>>>> this:
>> > > >> > > > > > >> > > > > >> > >>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new
>> Properties();
>> > > >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
>> > > >> > > > "localhost:4242");
>> > > >> > > > > > >> > > > > >> StreamingConfig
>> > > >> > > > > > >> > > > > >> > >>>>>>> config = new
>> StreamingConfig(props);
>> > > >> > > > > > >> > > > > >> > >>>>>>> 
>>config.subscribe("test-topic-1",
>> > > >> > > > "test-topic-2");
>> > > >> > > > > > >> > > > > >> > >>>>>>>
>> > > >> > > config.processor(ExampleStreamProcessor.class);
>> > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
>> > > >> > StringSerializer(),
>> > > >> > > > new
>> > > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
>> > KafkaStreaming
>> > > >> > > > container =
>> > > >> > > > > > new
>> > > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
>> > > container.run();
>> > > >> > > > > > >> > > > > >> > >>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>> -Jay
>> > > >> > > > > > >> > > > > >> > >>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 
>>PM,
>> > Jay
>> > > >> > Kreps <
>> > > >> > > > > > >> > > > jay@confluent.io
>> > > >> > > > > > >> > > > > >
>> > > >> > > > > > >> > > > > >> > >>>> wrote:
>> > > >> > > > > > >> > > > > >> > >>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some
>> conversations
>> > > >> Chris
>> > > >> > > and
>> > > >> > > > I
>> > > >> > > > > > were
>> > > >> > > > > > >> > > having
>> > > >> > > > > > >> > > > > >> > >>>>>>>> around
>> > > >> > > > > > >> > > > > >> > >>>>>>> whether
>> > > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use 
>>Samza
>> > as a
>> > > >> kind
>> > > >> > > of
>> > > >> > > > > data
>> > > >> > > > > > >> > > > ingestion
>> > > >> > > > > > >> > > > > >> > >>>>> framework
>> > > >> > > > > > >> > > > > >> > >>>>>>> for
>> > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead 
>>to
>> > > KIP-26
>> > > >> > > > > "copycat").
>> > > >> > > > > > >> This
>> > > >> > > > > > >> > > > kind
>> > > >> > > > > > >> > > > > of
>> > > >> > > > > > >> > > > > >> > >>>>>> combined
>> > > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config 
>>and
>> > > YARN
>> > > >> and
>> > > >> > > the
>> > > >> > > > > > >> > discussion
>> > > >> > > > > > >> > > > > >> around
>> > > >> > > > > > >> > > > > >> > >>>>>>>> how
>> > > >> > > > > > >> > > > > >> > >>>>> to
>> > > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was,
>> > given
>> > > >> that
>> > > >> > > > Samza
>> > > >> > > > > > was
>> > > >> > > > > > >> > > > basically
>> > > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka 
>>specific,
>> > what
>> > > if
>> > > >> > you
>> > > >> > > > just
>> > > >> > > > > > >> > embraced
>> > > >> > > > > > >> > > > > that
>> > > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
>> > > >> > > > > > >> > > > > >> > >>>>>> into
>> > > >> > > > > > >> > > > > >> > >>>>>>>> something less like a 
>>heavyweight
>> > > >> > framework
>> > > >> > > > and
>> > > >> > > > > > more
>> > > >> > > > > > >> > > like a
>> > > >> > > > > > >> > > > > >> > >>>>>>>> third
>> > > >> > > > > > >> > > > > >> > >>>>> Kafka
>> > > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
>> > > consumer"
>> > > >> > with
>> > > >> > > > > state
>> > > >> > > > > > >> > > > management
>> > > >> > > > > > >> > > > > >> > >>>>>> facilities.
>> > > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead 
>>of a
>> > > >> complex
>> > > >> > > > stream
>> > > >> > > > > > >> > > processing
>> > > >> > > > > > >> > > > > >> > >>>>>>>> framework
>> > > >> > > > > > >> > > > > >> > >>>>>>> this
>> > > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very 
>>simple
>> > > thing,
>> > > >> not
>> > > >> > > > much
>> > > >> > > > > > more
>> > > >> > > > > > >> > > > > >> complicated
>> > > >> > > > > > >> > > > > >> > >>>>>>>> to
>> > > >> > > > > > >> > > > > >> > >>>>> use
>> > > >> > > > > > >> > > > > >> > >>>>>>> or
>> > > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka 
>>consumer. As
>> > > Chris
>> > > >> > said
>> > > >> > > > we
>> > > >> > > > > > >> thought
>> > > >> > > > > > >> > > > about
>> > > >> > > > > > >> > > > > >> it
>> > > >> > > > > > >> > > > > >> > >>>>>>>> a
>> > > >> > > > > > >> > > > > >> > >>>>> lot
>> > > >> > > > > > >> > > > > >> > >>>>>> of
>> > > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other 
>>stream
>> > > >> > processing
>> > > >> > > > > > systems
>> > > >> > > > > > >> > were
>> > > >> > > > > > >> > > > > doing)
>> > > >> > > > > > >> > > > > >> > >>>>> seemed
>> > > >> > > > > > >> > > > > >> > >>>>>>> like
>> > > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from
>> MapReduce.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to
>> ingest/output
>> > > >> data
>> > > >> > to
>> > > >> > > > and
>> > > >> > > > > > from
>> > > >> > > > > > >> > the
>> > > >> > > > > > >> > > > > stream
>> > > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we 
>>actually
>> > > looked
>> > > >> > into
>> > > >> > > > how
>> > > >> > > > > > that
>> > > >> > > > > > >> > > would
>> > > >> > > > > > >> > > > > >> > >>>>>>>> work,
>> > > >> > > > > > >> > > > > >> > >>>>> Samza
>> > > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
>> > ingestion
>> > > >> > > framework
>> > > >> > > > > > for a
>> > > >> > > > > > >> > > bunch
>> > > >> > > > > > >> > > > of
>> > > >> > > > > > >> > > > > >> > >>>>> reasons.
>> > > >> > > > > > >> > > > > >> > >>>>>> To
>> > > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need 
>>a
>> > > pretty
>> > > >> > > > different
>> > > >> > > > > > >> > internal
>> > > >> > > > > > >> > > > > data
>> > > >> > > > > > >> > > > > >> > >>>>>>>> model
>> > > >> > > > > > >> > > > > >> > >>>>>> and
>> > > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you 
>>split
>> > > them
>> > > >> and
>> > > >> > > had
>> > > >> > > > > an
>> > > >> > > > > > api
>> > > >> > > > > > >> > for
>> > > >> > > > > > >> > > > > Kafka
>> > > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA
>> KIP-26)
>> > > >> and a
>> > > >> > > > > separate
>> > > >> > > > > > >> api
>> > > >> > > > > > >> > > for
>> > > >> > > > > > >> > > > > >> Kafka
>> > > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
>> > > embracing
>> > > >> the
>> > > >> > > > same
>> > > >> > > > > > >> > > terminology
>> > > >> > > > > > >> > > > > and
>> > > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint 
>>about
>> > the
>> > > >> > current
>> > > >> > > > > > state is
>> > > >> > > > > > >> > > that
>> > > >> > > > > > >> > > > > the
>> > > >> > > > > > >> > > > > >> > >>>>>>>> two
>> > > >> > > > > > >> > > > > >> > >>>>>>> systems
>> > > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on.
>> Terminology
>> > > >> like
>> > > >> > > > > "stream"
>> > > >> > > > > > vs
>> > > >> > > > > > >> > > > "topic"
>> > > >> > > > > > >> > > > > >> and
>> > > >> > > > > > >> > > > > >> > >>>>>>> different
>> > > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems
>> means
>> > > you
>> > > >> > kind
>> > > >> > > > of
>> > > >> > > > > > have
>> > > >> > > > > > >> to
>> > > >> > > > > > >> > > > learn
>> > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
>> > > >> > > > > > >> > > > > >> > >>>>>>> way,
>> > > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
>> > different
>> > > >> way,
>> > > >> > > > then
>> > > >> > > > > > kind
>> > > >> > > > > > >> of
>> > > >> > > > > > >> > > > > >> > >>>>>>>> understand
>> > > >> > > > > > >> > > > > >> > >>>>> how
>> > > >> > > > > > >> > > > > >> > >>>>>>> they
>> > > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which 
>>having
>> > > walked
>> > > >> a
>> > > >> > few
>> > > >> > > > > > people
>> > > >> > > > > > >> > > through
>> > > >> > > > > > >> > > > > >> this
>> > > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for 
>>folks
>> to
>> > > >> get.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a 
>>lot
>> of
>> > > >> time
>> > > >> > on
>> > > >> > > > > > >> airplanes I
>> > > >> > > > > > >> > > > > hacked
>> > > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still 
>>somewhat
>> > > >> incomplete
>> > > >> > > > > > prototype
>> > > >> > > > > > >> of
>> > > >> > > > > > >> > > > what
>> > > >> > > > > > >> > > > > >> > >>>>>>>> this would
>> > > >> > > > > > >> > > > > >> > >>>>> look
>> > > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just
>> unceremoniously
>> > > >> dumped
>> > > >> > > into
>> > > >> > > > > > Kafka
>> > > >> > > > > > >> as
>> > > >> > > > > > >> > > it
>> > > >> > > > > > >> > > > > >> > >>>>>>>> required a
>> > > >> > > > > > >> > > > > >> > >>>>>> few
>> > > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. 
>>Here
>> > is
>> > > >> the
>> > > >> > > code:
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > >
>> > > >> > > > > > >> >
>> > > >> > > > > >
>> > > >> > >
>> > > >>
>> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> > > >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the 
>>prototype
>> I
>> > > just
>> > > >> > > > > liberally
>> > > >> > > > > > >> > renamed
>> > > >> > > > > > >> > > > > >> > >>>>>>>> everything
>> > > >> > > > > > >> > > > > >> > >>>>> to
>> > > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka 
>>with
>> no
>> > > >> regard
>> > > >> > > for
>> > > >> > > > > > >> > > > compatibility.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something
>> like
>> > > >> this:
>> > > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
>> > Properties();
>> > > >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
>> > > >> > > > > "localhost:4242");
>> > > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
>> > > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
>> > > >> > > > > > >> > > > > >> > >>>>>>> 
>>config.subscribe("test-topic-1",
>> > > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
>> > > >> > > > > > >> > > > > >>
>> > config.processor(ExampleStreamProcessor.class);
>> > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
>> > > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
>> > > >> > > StringDeserializer());
>> > > >> > > > > > >> > > > KafkaStreaming
>> > > >> > > > > > >> > > > > >> > >>>>>> container =
>> > > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
>> > > >> > container.run();
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically 
>>the
>> > > >> > > > SamzaContainer;
>> > > >> > > > > > >> > > > > StreamProcessor
>> > > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the
>> > class
>> > > >> names
>> > > >> > > in
>> > > >> > > > a
>> > > >> > > > > > file
>> > > >> > > > > > >> > and
>> > > >> > > > > > >> > > > then
>> > > >> > > > > > >> > > > > >> > >>>>>>>> having
>> > > >> > > > > > >> > > > > >> > >>>>>> the
>> > > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, 
>>you
>> > just
>> > > >> > > > > instantiate
>> > > >> > > > > > the
>> > > >> > > > > > >> > > > > container
>> > > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is
>> balanced
>> > > over
>> > > >> > > > however
>> > > >> > > > > > many
>> > > >> > > > > > >> > > > > instances
>> > > >> > > > > > >> > > > > >> > >>>>>>>> of
>> > > >> > > > > > >> > > > > >> > >>>>> this
>> > > >> > > > > > >> > > > > >> > >>>>>>> are
>> > > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
>> > > instance
>> > > >> > dies,
>> > > >> > > > new
>> > > >> > > > > > >> tasks
>> > > >> > > > > > >> > > are
>> > > >> > > > > > >> > > > > >> added
>> > > >> > > > > > >> > > > > >> > >>>>>>>> to
>> > > >> > > > > > >> > > > > >> > >>>>> the
>> > > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
>> > shutting
>> > > >> them
>> > > >> > > > down).
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
>> > > running
>> > > >> > this
>> > > >> > > > > stuff
>> > > >> > > > > > in
>> > > >> > > > > > >> > YARN
>> > > >> > > > > > >> > > > via
>> > > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, 
>>and
>> AWS
>> > > >> using
>> > > >> > > some
>> > > >> > > > > of
>> > > >> > > > > > >> their
>> > > >> > > > > > >> > > > tools
>> > > >> > > > > > >> > > > > >> > >>>>>>>> but from the
>> > > >> > > > > > >> > > > > >> > >>>>>> point
>> > > >> > > > > > >> > > > > >> > >>>>>>> of
>> > > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these
>> > stream
>> > > >> > > > processing
>> > > >> > > > > > jobs
>> > > >> > > > > > >> > are
>> > > >> > > > > > >> > > > > just
>> > > >> > > > > > >> > > > > >> > >>>>>> stateless
>> > > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go 
>>and
>> > > >> expand
>> > > >> > and
>> > > >> > > > > > contract
>> > > >> > > > > > >> > at
>> > > >> > > > > > >> > > > > will.
>> > > >> > > > > > >> > > > > >> > >>>>>>>> There
>> > > >> > > > > > >> > > > > >> > >>>>> is
>> > > >> > > > > > >> > > > > >> > >>>>>>> no
>> > > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant 
>>details:
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of
>> code,
>> > > it
>> > > >> > would
>> > > >> > > > get
>> > > >> > > > > > >> larger
>> > > >> > > > > > >> > > if
>> > > >> > > > > > >> > > > we
>> > > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
>> > > larger.
>> > > >> We
>> > > >> > > > really
>> > > >> > > > > > do
>> > > >> > > > > > >> > get a
>> > > >> > > > > > >> > > > ton
>> > > >> > > > > > >> > > > > >> > >>>>>>>> of
>> > > >> > > > > > >> > > > > >> > >>>>>>> leverage
>> > > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is 
>>fully
>> > > >> > delegated
>> > > >> > > to
>> > > >> > > > > the
>> > > >> > > > > > >> new
>> > > >> > > > > > >> > > > > >> consumer.
>> > > >> > > > > > >> > > > > >> > >>>>> This
>> > > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any 
>>partition
>> > > >> > management
>> > > >> > > > > > strategy
>> > > >> > > > > > >> > > > > available
>> > > >> > > > > > >> > > > > >> > >>>>>>>> to
>> > > >> > > > > > >> > > > > >> > >>>>>> Kafka
>> > > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to
>> > Samza
>> > > >> (and
>> > > >> > > vice
>> > > >> > > > > > versa)
>> > > >> > > > > > >> > and
>> > > >> > > > > > >> > > > > with
>> > > >> > > > > > >> > > > > >> > >>>>>>>> the
>> > > >> > > > > > >> > > > > >> > >>>>>>> exact
>> > > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well 
>>as
>> > > state
>> > > >> > reuse
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully 
>>it
>> is
>> > > >> > thought
>> > > >> > > > > > >> provoking.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 
>>PM,
>> > > Chris
>> > > >> > > > > Riccomini <
>> > > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
>> > > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions 
>>with
>> > > Samza
>> > > >> > > > > engineers
>> > > >> > > > > > at
>> > > >> > > > > > >> > > > LinkedIn
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
>> > > >> > > > > > >> > > > > >> > >>>>>>> Confluent
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
>> > > observations
>> > > >> > and
>> > > >> > > > > would
>> > > >> > > > > > >> like
>> > > >> > > > > > >> > to
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> propose
>> > > >> > > > > > >> > > > > >> > >>>>> some
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things 
>>that
>> I
>> > > >> want to
>> > > >> > > > call
>> > > >> > > > > > out
>> > > >> > > > > > >> > about
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
>> > > >> > > > > > >> > > > > >> > >>>>>> design,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
>> > > changes.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a
>> > dynamic
>> > > >> > > > deployment
>> > > >> > > > > > >> system.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
>> > > >> SystemConsumer/SystemProducer
>> > > >> > and
>> > > >> > > > > > Kafka's
>> > > >> > > > > > >> > > > consumer
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
>> > > >> > > > > > >> > > > > >> > >>>>> are
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the
>> same
>> > > >> > problems.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
>> > > related,
>> > > >> > but
>> > > >> > > > I'll
>> > > >> > > > > > >> > address
>> > > >> > > > > > >> > > > them
>> > > >> > > > > > >> > > > > >> in
>> > > >> > > > > > >> > > > > >> > >>>>> order.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the
>> use
>> > > of a
>> > > >> > > > dynamic
>> > > >> > > > > > >> > > deployment
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
>> > > >> > > > > > >> > > > > >> > >>>>>> such
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> as
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
>> > initially
>> > > >> built
>> > > >> > > > > Samza,
>> > > >> > > > > > we
>> > > >> > > > > > >> > bet
>> > > >> > > > > > >> > > > that
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> there
>> > > >> > > > > > >> > > > > >> > >>>>>> would
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> be
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this 
>>area,
>> > and
>> > > >> we
>> > > >> > > could
>> > > >> > > > > > >> support
>> > > >> > > > > > >> > > > them,
>> > > >> > > > > > >> > > > > >> and
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > >> > > > > > >> > > > > >> > >>>>>> rest
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, 
>>there
>> > are
>> > > >> many
>> > > >> > > > > > >> variations.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
>> > > >> > > > > > >> > > > > >> > >>>>>> many
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just
>> start
>> > > >> their
>> > > >> > > > > > processors
>> > > >> > > > > > >> > like
>> > > >> > > > > > >> > > > > normal
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
>> > traditional
>> > > >> > > > deployment
>> > > >> > > > > > >> scripts
>> > > >> > > > > > >> > > > such
>> > > >> > > > > > >> > > > > as
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
>> > > >> > > > > > >> > > > > >> > >>>>>> Chef,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a
>> deployment
>> > > >> system
>> > > >> > > on
>> > > >> > > > > > users
>> > > >> > > > > > >> > makes
>> > > >> > > > > > >> > > > the
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
>> > > painful
>> > > >> for
>> > > >> > > > first
>> > > >> > > > > > time
>> > > >> > > > > > >> > > > users.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
>> > requirement
>> > > >> was
>> > > >> > > also
>> > > >> > > > a
>> > > >> > > > > > bit
>> > > >> > > > > > >> of
>> > > >> > > > > > >> > a
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
>> > > >> > > > > > >> > > > > >> > >>>>>> because
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> of
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental 
>>misunderstanding
>> > > between
>> > > >> > the
>> > > >> > > > > > nature of
>> > > >> > > > > > >> > > batch
>> > > >> > > > > > >> > > > > >> jobs
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
>> > > >> > > > > > >> > > > > >> > >>>>>>> stream
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we
>> made
>> > > >> > > conscious
>> > > >> > > > > > effort
>> > > >> > > > > > >> to
>> > > >> > > > > > >> > > > favor
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > >> > > > > > >> > > > > >> > >>>>>> Hadoop
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing
>> things,
>> > > >> since
>> > > >> > it
>> > > >> > > > > worked
>> > > >> > > > > > >> and
>> > > >> > > > > > >> > > was
>> > > >> > > > > > >> > > > > well
>> > > >> > > > > > >> > > > > >> > >>>>>>> understood.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was
>> that
>> > > >> batch
>> > > >> > > jobs
>> > > >> > > > > > have a
>> > > >> > > > > > >> > > > definite
>> > > >> > > > > > >> > > > > >> > >>>>>> beginning,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing 
>>jobs
>> > > don't
>> > > >> > > > > (usually).
>> > > >> > > > > > >> This
>> > > >> > > > > > >> > > > leads
>> > > >> > > > > > >> > > > > to
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> a
>> > > >> > > > > > >> > > > > >> > >>>>> much
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem 
>>for
>> > > stream
>> > > >> > > > > processors.
>> > > >> > > > > > >> You
>> > > >> > > > > > >> > > > > >> basically
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> just
>> > > >> > > > > > >> > > > > >> > >>>>>>> need
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
>> > > >> processor,
>> > > >> > and
>> > > >> > > > > start
>> > > >> > > > > > >> it.
>> > > >> > > > > > >> > > The
>> > > >> > > > > > >> > > > > way
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn,
>> there's
>> > > no
>> > > >> > > concept
>> > > >> > > > > of
>> > > >> > > > > > a
>> > > >> > > > > > >> > > cluster
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
>> > > >> > > > > > >> > > > > >> > >>>>>> add
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem 
>>with
>> > > >> coupling
>> > > >> > > > Samza
>> > > >> > > > > > with
>> > > >> > > > > > >> a
>> > > >> > > > > > >> > > > > >> scheduler
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> is
>> > > >> > > > > > >> > > > > >> > >>>>>> that
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now 
>>has
>> to
>> > > >> handle
>> > > >> > > > > > deployment.
>> > > >> > > > > > >> > > This
>> > > >> > > > > > >> > > > > >> pulls
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> in a
>> > > >> > > > > > >> > > > > >> > >>>>>>> bunch
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as 
>>configuration
>> > > >> > > distribution
>> > > >> > > > > > (config
>> > > >> > > > > > >> > > > > stream),
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> shell
>> > > >> > > > > > >> > > > > >> > >>>>>>> scrips
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
>> > > packaging
>> > > >> > (all
>> > > >> > > > the
>> > > >> > > > > > .tgz
>> > > >> > > > > > >> > > > stuff),
>> > > >> > > > > > >> > > > > >> etc.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring
>> > dynamic
>> > > >> > > > deployment
>> > > >> > > > > > was
>> > > >> > > > > > >> to
>> > > >> > > > > > >> > > > > support
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to
>> have
>> > > >> > > locality,
>> > > >> > > > > you
>> > > >> > > > > > >> need
>> > > >> > > > > > >> > to
>> > > >> > > > > > >> > > > put
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> your
>> > > >> > > > > > >> > > > > >> > >>>>>> processors
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
>> > > processing.
>> > > >> > Upon
>> > > >> > > > > > further
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
>> > > >> > > > > > >> > > > > >> > >>>>>>> though,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
>> > beneficial.
>> > > >> > There
>> > > >> > > is
>> > > >> > > > > > some
>> > > >> > > > > > >> > good
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
>> > > >> > > > > > >> > > > > >> > >>>>>> about
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
>> > SAMZA-335.
>> > > >> > Again,
>> > > >> > > we
>> > > >> > > > > > took
>> > > >> > > > > > >> the
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
>> > > >> > > > > > >> > > > > >> > >>>>>> path,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> but
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
>> > > differences
>> > > >> > > > between
>> > > >> > > > > > HDFS
>> > > >> > > > > > >> > and
>> > > >> > > > > > >> > > > > Kafka.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
>> > > >> > > > > > >> > > > > >> > >>>>>> has
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has
>> > partitions.
>> > > >> This
>> > > >> > > > leads
>> > > >> > > > > to
>> > > >> > > > > > >> less
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with
>> stream
>> > > >> > > processors
>> > > >> > > > > on
>> > > >> > > > > > top
>> > > >> > > > > > >> > of
>> > > >> > > > > > >> > > > > Kafka.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as 
>>a
>> > > crutch.
>> > > >> > > Samza
>> > > >> > > > > > doesn't
>> > > >> > > > > > >> > > have
>> > > >> > > > > > >> > > > > any
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> built
>> > > >> > > > > > >> > > > > >> > >>>>> in
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. 
>>Instead,
>> it
>> > > >> > depends
>> > > >> > > on
>> > > >> > > > > the
>> > > >> > > > > > >> > > dynamic
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system 
>>to
>> > > handle
>> > > >> > > > restarts
>> > > >> > > > > > >> when a
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
>> > > >> > > > > > >> > > > > >> > >>>>>>> made
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
>> > > >> standalone
>> > > >> > > Samza
>> > > >> > > > > > >> > container
>> > > >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is
>> > good,
>> > > >> but I
>> > > >> > > > think
>> > > >> > > > > > that
>> > > >> > > > > > >> > > we've
>> > > >> > > > > > >> > > > > >> gone
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> too
>> > > >> > > > > > >> > > > > >> > >>>>>> far
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza 
>>has:
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment 
>>systems.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
>> > > >> > > > (SystemConsumer,
>> > > >> > > > > > >> > > > > SystemProducer,
>> > > >> > > > > > >> > > > > >> > >>>> etc).
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for 
>>just
>> > > about
>> > > >> > every
>> > > >> > > > > > >> component
>> > > >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
>> > > >> > > ConfigRewriter,
>> > > >> > > > > > etc).
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that 
>>I've
>> > > >> > forgotten,
>> > > >> > > as
>> > > >> > > > > > well.
>> > > >> > > > > > >> > Some
>> > > >> > > > > > >> > > > of
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> these
>> > > >> > > > > > >> > > > > >> > >>>>> are
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven 
>>not
>> > to
>> > > >> be.
>> > > >> > > This
>> > > >> > > > > all
>> > > >> > > > > > >> comes
>> > > >> > > > > > >> > > at
>> > > >> > > > > > >> > > > a
>> > > >> > > > > > >> > > > > >> cost:
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity 
>>is
>> > > making
>> > > >> it
>> > > >> > > > harder
>> > > >> > > > > > for
>> > > >> > > > > > >> > our
>> > > >> > > > > > >> > > > > users
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> to
>> > > >> > > > > > >> > > > > >> > >>>>> pick
>> > > >> > > > > > >> > > > > >> > >>>>>> up
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the 
>>box. It
>> > > also
>> > > >> > makes
>> > > >> > > > it
>> > > >> > > > > > >> > difficult
>> > > >> > > > > > >> > > > for
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason 
>>about
>> > > what
>> > > >> the
>> > > >> > > > > > >> > > characteristics
>> > > >> > > > > > >> > > > of
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
>> > > >> characteristics
>> > > >> > > > change
>> > > >> > > > > > >> > > depending
>> > > >> > > > > > >> > > > on
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability 
>>are
>> > > most
>> > > >> > > visible
>> > > >> > > > > in
>> > > >> > > > > > the
>> > > >> > > > > > >> > > > System
>> > > >> > > > > > >> > > > > >> APIs.
>> > > >> > > > > > >> > > > > >> > >>>>> What
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
>> > > >> functional is
>> > > >> > > > Kafka
>> > > >> > > > > > as
>> > > >> > > > > > >> its
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> transport
>> > > >> > > > > > >> > > > > >> > >>>>>> layer.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> But
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated
>> use
>> > > >> cases
>> > > >> > > into
>> > > >> > > > > one
>> > > >> > > > > > >> API:
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of 
>>Kafka.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> The current System API 
>>supports
>> > both
>> > > >> of
>> > > >> > > these
>> > > >> > > > > use
>> > > >> > > > > > >> > cases.
>> > > >> > > > > > >> > > > The
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> problem
>> > > >> > > > > > >> > > > > >> > >>>>>> is,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> we
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> actually want different 
>>features
>> > for
>> > > >> each
>> > > >> > > use
>> > > >> > > > > > case.
>> > > >> > > > > > >> By
>> > > >> > > > > > >> > > > > >> papering
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> over
>> > > >> > > > > > >> > > > > >> > >>>>>>> these
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing 
>>a
>> > > single
>> > > >> > API,
>> > > >> > > > > we've
>> > > >> > > > > > >> > > > introduced
>> > > >> > > > > > >> > > > > a
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
>> > > >> > > > > > >> > > > > >> > >>>>>>> leaky
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really
>> like
>> > > in
>> > > >> (2)
>> > > >> > > is
>> > > >> > > > to
>> > > >> > > > > > have
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing 
>>longs
>> for
>> > > >> > offsets
>> > > >> > > > > (like
>> > > >> > > > > > >> > Kafka).
>> > > >> > > > > > >> > > > > This
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
>> > > >> > > > > > >> > > > > >> > >>>>> with
>> > > >> > > > > > >> > > > > >> > >>>>>>> (1),
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> though, since different 
>>systems
>> > have
>> > > >> > > > different
>> > > >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on 
>>the
>> > > >> mailing
>> > > >> > > list
>> > > >> > > > > and
>> > > >> > > > > > >> the
>> > > >> > > > > > >> > > SQL
>> > > >> > > > > > >> > > > > >> JIRAs
>> > > >> > > > > > >> > > > > >> > >>>>> about
>> > > >> > > > > > >> > > > > >> > >>>>>>> the
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
>> > > >> > > replayability.
>> > > >> > > > > > Kafka
>> > > >> > > > > > >> > > allows
>> > > >> > > > > > >> > > > us
>> > > >> > > > > > >> > > > > >> to
>> > > >> > > > > > >> > > > > >> > >>>>> rewind
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> when
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other
>> > > systems
>> > > >> > > don't.
>> > > >> > > > In
>> > > >> > > > > > some
>> > > >> > > > > > >> > > > cases,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
>> > > >> > > > > > >> > > > > >> > >>>>>>> return
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
>> > > >> > > > > > >> WikipediaSystemConsumer)
>> > > >> > > > > > >> > > > > because
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> they
>> > > >> > > > > > >> > > > > >> > >>>>>> have
>> > > >> > > > > > >> > > > > >> > >>>>>>> no
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another 
>>example.
>> > > Kafka
>> > > >> > > > supports
>> > > >> > > > > > >> > > > > partitioning,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> but
>> > > >> > > > > > >> > > > > >> > >>>>> many
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this 
>>by
>> > > >> having a
>> > > >> > > > single
>> > > >> > > > > > >> > > partition
>> > > >> > > > > > >> > > > > for
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other
>> > systems
>> > > >> model
>> > > >> > > > > > >> partitioning
>> > > >> > > > > > >> > > > > >> > >>>> differently (e.g.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is
>> also
>> > a
>> > > >> mess.
>> > > >> > > > > > Creating
>> > > >> > > > > > >> > > streams
>> > > >> > > > > > >> > > > > in
>> > > >> > > > > > >> > > > > >> a
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
>> > > >> impossible.
>> > > >> > > As
>> > > >> > > > is
>> > > >> > > > > > >> > modeling
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> metadata
>> > > >> > > > > > >> > > > > >> > >>>>> for
>> > > >> > > > > > >> > > > > >> > >>>>>>> the
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
>> > > >> partitions,
>> > > >> > > > > location,
>> > > >> > > > > > >> > etc).
>> > > >> > > > > > >> > > > The
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> list
>> > > >> > > > > > >> > > > > >> > >>>>> goes
>> > > >> > > > > > >> > > > > >> > >>>>>>> on.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began
>> writing
>> > > >> Samza,
>> > > >> > > > > Kafka's
>> > > >> > > > > > >> > > consumer
>> > > >> > > > > > >> > > > > and
>> > > >> > > > > > >> > > > > >> > >>>>> producer
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature
>> set.
>> > > On
>> > > >> the
>> > > >> > > > > > >> > consumer-side,
>> > > >> > > > > > >> > > > you
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> had two
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
>> > > consumer,
>> > > >> or
>> > > >> > > the
>> > > >> > > > > > simple
>> > > >> > > > > > >> > > > > consumer.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> The
>> > > >> > > > > > >> > > > > >> > >>>>>>> problem
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer 
>>was
>> > > that
>> > > >> it
>> > > >> > > > > > controlled
>> > > >> > > > > > >> > your
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition 
>>assignments,
>> > and
>> > > >> the
>> > > >> > > order
>> > > >> > > > > in
>> > > >> > > > > > >> which
>> > > >> > > > > > >> > > you
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
>> > > >> > > > > > >> > > > > >> > >>>>> problem
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> with
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that 
>>it's
>> > not
>> > > >> > > simple.
>> > > >> > > > > It's
>> > > >> > > > > > >> > basic.
>> > > >> > > > > > >> > > > You
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> end up
>> > > >> > > > > > >> > > > > >> > >>>>>>> having
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really
>> > low-level
>> > > >> stuff
>> > > >> > > > that
>> > > >> > > > > > you
>> > > >> > > > > > >> > > > > shouldn't.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> We
>> > > >> > > > > > >> > > > > >> > >>>>>> spent a
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
>> > > >> > > > KafkaSystemConsumer
>> > > >> > > > > > very
>> > > >> > > > > > >> > > > robust.
>> > > >> > > > > > >> > > > > >> It
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support 
>>some
>> > cool
>> > > >> > > features:
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message 
>>ordering
>> > and
>> > > >> > > > > > prioritization.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over 
>>partition
>> > > >> assignment
>> > > >> > > to
>> > > >> > > > > > support
>> > > >> > > > > > >> > > > joins,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> global
>> > > >> > > > > > >> > > > > >> > >>>>>> state
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it 
>>:)),
>> > > etc.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
>> > > >> > checkpointing.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the
>> time
>> > > is
>> > > >> > that
>> > > >> > > > > these
>> > > >> > > > > > >> > > features
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> should
>> > > >> > > > > > >> > > > > >> > >>>>>>> actually
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
>> > > consumers
>> > > >> > (not
>> > > >> > > > just
>> > > >> > > > > > >> Samza
>> > > >> > > > > > >> > > > stream
>> > > >> > > > > > >> > > > > >> > >>>>>> processors)
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things 
>>like
>> > > joins
>> > > >> > and
>> > > >> > > > > > partition
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
>> > > >> > > > > > >> > > > > >> > >>>>>>> Kafka
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the 
>>same
>> > > >> > conclusion.
>> > > >> > > > > > They're
>> > > >> > > > > > >> > > adding
>> > > >> > > > > > >> > > > a
>> > > >> > > > > > >> > > > > >> ton
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new 
>>Kafka
>> > > >> consumer
>> > > >> > > > > > >> > > implementation.
>> > > >> > > > > > >> > > > > To a
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
>> > > >> > > > > > >> > > > > >> > >>>>> it's
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've
>> > already
>> > > >> done
>> > > >> > > in
>> > > >> > > > > > Samza.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended 
>>up
>> > > taking
>> > > >> a
>> > > >> > > very
>> > > >> > > > > > similar
>> > > >> > > > > > >> > > > > approach
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> to
>> > > >> > > > > > >> > > > > >> > >>>>>> Samza's
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
>> > > implementation
>> > > >> for
>> > > >> > > > > > handling
>> > > >> > > > > > >> > > offset
>> > > >> > > > > > >> > > > > >> > >>>>>> checkpointing.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new 
>>offset
>> > > >> management
>> > > >> > > > > feature
>> > > >> > > > > > >> > stores
>> > > >> > > > > > >> > > > > >> offset
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and
>> allows
>> > > >> you to
>> > > >> > > > fetch
>> > > >> > > > > > them
>> > > >> > > > > > >> > > from
>> > > >> > > > > > >> > > > > the
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> broker.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a
>> waste,
>> > > >> since
>> > > >> > we
>> > > >> > > > > could
>> > > >> > > > > > >> have
>> > > >> > > > > > >> > > > shared
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > >> > > > > > >> > > > > >> > >>>>> work
>> > > >> > > > > > >> > > > > >> > >>>>>> if
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> it
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from 
>>the
>> > > >> get-go.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Vision
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a 
>>rather
>> > > >> radical
>> > > >> > > > > > proposal.
>> > > >> > > > > > >> > Samza
>> > > >> > > > > > >> > > > is
>> > > >> > > > > > >> > > > > >> > >>>>> relatively
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd
>> venture
>> > to
>> > > >> say
>> > > >> > > that
>> > > >> > > > > > we're
>> > > >> > > > > > >> > > near a
>> > > >> > > > > > >> > > > > 1.0
>> > > >> > > > > > >> > > > > >> > >>>>>> release.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> I'd
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take
>> what
>> > > >> we've
>> > > >> > > > > learned,
>> > > >> > > > > > and
>> > > >> > > > > > >> > > begin
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> thinking
>> > > >> > > > > > >> > > > > >> > >>>>>>> about
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would 
>>we
>> > > >> change if
>> > > >> > > we
>> > > >> > > > > were
>> > > >> > > > > > >> > > starting
>> > > >> > > > > > >> > > > > >> from
>> > > >> > > > > > >> > > > > >> > >>>>>> scratch?
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> My
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the
>> > *only*
>> > > >> way
>> > > >> > to
>> > > >> > > > run
>> > > >> > > > > > Samza
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all
>> > direct
>> > > >> > > > > dependences
>> > > >> > > > > > on
>> > > >> > > > > > >> > > YARN,
>> > > >> > > > > > >> > > > > >> Mesos,
>> > > >> > > > > > >> > > > > >> > >>>> etc.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to
>> > support
>> > > >> only
>> > > >> > > > Kafka
>> > > >> > > > > > as
>> > > >> > > > > > >> the
>> > > >> > > > > > >> > > > > stream
>> > > >> > > > > > >> > > > > >> > >>>>>> processing
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> layer.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics,
>> > > logging,
>> > > >> > > > > > >> serialization,
>> > > >> > > > > > >> > > and
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> config
>> > > >> > > > > > >> > > > > >> > >>>>>>> systems,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's 
>>instead.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the 
>>issues
>> > > that
>> > > >> I
>> > > >> > > > > outlined
>> > > >> > > > > > >> > above.
>> > > >> > > > > > >> > > It
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> should
>> > > >> > > > > > >> > > > > >> > >>>>> also
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base
>> pretty
>> > > >> > > > dramatically.
>> > > >> > > > > > >> > > Supporting
>> > > >> > > > > > >> > > > > >> only
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will
>> allow
>> > > >> Samza
>> > > >> > to
>> > > >> > > be
>> > > >> > > > > > >> executed
>> > > >> > > > > > >> > > on
>> > > >> > > > > > >> > > > > YARN
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
>> > > >> > > > Marathon/Aurora),
>> > > >> > > > > or
>> > > >> > > > > > >> most
>> > > >> > > > > > >> > > > other
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> in-house
>> > > >> > > > > > >> > > > > >> > >>>>>>> deployment
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make 
>>life a
>> > lot
>> > > >> > easier
>> > > >> > > > for
>> > > >> > > > > > new
>> > > >> > > > > > >> > > users.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
>> > > >> > > > > > >> > > > > >> > >>>>>>> having
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial 
>>without
>> > > YARN.
>> > > >> > The
>> > > >> > > > drop
>> > > >> > > > > > in
>> > > >> > > > > > >> > > mailing
>> > > >> > > > > > >> > > > > >> list
>> > > >> > > > > > >> > > > > >> > >>>>>> traffic
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems 
>>long
>> > > >> overdue to
>> > > >> > > me.
>> > > >> > > > > The
>> > > >> > > > > > >> > > reality
>> > > >> > > > > > >> > > > > is,
>> > > >> > > > > > >> > > > > >> > >>>>> everyone
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> that
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza 
>>with
>> > > >> Kafka.
>> > > >> > We
>> > > >> > > > > > basically
>> > > >> > > > > > >> > > > require
>> > > >> > > > > > >> > > > > >> it
>> > > >> > > > > > >> > > > > >> > >>>>>> already
>> > > >> > > > > > >> > > > > >> > >>>>>>> in
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to 
>>work.
>> > > Those
>> > > >> > that
>> > > >> > > > are
>> > > >> > > > > > >> using
>> > > >> > > > > > >> > > > other
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
>> > > >> > > > > > >> > > > > >> > >>>>>> are
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest
>> into
>> > > >> Kafka
>> > > >> > > (1),
>> > > >> > > > > and
>> > > >> > > > > > >> then
>> > > >> > > > > > >> > > > they
>> > > >> > > > > > >> > > > > do
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There 
>>is
>> > > >> already
>> > > >> > > > > > discussion (
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > >
>> > > >> > > > > > >> >
>> > > >> > > > > >
>> > > >> > >
>> > > >>
>> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> > > >> > > > > > >> > > > > >> > >>>>> 767
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> )
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting 
>>into
>> > > Kafka
>> > > >> > > > extremely
>> > > >> > > > > > >> easy.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to 
>>couple
>> > with
>> > > >> > Kafka,
>> > > >> > > > we
>> > > >> > > > > > can
>> > > >> > > > > > >> > > > leverage
>> > > >> > > > > > >> > > > > a
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
>> > > >> > > > > > >> > > > > >> > >>>>>>> their
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have 
>>to
>> > > >> maintain
>> > > >> > > our
>> > > >> > > > > own
>> > > >> > > > > > >> > config,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
>> > > >> > > > > > >> > > > > >> > >>>>> etc.
>> > > >> > > > > > >> > > > > >> > >>>>>>> We
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same
>> libraries,
>> > > and
>> > > >> > make
>> > > >> > > > them
>> > > >> > > > > > >> > better.
>> > > >> > > > > > >> > > > This
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> will
>> > > >> > > > > > >> > > > > >> > >>>>> also
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> allow us to share the
>> > > >> consumer/producer
>> > > >> > > APIs,
>> > > >> > > > > and
>> > > >> > > > > > >> will
>> > > >> > > > > > >> > > let
>> > > >> > > > > > >> > > > > us
>> > > >> > > > > > >> > > > > >> > >>>>> leverage
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> their offset management and
>> > > partition
>> > > >> > > > > management,
>> > > >> > > > > > >> > rather
>> > > >> > > > > > >> > > > > than
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> having
>> > > >> > > > > > >> > > > > >> > >>>>>> our
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> own. All of the coordinator
>> stream
>> > > >> code
>> > > >> > > would
>> > > >> > > > > go
>> > > >> > > > > > >> away,
>> > > >> > > > > > >> > > as
>> > > >> > > > > > >> > > > > >> would
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> most
>> > > >> > > > > > >> > > > > >> > >>>>>> of
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd
>> probably
>> > > >> have
>> > > >> > to
>> > > >> > > > push
>> > > >> > > > > > some
>> > > >> > > > > > >> > > > > partition
>> > > >> > > > > > >> > > > > >> > >>>>>>> management
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> features into the Kafka 
>>broker,
>> > but
>> > > >> > they're
>> > > >> > > > > > already
>> > > >> > > > > > >> > > moving
>> > > >> > > > > > >> > > > > in
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> that direction with the new
>> > consumer
>> > > >> API.
>> > > >> > > The
>> > > >> > > > > > >> features
>> > > >> > > > > > >> > > we
>> > > >> > > > > > >> > > > > have
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> for
>> > > >> > > > > > >> > > > > >> > >>>>>> partition
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> assignment aren't unique to
>> Samza,
>> > > and
>> > > >> > seem
>> > > >> > > > > like
>> > > >> > > > > > >> they
>> > > >> > > > > > >> > > > should
>> > > >> > > > > > >> > > > > >> be
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> in
>> > > >> > > > > > >> > > > > >> > >>>>>> Kafka
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> anyway. There will always be
>> some
>> > > >> niche
>> > > >> > > > usages
>> > > >> > > > > > which
>> > > >> > > > > > >> > > will
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> require
>> > > >> > > > > > >> > > > > >> > >>>>>> extra
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> care and hence full control 
>>over
>> > > >> > partition
>> > > >> > > > > > >> assignments
>> > > >> > > > > > >> > > > much
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> like the
>> > > >> > > > > > >> > > > > >> > >>>>>>> Kafka
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> low level consumer api. These
>> > would
>> > > >> > > continue
>> > > >> > > > to
>> > > >> > > > > > be
>> > > >> > > > > > >> > > > > supported.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> These items will be good for 
>>the
>> > > Samza
>> > > >> > > > > community.
>> > > >> > > > > > >> > > They'll
>> > > >> > > > > > >> > > > > make
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza easier to use, and 
>>make it
>> > > >> easier
>> > > >> > for
>> > > >> > > > > > >> developers
>> > > >> > > > > > >> > > to
>> > > >> > > > > > >> > > > > add
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> new features.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly 
>>large
>> > > (and
>> > > >> > > > somewhat
>> > > >> > > > > > >> > backwards
>> > > >> > > > > > >> > > > > >> > >>>>> incompatible
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> change). If we choose to go 
>>this
>> > > >> route,
>> > > >> > > it's
>> > > >> > > > > > >> important
>> > > >> > > > > > >> > > > that
>> > > >> > > > > > >> > > > > we
>> > > >> > > > > > >> > > > > >> > >>>>> openly
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> communicate how we're going 
>>to
>> > > >> provide a
>> > > >> > > > > > migration
>> > > >> > > > > > >> > path
>> > > >> > > > > > >> > > > from
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > >> > > > > > >> > > > > >> > >>>>>>> existing
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we 
>>make
>> > > >> > > incompatible
>> > > >> > > > > > >> > changes).
>> > > >> > > > > > >> > > I
>> > > >> > > > > > >> > > > > >> think
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably 
>>need
>> > to
>> > > >> > > provide a
>> > > >> > > > > > >> wrapper
>> > > >> > > > > > >> > to
>> > > >> > > > > > >> > > > > allow
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> existing StreamTask
>> > implementations
>> > > to
>> > > >> > > > continue
>> > > >> > > > > > >> > running
>> > > >> > > > > > >> > > on
>> > > >> > > > > > >> > > > > the
>> > > >> > > > > > >> > > > > >> > >>>> new container.
>> > > >> > > > > > >> > > > > >> > >>>>>>> It's
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> also important that we openly
>> > > >> communicate
>> > > >> > > > about
>> > > >> > > > > > >> > timing,
>> > > >> > > > > > >> > > > and
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> stages
>> > > >> > > > > > >> > > > > >> > >>>>> of
>> > > >> > > > > > >> > > > > >> > >>>>>>> the
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> migration.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm
>> sure
>> > > you
>> > > >> > have
>> > > >> > > > > > opinions.
>> > > >> > > > > > >> > :)
>> > > >> > > > > > >> > > > > Please
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> send
>> > > >> > > > > > >> > > > > >> > >>>>>> your
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Cheers,
>> > > >> > > > > > >> > > > > >> > >>>>>>>>> Chris
>> > > >> > > > > > >> > > > > >> > >>>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>> --
>> > > >> > > > > > >> > > > > >> > >>>>>> -- Guozhang
>> > > >> > > > > > >> > > > > >> > >>>>>>
>> > > >> > > > > > >> > > > > >> > >>>>>
>> > > >> > > > > > >> > > > > >> > >>>>
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> > >>
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >> >
>> > > >> > > > > > >> > > > > >>
>> > > >> > > > > > >> > > > > >
>> > > >> > > > > > >> > > > > >
>> > > >> > > > > > >> > > > >
>> > > >> > > > > > >> > > >
>> > > >> > > > > > >> > >
>> > > >> > > > > > >> >
>> > > >> > > > > > >>
>> > > >> > > > > >
>> > > >> > > > >
>> > > >> > > >
>> > > >> > >
>> > > >> >
>> > > >>
>> > > >
>> > > >
>> > >
>> >
>>
>
>
>
>-- 
>Jordan Shaw
>Full Stack Software Engineer
>PubNub Inc
>1045 17th St
>San Francisco, CA 94107


Re: Thoughts and obesrvations on Samza

Posted by Jordan Shaw <jo...@pubnub.com>.
Hi Everyone,
I still don't completely understand why there's a need to make Samza
"part-of" or rely on Kafka as a transmission medium? Is it not correct to
say that given proper API's and proper implementation of those API's things
will work just fine? We've had really good results from using the current
API's adding some SystemProducer/SystemConsumer functionality. I wouldn't
mind if those API's were refined or even broken in order to make what the
community thinks is better but to completely abandon something that really
isn't broken I don't understand. I also don't understand placing a bet that
Kafka will be the defacto transmission media form now and forever.

As I outlined before switching to Kafka wasn't an obvious decision for us
and we had to spend significant resources to make that decision. Where as
the decision on Samza was much easier. Just to reiterate transmission
mediums like ZeroMQ do streaming just as well as Kafka, or better for some
use cases but do it in a very different way, with very different scaling
characteristics so why wouldn't Samza want to have the option to support
that? (which it already does)

So I agree with Jakob on the Samza 2.0 approach, breaking API's to make the
code better but not abandoning, and to reiterate my questions:
1) What is so broken with the current API's that can't be fixed by
refactoring in the current context of Samza?
2) Why should Kafka be the defacto transmission medium for Samza when
something like ZeroMQ would work just fine and support different
infrastructure requirements?

-Jordan

On Sun, Jul 12, 2015 at 8:57 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey guys,
>
> There seems to be some confusion in the last few emails: there is no plan
> whatsoever to remove YARN support. The change suggested was to move the
> partition management out of the YARN app master and rely on Kafka's
> partition management. The advantage of this would be to make the vast
> majority of Samza totally cluster manager agnostic, and make it possible to
> implement a high quality "stand-alone" mode and support other frameworks.
> It is possible this will let us run in YARN without *any* samza-specific
> code using something generic like Slider, or maybe not. I don't think
> anyone has tried this so it is probably premature to say how much
> YARN-specific code could be killed. If that works out, then a bunch of the
> YARN-specific code will disappear, but this doesn't mean YARN support will
> disappear, just that we would retain less code to implement the same
> thing.  Either way Samza in YARN isn't going away.
>
> -Jay
>
> On Sun, Jul 12, 2015 at 7:48 PM, Garrett Barton <ga...@gmail.com>
> wrote:
>
> > Yi,
> >
> >  What you just summarized makes a whole lot more sense to me.
> Shamelessly
> > I am looking at this shift as a customer with a production workflow
> riding
> > on it so I am looking for some kind of consistency into the future of
> > Samza.  This makes me feel a lot better about it.
> >
> >
> > Thank you!
> >
> > On Sun, Jul 12, 2015 at 10:44 PM, Yi Pan <ni...@gmail.com> wrote:
> >
> > > Just to make it explicitly clear what I am proposing, here is a version
> > of
> > > more detailed description:
> > >
> > > The fourth option (in addition to what Jakob summarized) we are
> proposing
> > > is:
> > >
> > > - Recharter Samza to “stream processing as a service”
> > >
> > > - The current Samza core (the basic transformation API w/ basic
> partition
> > > and offset management build-in) will be moved to Kafka Streams (i.e.
> part
> > > of Kafka) and supports “run-as-a-library”
> > >
> > > - Deprecate the SystemConsumers and SystemProducers APIs and move them
> to
> > > Copycat
> > >
> > > - The current SQL development:
> > >
> > >    * physical operators and a Trident-like stream API should stay in
> > Kafka
> > > Streams as libraries, enabling any standalone deployment to use the
> core
> > > window/join functions
> > >
> > >    * query parser/planner and execution on top of a distributed service
> > > should stay in new Samza (i.e. “stream processing as a service”)
> > >
> > > - Advanced features related to job scheduling/state management stays in
> > new
> > > Samza (i.e. “streaming processing as a service”)
> > >
> > >    * Any advanced PartitionManager implementation that can be plugged
> > into
> > > Kafka Streams
> > >
> > >    * Any auto-scaling, dynamic configuration via coordinator stream
> > >
> > >    * Any advanced state management s.t. host-affinity etc.
> > >
> > >
> > > Pros:
> > >
> > > - W/ the current Samza core as Kafka Streams and move the ingestion to
> > > Copycat, we achieved most of the goals in the initial proposal:
> > >
> > >    * Tighter coupling w/ Kafka
> > >
> > >    * Reuse Kafka’s build-in functionalities, such as offset manager,
> > basic
> > > partition distribution
> > >
> > >    * Separation of ingestion vs transformation APIs
> > >
> > >    * offload a lot of system-specific configuration to Kafka Streams
> and
> > > Copycat (i.e. SystemFactory configure, serde configure, etc.)
> > >
> > >    * remove YARN dependency and make standalone deployment easy. As
> > > Guozhang mentioned, it would be really easy to start a process that
> > > internally run Kafka Streams as library.
> > >
> > > - By re-chartering Samza as “stream processing as a service”, we
> address
> > > the concern regarding to
> > >
> > >    * Pluggable partition management
> > >
> > >    * Running in a distributed cluster to manage process lifecycle,
> > > fault-tolerance, resource-allocation, etc.
> > >
> > >    * More advanced features s.t. host-affinity, auto-scaling, and
> dynamic
> > > configure changes, etc.
> > >
> > >
> > > Regarding to the code and community organization, I think the following
> > may
> > > be the best:
> > >
> > > Code:
> > >
> > > - A Kafka sub-project Kafka Streams to hold samza-core, samza-kv-store,
> > and
> > > the physical operator layer as library in SQL: this would allow better
> > > alignment w/ Kafka, in code, doc, and branding
> > >
> > > - Retain the current Samza project just to keep
> > >
> > >    * A pluggable explicit partition management in Kafka Streams client
> > >
> > >    * Integration w/ cluster-management systems for advanced features:
> > >
> > >       * host-affinity, auto-scaling,, dynamic configuration, etc.
> > >
> > >    * It will fully depend on the Kafka Streams API and remove all
> support
> > > for SystemConsumers/SystemProducers in the future
> > >
> > > Community: (this is almost the same as what Chris proposed)
> > >
> > > - Kafka Streams: the current Samza community should be supporting this
> > > effort together with some Kafka members, since most of the code here
> will
> > > be from samza-core, samza-kv-store, and samza-sql.
> > >
> > > - new Samza: the current Samza community should continue serve the
> course
> > > to support more advanced features to run Kafka Streams as a service.
> > > Arguably, the new Samza framework may be used to run Copycat workers as
> > > well, at least to manage Copycat worker’s lifecycle in a clustered
> > > environment. Hence, it would stay as a general stream processing
> > framework
> > > that takes in any source and output to any destination, just the
> > transport
> > > system is fixed to Kafka.
> > >
> > > On Sun, Jul 12, 2015 at 7:29 PM, Yi Pan <ni...@gmail.com> wrote:
> > >
> > > > Hi, Chris,
> > > >
> > > > Thanks for sending out this concrete set of points here. I agree w/
> all
> > > > but have a slight different point view on 8).
> > > >
> > > > My view on this is: instead of sunset Samza as TLP, can we re-charter
> > the
> > > > scope of Samza to be the home for "running streaming process as a
> > > service"?
> > > >
> > > > My main motivation is from the following points from a long internal
> > > > discussion in LinkedIn:
> > > >
> > > > - There is a clear ask for pluggable partition management, like we do
> > in
> > > > LinkedIn, and as Ben Kirwin has mentioned in
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCACuX-D-yJX++2GnF_1Laf10KYUVyamg7UP_Dt19v0ZNmmhbCPQ@mail.gmail.com%3E
> > > > - There are concerns on lack of support for running stream processing
> > in
> > > a
> > > > cluster: lifecycle management, resource allocation, fault tolerance,
> > etc.
> > > > - There is a question to how to support more advanced features s.t.
> > > > host-affinity, auto-scaling, and dynamic configuration in Samza jobs,
> > as
> > > > raised by Martin here:
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3C0D66EFD0-B7CD-4E4E-8B2F-2716167C313C@kleppmann.com%3E
> > > >
> > > > We have use cases that need to address all the above three cases and
> > most
> > > > of the functions are all in the current Samza project, in some
> flavor.
> > We
> > > > are all supporting to merge the samza-core functionalities into Kafka
> > > > Streams, but there is a question where we keep these functions in the
> > > > future. One option is to start a new project that includes these
> > > functions
> > > > that are closely related w/ "run stream-processing as-a-service",
> while
> > > > another personally more attractive option is to re-charter Samza
> > project
> > > > just do "run stream processing as-a-service". We can avoid the
> overhead
> > > of
> > > > re-starting another community for this project. Personally, I felt
> that
> > > > here are the benefits we should be getting:
> > > >
> > > > 1. We have already agreed mostly that Kafka Streams API would allow
> > some
> > > > pluggable partition management functions. Hence, the advanced
> partition
> > > > management can live out-side the new Kafka Streams core w/o affecting
> > the
> > > > run-as-a-library model in Kafka Streams.
> > > > 2. The integration w/ cluster management system and advanced features
> > > > listed above stays in the same project and allow existing users enjoy
> > > > no-impact migration to Kafka Stream as the core. That also addresses
> > > Tim's
> > > > question on "removing the support for YARN".
> > > > 3. A separate project for stream-processing-as-a-service also allow
> the
> > > > new Kafka Streams being independent to any cluster management and
> just
> > > > focusing on stream process core functions, while leaving the
> functions
> > > that
> > > > requires cluster-resource and state management to a separate layer.
> > > >
> > > > Please feel free to comment. Thanks!
> > > >
> > > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
> > criccomini@apache.org>
> > > > wrote:
> > > >
> > > >> Hey all,
> > > >>
> > > >> I want to start by saying that I'm absolutely thrilled to be a part
> of
> > > >> this
> > > >> community. The amount of level-headed, thoughtful, educated
> discussion
> > > >> that's gone on over the past ~10 days is overwhelming. Wonderful.
> > > >>
> > > >> It seems like discussion is waning a bit, and we've reached some
> > > >> conclusions. There are several key emails in this threat, which I
> want
> > > to
> > > >> call out:
> > > >>
> > > >> 1. Jakob's summary of the three potential ways forward.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > > >> 2. Julian's call out that we should be focusing on community over
> > code.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > > >> 3. Martin's summary about the benefits of merging communities.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > > >> 4. Jakob's comments about the distinction between community and code
> > > >> paths.
> > > >>
> > > >>
> > > >>
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> > > >>
> > > >> I agree with the comments on all of these emails. I think Martin's
> > > summary
> > > >> of his position aligns very closely with my own. To that end, I
> think
> > we
> > > >> should get concrete about what the proposal is, and call a vote on
> it.
> > > >> Given that Jay, Martin, and I seem to be aligning fairly closely, I
> > > think
> > > >> we should start with:
> > > >>
> > > >> 1. [community] Make Samza a subproject of Kafka.
> > > >> 2. [community] Make all Samza PMC/committers committers of the
> > > subproject.
> > > >> 3. [community] Migrate Samza's website/documentation into Kafka's.
> > > >> 4. [code] Have the Samza community and the Kafka community start a
> > > >> from-scratch reboot together in the new Kafka subproject. We can
> > > >> borrow/copy &  paste significant chunks of code from Samza's code
> > base.
> > > >> 5. [code] The subproject would intentionally eliminate support for
> > both
> > > >> other streaming systems and all deployment systems.
> > > >> 6. [code] Attempt to provide a bridge from our SystemConsumer to
> > KIP-26
> > > >> (copy cat)
> > > >> 7. [code] Attempt to provide a bridge from the new subproject's
> > > processor
> > > >> interface to our legacy StreamTask interface.
> > > >> 8. [code/community] Sunset Samza as a TLP when we have a working
> Kafka
> > > >> subproject that has a fault-tolerant container with state
> management.
> > > >>
> > > >> It's likely that (6) and (7) won't be fully drop-in. Still, the
> closer
> > > we
> > > >> can get, the better it's going to be for our existing community.
> > > >>
> > > >> One thing that I didn't touch on with (2) is whether any Samza PMC
> > > members
> > > >> should be rolled into Kafka PMC membership as well (though, Jay and
> > > Jakob
> > > >> are already PMC members on both). I think that Samza's community
> > > deserves
> > > >> a
> > > >> voice on the PMC, so I'd propose that we roll at least a few PMC
> > members
> > > >> into the Kafka PMC, but I don't have a strong framework for which
> > people
> > > >> to
> > > >> pick.
> > > >>
> > > >> Before (8), I think that Samza's TLP can continue to commit bug
> fixes
> > > and
> > > >> patches as it sees fit, provided that we openly communicate that we
> > > won't
> > > >> necessarily migrate new features to the new subproject, and that the
> > TLP
> > > >> will be shut down after the migration to the Kafka subproject
> occurs.
> > > >>
> > > >> Jakob, I could use your guidance here about about how to achieve
> this
> > > from
> > > >> an Apache process perspective (sorry).
> > > >>
> > > >> * Should I just call a vote on this proposal?
> > > >> * Should it happen on dev or private?
> > > >> * Do committers have binding votes, or just PMC?
> > > >>
> > > >> Having trouble finding much detail on the Apache wikis. :(
> > > >>
> > > >> Cheers,
> > > >> Chris
> > > >>
> > > >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com>
> > wrote:
> > > >>
> > > >> > Thanks, Jay. This argument persuaded me actually. :)
> > > >> >
> > > >> > Fang, Yan
> > > >> > yanfang724@gmail.com
> > > >> >
> > > >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io>
> > wrote:
> > > >> >
> > > >> > > Hey Yan,
> > > >> > >
> > > >> > > Yeah philosophically I think the argument is that you should
> > capture
> > > >> the
> > > >> > > stream in Kafka independent of the transformation. This is
> > > obviously a
> > > >> > > Kafka-centric view point.
> > > >> > >
> > > >> > > Advantages of this:
> > > >> > > - In practice I think this is what e.g. Storm people often end
> up
> > > >> doing
> > > >> > > anyway. You usually need to throttle any access to a live
> serving
> > > >> > database.
> > > >> > > - Can have multiple subscribers and they get the same thing
> > without
> > > >> > > additional load on the source system.
> > > >> > > - Applications can tap into the stream if need be by
> subscribing.
> > > >> > > - You can debug your transformation by tailing the Kafka topic
> > with
> > > >> the
> > > >> > > console consumer
> > > >> > > - Can tee off the same data stream for batch analysis or Lambda
> > arch
> > > >> > style
> > > >> > > re-processing
> > > >> > >
> > > >> > > The disadvantage is that it will use Kafka resources. But the
> idea
> > > is
> > > >> > > eventually you will have multiple subscribers to any data source
> > (at
> > > >> > least
> > > >> > > for monitoring) so you will end up there soon enough anyway.
> > > >> > >
> > > >> > > Down the road the technical benefit is that I think it gives us
> a
> > > good
> > > >> > path
> > > >> > > towards end-to-end exactly once semantics from source to
> > > destination.
> > > >> > > Basically the connectors need to support idempotence when
> talking
> > to
> > > >> > Kafka
> > > >> > > and we need the transactional write feature in Kafka to make the
> > > >> > > transformation atomic. This is actually pretty doable if you
> > > separate
> > > >> > > connector=>kafka problem from the generic transformations which
> > are
> > > >> > always
> > > >> > > kafka=>kafka. However I think it is quite impossible to do in a
> > > >> > all_things
> > > >> > > => all_things environment. Today you can say "well the semantics
> > of
> > > >> the
> > > >> > > Samza APIs depend on the connectors you use" but it is actually
> > > worse
> > > >> > then
> > > >> > > that because the semantics actually depend on the pairing of
> > > >> > connectors--so
> > > >> > > not only can you probably not get a usable "exactly once"
> > guarantee
> > > >> > > end-to-end it can actually be quite hard to reverse engineer
> what
> > > >> > property
> > > >> > > (if any) your end-to-end flow has if you have heterogenous
> > systems.
> > > >> > >
> > > >> > > -Jay
> > > >> > >
> > > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <yanfang724@gmail.com
> >
> > > >> wrote:
> > > >> > >
> > > >> > > > {quote}
> > > >> > > > maintained in a separate repository and retaining the existing
> > > >> > > > committership but sharing as much else as possible (website,
> > etc)
> > > >> > > > {quote}
> > > >> > > >
> > > >> > > > Overall, I agree on this idea. Now the question is more about
> > "how
> > > >> to
> > > >> > do
> > > >> > > > it".
> > > >> > > >
> > > >> > > > On the other hand, one thing I want to point out is that, if
> we
> > > >> decide
> > > >> > to
> > > >> > > > go this way, how do we want to support
> > > >> > > > otherSystem-transformation-otherSystem use case?
> > > >> > > >
> > > >> > > > Basically, there are four user groups here:
> > > >> > > >
> > > >> > > > 1. Kafka-transformation-Kafka
> > > >> > > > 2. Kafka-transformation-otherSystem
> > > >> > > > 3. otherSystem-transformation-Kafka
> > > >> > > > 4. otherSystem-transformation-otherSystem
> > > >> > > >
> > > >> > > > For group 1, they can easily use the new Samza library to
> > achieve.
> > > >> For
> > > >> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka
> > or
> > > >> > Kafka->
> > > >> > > > transformation -> copyCat.
> > > >> > > >
> > > >> > > > The problem is for group 4. Do we want to abandon this or
> still
> > > >> support
> > > >> > > it?
> > > >> > > > Of course, this use case can be achieved by using copyCat ->
> > > >> > > transformation
> > > >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> > > persuade
> > > >> > them
> > > >> > > to
> > > >> > > > do this long chain. If yes, it will also be a win for Kafka
> too.
> > > Or
> > > >> if
> > > >> > > > there is no one in this community actually doing this so far,
> > > maybe
> > > >> ok
> > > >> > to
> > > >> > > > not support the group 4 directly.
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > >
> > > >> > > > Fang, Yan
> > > >> > > > yanfang724@gmail.com
> > > >> > > >
> > > >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <jay@confluent.io
> >
> > > >> wrote:
> > > >> > > >
> > > >> > > > > Yeah I agree with this summary. I think there are kind of
> two
> > > >> > questions
> > > >> > > > > here:
> > > >> > > > > 1. Technically does alignment/reliance on Kafka make sense
> > > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
> > alignment
> > > >> with
> > > >> > > > Kafka
> > > >> > > > > make sense
> > > >> > > > >
> > > >> > > > > Personally I do think both of these things would be really
> > > >> valuable,
> > > >> > > and
> > > >> > > > > would dramatically alter the trajectory of the project.
> > > >> > > > >
> > > >> > > > > My preference would be to see if people can mostly agree on
> a
> > > >> > direction
> > > >> > > > > rather than splintering things off. From my point of view
> the
> > > >> ideal
> > > >> > > > outcome
> > > >> > > > > of all the options discussed would be to make Samza a
> closely
> > > >> aligned
> > > >> > > > > subproject, maintained in a separate repository and
> retaining
> > > the
> > > >> > > > existing
> > > >> > > > > committership but sharing as much else as possible (website,
> > > >> etc). No
> > > >> > > > idea
> > > >> > > > > about how these things work, Jacob, you probably know more.
> > > >> > > > >
> > > >> > > > > No discussion amongst the Kafka folks has happened on this,
> > but
> > > >> > likely
> > > >> > > we
> > > >> > > > > should figure out what the Samza community actually wants
> > first.
> > > >> > > > >
> > > >> > > > > I admit that this is a fairly radical departure from how
> > things
> > > >> are.
> > > >> > > > >
> > > >> > > > > If that doesn't fly, I think, yeah we could leave Samza as
> it
> > is
> > > >> and
> > > >> > do
> > > >> > > > the
> > > >> > > > > more radical reboot inside Kafka. From my point of view that
> > > does
> > > >> > leave
> > > >> > > > > things in a somewhat confusing state since now there are two
> > > >> stream
> > > >> > > > > processing systems more or less coupled to Kafka in large
> part
> > > >> made
> > > >> > by
> > > >> > > > the
> > > >> > > > > same people. But, arguably that might be a cleaner way to
> make
> > > the
> > > >> > > > cut-over
> > > >> > > > > and perhaps less risky for Samza community since if it works
> > > >> people
> > > >> > can
> > > >> > > > > switch and if it doesn't nothing will have changed. Dunno,
> how
> > > do
> > > >> > > people
> > > >> > > > > feel about this?
> > > >> > > > >
> > > >> > > > > -Jay
> > > >> > > > >
> > > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> > > jghoman@gmail.com>
> > > >> > > wrote:
> > > >> > > > >
> > > >> > > > > > >  This leads me to thinking that merging projects and
> > > >> communities
> > > >> > > > might
> > > >> > > > > > be a good idea: with the union of experience from both
> > > >> communities,
> > > >> > > we
> > > >> > > > > will
> > > >> > > > > > probably build a better system that is better for users.
> > > >> > > > > > Is this what's being proposed though? Merging the projects
> > > seems
> > > >> > like
> > > >> > > > > > a consequence of at most one of the three directions under
> > > >> > > discussion:
> > > >> > > > > > 1) Samza 2.0: The Samza community relies more heavily on
> > Kafka
> > > >> for
> > > >> > > > > > configuration, etc. (to a greater or lesser extent to be
> > > >> > determined)
> > > >> > > > > > but the Samza community would not automatically merge
> withe
> > > >> Kafka
> > > >> > > > > > community (the Phoenix/HBase example is a good one here).
> > > >> > > > > > 2) Samza Reboot: The Samza community continues to exist
> > with a
> > > >> > > limited
> > > >> > > > > > project scope, but similarly would not need to be part of
> > the
> > > >> Kafka
> > > >> > > > > > community (ie given committership) to progress.  Here,
> maybe
> > > the
> > > >> > > Samza
> > > >> > > > > > team would become a subproject of Kafka (the Board frowns
> on
> > > >> > > > > > subprojects at the moment, so I'm not sure if that's even
> > > >> > feasible),
> > > >> > > > > > but that would not be required.
> > > >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this
> option
> > > the
> > > >> > Kafka
> > > >> > > > > > team builds its own streaming library, possibly off of
> Jay's
> > > >> > > > > > prototype, which has not direct lineage to the Samza team.
> > > >> There's
> > > >> > > no
> > > >> > > > > > reason for the Kafka team to bring in the Samza team.
> > > >> > > > > >
> > > >> > > > > > Is the Kafka community on board with this?
> > > >> > > > > >
> > > >> > > > > > To be clear, all three options under discussion are
> > > interesting,
> > > >> > > > > > technically valid and likely healthy directions for the
> > > project.
> > > >> > > > > > Also, they are not mutually exclusive.  The Samza
> community
> > > >> could
> > > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
> > community
> > > >> went
> > > >> > > > > > forward with 'Hey Samza!'  My points above are directed
> > > >> entirely at
> > > >> > > > > > the community aspect of these choices.
> > > >> > > > > > -Jakob
> > > >> > > > > >
> > > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> > > roger.hoover@gmail.com>
> > > >> > > wrote:
> > > >> > > > > > > That's great.  Thanks, Jay.
> > > >> > > > > > >
> > > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> > > jay@confluent.io>
> > > >> > > wrote:
> > > >> > > > > > >
> > > >> > > > > > >> Yeah totally agree. I think you have this issue even
> > today,
> > > >> > right?
> > > >> > > > > I.e.
> > > >> > > > > > if
> > > >> > > > > > >> you need to make a simple config change and you're
> > running
> > > in
> > > >> > YARN
> > > >> > > > > today
> > > >> > > > > > >> you end up bouncing the job which then rebuilds state.
> I
> > > >> think
> > > >> > the
> > > >> > > > fix
> > > >> > > > > > is
> > > >> > > > > > >> exactly what you described which is to have a long
> > timeout
> > > on
> > > >> > > > > partition
> > > >> > > > > > >> movement for stateful jobs so that if a job is just
> > getting
> > > >> > > bounced,
> > > >> > > > > and
> > > >> > > > > > >> the cluster manager (or admin) is smart enough to
> restart
> > > it
> > > >> on
> > > >> > > the
> > > >> > > > > same
> > > >> > > > > > >> host when possible, it can optimistically reuse any
> > > existing
> > > >> > state
> > > >> > > > it
> > > >> > > > > > finds
> > > >> > > > > > >> on disk (if it is valid).
> > > >> > > > > > >>
> > > >> > > > > > >> So in this model the charter of the CM is to place
> > > processes
> > > >> as
> > > >> > > > > > stickily as
> > > >> > > > > > >> possible and to restart or re-place failed processes.
> The
> > > >> > charter
> > > >> > > of
> > > >> > > > > the
> > > >> > > > > > >> partition management system is to control the
> assignment
> > of
> > > >> work
> > > >> > > to
> > > >> > > > > > these
> > > >> > > > > > >> processes. The nice thing about this is that the work
> > > >> > assignment,
> > > >> > > > > > timeouts,
> > > >> > > > > > >> behavior, configs, and code will all be the same across
> > all
> > > >> > > cluster
> > > >> > > > > > >> managers.
> > > >> > > > > > >>
> > > >> > > > > > >> So I think that prototype would actually give you
> exactly
> > > >> what
> > > >> > you
> > > >> > > > > want
> > > >> > > > > > >> today for any cluster manager (or manual placement +
> > > restart
> > > >> > > script)
> > > >> > > > > > that
> > > >> > > > > > >> was sticky in terms of host placement since there is
> > > already
> > > >> a
> > > >> > > > > > configurable
> > > >> > > > > > >> partition movement timeout and task-by-task state reuse
> > > with
> > > >> a
> > > >> > > check
> > > >> > > > > on
> > > >> > > > > > >> state validity.
> > > >> > > > > > >>
> > > >> > > > > > >> -Jay
> > > >> > > > > > >>
> > > >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > > >> > > > roger.hoover@gmail.com
> > > >> > > > > >
> > > >> > > > > > >> wrote:
> > > >> > > > > > >>
> > > >> > > > > > >> > That would be great to let Kafka do as much heavy
> > lifting
> > > >> as
> > > >> > > > > possible
> > > >> > > > > > and
> > > >> > > > > > >> > make it easier for other languages to implement Samza
> > > apis.
> > > >> > > > > > >> >
> > > >> > > > > > >> > One thing to watch out for is the interplay between
> > > Kafka's
> > > >> > > group
> > > >> > > > > > >> > management and the external scheduler/process
> manager's
> > > >> fault
> > > >> > > > > > tolerance.
> > > >> > > > > > >> > If a container dies, the Kafka group membership
> > protocol
> > > >> will
> > > >> > > try
> > > >> > > > to
> > > >> > > > > > >> assign
> > > >> > > > > > >> > it's tasks to other containers while at the same time
> > the
> > > >> > > process
> > > >> > > > > > manager
> > > >> > > > > > >> > is trying to relaunch the container.  Without some
> > > >> > consideration
> > > >> > > > for
> > > >> > > > > > this
> > > >> > > > > > >> > (like a configurable amount of time to wait before
> > Kafka
> > > >> > alters
> > > >> > > > the
> > > >> > > > > > group
> > > >> > > > > > >> > membership), there may be thrashing going on which is
> > > >> > especially
> > > >> > > > bad
> > > >> > > > > > for
> > > >> > > > > > >> > containers with large amounts of local state.
> > > >> > > > > > >> >
> > > >> > > > > > >> > Someone else pointed this out already but I thought
> it
> > > >> might
> > > >> > be
> > > >> > > > > worth
> > > >> > > > > > >> > calling out again.
> > > >> > > > > > >> >
> > > >> > > > > > >> > Cheers,
> > > >> > > > > > >> >
> > > >> > > > > > >> > Roger
> > > >> > > > > > >> >
> > > >> > > > > > >> >
> > > >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> > > >> jay@confluent.io>
> > > >> > > > > wrote:
> > > >> > > > > > >> >
> > > >> > > > > > >> > > Hey Roger,
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > I couldn't agree more. We spent a bunch of time
> > talking
> > > >> to
> > > >> > > > people
> > > >> > > > > > and
> > > >> > > > > > >> > that
> > > >> > > > > > >> > > is exactly the stuff we heard time and again. What
> > > makes
> > > >> it
> > > >> > > > hard,
> > > >> > > > > of
> > > >> > > > > > >> > > course, is that there is some tension between
> > > >> compatibility
> > > >> > > with
> > > >> > > > > > what's
> > > >> > > > > > >> > > there now and making things better for new users.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > I also strongly agree with the importance of
> > > >> multi-language
> > > >> > > > > > support. We
> > > >> > > > > > >> > are
> > > >> > > > > > >> > > talking now about Java, but for application
> > development
> > > >> use
> > > >> > > > cases
> > > >> > > > > > >> people
> > > >> > > > > > >> > > want to work in whatever language they are using
> > > >> elsewhere.
> > > >> > I
> > > >> > > > > think
> > > >> > > > > > >> > moving
> > > >> > > > > > >> > > to a model where Kafka itself does the group
> > > membership,
> > > >> > > > lifecycle
> > > >> > > > > > >> > control,
> > > >> > > > > > >> > > and partition assignment has the advantage of
> putting
> > > all
> > > >> > that
> > > >> > > > > > complex
> > > >> > > > > > >> > > stuff behind a clean api that the clients are
> already
> > > >> going
> > > >> > to
> > > >> > > > be
> > > >> > > > > > >> > > implementing for their consumer, so the added
> > > >> functionality
> > > >> > > for
> > > >> > > > > > stream
> > > >> > > > > > >> > > processing beyond a consumer becomes very minor.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > -Jay
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > > >> > > > > > roger.hoover@gmail.com>
> > > >> > > > > > >> > > wrote:
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > > Metamorphosis...nice. :)
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > This has been a great discussion.  As a user of
> > Samza
> > > >> > who's
> > > >> > > > > > recently
> > > >> > > > > > >> > > > integrated it into a relatively large
> > organization, I
> > > >> just
> > > >> > > > want
> > > >> > > > > to
> > > >> > > > > > >> add
> > > >> > > > > > >> > > > support to a few points already made.
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
> > > >> currently
> > > >> > > > exists
> > > >> > > > > > that
> > > >> > > > > > >> > I've
> > > >> > > > > > >> > > > experienced are:
> > > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
> > environments
> > > >> > where
> > > >> > > > > Puppet
> > > >> > > > > > >> > would
> > > >> > > > > > >> > > do
> > > >> > > > > > >> > > > just fine but it was the only mechanism to get
> > fault
> > > >> > > > tolerance.
> > > >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> > > >> configuring
> > > >> > > most
> > > >> > > > > of
> > > >> > > > > > the
> > > >> > > > > > >> > job
> > > >> > > > > > >> > > > in code rather than config files.  In general, I
> > > think
> > > >> the
> > > >> > > > goal
> > > >> > > > > > >> should
> > > >> > > > > > >> > be
> > > >> > > > > > >> > > > to make it harder to make mistakes, especially of
> > the
> > > >> kind
> > > >> > > > where
> > > >> > > > > > the
> > > >> > > > > > >> > code
> > > >> > > > > > >> > > > expects something and the config doesn't match.
> > The
> > > >> > current
> > > >> > > > > > config
> > > >> > > > > > >> is
> > > >> > > > > > >> > > > quite intricate and error-prone.  For example,
> the
> > > >> > > application
> > > >> > > > > > logic
> > > >> > > > > > >> > may
> > > >> > > > > > >> > > > depend on bootstrapping a topic but rather than
> > > >> asserting
> > > >> > > that
> > > >> > > > > in
> > > >> > > > > > the
> > > >> > > > > > >> > > code,
> > > >> > > > > > >> > > > you have to rely on getting the config right.
> > > Likewise
> > > >> > with
> > > >> > > > > > serdes,
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > Java representations produced by various serdes
> > > (JSON,
> > > >> > Avro,
> > > >> > > > > etc.)
> > > >> > > > > > >> are
> > > >> > > > > > >> > > not
> > > >> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> > > >> without
> > > >> > > > > changing
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > code.   It would be nice for jobs to be able to
> > > assert
> > > >> > what
> > > >> > > > they
> > > >> > > > > > >> expect
> > > >> > > > > > >> > > > from their input topics in terms of partitioning.
> > > >> This is
> > > >> > > > > > getting a
> > > >> > > > > > >> > > little
> > > >> > > > > > >> > > > off topic but I was even thinking about creating
> a
> > > >> "Samza
> > > >> > > > config
> > > >> > > > > > >> > linter"
> > > >> > > > > > >> > > > that would sanity check a set of configs.
> > Especially
> > > >> in
> > > >> > > > > > >> organizations
> > > >> > > > > > >> > > > where config is managed by a different team than
> > the
> > > >> > > > application
> > > >> > > > > > >> > > developer,
> > > >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > > >> > > > > > >> > > > 3) Java/Scala centric - for many teams
> (especially
> > > >> > > DevOps-type
> > > >> > > > > > >> folks),
> > > >> > > > > > >> > > the
> > > >> > > > > > >> > > > pain of the Java toolchain (maven, slow builds,
> > weak
> > > >> > command
> > > >> > > > > line
> > > >> > > > > > >> > > support,
> > > >> > > > > > >> > > > configuration over convention) really inhibits
> > > >> > productivity.
> > > >> > > > As
> > > >> > > > > > more
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > more high-quality clients become available for
> > > Kafka, I
> > > >> > hope
> > > >> > > > > > they'll
> > > >> > > > > > >> > > follow
> > > >> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> > > >> proposals
> > > >> > > in
> > > >> > > > > this
> > > >> > > > > > >> > thread
> > > >> > > > > > >> > > > but please consider other languages in the
> > ecosystem
> > > as
> > > >> > > well.
> > > >> > > > > > From
> > > >> > > > > > >> > what
> > > >> > > > > > >> > > > I've heard, Spark has more Python users than
> > > >> Java/Scala.
> > > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > >
> > > >> > > > > > >> >
> > > >> > > > > > >>
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > >> > > > > > >> > > > and are working on a Yeoman generator
> > > >> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> > > >> > > Jython/Samza
> > > >> > > > > > >> projects
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > alleviate some of the pain)
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > I also want to underscore Jay's point about
> > improving
> > > >> the
> > > >> > > user
> > > >> > > > > > >> > > experience.
> > > >> > > > > > >> > > > That's a very important factor for adoption.  I
> > think
> > > >> the
> > > >> > > goal
> > > >> > > > > > should
> > > >> > > > > > >> > be
> > > >> > > > > > >> > > to
> > > >> > > > > > >> > > > make Samza as easy to get started with as
> something
> > > >> like
> > > >> > > > > Logstash.
> > > >> > > > > > >> > > > Logstash is vastly inferior in terms of
> > capabilities
> > > to
> > > >> > > Samza
> > > >> > > > > but
> > > >> > > > > > >> it's
> > > >> > > > > > >> > > easy
> > > >> > > > > > >> > > > to get started and that makes a big difference.
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > Cheers,
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > Roger
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> > > Francisci
> > > >> > > > Morales <
> > > >> > > > > > >> > > > gdfm@apache.org> wrote:
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> > > >> Metamorphosis
> > > >> > > is
> > > >> > > > a
> > > >> > > > > > clear
> > > >> > > > > > >> > > > winner
> > > >> > > > > > >> > > > > :)
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > --
> > > >> > > > > > >> > > > > Gianmarco
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> > > >> Morales
> > > >> > <
> > > >> > > > > > >> > > gdfm@apache.org
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > wrote:
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > > > > Hi,
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > @Martin, thanks for you comments.
> > > >> > > > > > >> > > > > > Maybe I'm missing some important point, but I
> > > think
> > > >> > > > coupling
> > > >> > > > > > the
> > > >> > > > > > >> > > > releases
> > > >> > > > > > >> > > > > > is actually a *good* thing.
> > > >> > > > > > >> > > > > > To make an example, would it be better if the
> > MR
> > > >> and
> > > >> > > HDFS
> > > >> > > > > > >> > components
> > > >> > > > > > >> > > of
> > > >> > > > > > >> > > > > > Hadoop had different release schedules?
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > Actually, keeping the discussion in a single
> > > place
> > > >> > would
> > > >> > > > > make
> > > >> > > > > > >> > > agreeing
> > > >> > > > > > >> > > > on
> > > >> > > > > > >> > > > > > releases (and backwards compatibility) much
> > > >> easier, as
> > > >> > > > > > everybody
> > > >> > > > > > >> > > would
> > > >> > > > > > >> > > > be
> > > >> > > > > > >> > > > > > responsible for the whole codebase.
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > That said, I like the idea of absorbing
> > > samza-core
> > > >> as
> > > >> > a
> > > >> > > > > > >> > sub-project,
> > > >> > > > > > >> > > > and
> > > >> > > > > > >> > > > > > leave the fancy stuff separate.
> > > >> > > > > > >> > > > > > It probably gives 90% of the benefits we have
> > > been
> > > >> > > > > discussing
> > > >> > > > > > >> here.
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > Cheers,
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > --
> > > >> > > > > > >> > > > > > Gianmarco
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > > >> > jay.kreps@gmail.com
> > > >> > > >
> > > >> > > > > > wrote:
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > >> Hey Martin,
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> I agree coupling release schedules is a
> > > downside.
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> Definitely we can try to solve some of the
> > > >> > integration
> > > >> > > > > > problems
> > > >> > > > > > >> in
> > > >> > > > > > >> > > > > >> Confluent Platform or in other
> distributions.
> > > But
> > > >> I
> > > >> > > think
> > > >> > > > > > this
> > > >> > > > > > >> > ends
> > > >> > > > > > >> > > up
> > > >> > > > > > >> > > > > >> being really shallow. I guess I feel to
> really
> > > >> get a
> > > >> > > good
> > > >> > > > > > user
> > > >> > > > > > >> > > > > experience
> > > >> > > > > > >> > > > > >> the two systems have to kind of feel like
> part
> > > of
> > > >> the
> > > >> > > > same
> > > >> > > > > > thing
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> can't really add that in later--you can put
> > both
> > > >> in
> > > >> > the
> > > >> > > > > same
> > > >> > > > > > >> > > > > downloadable
> > > >> > > > > > >> > > > > >> tar file but it doesn't really give a very
> > > >> cohesive
> > > >> > > > > feeling.
> > > >> > > > > > I
> > > >> > > > > > >> > agree
> > > >> > > > > > >> > > > > that
> > > >> > > > > > >> > > > > >> ultimately any of the project stuff is as
> much
> > > >> social
> > > >> > > and
> > > >> > > > > > naming
> > > >> > > > > > >> > as
> > > >> > > > > > >> > > > > >> anything else--theoretically two totally
> > > >> independent
> > > >> > > > > projects
> > > >> > > > > > >> > could
> > > >> > > > > > >> > > > work
> > > >> > > > > > >> > > > > >> to
> > > >> > > > > > >> > > > > >> tightly align. In practice this seems to be
> > > quite
> > > >> > > > difficult
> > > >> > > > > > >> > though.
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> For the frameworks--totally agree it would
> be
> > > >> good to
> > > >> > > > > > maintain
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > > >> framework support with the project. In some
> > > cases
> > > >> > there
> > > >> > > > may
> > > >> > > > > > not
> > > >> > > > > > >> be
> > > >> > > > > > >> > > too
> > > >> > > > > > >> > > > > >> much
> > > >> > > > > > >> > > > > >> there since the integration gets lighter
> but I
> > > >> think
> > > >> > > > > whatever
> > > >> > > > > > >> > stubs
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> need should be included. So no I definitely
> > > wasn't
> > > >> > > trying
> > > >> > > > > to
> > > >> > > > > > >> imply
> > > >> > > > > > >> > > > > >> dropping
> > > >> > > > > > >> > > > > >> support for these frameworks, just making
> the
> > > >> > > integration
> > > >> > > > > > >> lighter
> > > >> > > > > > >> > by
> > > >> > > > > > >> > > > > >> separating process management from partition
> > > >> > > management.
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> You raise two good points we would have to
> > > figure
> > > >> out
> > > >> > > if
> > > >> > > > we
> > > >> > > > > > went
> > > >> > > > > > >> > > down
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> alignment path:
> > > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think
> the
> > > >> first
> > > >> > > > > question
> > > >> > > > > > is
> > > >> > > > > > >> > > > whether
> > > >> > > > > > >> > > > > >> some "re-branding" would be worth it. If so
> > > then I
> > > >> > > think
> > > >> > > > we
> > > >> > > > > > can
> > > >> > > > > > >> > > have a
> > > >> > > > > > >> > > > > big
> > > >> > > > > > >> > > > > >> thread on the name. I'm definitely not set
> on
> > > >> Kafka
> > > >> > > > > > Streaming or
> > > >> > > > > > >> > > Kafka
> > > >> > > > > > >> > > > > >> Streams I was just using them to be kind of
> > > >> > > > illustrative. I
> > > >> > > > > > >> agree
> > > >> > > > > > >> > > with
> > > >> > > > > > >> > > > > >> your
> > > >> > > > > > >> > > > > >> critique of these names, though I think
> people
> > > >> would
> > > >> > > get
> > > >> > > > > the
> > > >> > > > > > >> idea.
> > > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about
> how
> > to
> > > >> > > "factor"
> > > >> > > > > it.
> > > >> > > > > > >> Here
> > > >> > > > > > >> > > are
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> options I see (I could get enthusiastic
> about
> > > any
> > > >> of
> > > >> > > > them):
> > > >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> > > seperation
> > > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api
> > and
> > > >> > > > samza-core
> > > >> > > > > > is
> > > >> > > > > > >> > > > absorbed
> > > >> > > > > > >> > > > > >> almost like a third client
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> Cheers,
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> -Jay
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> > > Kleppmann <
> > > >> > > > > > >> > > > martin@kleppmann.com>
> > > >> > > > > > >> > > > > >> wrote:
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a
> > few
> > > >> > > follow-up
> > > >> > > > > > >> > comments.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka
> or
> > > >> > becoming
> > > >> > > a
> > > >> > > > > > >> > subproject:
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > reasons you mention are good. The risk I
> see
> > > is
> > > >> > that
> > > >> > > > > > release
> > > >> > > > > > >> > > > schedules
> > > >> > > > > > >> > > > > >> > become coupled to each other, which can
> slow
> > > >> > everyone
> > > >> > > > > down,
> > > >> > > > > > >> and
> > > >> > > > > > >> > > > large
> > > >> > > > > > >> > > > > >> > projects with many contributors are harder
> > to
> > > >> > manage.
> > > >> > > > > > (Jakob,
> > > >> > > > > > >> > can
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> speak
> > > >> > > > > > >> > > > > >> > from experience, having seen a wider range
> > of
> > > >> > Hadoop
> > > >> > > > > > ecosystem
> > > >> > > > > > >> > > > > >> projects?)
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > Some of the goals of a better unified
> > > developer
> > > >> > > > > experience
> > > >> > > > > > >> could
> > > >> > > > > > >> > > > also
> > > >> > > > > > >> > > > > be
> > > >> > > > > > >> > > > > >> > solved by integrating Samza nicely into a
> > > Kafka
> > > >> > > > > > distribution
> > > >> > > > > > >> > (such
> > > >> > > > > > >> > > > as
> > > >> > > > > > >> > > > > >> > Confluent's). I'm not against merging
> > projects
> > > >> if
> > > >> > we
> > > >> > > > > decide
> > > >> > > > > > >> > that's
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> way
> > > >> > > > > > >> > > > > >> > to go, just pointing out the same goals
> can
> > > >> perhaps
> > > >> > > > also
> > > >> > > > > be
> > > >> > > > > > >> > > achieved
> > > >> > > > > > >> > > > > in
> > > >> > > > > > >> > > > > >> > other ways.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
> > dependency:
> > > >> are
> > > >> > > you
> > > >> > > > > > >> proposing
> > > >> > > > > > >> > > > that
> > > >> > > > > > >> > > > > >> > Samza doesn't give any help to people
> > wanting
> > > to
> > > >> > run
> > > >> > > on
> > > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > > >> > > > > > >> > > > > >> > So the docs would basically have a link to
> > > >> Slider
> > > >> > and
> > > >> > > > > > nothing
> > > >> > > > > > >> > > else?
> > > >> > > > > > >> > > > Or
> > > >> > > > > > >> > > > > >> > would we maintain integrations with a
> bunch
> > of
> > > >> > > popular
> > > >> > > > > > >> > deployment
> > > >> > > > > > >> > > > > >> methods
> > > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts
> > to
> > > >> make
> > > >> > > > Samza
> > > >> > > > > > work
> > > >> > > > > > >> > with
> > > >> > > > > > >> > > > > >> Slider)?
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > I absolutely think it's a good idea to
> have
> > > the
> > > >> > "as a
> > > >> > > > > > library"
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > "as a
> > > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
> > > >> people
> > > >> > who
> > > >> > > > > want
> > > >> > > > > > >> them,
> > > >> > > > > > >> > > > but I
> > > >> > > > > > >> > > > > >> > think there should also be a low-friction
> > path
> > > >> for
> > > >> > > > common
> > > >> > > > > > "as
> > > >> > > > > > >> a
> > > >> > > > > > >> > > > > service"
> > > >> > > > > > >> > > > > >> > deployment methods, for which we probably
> > need
> > > >> to
> > > >> > > > > maintain
> > > >> > > > > > >> > > > > integrations.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems
> odd
> > to
> > > >> me,
> > > >> > > > > because
> > > >> > > > > > >> Kafka
> > > >> > > > > > >> > > is
> > > >> > > > > > >> > > > > all
> > > >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> > > >> Transformers"
> > > >> > > or
> > > >> > > > > > "Kafka
> > > >> > > > > > >> > > > Filters"
> > > >> > > > > > >> > > > > >> > would be more apt?
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
> > > >> (stream
> > > >> > > > > > >> transformation
> > > >> > > > > > >> > > > with
> > > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> > > >> library"
> > > >> > > bit)
> > > >> > > > > > could
> > > >> > > > > > >> > > become
> > > >> > > > > > >> > > > > >> part of
> > > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
> > > >> streaming
> > > >> > SQL
> > > >> > > > and
> > > >> > > > > > >> > > > integrations
> > > >> > > > > > >> > > > > >> with
> > > >> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> > > >> project?
> > > >> > > In
> > > >> > > > > > other
> > > >> > > > > > >> > > words,
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > would absorb the proven, stable core of
> > Samza,
> > > >> > which
> > > >> > > > > would
> > > >> > > > > > >> > become
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in
> this
> > > >> > thread.
> > > >> > > > The
> > > >> > > > > > Samza
> > > >> > > > > > >> > > > project
> > > >> > > > > > >> > > > > >> > would then target that third Kafka client
> as
> > > its
> > > >> > base
> > > >> > > > > API,
> > > >> > > > > > and
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > > >> project
> > > >> > > > > > >> > > > > >> > would be freed up to explore more
> > experimental
> > > >> new
> > > >> > > > > > horizons.
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > Martin
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > > >> > > > jay.kreps@gmail.com>
> > > >> > > > > > >> wrote:
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > > Hey Martin,
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I
> > actually
> > > >> > don't
> > > >> > > > > think
> > > >> > > > > > it
> > > >> > > > > > >> > ties
> > > >> > > > > > >> > > > our
> > > >> > > > > > >> > > > > >> > hands
> > > >> > > > > > >> > > > > >> > > at all, all it does is refactor things.
> > The
> > > >> > > division
> > > >> > > > of
> > > >> > > > > > >> > > > > >> responsibility is
> > > >> > > > > > >> > > > > >> > > that Samza core is responsible for task
> > > >> > lifecycle,
> > > >> > > > > state,
> > > >> > > > > > >> and
> > > >> > > > > > >> > > > > >> partition
> > > >> > > > > > >> > > > > >> > > management (using the Kafka
> co-ordinator)
> > > but
> > > >> it
> > > >> > is
> > > >> > > > NOT
> > > >> > > > > > >> > > > responsible
> > > >> > > > > > >> > > > > >> for
> > > >> > > > > > >> > > > > >> > > packaging, configuration deployment or
> > > >> execution
> > > >> > of
> > > >> > > > > > >> processes.
> > > >> > > > > > >> > > The
> > > >> > > > > > >> > > > > >> > problem
> > > >> > > > > > >> > > > > >> > > of packaging and starting these
> processes
> > is
> > > >> > > > > > >> > > > > >> > > framework/environment-specific. This
> > leaves
> > > >> > > > individual
> > > >> > > > > > >> > > frameworks
> > > >> > > > > > >> > > > to
> > > >> > > > > > >> > > > > >> be
> > > >> > > > > > >> > > > > >> > as
> > > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you
> can
> > > get
> > > >> > > simple
> > > >> > > > > > >> stateless
> > > >> > > > > > >> > > > > >> support in
> > > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their
> off-the-shelf
> > > app
> > > >> > > > > framework
> > > >> > > > > > >> > > (Slider,
> > > >> > > > > > >> > > > > >> > Marathon,
> > > >> > > > > > >> > > > > >> > > etc). These are well known by people and
> > > have
> > > >> > nice
> > > >> > > > UIs
> > > >> > > > > > and a
> > > >> > > > > > >> > lot
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > > flexibility. I don't think they have
> node
> > > >> > affinity
> > > >> > > > as a
> > > >> > > > > > >> built
> > > >> > > > > > >> > in
> > > >> > > > > > >> > > > > >> option
> > > >> > > > > > >> > > > > >> > > (though I could be wrong). So if we want
> > > that
> > > >> we
> > > >> > > can
> > > >> > > > > > either
> > > >> > > > > > >> > wait
> > > >> > > > > > >> > > > for
> > > >> > > > > > >> > > > > >> them
> > > >> > > > > > >> > > > > >> > > to add it or do a custom framework to
> add
> > > that
> > > >> > > > feature
> > > >> > > > > > (as
> > > >> > > > > > >> > now).
> > > >> > > > > > >> > > > > >> > Obviously
> > > >> > > > > > >> > > > > >> > > if you manage things with old-school ops
> > > tools
> > > >> > > > > > >> > (puppet/chef/etc)
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> get
> > > >> > > > > > >> > > > > >> > > locality easily. The nice thing, though,
> > is
> > > >> that
> > > >> > > all
> > > >> > > > > the
> > > >> > > > > > >> samza
> > > >> > > > > > >> > > > > >> "business
> > > >> > > > > > >> > > > > >> > > logic" around partition management and
> > fault
> > > >> > > > tolerance
> > > >> > > > > > is in
> > > >> > > > > > >> > > Samza
> > > >> > > > > > >> > > > > >> core
> > > >> > > > > > >> > > > > >> > so
> > > >> > > > > > >> > > > > >> > > it is shared across frameworks and the
> > > >> framework
> > > >> > > > > specific
> > > >> > > > > > >> bit
> > > >> > > > > > >> > is
> > > >> > > > > > >> > > > > just
> > > >> > > > > > >> > > > > >> > > whether it is smart enough to try to get
> > the
> > > >> same
> > > >> > > > host
> > > >> > > > > > when
> > > >> > > > > > >> a
> > > >> > > > > > >> > > job
> > > >> > > > > > >> > > > is
> > > >> > > > > > >> > > > > >> > > restarted.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment,
> yeah
> > I
> > > >> think
> > > >> > > the
> > > >> > > > > > goal
> > > >> > > > > > >> > would
> > > >> > > > > > >> > > > be
> > > >> > > > > > >> > > > > >> (a)
> > > >> > > > > > >> > > > > >> > > actually get better alignment in user
> > > >> experience,
> > > >> > > and
> > > >> > > > > (b)
> > > >> > > > > > >> > > express
> > > >> > > > > > >> > > > > >> this in
> > > >> > > > > > >> > > > > >> > > the naming and project branding.
> > > Specifically:
> > > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for
> the
> > > >> > > > > > "transformation"
> > > >> > > > > > >> api
> > > >> > > > > > >> > > to
> > > >> > > > > > >> > > > be
> > > >> > > > > > >> > > > > >> > > discoverable in the main Kafka
> docs--i.e.
> > be
> > > >> able
> > > >> > > to
> > > >> > > > > > explain
> > > >> > > > > > >> > > when
> > > >> > > > > > >> > > > to
> > > >> > > > > > >> > > > > >> use
> > > >> > > > > > >> > > > > >> > > the consumer and when to use the stream
> > > >> > processing
> > > >> > > > > > >> > functionality
> > > >> > > > > > >> > > > and
> > > >> > > > > > >> > > > > >> lead
> > > >> > > > > > >> > > > > >> > > people into that experience.
> > > >> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza
> > 1.4.2
> > > >> (or
> > > >> > > > > > whatever)
> > > >> > > > > > >> > that
> > > >> > > > > > >> > > > has
> > > >> > > > > > >> > > > > >> both
> > > >> > > > > > >> > > > > >> > > Kafka and the stream processing part and
> > > they
> > > >> > > > actually
> > > >> > > > > > work
> > > >> > > > > > >> > > > > together.
> > > >> > > > > > >> > > > > >> > > 3. Unify the programming experience so
> the
> > > >> client
> > > >> > > and
> > > >> > > > > > Samza
> > > >> > > > > > >> > api
> > > >> > > > > > >> > > > > share
> > > >> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> > > committers
> > > >> and
> > > >> > > can
> > > >> > > > > > have a
> > > >> > > > > > >> > > > > separate
> > > >> > > > > > >> > > > > >> > repo,
> > > >> > > > > > >> > > > > >> > > but I'm actually not really sure (I
> can't
> > > >> find a
> > > >> > > > > > definition
> > > >> > > > > > >> > of a
> > > >> > > > > > >> > > > > >> > subproject
> > > >> > > > > > >> > > > > >> > > in Apache).
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > Basically at a high-level you want the
> > > >> experience
> > > >> > > to
> > > >> > > > > > "feel"
> > > >> > > > > > >> > > like a
> > > >> > > > > > >> > > > > >> single
> > > >> > > > > > >> > > > > >> > > system, not to relatively independent
> > things
> > > >> that
> > > >> > > are
> > > >> > > > > > kind
> > > >> > > > > > >> of
> > > >> > > > > > >> > > > > >> awkwardly
> > > >> > > > > > >> > > > > >> > > glued together.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > I think if we did that they having
> naming
> > or
> > > >> > > branding
> > > >> > > > > > like
> > > >> > > > > > >> > > "kafka
> > > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or
> something
> > > >> like
> > > >> > > that
> > > >> > > > > > would
> > > >> > > > > > >> > > > actually
> > > >> > > > > > >> > > > > >> do a
> > > >> > > > > > >> > > > > >> > > good job of conveying what it is. I do
> > that
> > > >> this
> > > >> > > > would
> > > >> > > > > > help
> > > >> > > > > > >> > > > adoption
> > > >> > > > > > >> > > > > >> > quite
> > > >> > > > > > >> > > > > >> > > a lot as it would correctly convey that
> > > using
> > > >> > Kafka
> > > >> > > > > > >> Streaming
> > > >> > > > > > >> > > with
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > is
> > > >> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka
> is
> > > >> pretty
> > > >> > > > > heavily
> > > >> > > > > > >> > adopted
> > > >> > > > > > >> > > > at
> > > >> > > > > > >> > > > > >> this
> > > >> > > > > > >> > > > > >> > > point.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> > > >> originally
> > > >> > > > when
> > > >> > > > > > open
> > > >> > > > > > >> > > > sourcing
> > > >> > > > > > >> > > > > >> > Samza,
> > > >> > > > > > >> > > > > >> > > however at that time Kafka was
> relatively
> > > >> unknown
> > > >> > > and
> > > >> > > > > we
> > > >> > > > > > >> > decided
> > > >> > > > > > >> > > > not
> > > >> > > > > > >> > > > > >> to
> > > >> > > > > > >> > > > > >> > do
> > > >> > > > > > >> > > > > >> > > it since we felt it would be limiting.
> > From
> > > my
> > > >> > > point
> > > >> > > > of
> > > >> > > > > > view
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > > three
> > > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now
> > really
> > > >> > heavily
> > > >> > > > > used
> > > >> > > > > > for
> > > >> > > > > > >> > > > stream
> > > >> > > > > > >> > > > > >> > > processing, (2) we learned that
> > abstracting
> > > >> out
> > > >> > the
> > > >> > > > > > stream
> > > >> > > > > > >> > well
> > > >> > > > > > >> > > is
> > > >> > > > > > >> > > > > >> > > basically impossible, (3) we learned it
> is
> > > >> really
> > > >> > > > hard
> > > >> > > > > to
> > > >> > > > > > >> keep
> > > >> > > > > > >> > > the
> > > >> > > > > > >> > > > > two
> > > >> > > > > > >> > > > > >> > > things feeling like a single product.
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > -Jay
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> > > >> Kleppmann
> > > >> > <
> > > >> > > > > > >> > > > > >> martin@kleppmann.com>
> > > >> > > > > > >> > > > > >> > > wrote:
> > > >> > > > > > >> > > > > >> > >
> > > >> > > > > > >> > > > > >> > >> Hi all,
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> I agree with the general philosophy of
> > > tying
> > > >> > Samza
> > > >> > > > > more
> > > >> > > > > > >> > firmly
> > > >> > > > > > >> > > to
> > > >> > > > > > >> > > > > >> Kafka.
> > > >> > > > > > >> > > > > >> > >> After I spent a while looking at
> > > integrating
> > > >> > other
> > > >> > > > > > message
> > > >> > > > > > >> > > > brokers
> > > >> > > > > > >> > > > > >> (e.g.
> > > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to
> > the
> > > >> > > > conclusion
> > > >> > > > > > that
> > > >> > > > > > >> > > > > >> > SystemConsumer
> > > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
> > > Kafka's
> > > >> > that
> > > >> > > > > pretty
> > > >> > > > > > >> much
> > > >> > > > > > >> > > > > nobody
> > > >> > > > > > >> > > > > >> but
> > > >> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus
> is
> > > >> > perhaps
> > > >> > > an
> > > >> > > > > > >> > exception,
> > > >> > > > > > >> > > > but
> > > >> > > > > > >> > > > > >> it
> > > >> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.)
> > > Thus,
> > > >> > > making
> > > >> > > > > > Samza
> > > >> > > > > > >> > > fully
> > > >> > > > > > >> > > > > >> > dependent
> > > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > > >> > system-independence
> > > >> > > > was
> > > >> > > > > > >> never
> > > >> > > > > > >> > as
> > > >> > > > > > >> > > > > real
> > > >> > > > > > >> > > > > >> as
> > > >> > > > > > >> > > > > >> > we
> > > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of
> > > code
> > > >> > reuse
> > > >> > > > are
> > > >> > > > > > >> real.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN
> > has
> > > >> also
> > > >> > > > always
> > > >> > > > > > been
> > > >> > > > > > >> > > > > >> appealing to
> > > >> > > > > > >> > > > > >> > >> me, for various reasons already
> mentioned
> > > in
> > > >> > this
> > > >> > > > > > thread.
> > > >> > > > > > >> > > > Although
> > > >> > > > > > >> > > > > >> > making
> > > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > > >> > > > (YARN/Mesos/AWS/etc)
> > > >> > > > > > >> seems
> > > >> > > > > > >> > > > > >> laudable,
> > > >> > > > > > >> > > > > >> > I am
> > > >> > > > > > >> > > > > >> > >> a little concerned that it will
> restrict
> > us
> > > >> to a
> > > >> > > > > lowest
> > > >> > > > > > >> > common
> > > >> > > > > > >> > > > > >> > denominator.
> > > >> > > > > > >> > > > > >> > >> For example, would host affinity
> > > (SAMZA-617)
> > > >> > still
> > > >> > > > be
> > > >> > > > > > >> > possible?
> > > >> > > > > > >> > > > For
> > > >> > > > > > >> > > > > >> jobs
> > > >> > > > > > >> > > > > >> > >> with large amounts of state, I think
> > > >> SAMZA-617
> > > >> > > would
> > > >> > > > > be
> > > >> > > > > > a
> > > >> > > > > > >> big
> > > >> > > > > > >> > > > boon,
> > > >> > > > > > >> > > > > >> > since
> > > >> > > > > > >> > > > > >> > >> restoring state off the changelog on
> > every
> > > >> > single
> > > >> > > > > > restart
> > > >> > > > > > >> is
> > > >> > > > > > >> > > > > painful,
> > > >> > > > > > >> > > > > >> > due
> > > >> > > > > > >> > > > > >> > >> to long recovery times. It would be a
> > shame
> > > >> if
> > > >> > the
> > > >> > > > > > >> decoupling
> > > >> > > > > > >> > > > from
> > > >> > > > > > >> > > > > >> YARN
> > > >> > > > > > >> > > > > >> > >> made host affinity impossible.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Jay, a question about the proposed API
> > for
> > > >> > > > > > instantiating a
> > > >> > > > > > >> > job
> > > >> > > > > > >> > > in
> > > >> > > > > > >> > > > > >> code
> > > >> > > > > > >> > > > > >> > >> (rather than a properties file): when
> > > >> > submitting a
> > > >> > > > job
> > > >> > > > > > to a
> > > >> > > > > > >> > > > > cluster,
> > > >> > > > > > >> > > > > >> is
> > > >> > > > > > >> > > > > >> > the
> > > >> > > > > > >> > > > > >> > >> idea that the instantiation code runs
> on
> > a
> > > >> > client
> > > >> > > > > > >> somewhere,
> > > >> > > > > > >> > > > which
> > > >> > > > > > >> > > > > >> then
> > > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > > >> > > YARN/Mesos/AWS/etc?
> > > >> > > > > Or
> > > >> > > > > > >> does
> > > >> > > > > > >> > > that
> > > >> > > > > > >> > > > > >> code
> > > >> > > > > > >> > > > > >> > run
> > > >> > > > > > >> > > > > >> > >> on each container that is part of the
> job
> > > (in
> > > >> > > which
> > > >> > > > > > case,
> > > >> > > > > > >> how
> > > >> > > > > > >> > > > does
> > > >> > > > > > >> > > > > >> the
> > > >> > > > > > >> > > > > >> > job
> > > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
> > > >> right to
> > > >> > > > make
> > > >> > > > > a
> > > >> > > > > > 1.0
> > > >> > > > > > >> > > > release
> > > >> > > > > > >> > > > > >> > with a
> > > >> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete.
> > So
> > > if
> > > >> > this
> > > >> > > > is
> > > >> > > > > > going
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> happen, I
> > > >> > > > > > >> > > > > >> > >> think it would be more honest to stick
> > with
> > > >> 0.*
> > > >> > > > > version
> > > >> > > > > > >> > numbers
> > > >> > > > > > >> > > > > until
> > > >> > > > > > >> > > > > >> > the
> > > >> > > > > > >> > > > > >> > >> library-ified Samza has been
> implemented,
> > > is
> > > >> > > stable
> > > >> > > > > and
> > > >> > > > > > >> > widely
> > > >> > > > > > >> > > > > used.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
> > > >> Kafka?
> > > >> > > There
> > > >> > > > > is
> > > >> > > > > > >> > > precedent
> > > >> > > > > > >> > > > > for
> > > >> > > > > > >> > > > > >> > >> tight coupling between different Apache
> > > >> projects
> > > >> > > > (e.g.
> > > >> > > > > > >> > Curator
> > > >> > > > > > >> > > > and
> > > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I
> > think
> > > >> > > remaining
> > > >> > > > > > >> separate
> > > >> > > > > > >> > > > would
> > > >> > > > > > >> > > > > >> be
> > > >> > > > > > >> > > > > >> > ok.
> > > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on
> > Kafka,
> > > >> there
> > > >> > > is
> > > >> > > > > > enough
> > > >> > > > > > >> > > > > substance
> > > >> > > > > > >> > > > > >> in
> > > >> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> > > >> project.
> > > >> > > An
> > > >> > > > > > >> argument
> > > >> > > > > > >> > in
> > > >> > > > > > >> > > > > >> favour
> > > >> > > > > > >> > > > > >> > of
> > > >> > > > > > >> > > > > >> > >> merging would be if we think Kafka has
> a
> > > much
> > > >> > > > stronger
> > > >> > > > > > >> "brand
> > > >> > > > > > >> > > > > >> presence"
> > > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one.
> > If
> > > >> the
> > > >> > > Kafka
> > > >> > > > > > >> project
> > > >> > > > > > >> > is
> > > >> > > > > > >> > > > > >> willing
> > > >> > > > > > >> > > > > >> > to
> > > >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of
> > > doing
> > > >> > > > stateful
> > > >> > > > > > >> stream
> > > >> > > > > > >> > > > > >> > >> transformations, that would probably
> have
> > > >> much
> > > >> > the
> > > >> > > > > same
> > > >> > > > > > >> > effect
> > > >> > > > > > >> > > as
> > > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> > > >> Processors"
> > > >> > or
> > > >> > > > > > suchlike.
> > > >> > > > > > >> > > Close
> > > >> > > > > > >> > > > > >> > >> collaboration between the two projects
> > will
> > > >> be
> > > >> > > > needed
> > > >> > > > > in
> > > >> > > > > > >> any
> > > >> > > > > > >> > > > case.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> From a project management perspective,
> I
> > > >> guess
> > > >> > the
> > > >> > > > > "new
> > > >> > > > > > >> > Samza"
> > > >> > > > > > >> > > > > would
> > > >> > > > > > >> > > > > >> > have
> > > >> > > > > > >> > > > > >> > >> to be developed on a branch alongside
> > > ongoing
> > > >> > > > > > maintenance
> > > >> > > > > > >> of
> > > >> > > > > > >> > > the
> > > >> > > > > > >> > > > > >> current
> > > >> > > > > > >> > > > > >> > >> line of development? I think it would
> be
> > > >> > important
> > > >> > > > to
> > > >> > > > > > >> > continue
> > > >> > > > > > >> > > > > >> > supporting
> > > >> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> > > >> migration
> > > >> > > > path
> > > >> > > > > to
> > > >> > > > > > >> the
> > > >> > > > > > >> > > new
> > > >> > > > > > >> > > > > >> > version.
> > > >> > > > > > >> > > > > >> > >> Leaving the current versions
> unsupported
> > > and
> > > >> > > forcing
> > > >> > > > > > people
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> rewrite
> > > >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> Best,
> > > >> > > > > > >> > > > > >> > >> Martin
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > > >> > > > jay@confluent.io>
> > > >> > > > > > >> wrote:
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >>> Hey Garry,
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be
> > > happy
> > > >> to
> > > >> > > chat
> > > >> > > > > > more
> > > >> > > > > > >> > about
> > > >> > > > > > >> > > > > this
> > > >> > > > > > >> > > > > >> if
> > > >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris
> and I
> > > >> > started
> > > >> > > > with
> > > >> > > > > > the
> > > >> > > > > > >> > idea
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> "what
> > > >> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> > > >> > ingestion
> > > >> > > > > tool"
> > > >> > > > > > but
> > > >> > > > > > >> > > > > >> ultimately
> > > >> > > > > > >> > > > > >> > we
> > > >> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> > > >> ingestion
> > > >> > > and
> > > >> > > > > > >> > > > transformation
> > > >> > > > > > >> > > > > >> had
> > > >> > > > > > >> > > > > >> > >>> pretty different needs and coupling
> the
> > > two
> > > >> > made
> > > >> > > > > things
> > > >> > > > > > >> > hard.
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
> > > (KIP-26)
> > > >> > > > actually
> > > >> > > > > > will
> > > >> > > > > > >> > do
> > > >> > > > > > >> > > > what
> > > >> > > > > > >> > > > > >> you
> > > >> > > > > > >> > > > > >> > >> are
> > > >> > > > > > >> > > > > >> > >>> looking for.
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> With regard to your point about
> slider,
> > I
> > > >> don't
> > > >> > > > > > >> necessarily
> > > >> > > > > > >> > > > > >> disagree.
> > > >> > > > > > >> > > > > >> > >> But I
> > > >> > > > > > >> > > > > >> > >>> think getting good YARN support is
> quite
> > > >> doable
> > > >> > > > and I
> > > >> > > > > > >> think
> > > >> > > > > > >> > we
> > > >> > > > > > >> > > > can
> > > >> > > > > > >> > > > > >> make
> > > >> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> > > >> proposal
> > > >> > > > > solves
> > > >> > > > > > is
> > > >> > > > > > >> > that
> > > >> > > > > > >> > > > > >> > >> technically
> > > >> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
> > > >> cluster
> > > >> > > > > > management
> > > >> > > > > > >> > > systems
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > way
> > > >> > > > > > >> > > > > >> > >>> things are now, you need to write an
> > "app
> > > >> > master"
> > > >> > > > or
> > > >> > > > > > >> > > "framework"
> > > >> > > > > > >> > > > > for
> > > >> > > > > > >> > > > > >> > each
> > > >> > > > > > >> > > > > >> > >>> and they are all a little different so
> > > >> testing
> > > >> > is
> > > >> > > > > > really
> > > >> > > > > > >> > hard.
> > > >> > > > > > >> > > > In
> > > >> > > > > > >> > > > > >> the
> > > >> > > > > > >> > > > > >> > >>> absence of this we have been stuck
> with
> > > just
> > > >> > YARN
> > > >> > > > > which
> > > >> > > > > > >> has
> > > >> > > > > > >> > > > > >> fantastic
> > > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the
> > > org,
> > > >> but
> > > >> > > > zero
> > > >> > > > > > >> > > penetration
> > > >> > > > > > >> > > > > >> > >> elsewhere.
> > > >> > > > > > >> > > > > >> > >>> Given the huge amount of work being
> put
> > in
> > > >> to
> > > >> > > > slider,
> > > >> > > > > > >> > > marathon,
> > > >> > > > > > >> > > > > aws
> > > >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
> > > related
> > > >> > > > packaging
> > > >> > > > > > >> > > > technologies
> > > >> > > > > > >> > > > > >> > people
> > > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes,
> various
> > > >> > > > > cloud-specific
> > > >> > > > > > >> > deploy
> > > >> > > > > > >> > > > > >> tools,
> > > >> > > > > > >> > > > > >> > >> etc)
> > > >> > > > > > >> > > > > >> > >>> I really think it is important to get
> > this
> > > >> > right.
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> -Jay
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> > > >> > Turkington
> > > >> > > <
> > > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com>
> wrote:
> > > >> > > > > > >> > > > > >> > >>>
> > > >> > > > > > >> > > > > >> > >>>> Hi all,
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> I think the question below re does
> > Samza
> > > >> > become
> > > >> > > a
> > > >> > > > > > >> > sub-project
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > >>>> highlights the broader point around
> > > >> migration.
> > > >> > > > Chris
> > > >> > > > > > >> > mentions
> > > >> > > > > > >> > > > > >> Samza's
> > > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1
> > release
> > > >> but
> > > >> > I'm
> > > >> > > > not
> > > >> > > > > > sure
> > > >> > > > > > >> > it
> > > >> > > > > > >> > > > > feels
> > > >> > > > > > >> > > > > >> > >> right to
> > > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> > > >> deprecate
> > > >> > > > most
> > > >> > > > > of
> > > >> > > > > > >> it.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have
> some
> > > guys
> > > >> > who
> > > >> > > > have
> > > >> > > > > > >> > started
> > > >> > > > > > >> > > > > >> working
> > > >> > > > > > >> > > > > >> > >> with
> > > >> > > > > > >> > > > > >> > >>>> Samza and building some new
> > > >> > consumers/producers
> > > >> > > > was
> > > >> > > > > > next
> > > >> > > > > > >> > up.
> > > >> > > > > > >> > > > > Sounds
> > > >> > > > > > >> > > > > >> > like
> > > >> > > > > > >> > > > > >> > >>>> that is absolutely not the direction
> to
> > > >> go. I
> > > >> > > need
> > > >> > > > > to
> > > >> > > > > > >> look
> > > >> > > > > > >> > > into
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > KIP
> > > >> > > > > > >> > > > > >> > >> in
> > > >> > > > > > >> > > > > >> > >>>> more detail but for me the
> > attractiveness
> > > >> of
> > > >> > > > adding
> > > >> > > > > > new
> > > >> > > > > > >> > Samza
> > > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all
> > > they
> > > >> > were
> > > >> > > > > doing
> > > >> > > > > > was
> > > >> > > > > > >> > > > really
> > > >> > > > > > >> > > > > >> > getting
> > > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to
> > > avoid
> > > >> > > > having
> > > >> > > > > to
> > > >> > > > > > >> > worry
> > > >> > > > > > >> > > > > about
> > > >> > > > > > >> > > > > >> the
> > > >> > > > > > >> > > > > >> > >>>> lifecycle management of external
> > clients.
> > > >> If
> > > >> > > there
> > > >> > > > > is
> > > >> > > > > > a
> > > >> > > > > > >> > > generic
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug
> a
> > > new
> > > >> > > > connector
> > > >> > > > > > into
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > have
> > > >> > > > > > >> > > > > >> a
> > > >> > > > > > >> > > > > >> > >> lot of
> > > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> > > reliability
> > > >> > done
> > > >> > > > for
> > > >> > > > > me
> > > >> > > > > > >> then
> > > >> > > > > > >> > > it
> > > >> > > > > > >> > > > > >> gives
> > > >> > > > > > >> > > > > >> > me
> > > >> > > > > > >> > > > > >> > >> all
> > > >> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers
> > > would.
> > > >> If
> > > >> > > not
> > > >> > > > > > then it
> > > >> > > > > > >> > > > > >> complicates
> > > >> > > > > > >> > > > > >> > my
> > > >> > > > > > >> > > > > >> > >>>> operational deployments.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Which is similar to my other question
> > > with
> > > >> the
> > > >> > > > > > proposal
> > > >> > > > > > >> --
> > > >> > > > > > >> > if
> > > >> > > > > > >> > > > we
> > > >> > > > > > >> > > > > >> > build a
> > > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza
> plus
> > > the
> > > >> > > > requisite
> > > >> > > > > > >> shims
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> > integrate
> > > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former
> > may
> > > >> be a
> > > >> > > lot
> > > >> > > > > more
> > > >> > > > > > >> work
> > > >> > > > > > >> > > > than
> > > >> > > > > > >> > > > > we
> > > >> > > > > > >> > > > > >> > >> think.
> > > >> > > > > > >> > > > > >> > >>>> We may make it much easier for a
> > newcomer
> > > >> to
> > > >> > get
> > > >> > > > > > >> something
> > > >> > > > > > >> > > > > running
> > > >> > > > > > >> > > > > >> but
> > > >> > > > > > >> > > > > >> > >>>> having them step up and get a
> reliable
> > > >> > > production
> > > >> > > > > > >> > deployment
> > > >> > > > > > >> > > > may
> > > >> > > > > > >> > > > > >> still
> > > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if
> for
> > > >> > different
> > > >> > > > > > reasons
> > > >> > > > > > >> > than
> > > >> > > > > > >> > > > > >> today.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable
> > > with
> > > >> > > making
> > > >> > > > > the
> > > >> > > > > > >> Samza
> > > >> > > > > > >> > > > > >> dependency
> > > >> > > > > > >> > > > > >> > >> on
> > > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
> > absolutely
> > > >> see
> > > >> > > the
> > > >> > > > > > >> benefits
> > > >> > > > > > >> > > in
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > > >> > > > > > >> > > > terminologies/abstractions
> > > >> > > > > > >> > > > > >> that
> > > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a
> library
> > > >> would
> > > >> > > > likely
> > > >> > > > > > be a
> > > >> > > > > > >> > very
> > > >> > > > > > >> > > > > nice
> > > >> > > > > > >> > > > > >> > tool
> > > >> > > > > > >> > > > > >> > >> to
> > > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just
> have
> > > the
> > > >> > > > concerns
> > > >> > > > > > >> above
> > > >> > > > > > >> > re
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > >>>> operational side.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Garry
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> > > >> [mailto:
> > > >> > > > > > >> > gdfm@apache.org
> > > >> > > > > > >> > > ]
> > > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and
> obesrvations
> > on
> > > >> > Samza
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > > >> > > > > > >> > > > > >> > >>>> From outside, I have always perceived
> > > Samza
> > > >> > as a
> > > >> > > > > > >> computing
> > > >> > > > > > >> > > > layer
> > > >> > > > > > >> > > > > >> over
> > > >> > > > > > >> > > > > >> > >>>> Kafka.
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit
> provocative,
> > is
> > > >> > > "should
> > > >> > > > > > Samza
> > > >> > > > > > >> be
> > > >> > > > > > >> > a
> > > >> > > > > > >> > > > > >> > sub-project
> > > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> > > >> separate
> > > >> > > > > project
> > > >> > > > > > >> > with a
> > > >> > > > > > >> > > > > >> separate
> > > >> > > > > > >> > > > > >> > >>>> governance?
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> Cheers,
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> --
> > > >> > > > > > >> > > > > >> > >>>> Gianmarco
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > > >> > > > > > yanfang724@gmail.com>
> > > >> > > > > > >> > > > wrote:
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with
> Kafka
> > > more
> > > >> > > > tightly.
> > > >> > > > > > >> > Because
> > > >> > > > > > >> > > > > Samza
> > > >> > > > > > >> > > > > >> de
> > > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it
> should
> > > >> > leverage
> > > >> > > > > what
> > > >> > > > > > >> Kafka
> > > >> > > > > > >> > > > has.
> > > >> > > > > > >> > > > > At
> > > >> > > > > > >> > > > > >> > the
> > > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
> > > reinvent
> > > >> > what
> > > >> > > > > Samza
> > > >> > > > > > >> > > already
> > > >> > > > > > >> > > > > >> has. I
> > > >> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> > > >> > ingestion
> > > >> > > > and
> > > >> > > > > > >> > > > > transformation.
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me
> to
> > > >> image
> > > >> > > how
> > > >> > > > > the
> > > >> > > > > > >> Samza
> > > >> > > > > > >> > > > will
> > > >> > > > > > >> > > > > >> look
> > > >> > > > > > >> > > > > >> > >>>> like.
> > > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a
> little
> > > >> > > difference
> > > >> > > > > in
> > > >> > > > > > >> terms
> > > >> > > > > > >> > > of
> > > >> > > > > > >> > > > > how
> > > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's
> code
> > > >> shows
> > > >> > (A
> > > >> > > > > > client of
> > > >> > > > > > >> > > > Kakfa)
> > > >> > > > > > >> > > > > ?
> > > >> > > > > > >> > > > > >> And
> > > >> > > > > > >> > > > > >> > >>>>> user's application code calls this
> > > client?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of
> > > Kafka
> > > >> > (like
> > > >> > > > > what
> > > >> > > > > > the
> > > >> > > > > > >> > > code
> > > >> > > > > > >> > > > > >> shows),
> > > >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > > >> > > > > fault-tolerance?
> > > >> > > > > > >> Are
> > > >> > > > > > >> > > they
> > > >> > > > > > >> > > > > >> taken
> > > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> > > >> mechanism,
> > > >> > > such
> > > >> > > > > as
> > > >> > > > > > >> > "Samza
> > > >> > > > > > >> > > > > >> worker"
> > > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 2. What about other features, such
> as
> > > >> > > > auto-scaling,
> > > >> > > > > > >> shared
> > > >> > > > > > >> > > > > state,
> > > >> > > > > > >> > > > > >> > >>>>> monitoring?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is
> > > this
> > > >> > what
> > > >> > > > > Chris
> > > >> > > > > > >> > > > suggests?)
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from
> > > Kakfa
> > > >> > and
> > > >> > > > > > produce
> > > >> > > > > > >> to
> > > >> > > > > > >> > > it.
> > > >> > > > > > >> > > > > >> Then it
> > > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks
> > > like
> > > >> > now,
> > > >> > > > > > except it
> > > >> > > > > > >> > > does
> > > >> > > > > > >> > > > > not
> > > >> > > > > > >> > > > > >> > rely
> > > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> > > >> leverage
> > > >> > > > Kafka's
> > > >> > > > > > >> > metrics,
> > > >> > > > > > >> > > > > logs,
> > > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the
> dependency?
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> Thanks,
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM,
> > Guozhang
> > > >> > Wang <
> > > >> > > > > > >> > > > > wangguoz@gmail.com
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> Read through the code example and
> it
> > > >> looks
> > > >> > > good
> > > >> > > > to
> > > >> > > > > > me.
> > > >> > > > > > >> A
> > > >> > > > > > >> > > few
> > > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> > > >> runnable
> > > >> > > like:
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > > >> > > --config-factory=...
> > > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> > > deploying
> > > >> > Samza
> > > >> > > > > more
> > > >> > > > > > as
> > > >> > > > > > >> > > > embedded
> > > >> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> > > >> (ignoring
> > > >> > > the
> > > >> > > > > > >> > > terminology
> > > >> > > > > > >> > > > > >> since
> > > >> > > > > > >> > > > > >> > >>>>>> it is not the
> > > >> > > > > > >> > > > > >> > >>>>> same
> > > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> > > >> MyStreamTask(configs);
> > > >> > > > > Thread
> > > >> > > > > > >> > thread
> > > >> > > > > > >> > > =
> > > >> > > > > > >> > > > > new
> > > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment
> > modes
> > > >> are
> > > >> > > > > important
> > > >> > > > > > >> for
> > > >> > > > > > >> > > > > >> different
> > > >> > > > > > >> > > > > >> > >>>>>> types
> > > >> > > > > > >> > > > > >> > >>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making
> > Samza
> > > >> > purely
> > > >> > > > > > >> standalone
> > > >> > > > > > >> > is
> > > >> > > > > > >> > > > > still
> > > >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or
> > > library
> > > >> > > modes.
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> Guozhang
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM,
> Jay
> > > >> Kreps
> > > >> > <
> > > >> > > > > > >> > > > jay@confluent.io>
> > > >> > > > > > >> > > > > >> > wrote:
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> > > >> example,
> > > >> > it
> > > >> > > > was
> > > >> > > > > > >> > supposed
> > > >> > > > > > >> > > > to
> > > >> > > > > > >> > > > > >> look
> > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > >> > > > > > >> > > > > >> > >>>>>>> this:
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new
> Properties();
> > > >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > > >> > > > "localhost:4242");
> > > >> > > > > > >> > > > > >> StreamingConfig
> > > >> > > > > > >> > > > > >> > >>>>>>> config = new
> StreamingConfig(props);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > >> > > > "test-topic-2");
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > config.processor(ExampleStreamProcessor.class);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > >> > StringSerializer(),
> > > >> > > > new
> > > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
> > KafkaStreaming
> > > >> > > > container =
> > > >> > > > > > new
> > > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> > > container.run();
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> -Jay
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM,
> > Jay
> > > >> > Kreps <
> > > >> > > > > > >> > > > jay@confluent.io
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > >> > >>>> wrote:
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some
> conversations
> > > >> Chris
> > > >> > > and
> > > >> > > > I
> > > >> > > > > > were
> > > >> > > > > > >> > > having
> > > >> > > > > > >> > > > > >> > >>>>>>>> around
> > > >> > > > > > >> > > > > >> > >>>>>>> whether
> > > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza
> > as a
> > > >> kind
> > > >> > > of
> > > >> > > > > data
> > > >> > > > > > >> > > > ingestion
> > > >> > > > > > >> > > > > >> > >>>>> framework
> > > >> > > > > > >> > > > > >> > >>>>>>> for
> > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to
> > > KIP-26
> > > >> > > > > "copycat").
> > > >> > > > > > >> This
> > > >> > > > > > >> > > > kind
> > > >> > > > > > >> > > > > of
> > > >> > > > > > >> > > > > >> > >>>>>> combined
> > > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and
> > > YARN
> > > >> and
> > > >> > > the
> > > >> > > > > > >> > discussion
> > > >> > > > > > >> > > > > >> around
> > > >> > > > > > >> > > > > >> > >>>>>>>> how
> > > >> > > > > > >> > > > > >> > >>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was,
> > given
> > > >> that
> > > >> > > > Samza
> > > >> > > > > > was
> > > >> > > > > > >> > > > basically
> > > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific,
> > what
> > > if
> > > >> > you
> > > >> > > > just
> > > >> > > > > > >> > embraced
> > > >> > > > > > >> > > > > that
> > > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > > >> > > > > > >> > > > > >> > >>>>>> into
> > > >> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> > > >> > framework
> > > >> > > > and
> > > >> > > > > > more
> > > >> > > > > > >> > > like a
> > > >> > > > > > >> > > > > >> > >>>>>>>> third
> > > >> > > > > > >> > > > > >> > >>>>> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> > > consumer"
> > > >> > with
> > > >> > > > > state
> > > >> > > > > > >> > > > management
> > > >> > > > > > >> > > > > >> > >>>>>> facilities.
> > > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> > > >> complex
> > > >> > > > stream
> > > >> > > > > > >> > > processing
> > > >> > > > > > >> > > > > >> > >>>>>>>> framework
> > > >> > > > > > >> > > > > >> > >>>>>>> this
> > > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple
> > > thing,
> > > >> not
> > > >> > > > much
> > > >> > > > > > more
> > > >> > > > > > >> > > > > >> complicated
> > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>> use
> > > >> > > > > > >> > > > > >> > >>>>>>> or
> > > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As
> > > Chris
> > > >> > said
> > > >> > > > we
> > > >> > > > > > >> thought
> > > >> > > > > > >> > > > about
> > > >> > > > > > >> > > > > >> it
> > > >> > > > > > >> > > > > >> > >>>>>>>> a
> > > >> > > > > > >> > > > > >> > >>>>> lot
> > > >> > > > > > >> > > > > >> > >>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> > > >> > processing
> > > >> > > > > > systems
> > > >> > > > > > >> > were
> > > >> > > > > > >> > > > > doing)
> > > >> > > > > > >> > > > > >> > >>>>> seemed
> > > >> > > > > > >> > > > > >> > >>>>>>> like
> > > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from
> MapReduce.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to
> ingest/output
> > > >> data
> > > >> > to
> > > >> > > > and
> > > >> > > > > > from
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > > > stream
> > > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually
> > > looked
> > > >> > into
> > > >> > > > how
> > > >> > > > > > that
> > > >> > > > > > >> > > would
> > > >> > > > > > >> > > > > >> > >>>>>>>> work,
> > > >> > > > > > >> > > > > >> > >>>>> Samza
> > > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
> > ingestion
> > > >> > > framework
> > > >> > > > > > for a
> > > >> > > > > > >> > > bunch
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > >>>>> reasons.
> > > >> > > > > > >> > > > > >> > >>>>>> To
> > > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a
> > > pretty
> > > >> > > > different
> > > >> > > > > > >> > internal
> > > >> > > > > > >> > > > > data
> > > >> > > > > > >> > > > > >> > >>>>>>>> model
> > > >> > > > > > >> > > > > >> > >>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split
> > > them
> > > >> and
> > > >> > > had
> > > >> > > > > an
> > > >> > > > > > api
> > > >> > > > > > >> > for
> > > >> > > > > > >> > > > > Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA
> KIP-26)
> > > >> and a
> > > >> > > > > separate
> > > >> > > > > > >> api
> > > >> > > > > > >> > > for
> > > >> > > > > > >> > > > > >> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> > > embracing
> > > >> the
> > > >> > > > same
> > > >> > > > > > >> > > terminology
> > > >> > > > > > >> > > > > and
> > > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about
> > the
> > > >> > current
> > > >> > > > > > state is
> > > >> > > > > > >> > > that
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > >>>>>>>> two
> > > >> > > > > > >> > > > > >> > >>>>>>> systems
> > > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on.
> Terminology
> > > >> like
> > > >> > > > > "stream"
> > > >> > > > > > vs
> > > >> > > > > > >> > > > "topic"
> > > >> > > > > > >> > > > > >> and
> > > >> > > > > > >> > > > > >> > >>>>>>> different
> > > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems
> means
> > > you
> > > >> > kind
> > > >> > > > of
> > > >> > > > > > have
> > > >> > > > > > >> to
> > > >> > > > > > >> > > > learn
> > > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > > >> > > > > > >> > > > > >> > >>>>>>> way,
> > > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
> > different
> > > >> way,
> > > >> > > > then
> > > >> > > > > > kind
> > > >> > > > > > >> of
> > > >> > > > > > >> > > > > >> > >>>>>>>> understand
> > > >> > > > > > >> > > > > >> > >>>>> how
> > > >> > > > > > >> > > > > >> > >>>>>>> they
> > > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having
> > > walked
> > > >> a
> > > >> > few
> > > >> > > > > > people
> > > >> > > > > > >> > > through
> > > >> > > > > > >> > > > > >> this
> > > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks
> to
> > > >> get.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot
> of
> > > >> time
> > > >> > on
> > > >> > > > > > >> airplanes I
> > > >> > > > > > >> > > > > hacked
> > > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> > > >> incomplete
> > > >> > > > > > prototype
> > > >> > > > > > >> of
> > > >> > > > > > >> > > > what
> > > >> > > > > > >> > > > > >> > >>>>>>>> this would
> > > >> > > > > > >> > > > > >> > >>>>> look
> > > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just
> unceremoniously
> > > >> dumped
> > > >> > > into
> > > >> > > > > > Kafka
> > > >> > > > > > >> as
> > > >> > > > > > >> > > it
> > > >> > > > > > >> > > > > >> > >>>>>>>> required a
> > > >> > > > > > >> > > > > >> > >>>>>> few
> > > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here
> > is
> > > >> the
> > > >> > > code:
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> >
> > > >> > > > > >
> > > >> > >
> > > >>
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype
> I
> > > just
> > > >> > > > > liberally
> > > >> > > > > > >> > renamed
> > > >> > > > > > >> > > > > >> > >>>>>>>> everything
> > > >> > > > > > >> > > > > >> > >>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with
> no
> > > >> regard
> > > >> > > for
> > > >> > > > > > >> > > > compatibility.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something
> like
> > > >> this:
> > > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
> > Properties();
> > > >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > > >> > > > > "localhost:4242");
> > > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > > >> > > > > > >> > > > > >>
> > config.processor(ExampleStreamProcessor.class);
> > > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > > >> > > StringDeserializer());
> > > >> > > > > > >> > > > KafkaStreaming
> > > >> > > > > > >> > > > > >> > >>>>>> container =
> > > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > > >> > container.run();
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > > >> > > > SamzaContainer;
> > > >> > > > > > >> > > > > StreamProcessor
> > > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the
> > class
> > > >> names
> > > >> > > in
> > > >> > > > a
> > > >> > > > > > file
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > then
> > > >> > > > > > >> > > > > >> > >>>>>>>> having
> > > >> > > > > > >> > > > > >> > >>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you
> > just
> > > >> > > > > instantiate
> > > >> > > > > > the
> > > >> > > > > > >> > > > > container
> > > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is
> balanced
> > > over
> > > >> > > > however
> > > >> > > > > > many
> > > >> > > > > > >> > > > > instances
> > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>> this
> > > >> > > > > > >> > > > > >> > >>>>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> > > instance
> > > >> > dies,
> > > >> > > > new
> > > >> > > > > > >> tasks
> > > >> > > > > > >> > > are
> > > >> > > > > > >> > > > > >> added
> > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
> > shutting
> > > >> them
> > > >> > > > down).
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
> > > running
> > > >> > this
> > > >> > > > > stuff
> > > >> > > > > > in
> > > >> > > > > > >> > YARN
> > > >> > > > > > >> > > > via
> > > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and
> AWS
> > > >> using
> > > >> > > some
> > > >> > > > > of
> > > >> > > > > > >> their
> > > >> > > > > > >> > > > tools
> > > >> > > > > > >> > > > > >> > >>>>>>>> but from the
> > > >> > > > > > >> > > > > >> > >>>>>> point
> > > >> > > > > > >> > > > > >> > >>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these
> > stream
> > > >> > > > processing
> > > >> > > > > > jobs
> > > >> > > > > > >> > are
> > > >> > > > > > >> > > > > just
> > > >> > > > > > >> > > > > >> > >>>>>> stateless
> > > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
> > > >> expand
> > > >> > and
> > > >> > > > > > contract
> > > >> > > > > > >> > at
> > > >> > > > > > >> > > > > will.
> > > >> > > > > > >> > > > > >> > >>>>>>>> There
> > > >> > > > > > >> > > > > >> > >>>>> is
> > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of
> code,
> > > it
> > > >> > would
> > > >> > > > get
> > > >> > > > > > >> larger
> > > >> > > > > > >> > > if
> > > >> > > > > > >> > > > we
> > > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
> > > larger.
> > > >> We
> > > >> > > > really
> > > >> > > > > > do
> > > >> > > > > > >> > get a
> > > >> > > > > > >> > > > ton
> > > >> > > > > > >> > > > > >> > >>>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>> leverage
> > > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> > > >> > delegated
> > > >> > > to
> > > >> > > > > the
> > > >> > > > > > >> new
> > > >> > > > > > >> > > > > >> consumer.
> > > >> > > > > > >> > > > > >> > >>>>> This
> > > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> > > >> > management
> > > >> > > > > > strategy
> > > >> > > > > > >> > > > > available
> > > >> > > > > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to
> > Samza
> > > >> (and
> > > >> > > vice
> > > >> > > > > > versa)
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > with
> > > >> > > > > > >> > > > > >> > >>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>> exact
> > > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as
> > > state
> > > >> > reuse
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it
> is
> > > >> > thought
> > > >> > > > > > >> provoking.
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM,
> > > Chris
> > > >> > > > > Riccomini <
> > > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with
> > > Samza
> > > >> > > > > engineers
> > > >> > > > > > at
> > > >> > > > > > >> > > > LinkedIn
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>> Confluent
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> > > observations
> > > >> > and
> > > >> > > > > would
> > > >> > > > > > >> like
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > >> > >>>>>>>>> propose
> > > >> > > > > > >> > > > > >> > >>>>> some
> > > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that
> I
> > > >> want to
> > > >> > > > call
> > > >> > > > > > out
> > > >> > > > > > >> > about
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > > >> > > > > > >> > > > > >> > >>>>>> design,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> > > changes.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a
> > dynamic
> > > >> > > > deployment
> > > >> > > > > > >> system.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> > > >> SystemConsumer/SystemProducer
> > > >> > and
> > > >> > > > > > Kafka's
> > > >> > > > > > >> > > > consumer
> > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > >> > > > > > >> > > > > >> > >>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the
> same
> > > >> > problems.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
> > > related,
> > > >> > but
> > > >> > > > I'll
> > > >> > > > > > >> > address
> > > >> > > > > > >> > > > them
> > > >> > > > > > >> > > > > >> in
> > > >> > > > > > >> > > > > >> > >>>>> order.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the
> use
> > > of a
> > > >> > > > dynamic
> > > >> > > > > > >> > > deployment
> > > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > > >> > > > > > >> > > > > >> > >>>>>> such
> > > >> > > > > > >> > > > > >> > >>>>>>>>> as
> > > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
> > initially
> > > >> built
> > > >> > > > > Samza,
> > > >> > > > > > we
> > > >> > > > > > >> > bet
> > > >> > > > > > >> > > > that
> > > >> > > > > > >> > > > > >> > >>>>>>>>> there
> > > >> > > > > > >> > > > > >> > >>>>>> would
> > > >> > > > > > >> > > > > >> > >>>>>>>>> be
> > > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area,
> > and
> > > >> we
> > > >> > > could
> > > >> > > > > > >> support
> > > >> > > > > > >> > > > them,
> > > >> > > > > > >> > > > > >> and
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>> rest
> > > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there
> > are
> > > >> many
> > > >> > > > > > >> variations.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > > >> > > > > > >> > > > > >> > >>>>>> many
> > > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just
> start
> > > >> their
> > > >> > > > > > processors
> > > >> > > > > > >> > like
> > > >> > > > > > >> > > > > normal
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
> > traditional
> > > >> > > > deployment
> > > >> > > > > > >> scripts
> > > >> > > > > > >> > > > such
> > > >> > > > > > >> > > > > as
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > > >> > > > > > >> > > > > >> > >>>>>> Chef,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a
> deployment
> > > >> system
> > > >> > > on
> > > >> > > > > > users
> > > >> > > > > > >> > makes
> > > >> > > > > > >> > > > the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
> > > painful
> > > >> for
> > > >> > > > first
> > > >> > > > > > time
> > > >> > > > > > >> > > > users.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
> > requirement
> > > >> was
> > > >> > > also
> > > >> > > > a
> > > >> > > > > > bit
> > > >> > > > > > >> of
> > > >> > > > > > >> > a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > > >> > > > > > >> > > > > >> > >>>>>> because
> > > >> > > > > > >> > > > > >> > >>>>>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding
> > > between
> > > >> > the
> > > >> > > > > > nature of
> > > >> > > > > > >> > > batch
> > > >> > > > > > >> > > > > >> jobs
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>> stream
> > > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we
> made
> > > >> > > conscious
> > > >> > > > > > effort
> > > >> > > > > > >> to
> > > >> > > > > > >> > > > favor
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>> Hadoop
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing
> things,
> > > >> since
> > > >> > it
> > > >> > > > > worked
> > > >> > > > > > >> and
> > > >> > > > > > >> > > was
> > > >> > > > > > >> > > > > well
> > > >> > > > > > >> > > > > >> > >>>>>>> understood.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was
> that
> > > >> batch
> > > >> > > jobs
> > > >> > > > > > have a
> > > >> > > > > > >> > > > definite
> > > >> > > > > > >> > > > > >> > >>>>>> beginning,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs
> > > don't
> > > >> > > > > (usually).
> > > >> > > > > > >> This
> > > >> > > > > > >> > > > leads
> > > >> > > > > > >> > > > > to
> > > >> > > > > > >> > > > > >> > >>>>>>>>> a
> > > >> > > > > > >> > > > > >> > >>>>> much
> > > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for
> > > stream
> > > >> > > > > processors.
> > > >> > > > > > >> You
> > > >> > > > > > >> > > > > >> basically
> > > >> > > > > > >> > > > > >> > >>>>>>>>> just
> > > >> > > > > > >> > > > > >> > >>>>>>> need
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> > > >> processor,
> > > >> > and
> > > >> > > > > start
> > > >> > > > > > >> it.
> > > >> > > > > > >> > > The
> > > >> > > > > > >> > > > > way
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn,
> there's
> > > no
> > > >> > > concept
> > > >> > > > > of
> > > >> > > > > > a
> > > >> > > > > > >> > > cluster
> > > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > > >> > > > > > >> > > > > >> > >>>>>> add
> > > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> > > >> coupling
> > > >> > > > Samza
> > > >> > > > > > with
> > > >> > > > > > >> a
> > > >> > > > > > >> > > > > >> scheduler
> > > >> > > > > > >> > > > > >> > >>>>>>>>> is
> > > >> > > > > > >> > > > > >> > >>>>>> that
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has
> to
> > > >> handle
> > > >> > > > > > deployment.
> > > >> > > > > > >> > > This
> > > >> > > > > > >> > > > > >> pulls
> > > >> > > > > > >> > > > > >> > >>>>>>>>> in a
> > > >> > > > > > >> > > > > >> > >>>>>>> bunch
> > > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> > > >> > > distribution
> > > >> > > > > > (config
> > > >> > > > > > >> > > > > stream),
> > > >> > > > > > >> > > > > >> > >>>>>>>>> shell
> > > >> > > > > > >> > > > > >> > >>>>>>> scrips
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> > > packaging
> > > >> > (all
> > > >> > > > the
> > > >> > > > > > .tgz
> > > >> > > > > > >> > > > stuff),
> > > >> > > > > > >> > > > > >> etc.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring
> > dynamic
> > > >> > > > deployment
> > > >> > > > > > was
> > > >> > > > > > >> to
> > > >> > > > > > >> > > > > support
> > > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to
> have
> > > >> > > locality,
> > > >> > > > > you
> > > >> > > > > > >> need
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > put
> > > >> > > > > > >> > > > > >> > >>>>>>>>> your
> > > >> > > > > > >> > > > > >> > >>>>>> processors
> > > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> > > processing.
> > > >> > Upon
> > > >> > > > > > further
> > > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > > >> > > > > > >> > > > > >> > >>>>>>> though,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
> > beneficial.
> > > >> > There
> > > >> > > is
> > > >> > > > > > some
> > > >> > > > > > >> > good
> > > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > > >> > > > > > >> > > > > >> > >>>>>> about
> > > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
> > SAMZA-335.
> > > >> > Again,
> > > >> > > we
> > > >> > > > > > took
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > > >> > > > > > >> > > > > >> > >>>>>> path,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> > > differences
> > > >> > > > between
> > > >> > > > > > HDFS
> > > >> > > > > > >> > and
> > > >> > > > > > >> > > > > Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > > >> > > > > > >> > > > > >> > >>>>>> has
> > > >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has
> > partitions.
> > > >> This
> > > >> > > > leads
> > > >> > > > > to
> > > >> > > > > > >> less
> > > >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with
> stream
> > > >> > > processors
> > > >> > > > > on
> > > >> > > > > > top
> > > >> > > > > > >> > of
> > > >> > > > > > >> > > > > Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a
> > > crutch.
> > > >> > > Samza
> > > >> > > > > > doesn't
> > > >> > > > > > >> > > have
> > > >> > > > > > >> > > > > any
> > > >> > > > > > >> > > > > >> > >>>>>>>>> built
> > > >> > > > > > >> > > > > >> > >>>>> in
> > > >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead,
> it
> > > >> > depends
> > > >> > > on
> > > >> > > > > the
> > > >> > > > > > >> > > dynamic
> > > >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to
> > > handle
> > > >> > > > restarts
> > > >> > > > > > >> when a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > > >> > > > > > >> > > > > >> > >>>>>>> made
> > > >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> > > >> standalone
> > > >> > > Samza
> > > >> > > > > > >> > container
> > > >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is
> > good,
> > > >> but I
> > > >> > > > think
> > > >> > > > > > that
> > > >> > > > > > >> > > we've
> > > >> > > > > > >> > > > > >> gone
> > > >> > > > > > >> > > > > >> > >>>>>>>>> too
> > > >> > > > > > >> > > > > >> > >>>>>> far
> > > >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > > >> > > > (SystemConsumer,
> > > >> > > > > > >> > > > > SystemProducer,
> > > >> > > > > > >> > > > > >> > >>>> etc).
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just
> > > about
> > > >> > every
> > > >> > > > > > >> component
> > > >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> > > >> > > ConfigRewriter,
> > > >> > > > > > etc).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> > > >> > forgotten,
> > > >> > > as
> > > >> > > > > > well.
> > > >> > > > > > >> > Some
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > >>>>>>>>> these
> > > >> > > > > > >> > > > > >> > >>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not
> > to
> > > >> be.
> > > >> > > This
> > > >> > > > > all
> > > >> > > > > > >> comes
> > > >> > > > > > >> > > at
> > > >> > > > > > >> > > > a
> > > >> > > > > > >> > > > > >> cost:
> > > >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is
> > > making
> > > >> it
> > > >> > > > harder
> > > >> > > > > > for
> > > >> > > > > > >> > our
> > > >> > > > > > >> > > > > users
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>> pick
> > > >> > > > > > >> > > > > >> > >>>>>> up
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It
> > > also
> > > >> > makes
> > > >> > > > it
> > > >> > > > > > >> > difficult
> > > >> > > > > > >> > > > for
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about
> > > what
> > > >> the
> > > >> > > > > > >> > > characteristics
> > > >> > > > > > >> > > > of
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> > > >> characteristics
> > > >> > > > change
> > > >> > > > > > >> > > depending
> > > >> > > > > > >> > > > on
> > > >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are
> > > most
> > > >> > > visible
> > > >> > > > > in
> > > >> > > > > > the
> > > >> > > > > > >> > > > System
> > > >> > > > > > >> > > > > >> APIs.
> > > >> > > > > > >> > > > > >> > >>>>> What
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
> > > >> functional is
> > > >> > > > Kafka
> > > >> > > > > > as
> > > >> > > > > > >> its
> > > >> > > > > > >> > > > > >> > >>>>>>>>> transport
> > > >> > > > > > >> > > > > >> > >>>>>> layer.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> But
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated
> use
> > > >> cases
> > > >> > > into
> > > >> > > > > one
> > > >> > > > > > >> API:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports
> > both
> > > >> of
> > > >> > > these
> > > >> > > > > use
> > > >> > > > > > >> > cases.
> > > >> > > > > > >> > > > The
> > > >> > > > > > >> > > > > >> > >>>>>>>>> problem
> > > >> > > > > > >> > > > > >> > >>>>>> is,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we
> > > >> > > > > > >> > > > > >> > >>>>>>>>> actually want different features
> > for
> > > >> each
> > > >> > > use
> > > >> > > > > > case.
> > > >> > > > > > >> By
> > > >> > > > > > >> > > > > >> papering
> > > >> > > > > > >> > > > > >> > >>>>>>>>> over
> > > >> > > > > > >> > > > > >> > >>>>>>> these
> > > >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a
> > > single
> > > >> > API,
> > > >> > > > > we've
> > > >> > > > > > >> > > > introduced
> > > >> > > > > > >> > > > > a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > >> > > > > > >> > > > > >> > >>>>>>> leaky
> > > >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really
> like
> > > in
> > > >> (2)
> > > >> > > is
> > > >> > > > to
> > > >> > > > > > have
> > > >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs
> for
> > > >> > offsets
> > > >> > > > > (like
> > > >> > > > > > >> > Kafka).
> > > >> > > > > > >> > > > > This
> > > >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > > >> > > > > > >> > > > > >> > >>>>> with
> > > >> > > > > > >> > > > > >> > >>>>>>> (1),
> > > >> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems
> > have
> > > >> > > > different
> > > >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> > > >> mailing
> > > >> > > list
> > > >> > > > > and
> > > >> > > > > > >> the
> > > >> > > > > > >> > > SQL
> > > >> > > > > > >> > > > > >> JIRAs
> > > >> > > > > > >> > > > > >> > >>>>> about
> > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> > > >> > > replayability.
> > > >> > > > > > Kafka
> > > >> > > > > > >> > > allows
> > > >> > > > > > >> > > > us
> > > >> > > > > > >> > > > > >> to
> > > >> > > > > > >> > > > > >> > >>>>> rewind
> > > >> > > > > > >> > > > > >> > >>>>>>>>> when
> > > >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other
> > > systems
> > > >> > > don't.
> > > >> > > > In
> > > >> > > > > > some
> > > >> > > > > > >> > > > cases,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > >> > > > > > >> > > > > >> > >>>>>>> return
> > > >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > > >> > > > > > >> WikipediaSystemConsumer)
> > > >> > > > > > >> > > > > because
> > > >> > > > > > >> > > > > >> > >>>>>>>>> they
> > > >> > > > > > >> > > > > >> > >>>>>> have
> > > >> > > > > > >> > > > > >> > >>>>>>> no
> > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example.
> > > Kafka
> > > >> > > > supports
> > > >> > > > > > >> > > > > partitioning,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > > >> > > > > > >> > > > > >> > >>>>> many
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
> > > >> having a
> > > >> > > > single
> > > >> > > > > > >> > > partition
> > > >> > > > > > >> > > > > for
> > > >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other
> > systems
> > > >> model
> > > >> > > > > > >> partitioning
> > > >> > > > > > >> > > > > >> > >>>> differently (e.g.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is
> also
> > a
> > > >> mess.
> > > >> > > > > > Creating
> > > >> > > > > > >> > > streams
> > > >> > > > > > >> > > > > in
> > > >> > > > > > >> > > > > >> a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> > > >> impossible.
> > > >> > > As
> > > >> > > > is
> > > >> > > > > > >> > modeling
> > > >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > > >> > > > > > >> > > > > >> > >>>>> for
> > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> > > >> partitions,
> > > >> > > > > location,
> > > >> > > > > > >> > etc).
> > > >> > > > > > >> > > > The
> > > >> > > > > > >> > > > > >> > >>>>>>>>> list
> > > >> > > > > > >> > > > > >> > >>>>> goes
> > > >> > > > > > >> > > > > >> > >>>>>>> on.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began
> writing
> > > >> Samza,
> > > >> > > > > Kafka's
> > > >> > > > > > >> > > consumer
> > > >> > > > > > >> > > > > and
> > > >> > > > > > >> > > > > >> > >>>>> producer
> > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature
> set.
> > > On
> > > >> the
> > > >> > > > > > >> > consumer-side,
> > > >> > > > > > >> > > > you
> > > >> > > > > > >> > > > > >> > >>>>>>>>> had two
> > > >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
> > > consumer,
> > > >> or
> > > >> > > the
> > > >> > > > > > simple
> > > >> > > > > > >> > > > > consumer.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> The
> > > >> > > > > > >> > > > > >> > >>>>>>> problem
> > > >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was
> > > that
> > > >> it
> > > >> > > > > > controlled
> > > >> > > > > > >> > your
> > > >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments,
> > and
> > > >> the
> > > >> > > order
> > > >> > > > > in
> > > >> > > > > > >> which
> > > >> > > > > > >> > > you
> > > >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > > >> > > > > > >> > > > > >> > >>>>> problem
> > > >> > > > > > >> > > > > >> > >>>>>>>>> with
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's
> > not
> > > >> > > simple.
> > > >> > > > > It's
> > > >> > > > > > >> > basic.
> > > >> > > > > > >> > > > You
> > > >> > > > > > >> > > > > >> > >>>>>>>>> end up
> > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really
> > low-level
> > > >> stuff
> > > >> > > > that
> > > >> > > > > > you
> > > >> > > > > > >> > > > > shouldn't.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> We
> > > >> > > > > > >> > > > > >> > >>>>>> spent a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > > >> > > > KafkaSystemConsumer
> > > >> > > > > > very
> > > >> > > > > > >> > > > robust.
> > > >> > > > > > >> > > > > >> It
> > > >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some
> > cool
> > > >> > > features:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering
> > and
> > > >> > > > > > prioritization.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> > > >> assignment
> > > >> > > to
> > > >> > > > > > support
> > > >> > > > > > >> > > > joins,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> global
> > > >> > > > > > >> > > > > >> > >>>>>> state
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)),
> > > etc.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> > > >> > checkpointing.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the
> time
> > > is
> > > >> > that
> > > >> > > > > these
> > > >> > > > > > >> > > features
> > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > >> > > > > > >> > > > > >> > >>>>>>> actually
> > > >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
> > > consumers
> > > >> > (not
> > > >> > > > just
> > > >> > > > > > >> Samza
> > > >> > > > > > >> > > > stream
> > > >> > > > > > >> > > > > >> > >>>>>> processors)
> > > >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like
> > > joins
> > > >> > and
> > > >> > > > > > partition
> > > >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> > > >> > conclusion.
> > > >> > > > > > They're
> > > >> > > > > > >> > > adding
> > > >> > > > > > >> > > > a
> > > >> > > > > > >> > > > > >> ton
> > > >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> > > >> consumer
> > > >> > > > > > >> > > implementation.
> > > >> > > > > > >> > > > > To a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > > >> > > > > > >> > > > > >> > >>>>> it's
> > > >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've
> > already
> > > >> done
> > > >> > > in
> > > >> > > > > > Samza.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up
> > > taking
> > > >> a
> > > >> > > very
> > > >> > > > > > similar
> > > >> > > > > > >> > > > > approach
> > > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > > >> > > > > > >> > > > > >> > >>>>>> Samza's
> > > >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> > > implementation
> > > >> for
> > > >> > > > > > handling
> > > >> > > > > > >> > > offset
> > > >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> > > >> management
> > > >> > > > > feature
> > > >> > > > > > >> > stores
> > > >> > > > > > >> > > > > >> offset
> > > >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and
> allows
> > > >> you to
> > > >> > > > fetch
> > > >> > > > > > them
> > > >> > > > > > >> > > from
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a
> waste,
> > > >> since
> > > >> > we
> > > >> > > > > could
> > > >> > > > > > >> have
> > > >> > > > > > >> > > > shared
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>> work
> > > >> > > > > > >> > > > > >> > >>>>>> if
> > > >> > > > > > >> > > > > >> > >>>>>>>>> it
> > > >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
> > > >> get-go.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> > > >> radical
> > > >> > > > > > proposal.
> > > >> > > > > > >> > Samza
> > > >> > > > > > >> > > > is
> > > >> > > > > > >> > > > > >> > >>>>> relatively
> > > >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd
> venture
> > to
> > > >> say
> > > >> > > that
> > > >> > > > > > we're
> > > >> > > > > > >> > > near a
> > > >> > > > > > >> > > > > 1.0
> > > >> > > > > > >> > > > > >> > >>>>>> release.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > > >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take
> what
> > > >> we've
> > > >> > > > > learned,
> > > >> > > > > > and
> > > >> > > > > > >> > > begin
> > > >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > > >> > > > > > >> > > > > >> > >>>>>>> about
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we
> > > >> change if
> > > >> > > we
> > > >> > > > > were
> > > >> > > > > > >> > > starting
> > > >> > > > > > >> > > > > >> from
> > > >> > > > > > >> > > > > >> > >>>>>> scratch?
> > > >> > > > > > >> > > > > >> > >>>>>>>>> My
> > > >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the
> > *only*
> > > >> way
> > > >> > to
> > > >> > > > run
> > > >> > > > > > Samza
> > > >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all
> > direct
> > > >> > > > > dependences
> > > >> > > > > > on
> > > >> > > > > > >> > > YARN,
> > > >> > > > > > >> > > > > >> Mesos,
> > > >> > > > > > >> > > > > >> > >>>> etc.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to
> > support
> > > >> only
> > > >> > > > Kafka
> > > >> > > > > > as
> > > >> > > > > > >> the
> > > >> > > > > > >> > > > > stream
> > > >> > > > > > >> > > > > >> > >>>>>> processing
> > > >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics,
> > > logging,
> > > >> > > > > > >> serialization,
> > > >> > > > > > >> > > and
> > > >> > > > > > >> > > > > >> > >>>>>>>>> config
> > > >> > > > > > >> > > > > >> > >>>>>>> systems,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues
> > > that
> > > >> I
> > > >> > > > > outlined
> > > >> > > > > > >> > above.
> > > >> > > > > > >> > > It
> > > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > > >> > > > > > >> > > > > >> > >>>>> also
> > > >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base
> pretty
> > > >> > > > dramatically.
> > > >> > > > > > >> > > Supporting
> > > >> > > > > > >> > > > > >> only
> > > >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will
> allow
> > > >> Samza
> > > >> > to
> > > >> > > be
> > > >> > > > > > >> executed
> > > >> > > > > > >> > > on
> > > >> > > > > > >> > > > > YARN
> > > >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > > >> > > > Marathon/Aurora),
> > > >> > > > > or
> > > >> > > > > > >> most
> > > >> > > > > > >> > > > other
> > > >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > > >> > > > > > >> > > > > >> > >>>>>>> deployment
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a
> > lot
> > > >> > easier
> > > >> > > > for
> > > >> > > > > > new
> > > >> > > > > > >> > > users.
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > > >> > > > > > >> > > > > >> > >>>>>>> having
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without
> > > YARN.
> > > >> > The
> > > >> > > > drop
> > > >> > > > > > in
> > > >> > > > > > >> > > mailing
> > > >> > > > > > >> > > > > >> list
> > > >> > > > > > >> > > > > >> > >>>>>> traffic
> > > >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
> > > >> overdue to
> > > >> > > me.
> > > >> > > > > The
> > > >> > > > > > >> > > reality
> > > >> > > > > > >> > > > > is,
> > > >> > > > > > >> > > > > >> > >>>>> everyone
> > > >> > > > > > >> > > > > >> > >>>>>>>>> that
> > > >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
> > > >> Kafka.
> > > >> > We
> > > >> > > > > > basically
> > > >> > > > > > >> > > > require
> > > >> > > > > > >> > > > > >> it
> > > >> > > > > > >> > > > > >> > >>>>>> already
> > > >> > > > > > >> > > > > >> > >>>>>>> in
> > > >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work.
> > > Those
> > > >> > that
> > > >> > > > are
> > > >> > > > > > >> using
> > > >> > > > > > >> > > > other
> > > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > >> > > > > > >> > > > > >> > >>>>>> are
> > > >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest
> into
> > > >> Kafka
> > > >> > > (1),
> > > >> > > > > and
> > > >> > > > > > >> then
> > > >> > > > > > >> > > > they
> > > >> > > > > > >> > > > > do
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
> > > >> already
> > > >> > > > > > discussion (
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> >
> > > >> > > > > >
> > > >> > >
> > > >>
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > >> > > > > > >> > > > > >> > >>>>> 767
> > > >> > > > > > >> > > > > >> > >>>>>>>>> )
> > > >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into
> > > Kafka
> > > >> > > > extremely
> > > >> > > > > > >> easy.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple
> > with
> > > >> > Kafka,
> > > >> > > > we
> > > >> > > > > > can
> > > >> > > > > > >> > > > leverage
> > > >> > > > > > >> > > > > a
> > > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > >> > > > > > >> > > > > >> > >>>>>>> their
> > > >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> > > >> maintain
> > > >> > > our
> > > >> > > > > own
> > > >> > > > > > >> > config,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > > >> > > > > > >> > > > > >> > >>>>> etc.
> > > >> > > > > > >> > > > > >> > >>>>>>> We
> > > >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same
> libraries,
> > > and
> > > >> > make
> > > >> > > > them
> > > >> > > > > > >> > better.
> > > >> > > > > > >> > > > This
> > > >> > > > > > >> > > > > >> > >>>>>>>>> will
> > > >> > > > > > >> > > > > >> > >>>>> also
> > > >> > > > > > >> > > > > >> > >>>>>>>>> allow us to share the
> > > >> consumer/producer
> > > >> > > APIs,
> > > >> > > > > and
> > > >> > > > > > >> will
> > > >> > > > > > >> > > let
> > > >> > > > > > >> > > > > us
> > > >> > > > > > >> > > > > >> > >>>>> leverage
> > > >> > > > > > >> > > > > >> > >>>>>>>>> their offset management and
> > > partition
> > > >> > > > > management,
> > > >> > > > > > >> > rather
> > > >> > > > > > >> > > > > than
> > > >> > > > > > >> > > > > >> > >>>>>>>>> having
> > > >> > > > > > >> > > > > >> > >>>>>> our
> > > >> > > > > > >> > > > > >> > >>>>>>>>> own. All of the coordinator
> stream
> > > >> code
> > > >> > > would
> > > >> > > > > go
> > > >> > > > > > >> away,
> > > >> > > > > > >> > > as
> > > >> > > > > > >> > > > > >> would
> > > >> > > > > > >> > > > > >> > >>>>>>>>> most
> > > >> > > > > > >> > > > > >> > >>>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd
> probably
> > > >> have
> > > >> > to
> > > >> > > > push
> > > >> > > > > > some
> > > >> > > > > > >> > > > > partition
> > > >> > > > > > >> > > > > >> > >>>>>>> management
> > > >> > > > > > >> > > > > >> > >>>>>>>>> features into the Kafka broker,
> > but
> > > >> > they're
> > > >> > > > > > already
> > > >> > > > > > >> > > moving
> > > >> > > > > > >> > > > > in
> > > >> > > > > > >> > > > > >> > >>>>>>>>> that direction with the new
> > consumer
> > > >> API.
> > > >> > > The
> > > >> > > > > > >> features
> > > >> > > > > > >> > > we
> > > >> > > > > > >> > > > > have
> > > >> > > > > > >> > > > > >> > >>>>>>>>> for
> > > >> > > > > > >> > > > > >> > >>>>>> partition
> > > >> > > > > > >> > > > > >> > >>>>>>>>> assignment aren't unique to
> Samza,
> > > and
> > > >> > seem
> > > >> > > > > like
> > > >> > > > > > >> they
> > > >> > > > > > >> > > > should
> > > >> > > > > > >> > > > > >> be
> > > >> > > > > > >> > > > > >> > >>>>>>>>> in
> > > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>>> anyway. There will always be
> some
> > > >> niche
> > > >> > > > usages
> > > >> > > > > > which
> > > >> > > > > > >> > > will
> > > >> > > > > > >> > > > > >> > >>>>>>>>> require
> > > >> > > > > > >> > > > > >> > >>>>>> extra
> > > >> > > > > > >> > > > > >> > >>>>>>>>> care and hence full control over
> > > >> > partition
> > > >> > > > > > >> assignments
> > > >> > > > > > >> > > > much
> > > >> > > > > > >> > > > > >> > >>>>>>>>> like the
> > > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > > >> > > > > > >> > > > > >> > >>>>>>>>> low level consumer api. These
> > would
> > > >> > > continue
> > > >> > > > to
> > > >> > > > > > be
> > > >> > > > > > >> > > > > supported.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> These items will be good for the
> > > Samza
> > > >> > > > > community.
> > > >> > > > > > >> > > They'll
> > > >> > > > > > >> > > > > make
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it
> > > >> easier
> > > >> > for
> > > >> > > > > > >> developers
> > > >> > > > > > >> > > to
> > > >> > > > > > >> > > > > add
> > > >> > > > > > >> > > > > >> > >>>>>>>>> new features.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large
> > > (and
> > > >> > > > somewhat
> > > >> > > > > > >> > backwards
> > > >> > > > > > >> > > > > >> > >>>>> incompatible
> > > >> > > > > > >> > > > > >> > >>>>>>>>> change). If we choose to go this
> > > >> route,
> > > >> > > it's
> > > >> > > > > > >> important
> > > >> > > > > > >> > > > that
> > > >> > > > > > >> > > > > we
> > > >> > > > > > >> > > > > >> > >>>>> openly
> > > >> > > > > > >> > > > > >> > >>>>>>>>> communicate how we're going to
> > > >> provide a
> > > >> > > > > > migration
> > > >> > > > > > >> > path
> > > >> > > > > > >> > > > from
> > > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>> existing
> > > >> > > > > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make
> > > >> > > incompatible
> > > >> > > > > > >> > changes).
> > > >> > > > > > >> > > I
> > > >> > > > > > >> > > > > >> think
> > > >> > > > > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need
> > to
> > > >> > > provide a
> > > >> > > > > > >> wrapper
> > > >> > > > > > >> > to
> > > >> > > > > > >> > > > > allow
> > > >> > > > > > >> > > > > >> > >>>>>>>>> existing StreamTask
> > implementations
> > > to
> > > >> > > > continue
> > > >> > > > > > >> > running
> > > >> > > > > > >> > > on
> > > >> > > > > > >> > > > > the
> > > >> > > > > > >> > > > > >> > >>>> new container.
> > > >> > > > > > >> > > > > >> > >>>>>>> It's
> > > >> > > > > > >> > > > > >> > >>>>>>>>> also important that we openly
> > > >> communicate
> > > >> > > > about
> > > >> > > > > > >> > timing,
> > > >> > > > > > >> > > > and
> > > >> > > > > > >> > > > > >> > >>>>>>>>> stages
> > > >> > > > > > >> > > > > >> > >>>>> of
> > > >> > > > > > >> > > > > >> > >>>>>>> the
> > > >> > > > > > >> > > > > >> > >>>>>>>>> migration.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm
> sure
> > > you
> > > >> > have
> > > >> > > > > > opinions.
> > > >> > > > > > >> > :)
> > > >> > > > > > >> > > > > Please
> > > >> > > > > > >> > > > > >> > >>>>>>>>> send
> > > >> > > > > > >> > > > > >> > >>>>>> your
> > > >> > > > > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Cheers,
> > > >> > > > > > >> > > > > >> > >>>>>>>>> Chris
> > > >> > > > > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>> --
> > > >> > > > > > >> > > > > >> > >>>>>> -- Guozhang
> > > >> > > > > > >> > > > > >> > >>>>>>
> > > >> > > > > > >> > > > > >> > >>>>>
> > > >> > > > > > >> > > > > >> > >>>>
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> > >>
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >> >
> > > >> > > > > > >> > > > > >>
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > > >
> > > >> > > > > > >> > > > >
> > > >> > > > > > >> > > >
> > > >> > > > > > >> > >
> > > >> > > > > > >> >
> > > >> > > > > > >>
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>



-- 
Jordan Shaw
Full Stack Software Engineer
PubNub Inc
1045 17th St
San Francisco, CA 94107

RE: Thoughts and obesrvations on Samza

Posted by Kartik Paramasivam <kp...@linkedin.com.INVALID>.
I think there are some details here that we should discuss more. 

It will be good to define what we mean by "Stream processing as a service"  (like Yi mentions in his mail).

Stream processing as a service is very much in the likes of AWS Lambda, Google Cloud DataFlow and Azure StreamAnalytics.
Each of these services in essence do some sliver of stream processing and have their own areas of focus.

In LinkedIn we think of the following things when we think of "stream processing as a service" :  

1. a common way to manage, deploy, monitor Samza jobs in such a way that samza users at LinkedIn don't have to know about details of Yarn or our cluster management software of choice. This would include:
    - A common dashboard that lists the samza jobs in the cluster
    - Ability to start/stop individual jobs from the dashboard.
    - Ability to look at metrics and logs for the job without having to worry about which machine in the cluster the job is running. (for this we pipe logs into ELK)
    - Change configuration for a samza job  - coordinator stream (and even the broadcast stream) work that we have been doing helps towards this goal.
    - support for changing configuration for maybe only 10% of the containers (canarying)
    - etc.
    
Some of this today here at LinkedIn is very specific to what we do, but it should be possible for us to generalize a lot of this over time.

2. Partition distribution is an important part of both Samza as a library and Samza as a service.

Partition management that exists in Kafka as of right now would satisfy most of the stateless scenarios.   As far as the stateful scenarios is concerned, there are big gaps in Kafka partition distribution as of today. Many of these can be fixed over time, but not all.   Here are some interesting scenarios pertaining partition distribution :
a. Ensuring that a single consumer joining or leaving the cluster doesn't fully rebalance all partitions.  This is a fixable problem in Kafka partition distribution. 
b. To perform any meaningful join semantics between streams we need co-partitioning for the consumer.  I imagine that kafka could provide this in the future.
c. Host affinity and stickiness is another thing that doesn't exist in Kafka today.. and I believe the new consumer can be used with appropriate configuration to solve this.
d. We do have situations where the input streams come from different kafka clusters (e.g. profile changes come in from a kafka cluster dedicated to our database change capture (espresso)) whereas the primary events come from a different kafka cluster (tracking/queuing clusters).  Moving partition distribution to Kafka doesn't work in this situation.  Moving data between clusters doubles costs.
e. This is not common, but every so often there are scenarios where some partitions are hot, in such a situation having more control over redistributing partitions across containers is beneficial.

All in all, at LinkedIn we expect to do custom partition distribution in our Stream processing as a service (Samza) offering for a good amount of time to come. As and when new partition distribution features come up and become stable in Kafka we would surely move to using them wherever possible.

3.  "auto-scaling" the number of Samza containers.
We currently have an intern who is trying to perform auto-scaling based on memory and partition lag in Samza.
In general it should be easy to add containers with Slider or such. However, it is important that partitions get moved to this new container slowly and carefully in the case of stateful jobs.
Basically we don't want a large numbers of partitions to get moved to this new container in one shot as it would cause reseeding of all the local state. Instead we would want to add a few partitions at a time and over time get it to a place where the partitions are evenly distributed across the containers.

4. Samza customers at LinkedIn really don't like to keep upgrading Samza in their jobs. They only care about upgrading their own stream processor code.
Samza as a product is young and Samza as a library is even more in its infancy. One of the attractions of running as a service is that we don't have to make our users think about upgrading Samza all the time.  We can upgrade Samza when we have a blessed version of Samza.    If the library was super-mature and samza users only have to upgrade the library once in a year then then it would be a different story.  

5. There are some scenarios e.g. SQL on streams where we need a dashboard where a query can be submitted and like Yi explains the query optimizer would figure out the number of containers to run and then the deployer would then materialize the query.  This is clearly something that requires close integration of stream processing with the cluster management software.  This would also be one of the things that the Stream processing service would focus on in the future.  I don't believe this is something that can be done in a library with cluster wide knowledge.

With the above explanation of "stream processing as a service" in mind, the question that Yi and I were grappling with last week is : what would be the point of deprecating the current Samza code base in the open source and forking it and continuing to use it and enhance it to run Stream processing as a service at LinkedIn.   Wouldn't it make more sense to keep developing Stream Processing as a Service in the open source.  This discussion has made us realize that clearly the answer has to be yes.

Now the question is where does this functionality go.

It is easier to think of a future where there are only 2 projects : Kafka and KafkaStreams (or whatever KafkaXX name we come up with) ,  
as opposed to 3 projects:  Kafka,  KafkaStreams ( core stream processor library) and say Samza (which becomes "stream processing as a service" built on top of KafkaStreams).
If we have 3 projects, then we are now making  the Samza community worry about two projects "Samza" and also "KafkaStreams".  How will that work in terms of the mailing lists etc.

Is there a way to avoid this.

If we put "stream processing as a service" functionality into KafkaStreams then that is same as Samza 2.0 (borrowed from Jacob's list).  It would be everything that Samza has today (- non kafka consumers) with a heavy focus on the library aspects.   Will this pollute our original goal of a thin stream processing library ?

I wonder if there is another middle ground where we put the really core library (similar to Jay's prototype) in Kafka and work on Samza 2.0 which leverages these new Kafka apis to deliver Stream processing as a service. In this world, all the SQL stuff and the operator stuff would be in Samza 2.0.  For e.g. Would it be possible here to have some of the Samza committers be able to commit to Kafka but only to this new library part of it ?

Thoughts?

Kartik
________________________________________
From: Jay Kreps [jay.kreps@gmail.com]
Sent: Sunday, July 12, 2015 8:57 PM
To: dev@samza.apache.org
Subject: Re: Thoughts and obesrvations on Samza

Hey guys,

There seems to be some confusion in the last few emails: there is no plan
whatsoever to remove YARN support. The change suggested was to move the
partition management out of the YARN app master and rely on Kafka's
partition management. The advantage of this would be to make the vast
majority of Samza totally cluster manager agnostic, and make it possible to
implement a high quality "stand-alone" mode and support other frameworks.
It is possible this will let us run in YARN without *any* samza-specific
code using something generic like Slider, or maybe not. I don't think
anyone has tried this so it is probably premature to say how much
YARN-specific code could be killed. If that works out, then a bunch of the
YARN-specific code will disappear, but this doesn't mean YARN support will
disappear, just that we would retain less code to implement the same
thing.  Either way Samza in YARN isn't going away.

-Jay

On Sun, Jul 12, 2015 at 7:48 PM, Garrett Barton <ga...@gmail.com>
wrote:

> Yi,
>
>  What you just summarized makes a whole lot more sense to me.  Shamelessly
> I am looking at this shift as a customer with a production workflow riding
> on it so I am looking for some kind of consistency into the future of
> Samza.  This makes me feel a lot better about it.
>
>
> Thank you!
>
> On Sun, Jul 12, 2015 at 10:44 PM, Yi Pan <ni...@gmail.com> wrote:
>
> > Just to make it explicitly clear what I am proposing, here is a version
> of
> > more detailed description:
> >
> > The fourth option (in addition to what Jakob summarized) we are proposing
> > is:
> >
> > - Recharter Samza to “stream processing as a service”
> >
> > - The current Samza core (the basic transformation API w/ basic partition
> > and offset management build-in) will be moved to Kafka Streams (i.e. part
> > of Kafka) and supports “run-as-a-library”
> >
> > - Deprecate the SystemConsumers and SystemProducers APIs and move them to
> > Copycat
> >
> > - The current SQL development:
> >
> >    * physical operators and a Trident-like stream API should stay in
> Kafka
> > Streams as libraries, enabling any standalone deployment to use the core
> > window/join functions
> >
> >    * query parser/planner and execution on top of a distributed service
> > should stay in new Samza (i.e. “stream processing as a service”)
> >
> > - Advanced features related to job scheduling/state management stays in
> new
> > Samza (i.e. “streaming processing as a service”)
> >
> >    * Any advanced PartitionManager implementation that can be plugged
> into
> > Kafka Streams
> >
> >    * Any auto-scaling, dynamic configuration via coordinator stream
> >
> >    * Any advanced state management s.t. host-affinity etc.
> >
> >
> > Pros:
> >
> > - W/ the current Samza core as Kafka Streams and move the ingestion to
> > Copycat, we achieved most of the goals in the initial proposal:
> >
> >    * Tighter coupling w/ Kafka
> >
> >    * Reuse Kafka’s build-in functionalities, such as offset manager,
> basic
> > partition distribution
> >
> >    * Separation of ingestion vs transformation APIs
> >
> >    * offload a lot of system-specific configuration to Kafka Streams and
> > Copycat (i.e. SystemFactory configure, serde configure, etc.)
> >
> >    * remove YARN dependency and make standalone deployment easy. As
> > Guozhang mentioned, it would be really easy to start a process that
> > internally run Kafka Streams as library.
> >
> > - By re-chartering Samza as “stream processing as a service”, we address
> > the concern regarding to
> >
> >    * Pluggable partition management
> >
> >    * Running in a distributed cluster to manage process lifecycle,
> > fault-tolerance, resource-allocation, etc.
> >
> >    * More advanced features s.t. host-affinity, auto-scaling, and dynamic
> > configure changes, etc.
> >
> >
> > Regarding to the code and community organization, I think the following
> may
> > be the best:
> >
> > Code:
> >
> > - A Kafka sub-project Kafka Streams to hold samza-core, samza-kv-store,
> and
> > the physical operator layer as library in SQL: this would allow better
> > alignment w/ Kafka, in code, doc, and branding
> >
> > - Retain the current Samza project just to keep
> >
> >    * A pluggable explicit partition management in Kafka Streams client
> >
> >    * Integration w/ cluster-management systems for advanced features:
> >
> >       * host-affinity, auto-scaling,, dynamic configuration, etc.
> >
> >    * It will fully depend on the Kafka Streams API and remove all support
> > for SystemConsumers/SystemProducers in the future
> >
> > Community: (this is almost the same as what Chris proposed)
> >
> > - Kafka Streams: the current Samza community should be supporting this
> > effort together with some Kafka members, since most of the code here will
> > be from samza-core, samza-kv-store, and samza-sql.
> >
> > - new Samza: the current Samza community should continue serve the course
> > to support more advanced features to run Kafka Streams as a service.
> > Arguably, the new Samza framework may be used to run Copycat workers as
> > well, at least to manage Copycat worker’s lifecycle in a clustered
> > environment. Hence, it would stay as a general stream processing
> framework
> > that takes in any source and output to any destination, just the
> transport
> > system is fixed to Kafka.
> >
> > On Sun, Jul 12, 2015 at 7:29 PM, Yi Pan <ni...@gmail.com> wrote:
> >
> > > Hi, Chris,
> > >
> > > Thanks for sending out this concrete set of points here. I agree w/ all
> > > but have a slight different point view on 8).
> > >
> > > My view on this is: instead of sunset Samza as TLP, can we re-charter
> the
> > > scope of Samza to be the home for "running streaming process as a
> > service"?
> > >
> > > My main motivation is from the following points from a long internal
> > > discussion in LinkedIn:
> > >
> > > - There is a clear ask for pluggable partition management, like we do
> in
> > > LinkedIn, and as Ben Kirwin has mentioned in
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCACuX-D-yJX++2GnF_1Laf10KYUVyamg7UP_Dt19v0ZNmmhbCPQ@mail.gmail.com%3E
> > > - There are concerns on lack of support for running stream processing
> in
> > a
> > > cluster: lifecycle management, resource allocation, fault tolerance,
> etc.
> > > - There is a question to how to support more advanced features s.t.
> > > host-affinity, auto-scaling, and dynamic configuration in Samza jobs,
> as
> > > raised by Martin here:
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3C0D66EFD0-B7CD-4E4E-8B2F-2716167C313C@kleppmann.com%3E
> > >
> > > We have use cases that need to address all the above three cases and
> most
> > > of the functions are all in the current Samza project, in some flavor.
> We
> > > are all supporting to merge the samza-core functionalities into Kafka
> > > Streams, but there is a question where we keep these functions in the
> > > future. One option is to start a new project that includes these
> > functions
> > > that are closely related w/ "run stream-processing as-a-service", while
> > > another personally more attractive option is to re-charter Samza
> project
> > > just do "run stream processing as-a-service". We can avoid the overhead
> > of
> > > re-starting another community for this project. Personally, I felt that
> > > here are the benefits we should be getting:
> > >
> > > 1. We have already agreed mostly that Kafka Streams API would allow
> some
> > > pluggable partition management functions. Hence, the advanced partition
> > > management can live out-side the new Kafka Streams core w/o affecting
> the
> > > run-as-a-library model in Kafka Streams.
> > > 2. The integration w/ cluster management system and advanced features
> > > listed above stays in the same project and allow existing users enjoy
> > > no-impact migration to Kafka Stream as the core. That also addresses
> > Tim's
> > > question on "removing the support for YARN".
> > > 3. A separate project for stream-processing-as-a-service also allow the
> > > new Kafka Streams being independent to any cluster management and just
> > > focusing on stream process core functions, while leaving the functions
> > that
> > > requires cluster-resource and state management to a separate layer.
> > >
> > > Please feel free to comment. Thanks!
> > >
> > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
> criccomini@apache.org>
> > > wrote:
> > >
> > >> Hey all,
> > >>
> > >> I want to start by saying that I'm absolutely thrilled to be a part of
> > >> this
> > >> community. The amount of level-headed, thoughtful, educated discussion
> > >> that's gone on over the past ~10 days is overwhelming. Wonderful.
> > >>
> > >> It seems like discussion is waning a bit, and we've reached some
> > >> conclusions. There are several key emails in this threat, which I want
> > to
> > >> call out:
> > >>
> > >> 1. Jakob's summary of the three potential ways forward.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > >> 2. Julian's call out that we should be focusing on community over
> code.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > >> 3. Martin's summary about the benefits of merging communities.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > >> 4. Jakob's comments about the distinction between community and code
> > >> paths.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> > >>
> > >> I agree with the comments on all of these emails. I think Martin's
> > summary
> > >> of his position aligns very closely with my own. To that end, I think
> we
> > >> should get concrete about what the proposal is, and call a vote on it.
> > >> Given that Jay, Martin, and I seem to be aligning fairly closely, I
> > think
> > >> we should start with:
> > >>
> > >> 1. [community] Make Samza a subproject of Kafka.
> > >> 2. [community] Make all Samza PMC/committers committers of the
> > subproject.
> > >> 3. [community] Migrate Samza's website/documentation into Kafka's.
> > >> 4. [code] Have the Samza community and the Kafka community start a
> > >> from-scratch reboot together in the new Kafka subproject. We can
> > >> borrow/copy &  paste significant chunks of code from Samza's code
> base.
> > >> 5. [code] The subproject would intentionally eliminate support for
> both
> > >> other streaming systems and all deployment systems.
> > >> 6. [code] Attempt to provide a bridge from our SystemConsumer to
> KIP-26
> > >> (copy cat)
> > >> 7. [code] Attempt to provide a bridge from the new subproject's
> > processor
> > >> interface to our legacy StreamTask interface.
> > >> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> > >> subproject that has a fault-tolerant container with state management.
> > >>
> > >> It's likely that (6) and (7) won't be fully drop-in. Still, the closer
> > we
> > >> can get, the better it's going to be for our existing community.
> > >>
> > >> One thing that I didn't touch on with (2) is whether any Samza PMC
> > members
> > >> should be rolled into Kafka PMC membership as well (though, Jay and
> > Jakob
> > >> are already PMC members on both). I think that Samza's community
> > deserves
> > >> a
> > >> voice on the PMC, so I'd propose that we roll at least a few PMC
> members
> > >> into the Kafka PMC, but I don't have a strong framework for which
> people
> > >> to
> > >> pick.
> > >>
> > >> Before (8), I think that Samza's TLP can continue to commit bug fixes
> > and
> > >> patches as it sees fit, provided that we openly communicate that we
> > won't
> > >> necessarily migrate new features to the new subproject, and that the
> TLP
> > >> will be shut down after the migration to the Kafka subproject occurs.
> > >>
> > >> Jakob, I could use your guidance here about about how to achieve this
> > from
> > >> an Apache process perspective (sorry).
> > >>
> > >> * Should I just call a vote on this proposal?
> > >> * Should it happen on dev or private?
> > >> * Do committers have binding votes, or just PMC?
> > >>
> > >> Having trouble finding much detail on the Apache wikis. :(
> > >>
> > >> Cheers,
> > >> Chris
> > >>
> > >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com>
> wrote:
> > >>
> > >> > Thanks, Jay. This argument persuaded me actually. :)
> > >> >
> > >> > Fang, Yan
> > >> > yanfang724@gmail.com
> > >> >
> > >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > >> >
> > >> > > Hey Yan,
> > >> > >
> > >> > > Yeah philosophically I think the argument is that you should
> capture
> > >> the
> > >> > > stream in Kafka independent of the transformation. This is
> > obviously a
> > >> > > Kafka-centric view point.
> > >> > >
> > >> > > Advantages of this:
> > >> > > - In practice I think this is what e.g. Storm people often end up
> > >> doing
> > >> > > anyway. You usually need to throttle any access to a live serving
> > >> > database.
> > >> > > - Can have multiple subscribers and they get the same thing
> without
> > >> > > additional load on the source system.
> > >> > > - Applications can tap into the stream if need be by subscribing.
> > >> > > - You can debug your transformation by tailing the Kafka topic
> with
> > >> the
> > >> > > console consumer
> > >> > > - Can tee off the same data stream for batch analysis or Lambda
> arch
> > >> > style
> > >> > > re-processing
> > >> > >
> > >> > > The disadvantage is that it will use Kafka resources. But the idea
> > is
> > >> > > eventually you will have multiple subscribers to any data source
> (at
> > >> > least
> > >> > > for monitoring) so you will end up there soon enough anyway.
> > >> > >
> > >> > > Down the road the technical benefit is that I think it gives us a
> > good
> > >> > path
> > >> > > towards end-to-end exactly once semantics from source to
> > destination.
> > >> > > Basically the connectors need to support idempotence when talking
> to
> > >> > Kafka
> > >> > > and we need the transactional write feature in Kafka to make the
> > >> > > transformation atomic. This is actually pretty doable if you
> > separate
> > >> > > connector=>kafka problem from the generic transformations which
> are
> > >> > always
> > >> > > kafka=>kafka. However I think it is quite impossible to do in a
> > >> > all_things
> > >> > > => all_things environment. Today you can say "well the semantics
> of
> > >> the
> > >> > > Samza APIs depend on the connectors you use" but it is actually
> > worse
> > >> > then
> > >> > > that because the semantics actually depend on the pairing of
> > >> > connectors--so
> > >> > > not only can you probably not get a usable "exactly once"
> guarantee
> > >> > > end-to-end it can actually be quite hard to reverse engineer what
> > >> > property
> > >> > > (if any) your end-to-end flow has if you have heterogenous
> systems.
> > >> > >
> > >> > > -Jay
> > >> > >
> > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > > > {quote}
> > >> > > > maintained in a separate repository and retaining the existing
> > >> > > > committership but sharing as much else as possible (website,
> etc)
> > >> > > > {quote}
> > >> > > >
> > >> > > > Overall, I agree on this idea. Now the question is more about
> "how
> > >> to
> > >> > do
> > >> > > > it".
> > >> > > >
> > >> > > > On the other hand, one thing I want to point out is that, if we
> > >> decide
> > >> > to
> > >> > > > go this way, how do we want to support
> > >> > > > otherSystem-transformation-otherSystem use case?
> > >> > > >
> > >> > > > Basically, there are four user groups here:
> > >> > > >
> > >> > > > 1. Kafka-transformation-Kafka
> > >> > > > 2. Kafka-transformation-otherSystem
> > >> > > > 3. otherSystem-transformation-Kafka
> > >> > > > 4. otherSystem-transformation-otherSystem
> > >> > > >
> > >> > > > For group 1, they can easily use the new Samza library to
> achieve.
> > >> For
> > >> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka
> or
> > >> > Kafka->
> > >> > > > transformation -> copyCat.
> > >> > > >
> > >> > > > The problem is for group 4. Do we want to abandon this or still
> > >> support
> > >> > > it?
> > >> > > > Of course, this use case can be achieved by using copyCat ->
> > >> > > transformation
> > >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> > persuade
> > >> > them
> > >> > > to
> > >> > > > do this long chain. If yes, it will also be a win for Kafka too.
> > Or
> > >> if
> > >> > > > there is no one in this community actually doing this so far,
> > maybe
> > >> ok
> > >> > to
> > >> > > > not support the group 4 directly.
> > >> > > >
> > >> > > > Thanks,
> > >> > > >
> > >> > > > Fang, Yan
> > >> > > > yanfang724@gmail.com
> > >> > > >
> > >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
> > >> wrote:
> > >> > > >
> > >> > > > > Yeah I agree with this summary. I think there are kind of two
> > >> > questions
> > >> > > > > here:
> > >> > > > > 1. Technically does alignment/reliance on Kafka make sense
> > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
> alignment
> > >> with
> > >> > > > Kafka
> > >> > > > > make sense
> > >> > > > >
> > >> > > > > Personally I do think both of these things would be really
> > >> valuable,
> > >> > > and
> > >> > > > > would dramatically alter the trajectory of the project.
> > >> > > > >
> > >> > > > > My preference would be to see if people can mostly agree on a
> > >> > direction
> > >> > > > > rather than splintering things off. From my point of view the
> > >> ideal
> > >> > > > outcome
> > >> > > > > of all the options discussed would be to make Samza a closely
> > >> aligned
> > >> > > > > subproject, maintained in a separate repository and retaining
> > the
> > >> > > > existing
> > >> > > > > committership but sharing as much else as possible (website,
> > >> etc). No
> > >> > > > idea
> > >> > > > > about how these things work, Jacob, you probably know more.
> > >> > > > >
> > >> > > > > No discussion amongst the Kafka folks has happened on this,
> but
> > >> > likely
> > >> > > we
> > >> > > > > should figure out what the Samza community actually wants
> first.
> > >> > > > >
> > >> > > > > I admit that this is a fairly radical departure from how
> things
> > >> are.
> > >> > > > >
> > >> > > > > If that doesn't fly, I think, yeah we could leave Samza as it
> is
> > >> and
> > >> > do
> > >> > > > the
> > >> > > > > more radical reboot inside Kafka. From my point of view that
> > does
> > >> > leave
> > >> > > > > things in a somewhat confusing state since now there are two
> > >> stream
> > >> > > > > processing systems more or less coupled to Kafka in large part
> > >> made
> > >> > by
> > >> > > > the
> > >> > > > > same people. But, arguably that might be a cleaner way to make
> > the
> > >> > > > cut-over
> > >> > > > > and perhaps less risky for Samza community since if it works
> > >> people
> > >> > can
> > >> > > > > switch and if it doesn't nothing will have changed. Dunno, how
> > do
> > >> > > people
> > >> > > > > feel about this?
> > >> > > > >
> > >> > > > > -Jay
> > >> > > > >
> > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> > jghoman@gmail.com>
> > >> > > wrote:
> > >> > > > >
> > >> > > > > > >  This leads me to thinking that merging projects and
> > >> communities
> > >> > > > might
> > >> > > > > > be a good idea: with the union of experience from both
> > >> communities,
> > >> > > we
> > >> > > > > will
> > >> > > > > > probably build a better system that is better for users.
> > >> > > > > > Is this what's being proposed though? Merging the projects
> > seems
> > >> > like
> > >> > > > > > a consequence of at most one of the three directions under
> > >> > > discussion:
> > >> > > > > > 1) Samza 2.0: The Samza community relies more heavily on
> Kafka
> > >> for
> > >> > > > > > configuration, etc. (to a greater or lesser extent to be
> > >> > determined)
> > >> > > > > > but the Samza community would not automatically merge withe
> > >> Kafka
> > >> > > > > > community (the Phoenix/HBase example is a good one here).
> > >> > > > > > 2) Samza Reboot: The Samza community continues to exist
> with a
> > >> > > limited
> > >> > > > > > project scope, but similarly would not need to be part of
> the
> > >> Kafka
> > >> > > > > > community (ie given committership) to progress.  Here, maybe
> > the
> > >> > > Samza
> > >> > > > > > team would become a subproject of Kafka (the Board frowns on
> > >> > > > > > subprojects at the moment, so I'm not sure if that's even
> > >> > feasible),
> > >> > > > > > but that would not be required.
> > >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option
> > the
> > >> > Kafka
> > >> > > > > > team builds its own streaming library, possibly off of Jay's
> > >> > > > > > prototype, which has not direct lineage to the Samza team.
> > >> There's
> > >> > > no
> > >> > > > > > reason for the Kafka team to bring in the Samza team.
> > >> > > > > >
> > >> > > > > > Is the Kafka community on board with this?
> > >> > > > > >
> > >> > > > > > To be clear, all three options under discussion are
> > interesting,
> > >> > > > > > technically valid and likely healthy directions for the
> > project.
> > >> > > > > > Also, they are not mutually exclusive.  The Samza community
> > >> could
> > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
> community
> > >> went
> > >> > > > > > forward with 'Hey Samza!'  My points above are directed
> > >> entirely at
> > >> > > > > > the community aspect of these choices.
> > >> > > > > > -Jakob
> > >> > > > > >
> > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> > roger.hoover@gmail.com>
> > >> > > wrote:
> > >> > > > > > > That's great.  Thanks, Jay.
> > >> > > > > > >
> > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> > jay@confluent.io>
> > >> > > wrote:
> > >> > > > > > >
> > >> > > > > > >> Yeah totally agree. I think you have this issue even
> today,
> > >> > right?
> > >> > > > > I.e.
> > >> > > > > > if
> > >> > > > > > >> you need to make a simple config change and you're
> running
> > in
> > >> > YARN
> > >> > > > > today
> > >> > > > > > >> you end up bouncing the job which then rebuilds state. I
> > >> think
> > >> > the
> > >> > > > fix
> > >> > > > > > is
> > >> > > > > > >> exactly what you described which is to have a long
> timeout
> > on
> > >> > > > > partition
> > >> > > > > > >> movement for stateful jobs so that if a job is just
> getting
> > >> > > bounced,
> > >> > > > > and
> > >> > > > > > >> the cluster manager (or admin) is smart enough to restart
> > it
> > >> on
> > >> > > the
> > >> > > > > same
> > >> > > > > > >> host when possible, it can optimistically reuse any
> > existing
> > >> > state
> > >> > > > it
> > >> > > > > > finds
> > >> > > > > > >> on disk (if it is valid).
> > >> > > > > > >>
> > >> > > > > > >> So in this model the charter of the CM is to place
> > processes
> > >> as
> > >> > > > > > stickily as
> > >> > > > > > >> possible and to restart or re-place failed processes. The
> > >> > charter
> > >> > > of
> > >> > > > > the
> > >> > > > > > >> partition management system is to control the assignment
> of
> > >> work
> > >> > > to
> > >> > > > > > these
> > >> > > > > > >> processes. The nice thing about this is that the work
> > >> > assignment,
> > >> > > > > > timeouts,
> > >> > > > > > >> behavior, configs, and code will all be the same across
> all
> > >> > > cluster
> > >> > > > > > >> managers.
> > >> > > > > > >>
> > >> > > > > > >> So I think that prototype would actually give you exactly
> > >> what
> > >> > you
> > >> > > > > want
> > >> > > > > > >> today for any cluster manager (or manual placement +
> > restart
> > >> > > script)
> > >> > > > > > that
> > >> > > > > > >> was sticky in terms of host placement since there is
> > already
> > >> a
> > >> > > > > > configurable
> > >> > > > > > >> partition movement timeout and task-by-task state reuse
> > with
> > >> a
> > >> > > check
> > >> > > > > on
> > >> > > > > > >> state validity.
> > >> > > > > > >>
> > >> > > > > > >> -Jay
> > >> > > > > > >>
> > >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > >> > > > roger.hoover@gmail.com
> > >> > > > > >
> > >> > > > > > >> wrote:
> > >> > > > > > >>
> > >> > > > > > >> > That would be great to let Kafka do as much heavy
> lifting
> > >> as
> > >> > > > > possible
> > >> > > > > > and
> > >> > > > > > >> > make it easier for other languages to implement Samza
> > apis.
> > >> > > > > > >> >
> > >> > > > > > >> > One thing to watch out for is the interplay between
> > Kafka's
> > >> > > group
> > >> > > > > > >> > management and the external scheduler/process manager's
> > >> fault
> > >> > > > > > tolerance.
> > >> > > > > > >> > If a container dies, the Kafka group membership
> protocol
> > >> will
> > >> > > try
> > >> > > > to
> > >> > > > > > >> assign
> > >> > > > > > >> > it's tasks to other containers while at the same time
> the
> > >> > > process
> > >> > > > > > manager
> > >> > > > > > >> > is trying to relaunch the container.  Without some
> > >> > consideration
> > >> > > > for
> > >> > > > > > this
> > >> > > > > > >> > (like a configurable amount of time to wait before
> Kafka
> > >> > alters
> > >> > > > the
> > >> > > > > > group
> > >> > > > > > >> > membership), there may be thrashing going on which is
> > >> > especially
> > >> > > > bad
> > >> > > > > > for
> > >> > > > > > >> > containers with large amounts of local state.
> > >> > > > > > >> >
> > >> > > > > > >> > Someone else pointed this out already but I thought it
> > >> might
> > >> > be
> > >> > > > > worth
> > >> > > > > > >> > calling out again.
> > >> > > > > > >> >
> > >> > > > > > >> > Cheers,
> > >> > > > > > >> >
> > >> > > > > > >> > Roger
> > >> > > > > > >> >
> > >> > > > > > >> >
> > >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> > >> jay@confluent.io>
> > >> > > > > wrote:
> > >> > > > > > >> >
> > >> > > > > > >> > > Hey Roger,
> > >> > > > > > >> > >
> > >> > > > > > >> > > I couldn't agree more. We spent a bunch of time
> talking
> > >> to
> > >> > > > people
> > >> > > > > > and
> > >> > > > > > >> > that
> > >> > > > > > >> > > is exactly the stuff we heard time and again. What
> > makes
> > >> it
> > >> > > > hard,
> > >> > > > > of
> > >> > > > > > >> > > course, is that there is some tension between
> > >> compatibility
> > >> > > with
> > >> > > > > > what's
> > >> > > > > > >> > > there now and making things better for new users.
> > >> > > > > > >> > >
> > >> > > > > > >> > > I also strongly agree with the importance of
> > >> multi-language
> > >> > > > > > support. We
> > >> > > > > > >> > are
> > >> > > > > > >> > > talking now about Java, but for application
> development
> > >> use
> > >> > > > cases
> > >> > > > > > >> people
> > >> > > > > > >> > > want to work in whatever language they are using
> > >> elsewhere.
> > >> > I
> > >> > > > > think
> > >> > > > > > >> > moving
> > >> > > > > > >> > > to a model where Kafka itself does the group
> > membership,
> > >> > > > lifecycle
> > >> > > > > > >> > control,
> > >> > > > > > >> > > and partition assignment has the advantage of putting
> > all
> > >> > that
> > >> > > > > > complex
> > >> > > > > > >> > > stuff behind a clean api that the clients are already
> > >> going
> > >> > to
> > >> > > > be
> > >> > > > > > >> > > implementing for their consumer, so the added
> > >> functionality
> > >> > > for
> > >> > > > > > stream
> > >> > > > > > >> > > processing beyond a consumer becomes very minor.
> > >> > > > > > >> > >
> > >> > > > > > >> > > -Jay
> > >> > > > > > >> > >
> > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > >> > > > > > roger.hoover@gmail.com>
> > >> > > > > > >> > > wrote:
> > >> > > > > > >> > >
> > >> > > > > > >> > > > Metamorphosis...nice. :)
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > This has been a great discussion.  As a user of
> Samza
> > >> > who's
> > >> > > > > > recently
> > >> > > > > > >> > > > integrated it into a relatively large
> organization, I
> > >> just
> > >> > > > want
> > >> > > > > to
> > >> > > > > > >> add
> > >> > > > > > >> > > > support to a few points already made.
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
> > >> currently
> > >> > > > exists
> > >> > > > > > that
> > >> > > > > > >> > I've
> > >> > > > > > >> > > > experienced are:
> > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
> environments
> > >> > where
> > >> > > > > Puppet
> > >> > > > > > >> > would
> > >> > > > > > >> > > do
> > >> > > > > > >> > > > just fine but it was the only mechanism to get
> fault
> > >> > > > tolerance.
> > >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> > >> configuring
> > >> > > most
> > >> > > > > of
> > >> > > > > > the
> > >> > > > > > >> > job
> > >> > > > > > >> > > > in code rather than config files.  In general, I
> > think
> > >> the
> > >> > > > goal
> > >> > > > > > >> should
> > >> > > > > > >> > be
> > >> > > > > > >> > > > to make it harder to make mistakes, especially of
> the
> > >> kind
> > >> > > > where
> > >> > > > > > the
> > >> > > > > > >> > code
> > >> > > > > > >> > > > expects something and the config doesn't match.
> The
> > >> > current
> > >> > > > > > config
> > >> > > > > > >> is
> > >> > > > > > >> > > > quite intricate and error-prone.  For example, the
> > >> > > application
> > >> > > > > > logic
> > >> > > > > > >> > may
> > >> > > > > > >> > > > depend on bootstrapping a topic but rather than
> > >> asserting
> > >> > > that
> > >> > > > > in
> > >> > > > > > the
> > >> > > > > > >> > > code,
> > >> > > > > > >> > > > you have to rely on getting the config right.
> > Likewise
> > >> > with
> > >> > > > > > serdes,
> > >> > > > > > >> > the
> > >> > > > > > >> > > > Java representations produced by various serdes
> > (JSON,
> > >> > Avro,
> > >> > > > > etc.)
> > >> > > > > > >> are
> > >> > > > > > >> > > not
> > >> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> > >> without
> > >> > > > > changing
> > >> > > > > > >> the
> > >> > > > > > >> > > > code.   It would be nice for jobs to be able to
> > assert
> > >> > what
> > >> > > > they
> > >> > > > > > >> expect
> > >> > > > > > >> > > > from their input topics in terms of partitioning.
> > >> This is
> > >> > > > > > getting a
> > >> > > > > > >> > > little
> > >> > > > > > >> > > > off topic but I was even thinking about creating a
> > >> "Samza
> > >> > > > config
> > >> > > > > > >> > linter"
> > >> > > > > > >> > > > that would sanity check a set of configs.
> Especially
> > >> in
> > >> > > > > > >> organizations
> > >> > > > > > >> > > > where config is managed by a different team than
> the
> > >> > > > application
> > >> > > > > > >> > > developer,
> > >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > >> > > > > > >> > > > 3) Java/Scala centric - for many teams (especially
> > >> > > DevOps-type
> > >> > > > > > >> folks),
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > pain of the Java toolchain (maven, slow builds,
> weak
> > >> > command
> > >> > > > > line
> > >> > > > > > >> > > support,
> > >> > > > > > >> > > > configuration over convention) really inhibits
> > >> > productivity.
> > >> > > > As
> > >> > > > > > more
> > >> > > > > > >> > and
> > >> > > > > > >> > > > more high-quality clients become available for
> > Kafka, I
> > >> > hope
> > >> > > > > > they'll
> > >> > > > > > >> > > follow
> > >> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> > >> proposals
> > >> > > in
> > >> > > > > this
> > >> > > > > > >> > thread
> > >> > > > > > >> > > > but please consider other languages in the
> ecosystem
> > as
> > >> > > well.
> > >> > > > > > From
> > >> > > > > > >> > what
> > >> > > > > > >> > > > I've heard, Spark has more Python users than
> > >> Java/Scala.
> > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > >
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > >> > > > > > >> > > > and are working on a Yeoman generator
> > >> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> > >> > > Jython/Samza
> > >> > > > > > >> projects
> > >> > > > > > >> > to
> > >> > > > > > >> > > > alleviate some of the pain)
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > I also want to underscore Jay's point about
> improving
> > >> the
> > >> > > user
> > >> > > > > > >> > > experience.
> > >> > > > > > >> > > > That's a very important factor for adoption.  I
> think
> > >> the
> > >> > > goal
> > >> > > > > > should
> > >> > > > > > >> > be
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > make Samza as easy to get started with as something
> > >> like
> > >> > > > > Logstash.
> > >> > > > > > >> > > > Logstash is vastly inferior in terms of
> capabilities
> > to
> > >> > > Samza
> > >> > > > > but
> > >> > > > > > >> it's
> > >> > > > > > >> > > easy
> > >> > > > > > >> > > > to get started and that makes a big difference.
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > Cheers,
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > Roger
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> > Francisci
> > >> > > > Morales <
> > >> > > > > > >> > > > gdfm@apache.org> wrote:
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> > >> Metamorphosis
> > >> > > is
> > >> > > > a
> > >> > > > > > clear
> > >> > > > > > >> > > > winner
> > >> > > > > > >> > > > > :)
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > --
> > >> > > > > > >> > > > > Gianmarco
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> > >> Morales
> > >> > <
> > >> > > > > > >> > > gdfm@apache.org
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > wrote:
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > > Hi,
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > @Martin, thanks for you comments.
> > >> > > > > > >> > > > > > Maybe I'm missing some important point, but I
> > think
> > >> > > > coupling
> > >> > > > > > the
> > >> > > > > > >> > > > releases
> > >> > > > > > >> > > > > > is actually a *good* thing.
> > >> > > > > > >> > > > > > To make an example, would it be better if the
> MR
> > >> and
> > >> > > HDFS
> > >> > > > > > >> > components
> > >> > > > > > >> > > of
> > >> > > > > > >> > > > > > Hadoop had different release schedules?
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > Actually, keeping the discussion in a single
> > place
> > >> > would
> > >> > > > > make
> > >> > > > > > >> > > agreeing
> > >> > > > > > >> > > > on
> > >> > > > > > >> > > > > > releases (and backwards compatibility) much
> > >> easier, as
> > >> > > > > > everybody
> > >> > > > > > >> > > would
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > > responsible for the whole codebase.
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > That said, I like the idea of absorbing
> > samza-core
> > >> as
> > >> > a
> > >> > > > > > >> > sub-project,
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > > leave the fancy stuff separate.
> > >> > > > > > >> > > > > > It probably gives 90% of the benefits we have
> > been
> > >> > > > > discussing
> > >> > > > > > >> here.
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > Cheers,
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > --
> > >> > > > > > >> > > > > > Gianmarco
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > >> > jay.kreps@gmail.com
> > >> > > >
> > >> > > > > > wrote:
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >> Hey Martin,
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> I agree coupling release schedules is a
> > downside.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> Definitely we can try to solve some of the
> > >> > integration
> > >> > > > > > problems
> > >> > > > > > >> in
> > >> > > > > > >> > > > > >> Confluent Platform or in other distributions.
> > But
> > >> I
> > >> > > think
> > >> > > > > > this
> > >> > > > > > >> > ends
> > >> > > > > > >> > > up
> > >> > > > > > >> > > > > >> being really shallow. I guess I feel to really
> > >> get a
> > >> > > good
> > >> > > > > > user
> > >> > > > > > >> > > > > experience
> > >> > > > > > >> > > > > >> the two systems have to kind of feel like part
> > of
> > >> the
> > >> > > > same
> > >> > > > > > thing
> > >> > > > > > >> > and
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> can't really add that in later--you can put
> both
> > >> in
> > >> > the
> > >> > > > > same
> > >> > > > > > >> > > > > downloadable
> > >> > > > > > >> > > > > >> tar file but it doesn't really give a very
> > >> cohesive
> > >> > > > > feeling.
> > >> > > > > > I
> > >> > > > > > >> > agree
> > >> > > > > > >> > > > > that
> > >> > > > > > >> > > > > >> ultimately any of the project stuff is as much
> > >> social
> > >> > > and
> > >> > > > > > naming
> > >> > > > > > >> > as
> > >> > > > > > >> > > > > >> anything else--theoretically two totally
> > >> independent
> > >> > > > > projects
> > >> > > > > > >> > could
> > >> > > > > > >> > > > work
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> tightly align. In practice this seems to be
> > quite
> > >> > > > difficult
> > >> > > > > > >> > though.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> For the frameworks--totally agree it would be
> > >> good to
> > >> > > > > > maintain
> > >> > > > > > >> the
> > >> > > > > > >> > > > > >> framework support with the project. In some
> > cases
> > >> > there
> > >> > > > may
> > >> > > > > > not
> > >> > > > > > >> be
> > >> > > > > > >> > > too
> > >> > > > > > >> > > > > >> much
> > >> > > > > > >> > > > > >> there since the integration gets lighter but I
> > >> think
> > >> > > > > whatever
> > >> > > > > > >> > stubs
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> need should be included. So no I definitely
> > wasn't
> > >> > > trying
> > >> > > > > to
> > >> > > > > > >> imply
> > >> > > > > > >> > > > > >> dropping
> > >> > > > > > >> > > > > >> support for these frameworks, just making the
> > >> > > integration
> > >> > > > > > >> lighter
> > >> > > > > > >> > by
> > >> > > > > > >> > > > > >> separating process management from partition
> > >> > > management.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> You raise two good points we would have to
> > figure
> > >> out
> > >> > > if
> > >> > > > we
> > >> > > > > > went
> > >> > > > > > >> > > down
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> alignment path:
> > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think the
> > >> first
> > >> > > > > question
> > >> > > > > > is
> > >> > > > > > >> > > > whether
> > >> > > > > > >> > > > > >> some "re-branding" would be worth it. If so
> > then I
> > >> > > think
> > >> > > > we
> > >> > > > > > can
> > >> > > > > > >> > > have a
> > >> > > > > > >> > > > > big
> > >> > > > > > >> > > > > >> thread on the name. I'm definitely not set on
> > >> Kafka
> > >> > > > > > Streaming or
> > >> > > > > > >> > > Kafka
> > >> > > > > > >> > > > > >> Streams I was just using them to be kind of
> > >> > > > illustrative. I
> > >> > > > > > >> agree
> > >> > > > > > >> > > with
> > >> > > > > > >> > > > > >> your
> > >> > > > > > >> > > > > >> critique of these names, though I think people
> > >> would
> > >> > > get
> > >> > > > > the
> > >> > > > > > >> idea.
> > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about how
> to
> > >> > > "factor"
> > >> > > > > it.
> > >> > > > > > >> Here
> > >> > > > > > >> > > are
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> options I see (I could get enthusiastic about
> > any
> > >> of
> > >> > > > them):
> > >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> > seperation
> > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api
> and
> > >> > > > samza-core
> > >> > > > > > is
> > >> > > > > > >> > > > absorbed
> > >> > > > > > >> > > > > >> almost like a third client
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> Cheers,
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> -Jay
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> > Kleppmann <
> > >> > > > > > >> > > > martin@kleppmann.com>
> > >> > > > > > >> > > > > >> wrote:
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a
> few
> > >> > > follow-up
> > >> > > > > > >> > comments.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
> > >> > becoming
> > >> > > a
> > >> > > > > > >> > subproject:
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > reasons you mention are good. The risk I see
> > is
> > >> > that
> > >> > > > > > release
> > >> > > > > > >> > > > schedules
> > >> > > > > > >> > > > > >> > become coupled to each other, which can slow
> > >> > everyone
> > >> > > > > down,
> > >> > > > > > >> and
> > >> > > > > > >> > > > large
> > >> > > > > > >> > > > > >> > projects with many contributors are harder
> to
> > >> > manage.
> > >> > > > > > (Jakob,
> > >> > > > > > >> > can
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> speak
> > >> > > > > > >> > > > > >> > from experience, having seen a wider range
> of
> > >> > Hadoop
> > >> > > > > > ecosystem
> > >> > > > > > >> > > > > >> projects?)
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > Some of the goals of a better unified
> > developer
> > >> > > > > experience
> > >> > > > > > >> could
> > >> > > > > > >> > > > also
> > >> > > > > > >> > > > > be
> > >> > > > > > >> > > > > >> > solved by integrating Samza nicely into a
> > Kafka
> > >> > > > > > distribution
> > >> > > > > > >> > (such
> > >> > > > > > >> > > > as
> > >> > > > > > >> > > > > >> > Confluent's). I'm not against merging
> projects
> > >> if
> > >> > we
> > >> > > > > decide
> > >> > > > > > >> > that's
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> way
> > >> > > > > > >> > > > > >> > to go, just pointing out the same goals can
> > >> perhaps
> > >> > > > also
> > >> > > > > be
> > >> > > > > > >> > > achieved
> > >> > > > > > >> > > > > in
> > >> > > > > > >> > > > > >> > other ways.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
> dependency:
> > >> are
> > >> > > you
> > >> > > > > > >> proposing
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > >> > Samza doesn't give any help to people
> wanting
> > to
> > >> > run
> > >> > > on
> > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > >> > > > > > >> > > > > >> > So the docs would basically have a link to
> > >> Slider
> > >> > and
> > >> > > > > > nothing
> > >> > > > > > >> > > else?
> > >> > > > > > >> > > > Or
> > >> > > > > > >> > > > > >> > would we maintain integrations with a bunch
> of
> > >> > > popular
> > >> > > > > > >> > deployment
> > >> > > > > > >> > > > > >> methods
> > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts
> to
> > >> make
> > >> > > > Samza
> > >> > > > > > work
> > >> > > > > > >> > with
> > >> > > > > > >> > > > > >> Slider)?
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > I absolutely think it's a good idea to have
> > the
> > >> > "as a
> > >> > > > > > library"
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > "as a
> > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
> > >> people
> > >> > who
> > >> > > > > want
> > >> > > > > > >> them,
> > >> > > > > > >> > > > but I
> > >> > > > > > >> > > > > >> > think there should also be a low-friction
> path
> > >> for
> > >> > > > common
> > >> > > > > > "as
> > >> > > > > > >> a
> > >> > > > > > >> > > > > service"
> > >> > > > > > >> > > > > >> > deployment methods, for which we probably
> need
> > >> to
> > >> > > > > maintain
> > >> > > > > > >> > > > > integrations.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd
> to
> > >> me,
> > >> > > > > because
> > >> > > > > > >> Kafka
> > >> > > > > > >> > > is
> > >> > > > > > >> > > > > all
> > >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> > >> Transformers"
> > >> > > or
> > >> > > > > > "Kafka
> > >> > > > > > >> > > > Filters"
> > >> > > > > > >> > > > > >> > would be more apt?
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
> > >> (stream
> > >> > > > > > >> transformation
> > >> > > > > > >> > > > with
> > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> > >> library"
> > >> > > bit)
> > >> > > > > > could
> > >> > > > > > >> > > become
> > >> > > > > > >> > > > > >> part of
> > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
> > >> streaming
> > >> > SQL
> > >> > > > and
> > >> > > > > > >> > > > integrations
> > >> > > > > > >> > > > > >> with
> > >> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> > >> project?
> > >> > > In
> > >> > > > > > other
> > >> > > > > > >> > > words,
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > would absorb the proven, stable core of
> Samza,
> > >> > which
> > >> > > > > would
> > >> > > > > > >> > become
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in this
> > >> > thread.
> > >> > > > The
> > >> > > > > > Samza
> > >> > > > > > >> > > > project
> > >> > > > > > >> > > > > >> > would then target that third Kafka client as
> > its
> > >> > base
> > >> > > > > API,
> > >> > > > > > and
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > >> project
> > >> > > > > > >> > > > > >> > would be freed up to explore more
> experimental
> > >> new
> > >> > > > > > horizons.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > Martin
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > >> > > > jay.kreps@gmail.com>
> > >> > > > > > >> wrote:
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > > Hey Martin,
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I
> actually
> > >> > don't
> > >> > > > > think
> > >> > > > > > it
> > >> > > > > > >> > ties
> > >> > > > > > >> > > > our
> > >> > > > > > >> > > > > >> > hands
> > >> > > > > > >> > > > > >> > > at all, all it does is refactor things.
> The
> > >> > > division
> > >> > > > of
> > >> > > > > > >> > > > > >> responsibility is
> > >> > > > > > >> > > > > >> > > that Samza core is responsible for task
> > >> > lifecycle,
> > >> > > > > state,
> > >> > > > > > >> and
> > >> > > > > > >> > > > > >> partition
> > >> > > > > > >> > > > > >> > > management (using the Kafka co-ordinator)
> > but
> > >> it
> > >> > is
> > >> > > > NOT
> > >> > > > > > >> > > > responsible
> > >> > > > > > >> > > > > >> for
> > >> > > > > > >> > > > > >> > > packaging, configuration deployment or
> > >> execution
> > >> > of
> > >> > > > > > >> processes.
> > >> > > > > > >> > > The
> > >> > > > > > >> > > > > >> > problem
> > >> > > > > > >> > > > > >> > > of packaging and starting these processes
> is
> > >> > > > > > >> > > > > >> > > framework/environment-specific. This
> leaves
> > >> > > > individual
> > >> > > > > > >> > > frameworks
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > as
> > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you can
> > get
> > >> > > simple
> > >> > > > > > >> stateless
> > >> > > > > > >> > > > > >> support in
> > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf
> > app
> > >> > > > > framework
> > >> > > > > > >> > > (Slider,
> > >> > > > > > >> > > > > >> > Marathon,
> > >> > > > > > >> > > > > >> > > etc). These are well known by people and
> > have
> > >> > nice
> > >> > > > UIs
> > >> > > > > > and a
> > >> > > > > > >> > lot
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > > flexibility. I don't think they have node
> > >> > affinity
> > >> > > > as a
> > >> > > > > > >> built
> > >> > > > > > >> > in
> > >> > > > > > >> > > > > >> option
> > >> > > > > > >> > > > > >> > > (though I could be wrong). So if we want
> > that
> > >> we
> > >> > > can
> > >> > > > > > either
> > >> > > > > > >> > wait
> > >> > > > > > >> > > > for
> > >> > > > > > >> > > > > >> them
> > >> > > > > > >> > > > > >> > > to add it or do a custom framework to add
> > that
> > >> > > > feature
> > >> > > > > > (as
> > >> > > > > > >> > now).
> > >> > > > > > >> > > > > >> > Obviously
> > >> > > > > > >> > > > > >> > > if you manage things with old-school ops
> > tools
> > >> > > > > > >> > (puppet/chef/etc)
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> get
> > >> > > > > > >> > > > > >> > > locality easily. The nice thing, though,
> is
> > >> that
> > >> > > all
> > >> > > > > the
> > >> > > > > > >> samza
> > >> > > > > > >> > > > > >> "business
> > >> > > > > > >> > > > > >> > > logic" around partition management and
> fault
> > >> > > > tolerance
> > >> > > > > > is in
> > >> > > > > > >> > > Samza
> > >> > > > > > >> > > > > >> core
> > >> > > > > > >> > > > > >> > so
> > >> > > > > > >> > > > > >> > > it is shared across frameworks and the
> > >> framework
> > >> > > > > specific
> > >> > > > > > >> bit
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > just
> > >> > > > > > >> > > > > >> > > whether it is smart enough to try to get
> the
> > >> same
> > >> > > > host
> > >> > > > > > when
> > >> > > > > > >> a
> > >> > > > > > >> > > job
> > >> > > > > > >> > > > is
> > >> > > > > > >> > > > > >> > > restarted.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah
> I
> > >> think
> > >> > > the
> > >> > > > > > goal
> > >> > > > > > >> > would
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > >> (a)
> > >> > > > > > >> > > > > >> > > actually get better alignment in user
> > >> experience,
> > >> > > and
> > >> > > > > (b)
> > >> > > > > > >> > > express
> > >> > > > > > >> > > > > >> this in
> > >> > > > > > >> > > > > >> > > the naming and project branding.
> > Specifically:
> > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> > >> > > > > > "transformation"
> > >> > > > > > >> api
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e.
> be
> > >> able
> > >> > > to
> > >> > > > > > explain
> > >> > > > > > >> > > when
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> use
> > >> > > > > > >> > > > > >> > > the consumer and when to use the stream
> > >> > processing
> > >> > > > > > >> > functionality
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> lead
> > >> > > > > > >> > > > > >> > > people into that experience.
> > >> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza
> 1.4.2
> > >> (or
> > >> > > > > > whatever)
> > >> > > > > > >> > that
> > >> > > > > > >> > > > has
> > >> > > > > > >> > > > > >> both
> > >> > > > > > >> > > > > >> > > Kafka and the stream processing part and
> > they
> > >> > > > actually
> > >> > > > > > work
> > >> > > > > > >> > > > > together.
> > >> > > > > > >> > > > > >> > > 3. Unify the programming experience so the
> > >> client
> > >> > > and
> > >> > > > > > Samza
> > >> > > > > > >> > api
> > >> > > > > > >> > > > > share
> > >> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> > committers
> > >> and
> > >> > > can
> > >> > > > > > have a
> > >> > > > > > >> > > > > separate
> > >> > > > > > >> > > > > >> > repo,
> > >> > > > > > >> > > > > >> > > but I'm actually not really sure (I can't
> > >> find a
> > >> > > > > > definition
> > >> > > > > > >> > of a
> > >> > > > > > >> > > > > >> > subproject
> > >> > > > > > >> > > > > >> > > in Apache).
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > Basically at a high-level you want the
> > >> experience
> > >> > > to
> > >> > > > > > "feel"
> > >> > > > > > >> > > like a
> > >> > > > > > >> > > > > >> single
> > >> > > > > > >> > > > > >> > > system, not to relatively independent
> things
> > >> that
> > >> > > are
> > >> > > > > > kind
> > >> > > > > > >> of
> > >> > > > > > >> > > > > >> awkwardly
> > >> > > > > > >> > > > > >> > > glued together.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > I think if we did that they having naming
> or
> > >> > > branding
> > >> > > > > > like
> > >> > > > > > >> > > "kafka
> > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or something
> > >> like
> > >> > > that
> > >> > > > > > would
> > >> > > > > > >> > > > actually
> > >> > > > > > >> > > > > >> do a
> > >> > > > > > >> > > > > >> > > good job of conveying what it is. I do
> that
> > >> this
> > >> > > > would
> > >> > > > > > help
> > >> > > > > > >> > > > adoption
> > >> > > > > > >> > > > > >> > quite
> > >> > > > > > >> > > > > >> > > a lot as it would correctly convey that
> > using
> > >> > Kafka
> > >> > > > > > >> Streaming
> > >> > > > > > >> > > with
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > is
> > >> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
> > >> pretty
> > >> > > > > heavily
> > >> > > > > > >> > adopted
> > >> > > > > > >> > > > at
> > >> > > > > > >> > > > > >> this
> > >> > > > > > >> > > > > >> > > point.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> > >> originally
> > >> > > > when
> > >> > > > > > open
> > >> > > > > > >> > > > sourcing
> > >> > > > > > >> > > > > >> > Samza,
> > >> > > > > > >> > > > > >> > > however at that time Kafka was relatively
> > >> unknown
> > >> > > and
> > >> > > > > we
> > >> > > > > > >> > decided
> > >> > > > > > >> > > > not
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> > do
> > >> > > > > > >> > > > > >> > > it since we felt it would be limiting.
> From
> > my
> > >> > > point
> > >> > > > of
> > >> > > > > > view
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > three
> > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now
> really
> > >> > heavily
> > >> > > > > used
> > >> > > > > > for
> > >> > > > > > >> > > > stream
> > >> > > > > > >> > > > > >> > > processing, (2) we learned that
> abstracting
> > >> out
> > >> > the
> > >> > > > > > stream
> > >> > > > > > >> > well
> > >> > > > > > >> > > is
> > >> > > > > > >> > > > > >> > > basically impossible, (3) we learned it is
> > >> really
> > >> > > > hard
> > >> > > > > to
> > >> > > > > > >> keep
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > > two
> > >> > > > > > >> > > > > >> > > things feeling like a single product.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > -Jay
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> > >> Kleppmann
> > >> > <
> > >> > > > > > >> > > > > >> martin@kleppmann.com>
> > >> > > > > > >> > > > > >> > > wrote:
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > >> Hi all,
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> I agree with the general philosophy of
> > tying
> > >> > Samza
> > >> > > > > more
> > >> > > > > > >> > firmly
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > > >> Kafka.
> > >> > > > > > >> > > > > >> > >> After I spent a while looking at
> > integrating
> > >> > other
> > >> > > > > > message
> > >> > > > > > >> > > > brokers
> > >> > > > > > >> > > > > >> (e.g.
> > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to
> the
> > >> > > > conclusion
> > >> > > > > > that
> > >> > > > > > >> > > > > >> > SystemConsumer
> > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
> > Kafka's
> > >> > that
> > >> > > > > pretty
> > >> > > > > > >> much
> > >> > > > > > >> > > > > nobody
> > >> > > > > > >> > > > > >> but
> > >> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
> > >> > perhaps
> > >> > > an
> > >> > > > > > >> > exception,
> > >> > > > > > >> > > > but
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.)
> > Thus,
> > >> > > making
> > >> > > > > > Samza
> > >> > > > > > >> > > fully
> > >> > > > > > >> > > > > >> > dependent
> > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > >> > system-independence
> > >> > > > was
> > >> > > > > > >> never
> > >> > > > > > >> > as
> > >> > > > > > >> > > > > real
> > >> > > > > > >> > > > > >> as
> > >> > > > > > >> > > > > >> > we
> > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of
> > code
> > >> > reuse
> > >> > > > are
> > >> > > > > > >> real.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN
> has
> > >> also
> > >> > > > always
> > >> > > > > > been
> > >> > > > > > >> > > > > >> appealing to
> > >> > > > > > >> > > > > >> > >> me, for various reasons already mentioned
> > in
> > >> > this
> > >> > > > > > thread.
> > >> > > > > > >> > > > Although
> > >> > > > > > >> > > > > >> > making
> > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > >> > > > (YARN/Mesos/AWS/etc)
> > >> > > > > > >> seems
> > >> > > > > > >> > > > > >> laudable,
> > >> > > > > > >> > > > > >> > I am
> > >> > > > > > >> > > > > >> > >> a little concerned that it will restrict
> us
> > >> to a
> > >> > > > > lowest
> > >> > > > > > >> > common
> > >> > > > > > >> > > > > >> > denominator.
> > >> > > > > > >> > > > > >> > >> For example, would host affinity
> > (SAMZA-617)
> > >> > still
> > >> > > > be
> > >> > > > > > >> > possible?
> > >> > > > > > >> > > > For
> > >> > > > > > >> > > > > >> jobs
> > >> > > > > > >> > > > > >> > >> with large amounts of state, I think
> > >> SAMZA-617
> > >> > > would
> > >> > > > > be
> > >> > > > > > a
> > >> > > > > > >> big
> > >> > > > > > >> > > > boon,
> > >> > > > > > >> > > > > >> > since
> > >> > > > > > >> > > > > >> > >> restoring state off the changelog on
> every
> > >> > single
> > >> > > > > > restart
> > >> > > > > > >> is
> > >> > > > > > >> > > > > painful,
> > >> > > > > > >> > > > > >> > due
> > >> > > > > > >> > > > > >> > >> to long recovery times. It would be a
> shame
> > >> if
> > >> > the
> > >> > > > > > >> decoupling
> > >> > > > > > >> > > > from
> > >> > > > > > >> > > > > >> YARN
> > >> > > > > > >> > > > > >> > >> made host affinity impossible.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Jay, a question about the proposed API
> for
> > >> > > > > > instantiating a
> > >> > > > > > >> > job
> > >> > > > > > >> > > in
> > >> > > > > > >> > > > > >> code
> > >> > > > > > >> > > > > >> > >> (rather than a properties file): when
> > >> > submitting a
> > >> > > > job
> > >> > > > > > to a
> > >> > > > > > >> > > > > cluster,
> > >> > > > > > >> > > > > >> is
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >> idea that the instantiation code runs on
> a
> > >> > client
> > >> > > > > > >> somewhere,
> > >> > > > > > >> > > > which
> > >> > > > > > >> > > > > >> then
> > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > >> > > YARN/Mesos/AWS/etc?
> > >> > > > > Or
> > >> > > > > > >> does
> > >> > > > > > >> > > that
> > >> > > > > > >> > > > > >> code
> > >> > > > > > >> > > > > >> > run
> > >> > > > > > >> > > > > >> > >> on each container that is part of the job
> > (in
> > >> > > which
> > >> > > > > > case,
> > >> > > > > > >> how
> > >> > > > > > >> > > > does
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > job
> > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
> > >> right to
> > >> > > > make
> > >> > > > > a
> > >> > > > > > 1.0
> > >> > > > > > >> > > > release
> > >> > > > > > >> > > > > >> > with a
> > >> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete.
> So
> > if
> > >> > this
> > >> > > > is
> > >> > > > > > going
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> happen, I
> > >> > > > > > >> > > > > >> > >> think it would be more honest to stick
> with
> > >> 0.*
> > >> > > > > version
> > >> > > > > > >> > numbers
> > >> > > > > > >> > > > > until
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >> library-ified Samza has been implemented,
> > is
> > >> > > stable
> > >> > > > > and
> > >> > > > > > >> > widely
> > >> > > > > > >> > > > > used.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
> > >> Kafka?
> > >> > > There
> > >> > > > > is
> > >> > > > > > >> > > precedent
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > >> tight coupling between different Apache
> > >> projects
> > >> > > > (e.g.
> > >> > > > > > >> > Curator
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I
> think
> > >> > > remaining
> > >> > > > > > >> separate
> > >> > > > > > >> > > > would
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > ok.
> > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on
> Kafka,
> > >> there
> > >> > > is
> > >> > > > > > enough
> > >> > > > > > >> > > > > substance
> > >> > > > > > >> > > > > >> in
> > >> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> > >> project.
> > >> > > An
> > >> > > > > > >> argument
> > >> > > > > > >> > in
> > >> > > > > > >> > > > > >> favour
> > >> > > > > > >> > > > > >> > of
> > >> > > > > > >> > > > > >> > >> merging would be if we think Kafka has a
> > much
> > >> > > > stronger
> > >> > > > > > >> "brand
> > >> > > > > > >> > > > > >> presence"
> > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one.
> If
> > >> the
> > >> > > Kafka
> > >> > > > > > >> project
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > >> willing
> > >> > > > > > >> > > > > >> > to
> > >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of
> > doing
> > >> > > > stateful
> > >> > > > > > >> stream
> > >> > > > > > >> > > > > >> > >> transformations, that would probably have
> > >> much
> > >> > the
> > >> > > > > same
> > >> > > > > > >> > effect
> > >> > > > > > >> > > as
> > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> > >> Processors"
> > >> > or
> > >> > > > > > suchlike.
> > >> > > > > > >> > > Close
> > >> > > > > > >> > > > > >> > >> collaboration between the two projects
> will
> > >> be
> > >> > > > needed
> > >> > > > > in
> > >> > > > > > >> any
> > >> > > > > > >> > > > case.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> From a project management perspective, I
> > >> guess
> > >> > the
> > >> > > > > "new
> > >> > > > > > >> > Samza"
> > >> > > > > > >> > > > > would
> > >> > > > > > >> > > > > >> > have
> > >> > > > > > >> > > > > >> > >> to be developed on a branch alongside
> > ongoing
> > >> > > > > > maintenance
> > >> > > > > > >> of
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > > >> current
> > >> > > > > > >> > > > > >> > >> line of development? I think it would be
> > >> > important
> > >> > > > to
> > >> > > > > > >> > continue
> > >> > > > > > >> > > > > >> > supporting
> > >> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> > >> migration
> > >> > > > path
> > >> > > > > to
> > >> > > > > > >> the
> > >> > > > > > >> > > new
> > >> > > > > > >> > > > > >> > version.
> > >> > > > > > >> > > > > >> > >> Leaving the current versions unsupported
> > and
> > >> > > forcing
> > >> > > > > > people
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> rewrite
> > >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Best,
> > >> > > > > > >> > > > > >> > >> Martin
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > >> > > > jay@confluent.io>
> > >> > > > > > >> wrote:
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >>> Hey Garry,
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be
> > happy
> > >> to
> > >> > > chat
> > >> > > > > > more
> > >> > > > > > >> > about
> > >> > > > > > >> > > > > this
> > >> > > > > > >> > > > > >> if
> > >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
> > >> > started
> > >> > > > with
> > >> > > > > > the
> > >> > > > > > >> > idea
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> "what
> > >> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> > >> > ingestion
> > >> > > > > tool"
> > >> > > > > > but
> > >> > > > > > >> > > > > >> ultimately
> > >> > > > > > >> > > > > >> > we
> > >> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> > >> ingestion
> > >> > > and
> > >> > > > > > >> > > > transformation
> > >> > > > > > >> > > > > >> had
> > >> > > > > > >> > > > > >> > >>> pretty different needs and coupling the
> > two
> > >> > made
> > >> > > > > things
> > >> > > > > > >> > hard.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
> > (KIP-26)
> > >> > > > actually
> > >> > > > > > will
> > >> > > > > > >> > do
> > >> > > > > > >> > > > what
> > >> > > > > > >> > > > > >> you
> > >> > > > > > >> > > > > >> > >> are
> > >> > > > > > >> > > > > >> > >>> looking for.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> With regard to your point about slider,
> I
> > >> don't
> > >> > > > > > >> necessarily
> > >> > > > > > >> > > > > >> disagree.
> > >> > > > > > >> > > > > >> > >> But I
> > >> > > > > > >> > > > > >> > >>> think getting good YARN support is quite
> > >> doable
> > >> > > > and I
> > >> > > > > > >> think
> > >> > > > > > >> > we
> > >> > > > > > >> > > > can
> > >> > > > > > >> > > > > >> make
> > >> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> > >> proposal
> > >> > > > > solves
> > >> > > > > > is
> > >> > > > > > >> > that
> > >> > > > > > >> > > > > >> > >> technically
> > >> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
> > >> cluster
> > >> > > > > > management
> > >> > > > > > >> > > systems
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > way
> > >> > > > > > >> > > > > >> > >>> things are now, you need to write an
> "app
> > >> > master"
> > >> > > > or
> > >> > > > > > >> > > "framework"
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > each
> > >> > > > > > >> > > > > >> > >>> and they are all a little different so
> > >> testing
> > >> > is
> > >> > > > > > really
> > >> > > > > > >> > hard.
> > >> > > > > > >> > > > In
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > >>> absence of this we have been stuck with
> > just
> > >> > YARN
> > >> > > > > which
> > >> > > > > > >> has
> > >> > > > > > >> > > > > >> fantastic
> > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the
> > org,
> > >> but
> > >> > > > zero
> > >> > > > > > >> > > penetration
> > >> > > > > > >> > > > > >> > >> elsewhere.
> > >> > > > > > >> > > > > >> > >>> Given the huge amount of work being put
> in
> > >> to
> > >> > > > slider,
> > >> > > > > > >> > > marathon,
> > >> > > > > > >> > > > > aws
> > >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
> > related
> > >> > > > packaging
> > >> > > > > > >> > > > technologies
> > >> > > > > > >> > > > > >> > people
> > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> > >> > > > > cloud-specific
> > >> > > > > > >> > deploy
> > >> > > > > > >> > > > > >> tools,
> > >> > > > > > >> > > > > >> > >> etc)
> > >> > > > > > >> > > > > >> > >>> I really think it is important to get
> this
> > >> > right.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> -Jay
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> > >> > Turkington
> > >> > > <
> > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>>> Hi all,
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> I think the question below re does
> Samza
> > >> > become
> > >> > > a
> > >> > > > > > >> > sub-project
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>> highlights the broader point around
> > >> migration.
> > >> > > > Chris
> > >> > > > > > >> > mentions
> > >> > > > > > >> > > > > >> Samza's
> > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1
> release
> > >> but
> > >> > I'm
> > >> > > > not
> > >> > > > > > sure
> > >> > > > > > >> > it
> > >> > > > > > >> > > > > feels
> > >> > > > > > >> > > > > >> > >> right to
> > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> > >> deprecate
> > >> > > > most
> > >> > > > > of
> > >> > > > > > >> it.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have some
> > guys
> > >> > who
> > >> > > > have
> > >> > > > > > >> > started
> > >> > > > > > >> > > > > >> working
> > >> > > > > > >> > > > > >> > >> with
> > >> > > > > > >> > > > > >> > >>>> Samza and building some new
> > >> > consumers/producers
> > >> > > > was
> > >> > > > > > next
> > >> > > > > > >> > up.
> > >> > > > > > >> > > > > Sounds
> > >> > > > > > >> > > > > >> > like
> > >> > > > > > >> > > > > >> > >>>> that is absolutely not the direction to
> > >> go. I
> > >> > > need
> > >> > > > > to
> > >> > > > > > >> look
> > >> > > > > > >> > > into
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > KIP
> > >> > > > > > >> > > > > >> > >> in
> > >> > > > > > >> > > > > >> > >>>> more detail but for me the
> attractiveness
> > >> of
> > >> > > > adding
> > >> > > > > > new
> > >> > > > > > >> > Samza
> > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all
> > they
> > >> > were
> > >> > > > > doing
> > >> > > > > > was
> > >> > > > > > >> > > > really
> > >> > > > > > >> > > > > >> > getting
> > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to
> > avoid
> > >> > > > having
> > >> > > > > to
> > >> > > > > > >> > worry
> > >> > > > > > >> > > > > about
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > >>>> lifecycle management of external
> clients.
> > >> If
> > >> > > there
> > >> > > > > is
> > >> > > > > > a
> > >> > > > > > >> > > generic
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a
> > new
> > >> > > > connector
> > >> > > > > > into
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > have
> > >> > > > > > >> > > > > >> a
> > >> > > > > > >> > > > > >> > >> lot of
> > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> > reliability
> > >> > done
> > >> > > > for
> > >> > > > > me
> > >> > > > > > >> then
> > >> > > > > > >> > > it
> > >> > > > > > >> > > > > >> gives
> > >> > > > > > >> > > > > >> > me
> > >> > > > > > >> > > > > >> > >> all
> > >> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers
> > would.
> > >> If
> > >> > > not
> > >> > > > > > then it
> > >> > > > > > >> > > > > >> complicates
> > >> > > > > > >> > > > > >> > my
> > >> > > > > > >> > > > > >> > >>>> operational deployments.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Which is similar to my other question
> > with
> > >> the
> > >> > > > > > proposal
> > >> > > > > > >> --
> > >> > > > > > >> > if
> > >> > > > > > >> > > > we
> > >> > > > > > >> > > > > >> > build a
> > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus
> > the
> > >> > > > requisite
> > >> > > > > > >> shims
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> > integrate
> > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former
> may
> > >> be a
> > >> > > lot
> > >> > > > > more
> > >> > > > > > >> work
> > >> > > > > > >> > > > than
> > >> > > > > > >> > > > > we
> > >> > > > > > >> > > > > >> > >> think.
> > >> > > > > > >> > > > > >> > >>>> We may make it much easier for a
> newcomer
> > >> to
> > >> > get
> > >> > > > > > >> something
> > >> > > > > > >> > > > > running
> > >> > > > > > >> > > > > >> but
> > >> > > > > > >> > > > > >> > >>>> having them step up and get a reliable
> > >> > > production
> > >> > > > > > >> > deployment
> > >> > > > > > >> > > > may
> > >> > > > > > >> > > > > >> still
> > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
> > >> > different
> > >> > > > > > reasons
> > >> > > > > > >> > than
> > >> > > > > > >> > > > > >> today.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable
> > with
> > >> > > making
> > >> > > > > the
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > > >> dependency
> > >> > > > > > >> > > > > >> > >> on
> > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
> absolutely
> > >> see
> > >> > > the
> > >> > > > > > >> benefits
> > >> > > > > > >> > > in
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > >> > > > > > >> > > > terminologies/abstractions
> > >> > > > > > >> > > > > >> that
> > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library
> > >> would
> > >> > > > likely
> > >> > > > > > be a
> > >> > > > > > >> > very
> > >> > > > > > >> > > > > nice
> > >> > > > > > >> > > > > >> > tool
> > >> > > > > > >> > > > > >> > >> to
> > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have
> > the
> > >> > > > concerns
> > >> > > > > > >> above
> > >> > > > > > >> > re
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > >>>> operational side.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Garry
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> > >> [mailto:
> > >> > > > > > >> > gdfm@apache.org
> > >> > > > > > >> > > ]
> > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations
> on
> > >> > Samza
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > >> > > > > > >> > > > > >> > >>>> From outside, I have always perceived
> > Samza
> > >> > as a
> > >> > > > > > >> computing
> > >> > > > > > >> > > > layer
> > >> > > > > > >> > > > > >> over
> > >> > > > > > >> > > > > >> > >>>> Kafka.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit provocative,
> is
> > >> > > "should
> > >> > > > > > Samza
> > >> > > > > > >> be
> > >> > > > > > >> > a
> > >> > > > > > >> > > > > >> > sub-project
> > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> > >> separate
> > >> > > > > project
> > >> > > > > > >> > with a
> > >> > > > > > >> > > > > >> separate
> > >> > > > > > >> > > > > >> > >>>> governance?
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Cheers,
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> --
> > >> > > > > > >> > > > > >> > >>>> Gianmarco
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > >> > > > > > yanfang724@gmail.com>
> > >> > > > > > >> > > > wrote:
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka
> > more
> > >> > > > tightly.
> > >> > > > > > >> > Because
> > >> > > > > > >> > > > > Samza
> > >> > > > > > >> > > > > >> de
> > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
> > >> > leverage
> > >> > > > > what
> > >> > > > > > >> Kafka
> > >> > > > > > >> > > > has.
> > >> > > > > > >> > > > > At
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
> > reinvent
> > >> > what
> > >> > > > > Samza
> > >> > > > > > >> > > already
> > >> > > > > > >> > > > > >> has. I
> > >> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> > >> > ingestion
> > >> > > > and
> > >> > > > > > >> > > > > transformation.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
> > >> image
> > >> > > how
> > >> > > > > the
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > will
> > >> > > > > > >> > > > > >> look
> > >> > > > > > >> > > > > >> > >>>> like.
> > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> > >> > > difference
> > >> > > > > in
> > >> > > > > > >> terms
> > >> > > > > > >> > > of
> > >> > > > > > >> > > > > how
> > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code
> > >> shows
> > >> > (A
> > >> > > > > > client of
> > >> > > > > > >> > > > Kakfa)
> > >> > > > > > >> > > > > ?
> > >> > > > > > >> > > > > >> And
> > >> > > > > > >> > > > > >> > >>>>> user's application code calls this
> > client?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of
> > Kafka
> > >> > (like
> > >> > > > > what
> > >> > > > > > the
> > >> > > > > > >> > > code
> > >> > > > > > >> > > > > >> shows),
> > >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > >> > > > > fault-tolerance?
> > >> > > > > > >> Are
> > >> > > > > > >> > > they
> > >> > > > > > >> > > > > >> taken
> > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> > >> mechanism,
> > >> > > such
> > >> > > > > as
> > >> > > > > > >> > "Samza
> > >> > > > > > >> > > > > >> worker"
> > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 2. What about other features, such as
> > >> > > > auto-scaling,
> > >> > > > > > >> shared
> > >> > > > > > >> > > > > state,
> > >> > > > > > >> > > > > >> > >>>>> monitoring?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is
> > this
> > >> > what
> > >> > > > > Chris
> > >> > > > > > >> > > > suggests?)
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from
> > Kakfa
> > >> > and
> > >> > > > > > produce
> > >> > > > > > >> to
> > >> > > > > > >> > > it.
> > >> > > > > > >> > > > > >> Then it
> > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks
> > like
> > >> > now,
> > >> > > > > > except it
> > >> > > > > > >> > > does
> > >> > > > > > >> > > > > not
> > >> > > > > > >> > > > > >> > rely
> > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> > >> leverage
> > >> > > > Kafka's
> > >> > > > > > >> > metrics,
> > >> > > > > > >> > > > > logs,
> > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> Thanks,
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM,
> Guozhang
> > >> > Wang <
> > >> > > > > > >> > > > > wangguoz@gmail.com
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > >>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Read through the code example and it
> > >> looks
> > >> > > good
> > >> > > > to
> > >> > > > > > me.
> > >> > > > > > >> A
> > >> > > > > > >> > > few
> > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> > >> runnable
> > >> > > like:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > >> > > --config-factory=...
> > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> > deploying
> > >> > Samza
> > >> > > > > more
> > >> > > > > > as
> > >> > > > > > >> > > > embedded
> > >> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> > >> (ignoring
> > >> > > the
> > >> > > > > > >> > > terminology
> > >> > > > > > >> > > > > >> since
> > >> > > > > > >> > > > > >> > >>>>>> it is not the
> > >> > > > > > >> > > > > >> > >>>>> same
> > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> > >> MyStreamTask(configs);
> > >> > > > > Thread
> > >> > > > > > >> > thread
> > >> > > > > > >> > > =
> > >> > > > > > >> > > > > new
> > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment
> modes
> > >> are
> > >> > > > > important
> > >> > > > > > >> for
> > >> > > > > > >> > > > > >> different
> > >> > > > > > >> > > > > >> > >>>>>> types
> > >> > > > > > >> > > > > >> > >>>>> of
> > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making
> Samza
> > >> > purely
> > >> > > > > > >> standalone
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > still
> > >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or
> > library
> > >> > > modes.
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Guozhang
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
> > >> Kreps
> > >> > <
> > >> > > > > > >> > > > jay@confluent.io>
> > >> > > > > > >> > > > > >> > wrote:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> > >> example,
> > >> > it
> > >> > > > was
> > >> > > > > > >> > supposed
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> look
> > >> > > > > > >> > > > > >> > >>>>>>> like
> > >> > > > > > >> > > > > >> > >>>>>>> this:
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> > >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > >> > > > "localhost:4242");
> > >> > > > > > >> > > > > >> StreamingConfig
> > >> > > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > > > "test-topic-2");
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > config.processor(ExampleStreamProcessor.class);
> > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > >> > StringSerializer(),
> > >> > > > new
> > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
> KafkaStreaming
> > >> > > > container =
> > >> > > > > > new
> > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> > container.run();
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> -Jay
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM,
> Jay
> > >> > Kreps <
> > >> > > > > > >> > > > jay@confluent.io
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >> > >>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
> > >> Chris
> > >> > > and
> > >> > > > I
> > >> > > > > > were
> > >> > > > > > >> > > having
> > >> > > > > > >> > > > > >> > >>>>>>>> around
> > >> > > > > > >> > > > > >> > >>>>>>> whether
> > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza
> as a
> > >> kind
> > >> > > of
> > >> > > > > data
> > >> > > > > > >> > > > ingestion
> > >> > > > > > >> > > > > >> > >>>>> framework
> > >> > > > > > >> > > > > >> > >>>>>>> for
> > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to
> > KIP-26
> > >> > > > > "copycat").
> > >> > > > > > >> This
> > >> > > > > > >> > > > kind
> > >> > > > > > >> > > > > of
> > >> > > > > > >> > > > > >> > >>>>>> combined
> > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and
> > YARN
> > >> and
> > >> > > the
> > >> > > > > > >> > discussion
> > >> > > > > > >> > > > > >> around
> > >> > > > > > >> > > > > >> > >>>>>>>> how
> > >> > > > > > >> > > > > >> > >>>>> to
> > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was,
> given
> > >> that
> > >> > > > Samza
> > >> > > > > > was
> > >> > > > > > >> > > > basically
> > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific,
> what
> > if
> > >> > you
> > >> > > > just
> > >> > > > > > >> > embraced
> > >> > > > > > >> > > > > that
> > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > >> > > > > > >> > > > > >> > >>>>>> into
> > >> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> > >> > framework
> > >> > > > and
> > >> > > > > > more
> > >> > > > > > >> > > like a
> > >> > > > > > >> > > > > >> > >>>>>>>> third
> > >> > > > > > >> > > > > >> > >>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> > consumer"
> > >> > with
> > >> > > > > state
> > >> > > > > > >> > > > management
> > >> > > > > > >> > > > > >> > >>>>>> facilities.
> > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> > >> complex
> > >> > > > stream
> > >> > > > > > >> > > processing
> > >> > > > > > >> > > > > >> > >>>>>>>> framework
> > >> > > > > > >> > > > > >> > >>>>>>> this
> > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple
> > thing,
> > >> not
> > >> > > > much
> > >> > > > > > more
> > >> > > > > > >> > > > > >> complicated
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> use
> > >> > > > > > >> > > > > >> > >>>>>>> or
> > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As
> > Chris
> > >> > said
> > >> > > > we
> > >> > > > > > >> thought
> > >> > > > > > >> > > > about
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >>>>>>>> a
> > >> > > > > > >> > > > > >> > >>>>> lot
> > >> > > > > > >> > > > > >> > >>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> > >> > processing
> > >> > > > > > systems
> > >> > > > > > >> > were
> > >> > > > > > >> > > > > doing)
> > >> > > > > > >> > > > > >> > >>>>> seemed
> > >> > > > > > >> > > > > >> > >>>>>>> like
> > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output
> > >> data
> > >> > to
> > >> > > > and
> > >> > > > > > from
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > stream
> > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually
> > looked
> > >> > into
> > >> > > > how
> > >> > > > > > that
> > >> > > > > > >> > > would
> > >> > > > > > >> > > > > >> > >>>>>>>> work,
> > >> > > > > > >> > > > > >> > >>>>> Samza
> > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
> ingestion
> > >> > > framework
> > >> > > > > > for a
> > >> > > > > > >> > > bunch
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>> reasons.
> > >> > > > > > >> > > > > >> > >>>>>> To
> > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a
> > pretty
> > >> > > > different
> > >> > > > > > >> > internal
> > >> > > > > > >> > > > > data
> > >> > > > > > >> > > > > >> > >>>>>>>> model
> > >> > > > > > >> > > > > >> > >>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split
> > them
> > >> and
> > >> > > had
> > >> > > > > an
> > >> > > > > > api
> > >> > > > > > >> > for
> > >> > > > > > >> > > > > Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26)
> > >> and a
> > >> > > > > separate
> > >> > > > > > >> api
> > >> > > > > > >> > > for
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> > embracing
> > >> the
> > >> > > > same
> > >> > > > > > >> > > terminology
> > >> > > > > > >> > > > > and
> > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about
> the
> > >> > current
> > >> > > > > > state is
> > >> > > > > > >> > > that
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>> two
> > >> > > > > > >> > > > > >> > >>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology
> > >> like
> > >> > > > > "stream"
> > >> > > > > > vs
> > >> > > > > > >> > > > "topic"
> > >> > > > > > >> > > > > >> and
> > >> > > > > > >> > > > > >> > >>>>>>> different
> > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means
> > you
> > >> > kind
> > >> > > > of
> > >> > > > > > have
> > >> > > > > > >> to
> > >> > > > > > >> > > > learn
> > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > >> > > > > > >> > > > > >> > >>>>>>> way,
> > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
> different
> > >> way,
> > >> > > > then
> > >> > > > > > kind
> > >> > > > > > >> of
> > >> > > > > > >> > > > > >> > >>>>>>>> understand
> > >> > > > > > >> > > > > >> > >>>>> how
> > >> > > > > > >> > > > > >> > >>>>>>> they
> > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having
> > walked
> > >> a
> > >> > few
> > >> > > > > > people
> > >> > > > > > >> > > through
> > >> > > > > > >> > > > > >> this
> > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to
> > >> get.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of
> > >> time
> > >> > on
> > >> > > > > > >> airplanes I
> > >> > > > > > >> > > > > hacked
> > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> > >> incomplete
> > >> > > > > > prototype
> > >> > > > > > >> of
> > >> > > > > > >> > > > what
> > >> > > > > > >> > > > > >> > >>>>>>>> this would
> > >> > > > > > >> > > > > >> > >>>>> look
> > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
> > >> dumped
> > >> > > into
> > >> > > > > > Kafka
> > >> > > > > > >> as
> > >> > > > > > >> > > it
> > >> > > > > > >> > > > > >> > >>>>>>>> required a
> > >> > > > > > >> > > > > >> > >>>>>> few
> > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here
> is
> > >> the
> > >> > > code:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > >
> > >> > > > > > >> >
> > >> > > > > >
> > >> > >
> > >>
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I
> > just
> > >> > > > > liberally
> > >> > > > > > >> > renamed
> > >> > > > > > >> > > > > >> > >>>>>>>> everything
> > >> > > > > > >> > > > > >> > >>>>> to
> > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
> > >> regard
> > >> > > for
> > >> > > > > > >> > > > compatibility.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something like
> > >> this:
> > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
> Properties();
> > >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > >> > > > > "localhost:4242");
> > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > >> > > > > > >> > > > > >>
> config.processor(ExampleStreamProcessor.class);
> > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > >> > > StringDeserializer());
> > >> > > > > > >> > > > KafkaStreaming
> > >> > > > > > >> > > > > >> > >>>>>> container =
> > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > >> > container.run();
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > >> > > > SamzaContainer;
> > >> > > > > > >> > > > > StreamProcessor
> > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the
> class
> > >> names
> > >> > > in
> > >> > > > a
> > >> > > > > > file
> > >> > > > > > >> > and
> > >> > > > > > >> > > > then
> > >> > > > > > >> > > > > >> > >>>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you
> just
> > >> > > > > instantiate
> > >> > > > > > the
> > >> > > > > > >> > > > > container
> > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced
> > over
> > >> > > > however
> > >> > > > > > many
> > >> > > > > > >> > > > > instances
> > >> > > > > > >> > > > > >> > >>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>> this
> > >> > > > > > >> > > > > >> > >>>>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> > instance
> > >> > dies,
> > >> > > > new
> > >> > > > > > >> tasks
> > >> > > > > > >> > > are
> > >> > > > > > >> > > > > >> added
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
> shutting
> > >> them
> > >> > > > down).
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
> > running
> > >> > this
> > >> > > > > stuff
> > >> > > > > > in
> > >> > > > > > >> > YARN
> > >> > > > > > >> > > > via
> > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
> > >> using
> > >> > > some
> > >> > > > > of
> > >> > > > > > >> their
> > >> > > > > > >> > > > tools
> > >> > > > > > >> > > > > >> > >>>>>>>> but from the
> > >> > > > > > >> > > > > >> > >>>>>> point
> > >> > > > > > >> > > > > >> > >>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these
> stream
> > >> > > > processing
> > >> > > > > > jobs
> > >> > > > > > >> > are
> > >> > > > > > >> > > > > just
> > >> > > > > > >> > > > > >> > >>>>>> stateless
> > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
> > >> expand
> > >> > and
> > >> > > > > > contract
> > >> > > > > > >> > at
> > >> > > > > > >> > > > > will.
> > >> > > > > > >> > > > > >> > >>>>>>>> There
> > >> > > > > > >> > > > > >> > >>>>> is
> > >> > > > > > >> > > > > >> > >>>>>>> no
> > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code,
> > it
> > >> > would
> > >> > > > get
> > >> > > > > > >> larger
> > >> > > > > > >> > > if
> > >> > > > > > >> > > > we
> > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
> > larger.
> > >> We
> > >> > > > really
> > >> > > > > > do
> > >> > > > > > >> > get a
> > >> > > > > > >> > > > ton
> > >> > > > > > >> > > > > >> > >>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>> leverage
> > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> > >> > delegated
> > >> > > to
> > >> > > > > the
> > >> > > > > > >> new
> > >> > > > > > >> > > > > >> consumer.
> > >> > > > > > >> > > > > >> > >>>>> This
> > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> > >> > management
> > >> > > > > > strategy
> > >> > > > > > >> > > > > available
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to
> Samza
> > >> (and
> > >> > > vice
> > >> > > > > > versa)
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > with
> > >> > > > > > >> > > > > >> > >>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>> exact
> > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as
> > state
> > >> > reuse
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
> > >> > thought
> > >> > > > > > >> provoking.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM,
> > Chris
> > >> > > > > Riccomini <
> > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with
> > Samza
> > >> > > > > engineers
> > >> > > > > > at
> > >> > > > > > >> > > > LinkedIn
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>> Confluent
> > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> > observations
> > >> > and
> > >> > > > > would
> > >> > > > > > >> like
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> > >>>>>>>>> propose
> > >> > > > > > >> > > > > >> > >>>>> some
> > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I
> > >> want to
> > >> > > > call
> > >> > > > > > out
> > >> > > > > > >> > about
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > >> > > > > > >> > > > > >> > >>>>>> design,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> > changes.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a
> dynamic
> > >> > > > deployment
> > >> > > > > > >> system.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> > >> SystemConsumer/SystemProducer
> > >> > and
> > >> > > > > > Kafka's
> > >> > > > > > >> > > > consumer
> > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > >> > > > > > >> > > > > >> > >>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
> > >> > problems.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
> > related,
> > >> > but
> > >> > > > I'll
> > >> > > > > > >> > address
> > >> > > > > > >> > > > them
> > >> > > > > > >> > > > > >> in
> > >> > > > > > >> > > > > >> > >>>>> order.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use
> > of a
> > >> > > > dynamic
> > >> > > > > > >> > > deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > >> > > > > > >> > > > > >> > >>>>>> such
> > >> > > > > > >> > > > > >> > >>>>>>>>> as
> > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
> initially
> > >> built
> > >> > > > > Samza,
> > >> > > > > > we
> > >> > > > > > >> > bet
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > >> > >>>>>>>>> there
> > >> > > > > > >> > > > > >> > >>>>>> would
> > >> > > > > > >> > > > > >> > >>>>>>>>> be
> > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area,
> and
> > >> we
> > >> > > could
> > >> > > > > > >> support
> > >> > > > > > >> > > > them,
> > >> > > > > > >> > > > > >> and
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>> rest
> > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there
> are
> > >> many
> > >> > > > > > >> variations.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > >> > > > > > >> > > > > >> > >>>>>> many
> > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start
> > >> their
> > >> > > > > > processors
> > >> > > > > > >> > like
> > >> > > > > > >> > > > > normal
> > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
> traditional
> > >> > > > deployment
> > >> > > > > > >> scripts
> > >> > > > > > >> > > > such
> > >> > > > > > >> > > > > as
> > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > >> > > > > > >> > > > > >> > >>>>>> Chef,
> > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
> > >> system
> > >> > > on
> > >> > > > > > users
> > >> > > > > > >> > makes
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
> > painful
> > >> for
> > >> > > > first
> > >> > > > > > time
> > >> > > > > > >> > > > users.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
> requirement
> > >> was
> > >> > > also
> > >> > > > a
> > >> > > > > > bit
> > >> > > > > > >> of
> > >> > > > > > >> > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > >> > > > > > >> > > > > >> > >>>>>> because
> > >> > > > > > >> > > > > >> > >>>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding
> > between
> > >> > the
> > >> > > > > > nature of
> > >> > > > > > >> > > batch
> > >> > > > > > >> > > > > >> jobs
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>> stream
> > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> > >> > > conscious
> > >> > > > > > effort
> > >> > > > > > >> to
> > >> > > > > > >> > > > favor
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>> Hadoop
> > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things,
> > >> since
> > >> > it
> > >> > > > > worked
> > >> > > > > > >> and
> > >> > > > > > >> > > was
> > >> > > > > > >> > > > > well
> > >> > > > > > >> > > > > >> > >>>>>>> understood.
> > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that
> > >> batch
> > >> > > jobs
> > >> > > > > > have a
> > >> > > > > > >> > > > definite
> > >> > > > > > >> > > > > >> > >>>>>> beginning,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs
> > don't
> > >> > > > > (usually).
> > >> > > > > > >> This
> > >> > > > > > >> > > > leads
> > >> > > > > > >> > > > > to
> > >> > > > > > >> > > > > >> > >>>>>>>>> a
> > >> > > > > > >> > > > > >> > >>>>> much
> > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for
> > stream
> > >> > > > > processors.
> > >> > > > > > >> You
> > >> > > > > > >> > > > > >> basically
> > >> > > > > > >> > > > > >> > >>>>>>>>> just
> > >> > > > > > >> > > > > >> > >>>>>>> need
> > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> > >> processor,
> > >> > and
> > >> > > > > start
> > >> > > > > > >> it.
> > >> > > > > > >> > > The
> > >> > > > > > >> > > > > way
> > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's
> > no
> > >> > > concept
> > >> > > > > of
> > >> > > > > > a
> > >> > > > > > >> > > cluster
> > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > >> > > > > > >> > > > > >> > >>>>>> add
> > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> > >> coupling
> > >> > > > Samza
> > >> > > > > > with
> > >> > > > > > >> a
> > >> > > > > > >> > > > > >> scheduler
> > >> > > > > > >> > > > > >> > >>>>>>>>> is
> > >> > > > > > >> > > > > >> > >>>>>> that
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
> > >> handle
> > >> > > > > > deployment.
> > >> > > > > > >> > > This
> > >> > > > > > >> > > > > >> pulls
> > >> > > > > > >> > > > > >> > >>>>>>>>> in a
> > >> > > > > > >> > > > > >> > >>>>>>> bunch
> > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> > >> > > distribution
> > >> > > > > > (config
> > >> > > > > > >> > > > > stream),
> > >> > > > > > >> > > > > >> > >>>>>>>>> shell
> > >> > > > > > >> > > > > >> > >>>>>>> scrips
> > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> > packaging
> > >> > (all
> > >> > > > the
> > >> > > > > > .tgz
> > >> > > > > > >> > > > stuff),
> > >> > > > > > >> > > > > >> etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring
> dynamic
> > >> > > > deployment
> > >> > > > > > was
> > >> > > > > > >> to
> > >> > > > > > >> > > > > support
> > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> > >> > > locality,
> > >> > > > > you
> > >> > > > > > >> need
> > >> > > > > > >> > to
> > >> > > > > > >> > > > put
> > >> > > > > > >> > > > > >> > >>>>>>>>> your
> > >> > > > > > >> > > > > >> > >>>>>> processors
> > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> > processing.
> > >> > Upon
> > >> > > > > > further
> > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > >> > > > > > >> > > > > >> > >>>>>>> though,
> > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
> beneficial.
> > >> > There
> > >> > > is
> > >> > > > > > some
> > >> > > > > > >> > good
> > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > >> > > > > > >> > > > > >> > >>>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
> SAMZA-335.
> > >> > Again,
> > >> > > we
> > >> > > > > > took
> > >> > > > > > >> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > >> > > > > > >> > > > > >> > >>>>>> path,
> > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> > differences
> > >> > > > between
> > >> > > > > > HDFS
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > >> > > > > > >> > > > > >> > >>>>>> has
> > >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has
> partitions.
> > >> This
> > >> > > > leads
> > >> > > > > to
> > >> > > > > > >> less
> > >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
> > >> > > processors
> > >> > > > > on
> > >> > > > > > top
> > >> > > > > > >> > of
> > >> > > > > > >> > > > > Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a
> > crutch.
> > >> > > Samza
> > >> > > > > > doesn't
> > >> > > > > > >> > > have
> > >> > > > > > >> > > > > any
> > >> > > > > > >> > > > > >> > >>>>>>>>> built
> > >> > > > > > >> > > > > >> > >>>>> in
> > >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
> > >> > depends
> > >> > > on
> > >> > > > > the
> > >> > > > > > >> > > dynamic
> > >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to
> > handle
> > >> > > > restarts
> > >> > > > > > >> when a
> > >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > >> > > > > > >> > > > > >> > >>>>>>> made
> > >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> > >> standalone
> > >> > > Samza
> > >> > > > > > >> > container
> > >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is
> good,
> > >> but I
> > >> > > > think
> > >> > > > > > that
> > >> > > > > > >> > > we've
> > >> > > > > > >> > > > > >> gone
> > >> > > > > > >> > > > > >> > >>>>>>>>> too
> > >> > > > > > >> > > > > >> > >>>>>> far
> > >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > >> > > > (SystemConsumer,
> > >> > > > > > >> > > > > SystemProducer,
> > >> > > > > > >> > > > > >> > >>>> etc).
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just
> > about
> > >> > every
> > >> > > > > > >> component
> > >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > >> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> > >> > > ConfigRewriter,
> > >> > > > > > etc).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> > >> > forgotten,
> > >> > > as
> > >> > > > > > well.
> > >> > > > > > >> > Some
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>>>>>> these
> > >> > > > > > >> > > > > >> > >>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not
> to
> > >> be.
> > >> > > This
> > >> > > > > all
> > >> > > > > > >> comes
> > >> > > > > > >> > > at
> > >> > > > > > >> > > > a
> > >> > > > > > >> > > > > >> cost:
> > >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is
> > making
> > >> it
> > >> > > > harder
> > >> > > > > > for
> > >> > > > > > >> > our
> > >> > > > > > >> > > > > users
> > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> pick
> > >> > > > > > >> > > > > >> > >>>>>> up
> > >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It
> > also
> > >> > makes
> > >> > > > it
> > >> > > > > > >> > difficult
> > >> > > > > > >> > > > for
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about
> > what
> > >> the
> > >> > > > > > >> > > characteristics
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> > >> characteristics
> > >> > > > change
> > >> > > > > > >> > > depending
> > >> > > > > > >> > > > on
> > >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are
> > most
> > >> > > visible
> > >> > > > > in
> > >> > > > > > the
> > >> > > > > > >> > > > System
> > >> > > > > > >> > > > > >> APIs.
> > >> > > > > > >> > > > > >> > >>>>> What
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
> > >> functional is
> > >> > > > Kafka
> > >> > > > > > as
> > >> > > > > > >> its
> > >> > > > > > >> > > > > >> > >>>>>>>>> transport
> > >> > > > > > >> > > > > >> > >>>>>> layer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> But
> > >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use
> > >> cases
> > >> > > into
> > >> > > > > one
> > >> > > > > > >> API:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports
> both
> > >> of
> > >> > > these
> > >> > > > > use
> > >> > > > > > >> > cases.
> > >> > > > > > >> > > > The
> > >> > > > > > >> > > > > >> > >>>>>>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>> is,
> > >> > > > > > >> > > > > >> > >>>>>>>>> we
> > >> > > > > > >> > > > > >> > >>>>>>>>> actually want different features
> for
> > >> each
> > >> > > use
> > >> > > > > > case.
> > >> > > > > > >> By
> > >> > > > > > >> > > > > >> papering
> > >> > > > > > >> > > > > >> > >>>>>>>>> over
> > >> > > > > > >> > > > > >> > >>>>>>> these
> > >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a
> > single
> > >> > API,
> > >> > > > > we've
> > >> > > > > > >> > > > introduced
> > >> > > > > > >> > > > > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > >> > > > > > >> > > > > >> > >>>>>>> leaky
> > >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like
> > in
> > >> (2)
> > >> > > is
> > >> > > > to
> > >> > > > > > have
> > >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
> > >> > offsets
> > >> > > > > (like
> > >> > > > > > >> > Kafka).
> > >> > > > > > >> > > > > This
> > >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > >> > > > > > >> > > > > >> > >>>>> with
> > >> > > > > > >> > > > > >> > >>>>>>> (1),
> > >> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems
> have
> > >> > > > different
> > >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> > >> mailing
> > >> > > list
> > >> > > > > and
> > >> > > > > > >> the
> > >> > > > > > >> > > SQL
> > >> > > > > > >> > > > > >> JIRAs
> > >> > > > > > >> > > > > >> > >>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> > >> > > replayability.
> > >> > > > > > Kafka
> > >> > > > > > >> > > allows
> > >> > > > > > >> > > > us
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> > >>>>> rewind
> > >> > > > > > >> > > > > >> > >>>>>>>>> when
> > >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other
> > systems
> > >> > > don't.
> > >> > > > In
> > >> > > > > > some
> > >> > > > > > >> > > > cases,
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>>> return
> > >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > >> > > > > > >> WikipediaSystemConsumer)
> > >> > > > > > >> > > > > because
> > >> > > > > > >> > > > > >> > >>>>>>>>> they
> > >> > > > > > >> > > > > >> > >>>>>> have
> > >> > > > > > >> > > > > >> > >>>>>>> no
> > >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example.
> > Kafka
> > >> > > > supports
> > >> > > > > > >> > > > > partitioning,
> > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > >> > > > > > >> > > > > >> > >>>>> many
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
> > >> having a
> > >> > > > single
> > >> > > > > > >> > > partition
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other
> systems
> > >> model
> > >> > > > > > >> partitioning
> > >> > > > > > >> > > > > >> > >>>> differently (e.g.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also
> a
> > >> mess.
> > >> > > > > > Creating
> > >> > > > > > >> > > streams
> > >> > > > > > >> > > > > in
> > >> > > > > > >> > > > > >> a
> > >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> > >> impossible.
> > >> > > As
> > >> > > > is
> > >> > > > > > >> > modeling
> > >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > >> > > > > > >> > > > > >> > >>>>> for
> > >> > > > > > >> > > > > >> > >>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> > >> partitions,
> > >> > > > > location,
> > >> > > > > > >> > etc).
> > >> > > > > > >> > > > The
> > >> > > > > > >> > > > > >> > >>>>>>>>> list
> > >> > > > > > >> > > > > >> > >>>>> goes
> > >> > > > > > >> > > > > >> > >>>>>>> on.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing
> > >> Samza,
> > >> > > > > Kafka's
> > >> > > > > > >> > > consumer
> > >> > > > > > >> > > > > and
> > >> > > > > > >> > > > > >> > >>>>> producer
> > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set.
> > On
> > >> the
> > >> > > > > > >> > consumer-side,
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> > >>>>>>>>> had two
> > >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
> > consumer,
> > >> or
> > >> > > the
> > >> > > > > > simple
> > >> > > > > > >> > > > > consumer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> The
> > >> > > > > > >> > > > > >> > >>>>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was
> > that
> > >> it
> > >> > > > > > controlled
> > >> > > > > > >> > your
> > >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments,
> and
> > >> the
> > >> > > order
> > >> > > > > in
> > >> > > > > > >> which
> > >> > > > > > >> > > you
> > >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > >> > > > > > >> > > > > >> > >>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>>>>> with
> > >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's
> not
> > >> > > simple.
> > >> > > > > It's
> > >> > > > > > >> > basic.
> > >> > > > > > >> > > > You
> > >> > > > > > >> > > > > >> > >>>>>>>>> end up
> > >> > > > > > >> > > > > >> > >>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really
> low-level
> > >> stuff
> > >> > > > that
> > >> > > > > > you
> > >> > > > > > >> > > > > shouldn't.
> > >> > > > > > >> > > > > >> > >>>>>>>>> We
> > >> > > > > > >> > > > > >> > >>>>>> spent a
> > >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > >> > > > KafkaSystemConsumer
> > >> > > > > > very
> > >> > > > > > >> > > > robust.
> > >> > > > > > >> > > > > >> It
> > >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some
> cool
> > >> > > features:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering
> and
> > >> > > > > > prioritization.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> > >> assignment
> > >> > > to
> > >> > > > > > support
> > >> > > > > > >> > > > joins,
> > >> > > > > > >> > > > > >> > >>>>>>>>> global
> > >> > > > > > >> > > > > >> > >>>>>> state
> > >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)),
> > etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> > >> > checkpointing.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time
> > is
> > >> > that
> > >> > > > > these
> > >> > > > > > >> > > features
> > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > >> > > > > > >> > > > > >> > >>>>>>> actually
> > >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
> > consumers
> > >> > (not
> > >> > > > just
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > stream
> > >> > > > > > >> > > > > >> > >>>>>> processors)
> > >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like
> > joins
> > >> > and
> > >> > > > > > partition
> > >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> > >> > conclusion.
> > >> > > > > > They're
> > >> > > > > > >> > > adding
> > >> > > > > > >> > > > a
> > >> > > > > > >> > > > > >> ton
> > >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> > >> consumer
> > >> > > > > > >> > > implementation.
> > >> > > > > > >> > > > > To a
> > >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > >> > > > > > >> > > > > >> > >>>>> it's
> > >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've
> already
> > >> done
> > >> > > in
> > >> > > > > > Samza.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up
> > taking
> > >> a
> > >> > > very
> > >> > > > > > similar
> > >> > > > > > >> > > > > approach
> > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>>> Samza's
> > >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> > implementation
> > >> for
> > >> > > > > > handling
> > >> > > > > > >> > > offset
> > >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> > >> management
> > >> > > > > feature
> > >> > > > > > >> > stores
> > >> > > > > > >> > > > > >> offset
> > >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows
> > >> you to
> > >> > > > fetch
> > >> > > > > > them
> > >> > > > > > >> > > from
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste,
> > >> since
> > >> > we
> > >> > > > > could
> > >> > > > > > >> have
> > >> > > > > > >> > > > shared
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>> work
> > >> > > > > > >> > > > > >> > >>>>>> if
> > >> > > > > > >> > > > > >> > >>>>>>>>> it
> > >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
> > >> get-go.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> > >> radical
> > >> > > > > > proposal.
> > >> > > > > > >> > Samza
> > >> > > > > > >> > > > is
> > >> > > > > > >> > > > > >> > >>>>> relatively
> > >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture
> to
> > >> say
> > >> > > that
> > >> > > > > > we're
> > >> > > > > > >> > > near a
> > >> > > > > > >> > > > > 1.0
> > >> > > > > > >> > > > > >> > >>>>>> release.
> > >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what
> > >> we've
> > >> > > > > learned,
> > >> > > > > > and
> > >> > > > > > >> > > begin
> > >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > >> > > > > > >> > > > > >> > >>>>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we
> > >> change if
> > >> > > we
> > >> > > > > were
> > >> > > > > > >> > > starting
> > >> > > > > > >> > > > > >> from
> > >> > > > > > >> > > > > >> > >>>>>> scratch?
> > >> > > > > > >> > > > > >> > >>>>>>>>> My
> > >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the
> *only*
> > >> way
> > >> > to
> > >> > > > run
> > >> > > > > > Samza
> > >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all
> direct
> > >> > > > > dependences
> > >> > > > > > on
> > >> > > > > > >> > > YARN,
> > >> > > > > > >> > > > > >> Mesos,
> > >> > > > > > >> > > > > >> > >>>> etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to
> support
> > >> only
> > >> > > > Kafka
> > >> > > > > > as
> > >> > > > > > >> the
> > >> > > > > > >> > > > > stream
> > >> > > > > > >> > > > > >> > >>>>>> processing
> > >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics,
> > logging,
> > >> > > > > > >> serialization,
> > >> > > > > > >> > > and
> > >> > > > > > >> > > > > >> > >>>>>>>>> config
> > >> > > > > > >> > > > > >> > >>>>>>> systems,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues
> > that
> > >> I
> > >> > > > > outlined
> > >> > > > > > >> > above.
> > >> > > > > > >> > > It
> > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > >> > > > > > >> > > > > >> > >>>>> also
> > >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> > >> > > > dramatically.
> > >> > > > > > >> > > Supporting
> > >> > > > > > >> > > > > >> only
> > >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow
> > >> Samza
> > >> > to
> > >> > > be
> > >> > > > > > >> executed
> > >> > > > > > >> > > on
> > >> > > > > > >> > > > > YARN
> > >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > >> > > > Marathon/Aurora),
> > >> > > > > or
> > >> > > > > > >> most
> > >> > > > > > >> > > > other
> > >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > >> > > > > > >> > > > > >> > >>>>>>> deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a
> lot
> > >> > easier
> > >> > > > for
> > >> > > > > > new
> > >> > > > > > >> > > users.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > >> > > > > > >> > > > > >> > >>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without
> > YARN.
> > >> > The
> > >> > > > drop
> > >> > > > > > in
> > >> > > > > > >> > > mailing
> > >> > > > > > >> > > > > >> list
> > >> > > > > > >> > > > > >> > >>>>>> traffic
> > >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
> > >> overdue to
> > >> > > me.
> > >> > > > > The
> > >> > > > > > >> > > reality
> > >> > > > > > >> > > > > is,
> > >> > > > > > >> > > > > >> > >>>>> everyone
> > >> > > > > > >> > > > > >> > >>>>>>>>> that
> > >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
> > >> Kafka.
> > >> > We
> > >> > > > > > basically
> > >> > > > > > >> > > > require
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >>>>>> already
> > >> > > > > > >> > > > > >> > >>>>>>> in
> > >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work.
> > Those
> > >> > that
> > >> > > > are
> > >> > > > > > >> using
> > >> > > > > > >> > > > other
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into
> > >> Kafka
> > >> > > (1),
> > >> > > > > and
> > >> > > > > > >> then
> > >> > > > > > >> > > > they
> > >> > > > > > >> > > > > do
> > >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
> > >> already
> > >> > > > > > discussion (
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > >
> > >> > > > > > >> >
> > >> > > > > >
> > >> > >
> > >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >> > > > > > >> > > > > >> > >>>>> 767
> > >> > > > > > >> > > > > >> > >>>>>>>>> )
> > >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into
> > Kafka
> > >> > > > extremely
> > >> > > > > > >> easy.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple
> with
> > >> > Kafka,
> > >> > > > we
> > >> > > > > > can
> > >> > > > > > >> > > > leverage
> > >> > > > > > >> > > > > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > >> > > > > > >> > > > > >> > >>>>>>> their
> > >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> > >> maintain
> > >> > > our
> > >> > > > > own
> > >> > > > > > >> > config,
> > >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > >> > > > > > >> > > > > >> > >>>>> etc.
> > >> > > > > > >> > > > > >> > >>>>>>> We
> > >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries,
> > and
> > >> > make
> > >> > > > them
> > >> > > > > > >> > better.
> > >> > > > > > >> > > > This
> > >> > > > > > >> > > > > >> > >>>>>>>>> will
> > >> > > > > > >> > > > > >> > >>>>> also
> > >> > > > > > >> > > > > >> > >>>>>>>>> allow us to share the
> > >> consumer/producer
> > >> > > APIs,
> > >> > > > > and
> > >> > > > > > >> will
> > >> > > > > > >> > > let
> > >> > > > > > >> > > > > us
> > >> > > > > > >> > > > > >> > >>>>> leverage
> > >> > > > > > >> > > > > >> > >>>>>>>>> their offset management and
> > partition
> > >> > > > > management,
> > >> > > > > > >> > rather
> > >> > > > > > >> > > > > than
> > >> > > > > > >> > > > > >> > >>>>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>> our
> > >> > > > > > >> > > > > >> > >>>>>>>>> own. All of the coordinator stream
> > >> code
> > >> > > would
> > >> > > > > go
> > >> > > > > > >> away,
> > >> > > > > > >> > > as
> > >> > > > > > >> > > > > >> would
> > >> > > > > > >> > > > > >> > >>>>>>>>> most
> > >> > > > > > >> > > > > >> > >>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably
> > >> have
> > >> > to
> > >> > > > push
> > >> > > > > > some
> > >> > > > > > >> > > > > partition
> > >> > > > > > >> > > > > >> > >>>>>>> management
> > >> > > > > > >> > > > > >> > >>>>>>>>> features into the Kafka broker,
> but
> > >> > they're
> > >> > > > > > already
> > >> > > > > > >> > > moving
> > >> > > > > > >> > > > > in
> > >> > > > > > >> > > > > >> > >>>>>>>>> that direction with the new
> consumer
> > >> API.
> > >> > > The
> > >> > > > > > >> features
> > >> > > > > > >> > > we
> > >> > > > > > >> > > > > have
> > >> > > > > > >> > > > > >> > >>>>>>>>> for
> > >> > > > > > >> > > > > >> > >>>>>> partition
> > >> > > > > > >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza,
> > and
> > >> > seem
> > >> > > > > like
> > >> > > > > > >> they
> > >> > > > > > >> > > > should
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > >>>>>>>>> in
> > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>> anyway. There will always be some
> > >> niche
> > >> > > > usages
> > >> > > > > > which
> > >> > > > > > >> > > will
> > >> > > > > > >> > > > > >> > >>>>>>>>> require
> > >> > > > > > >> > > > > >> > >>>>>> extra
> > >> > > > > > >> > > > > >> > >>>>>>>>> care and hence full control over
> > >> > partition
> > >> > > > > > >> assignments
> > >> > > > > > >> > > > much
> > >> > > > > > >> > > > > >> > >>>>>>>>> like the
> > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>> low level consumer api. These
> would
> > >> > > continue
> > >> > > > to
> > >> > > > > > be
> > >> > > > > > >> > > > > supported.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> These items will be good for the
> > Samza
> > >> > > > > community.
> > >> > > > > > >> > > They'll
> > >> > > > > > >> > > > > make
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it
> > >> easier
> > >> > for
> > >> > > > > > >> developers
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > > add
> > >> > > > > > >> > > > > >> > >>>>>>>>> new features.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large
> > (and
> > >> > > > somewhat
> > >> > > > > > >> > backwards
> > >> > > > > > >> > > > > >> > >>>>> incompatible
> > >> > > > > > >> > > > > >> > >>>>>>>>> change). If we choose to go this
> > >> route,
> > >> > > it's
> > >> > > > > > >> important
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > we
> > >> > > > > > >> > > > > >> > >>>>> openly
> > >> > > > > > >> > > > > >> > >>>>>>>>> communicate how we're going to
> > >> provide a
> > >> > > > > > migration
> > >> > > > > > >> > path
> > >> > > > > > >> > > > from
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>> existing
> > >> > > > > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make
> > >> > > incompatible
> > >> > > > > > >> > changes).
> > >> > > > > > >> > > I
> > >> > > > > > >> > > > > >> think
> > >> > > > > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need
> to
> > >> > > provide a
> > >> > > > > > >> wrapper
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > allow
> > >> > > > > > >> > > > > >> > >>>>>>>>> existing StreamTask
> implementations
> > to
> > >> > > > continue
> > >> > > > > > >> > running
> > >> > > > > > >> > > on
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>> new container.
> > >> > > > > > >> > > > > >> > >>>>>>> It's
> > >> > > > > > >> > > > > >> > >>>>>>>>> also important that we openly
> > >> communicate
> > >> > > > about
> > >> > > > > > >> > timing,
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> > >>>>>>>>> stages
> > >> > > > > > >> > > > > >> > >>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> migration.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure
> > you
> > >> > have
> > >> > > > > > opinions.
> > >> > > > > > >> > :)
> > >> > > > > > >> > > > > Please
> > >> > > > > > >> > > > > >> > >>>>>>>>> send
> > >> > > > > > >> > > > > >> > >>>>>> your
> > >> > > > > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Cheers,
> > >> > > > > > >> > > > > >> > >>>>>>>>> Chris
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> --
> > >> > > > > > >> > > > > >> > >>>>>> -- Guozhang
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > >
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@gmail.com>.
Hey guys,

There seems to be some confusion in the last few emails: there is no plan
whatsoever to remove YARN support. The change suggested was to move the
partition management out of the YARN app master and rely on Kafka's
partition management. The advantage of this would be to make the vast
majority of Samza totally cluster manager agnostic, and make it possible to
implement a high quality "stand-alone" mode and support other frameworks.
It is possible this will let us run in YARN without *any* samza-specific
code using something generic like Slider, or maybe not. I don't think
anyone has tried this so it is probably premature to say how much
YARN-specific code could be killed. If that works out, then a bunch of the
YARN-specific code will disappear, but this doesn't mean YARN support will
disappear, just that we would retain less code to implement the same
thing.  Either way Samza in YARN isn't going away.

-Jay

On Sun, Jul 12, 2015 at 7:48 PM, Garrett Barton <ga...@gmail.com>
wrote:

> Yi,
>
>  What you just summarized makes a whole lot more sense to me.  Shamelessly
> I am looking at this shift as a customer with a production workflow riding
> on it so I am looking for some kind of consistency into the future of
> Samza.  This makes me feel a lot better about it.
>
>
> Thank you!
>
> On Sun, Jul 12, 2015 at 10:44 PM, Yi Pan <ni...@gmail.com> wrote:
>
> > Just to make it explicitly clear what I am proposing, here is a version
> of
> > more detailed description:
> >
> > The fourth option (in addition to what Jakob summarized) we are proposing
> > is:
> >
> > - Recharter Samza to “stream processing as a service”
> >
> > - The current Samza core (the basic transformation API w/ basic partition
> > and offset management build-in) will be moved to Kafka Streams (i.e. part
> > of Kafka) and supports “run-as-a-library”
> >
> > - Deprecate the SystemConsumers and SystemProducers APIs and move them to
> > Copycat
> >
> > - The current SQL development:
> >
> >    * physical operators and a Trident-like stream API should stay in
> Kafka
> > Streams as libraries, enabling any standalone deployment to use the core
> > window/join functions
> >
> >    * query parser/planner and execution on top of a distributed service
> > should stay in new Samza (i.e. “stream processing as a service”)
> >
> > - Advanced features related to job scheduling/state management stays in
> new
> > Samza (i.e. “streaming processing as a service”)
> >
> >    * Any advanced PartitionManager implementation that can be plugged
> into
> > Kafka Streams
> >
> >    * Any auto-scaling, dynamic configuration via coordinator stream
> >
> >    * Any advanced state management s.t. host-affinity etc.
> >
> >
> > Pros:
> >
> > - W/ the current Samza core as Kafka Streams and move the ingestion to
> > Copycat, we achieved most of the goals in the initial proposal:
> >
> >    * Tighter coupling w/ Kafka
> >
> >    * Reuse Kafka’s build-in functionalities, such as offset manager,
> basic
> > partition distribution
> >
> >    * Separation of ingestion vs transformation APIs
> >
> >    * offload a lot of system-specific configuration to Kafka Streams and
> > Copycat (i.e. SystemFactory configure, serde configure, etc.)
> >
> >    * remove YARN dependency and make standalone deployment easy. As
> > Guozhang mentioned, it would be really easy to start a process that
> > internally run Kafka Streams as library.
> >
> > - By re-chartering Samza as “stream processing as a service”, we address
> > the concern regarding to
> >
> >    * Pluggable partition management
> >
> >    * Running in a distributed cluster to manage process lifecycle,
> > fault-tolerance, resource-allocation, etc.
> >
> >    * More advanced features s.t. host-affinity, auto-scaling, and dynamic
> > configure changes, etc.
> >
> >
> > Regarding to the code and community organization, I think the following
> may
> > be the best:
> >
> > Code:
> >
> > - A Kafka sub-project Kafka Streams to hold samza-core, samza-kv-store,
> and
> > the physical operator layer as library in SQL: this would allow better
> > alignment w/ Kafka, in code, doc, and branding
> >
> > - Retain the current Samza project just to keep
> >
> >    * A pluggable explicit partition management in Kafka Streams client
> >
> >    * Integration w/ cluster-management systems for advanced features:
> >
> >       * host-affinity, auto-scaling,, dynamic configuration, etc.
> >
> >    * It will fully depend on the Kafka Streams API and remove all support
> > for SystemConsumers/SystemProducers in the future
> >
> > Community: (this is almost the same as what Chris proposed)
> >
> > - Kafka Streams: the current Samza community should be supporting this
> > effort together with some Kafka members, since most of the code here will
> > be from samza-core, samza-kv-store, and samza-sql.
> >
> > - new Samza: the current Samza community should continue serve the course
> > to support more advanced features to run Kafka Streams as a service.
> > Arguably, the new Samza framework may be used to run Copycat workers as
> > well, at least to manage Copycat worker’s lifecycle in a clustered
> > environment. Hence, it would stay as a general stream processing
> framework
> > that takes in any source and output to any destination, just the
> transport
> > system is fixed to Kafka.
> >
> > On Sun, Jul 12, 2015 at 7:29 PM, Yi Pan <ni...@gmail.com> wrote:
> >
> > > Hi, Chris,
> > >
> > > Thanks for sending out this concrete set of points here. I agree w/ all
> > > but have a slight different point view on 8).
> > >
> > > My view on this is: instead of sunset Samza as TLP, can we re-charter
> the
> > > scope of Samza to be the home for "running streaming process as a
> > service"?
> > >
> > > My main motivation is from the following points from a long internal
> > > discussion in LinkedIn:
> > >
> > > - There is a clear ask for pluggable partition management, like we do
> in
> > > LinkedIn, and as Ben Kirwin has mentioned in
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCACuX-D-yJX++2GnF_1Laf10KYUVyamg7UP_Dt19v0ZNmmhbCPQ@mail.gmail.com%3E
> > > - There are concerns on lack of support for running stream processing
> in
> > a
> > > cluster: lifecycle management, resource allocation, fault tolerance,
> etc.
> > > - There is a question to how to support more advanced features s.t.
> > > host-affinity, auto-scaling, and dynamic configuration in Samza jobs,
> as
> > > raised by Martin here:
> > >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3C0D66EFD0-B7CD-4E4E-8B2F-2716167C313C@kleppmann.com%3E
> > >
> > > We have use cases that need to address all the above three cases and
> most
> > > of the functions are all in the current Samza project, in some flavor.
> We
> > > are all supporting to merge the samza-core functionalities into Kafka
> > > Streams, but there is a question where we keep these functions in the
> > > future. One option is to start a new project that includes these
> > functions
> > > that are closely related w/ "run stream-processing as-a-service", while
> > > another personally more attractive option is to re-charter Samza
> project
> > > just do "run stream processing as-a-service". We can avoid the overhead
> > of
> > > re-starting another community for this project. Personally, I felt that
> > > here are the benefits we should be getting:
> > >
> > > 1. We have already agreed mostly that Kafka Streams API would allow
> some
> > > pluggable partition management functions. Hence, the advanced partition
> > > management can live out-side the new Kafka Streams core w/o affecting
> the
> > > run-as-a-library model in Kafka Streams.
> > > 2. The integration w/ cluster management system and advanced features
> > > listed above stays in the same project and allow existing users enjoy
> > > no-impact migration to Kafka Stream as the core. That also addresses
> > Tim's
> > > question on "removing the support for YARN".
> > > 3. A separate project for stream-processing-as-a-service also allow the
> > > new Kafka Streams being independent to any cluster management and just
> > > focusing on stream process core functions, while leaving the functions
> > that
> > > requires cluster-resource and state management to a separate layer.
> > >
> > > Please feel free to comment. Thanks!
> > >
> > > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <
> criccomini@apache.org>
> > > wrote:
> > >
> > >> Hey all,
> > >>
> > >> I want to start by saying that I'm absolutely thrilled to be a part of
> > >> this
> > >> community. The amount of level-headed, thoughtful, educated discussion
> > >> that's gone on over the past ~10 days is overwhelming. Wonderful.
> > >>
> > >> It seems like discussion is waning a bit, and we've reached some
> > >> conclusions. There are several key emails in this threat, which I want
> > to
> > >> call out:
> > >>
> > >> 1. Jakob's summary of the three potential ways forward.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > >> 2. Julian's call out that we should be focusing on community over
> code.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > >> 3. Martin's summary about the benefits of merging communities.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > >> 4. Jakob's comments about the distinction between community and code
> > >> paths.
> > >>
> > >>
> > >>
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> > >>
> > >> I agree with the comments on all of these emails. I think Martin's
> > summary
> > >> of his position aligns very closely with my own. To that end, I think
> we
> > >> should get concrete about what the proposal is, and call a vote on it.
> > >> Given that Jay, Martin, and I seem to be aligning fairly closely, I
> > think
> > >> we should start with:
> > >>
> > >> 1. [community] Make Samza a subproject of Kafka.
> > >> 2. [community] Make all Samza PMC/committers committers of the
> > subproject.
> > >> 3. [community] Migrate Samza's website/documentation into Kafka's.
> > >> 4. [code] Have the Samza community and the Kafka community start a
> > >> from-scratch reboot together in the new Kafka subproject. We can
> > >> borrow/copy &  paste significant chunks of code from Samza's code
> base.
> > >> 5. [code] The subproject would intentionally eliminate support for
> both
> > >> other streaming systems and all deployment systems.
> > >> 6. [code] Attempt to provide a bridge from our SystemConsumer to
> KIP-26
> > >> (copy cat)
> > >> 7. [code] Attempt to provide a bridge from the new subproject's
> > processor
> > >> interface to our legacy StreamTask interface.
> > >> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> > >> subproject that has a fault-tolerant container with state management.
> > >>
> > >> It's likely that (6) and (7) won't be fully drop-in. Still, the closer
> > we
> > >> can get, the better it's going to be for our existing community.
> > >>
> > >> One thing that I didn't touch on with (2) is whether any Samza PMC
> > members
> > >> should be rolled into Kafka PMC membership as well (though, Jay and
> > Jakob
> > >> are already PMC members on both). I think that Samza's community
> > deserves
> > >> a
> > >> voice on the PMC, so I'd propose that we roll at least a few PMC
> members
> > >> into the Kafka PMC, but I don't have a strong framework for which
> people
> > >> to
> > >> pick.
> > >>
> > >> Before (8), I think that Samza's TLP can continue to commit bug fixes
> > and
> > >> patches as it sees fit, provided that we openly communicate that we
> > won't
> > >> necessarily migrate new features to the new subproject, and that the
> TLP
> > >> will be shut down after the migration to the Kafka subproject occurs.
> > >>
> > >> Jakob, I could use your guidance here about about how to achieve this
> > from
> > >> an Apache process perspective (sorry).
> > >>
> > >> * Should I just call a vote on this proposal?
> > >> * Should it happen on dev or private?
> > >> * Do committers have binding votes, or just PMC?
> > >>
> > >> Having trouble finding much detail on the Apache wikis. :(
> > >>
> > >> Cheers,
> > >> Chris
> > >>
> > >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com>
> wrote:
> > >>
> > >> > Thanks, Jay. This argument persuaded me actually. :)
> > >> >
> > >> > Fang, Yan
> > >> > yanfang724@gmail.com
> > >> >
> > >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > >> >
> > >> > > Hey Yan,
> > >> > >
> > >> > > Yeah philosophically I think the argument is that you should
> capture
> > >> the
> > >> > > stream in Kafka independent of the transformation. This is
> > obviously a
> > >> > > Kafka-centric view point.
> > >> > >
> > >> > > Advantages of this:
> > >> > > - In practice I think this is what e.g. Storm people often end up
> > >> doing
> > >> > > anyway. You usually need to throttle any access to a live serving
> > >> > database.
> > >> > > - Can have multiple subscribers and they get the same thing
> without
> > >> > > additional load on the source system.
> > >> > > - Applications can tap into the stream if need be by subscribing.
> > >> > > - You can debug your transformation by tailing the Kafka topic
> with
> > >> the
> > >> > > console consumer
> > >> > > - Can tee off the same data stream for batch analysis or Lambda
> arch
> > >> > style
> > >> > > re-processing
> > >> > >
> > >> > > The disadvantage is that it will use Kafka resources. But the idea
> > is
> > >> > > eventually you will have multiple subscribers to any data source
> (at
> > >> > least
> > >> > > for monitoring) so you will end up there soon enough anyway.
> > >> > >
> > >> > > Down the road the technical benefit is that I think it gives us a
> > good
> > >> > path
> > >> > > towards end-to-end exactly once semantics from source to
> > destination.
> > >> > > Basically the connectors need to support idempotence when talking
> to
> > >> > Kafka
> > >> > > and we need the transactional write feature in Kafka to make the
> > >> > > transformation atomic. This is actually pretty doable if you
> > separate
> > >> > > connector=>kafka problem from the generic transformations which
> are
> > >> > always
> > >> > > kafka=>kafka. However I think it is quite impossible to do in a
> > >> > all_things
> > >> > > => all_things environment. Today you can say "well the semantics
> of
> > >> the
> > >> > > Samza APIs depend on the connectors you use" but it is actually
> > worse
> > >> > then
> > >> > > that because the semantics actually depend on the pairing of
> > >> > connectors--so
> > >> > > not only can you probably not get a usable "exactly once"
> guarantee
> > >> > > end-to-end it can actually be quite hard to reverse engineer what
> > >> > property
> > >> > > (if any) your end-to-end flow has if you have heterogenous
> systems.
> > >> > >
> > >> > > -Jay
> > >> > >
> > >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > > > {quote}
> > >> > > > maintained in a separate repository and retaining the existing
> > >> > > > committership but sharing as much else as possible (website,
> etc)
> > >> > > > {quote}
> > >> > > >
> > >> > > > Overall, I agree on this idea. Now the question is more about
> "how
> > >> to
> > >> > do
> > >> > > > it".
> > >> > > >
> > >> > > > On the other hand, one thing I want to point out is that, if we
> > >> decide
> > >> > to
> > >> > > > go this way, how do we want to support
> > >> > > > otherSystem-transformation-otherSystem use case?
> > >> > > >
> > >> > > > Basically, there are four user groups here:
> > >> > > >
> > >> > > > 1. Kafka-transformation-Kafka
> > >> > > > 2. Kafka-transformation-otherSystem
> > >> > > > 3. otherSystem-transformation-Kafka
> > >> > > > 4. otherSystem-transformation-otherSystem
> > >> > > >
> > >> > > > For group 1, they can easily use the new Samza library to
> achieve.
> > >> For
> > >> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka
> or
> > >> > Kafka->
> > >> > > > transformation -> copyCat.
> > >> > > >
> > >> > > > The problem is for group 4. Do we want to abandon this or still
> > >> support
> > >> > > it?
> > >> > > > Of course, this use case can be achieved by using copyCat ->
> > >> > > transformation
> > >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> > persuade
> > >> > them
> > >> > > to
> > >> > > > do this long chain. If yes, it will also be a win for Kafka too.
> > Or
> > >> if
> > >> > > > there is no one in this community actually doing this so far,
> > maybe
> > >> ok
> > >> > to
> > >> > > > not support the group 4 directly.
> > >> > > >
> > >> > > > Thanks,
> > >> > > >
> > >> > > > Fang, Yan
> > >> > > > yanfang724@gmail.com
> > >> > > >
> > >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
> > >> wrote:
> > >> > > >
> > >> > > > > Yeah I agree with this summary. I think there are kind of two
> > >> > questions
> > >> > > > > here:
> > >> > > > > 1. Technically does alignment/reliance on Kafka make sense
> > >> > > > > 2. Branding wise (naming, website, concepts, etc) does
> alignment
> > >> with
> > >> > > > Kafka
> > >> > > > > make sense
> > >> > > > >
> > >> > > > > Personally I do think both of these things would be really
> > >> valuable,
> > >> > > and
> > >> > > > > would dramatically alter the trajectory of the project.
> > >> > > > >
> > >> > > > > My preference would be to see if people can mostly agree on a
> > >> > direction
> > >> > > > > rather than splintering things off. From my point of view the
> > >> ideal
> > >> > > > outcome
> > >> > > > > of all the options discussed would be to make Samza a closely
> > >> aligned
> > >> > > > > subproject, maintained in a separate repository and retaining
> > the
> > >> > > > existing
> > >> > > > > committership but sharing as much else as possible (website,
> > >> etc). No
> > >> > > > idea
> > >> > > > > about how these things work, Jacob, you probably know more.
> > >> > > > >
> > >> > > > > No discussion amongst the Kafka folks has happened on this,
> but
> > >> > likely
> > >> > > we
> > >> > > > > should figure out what the Samza community actually wants
> first.
> > >> > > > >
> > >> > > > > I admit that this is a fairly radical departure from how
> things
> > >> are.
> > >> > > > >
> > >> > > > > If that doesn't fly, I think, yeah we could leave Samza as it
> is
> > >> and
> > >> > do
> > >> > > > the
> > >> > > > > more radical reboot inside Kafka. From my point of view that
> > does
> > >> > leave
> > >> > > > > things in a somewhat confusing state since now there are two
> > >> stream
> > >> > > > > processing systems more or less coupled to Kafka in large part
> > >> made
> > >> > by
> > >> > > > the
> > >> > > > > same people. But, arguably that might be a cleaner way to make
> > the
> > >> > > > cut-over
> > >> > > > > and perhaps less risky for Samza community since if it works
> > >> people
> > >> > can
> > >> > > > > switch and if it doesn't nothing will have changed. Dunno, how
> > do
> > >> > > people
> > >> > > > > feel about this?
> > >> > > > >
> > >> > > > > -Jay
> > >> > > > >
> > >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> > jghoman@gmail.com>
> > >> > > wrote:
> > >> > > > >
> > >> > > > > > >  This leads me to thinking that merging projects and
> > >> communities
> > >> > > > might
> > >> > > > > > be a good idea: with the union of experience from both
> > >> communities,
> > >> > > we
> > >> > > > > will
> > >> > > > > > probably build a better system that is better for users.
> > >> > > > > > Is this what's being proposed though? Merging the projects
> > seems
> > >> > like
> > >> > > > > > a consequence of at most one of the three directions under
> > >> > > discussion:
> > >> > > > > > 1) Samza 2.0: The Samza community relies more heavily on
> Kafka
> > >> for
> > >> > > > > > configuration, etc. (to a greater or lesser extent to be
> > >> > determined)
> > >> > > > > > but the Samza community would not automatically merge withe
> > >> Kafka
> > >> > > > > > community (the Phoenix/HBase example is a good one here).
> > >> > > > > > 2) Samza Reboot: The Samza community continues to exist
> with a
> > >> > > limited
> > >> > > > > > project scope, but similarly would not need to be part of
> the
> > >> Kafka
> > >> > > > > > community (ie given committership) to progress.  Here, maybe
> > the
> > >> > > Samza
> > >> > > > > > team would become a subproject of Kafka (the Board frowns on
> > >> > > > > > subprojects at the moment, so I'm not sure if that's even
> > >> > feasible),
> > >> > > > > > but that would not be required.
> > >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option
> > the
> > >> > Kafka
> > >> > > > > > team builds its own streaming library, possibly off of Jay's
> > >> > > > > > prototype, which has not direct lineage to the Samza team.
> > >> There's
> > >> > > no
> > >> > > > > > reason for the Kafka team to bring in the Samza team.
> > >> > > > > >
> > >> > > > > > Is the Kafka community on board with this?
> > >> > > > > >
> > >> > > > > > To be clear, all three options under discussion are
> > interesting,
> > >> > > > > > technically valid and likely healthy directions for the
> > project.
> > >> > > > > > Also, they are not mutually exclusive.  The Samza community
> > >> could
> > >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka
> community
> > >> went
> > >> > > > > > forward with 'Hey Samza!'  My points above are directed
> > >> entirely at
> > >> > > > > > the community aspect of these choices.
> > >> > > > > > -Jakob
> > >> > > > > >
> > >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> > roger.hoover@gmail.com>
> > >> > > wrote:
> > >> > > > > > > That's great.  Thanks, Jay.
> > >> > > > > > >
> > >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> > jay@confluent.io>
> > >> > > wrote:
> > >> > > > > > >
> > >> > > > > > >> Yeah totally agree. I think you have this issue even
> today,
> > >> > right?
> > >> > > > > I.e.
> > >> > > > > > if
> > >> > > > > > >> you need to make a simple config change and you're
> running
> > in
> > >> > YARN
> > >> > > > > today
> > >> > > > > > >> you end up bouncing the job which then rebuilds state. I
> > >> think
> > >> > the
> > >> > > > fix
> > >> > > > > > is
> > >> > > > > > >> exactly what you described which is to have a long
> timeout
> > on
> > >> > > > > partition
> > >> > > > > > >> movement for stateful jobs so that if a job is just
> getting
> > >> > > bounced,
> > >> > > > > and
> > >> > > > > > >> the cluster manager (or admin) is smart enough to restart
> > it
> > >> on
> > >> > > the
> > >> > > > > same
> > >> > > > > > >> host when possible, it can optimistically reuse any
> > existing
> > >> > state
> > >> > > > it
> > >> > > > > > finds
> > >> > > > > > >> on disk (if it is valid).
> > >> > > > > > >>
> > >> > > > > > >> So in this model the charter of the CM is to place
> > processes
> > >> as
> > >> > > > > > stickily as
> > >> > > > > > >> possible and to restart or re-place failed processes. The
> > >> > charter
> > >> > > of
> > >> > > > > the
> > >> > > > > > >> partition management system is to control the assignment
> of
> > >> work
> > >> > > to
> > >> > > > > > these
> > >> > > > > > >> processes. The nice thing about this is that the work
> > >> > assignment,
> > >> > > > > > timeouts,
> > >> > > > > > >> behavior, configs, and code will all be the same across
> all
> > >> > > cluster
> > >> > > > > > >> managers.
> > >> > > > > > >>
> > >> > > > > > >> So I think that prototype would actually give you exactly
> > >> what
> > >> > you
> > >> > > > > want
> > >> > > > > > >> today for any cluster manager (or manual placement +
> > restart
> > >> > > script)
> > >> > > > > > that
> > >> > > > > > >> was sticky in terms of host placement since there is
> > already
> > >> a
> > >> > > > > > configurable
> > >> > > > > > >> partition movement timeout and task-by-task state reuse
> > with
> > >> a
> > >> > > check
> > >> > > > > on
> > >> > > > > > >> state validity.
> > >> > > > > > >>
> > >> > > > > > >> -Jay
> > >> > > > > > >>
> > >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > >> > > > roger.hoover@gmail.com
> > >> > > > > >
> > >> > > > > > >> wrote:
> > >> > > > > > >>
> > >> > > > > > >> > That would be great to let Kafka do as much heavy
> lifting
> > >> as
> > >> > > > > possible
> > >> > > > > > and
> > >> > > > > > >> > make it easier for other languages to implement Samza
> > apis.
> > >> > > > > > >> >
> > >> > > > > > >> > One thing to watch out for is the interplay between
> > Kafka's
> > >> > > group
> > >> > > > > > >> > management and the external scheduler/process manager's
> > >> fault
> > >> > > > > > tolerance.
> > >> > > > > > >> > If a container dies, the Kafka group membership
> protocol
> > >> will
> > >> > > try
> > >> > > > to
> > >> > > > > > >> assign
> > >> > > > > > >> > it's tasks to other containers while at the same time
> the
> > >> > > process
> > >> > > > > > manager
> > >> > > > > > >> > is trying to relaunch the container.  Without some
> > >> > consideration
> > >> > > > for
> > >> > > > > > this
> > >> > > > > > >> > (like a configurable amount of time to wait before
> Kafka
> > >> > alters
> > >> > > > the
> > >> > > > > > group
> > >> > > > > > >> > membership), there may be thrashing going on which is
> > >> > especially
> > >> > > > bad
> > >> > > > > > for
> > >> > > > > > >> > containers with large amounts of local state.
> > >> > > > > > >> >
> > >> > > > > > >> > Someone else pointed this out already but I thought it
> > >> might
> > >> > be
> > >> > > > > worth
> > >> > > > > > >> > calling out again.
> > >> > > > > > >> >
> > >> > > > > > >> > Cheers,
> > >> > > > > > >> >
> > >> > > > > > >> > Roger
> > >> > > > > > >> >
> > >> > > > > > >> >
> > >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> > >> jay@confluent.io>
> > >> > > > > wrote:
> > >> > > > > > >> >
> > >> > > > > > >> > > Hey Roger,
> > >> > > > > > >> > >
> > >> > > > > > >> > > I couldn't agree more. We spent a bunch of time
> talking
> > >> to
> > >> > > > people
> > >> > > > > > and
> > >> > > > > > >> > that
> > >> > > > > > >> > > is exactly the stuff we heard time and again. What
> > makes
> > >> it
> > >> > > > hard,
> > >> > > > > of
> > >> > > > > > >> > > course, is that there is some tension between
> > >> compatibility
> > >> > > with
> > >> > > > > > what's
> > >> > > > > > >> > > there now and making things better for new users.
> > >> > > > > > >> > >
> > >> > > > > > >> > > I also strongly agree with the importance of
> > >> multi-language
> > >> > > > > > support. We
> > >> > > > > > >> > are
> > >> > > > > > >> > > talking now about Java, but for application
> development
> > >> use
> > >> > > > cases
> > >> > > > > > >> people
> > >> > > > > > >> > > want to work in whatever language they are using
> > >> elsewhere.
> > >> > I
> > >> > > > > think
> > >> > > > > > >> > moving
> > >> > > > > > >> > > to a model where Kafka itself does the group
> > membership,
> > >> > > > lifecycle
> > >> > > > > > >> > control,
> > >> > > > > > >> > > and partition assignment has the advantage of putting
> > all
> > >> > that
> > >> > > > > > complex
> > >> > > > > > >> > > stuff behind a clean api that the clients are already
> > >> going
> > >> > to
> > >> > > > be
> > >> > > > > > >> > > implementing for their consumer, so the added
> > >> functionality
> > >> > > for
> > >> > > > > > stream
> > >> > > > > > >> > > processing beyond a consumer becomes very minor.
> > >> > > > > > >> > >
> > >> > > > > > >> > > -Jay
> > >> > > > > > >> > >
> > >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > >> > > > > > roger.hoover@gmail.com>
> > >> > > > > > >> > > wrote:
> > >> > > > > > >> > >
> > >> > > > > > >> > > > Metamorphosis...nice. :)
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > This has been a great discussion.  As a user of
> Samza
> > >> > who's
> > >> > > > > > recently
> > >> > > > > > >> > > > integrated it into a relatively large
> organization, I
> > >> just
> > >> > > > want
> > >> > > > > to
> > >> > > > > > >> add
> > >> > > > > > >> > > > support to a few points already made.
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
> > >> currently
> > >> > > > exists
> > >> > > > > > that
> > >> > > > > > >> > I've
> > >> > > > > > >> > > > experienced are:
> > >> > > > > > >> > > > 1) YARN - YARN is overly complex in many
> environments
> > >> > where
> > >> > > > > Puppet
> > >> > > > > > >> > would
> > >> > > > > > >> > > do
> > >> > > > > > >> > > > just fine but it was the only mechanism to get
> fault
> > >> > > > tolerance.
> > >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> > >> configuring
> > >> > > most
> > >> > > > > of
> > >> > > > > > the
> > >> > > > > > >> > job
> > >> > > > > > >> > > > in code rather than config files.  In general, I
> > think
> > >> the
> > >> > > > goal
> > >> > > > > > >> should
> > >> > > > > > >> > be
> > >> > > > > > >> > > > to make it harder to make mistakes, especially of
> the
> > >> kind
> > >> > > > where
> > >> > > > > > the
> > >> > > > > > >> > code
> > >> > > > > > >> > > > expects something and the config doesn't match.
> The
> > >> > current
> > >> > > > > > config
> > >> > > > > > >> is
> > >> > > > > > >> > > > quite intricate and error-prone.  For example, the
> > >> > > application
> > >> > > > > > logic
> > >> > > > > > >> > may
> > >> > > > > > >> > > > depend on bootstrapping a topic but rather than
> > >> asserting
> > >> > > that
> > >> > > > > in
> > >> > > > > > the
> > >> > > > > > >> > > code,
> > >> > > > > > >> > > > you have to rely on getting the config right.
> > Likewise
> > >> > with
> > >> > > > > > serdes,
> > >> > > > > > >> > the
> > >> > > > > > >> > > > Java representations produced by various serdes
> > (JSON,
> > >> > Avro,
> > >> > > > > etc.)
> > >> > > > > > >> are
> > >> > > > > > >> > > not
> > >> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> > >> without
> > >> > > > > changing
> > >> > > > > > >> the
> > >> > > > > > >> > > > code.   It would be nice for jobs to be able to
> > assert
> > >> > what
> > >> > > > they
> > >> > > > > > >> expect
> > >> > > > > > >> > > > from their input topics in terms of partitioning.
> > >> This is
> > >> > > > > > getting a
> > >> > > > > > >> > > little
> > >> > > > > > >> > > > off topic but I was even thinking about creating a
> > >> "Samza
> > >> > > > config
> > >> > > > > > >> > linter"
> > >> > > > > > >> > > > that would sanity check a set of configs.
> Especially
> > >> in
> > >> > > > > > >> organizations
> > >> > > > > > >> > > > where config is managed by a different team than
> the
> > >> > > > application
> > >> > > > > > >> > > developer,
> > >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > >> > > > > > >> > > > 3) Java/Scala centric - for many teams (especially
> > >> > > DevOps-type
> > >> > > > > > >> folks),
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > pain of the Java toolchain (maven, slow builds,
> weak
> > >> > command
> > >> > > > > line
> > >> > > > > > >> > > support,
> > >> > > > > > >> > > > configuration over convention) really inhibits
> > >> > productivity.
> > >> > > > As
> > >> > > > > > more
> > >> > > > > > >> > and
> > >> > > > > > >> > > > more high-quality clients become available for
> > Kafka, I
> > >> > hope
> > >> > > > > > they'll
> > >> > > > > > >> > > follow
> > >> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> > >> proposals
> > >> > > in
> > >> > > > > this
> > >> > > > > > >> > thread
> > >> > > > > > >> > > > but please consider other languages in the
> ecosystem
> > as
> > >> > > well.
> > >> > > > > > From
> > >> > > > > > >> > what
> > >> > > > > > >> > > > I've heard, Spark has more Python users than
> > >> Java/Scala.
> > >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > >
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > >> > > > > > >> > > > and are working on a Yeoman generator
> > >> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> > >> > > Jython/Samza
> > >> > > > > > >> projects
> > >> > > > > > >> > to
> > >> > > > > > >> > > > alleviate some of the pain)
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > I also want to underscore Jay's point about
> improving
> > >> the
> > >> > > user
> > >> > > > > > >> > > experience.
> > >> > > > > > >> > > > That's a very important factor for adoption.  I
> think
> > >> the
> > >> > > goal
> > >> > > > > > should
> > >> > > > > > >> > be
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > make Samza as easy to get started with as something
> > >> like
> > >> > > > > Logstash.
> > >> > > > > > >> > > > Logstash is vastly inferior in terms of
> capabilities
> > to
> > >> > > Samza
> > >> > > > > but
> > >> > > > > > >> it's
> > >> > > > > > >> > > easy
> > >> > > > > > >> > > > to get started and that makes a big difference.
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > Cheers,
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > Roger
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> > Francisci
> > >> > > > Morales <
> > >> > > > > > >> > > > gdfm@apache.org> wrote:
> > >> > > > > > >> > > >
> > >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> > >> Metamorphosis
> > >> > > is
> > >> > > > a
> > >> > > > > > clear
> > >> > > > > > >> > > > winner
> > >> > > > > > >> > > > > :)
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > --
> > >> > > > > > >> > > > > Gianmarco
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> > >> Morales
> > >> > <
> > >> > > > > > >> > > gdfm@apache.org
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > wrote:
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > > > > Hi,
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > @Martin, thanks for you comments.
> > >> > > > > > >> > > > > > Maybe I'm missing some important point, but I
> > think
> > >> > > > coupling
> > >> > > > > > the
> > >> > > > > > >> > > > releases
> > >> > > > > > >> > > > > > is actually a *good* thing.
> > >> > > > > > >> > > > > > To make an example, would it be better if the
> MR
> > >> and
> > >> > > HDFS
> > >> > > > > > >> > components
> > >> > > > > > >> > > of
> > >> > > > > > >> > > > > > Hadoop had different release schedules?
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > Actually, keeping the discussion in a single
> > place
> > >> > would
> > >> > > > > make
> > >> > > > > > >> > > agreeing
> > >> > > > > > >> > > > on
> > >> > > > > > >> > > > > > releases (and backwards compatibility) much
> > >> easier, as
> > >> > > > > > everybody
> > >> > > > > > >> > > would
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > > responsible for the whole codebase.
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > That said, I like the idea of absorbing
> > samza-core
> > >> as
> > >> > a
> > >> > > > > > >> > sub-project,
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > > leave the fancy stuff separate.
> > >> > > > > > >> > > > > > It probably gives 90% of the benefits we have
> > been
> > >> > > > > discussing
> > >> > > > > > >> here.
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > Cheers,
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > --
> > >> > > > > > >> > > > > > Gianmarco
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > >> > jay.kreps@gmail.com
> > >> > > >
> > >> > > > > > wrote:
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >> Hey Martin,
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> I agree coupling release schedules is a
> > downside.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> Definitely we can try to solve some of the
> > >> > integration
> > >> > > > > > problems
> > >> > > > > > >> in
> > >> > > > > > >> > > > > >> Confluent Platform or in other distributions.
> > But
> > >> I
> > >> > > think
> > >> > > > > > this
> > >> > > > > > >> > ends
> > >> > > > > > >> > > up
> > >> > > > > > >> > > > > >> being really shallow. I guess I feel to really
> > >> get a
> > >> > > good
> > >> > > > > > user
> > >> > > > > > >> > > > > experience
> > >> > > > > > >> > > > > >> the two systems have to kind of feel like part
> > of
> > >> the
> > >> > > > same
> > >> > > > > > thing
> > >> > > > > > >> > and
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> can't really add that in later--you can put
> both
> > >> in
> > >> > the
> > >> > > > > same
> > >> > > > > > >> > > > > downloadable
> > >> > > > > > >> > > > > >> tar file but it doesn't really give a very
> > >> cohesive
> > >> > > > > feeling.
> > >> > > > > > I
> > >> > > > > > >> > agree
> > >> > > > > > >> > > > > that
> > >> > > > > > >> > > > > >> ultimately any of the project stuff is as much
> > >> social
> > >> > > and
> > >> > > > > > naming
> > >> > > > > > >> > as
> > >> > > > > > >> > > > > >> anything else--theoretically two totally
> > >> independent
> > >> > > > > projects
> > >> > > > > > >> > could
> > >> > > > > > >> > > > work
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> tightly align. In practice this seems to be
> > quite
> > >> > > > difficult
> > >> > > > > > >> > though.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> For the frameworks--totally agree it would be
> > >> good to
> > >> > > > > > maintain
> > >> > > > > > >> the
> > >> > > > > > >> > > > > >> framework support with the project. In some
> > cases
> > >> > there
> > >> > > > may
> > >> > > > > > not
> > >> > > > > > >> be
> > >> > > > > > >> > > too
> > >> > > > > > >> > > > > >> much
> > >> > > > > > >> > > > > >> there since the integration gets lighter but I
> > >> think
> > >> > > > > whatever
> > >> > > > > > >> > stubs
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> need should be included. So no I definitely
> > wasn't
> > >> > > trying
> > >> > > > > to
> > >> > > > > > >> imply
> > >> > > > > > >> > > > > >> dropping
> > >> > > > > > >> > > > > >> support for these frameworks, just making the
> > >> > > integration
> > >> > > > > > >> lighter
> > >> > > > > > >> > by
> > >> > > > > > >> > > > > >> separating process management from partition
> > >> > > management.
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> You raise two good points we would have to
> > figure
> > >> out
> > >> > > if
> > >> > > > we
> > >> > > > > > went
> > >> > > > > > >> > > down
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> alignment path:
> > >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think the
> > >> first
> > >> > > > > question
> > >> > > > > > is
> > >> > > > > > >> > > > whether
> > >> > > > > > >> > > > > >> some "re-branding" would be worth it. If so
> > then I
> > >> > > think
> > >> > > > we
> > >> > > > > > can
> > >> > > > > > >> > > have a
> > >> > > > > > >> > > > > big
> > >> > > > > > >> > > > > >> thread on the name. I'm definitely not set on
> > >> Kafka
> > >> > > > > > Streaming or
> > >> > > > > > >> > > Kafka
> > >> > > > > > >> > > > > >> Streams I was just using them to be kind of
> > >> > > > illustrative. I
> > >> > > > > > >> agree
> > >> > > > > > >> > > with
> > >> > > > > > >> > > > > >> your
> > >> > > > > > >> > > > > >> critique of these names, though I think people
> > >> would
> > >> > > get
> > >> > > > > the
> > >> > > > > > >> idea.
> > >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about how
> to
> > >> > > "factor"
> > >> > > > > it.
> > >> > > > > > >> Here
> > >> > > > > > >> > > are
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> options I see (I could get enthusiastic about
> > any
> > >> of
> > >> > > > them):
> > >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> > seperation
> > >> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api
> and
> > >> > > > samza-core
> > >> > > > > > is
> > >> > > > > > >> > > > absorbed
> > >> > > > > > >> > > > > >> almost like a third client
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> Cheers,
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> -Jay
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> > Kleppmann <
> > >> > > > > > >> > > > martin@kleppmann.com>
> > >> > > > > > >> > > > > >> wrote:
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a
> few
> > >> > > follow-up
> > >> > > > > > >> > comments.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
> > >> > becoming
> > >> > > a
> > >> > > > > > >> > subproject:
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > reasons you mention are good. The risk I see
> > is
> > >> > that
> > >> > > > > > release
> > >> > > > > > >> > > > schedules
> > >> > > > > > >> > > > > >> > become coupled to each other, which can slow
> > >> > everyone
> > >> > > > > down,
> > >> > > > > > >> and
> > >> > > > > > >> > > > large
> > >> > > > > > >> > > > > >> > projects with many contributors are harder
> to
> > >> > manage.
> > >> > > > > > (Jakob,
> > >> > > > > > >> > can
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> speak
> > >> > > > > > >> > > > > >> > from experience, having seen a wider range
> of
> > >> > Hadoop
> > >> > > > > > ecosystem
> > >> > > > > > >> > > > > >> projects?)
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > Some of the goals of a better unified
> > developer
> > >> > > > > experience
> > >> > > > > > >> could
> > >> > > > > > >> > > > also
> > >> > > > > > >> > > > > be
> > >> > > > > > >> > > > > >> > solved by integrating Samza nicely into a
> > Kafka
> > >> > > > > > distribution
> > >> > > > > > >> > (such
> > >> > > > > > >> > > > as
> > >> > > > > > >> > > > > >> > Confluent's). I'm not against merging
> projects
> > >> if
> > >> > we
> > >> > > > > decide
> > >> > > > > > >> > that's
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> way
> > >> > > > > > >> > > > > >> > to go, just pointing out the same goals can
> > >> perhaps
> > >> > > > also
> > >> > > > > be
> > >> > > > > > >> > > achieved
> > >> > > > > > >> > > > > in
> > >> > > > > > >> > > > > >> > other ways.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - With regard to dropping the YARN
> dependency:
> > >> are
> > >> > > you
> > >> > > > > > >> proposing
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > >> > Samza doesn't give any help to people
> wanting
> > to
> > >> > run
> > >> > > on
> > >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > >> > > > > > >> > > > > >> > So the docs would basically have a link to
> > >> Slider
> > >> > and
> > >> > > > > > nothing
> > >> > > > > > >> > > else?
> > >> > > > > > >> > > > Or
> > >> > > > > > >> > > > > >> > would we maintain integrations with a bunch
> of
> > >> > > popular
> > >> > > > > > >> > deployment
> > >> > > > > > >> > > > > >> methods
> > >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts
> to
> > >> make
> > >> > > > Samza
> > >> > > > > > work
> > >> > > > > > >> > with
> > >> > > > > > >> > > > > >> Slider)?
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > I absolutely think it's a good idea to have
> > the
> > >> > "as a
> > >> > > > > > library"
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > "as a
> > >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
> > >> people
> > >> > who
> > >> > > > > want
> > >> > > > > > >> them,
> > >> > > > > > >> > > > but I
> > >> > > > > > >> > > > > >> > think there should also be a low-friction
> path
> > >> for
> > >> > > > common
> > >> > > > > > "as
> > >> > > > > > >> a
> > >> > > > > > >> > > > > service"
> > >> > > > > > >> > > > > >> > deployment methods, for which we probably
> need
> > >> to
> > >> > > > > maintain
> > >> > > > > > >> > > > > integrations.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd
> to
> > >> me,
> > >> > > > > because
> > >> > > > > > >> Kafka
> > >> > > > > > >> > > is
> > >> > > > > > >> > > > > all
> > >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> > >> Transformers"
> > >> > > or
> > >> > > > > > "Kafka
> > >> > > > > > >> > > > Filters"
> > >> > > > > > >> > > > > >> > would be more apt?
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
> > >> (stream
> > >> > > > > > >> transformation
> > >> > > > > > >> > > > with
> > >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> > >> library"
> > >> > > bit)
> > >> > > > > > could
> > >> > > > > > >> > > become
> > >> > > > > > >> > > > > >> part of
> > >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
> > >> streaming
> > >> > SQL
> > >> > > > and
> > >> > > > > > >> > > > integrations
> > >> > > > > > >> > > > > >> with
> > >> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> > >> project?
> > >> > > In
> > >> > > > > > other
> > >> > > > > > >> > > words,
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > would absorb the proven, stable core of
> Samza,
> > >> > which
> > >> > > > > would
> > >> > > > > > >> > become
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in this
> > >> > thread.
> > >> > > > The
> > >> > > > > > Samza
> > >> > > > > > >> > > > project
> > >> > > > > > >> > > > > >> > would then target that third Kafka client as
> > its
> > >> > base
> > >> > > > > API,
> > >> > > > > > and
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > >> project
> > >> > > > > > >> > > > > >> > would be freed up to explore more
> experimental
> > >> new
> > >> > > > > > horizons.
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > Martin
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > >> > > > jay.kreps@gmail.com>
> > >> > > > > > >> wrote:
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > > Hey Martin,
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I
> actually
> > >> > don't
> > >> > > > > think
> > >> > > > > > it
> > >> > > > > > >> > ties
> > >> > > > > > >> > > > our
> > >> > > > > > >> > > > > >> > hands
> > >> > > > > > >> > > > > >> > > at all, all it does is refactor things.
> The
> > >> > > division
> > >> > > > of
> > >> > > > > > >> > > > > >> responsibility is
> > >> > > > > > >> > > > > >> > > that Samza core is responsible for task
> > >> > lifecycle,
> > >> > > > > state,
> > >> > > > > > >> and
> > >> > > > > > >> > > > > >> partition
> > >> > > > > > >> > > > > >> > > management (using the Kafka co-ordinator)
> > but
> > >> it
> > >> > is
> > >> > > > NOT
> > >> > > > > > >> > > > responsible
> > >> > > > > > >> > > > > >> for
> > >> > > > > > >> > > > > >> > > packaging, configuration deployment or
> > >> execution
> > >> > of
> > >> > > > > > >> processes.
> > >> > > > > > >> > > The
> > >> > > > > > >> > > > > >> > problem
> > >> > > > > > >> > > > > >> > > of packaging and starting these processes
> is
> > >> > > > > > >> > > > > >> > > framework/environment-specific. This
> leaves
> > >> > > > individual
> > >> > > > > > >> > > frameworks
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > as
> > >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you can
> > get
> > >> > > simple
> > >> > > > > > >> stateless
> > >> > > > > > >> > > > > >> support in
> > >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf
> > app
> > >> > > > > framework
> > >> > > > > > >> > > (Slider,
> > >> > > > > > >> > > > > >> > Marathon,
> > >> > > > > > >> > > > > >> > > etc). These are well known by people and
> > have
> > >> > nice
> > >> > > > UIs
> > >> > > > > > and a
> > >> > > > > > >> > lot
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > > flexibility. I don't think they have node
> > >> > affinity
> > >> > > > as a
> > >> > > > > > >> built
> > >> > > > > > >> > in
> > >> > > > > > >> > > > > >> option
> > >> > > > > > >> > > > > >> > > (though I could be wrong). So if we want
> > that
> > >> we
> > >> > > can
> > >> > > > > > either
> > >> > > > > > >> > wait
> > >> > > > > > >> > > > for
> > >> > > > > > >> > > > > >> them
> > >> > > > > > >> > > > > >> > > to add it or do a custom framework to add
> > that
> > >> > > > feature
> > >> > > > > > (as
> > >> > > > > > >> > now).
> > >> > > > > > >> > > > > >> > Obviously
> > >> > > > > > >> > > > > >> > > if you manage things with old-school ops
> > tools
> > >> > > > > > >> > (puppet/chef/etc)
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> get
> > >> > > > > > >> > > > > >> > > locality easily. The nice thing, though,
> is
> > >> that
> > >> > > all
> > >> > > > > the
> > >> > > > > > >> samza
> > >> > > > > > >> > > > > >> "business
> > >> > > > > > >> > > > > >> > > logic" around partition management and
> fault
> > >> > > > tolerance
> > >> > > > > > is in
> > >> > > > > > >> > > Samza
> > >> > > > > > >> > > > > >> core
> > >> > > > > > >> > > > > >> > so
> > >> > > > > > >> > > > > >> > > it is shared across frameworks and the
> > >> framework
> > >> > > > > specific
> > >> > > > > > >> bit
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > just
> > >> > > > > > >> > > > > >> > > whether it is smart enough to try to get
> the
> > >> same
> > >> > > > host
> > >> > > > > > when
> > >> > > > > > >> a
> > >> > > > > > >> > > job
> > >> > > > > > >> > > > is
> > >> > > > > > >> > > > > >> > > restarted.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah
> I
> > >> think
> > >> > > the
> > >> > > > > > goal
> > >> > > > > > >> > would
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > >> (a)
> > >> > > > > > >> > > > > >> > > actually get better alignment in user
> > >> experience,
> > >> > > and
> > >> > > > > (b)
> > >> > > > > > >> > > express
> > >> > > > > > >> > > > > >> this in
> > >> > > > > > >> > > > > >> > > the naming and project branding.
> > Specifically:
> > >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> > >> > > > > > "transformation"
> > >> > > > > > >> api
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > be
> > >> > > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e.
> be
> > >> able
> > >> > > to
> > >> > > > > > explain
> > >> > > > > > >> > > when
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> use
> > >> > > > > > >> > > > > >> > > the consumer and when to use the stream
> > >> > processing
> > >> > > > > > >> > functionality
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> lead
> > >> > > > > > >> > > > > >> > > people into that experience.
> > >> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza
> 1.4.2
> > >> (or
> > >> > > > > > whatever)
> > >> > > > > > >> > that
> > >> > > > > > >> > > > has
> > >> > > > > > >> > > > > >> both
> > >> > > > > > >> > > > > >> > > Kafka and the stream processing part and
> > they
> > >> > > > actually
> > >> > > > > > work
> > >> > > > > > >> > > > > together.
> > >> > > > > > >> > > > > >> > > 3. Unify the programming experience so the
> > >> client
> > >> > > and
> > >> > > > > > Samza
> > >> > > > > > >> > api
> > >> > > > > > >> > > > > share
> > >> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> > committers
> > >> and
> > >> > > can
> > >> > > > > > have a
> > >> > > > > > >> > > > > separate
> > >> > > > > > >> > > > > >> > repo,
> > >> > > > > > >> > > > > >> > > but I'm actually not really sure (I can't
> > >> find a
> > >> > > > > > definition
> > >> > > > > > >> > of a
> > >> > > > > > >> > > > > >> > subproject
> > >> > > > > > >> > > > > >> > > in Apache).
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > Basically at a high-level you want the
> > >> experience
> > >> > > to
> > >> > > > > > "feel"
> > >> > > > > > >> > > like a
> > >> > > > > > >> > > > > >> single
> > >> > > > > > >> > > > > >> > > system, not to relatively independent
> things
> > >> that
> > >> > > are
> > >> > > > > > kind
> > >> > > > > > >> of
> > >> > > > > > >> > > > > >> awkwardly
> > >> > > > > > >> > > > > >> > > glued together.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > I think if we did that they having naming
> or
> > >> > > branding
> > >> > > > > > like
> > >> > > > > > >> > > "kafka
> > >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or something
> > >> like
> > >> > > that
> > >> > > > > > would
> > >> > > > > > >> > > > actually
> > >> > > > > > >> > > > > >> do a
> > >> > > > > > >> > > > > >> > > good job of conveying what it is. I do
> that
> > >> this
> > >> > > > would
> > >> > > > > > help
> > >> > > > > > >> > > > adoption
> > >> > > > > > >> > > > > >> > quite
> > >> > > > > > >> > > > > >> > > a lot as it would correctly convey that
> > using
> > >> > Kafka
> > >> > > > > > >> Streaming
> > >> > > > > > >> > > with
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > is
> > >> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
> > >> pretty
> > >> > > > > heavily
> > >> > > > > > >> > adopted
> > >> > > > > > >> > > > at
> > >> > > > > > >> > > > > >> this
> > >> > > > > > >> > > > > >> > > point.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> > >> originally
> > >> > > > when
> > >> > > > > > open
> > >> > > > > > >> > > > sourcing
> > >> > > > > > >> > > > > >> > Samza,
> > >> > > > > > >> > > > > >> > > however at that time Kafka was relatively
> > >> unknown
> > >> > > and
> > >> > > > > we
> > >> > > > > > >> > decided
> > >> > > > > > >> > > > not
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> > do
> > >> > > > > > >> > > > > >> > > it since we felt it would be limiting.
> From
> > my
> > >> > > point
> > >> > > > of
> > >> > > > > > view
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > three
> > >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now
> really
> > >> > heavily
> > >> > > > > used
> > >> > > > > > for
> > >> > > > > > >> > > > stream
> > >> > > > > > >> > > > > >> > > processing, (2) we learned that
> abstracting
> > >> out
> > >> > the
> > >> > > > > > stream
> > >> > > > > > >> > well
> > >> > > > > > >> > > is
> > >> > > > > > >> > > > > >> > > basically impossible, (3) we learned it is
> > >> really
> > >> > > > hard
> > >> > > > > to
> > >> > > > > > >> keep
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > > two
> > >> > > > > > >> > > > > >> > > things feeling like a single product.
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > -Jay
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> > >> Kleppmann
> > >> > <
> > >> > > > > > >> > > > > >> martin@kleppmann.com>
> > >> > > > > > >> > > > > >> > > wrote:
> > >> > > > > > >> > > > > >> > >
> > >> > > > > > >> > > > > >> > >> Hi all,
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> I agree with the general philosophy of
> > tying
> > >> > Samza
> > >> > > > > more
> > >> > > > > > >> > firmly
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > > >> Kafka.
> > >> > > > > > >> > > > > >> > >> After I spent a while looking at
> > integrating
> > >> > other
> > >> > > > > > message
> > >> > > > > > >> > > > brokers
> > >> > > > > > >> > > > > >> (e.g.
> > >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to
> the
> > >> > > > conclusion
> > >> > > > > > that
> > >> > > > > > >> > > > > >> > SystemConsumer
> > >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
> > Kafka's
> > >> > that
> > >> > > > > pretty
> > >> > > > > > >> much
> > >> > > > > > >> > > > > nobody
> > >> > > > > > >> > > > > >> but
> > >> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
> > >> > perhaps
> > >> > > an
> > >> > > > > > >> > exception,
> > >> > > > > > >> > > > but
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.)
> > Thus,
> > >> > > making
> > >> > > > > > Samza
> > >> > > > > > >> > > fully
> > >> > > > > > >> > > > > >> > dependent
> > >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > >> > system-independence
> > >> > > > was
> > >> > > > > > >> never
> > >> > > > > > >> > as
> > >> > > > > > >> > > > > real
> > >> > > > > > >> > > > > >> as
> > >> > > > > > >> > > > > >> > we
> > >> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of
> > code
> > >> > reuse
> > >> > > > are
> > >> > > > > > >> real.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN
> has
> > >> also
> > >> > > > always
> > >> > > > > > been
> > >> > > > > > >> > > > > >> appealing to
> > >> > > > > > >> > > > > >> > >> me, for various reasons already mentioned
> > in
> > >> > this
> > >> > > > > > thread.
> > >> > > > > > >> > > > Although
> > >> > > > > > >> > > > > >> > making
> > >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > >> > > > (YARN/Mesos/AWS/etc)
> > >> > > > > > >> seems
> > >> > > > > > >> > > > > >> laudable,
> > >> > > > > > >> > > > > >> > I am
> > >> > > > > > >> > > > > >> > >> a little concerned that it will restrict
> us
> > >> to a
> > >> > > > > lowest
> > >> > > > > > >> > common
> > >> > > > > > >> > > > > >> > denominator.
> > >> > > > > > >> > > > > >> > >> For example, would host affinity
> > (SAMZA-617)
> > >> > still
> > >> > > > be
> > >> > > > > > >> > possible?
> > >> > > > > > >> > > > For
> > >> > > > > > >> > > > > >> jobs
> > >> > > > > > >> > > > > >> > >> with large amounts of state, I think
> > >> SAMZA-617
> > >> > > would
> > >> > > > > be
> > >> > > > > > a
> > >> > > > > > >> big
> > >> > > > > > >> > > > boon,
> > >> > > > > > >> > > > > >> > since
> > >> > > > > > >> > > > > >> > >> restoring state off the changelog on
> every
> > >> > single
> > >> > > > > > restart
> > >> > > > > > >> is
> > >> > > > > > >> > > > > painful,
> > >> > > > > > >> > > > > >> > due
> > >> > > > > > >> > > > > >> > >> to long recovery times. It would be a
> shame
> > >> if
> > >> > the
> > >> > > > > > >> decoupling
> > >> > > > > > >> > > > from
> > >> > > > > > >> > > > > >> YARN
> > >> > > > > > >> > > > > >> > >> made host affinity impossible.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Jay, a question about the proposed API
> for
> > >> > > > > > instantiating a
> > >> > > > > > >> > job
> > >> > > > > > >> > > in
> > >> > > > > > >> > > > > >> code
> > >> > > > > > >> > > > > >> > >> (rather than a properties file): when
> > >> > submitting a
> > >> > > > job
> > >> > > > > > to a
> > >> > > > > > >> > > > > cluster,
> > >> > > > > > >> > > > > >> is
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >> idea that the instantiation code runs on
> a
> > >> > client
> > >> > > > > > >> somewhere,
> > >> > > > > > >> > > > which
> > >> > > > > > >> > > > > >> then
> > >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > >> > > YARN/Mesos/AWS/etc?
> > >> > > > > Or
> > >> > > > > > >> does
> > >> > > > > > >> > > that
> > >> > > > > > >> > > > > >> code
> > >> > > > > > >> > > > > >> > run
> > >> > > > > > >> > > > > >> > >> on each container that is part of the job
> > (in
> > >> > > which
> > >> > > > > > case,
> > >> > > > > > >> how
> > >> > > > > > >> > > > does
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > job
> > >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
> > >> right to
> > >> > > > make
> > >> > > > > a
> > >> > > > > > 1.0
> > >> > > > > > >> > > > release
> > >> > > > > > >> > > > > >> > with a
> > >> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete.
> So
> > if
> > >> > this
> > >> > > > is
> > >> > > > > > going
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> happen, I
> > >> > > > > > >> > > > > >> > >> think it would be more honest to stick
> with
> > >> 0.*
> > >> > > > > version
> > >> > > > > > >> > numbers
> > >> > > > > > >> > > > > until
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >> library-ified Samza has been implemented,
> > is
> > >> > > stable
> > >> > > > > and
> > >> > > > > > >> > widely
> > >> > > > > > >> > > > > used.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
> > >> Kafka?
> > >> > > There
> > >> > > > > is
> > >> > > > > > >> > > precedent
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > >> tight coupling between different Apache
> > >> projects
> > >> > > > (e.g.
> > >> > > > > > >> > Curator
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I
> think
> > >> > > remaining
> > >> > > > > > >> separate
> > >> > > > > > >> > > > would
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > ok.
> > >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on
> Kafka,
> > >> there
> > >> > > is
> > >> > > > > > enough
> > >> > > > > > >> > > > > substance
> > >> > > > > > >> > > > > >> in
> > >> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> > >> project.
> > >> > > An
> > >> > > > > > >> argument
> > >> > > > > > >> > in
> > >> > > > > > >> > > > > >> favour
> > >> > > > > > >> > > > > >> > of
> > >> > > > > > >> > > > > >> > >> merging would be if we think Kafka has a
> > much
> > >> > > > stronger
> > >> > > > > > >> "brand
> > >> > > > > > >> > > > > >> presence"
> > >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one.
> If
> > >> the
> > >> > > Kafka
> > >> > > > > > >> project
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > >> willing
> > >> > > > > > >> > > > > >> > to
> > >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of
> > doing
> > >> > > > stateful
> > >> > > > > > >> stream
> > >> > > > > > >> > > > > >> > >> transformations, that would probably have
> > >> much
> > >> > the
> > >> > > > > same
> > >> > > > > > >> > effect
> > >> > > > > > >> > > as
> > >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> > >> Processors"
> > >> > or
> > >> > > > > > suchlike.
> > >> > > > > > >> > > Close
> > >> > > > > > >> > > > > >> > >> collaboration between the two projects
> will
> > >> be
> > >> > > > needed
> > >> > > > > in
> > >> > > > > > >> any
> > >> > > > > > >> > > > case.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> From a project management perspective, I
> > >> guess
> > >> > the
> > >> > > > > "new
> > >> > > > > > >> > Samza"
> > >> > > > > > >> > > > > would
> > >> > > > > > >> > > > > >> > have
> > >> > > > > > >> > > > > >> > >> to be developed on a branch alongside
> > ongoing
> > >> > > > > > maintenance
> > >> > > > > > >> of
> > >> > > > > > >> > > the
> > >> > > > > > >> > > > > >> current
> > >> > > > > > >> > > > > >> > >> line of development? I think it would be
> > >> > important
> > >> > > > to
> > >> > > > > > >> > continue
> > >> > > > > > >> > > > > >> > supporting
> > >> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> > >> migration
> > >> > > > path
> > >> > > > > to
> > >> > > > > > >> the
> > >> > > > > > >> > > new
> > >> > > > > > >> > > > > >> > version.
> > >> > > > > > >> > > > > >> > >> Leaving the current versions unsupported
> > and
> > >> > > forcing
> > >> > > > > > people
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> rewrite
> > >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> Best,
> > >> > > > > > >> > > > > >> > >> Martin
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > >> > > > jay@confluent.io>
> > >> > > > > > >> wrote:
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >>> Hey Garry,
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be
> > happy
> > >> to
> > >> > > chat
> > >> > > > > > more
> > >> > > > > > >> > about
> > >> > > > > > >> > > > > this
> > >> > > > > > >> > > > > >> if
> > >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
> > >> > started
> > >> > > > with
> > >> > > > > > the
> > >> > > > > > >> > idea
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> "what
> > >> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> > >> > ingestion
> > >> > > > > tool"
> > >> > > > > > but
> > >> > > > > > >> > > > > >> ultimately
> > >> > > > > > >> > > > > >> > we
> > >> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> > >> ingestion
> > >> > > and
> > >> > > > > > >> > > > transformation
> > >> > > > > > >> > > > > >> had
> > >> > > > > > >> > > > > >> > >>> pretty different needs and coupling the
> > two
> > >> > made
> > >> > > > > things
> > >> > > > > > >> > hard.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
> > (KIP-26)
> > >> > > > actually
> > >> > > > > > will
> > >> > > > > > >> > do
> > >> > > > > > >> > > > what
> > >> > > > > > >> > > > > >> you
> > >> > > > > > >> > > > > >> > >> are
> > >> > > > > > >> > > > > >> > >>> looking for.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> With regard to your point about slider,
> I
> > >> don't
> > >> > > > > > >> necessarily
> > >> > > > > > >> > > > > >> disagree.
> > >> > > > > > >> > > > > >> > >> But I
> > >> > > > > > >> > > > > >> > >>> think getting good YARN support is quite
> > >> doable
> > >> > > > and I
> > >> > > > > > >> think
> > >> > > > > > >> > we
> > >> > > > > > >> > > > can
> > >> > > > > > >> > > > > >> make
> > >> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> > >> proposal
> > >> > > > > solves
> > >> > > > > > is
> > >> > > > > > >> > that
> > >> > > > > > >> > > > > >> > >> technically
> > >> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
> > >> cluster
> > >> > > > > > management
> > >> > > > > > >> > > systems
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > way
> > >> > > > > > >> > > > > >> > >>> things are now, you need to write an
> "app
> > >> > master"
> > >> > > > or
> > >> > > > > > >> > > "framework"
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > each
> > >> > > > > > >> > > > > >> > >>> and they are all a little different so
> > >> testing
> > >> > is
> > >> > > > > > really
> > >> > > > > > >> > hard.
> > >> > > > > > >> > > > In
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > >>> absence of this we have been stuck with
> > just
> > >> > YARN
> > >> > > > > which
> > >> > > > > > >> has
> > >> > > > > > >> > > > > >> fantastic
> > >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the
> > org,
> > >> but
> > >> > > > zero
> > >> > > > > > >> > > penetration
> > >> > > > > > >> > > > > >> > >> elsewhere.
> > >> > > > > > >> > > > > >> > >>> Given the huge amount of work being put
> in
> > >> to
> > >> > > > slider,
> > >> > > > > > >> > > marathon,
> > >> > > > > > >> > > > > aws
> > >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
> > related
> > >> > > > packaging
> > >> > > > > > >> > > > technologies
> > >> > > > > > >> > > > > >> > people
> > >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> > >> > > > > cloud-specific
> > >> > > > > > >> > deploy
> > >> > > > > > >> > > > > >> tools,
> > >> > > > > > >> > > > > >> > >> etc)
> > >> > > > > > >> > > > > >> > >>> I really think it is important to get
> this
> > >> > right.
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> -Jay
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> > >> > Turkington
> > >> > > <
> > >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > >> > > > > > >> > > > > >> > >>>
> > >> > > > > > >> > > > > >> > >>>> Hi all,
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> I think the question below re does
> Samza
> > >> > become
> > >> > > a
> > >> > > > > > >> > sub-project
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>> highlights the broader point around
> > >> migration.
> > >> > > > Chris
> > >> > > > > > >> > mentions
> > >> > > > > > >> > > > > >> Samza's
> > >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1
> release
> > >> but
> > >> > I'm
> > >> > > > not
> > >> > > > > > sure
> > >> > > > > > >> > it
> > >> > > > > > >> > > > > feels
> > >> > > > > > >> > > > > >> > >> right to
> > >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> > >> deprecate
> > >> > > > most
> > >> > > > > of
> > >> > > > > > >> it.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have some
> > guys
> > >> > who
> > >> > > > have
> > >> > > > > > >> > started
> > >> > > > > > >> > > > > >> working
> > >> > > > > > >> > > > > >> > >> with
> > >> > > > > > >> > > > > >> > >>>> Samza and building some new
> > >> > consumers/producers
> > >> > > > was
> > >> > > > > > next
> > >> > > > > > >> > up.
> > >> > > > > > >> > > > > Sounds
> > >> > > > > > >> > > > > >> > like
> > >> > > > > > >> > > > > >> > >>>> that is absolutely not the direction to
> > >> go. I
> > >> > > need
> > >> > > > > to
> > >> > > > > > >> look
> > >> > > > > > >> > > into
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > KIP
> > >> > > > > > >> > > > > >> > >> in
> > >> > > > > > >> > > > > >> > >>>> more detail but for me the
> attractiveness
> > >> of
> > >> > > > adding
> > >> > > > > > new
> > >> > > > > > >> > Samza
> > >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all
> > they
> > >> > were
> > >> > > > > doing
> > >> > > > > > was
> > >> > > > > > >> > > > really
> > >> > > > > > >> > > > > >> > getting
> > >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to
> > avoid
> > >> > > > having
> > >> > > > > to
> > >> > > > > > >> > worry
> > >> > > > > > >> > > > > about
> > >> > > > > > >> > > > > >> the
> > >> > > > > > >> > > > > >> > >>>> lifecycle management of external
> clients.
> > >> If
> > >> > > there
> > >> > > > > is
> > >> > > > > > a
> > >> > > > > > >> > > generic
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a
> > new
> > >> > > > connector
> > >> > > > > > into
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > have
> > >> > > > > > >> > > > > >> a
> > >> > > > > > >> > > > > >> > >> lot of
> > >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> > reliability
> > >> > done
> > >> > > > for
> > >> > > > > me
> > >> > > > > > >> then
> > >> > > > > > >> > > it
> > >> > > > > > >> > > > > >> gives
> > >> > > > > > >> > > > > >> > me
> > >> > > > > > >> > > > > >> > >> all
> > >> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers
> > would.
> > >> If
> > >> > > not
> > >> > > > > > then it
> > >> > > > > > >> > > > > >> complicates
> > >> > > > > > >> > > > > >> > my
> > >> > > > > > >> > > > > >> > >>>> operational deployments.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Which is similar to my other question
> > with
> > >> the
> > >> > > > > > proposal
> > >> > > > > > >> --
> > >> > > > > > >> > if
> > >> > > > > > >> > > > we
> > >> > > > > > >> > > > > >> > build a
> > >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus
> > the
> > >> > > > requisite
> > >> > > > > > >> shims
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> > integrate
> > >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former
> may
> > >> be a
> > >> > > lot
> > >> > > > > more
> > >> > > > > > >> work
> > >> > > > > > >> > > > than
> > >> > > > > > >> > > > > we
> > >> > > > > > >> > > > > >> > >> think.
> > >> > > > > > >> > > > > >> > >>>> We may make it much easier for a
> newcomer
> > >> to
> > >> > get
> > >> > > > > > >> something
> > >> > > > > > >> > > > > running
> > >> > > > > > >> > > > > >> but
> > >> > > > > > >> > > > > >> > >>>> having them step up and get a reliable
> > >> > > production
> > >> > > > > > >> > deployment
> > >> > > > > > >> > > > may
> > >> > > > > > >> > > > > >> still
> > >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
> > >> > different
> > >> > > > > > reasons
> > >> > > > > > >> > than
> > >> > > > > > >> > > > > >> today.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable
> > with
> > >> > > making
> > >> > > > > the
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > > >> dependency
> > >> > > > > > >> > > > > >> > >> on
> > >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I
> absolutely
> > >> see
> > >> > > the
> > >> > > > > > >> benefits
> > >> > > > > > >> > > in
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > >> > > > > > >> > > > terminologies/abstractions
> > >> > > > > > >> > > > > >> that
> > >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library
> > >> would
> > >> > > > likely
> > >> > > > > > be a
> > >> > > > > > >> > very
> > >> > > > > > >> > > > > nice
> > >> > > > > > >> > > > > >> > tool
> > >> > > > > > >> > > > > >> > >> to
> > >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have
> > the
> > >> > > > concerns
> > >> > > > > > >> above
> > >> > > > > > >> > re
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > >>>> operational side.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Garry
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> > >> [mailto:
> > >> > > > > > >> > gdfm@apache.org
> > >> > > > > > >> > > ]
> > >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations
> on
> > >> > Samza
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > >> > > > > > >> > > > > >> > >>>> From outside, I have always perceived
> > Samza
> > >> > as a
> > >> > > > > > >> computing
> > >> > > > > > >> > > > layer
> > >> > > > > > >> > > > > >> over
> > >> > > > > > >> > > > > >> > >>>> Kafka.
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> The question, maybe a bit provocative,
> is
> > >> > > "should
> > >> > > > > > Samza
> > >> > > > > > >> be
> > >> > > > > > >> > a
> > >> > > > > > >> > > > > >> > sub-project
> > >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> > >> separate
> > >> > > > > project
> > >> > > > > > >> > with a
> > >> > > > > > >> > > > > >> separate
> > >> > > > > > >> > > > > >> > >>>> governance?
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> Cheers,
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> --
> > >> > > > > > >> > > > > >> > >>>> Gianmarco
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > >> > > > > > yanfang724@gmail.com>
> > >> > > > > > >> > > > wrote:
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka
> > more
> > >> > > > tightly.
> > >> > > > > > >> > Because
> > >> > > > > > >> > > > > Samza
> > >> > > > > > >> > > > > >> de
> > >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
> > >> > leverage
> > >> > > > > what
> > >> > > > > > >> Kafka
> > >> > > > > > >> > > > has.
> > >> > > > > > >> > > > > At
> > >> > > > > > >> > > > > >> > the
> > >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
> > reinvent
> > >> > what
> > >> > > > > Samza
> > >> > > > > > >> > > already
> > >> > > > > > >> > > > > >> has. I
> > >> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> > >> > ingestion
> > >> > > > and
> > >> > > > > > >> > > > > transformation.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
> > >> image
> > >> > > how
> > >> > > > > the
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > will
> > >> > > > > > >> > > > > >> look
> > >> > > > > > >> > > > > >> > >>>> like.
> > >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> > >> > > difference
> > >> > > > > in
> > >> > > > > > >> terms
> > >> > > > > > >> > > of
> > >> > > > > > >> > > > > how
> > >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code
> > >> shows
> > >> > (A
> > >> > > > > > client of
> > >> > > > > > >> > > > Kakfa)
> > >> > > > > > >> > > > > ?
> > >> > > > > > >> > > > > >> And
> > >> > > > > > >> > > > > >> > >>>>> user's application code calls this
> > client?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of
> > Kafka
> > >> > (like
> > >> > > > > what
> > >> > > > > > the
> > >> > > > > > >> > > code
> > >> > > > > > >> > > > > >> shows),
> > >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > >> > > > > fault-tolerance?
> > >> > > > > > >> Are
> > >> > > > > > >> > > they
> > >> > > > > > >> > > > > >> taken
> > >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> > >> mechanism,
> > >> > > such
> > >> > > > > as
> > >> > > > > > >> > "Samza
> > >> > > > > > >> > > > > >> worker"
> > >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 2. What about other features, such as
> > >> > > > auto-scaling,
> > >> > > > > > >> shared
> > >> > > > > > >> > > > > state,
> > >> > > > > > >> > > > > >> > >>>>> monitoring?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is
> > this
> > >> > what
> > >> > > > > Chris
> > >> > > > > > >> > > > suggests?)
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from
> > Kakfa
> > >> > and
> > >> > > > > > produce
> > >> > > > > > >> to
> > >> > > > > > >> > > it.
> > >> > > > > > >> > > > > >> Then it
> > >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks
> > like
> > >> > now,
> > >> > > > > > except it
> > >> > > > > > >> > > does
> > >> > > > > > >> > > > > not
> > >> > > > > > >> > > > > >> > rely
> > >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> > >> leverage
> > >> > > > Kafka's
> > >> > > > > > >> > metrics,
> > >> > > > > > >> > > > > logs,
> > >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> Thanks,
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM,
> Guozhang
> > >> > Wang <
> > >> > > > > > >> > > > > wangguoz@gmail.com
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> > >>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Read through the code example and it
> > >> looks
> > >> > > good
> > >> > > > to
> > >> > > > > > me.
> > >> > > > > > >> A
> > >> > > > > > >> > > few
> > >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> > >> runnable
> > >> > > like:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > >> > > --config-factory=...
> > >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> > deploying
> > >> > Samza
> > >> > > > > more
> > >> > > > > > as
> > >> > > > > > >> > > > embedded
> > >> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> > >> (ignoring
> > >> > > the
> > >> > > > > > >> > > terminology
> > >> > > > > > >> > > > > >> since
> > >> > > > > > >> > > > > >> > >>>>>> it is not the
> > >> > > > > > >> > > > > >> > >>>>> same
> > >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> > >> MyStreamTask(configs);
> > >> > > > > Thread
> > >> > > > > > >> > thread
> > >> > > > > > >> > > =
> > >> > > > > > >> > > > > new
> > >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment
> modes
> > >> are
> > >> > > > > important
> > >> > > > > > >> for
> > >> > > > > > >> > > > > >> different
> > >> > > > > > >> > > > > >> > >>>>>> types
> > >> > > > > > >> > > > > >> > >>>>> of
> > >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making
> Samza
> > >> > purely
> > >> > > > > > >> standalone
> > >> > > > > > >> > is
> > >> > > > > > >> > > > > still
> > >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or
> > library
> > >> > > modes.
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> Guozhang
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
> > >> Kreps
> > >> > <
> > >> > > > > > >> > > > jay@confluent.io>
> > >> > > > > > >> > > > > >> > wrote:
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> > >> example,
> > >> > it
> > >> > > > was
> > >> > > > > > >> > supposed
> > >> > > > > > >> > > > to
> > >> > > > > > >> > > > > >> look
> > >> > > > > > >> > > > > >> > >>>>>>> like
> > >> > > > > > >> > > > > >> > >>>>>>> this:
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> > >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > >> > > > "localhost:4242");
> > >> > > > > > >> > > > > >> StreamingConfig
> > >> > > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > > > "test-topic-2");
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > config.processor(ExampleStreamProcessor.class);
> > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > >> > StringSerializer(),
> > >> > > > new
> > >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer());
> KafkaStreaming
> > >> > > > container =
> > >> > > > > > new
> > >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> > container.run();
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> -Jay
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM,
> Jay
> > >> > Kreps <
> > >> > > > > > >> > > > jay@confluent.io
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >> > >>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
> > >> Chris
> > >> > > and
> > >> > > > I
> > >> > > > > > were
> > >> > > > > > >> > > having
> > >> > > > > > >> > > > > >> > >>>>>>>> around
> > >> > > > > > >> > > > > >> > >>>>>>> whether
> > >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza
> as a
> > >> kind
> > >> > > of
> > >> > > > > data
> > >> > > > > > >> > > > ingestion
> > >> > > > > > >> > > > > >> > >>>>> framework
> > >> > > > > > >> > > > > >> > >>>>>>> for
> > >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to
> > KIP-26
> > >> > > > > "copycat").
> > >> > > > > > >> This
> > >> > > > > > >> > > > kind
> > >> > > > > > >> > > > > of
> > >> > > > > > >> > > > > >> > >>>>>> combined
> > >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and
> > YARN
> > >> and
> > >> > > the
> > >> > > > > > >> > discussion
> > >> > > > > > >> > > > > >> around
> > >> > > > > > >> > > > > >> > >>>>>>>> how
> > >> > > > > > >> > > > > >> > >>>>> to
> > >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was,
> given
> > >> that
> > >> > > > Samza
> > >> > > > > > was
> > >> > > > > > >> > > > basically
> > >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific,
> what
> > if
> > >> > you
> > >> > > > just
> > >> > > > > > >> > embraced
> > >> > > > > > >> > > > > that
> > >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > >> > > > > > >> > > > > >> > >>>>>> into
> > >> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> > >> > framework
> > >> > > > and
> > >> > > > > > more
> > >> > > > > > >> > > like a
> > >> > > > > > >> > > > > >> > >>>>>>>> third
> > >> > > > > > >> > > > > >> > >>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> > consumer"
> > >> > with
> > >> > > > > state
> > >> > > > > > >> > > > management
> > >> > > > > > >> > > > > >> > >>>>>> facilities.
> > >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> > >> complex
> > >> > > > stream
> > >> > > > > > >> > > processing
> > >> > > > > > >> > > > > >> > >>>>>>>> framework
> > >> > > > > > >> > > > > >> > >>>>>>> this
> > >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple
> > thing,
> > >> not
> > >> > > > much
> > >> > > > > > more
> > >> > > > > > >> > > > > >> complicated
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> use
> > >> > > > > > >> > > > > >> > >>>>>>> or
> > >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As
> > Chris
> > >> > said
> > >> > > > we
> > >> > > > > > >> thought
> > >> > > > > > >> > > > about
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >>>>>>>> a
> > >> > > > > > >> > > > > >> > >>>>> lot
> > >> > > > > > >> > > > > >> > >>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> > >> > processing
> > >> > > > > > systems
> > >> > > > > > >> > were
> > >> > > > > > >> > > > > doing)
> > >> > > > > > >> > > > > >> > >>>>> seemed
> > >> > > > > > >> > > > > >> > >>>>>>> like
> > >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output
> > >> data
> > >> > to
> > >> > > > and
> > >> > > > > > from
> > >> > > > > > >> > the
> > >> > > > > > >> > > > > stream
> > >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually
> > looked
> > >> > into
> > >> > > > how
> > >> > > > > > that
> > >> > > > > > >> > > would
> > >> > > > > > >> > > > > >> > >>>>>>>> work,
> > >> > > > > > >> > > > > >> > >>>>> Samza
> > >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data
> ingestion
> > >> > > framework
> > >> > > > > > for a
> > >> > > > > > >> > > bunch
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>> reasons.
> > >> > > > > > >> > > > > >> > >>>>>> To
> > >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a
> > pretty
> > >> > > > different
> > >> > > > > > >> > internal
> > >> > > > > > >> > > > > data
> > >> > > > > > >> > > > > >> > >>>>>>>> model
> > >> > > > > > >> > > > > >> > >>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split
> > them
> > >> and
> > >> > > had
> > >> > > > > an
> > >> > > > > > api
> > >> > > > > > >> > for
> > >> > > > > > >> > > > > Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26)
> > >> and a
> > >> > > > > separate
> > >> > > > > > >> api
> > >> > > > > > >> > > for
> > >> > > > > > >> > > > > >> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> > embracing
> > >> the
> > >> > > > same
> > >> > > > > > >> > > terminology
> > >> > > > > > >> > > > > and
> > >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about
> the
> > >> > current
> > >> > > > > > state is
> > >> > > > > > >> > > that
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>> two
> > >> > > > > > >> > > > > >> > >>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology
> > >> like
> > >> > > > > "stream"
> > >> > > > > > vs
> > >> > > > > > >> > > > "topic"
> > >> > > > > > >> > > > > >> and
> > >> > > > > > >> > > > > >> > >>>>>>> different
> > >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means
> > you
> > >> > kind
> > >> > > > of
> > >> > > > > > have
> > >> > > > > > >> to
> > >> > > > > > >> > > > learn
> > >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > >> > > > > > >> > > > > >> > >>>>>>> way,
> > >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly
> different
> > >> way,
> > >> > > > then
> > >> > > > > > kind
> > >> > > > > > >> of
> > >> > > > > > >> > > > > >> > >>>>>>>> understand
> > >> > > > > > >> > > > > >> > >>>>> how
> > >> > > > > > >> > > > > >> > >>>>>>> they
> > >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having
> > walked
> > >> a
> > >> > few
> > >> > > > > > people
> > >> > > > > > >> > > through
> > >> > > > > > >> > > > > >> this
> > >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to
> > >> get.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of
> > >> time
> > >> > on
> > >> > > > > > >> airplanes I
> > >> > > > > > >> > > > > hacked
> > >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> > >> incomplete
> > >> > > > > > prototype
> > >> > > > > > >> of
> > >> > > > > > >> > > > what
> > >> > > > > > >> > > > > >> > >>>>>>>> this would
> > >> > > > > > >> > > > > >> > >>>>> look
> > >> > > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
> > >> dumped
> > >> > > into
> > >> > > > > > Kafka
> > >> > > > > > >> as
> > >> > > > > > >> > > it
> > >> > > > > > >> > > > > >> > >>>>>>>> required a
> > >> > > > > > >> > > > > >> > >>>>>> few
> > >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here
> is
> > >> the
> > >> > > code:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > >
> > >> > > > > > >> >
> > >> > > > > >
> > >> > >
> > >>
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I
> > just
> > >> > > > > liberally
> > >> > > > > > >> > renamed
> > >> > > > > > >> > > > > >> > >>>>>>>> everything
> > >> > > > > > >> > > > > >> > >>>>> to
> > >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
> > >> regard
> > >> > > for
> > >> > > > > > >> > > > compatibility.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something like
> > >> this:
> > >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new
> Properties();
> > >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > >> > > > > "localhost:4242");
> > >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > >> > > > > > >> > > > > >>
> config.processor(ExampleStreamProcessor.class);
> > >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > >> > > StringDeserializer());
> > >> > > > > > >> > > > KafkaStreaming
> > >> > > > > > >> > > > > >> > >>>>>> container =
> > >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > >> > container.run();
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > >> > > > SamzaContainer;
> > >> > > > > > >> > > > > StreamProcessor
> > >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the
> class
> > >> names
> > >> > > in
> > >> > > > a
> > >> > > > > > file
> > >> > > > > > >> > and
> > >> > > > > > >> > > > then
> > >> > > > > > >> > > > > >> > >>>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you
> just
> > >> > > > > instantiate
> > >> > > > > > the
> > >> > > > > > >> > > > > container
> > >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced
> > over
> > >> > > > however
> > >> > > > > > many
> > >> > > > > > >> > > > > instances
> > >> > > > > > >> > > > > >> > >>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>> this
> > >> > > > > > >> > > > > >> > >>>>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> > instance
> > >> > dies,
> > >> > > > new
> > >> > > > > > >> tasks
> > >> > > > > > >> > > are
> > >> > > > > > >> > > > > >> added
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>> existing containers without
> shutting
> > >> them
> > >> > > > down).
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
> > running
> > >> > this
> > >> > > > > stuff
> > >> > > > > > in
> > >> > > > > > >> > YARN
> > >> > > > > > >> > > > via
> > >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
> > >> using
> > >> > > some
> > >> > > > > of
> > >> > > > > > >> their
> > >> > > > > > >> > > > tools
> > >> > > > > > >> > > > > >> > >>>>>>>> but from the
> > >> > > > > > >> > > > > >> > >>>>>> point
> > >> > > > > > >> > > > > >> > >>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these
> stream
> > >> > > > processing
> > >> > > > > > jobs
> > >> > > > > > >> > are
> > >> > > > > > >> > > > > just
> > >> > > > > > >> > > > > >> > >>>>>> stateless
> > >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
> > >> expand
> > >> > and
> > >> > > > > > contract
> > >> > > > > > >> > at
> > >> > > > > > >> > > > > will.
> > >> > > > > > >> > > > > >> > >>>>>>>> There
> > >> > > > > > >> > > > > >> > >>>>> is
> > >> > > > > > >> > > > > >> > >>>>>>> no
> > >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code,
> > it
> > >> > would
> > >> > > > get
> > >> > > > > > >> larger
> > >> > > > > > >> > > if
> > >> > > > > > >> > > > we
> > >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
> > larger.
> > >> We
> > >> > > > really
> > >> > > > > > do
> > >> > > > > > >> > get a
> > >> > > > > > >> > > > ton
> > >> > > > > > >> > > > > >> > >>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>> leverage
> > >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> > >> > delegated
> > >> > > to
> > >> > > > > the
> > >> > > > > > >> new
> > >> > > > > > >> > > > > >> consumer.
> > >> > > > > > >> > > > > >> > >>>>> This
> > >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> > >> > management
> > >> > > > > > strategy
> > >> > > > > > >> > > > > available
> > >> > > > > > >> > > > > >> > >>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to
> Samza
> > >> (and
> > >> > > vice
> > >> > > > > > versa)
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > with
> > >> > > > > > >> > > > > >> > >>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>> exact
> > >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as
> > state
> > >> > reuse
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
> > >> > thought
> > >> > > > > > >> provoking.
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM,
> > Chris
> > >> > > > > Riccomini <
> > >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with
> > Samza
> > >> > > > > engineers
> > >> > > > > > at
> > >> > > > > > >> > > > LinkedIn
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>> Confluent
> > >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> > observations
> > >> > and
> > >> > > > > would
> > >> > > > > > >> like
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > >> > >>>>>>>>> propose
> > >> > > > > > >> > > > > >> > >>>>> some
> > >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I
> > >> want to
> > >> > > > call
> > >> > > > > > out
> > >> > > > > > >> > about
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > >> > > > > > >> > > > > >> > >>>>>> design,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> > changes.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a
> dynamic
> > >> > > > deployment
> > >> > > > > > >> system.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> > >> SystemConsumer/SystemProducer
> > >> > and
> > >> > > > > > Kafka's
> > >> > > > > > >> > > > consumer
> > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > >> > > > > > >> > > > > >> > >>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
> > >> > problems.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
> > related,
> > >> > but
> > >> > > > I'll
> > >> > > > > > >> > address
> > >> > > > > > >> > > > them
> > >> > > > > > >> > > > > >> in
> > >> > > > > > >> > > > > >> > >>>>> order.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use
> > of a
> > >> > > > dynamic
> > >> > > > > > >> > > deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > >> > > > > > >> > > > > >> > >>>>>> such
> > >> > > > > > >> > > > > >> > >>>>>>>>> as
> > >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we
> initially
> > >> built
> > >> > > > > Samza,
> > >> > > > > > we
> > >> > > > > > >> > bet
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > >> > >>>>>>>>> there
> > >> > > > > > >> > > > > >> > >>>>>> would
> > >> > > > > > >> > > > > >> > >>>>>>>>> be
> > >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area,
> and
> > >> we
> > >> > > could
> > >> > > > > > >> support
> > >> > > > > > >> > > > them,
> > >> > > > > > >> > > > > >> and
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>> rest
> > >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there
> are
> > >> many
> > >> > > > > > >> variations.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > >> > > > > > >> > > > > >> > >>>>>> many
> > >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start
> > >> their
> > >> > > > > > processors
> > >> > > > > > >> > like
> > >> > > > > > >> > > > > normal
> > >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use
> traditional
> > >> > > > deployment
> > >> > > > > > >> scripts
> > >> > > > > > >> > > > such
> > >> > > > > > >> > > > > as
> > >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > >> > > > > > >> > > > > >> > >>>>>> Chef,
> > >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
> > >> system
> > >> > > on
> > >> > > > > > users
> > >> > > > > > >> > makes
> > >> > > > > > >> > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
> > painful
> > >> for
> > >> > > > first
> > >> > > > > > time
> > >> > > > > > >> > > > users.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a
> requirement
> > >> was
> > >> > > also
> > >> > > > a
> > >> > > > > > bit
> > >> > > > > > >> of
> > >> > > > > > >> > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > >> > > > > > >> > > > > >> > >>>>>> because
> > >> > > > > > >> > > > > >> > >>>>>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding
> > between
> > >> > the
> > >> > > > > > nature of
> > >> > > > > > >> > > batch
> > >> > > > > > >> > > > > >> jobs
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>> stream
> > >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> > >> > > conscious
> > >> > > > > > effort
> > >> > > > > > >> to
> > >> > > > > > >> > > > favor
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>> Hadoop
> > >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things,
> > >> since
> > >> > it
> > >> > > > > worked
> > >> > > > > > >> and
> > >> > > > > > >> > > was
> > >> > > > > > >> > > > > well
> > >> > > > > > >> > > > > >> > >>>>>>> understood.
> > >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that
> > >> batch
> > >> > > jobs
> > >> > > > > > have a
> > >> > > > > > >> > > > definite
> > >> > > > > > >> > > > > >> > >>>>>> beginning,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and
> > >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs
> > don't
> > >> > > > > (usually).
> > >> > > > > > >> This
> > >> > > > > > >> > > > leads
> > >> > > > > > >> > > > > to
> > >> > > > > > >> > > > > >> > >>>>>>>>> a
> > >> > > > > > >> > > > > >> > >>>>> much
> > >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for
> > stream
> > >> > > > > processors.
> > >> > > > > > >> You
> > >> > > > > > >> > > > > >> basically
> > >> > > > > > >> > > > > >> > >>>>>>>>> just
> > >> > > > > > >> > > > > >> > >>>>>>> need
> > >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> > >> processor,
> > >> > and
> > >> > > > > start
> > >> > > > > > >> it.
> > >> > > > > > >> > > The
> > >> > > > > > >> > > > > way
> > >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's
> > no
> > >> > > concept
> > >> > > > > of
> > >> > > > > > a
> > >> > > > > > >> > > cluster
> > >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > >> > > > > > >> > > > > >> > >>>>>> add
> > >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> > >> coupling
> > >> > > > Samza
> > >> > > > > > with
> > >> > > > > > >> a
> > >> > > > > > >> > > > > >> scheduler
> > >> > > > > > >> > > > > >> > >>>>>>>>> is
> > >> > > > > > >> > > > > >> > >>>>>> that
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
> > >> handle
> > >> > > > > > deployment.
> > >> > > > > > >> > > This
> > >> > > > > > >> > > > > >> pulls
> > >> > > > > > >> > > > > >> > >>>>>>>>> in a
> > >> > > > > > >> > > > > >> > >>>>>>> bunch
> > >> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> > >> > > distribution
> > >> > > > > > (config
> > >> > > > > > >> > > > > stream),
> > >> > > > > > >> > > > > >> > >>>>>>>>> shell
> > >> > > > > > >> > > > > >> > >>>>>>> scrips
> > >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> > packaging
> > >> > (all
> > >> > > > the
> > >> > > > > > .tgz
> > >> > > > > > >> > > > stuff),
> > >> > > > > > >> > > > > >> etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring
> dynamic
> > >> > > > deployment
> > >> > > > > > was
> > >> > > > > > >> to
> > >> > > > > > >> > > > > support
> > >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> > >> > > locality,
> > >> > > > > you
> > >> > > > > > >> need
> > >> > > > > > >> > to
> > >> > > > > > >> > > > put
> > >> > > > > > >> > > > > >> > >>>>>>>>> your
> > >> > > > > > >> > > > > >> > >>>>>> processors
> > >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> > processing.
> > >> > Upon
> > >> > > > > > further
> > >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > >> > > > > > >> > > > > >> > >>>>>>> though,
> > >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that
> beneficial.
> > >> > There
> > >> > > is
> > >> > > > > > some
> > >> > > > > > >> > good
> > >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > >> > > > > > >> > > > > >> > >>>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on
> SAMZA-335.
> > >> > Again,
> > >> > > we
> > >> > > > > > took
> > >> > > > > > >> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > >> > > > > > >> > > > > >> > >>>>>> path,
> > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> > differences
> > >> > > > between
> > >> > > > > > HDFS
> > >> > > > > > >> > and
> > >> > > > > > >> > > > > Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > >> > > > > > >> > > > > >> > >>>>>> has
> > >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has
> partitions.
> > >> This
> > >> > > > leads
> > >> > > > > to
> > >> > > > > > >> less
> > >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
> > >> > > processors
> > >> > > > > on
> > >> > > > > > top
> > >> > > > > > >> > of
> > >> > > > > > >> > > > > Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a
> > crutch.
> > >> > > Samza
> > >> > > > > > doesn't
> > >> > > > > > >> > > have
> > >> > > > > > >> > > > > any
> > >> > > > > > >> > > > > >> > >>>>>>>>> built
> > >> > > > > > >> > > > > >> > >>>>> in
> > >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
> > >> > depends
> > >> > > on
> > >> > > > > the
> > >> > > > > > >> > > dynamic
> > >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to
> > handle
> > >> > > > restarts
> > >> > > > > > >> when a
> > >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > >> > > > > > >> > > > > >> > >>>>>>> made
> > >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> > >> standalone
> > >> > > Samza
> > >> > > > > > >> > container
> > >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is
> good,
> > >> but I
> > >> > > > think
> > >> > > > > > that
> > >> > > > > > >> > > we've
> > >> > > > > > >> > > > > >> gone
> > >> > > > > > >> > > > > >> > >>>>>>>>> too
> > >> > > > > > >> > > > > >> > >>>>>> far
> > >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > >> > > > (SystemConsumer,
> > >> > > > > > >> > > > > SystemProducer,
> > >> > > > > > >> > > > > >> > >>>> etc).
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just
> > about
> > >> > every
> > >> > > > > > >> component
> > >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > >> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> > >> > > ConfigRewriter,
> > >> > > > > > etc).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> > >> > forgotten,
> > >> > > as
> > >> > > > > > well.
> > >> > > > > > >> > Some
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>>>>>> these
> > >> > > > > > >> > > > > >> > >>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not
> to
> > >> be.
> > >> > > This
> > >> > > > > all
> > >> > > > > > >> comes
> > >> > > > > > >> > > at
> > >> > > > > > >> > > > a
> > >> > > > > > >> > > > > >> cost:
> > >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is
> > making
> > >> it
> > >> > > > harder
> > >> > > > > > for
> > >> > > > > > >> > our
> > >> > > > > > >> > > > > users
> > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>> pick
> > >> > > > > > >> > > > > >> > >>>>>> up
> > >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It
> > also
> > >> > makes
> > >> > > > it
> > >> > > > > > >> > difficult
> > >> > > > > > >> > > > for
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about
> > what
> > >> the
> > >> > > > > > >> > > characteristics
> > >> > > > > > >> > > > of
> > >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> > >> characteristics
> > >> > > > change
> > >> > > > > > >> > > depending
> > >> > > > > > >> > > > on
> > >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are
> > most
> > >> > > visible
> > >> > > > > in
> > >> > > > > > the
> > >> > > > > > >> > > > System
> > >> > > > > > >> > > > > >> APIs.
> > >> > > > > > >> > > > > >> > >>>>> What
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
> > >> functional is
> > >> > > > Kafka
> > >> > > > > > as
> > >> > > > > > >> its
> > >> > > > > > >> > > > > >> > >>>>>>>>> transport
> > >> > > > > > >> > > > > >> > >>>>>> layer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> But
> > >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use
> > >> cases
> > >> > > into
> > >> > > > > one
> > >> > > > > > >> API:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports
> both
> > >> of
> > >> > > these
> > >> > > > > use
> > >> > > > > > >> > cases.
> > >> > > > > > >> > > > The
> > >> > > > > > >> > > > > >> > >>>>>>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>> is,
> > >> > > > > > >> > > > > >> > >>>>>>>>> we
> > >> > > > > > >> > > > > >> > >>>>>>>>> actually want different features
> for
> > >> each
> > >> > > use
> > >> > > > > > case.
> > >> > > > > > >> By
> > >> > > > > > >> > > > > >> papering
> > >> > > > > > >> > > > > >> > >>>>>>>>> over
> > >> > > > > > >> > > > > >> > >>>>>>> these
> > >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a
> > single
> > >> > API,
> > >> > > > > we've
> > >> > > > > > >> > > > introduced
> > >> > > > > > >> > > > > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > >> > > > > > >> > > > > >> > >>>>>>> leaky
> > >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like
> > in
> > >> (2)
> > >> > > is
> > >> > > > to
> > >> > > > > > have
> > >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
> > >> > offsets
> > >> > > > > (like
> > >> > > > > > >> > Kafka).
> > >> > > > > > >> > > > > This
> > >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > >> > > > > > >> > > > > >> > >>>>> with
> > >> > > > > > >> > > > > >> > >>>>>>> (1),
> > >> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems
> have
> > >> > > > different
> > >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> > >> mailing
> > >> > > list
> > >> > > > > and
> > >> > > > > > >> the
> > >> > > > > > >> > > SQL
> > >> > > > > > >> > > > > >> JIRAs
> > >> > > > > > >> > > > > >> > >>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> > >> > > replayability.
> > >> > > > > > Kafka
> > >> > > > > > >> > > allows
> > >> > > > > > >> > > > us
> > >> > > > > > >> > > > > >> to
> > >> > > > > > >> > > > > >> > >>>>> rewind
> > >> > > > > > >> > > > > >> > >>>>>>>>> when
> > >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other
> > systems
> > >> > > don't.
> > >> > > > In
> > >> > > > > > some
> > >> > > > > > >> > > > cases,
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>>> return
> > >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > >> > > > > > >> WikipediaSystemConsumer)
> > >> > > > > > >> > > > > because
> > >> > > > > > >> > > > > >> > >>>>>>>>> they
> > >> > > > > > >> > > > > >> > >>>>>> have
> > >> > > > > > >> > > > > >> > >>>>>>> no
> > >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example.
> > Kafka
> > >> > > > supports
> > >> > > > > > >> > > > > partitioning,
> > >> > > > > > >> > > > > >> > >>>>>>>>> but
> > >> > > > > > >> > > > > >> > >>>>> many
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
> > >> having a
> > >> > > > single
> > >> > > > > > >> > > partition
> > >> > > > > > >> > > > > for
> > >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other
> systems
> > >> model
> > >> > > > > > >> partitioning
> > >> > > > > > >> > > > > >> > >>>> differently (e.g.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also
> a
> > >> mess.
> > >> > > > > > Creating
> > >> > > > > > >> > > streams
> > >> > > > > > >> > > > > in
> > >> > > > > > >> > > > > >> a
> > >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> > >> impossible.
> > >> > > As
> > >> > > > is
> > >> > > > > > >> > modeling
> > >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > >> > > > > > >> > > > > >> > >>>>> for
> > >> > > > > > >> > > > > >> > >>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> > >> partitions,
> > >> > > > > location,
> > >> > > > > > >> > etc).
> > >> > > > > > >> > > > The
> > >> > > > > > >> > > > > >> > >>>>>>>>> list
> > >> > > > > > >> > > > > >> > >>>>> goes
> > >> > > > > > >> > > > > >> > >>>>>>> on.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing
> > >> Samza,
> > >> > > > > Kafka's
> > >> > > > > > >> > > consumer
> > >> > > > > > >> > > > > and
> > >> > > > > > >> > > > > >> > >>>>> producer
> > >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set.
> > On
> > >> the
> > >> > > > > > >> > consumer-side,
> > >> > > > > > >> > > > you
> > >> > > > > > >> > > > > >> > >>>>>>>>> had two
> > >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
> > consumer,
> > >> or
> > >> > > the
> > >> > > > > > simple
> > >> > > > > > >> > > > > consumer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> The
> > >> > > > > > >> > > > > >> > >>>>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was
> > that
> > >> it
> > >> > > > > > controlled
> > >> > > > > > >> > your
> > >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments,
> and
> > >> the
> > >> > > order
> > >> > > > > in
> > >> > > > > > >> which
> > >> > > > > > >> > > you
> > >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > >> > > > > > >> > > > > >> > >>>>> problem
> > >> > > > > > >> > > > > >> > >>>>>>>>> with
> > >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's
> not
> > >> > > simple.
> > >> > > > > It's
> > >> > > > > > >> > basic.
> > >> > > > > > >> > > > You
> > >> > > > > > >> > > > > >> > >>>>>>>>> end up
> > >> > > > > > >> > > > > >> > >>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really
> low-level
> > >> stuff
> > >> > > > that
> > >> > > > > > you
> > >> > > > > > >> > > > > shouldn't.
> > >> > > > > > >> > > > > >> > >>>>>>>>> We
> > >> > > > > > >> > > > > >> > >>>>>> spent a
> > >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > >> > > > KafkaSystemConsumer
> > >> > > > > > very
> > >> > > > > > >> > > > robust.
> > >> > > > > > >> > > > > >> It
> > >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some
> cool
> > >> > > features:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering
> and
> > >> > > > > > prioritization.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> > >> assignment
> > >> > > to
> > >> > > > > > support
> > >> > > > > > >> > > > joins,
> > >> > > > > > >> > > > > >> > >>>>>>>>> global
> > >> > > > > > >> > > > > >> > >>>>>> state
> > >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)),
> > etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> > >> > checkpointing.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time
> > is
> > >> > that
> > >> > > > > these
> > >> > > > > > >> > > features
> > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > >> > > > > > >> > > > > >> > >>>>>>> actually
> > >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
> > consumers
> > >> > (not
> > >> > > > just
> > >> > > > > > >> Samza
> > >> > > > > > >> > > > stream
> > >> > > > > > >> > > > > >> > >>>>>> processors)
> > >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like
> > joins
> > >> > and
> > >> > > > > > partition
> > >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> > >> > conclusion.
> > >> > > > > > They're
> > >> > > > > > >> > > adding
> > >> > > > > > >> > > > a
> > >> > > > > > >> > > > > >> ton
> > >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> > >> consumer
> > >> > > > > > >> > > implementation.
> > >> > > > > > >> > > > > To a
> > >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > >> > > > > > >> > > > > >> > >>>>> it's
> > >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've
> already
> > >> done
> > >> > > in
> > >> > > > > > Samza.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up
> > taking
> > >> a
> > >> > > very
> > >> > > > > > similar
> > >> > > > > > >> > > > > approach
> > >> > > > > > >> > > > > >> > >>>>>>>>> to
> > >> > > > > > >> > > > > >> > >>>>>> Samza's
> > >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> > implementation
> > >> for
> > >> > > > > > handling
> > >> > > > > > >> > > offset
> > >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> > >> management
> > >> > > > > feature
> > >> > > > > > >> > stores
> > >> > > > > > >> > > > > >> offset
> > >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows
> > >> you to
> > >> > > > fetch
> > >> > > > > > them
> > >> > > > > > >> > > from
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste,
> > >> since
> > >> > we
> > >> > > > > could
> > >> > > > > > >> have
> > >> > > > > > >> > > > shared
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>> work
> > >> > > > > > >> > > > > >> > >>>>>> if
> > >> > > > > > >> > > > > >> > >>>>>>>>> it
> > >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
> > >> get-go.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> > >> radical
> > >> > > > > > proposal.
> > >> > > > > > >> > Samza
> > >> > > > > > >> > > > is
> > >> > > > > > >> > > > > >> > >>>>> relatively
> > >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture
> to
> > >> say
> > >> > > that
> > >> > > > > > we're
> > >> > > > > > >> > > near a
> > >> > > > > > >> > > > > 1.0
> > >> > > > > > >> > > > > >> > >>>>>> release.
> > >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what
> > >> we've
> > >> > > > > learned,
> > >> > > > > > and
> > >> > > > > > >> > > begin
> > >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > >> > > > > > >> > > > > >> > >>>>>>> about
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we
> > >> change if
> > >> > > we
> > >> > > > > were
> > >> > > > > > >> > > starting
> > >> > > > > > >> > > > > >> from
> > >> > > > > > >> > > > > >> > >>>>>> scratch?
> > >> > > > > > >> > > > > >> > >>>>>>>>> My
> > >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the
> *only*
> > >> way
> > >> > to
> > >> > > > run
> > >> > > > > > Samza
> > >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all
> direct
> > >> > > > > dependences
> > >> > > > > > on
> > >> > > > > > >> > > YARN,
> > >> > > > > > >> > > > > >> Mesos,
> > >> > > > > > >> > > > > >> > >>>> etc.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to
> support
> > >> only
> > >> > > > Kafka
> > >> > > > > > as
> > >> > > > > > >> the
> > >> > > > > > >> > > > > stream
> > >> > > > > > >> > > > > >> > >>>>>> processing
> > >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics,
> > logging,
> > >> > > > > > >> serialization,
> > >> > > > > > >> > > and
> > >> > > > > > >> > > > > >> > >>>>>>>>> config
> > >> > > > > > >> > > > > >> > >>>>>>> systems,
> > >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues
> > that
> > >> I
> > >> > > > > outlined
> > >> > > > > > >> > above.
> > >> > > > > > >> > > It
> > >> > > > > > >> > > > > >> > >>>>>>>>> should
> > >> > > > > > >> > > > > >> > >>>>> also
> > >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> > >> > > > dramatically.
> > >> > > > > > >> > > Supporting
> > >> > > > > > >> > > > > >> only
> > >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow
> > >> Samza
> > >> > to
> > >> > > be
> > >> > > > > > >> executed
> > >> > > > > > >> > > on
> > >> > > > > > >> > > > > YARN
> > >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > >> > > > Marathon/Aurora),
> > >> > > > > or
> > >> > > > > > >> most
> > >> > > > > > >> > > > other
> > >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > >> > > > > > >> > > > > >> > >>>>>>> deployment
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a
> lot
> > >> > easier
> > >> > > > for
> > >> > > > > > new
> > >> > > > > > >> > > users.
> > >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > >> > > > > > >> > > > > >> > >>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without
> > YARN.
> > >> > The
> > >> > > > drop
> > >> > > > > > in
> > >> > > > > > >> > > mailing
> > >> > > > > > >> > > > > >> list
> > >> > > > > > >> > > > > >> > >>>>>> traffic
> > >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
> > >> overdue to
> > >> > > me.
> > >> > > > > The
> > >> > > > > > >> > > reality
> > >> > > > > > >> > > > > is,
> > >> > > > > > >> > > > > >> > >>>>> everyone
> > >> > > > > > >> > > > > >> > >>>>>>>>> that
> > >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
> > >> Kafka.
> > >> > We
> > >> > > > > > basically
> > >> > > > > > >> > > > require
> > >> > > > > > >> > > > > >> it
> > >> > > > > > >> > > > > >> > >>>>>> already
> > >> > > > > > >> > > > > >> > >>>>>>> in
> > >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work.
> > Those
> > >> > that
> > >> > > > are
> > >> > > > > > >> using
> > >> > > > > > >> > > > other
> > >> > > > > > >> > > > > >> > >>>>>>>>> systems
> > >> > > > > > >> > > > > >> > >>>>>> are
> > >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into
> > >> Kafka
> > >> > > (1),
> > >> > > > > and
> > >> > > > > > >> then
> > >> > > > > > >> > > > they
> > >> > > > > > >> > > > > do
> > >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
> > >> already
> > >> > > > > > discussion (
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > >
> > >> > > > > > >> >
> > >> > > > > >
> > >> > >
> > >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >> > > > > > >> > > > > >> > >>>>> 767
> > >> > > > > > >> > > > > >> > >>>>>>>>> )
> > >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into
> > Kafka
> > >> > > > extremely
> > >> > > > > > >> easy.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple
> with
> > >> > Kafka,
> > >> > > > we
> > >> > > > > > can
> > >> > > > > > >> > > > leverage
> > >> > > > > > >> > > > > a
> > >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > >> > > > > > >> > > > > >> > >>>>>>> their
> > >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> > >> maintain
> > >> > > our
> > >> > > > > own
> > >> > > > > > >> > config,
> > >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > >> > > > > > >> > > > > >> > >>>>> etc.
> > >> > > > > > >> > > > > >> > >>>>>>> We
> > >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries,
> > and
> > >> > make
> > >> > > > them
> > >> > > > > > >> > better.
> > >> > > > > > >> > > > This
> > >> > > > > > >> > > > > >> > >>>>>>>>> will
> > >> > > > > > >> > > > > >> > >>>>> also
> > >> > > > > > >> > > > > >> > >>>>>>>>> allow us to share the
> > >> consumer/producer
> > >> > > APIs,
> > >> > > > > and
> > >> > > > > > >> will
> > >> > > > > > >> > > let
> > >> > > > > > >> > > > > us
> > >> > > > > > >> > > > > >> > >>>>> leverage
> > >> > > > > > >> > > > > >> > >>>>>>>>> their offset management and
> > partition
> > >> > > > > management,
> > >> > > > > > >> > rather
> > >> > > > > > >> > > > > than
> > >> > > > > > >> > > > > >> > >>>>>>>>> having
> > >> > > > > > >> > > > > >> > >>>>>> our
> > >> > > > > > >> > > > > >> > >>>>>>>>> own. All of the coordinator stream
> > >> code
> > >> > > would
> > >> > > > > go
> > >> > > > > > >> away,
> > >> > > > > > >> > > as
> > >> > > > > > >> > > > > >> would
> > >> > > > > > >> > > > > >> > >>>>>>>>> most
> > >> > > > > > >> > > > > >> > >>>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably
> > >> have
> > >> > to
> > >> > > > push
> > >> > > > > > some
> > >> > > > > > >> > > > > partition
> > >> > > > > > >> > > > > >> > >>>>>>> management
> > >> > > > > > >> > > > > >> > >>>>>>>>> features into the Kafka broker,
> but
> > >> > they're
> > >> > > > > > already
> > >> > > > > > >> > > moving
> > >> > > > > > >> > > > > in
> > >> > > > > > >> > > > > >> > >>>>>>>>> that direction with the new
> consumer
> > >> API.
> > >> > > The
> > >> > > > > > >> features
> > >> > > > > > >> > > we
> > >> > > > > > >> > > > > have
> > >> > > > > > >> > > > > >> > >>>>>>>>> for
> > >> > > > > > >> > > > > >> > >>>>>> partition
> > >> > > > > > >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza,
> > and
> > >> > seem
> > >> > > > > like
> > >> > > > > > >> they
> > >> > > > > > >> > > > should
> > >> > > > > > >> > > > > >> be
> > >> > > > > > >> > > > > >> > >>>>>>>>> in
> > >> > > > > > >> > > > > >> > >>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>> anyway. There will always be some
> > >> niche
> > >> > > > usages
> > >> > > > > > which
> > >> > > > > > >> > > will
> > >> > > > > > >> > > > > >> > >>>>>>>>> require
> > >> > > > > > >> > > > > >> > >>>>>> extra
> > >> > > > > > >> > > > > >> > >>>>>>>>> care and hence full control over
> > >> > partition
> > >> > > > > > >> assignments
> > >> > > > > > >> > > > much
> > >> > > > > > >> > > > > >> > >>>>>>>>> like the
> > >> > > > > > >> > > > > >> > >>>>>>> Kafka
> > >> > > > > > >> > > > > >> > >>>>>>>>> low level consumer api. These
> would
> > >> > > continue
> > >> > > > to
> > >> > > > > > be
> > >> > > > > > >> > > > > supported.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> These items will be good for the
> > Samza
> > >> > > > > community.
> > >> > > > > > >> > > They'll
> > >> > > > > > >> > > > > make
> > >> > > > > > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it
> > >> easier
> > >> > for
> > >> > > > > > >> developers
> > >> > > > > > >> > > to
> > >> > > > > > >> > > > > add
> > >> > > > > > >> > > > > >> > >>>>>>>>> new features.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large
> > (and
> > >> > > > somewhat
> > >> > > > > > >> > backwards
> > >> > > > > > >> > > > > >> > >>>>> incompatible
> > >> > > > > > >> > > > > >> > >>>>>>>>> change). If we choose to go this
> > >> route,
> > >> > > it's
> > >> > > > > > >> important
> > >> > > > > > >> > > > that
> > >> > > > > > >> > > > > we
> > >> > > > > > >> > > > > >> > >>>>> openly
> > >> > > > > > >> > > > > >> > >>>>>>>>> communicate how we're going to
> > >> provide a
> > >> > > > > > migration
> > >> > > > > > >> > path
> > >> > > > > > >> > > > from
> > >> > > > > > >> > > > > >> > >>>>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>> existing
> > >> > > > > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make
> > >> > > incompatible
> > >> > > > > > >> > changes).
> > >> > > > > > >> > > I
> > >> > > > > > >> > > > > >> think
> > >> > > > > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need
> to
> > >> > > provide a
> > >> > > > > > >> wrapper
> > >> > > > > > >> > to
> > >> > > > > > >> > > > > allow
> > >> > > > > > >> > > > > >> > >>>>>>>>> existing StreamTask
> implementations
> > to
> > >> > > > continue
> > >> > > > > > >> > running
> > >> > > > > > >> > > on
> > >> > > > > > >> > > > > the
> > >> > > > > > >> > > > > >> > >>>> new container.
> > >> > > > > > >> > > > > >> > >>>>>>> It's
> > >> > > > > > >> > > > > >> > >>>>>>>>> also important that we openly
> > >> communicate
> > >> > > > about
> > >> > > > > > >> > timing,
> > >> > > > > > >> > > > and
> > >> > > > > > >> > > > > >> > >>>>>>>>> stages
> > >> > > > > > >> > > > > >> > >>>>> of
> > >> > > > > > >> > > > > >> > >>>>>>> the
> > >> > > > > > >> > > > > >> > >>>>>>>>> migration.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure
> > you
> > >> > have
> > >> > > > > > opinions.
> > >> > > > > > >> > :)
> > >> > > > > > >> > > > > Please
> > >> > > > > > >> > > > > >> > >>>>>>>>> send
> > >> > > > > > >> > > > > >> > >>>>>> your
> > >> > > > > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>> Cheers,
> > >> > > > > > >> > > > > >> > >>>>>>>>> Chris
> > >> > > > > > >> > > > > >> > >>>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>> --
> > >> > > > > > >> > > > > >> > >>>>>> -- Guozhang
> > >> > > > > > >> > > > > >> > >>>>>>
> > >> > > > > > >> > > > > >> > >>>>>
> > >> > > > > > >> > > > > >> > >>>>
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> > >>
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >> >
> > >> > > > > > >> > > > > >>
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > > >
> > >> > > > > > >> > > > >
> > >> > > > > > >> > > >
> > >> > > > > > >> > >
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Garrett Barton <ga...@gmail.com>.
Yi,

 What you just summarized makes a whole lot more sense to me.  Shamelessly
I am looking at this shift as a customer with a production workflow riding
on it so I am looking for some kind of consistency into the future of
Samza.  This makes me feel a lot better about it.


Thank you!

On Sun, Jul 12, 2015 at 10:44 PM, Yi Pan <ni...@gmail.com> wrote:

> Just to make it explicitly clear what I am proposing, here is a version of
> more detailed description:
>
> The fourth option (in addition to what Jakob summarized) we are proposing
> is:
>
> - Recharter Samza to “stream processing as a service”
>
> - The current Samza core (the basic transformation API w/ basic partition
> and offset management build-in) will be moved to Kafka Streams (i.e. part
> of Kafka) and supports “run-as-a-library”
>
> - Deprecate the SystemConsumers and SystemProducers APIs and move them to
> Copycat
>
> - The current SQL development:
>
>    * physical operators and a Trident-like stream API should stay in Kafka
> Streams as libraries, enabling any standalone deployment to use the core
> window/join functions
>
>    * query parser/planner and execution on top of a distributed service
> should stay in new Samza (i.e. “stream processing as a service”)
>
> - Advanced features related to job scheduling/state management stays in new
> Samza (i.e. “streaming processing as a service”)
>
>    * Any advanced PartitionManager implementation that can be plugged into
> Kafka Streams
>
>    * Any auto-scaling, dynamic configuration via coordinator stream
>
>    * Any advanced state management s.t. host-affinity etc.
>
>
> Pros:
>
> - W/ the current Samza core as Kafka Streams and move the ingestion to
> Copycat, we achieved most of the goals in the initial proposal:
>
>    * Tighter coupling w/ Kafka
>
>    * Reuse Kafka’s build-in functionalities, such as offset manager, basic
> partition distribution
>
>    * Separation of ingestion vs transformation APIs
>
>    * offload a lot of system-specific configuration to Kafka Streams and
> Copycat (i.e. SystemFactory configure, serde configure, etc.)
>
>    * remove YARN dependency and make standalone deployment easy. As
> Guozhang mentioned, it would be really easy to start a process that
> internally run Kafka Streams as library.
>
> - By re-chartering Samza as “stream processing as a service”, we address
> the concern regarding to
>
>    * Pluggable partition management
>
>    * Running in a distributed cluster to manage process lifecycle,
> fault-tolerance, resource-allocation, etc.
>
>    * More advanced features s.t. host-affinity, auto-scaling, and dynamic
> configure changes, etc.
>
>
> Regarding to the code and community organization, I think the following may
> be the best:
>
> Code:
>
> - A Kafka sub-project Kafka Streams to hold samza-core, samza-kv-store, and
> the physical operator layer as library in SQL: this would allow better
> alignment w/ Kafka, in code, doc, and branding
>
> - Retain the current Samza project just to keep
>
>    * A pluggable explicit partition management in Kafka Streams client
>
>    * Integration w/ cluster-management systems for advanced features:
>
>       * host-affinity, auto-scaling,, dynamic configuration, etc.
>
>    * It will fully depend on the Kafka Streams API and remove all support
> for SystemConsumers/SystemProducers in the future
>
> Community: (this is almost the same as what Chris proposed)
>
> - Kafka Streams: the current Samza community should be supporting this
> effort together with some Kafka members, since most of the code here will
> be from samza-core, samza-kv-store, and samza-sql.
>
> - new Samza: the current Samza community should continue serve the course
> to support more advanced features to run Kafka Streams as a service.
> Arguably, the new Samza framework may be used to run Copycat workers as
> well, at least to manage Copycat worker’s lifecycle in a clustered
> environment. Hence, it would stay as a general stream processing framework
> that takes in any source and output to any destination, just the transport
> system is fixed to Kafka.
>
> On Sun, Jul 12, 2015 at 7:29 PM, Yi Pan <ni...@gmail.com> wrote:
>
> > Hi, Chris,
> >
> > Thanks for sending out this concrete set of points here. I agree w/ all
> > but have a slight different point view on 8).
> >
> > My view on this is: instead of sunset Samza as TLP, can we re-charter the
> > scope of Samza to be the home for "running streaming process as a
> service"?
> >
> > My main motivation is from the following points from a long internal
> > discussion in LinkedIn:
> >
> > - There is a clear ask for pluggable partition management, like we do in
> > LinkedIn, and as Ben Kirwin has mentioned in
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCACuX-D-yJX++2GnF_1Laf10KYUVyamg7UP_Dt19v0ZNmmhbCPQ@mail.gmail.com%3E
> > - There are concerns on lack of support for running stream processing in
> a
> > cluster: lifecycle management, resource allocation, fault tolerance, etc.
> > - There is a question to how to support more advanced features s.t.
> > host-affinity, auto-scaling, and dynamic configuration in Samza jobs, as
> > raised by Martin here:
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3C0D66EFD0-B7CD-4E4E-8B2F-2716167C313C@kleppmann.com%3E
> >
> > We have use cases that need to address all the above three cases and most
> > of the functions are all in the current Samza project, in some flavor. We
> > are all supporting to merge the samza-core functionalities into Kafka
> > Streams, but there is a question where we keep these functions in the
> > future. One option is to start a new project that includes these
> functions
> > that are closely related w/ "run stream-processing as-a-service", while
> > another personally more attractive option is to re-charter Samza project
> > just do "run stream processing as-a-service". We can avoid the overhead
> of
> > re-starting another community for this project. Personally, I felt that
> > here are the benefits we should be getting:
> >
> > 1. We have already agreed mostly that Kafka Streams API would allow some
> > pluggable partition management functions. Hence, the advanced partition
> > management can live out-side the new Kafka Streams core w/o affecting the
> > run-as-a-library model in Kafka Streams.
> > 2. The integration w/ cluster management system and advanced features
> > listed above stays in the same project and allow existing users enjoy
> > no-impact migration to Kafka Stream as the core. That also addresses
> Tim's
> > question on "removing the support for YARN".
> > 3. A separate project for stream-processing-as-a-service also allow the
> > new Kafka Streams being independent to any cluster management and just
> > focusing on stream process core functions, while leaving the functions
> that
> > requires cluster-resource and state management to a separate layer.
> >
> > Please feel free to comment. Thanks!
> >
> > On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <cr...@apache.org>
> > wrote:
> >
> >> Hey all,
> >>
> >> I want to start by saying that I'm absolutely thrilled to be a part of
> >> this
> >> community. The amount of level-headed, thoughtful, educated discussion
> >> that's gone on over the past ~10 days is overwhelming. Wonderful.
> >>
> >> It seems like discussion is waning a bit, and we've reached some
> >> conclusions. There are several key emails in this threat, which I want
> to
> >> call out:
> >>
> >> 1. Jakob's summary of the three potential ways forward.
> >>
> >>
> >>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> >> 2. Julian's call out that we should be focusing on community over code.
> >>
> >>
> >>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> >> 3. Martin's summary about the benefits of merging communities.
> >>
> >>
> >>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> >> 4. Jakob's comments about the distinction between community and code
> >> paths.
> >>
> >>
> >>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> >>
> >> I agree with the comments on all of these emails. I think Martin's
> summary
> >> of his position aligns very closely with my own. To that end, I think we
> >> should get concrete about what the proposal is, and call a vote on it.
> >> Given that Jay, Martin, and I seem to be aligning fairly closely, I
> think
> >> we should start with:
> >>
> >> 1. [community] Make Samza a subproject of Kafka.
> >> 2. [community] Make all Samza PMC/committers committers of the
> subproject.
> >> 3. [community] Migrate Samza's website/documentation into Kafka's.
> >> 4. [code] Have the Samza community and the Kafka community start a
> >> from-scratch reboot together in the new Kafka subproject. We can
> >> borrow/copy &  paste significant chunks of code from Samza's code base.
> >> 5. [code] The subproject would intentionally eliminate support for both
> >> other streaming systems and all deployment systems.
> >> 6. [code] Attempt to provide a bridge from our SystemConsumer to KIP-26
> >> (copy cat)
> >> 7. [code] Attempt to provide a bridge from the new subproject's
> processor
> >> interface to our legacy StreamTask interface.
> >> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> >> subproject that has a fault-tolerant container with state management.
> >>
> >> It's likely that (6) and (7) won't be fully drop-in. Still, the closer
> we
> >> can get, the better it's going to be for our existing community.
> >>
> >> One thing that I didn't touch on with (2) is whether any Samza PMC
> members
> >> should be rolled into Kafka PMC membership as well (though, Jay and
> Jakob
> >> are already PMC members on both). I think that Samza's community
> deserves
> >> a
> >> voice on the PMC, so I'd propose that we roll at least a few PMC members
> >> into the Kafka PMC, but I don't have a strong framework for which people
> >> to
> >> pick.
> >>
> >> Before (8), I think that Samza's TLP can continue to commit bug fixes
> and
> >> patches as it sees fit, provided that we openly communicate that we
> won't
> >> necessarily migrate new features to the new subproject, and that the TLP
> >> will be shut down after the migration to the Kafka subproject occurs.
> >>
> >> Jakob, I could use your guidance here about about how to achieve this
> from
> >> an Apache process perspective (sorry).
> >>
> >> * Should I just call a vote on this proposal?
> >> * Should it happen on dev or private?
> >> * Do committers have binding votes, or just PMC?
> >>
> >> Having trouble finding much detail on the Apache wikis. :(
> >>
> >> Cheers,
> >> Chris
> >>
> >> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com> wrote:
> >>
> >> > Thanks, Jay. This argument persuaded me actually. :)
> >> >
> >> > Fang, Yan
> >> > yanfang724@gmail.com
> >> >
> >> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> >> >
> >> > > Hey Yan,
> >> > >
> >> > > Yeah philosophically I think the argument is that you should capture
> >> the
> >> > > stream in Kafka independent of the transformation. This is
> obviously a
> >> > > Kafka-centric view point.
> >> > >
> >> > > Advantages of this:
> >> > > - In practice I think this is what e.g. Storm people often end up
> >> doing
> >> > > anyway. You usually need to throttle any access to a live serving
> >> > database.
> >> > > - Can have multiple subscribers and they get the same thing without
> >> > > additional load on the source system.
> >> > > - Applications can tap into the stream if need be by subscribing.
> >> > > - You can debug your transformation by tailing the Kafka topic with
> >> the
> >> > > console consumer
> >> > > - Can tee off the same data stream for batch analysis or Lambda arch
> >> > style
> >> > > re-processing
> >> > >
> >> > > The disadvantage is that it will use Kafka resources. But the idea
> is
> >> > > eventually you will have multiple subscribers to any data source (at
> >> > least
> >> > > for monitoring) so you will end up there soon enough anyway.
> >> > >
> >> > > Down the road the technical benefit is that I think it gives us a
> good
> >> > path
> >> > > towards end-to-end exactly once semantics from source to
> destination.
> >> > > Basically the connectors need to support idempotence when talking to
> >> > Kafka
> >> > > and we need the transactional write feature in Kafka to make the
> >> > > transformation atomic. This is actually pretty doable if you
> separate
> >> > > connector=>kafka problem from the generic transformations which are
> >> > always
> >> > > kafka=>kafka. However I think it is quite impossible to do in a
> >> > all_things
> >> > > => all_things environment. Today you can say "well the semantics of
> >> the
> >> > > Samza APIs depend on the connectors you use" but it is actually
> worse
> >> > then
> >> > > that because the semantics actually depend on the pairing of
> >> > connectors--so
> >> > > not only can you probably not get a usable "exactly once" guarantee
> >> > > end-to-end it can actually be quite hard to reverse engineer what
> >> > property
> >> > > (if any) your end-to-end flow has if you have heterogenous systems.
> >> > >
> >> > > -Jay
> >> > >
> >> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
> >> wrote:
> >> > >
> >> > > > {quote}
> >> > > > maintained in a separate repository and retaining the existing
> >> > > > committership but sharing as much else as possible (website, etc)
> >> > > > {quote}
> >> > > >
> >> > > > Overall, I agree on this idea. Now the question is more about "how
> >> to
> >> > do
> >> > > > it".
> >> > > >
> >> > > > On the other hand, one thing I want to point out is that, if we
> >> decide
> >> > to
> >> > > > go this way, how do we want to support
> >> > > > otherSystem-transformation-otherSystem use case?
> >> > > >
> >> > > > Basically, there are four user groups here:
> >> > > >
> >> > > > 1. Kafka-transformation-Kafka
> >> > > > 2. Kafka-transformation-otherSystem
> >> > > > 3. otherSystem-transformation-Kafka
> >> > > > 4. otherSystem-transformation-otherSystem
> >> > > >
> >> > > > For group 1, they can easily use the new Samza library to achieve.
> >> For
> >> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka or
> >> > Kafka->
> >> > > > transformation -> copyCat.
> >> > > >
> >> > > > The problem is for group 4. Do we want to abandon this or still
> >> support
> >> > > it?
> >> > > > Of course, this use case can be achieved by using copyCat ->
> >> > > transformation
> >> > > > -> Kafka -> transformation -> copyCat, the thing is how we
> persuade
> >> > them
> >> > > to
> >> > > > do this long chain. If yes, it will also be a win for Kafka too.
> Or
> >> if
> >> > > > there is no one in this community actually doing this so far,
> maybe
> >> ok
> >> > to
> >> > > > not support the group 4 directly.
> >> > > >
> >> > > > Thanks,
> >> > > >
> >> > > > Fang, Yan
> >> > > > yanfang724@gmail.com
> >> > > >
> >> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
> >> wrote:
> >> > > >
> >> > > > > Yeah I agree with this summary. I think there are kind of two
> >> > questions
> >> > > > > here:
> >> > > > > 1. Technically does alignment/reliance on Kafka make sense
> >> > > > > 2. Branding wise (naming, website, concepts, etc) does alignment
> >> with
> >> > > > Kafka
> >> > > > > make sense
> >> > > > >
> >> > > > > Personally I do think both of these things would be really
> >> valuable,
> >> > > and
> >> > > > > would dramatically alter the trajectory of the project.
> >> > > > >
> >> > > > > My preference would be to see if people can mostly agree on a
> >> > direction
> >> > > > > rather than splintering things off. From my point of view the
> >> ideal
> >> > > > outcome
> >> > > > > of all the options discussed would be to make Samza a closely
> >> aligned
> >> > > > > subproject, maintained in a separate repository and retaining
> the
> >> > > > existing
> >> > > > > committership but sharing as much else as possible (website,
> >> etc). No
> >> > > > idea
> >> > > > > about how these things work, Jacob, you probably know more.
> >> > > > >
> >> > > > > No discussion amongst the Kafka folks has happened on this, but
> >> > likely
> >> > > we
> >> > > > > should figure out what the Samza community actually wants first.
> >> > > > >
> >> > > > > I admit that this is a fairly radical departure from how things
> >> are.
> >> > > > >
> >> > > > > If that doesn't fly, I think, yeah we could leave Samza as it is
> >> and
> >> > do
> >> > > > the
> >> > > > > more radical reboot inside Kafka. From my point of view that
> does
> >> > leave
> >> > > > > things in a somewhat confusing state since now there are two
> >> stream
> >> > > > > processing systems more or less coupled to Kafka in large part
> >> made
> >> > by
> >> > > > the
> >> > > > > same people. But, arguably that might be a cleaner way to make
> the
> >> > > > cut-over
> >> > > > > and perhaps less risky for Samza community since if it works
> >> people
> >> > can
> >> > > > > switch and if it doesn't nothing will have changed. Dunno, how
> do
> >> > > people
> >> > > > > feel about this?
> >> > > > >
> >> > > > > -Jay
> >> > > > >
> >> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <
> jghoman@gmail.com>
> >> > > wrote:
> >> > > > >
> >> > > > > > >  This leads me to thinking that merging projects and
> >> communities
> >> > > > might
> >> > > > > > be a good idea: with the union of experience from both
> >> communities,
> >> > > we
> >> > > > > will
> >> > > > > > probably build a better system that is better for users.
> >> > > > > > Is this what's being proposed though? Merging the projects
> seems
> >> > like
> >> > > > > > a consequence of at most one of the three directions under
> >> > > discussion:
> >> > > > > > 1) Samza 2.0: The Samza community relies more heavily on Kafka
> >> for
> >> > > > > > configuration, etc. (to a greater or lesser extent to be
> >> > determined)
> >> > > > > > but the Samza community would not automatically merge withe
> >> Kafka
> >> > > > > > community (the Phoenix/HBase example is a good one here).
> >> > > > > > 2) Samza Reboot: The Samza community continues to exist with a
> >> > > limited
> >> > > > > > project scope, but similarly would not need to be part of the
> >> Kafka
> >> > > > > > community (ie given committership) to progress.  Here, maybe
> the
> >> > > Samza
> >> > > > > > team would become a subproject of Kafka (the Board frowns on
> >> > > > > > subprojects at the moment, so I'm not sure if that's even
> >> > feasible),
> >> > > > > > but that would not be required.
> >> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option
> the
> >> > Kafka
> >> > > > > > team builds its own streaming library, possibly off of Jay's
> >> > > > > > prototype, which has not direct lineage to the Samza team.
> >> There's
> >> > > no
> >> > > > > > reason for the Kafka team to bring in the Samza team.
> >> > > > > >
> >> > > > > > Is the Kafka community on board with this?
> >> > > > > >
> >> > > > > > To be clear, all three options under discussion are
> interesting,
> >> > > > > > technically valid and likely healthy directions for the
> project.
> >> > > > > > Also, they are not mutually exclusive.  The Samza community
> >> could
> >> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka community
> >> went
> >> > > > > > forward with 'Hey Samza!'  My points above are directed
> >> entirely at
> >> > > > > > the community aspect of these choices.
> >> > > > > > -Jakob
> >> > > > > >
> >> > > > > > On 10 July 2015 at 09:10, Roger Hoover <
> roger.hoover@gmail.com>
> >> > > wrote:
> >> > > > > > > That's great.  Thanks, Jay.
> >> > > > > > >
> >> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <
> jay@confluent.io>
> >> > > wrote:
> >> > > > > > >
> >> > > > > > >> Yeah totally agree. I think you have this issue even today,
> >> > right?
> >> > > > > I.e.
> >> > > > > > if
> >> > > > > > >> you need to make a simple config change and you're running
> in
> >> > YARN
> >> > > > > today
> >> > > > > > >> you end up bouncing the job which then rebuilds state. I
> >> think
> >> > the
> >> > > > fix
> >> > > > > > is
> >> > > > > > >> exactly what you described which is to have a long timeout
> on
> >> > > > > partition
> >> > > > > > >> movement for stateful jobs so that if a job is just getting
> >> > > bounced,
> >> > > > > and
> >> > > > > > >> the cluster manager (or admin) is smart enough to restart
> it
> >> on
> >> > > the
> >> > > > > same
> >> > > > > > >> host when possible, it can optimistically reuse any
> existing
> >> > state
> >> > > > it
> >> > > > > > finds
> >> > > > > > >> on disk (if it is valid).
> >> > > > > > >>
> >> > > > > > >> So in this model the charter of the CM is to place
> processes
> >> as
> >> > > > > > stickily as
> >> > > > > > >> possible and to restart or re-place failed processes. The
> >> > charter
> >> > > of
> >> > > > > the
> >> > > > > > >> partition management system is to control the assignment of
> >> work
> >> > > to
> >> > > > > > these
> >> > > > > > >> processes. The nice thing about this is that the work
> >> > assignment,
> >> > > > > > timeouts,
> >> > > > > > >> behavior, configs, and code will all be the same across all
> >> > > cluster
> >> > > > > > >> managers.
> >> > > > > > >>
> >> > > > > > >> So I think that prototype would actually give you exactly
> >> what
> >> > you
> >> > > > > want
> >> > > > > > >> today for any cluster manager (or manual placement +
> restart
> >> > > script)
> >> > > > > > that
> >> > > > > > >> was sticky in terms of host placement since there is
> already
> >> a
> >> > > > > > configurable
> >> > > > > > >> partition movement timeout and task-by-task state reuse
> with
> >> a
> >> > > check
> >> > > > > on
> >> > > > > > >> state validity.
> >> > > > > > >>
> >> > > > > > >> -Jay
> >> > > > > > >>
> >> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> >> > > > roger.hoover@gmail.com
> >> > > > > >
> >> > > > > > >> wrote:
> >> > > > > > >>
> >> > > > > > >> > That would be great to let Kafka do as much heavy lifting
> >> as
> >> > > > > possible
> >> > > > > > and
> >> > > > > > >> > make it easier for other languages to implement Samza
> apis.
> >> > > > > > >> >
> >> > > > > > >> > One thing to watch out for is the interplay between
> Kafka's
> >> > > group
> >> > > > > > >> > management and the external scheduler/process manager's
> >> fault
> >> > > > > > tolerance.
> >> > > > > > >> > If a container dies, the Kafka group membership protocol
> >> will
> >> > > try
> >> > > > to
> >> > > > > > >> assign
> >> > > > > > >> > it's tasks to other containers while at the same time the
> >> > > process
> >> > > > > > manager
> >> > > > > > >> > is trying to relaunch the container.  Without some
> >> > consideration
> >> > > > for
> >> > > > > > this
> >> > > > > > >> > (like a configurable amount of time to wait before Kafka
> >> > alters
> >> > > > the
> >> > > > > > group
> >> > > > > > >> > membership), there may be thrashing going on which is
> >> > especially
> >> > > > bad
> >> > > > > > for
> >> > > > > > >> > containers with large amounts of local state.
> >> > > > > > >> >
> >> > > > > > >> > Someone else pointed this out already but I thought it
> >> might
> >> > be
> >> > > > > worth
> >> > > > > > >> > calling out again.
> >> > > > > > >> >
> >> > > > > > >> > Cheers,
> >> > > > > > >> >
> >> > > > > > >> > Roger
> >> > > > > > >> >
> >> > > > > > >> >
> >> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> >> jay@confluent.io>
> >> > > > > wrote:
> >> > > > > > >> >
> >> > > > > > >> > > Hey Roger,
> >> > > > > > >> > >
> >> > > > > > >> > > I couldn't agree more. We spent a bunch of time talking
> >> to
> >> > > > people
> >> > > > > > and
> >> > > > > > >> > that
> >> > > > > > >> > > is exactly the stuff we heard time and again. What
> makes
> >> it
> >> > > > hard,
> >> > > > > of
> >> > > > > > >> > > course, is that there is some tension between
> >> compatibility
> >> > > with
> >> > > > > > what's
> >> > > > > > >> > > there now and making things better for new users.
> >> > > > > > >> > >
> >> > > > > > >> > > I also strongly agree with the importance of
> >> multi-language
> >> > > > > > support. We
> >> > > > > > >> > are
> >> > > > > > >> > > talking now about Java, but for application development
> >> use
> >> > > > cases
> >> > > > > > >> people
> >> > > > > > >> > > want to work in whatever language they are using
> >> elsewhere.
> >> > I
> >> > > > > think
> >> > > > > > >> > moving
> >> > > > > > >> > > to a model where Kafka itself does the group
> membership,
> >> > > > lifecycle
> >> > > > > > >> > control,
> >> > > > > > >> > > and partition assignment has the advantage of putting
> all
> >> > that
> >> > > > > > complex
> >> > > > > > >> > > stuff behind a clean api that the clients are already
> >> going
> >> > to
> >> > > > be
> >> > > > > > >> > > implementing for their consumer, so the added
> >> functionality
> >> > > for
> >> > > > > > stream
> >> > > > > > >> > > processing beyond a consumer becomes very minor.
> >> > > > > > >> > >
> >> > > > > > >> > > -Jay
> >> > > > > > >> > >
> >> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> >> > > > > > roger.hoover@gmail.com>
> >> > > > > > >> > > wrote:
> >> > > > > > >> > >
> >> > > > > > >> > > > Metamorphosis...nice. :)
> >> > > > > > >> > > >
> >> > > > > > >> > > > This has been a great discussion.  As a user of Samza
> >> > who's
> >> > > > > > recently
> >> > > > > > >> > > > integrated it into a relatively large organization, I
> >> just
> >> > > > want
> >> > > > > to
> >> > > > > > >> add
> >> > > > > > >> > > > support to a few points already made.
> >> > > > > > >> > > >
> >> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
> >> currently
> >> > > > exists
> >> > > > > > that
> >> > > > > > >> > I've
> >> > > > > > >> > > > experienced are:
> >> > > > > > >> > > > 1) YARN - YARN is overly complex in many environments
> >> > where
> >> > > > > Puppet
> >> > > > > > >> > would
> >> > > > > > >> > > do
> >> > > > > > >> > > > just fine but it was the only mechanism to get fault
> >> > > > tolerance.
> >> > > > > > >> > > > 2) Configuration - I think I like the idea of
> >> configuring
> >> > > most
> >> > > > > of
> >> > > > > > the
> >> > > > > > >> > job
> >> > > > > > >> > > > in code rather than config files.  In general, I
> think
> >> the
> >> > > > goal
> >> > > > > > >> should
> >> > > > > > >> > be
> >> > > > > > >> > > > to make it harder to make mistakes, especially of the
> >> kind
> >> > > > where
> >> > > > > > the
> >> > > > > > >> > code
> >> > > > > > >> > > > expects something and the config doesn't match.  The
> >> > current
> >> > > > > > config
> >> > > > > > >> is
> >> > > > > > >> > > > quite intricate and error-prone.  For example, the
> >> > > application
> >> > > > > > logic
> >> > > > > > >> > may
> >> > > > > > >> > > > depend on bootstrapping a topic but rather than
> >> asserting
> >> > > that
> >> > > > > in
> >> > > > > > the
> >> > > > > > >> > > code,
> >> > > > > > >> > > > you have to rely on getting the config right.
> Likewise
> >> > with
> >> > > > > > serdes,
> >> > > > > > >> > the
> >> > > > > > >> > > > Java representations produced by various serdes
> (JSON,
> >> > Avro,
> >> > > > > etc.)
> >> > > > > > >> are
> >> > > > > > >> > > not
> >> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> >> without
> >> > > > > changing
> >> > > > > > >> the
> >> > > > > > >> > > > code.   It would be nice for jobs to be able to
> assert
> >> > what
> >> > > > they
> >> > > > > > >> expect
> >> > > > > > >> > > > from their input topics in terms of partitioning.
> >> This is
> >> > > > > > getting a
> >> > > > > > >> > > little
> >> > > > > > >> > > > off topic but I was even thinking about creating a
> >> "Samza
> >> > > > config
> >> > > > > > >> > linter"
> >> > > > > > >> > > > that would sanity check a set of configs.  Especially
> >> in
> >> > > > > > >> organizations
> >> > > > > > >> > > > where config is managed by a different team than the
> >> > > > application
> >> > > > > > >> > > developer,
> >> > > > > > >> > > > it's very hard to get avoid config mistakes.
> >> > > > > > >> > > > 3) Java/Scala centric - for many teams (especially
> >> > > DevOps-type
> >> > > > > > >> folks),
> >> > > > > > >> > > the
> >> > > > > > >> > > > pain of the Java toolchain (maven, slow builds, weak
> >> > command
> >> > > > > line
> >> > > > > > >> > > support,
> >> > > > > > >> > > > configuration over convention) really inhibits
> >> > productivity.
> >> > > > As
> >> > > > > > more
> >> > > > > > >> > and
> >> > > > > > >> > > > more high-quality clients become available for
> Kafka, I
> >> > hope
> >> > > > > > they'll
> >> > > > > > >> > > follow
> >> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> >> proposals
> >> > > in
> >> > > > > this
> >> > > > > > >> > thread
> >> > > > > > >> > > > but please consider other languages in the ecosystem
> as
> >> > > well.
> >> > > > > > From
> >> > > > > > >> > what
> >> > > > > > >> > > > I've heard, Spark has more Python users than
> >> Java/Scala.
> >> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> >> > > > > > >> > > >
> >> > > > > > >> > > >
> >> > > > > > >> > >
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> >> > > > > > >> > > > and are working on a Yeoman generator
> >> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> >> > > Jython/Samza
> >> > > > > > >> projects
> >> > > > > > >> > to
> >> > > > > > >> > > > alleviate some of the pain)
> >> > > > > > >> > > >
> >> > > > > > >> > > > I also want to underscore Jay's point about improving
> >> the
> >> > > user
> >> > > > > > >> > > experience.
> >> > > > > > >> > > > That's a very important factor for adoption.  I think
> >> the
> >> > > goal
> >> > > > > > should
> >> > > > > > >> > be
> >> > > > > > >> > > to
> >> > > > > > >> > > > make Samza as easy to get started with as something
> >> like
> >> > > > > Logstash.
> >> > > > > > >> > > > Logstash is vastly inferior in terms of capabilities
> to
> >> > > Samza
> >> > > > > but
> >> > > > > > >> it's
> >> > > > > > >> > > easy
> >> > > > > > >> > > > to get started and that makes a big difference.
> >> > > > > > >> > > >
> >> > > > > > >> > > > Cheers,
> >> > > > > > >> > > >
> >> > > > > > >> > > > Roger
> >> > > > > > >> > > >
> >> > > > > > >> > > >
> >> > > > > > >> > > >
> >> > > > > > >> > > >
> >> > > > > > >> > > >
> >> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De
> Francisci
> >> > > > Morales <
> >> > > > > > >> > > > gdfm@apache.org> wrote:
> >> > > > > > >> > > >
> >> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> >> Metamorphosis
> >> > > is
> >> > > > a
> >> > > > > > clear
> >> > > > > > >> > > > winner
> >> > > > > > >> > > > > :)
> >> > > > > > >> > > > >
> >> > > > > > >> > > > > --
> >> > > > > > >> > > > > Gianmarco
> >> > > > > > >> > > > >
> >> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> >> Morales
> >> > <
> >> > > > > > >> > > gdfm@apache.org
> >> > > > > > >> > > > >
> >> > > > > > >> > > > > wrote:
> >> > > > > > >> > > > >
> >> > > > > > >> > > > > > Hi,
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > @Martin, thanks for you comments.
> >> > > > > > >> > > > > > Maybe I'm missing some important point, but I
> think
> >> > > > coupling
> >> > > > > > the
> >> > > > > > >> > > > releases
> >> > > > > > >> > > > > > is actually a *good* thing.
> >> > > > > > >> > > > > > To make an example, would it be better if the MR
> >> and
> >> > > HDFS
> >> > > > > > >> > components
> >> > > > > > >> > > of
> >> > > > > > >> > > > > > Hadoop had different release schedules?
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > Actually, keeping the discussion in a single
> place
> >> > would
> >> > > > > make
> >> > > > > > >> > > agreeing
> >> > > > > > >> > > > on
> >> > > > > > >> > > > > > releases (and backwards compatibility) much
> >> easier, as
> >> > > > > > everybody
> >> > > > > > >> > > would
> >> > > > > > >> > > > be
> >> > > > > > >> > > > > > responsible for the whole codebase.
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > That said, I like the idea of absorbing
> samza-core
> >> as
> >> > a
> >> > > > > > >> > sub-project,
> >> > > > > > >> > > > and
> >> > > > > > >> > > > > > leave the fancy stuff separate.
> >> > > > > > >> > > > > > It probably gives 90% of the benefits we have
> been
> >> > > > > discussing
> >> > > > > > >> here.
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > Cheers,
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > --
> >> > > > > > >> > > > > > Gianmarco
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> >> > jay.kreps@gmail.com
> >> > > >
> >> > > > > > wrote:
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > >> Hey Martin,
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> I agree coupling release schedules is a
> downside.
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> Definitely we can try to solve some of the
> >> > integration
> >> > > > > > problems
> >> > > > > > >> in
> >> > > > > > >> > > > > >> Confluent Platform or in other distributions.
> But
> >> I
> >> > > think
> >> > > > > > this
> >> > > > > > >> > ends
> >> > > > > > >> > > up
> >> > > > > > >> > > > > >> being really shallow. I guess I feel to really
> >> get a
> >> > > good
> >> > > > > > user
> >> > > > > > >> > > > > experience
> >> > > > > > >> > > > > >> the two systems have to kind of feel like part
> of
> >> the
> >> > > > same
> >> > > > > > thing
> >> > > > > > >> > and
> >> > > > > > >> > > > you
> >> > > > > > >> > > > > >> can't really add that in later--you can put both
> >> in
> >> > the
> >> > > > > same
> >> > > > > > >> > > > > downloadable
> >> > > > > > >> > > > > >> tar file but it doesn't really give a very
> >> cohesive
> >> > > > > feeling.
> >> > > > > > I
> >> > > > > > >> > agree
> >> > > > > > >> > > > > that
> >> > > > > > >> > > > > >> ultimately any of the project stuff is as much
> >> social
> >> > > and
> >> > > > > > naming
> >> > > > > > >> > as
> >> > > > > > >> > > > > >> anything else--theoretically two totally
> >> independent
> >> > > > > projects
> >> > > > > > >> > could
> >> > > > > > >> > > > work
> >> > > > > > >> > > > > >> to
> >> > > > > > >> > > > > >> tightly align. In practice this seems to be
> quite
> >> > > > difficult
> >> > > > > > >> > though.
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> For the frameworks--totally agree it would be
> >> good to
> >> > > > > > maintain
> >> > > > > > >> the
> >> > > > > > >> > > > > >> framework support with the project. In some
> cases
> >> > there
> >> > > > may
> >> > > > > > not
> >> > > > > > >> be
> >> > > > > > >> > > too
> >> > > > > > >> > > > > >> much
> >> > > > > > >> > > > > >> there since the integration gets lighter but I
> >> think
> >> > > > > whatever
> >> > > > > > >> > stubs
> >> > > > > > >> > > > you
> >> > > > > > >> > > > > >> need should be included. So no I definitely
> wasn't
> >> > > trying
> >> > > > > to
> >> > > > > > >> imply
> >> > > > > > >> > > > > >> dropping
> >> > > > > > >> > > > > >> support for these frameworks, just making the
> >> > > integration
> >> > > > > > >> lighter
> >> > > > > > >> > by
> >> > > > > > >> > > > > >> separating process management from partition
> >> > > management.
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> You raise two good points we would have to
> figure
> >> out
> >> > > if
> >> > > > we
> >> > > > > > went
> >> > > > > > >> > > down
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> alignment path:
> >> > > > > > >> > > > > >> 1. With respect to the name, yeah I think the
> >> first
> >> > > > > question
> >> > > > > > is
> >> > > > > > >> > > > whether
> >> > > > > > >> > > > > >> some "re-branding" would be worth it. If so
> then I
> >> > > think
> >> > > > we
> >> > > > > > can
> >> > > > > > >> > > have a
> >> > > > > > >> > > > > big
> >> > > > > > >> > > > > >> thread on the name. I'm definitely not set on
> >> Kafka
> >> > > > > > Streaming or
> >> > > > > > >> > > Kafka
> >> > > > > > >> > > > > >> Streams I was just using them to be kind of
> >> > > > illustrative. I
> >> > > > > > >> agree
> >> > > > > > >> > > with
> >> > > > > > >> > > > > >> your
> >> > > > > > >> > > > > >> critique of these names, though I think people
> >> would
> >> > > get
> >> > > > > the
> >> > > > > > >> idea.
> >> > > > > > >> > > > > >> 2. Yeah you also raise a good point about how to
> >> > > "factor"
> >> > > > > it.
> >> > > > > > >> Here
> >> > > > > > >> > > are
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> options I see (I could get enthusiastic about
> any
> >> of
> >> > > > them):
> >> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> >> > > > > > >> > > > > >>    b. Two repos, retaining the current
> seperation
> >> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api and
> >> > > > samza-core
> >> > > > > > is
> >> > > > > > >> > > > absorbed
> >> > > > > > >> > > > > >> almost like a third client
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> Cheers,
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> -Jay
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin
> Kleppmann <
> >> > > > > > >> > > > martin@kleppmann.com>
> >> > > > > > >> > > > > >> wrote:
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a few
> >> > > follow-up
> >> > > > > > >> > comments.
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
> >> > becoming
> >> > > a
> >> > > > > > >> > subproject:
> >> > > > > > >> > > > the
> >> > > > > > >> > > > > >> > reasons you mention are good. The risk I see
> is
> >> > that
> >> > > > > > release
> >> > > > > > >> > > > schedules
> >> > > > > > >> > > > > >> > become coupled to each other, which can slow
> >> > everyone
> >> > > > > down,
> >> > > > > > >> and
> >> > > > > > >> > > > large
> >> > > > > > >> > > > > >> > projects with many contributors are harder to
> >> > manage.
> >> > > > > > (Jakob,
> >> > > > > > >> > can
> >> > > > > > >> > > > you
> >> > > > > > >> > > > > >> speak
> >> > > > > > >> > > > > >> > from experience, having seen a wider range of
> >> > Hadoop
> >> > > > > > ecosystem
> >> > > > > > >> > > > > >> projects?)
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > Some of the goals of a better unified
> developer
> >> > > > > experience
> >> > > > > > >> could
> >> > > > > > >> > > > also
> >> > > > > > >> > > > > be
> >> > > > > > >> > > > > >> > solved by integrating Samza nicely into a
> Kafka
> >> > > > > > distribution
> >> > > > > > >> > (such
> >> > > > > > >> > > > as
> >> > > > > > >> > > > > >> > Confluent's). I'm not against merging projects
> >> if
> >> > we
> >> > > > > decide
> >> > > > > > >> > that's
> >> > > > > > >> > > > the
> >> > > > > > >> > > > > >> way
> >> > > > > > >> > > > > >> > to go, just pointing out the same goals can
> >> perhaps
> >> > > > also
> >> > > > > be
> >> > > > > > >> > > achieved
> >> > > > > > >> > > > > in
> >> > > > > > >> > > > > >> > other ways.
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > - With regard to dropping the YARN dependency:
> >> are
> >> > > you
> >> > > > > > >> proposing
> >> > > > > > >> > > > that
> >> > > > > > >> > > > > >> > Samza doesn't give any help to people wanting
> to
> >> > run
> >> > > on
> >> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> >> > > > > > >> > > > > >> > So the docs would basically have a link to
> >> Slider
> >> > and
> >> > > > > > nothing
> >> > > > > > >> > > else?
> >> > > > > > >> > > > Or
> >> > > > > > >> > > > > >> > would we maintain integrations with a bunch of
> >> > > popular
> >> > > > > > >> > deployment
> >> > > > > > >> > > > > >> methods
> >> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts to
> >> make
> >> > > > Samza
> >> > > > > > work
> >> > > > > > >> > with
> >> > > > > > >> > > > > >> Slider)?
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > I absolutely think it's a good idea to have
> the
> >> > "as a
> >> > > > > > library"
> >> > > > > > >> > and
> >> > > > > > >> > > > > "as a
> >> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
> >> people
> >> > who
> >> > > > > want
> >> > > > > > >> them,
> >> > > > > > >> > > > but I
> >> > > > > > >> > > > > >> > think there should also be a low-friction path
> >> for
> >> > > > common
> >> > > > > > "as
> >> > > > > > >> a
> >> > > > > > >> > > > > service"
> >> > > > > > >> > > > > >> > deployment methods, for which we probably need
> >> to
> >> > > > > maintain
> >> > > > > > >> > > > > integrations.
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to
> >> me,
> >> > > > > because
> >> > > > > > >> Kafka
> >> > > > > > >> > > is
> >> > > > > > >> > > > > all
> >> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> >> Transformers"
> >> > > or
> >> > > > > > "Kafka
> >> > > > > > >> > > > Filters"
> >> > > > > > >> > > > > >> > would be more apt?
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
> >> (stream
> >> > > > > > >> transformation
> >> > > > > > >> > > > with
> >> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> >> library"
> >> > > bit)
> >> > > > > > could
> >> > > > > > >> > > become
> >> > > > > > >> > > > > >> part of
> >> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
> >> streaming
> >> > SQL
> >> > > > and
> >> > > > > > >> > > > integrations
> >> > > > > > >> > > > > >> with
> >> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> >> project?
> >> > > In
> >> > > > > > other
> >> > > > > > >> > > words,
> >> > > > > > >> > > > > >> Kafka
> >> > > > > > >> > > > > >> > would absorb the proven, stable core of Samza,
> >> > which
> >> > > > > would
> >> > > > > > >> > become
> >> > > > > > >> > > > the
> >> > > > > > >> > > > > >> > "third Kafka client" mentioned early in this
> >> > thread.
> >> > > > The
> >> > > > > > Samza
> >> > > > > > >> > > > project
> >> > > > > > >> > > > > >> > would then target that third Kafka client as
> its
> >> > base
> >> > > > > API,
> >> > > > > > and
> >> > > > > > >> > the
> >> > > > > > >> > > > > >> project
> >> > > > > > >> > > > > >> > would be freed up to explore more experimental
> >> new
> >> > > > > > horizons.
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > Martin
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> >> > > > jay.kreps@gmail.com>
> >> > > > > > >> wrote:
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > > Hey Martin,
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually
> >> > don't
> >> > > > > think
> >> > > > > > it
> >> > > > > > >> > ties
> >> > > > > > >> > > > our
> >> > > > > > >> > > > > >> > hands
> >> > > > > > >> > > > > >> > > at all, all it does is refactor things. The
> >> > > division
> >> > > > of
> >> > > > > > >> > > > > >> responsibility is
> >> > > > > > >> > > > > >> > > that Samza core is responsible for task
> >> > lifecycle,
> >> > > > > state,
> >> > > > > > >> and
> >> > > > > > >> > > > > >> partition
> >> > > > > > >> > > > > >> > > management (using the Kafka co-ordinator)
> but
> >> it
> >> > is
> >> > > > NOT
> >> > > > > > >> > > > responsible
> >> > > > > > >> > > > > >> for
> >> > > > > > >> > > > > >> > > packaging, configuration deployment or
> >> execution
> >> > of
> >> > > > > > >> processes.
> >> > > > > > >> > > The
> >> > > > > > >> > > > > >> > problem
> >> > > > > > >> > > > > >> > > of packaging and starting these processes is
> >> > > > > > >> > > > > >> > > framework/environment-specific. This leaves
> >> > > > individual
> >> > > > > > >> > > frameworks
> >> > > > > > >> > > > to
> >> > > > > > >> > > > > >> be
> >> > > > > > >> > > > > >> > as
> >> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you can
> get
> >> > > simple
> >> > > > > > >> stateless
> >> > > > > > >> > > > > >> support in
> >> > > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf
> app
> >> > > > > framework
> >> > > > > > >> > > (Slider,
> >> > > > > > >> > > > > >> > Marathon,
> >> > > > > > >> > > > > >> > > etc). These are well known by people and
> have
> >> > nice
> >> > > > UIs
> >> > > > > > and a
> >> > > > > > >> > lot
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> > > flexibility. I don't think they have node
> >> > affinity
> >> > > > as a
> >> > > > > > >> built
> >> > > > > > >> > in
> >> > > > > > >> > > > > >> option
> >> > > > > > >> > > > > >> > > (though I could be wrong). So if we want
> that
> >> we
> >> > > can
> >> > > > > > either
> >> > > > > > >> > wait
> >> > > > > > >> > > > for
> >> > > > > > >> > > > > >> them
> >> > > > > > >> > > > > >> > > to add it or do a custom framework to add
> that
> >> > > > feature
> >> > > > > > (as
> >> > > > > > >> > now).
> >> > > > > > >> > > > > >> > Obviously
> >> > > > > > >> > > > > >> > > if you manage things with old-school ops
> tools
> >> > > > > > >> > (puppet/chef/etc)
> >> > > > > > >> > > > you
> >> > > > > > >> > > > > >> get
> >> > > > > > >> > > > > >> > > locality easily. The nice thing, though, is
> >> that
> >> > > all
> >> > > > > the
> >> > > > > > >> samza
> >> > > > > > >> > > > > >> "business
> >> > > > > > >> > > > > >> > > logic" around partition management and fault
> >> > > > tolerance
> >> > > > > > is in
> >> > > > > > >> > > Samza
> >> > > > > > >> > > > > >> core
> >> > > > > > >> > > > > >> > so
> >> > > > > > >> > > > > >> > > it is shared across frameworks and the
> >> framework
> >> > > > > specific
> >> > > > > > >> bit
> >> > > > > > >> > is
> >> > > > > > >> > > > > just
> >> > > > > > >> > > > > >> > > whether it is smart enough to try to get the
> >> same
> >> > > > host
> >> > > > > > when
> >> > > > > > >> a
> >> > > > > > >> > > job
> >> > > > > > >> > > > is
> >> > > > > > >> > > > > >> > > restarted.
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah I
> >> think
> >> > > the
> >> > > > > > goal
> >> > > > > > >> > would
> >> > > > > > >> > > > be
> >> > > > > > >> > > > > >> (a)
> >> > > > > > >> > > > > >> > > actually get better alignment in user
> >> experience,
> >> > > and
> >> > > > > (b)
> >> > > > > > >> > > express
> >> > > > > > >> > > > > >> this in
> >> > > > > > >> > > > > >> > > the naming and project branding.
> Specifically:
> >> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> >> > > > > > "transformation"
> >> > > > > > >> api
> >> > > > > > >> > > to
> >> > > > > > >> > > > be
> >> > > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be
> >> able
> >> > > to
> >> > > > > > explain
> >> > > > > > >> > > when
> >> > > > > > >> > > > to
> >> > > > > > >> > > > > >> use
> >> > > > > > >> > > > > >> > > the consumer and when to use the stream
> >> > processing
> >> > > > > > >> > functionality
> >> > > > > > >> > > > and
> >> > > > > > >> > > > > >> lead
> >> > > > > > >> > > > > >> > > people into that experience.
> >> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2
> >> (or
> >> > > > > > whatever)
> >> > > > > > >> > that
> >> > > > > > >> > > > has
> >> > > > > > >> > > > > >> both
> >> > > > > > >> > > > > >> > > Kafka and the stream processing part and
> they
> >> > > > actually
> >> > > > > > work
> >> > > > > > >> > > > > together.
> >> > > > > > >> > > > > >> > > 3. Unify the programming experience so the
> >> client
> >> > > and
> >> > > > > > Samza
> >> > > > > > >> > api
> >> > > > > > >> > > > > share
> >> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > I think sub-projects keep separate
> committers
> >> and
> >> > > can
> >> > > > > > have a
> >> > > > > > >> > > > > separate
> >> > > > > > >> > > > > >> > repo,
> >> > > > > > >> > > > > >> > > but I'm actually not really sure (I can't
> >> find a
> >> > > > > > definition
> >> > > > > > >> > of a
> >> > > > > > >> > > > > >> > subproject
> >> > > > > > >> > > > > >> > > in Apache).
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > Basically at a high-level you want the
> >> experience
> >> > > to
> >> > > > > > "feel"
> >> > > > > > >> > > like a
> >> > > > > > >> > > > > >> single
> >> > > > > > >> > > > > >> > > system, not to relatively independent things
> >> that
> >> > > are
> >> > > > > > kind
> >> > > > > > >> of
> >> > > > > > >> > > > > >> awkwardly
> >> > > > > > >> > > > > >> > > glued together.
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > I think if we did that they having naming or
> >> > > branding
> >> > > > > > like
> >> > > > > > >> > > "kafka
> >> > > > > > >> > > > > >> > > streaming" or "kafka streams" or something
> >> like
> >> > > that
> >> > > > > > would
> >> > > > > > >> > > > actually
> >> > > > > > >> > > > > >> do a
> >> > > > > > >> > > > > >> > > good job of conveying what it is. I do that
> >> this
> >> > > > would
> >> > > > > > help
> >> > > > > > >> > > > adoption
> >> > > > > > >> > > > > >> > quite
> >> > > > > > >> > > > > >> > > a lot as it would correctly convey that
> using
> >> > Kafka
> >> > > > > > >> Streaming
> >> > > > > > >> > > with
> >> > > > > > >> > > > > >> Kafka
> >> > > > > > >> > > > > >> > is
> >> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
> >> pretty
> >> > > > > heavily
> >> > > > > > >> > adopted
> >> > > > > > >> > > > at
> >> > > > > > >> > > > > >> this
> >> > > > > > >> > > > > >> > > point.
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> >> originally
> >> > > > when
> >> > > > > > open
> >> > > > > > >> > > > sourcing
> >> > > > > > >> > > > > >> > Samza,
> >> > > > > > >> > > > > >> > > however at that time Kafka was relatively
> >> unknown
> >> > > and
> >> > > > > we
> >> > > > > > >> > decided
> >> > > > > > >> > > > not
> >> > > > > > >> > > > > >> to
> >> > > > > > >> > > > > >> > do
> >> > > > > > >> > > > > >> > > it since we felt it would be limiting. From
> my
> >> > > point
> >> > > > of
> >> > > > > > view
> >> > > > > > >> > the
> >> > > > > > >> > > > > three
> >> > > > > > >> > > > > >> > > things have changed (1) Kafka is now really
> >> > heavily
> >> > > > > used
> >> > > > > > for
> >> > > > > > >> > > > stream
> >> > > > > > >> > > > > >> > > processing, (2) we learned that abstracting
> >> out
> >> > the
> >> > > > > > stream
> >> > > > > > >> > well
> >> > > > > > >> > > is
> >> > > > > > >> > > > > >> > > basically impossible, (3) we learned it is
> >> really
> >> > > > hard
> >> > > > > to
> >> > > > > > >> keep
> >> > > > > > >> > > the
> >> > > > > > >> > > > > two
> >> > > > > > >> > > > > >> > > things feeling like a single product.
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > -Jay
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> >> Kleppmann
> >> > <
> >> > > > > > >> > > > > >> martin@kleppmann.com>
> >> > > > > > >> > > > > >> > > wrote:
> >> > > > > > >> > > > > >> > >
> >> > > > > > >> > > > > >> > >> Hi all,
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> I agree with the general philosophy of
> tying
> >> > Samza
> >> > > > > more
> >> > > > > > >> > firmly
> >> > > > > > >> > > to
> >> > > > > > >> > > > > >> Kafka.
> >> > > > > > >> > > > > >> > >> After I spent a while looking at
> integrating
> >> > other
> >> > > > > > message
> >> > > > > > >> > > > brokers
> >> > > > > > >> > > > > >> (e.g.
> >> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
> >> > > > conclusion
> >> > > > > > that
> >> > > > > > >> > > > > >> > SystemConsumer
> >> > > > > > >> > > > > >> > >> tacitly assumes a model so much like
> Kafka's
> >> > that
> >> > > > > pretty
> >> > > > > > >> much
> >> > > > > > >> > > > > nobody
> >> > > > > > >> > > > > >> but
> >> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
> >> > perhaps
> >> > > an
> >> > > > > > >> > exception,
> >> > > > > > >> > > > but
> >> > > > > > >> > > > > >> it
> >> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.)
> Thus,
> >> > > making
> >> > > > > > Samza
> >> > > > > > >> > > fully
> >> > > > > > >> > > > > >> > dependent
> >> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> >> > system-independence
> >> > > > was
> >> > > > > > >> never
> >> > > > > > >> > as
> >> > > > > > >> > > > > real
> >> > > > > > >> > > > > >> as
> >> > > > > > >> > > > > >> > we
> >> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of
> code
> >> > reuse
> >> > > > are
> >> > > > > > >> real.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN has
> >> also
> >> > > > always
> >> > > > > > been
> >> > > > > > >> > > > > >> appealing to
> >> > > > > > >> > > > > >> > >> me, for various reasons already mentioned
> in
> >> > this
> >> > > > > > thread.
> >> > > > > > >> > > > Although
> >> > > > > > >> > > > > >> > making
> >> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> >> > > > (YARN/Mesos/AWS/etc)
> >> > > > > > >> seems
> >> > > > > > >> > > > > >> laudable,
> >> > > > > > >> > > > > >> > I am
> >> > > > > > >> > > > > >> > >> a little concerned that it will restrict us
> >> to a
> >> > > > > lowest
> >> > > > > > >> > common
> >> > > > > > >> > > > > >> > denominator.
> >> > > > > > >> > > > > >> > >> For example, would host affinity
> (SAMZA-617)
> >> > still
> >> > > > be
> >> > > > > > >> > possible?
> >> > > > > > >> > > > For
> >> > > > > > >> > > > > >> jobs
> >> > > > > > >> > > > > >> > >> with large amounts of state, I think
> >> SAMZA-617
> >> > > would
> >> > > > > be
> >> > > > > > a
> >> > > > > > >> big
> >> > > > > > >> > > > boon,
> >> > > > > > >> > > > > >> > since
> >> > > > > > >> > > > > >> > >> restoring state off the changelog on every
> >> > single
> >> > > > > > restart
> >> > > > > > >> is
> >> > > > > > >> > > > > painful,
> >> > > > > > >> > > > > >> > due
> >> > > > > > >> > > > > >> > >> to long recovery times. It would be a shame
> >> if
> >> > the
> >> > > > > > >> decoupling
> >> > > > > > >> > > > from
> >> > > > > > >> > > > > >> YARN
> >> > > > > > >> > > > > >> > >> made host affinity impossible.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> Jay, a question about the proposed API for
> >> > > > > > instantiating a
> >> > > > > > >> > job
> >> > > > > > >> > > in
> >> > > > > > >> > > > > >> code
> >> > > > > > >> > > > > >> > >> (rather than a properties file): when
> >> > submitting a
> >> > > > job
> >> > > > > > to a
> >> > > > > > >> > > > > cluster,
> >> > > > > > >> > > > > >> is
> >> > > > > > >> > > > > >> > the
> >> > > > > > >> > > > > >> > >> idea that the instantiation code runs on a
> >> > client
> >> > > > > > >> somewhere,
> >> > > > > > >> > > > which
> >> > > > > > >> > > > > >> then
> >> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> >> > > YARN/Mesos/AWS/etc?
> >> > > > > Or
> >> > > > > > >> does
> >> > > > > > >> > > that
> >> > > > > > >> > > > > >> code
> >> > > > > > >> > > > > >> > run
> >> > > > > > >> > > > > >> > >> on each container that is part of the job
> (in
> >> > > which
> >> > > > > > case,
> >> > > > > > >> how
> >> > > > > > >> > > > does
> >> > > > > > >> > > > > >> the
> >> > > > > > >> > > > > >> > job
> >> > > > > > >> > > > > >> > >> submission to the cluster work)?
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
> >> right to
> >> > > > make
> >> > > > > a
> >> > > > > > 1.0
> >> > > > > > >> > > > release
> >> > > > > > >> > > > > >> > with a
> >> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete. So
> if
> >> > this
> >> > > > is
> >> > > > > > going
> >> > > > > > >> > to
> >> > > > > > >> > > > > >> happen, I
> >> > > > > > >> > > > > >> > >> think it would be more honest to stick with
> >> 0.*
> >> > > > > version
> >> > > > > > >> > numbers
> >> > > > > > >> > > > > until
> >> > > > > > >> > > > > >> > the
> >> > > > > > >> > > > > >> > >> library-ified Samza has been implemented,
> is
> >> > > stable
> >> > > > > and
> >> > > > > > >> > widely
> >> > > > > > >> > > > > used.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
> >> Kafka?
> >> > > There
> >> > > > > is
> >> > > > > > >> > > precedent
> >> > > > > > >> > > > > for
> >> > > > > > >> > > > > >> > >> tight coupling between different Apache
> >> projects
> >> > > > (e.g.
> >> > > > > > >> > Curator
> >> > > > > > >> > > > and
> >> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think
> >> > > remaining
> >> > > > > > >> separate
> >> > > > > > >> > > > would
> >> > > > > > >> > > > > >> be
> >> > > > > > >> > > > > >> > ok.
> >> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on Kafka,
> >> there
> >> > > is
> >> > > > > > enough
> >> > > > > > >> > > > > substance
> >> > > > > > >> > > > > >> in
> >> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> >> project.
> >> > > An
> >> > > > > > >> argument
> >> > > > > > >> > in
> >> > > > > > >> > > > > >> favour
> >> > > > > > >> > > > > >> > of
> >> > > > > > >> > > > > >> > >> merging would be if we think Kafka has a
> much
> >> > > > stronger
> >> > > > > > >> "brand
> >> > > > > > >> > > > > >> presence"
> >> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If
> >> the
> >> > > Kafka
> >> > > > > > >> project
> >> > > > > > >> > is
> >> > > > > > >> > > > > >> willing
> >> > > > > > >> > > > > >> > to
> >> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of
> doing
> >> > > > stateful
> >> > > > > > >> stream
> >> > > > > > >> > > > > >> > >> transformations, that would probably have
> >> much
> >> > the
> >> > > > > same
> >> > > > > > >> > effect
> >> > > > > > >> > > as
> >> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> >> Processors"
> >> > or
> >> > > > > > suchlike.
> >> > > > > > >> > > Close
> >> > > > > > >> > > > > >> > >> collaboration between the two projects will
> >> be
> >> > > > needed
> >> > > > > in
> >> > > > > > >> any
> >> > > > > > >> > > > case.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> From a project management perspective, I
> >> guess
> >> > the
> >> > > > > "new
> >> > > > > > >> > Samza"
> >> > > > > > >> > > > > would
> >> > > > > > >> > > > > >> > have
> >> > > > > > >> > > > > >> > >> to be developed on a branch alongside
> ongoing
> >> > > > > > maintenance
> >> > > > > > >> of
> >> > > > > > >> > > the
> >> > > > > > >> > > > > >> current
> >> > > > > > >> > > > > >> > >> line of development? I think it would be
> >> > important
> >> > > > to
> >> > > > > > >> > continue
> >> > > > > > >> > > > > >> > supporting
> >> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> >> migration
> >> > > > path
> >> > > > > to
> >> > > > > > >> the
> >> > > > > > >> > > new
> >> > > > > > >> > > > > >> > version.
> >> > > > > > >> > > > > >> > >> Leaving the current versions unsupported
> and
> >> > > forcing
> >> > > > > > people
> >> > > > > > >> > to
> >> > > > > > >> > > > > >> rewrite
> >> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> Best,
> >> > > > > > >> > > > > >> > >> Martin
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> >> > > > jay@confluent.io>
> >> > > > > > >> wrote:
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >>> Hey Garry,
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be
> happy
> >> to
> >> > > chat
> >> > > > > > more
> >> > > > > > >> > about
> >> > > > > > >> > > > > this
> >> > > > > > >> > > > > >> if
> >> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
> >> > started
> >> > > > with
> >> > > > > > the
> >> > > > > > >> > idea
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> "what
> >> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> >> > ingestion
> >> > > > > tool"
> >> > > > > > but
> >> > > > > > >> > > > > >> ultimately
> >> > > > > > >> > > > > >> > we
> >> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> >> ingestion
> >> > > and
> >> > > > > > >> > > > transformation
> >> > > > > > >> > > > > >> had
> >> > > > > > >> > > > > >> > >>> pretty different needs and coupling the
> two
> >> > made
> >> > > > > things
> >> > > > > > >> > hard.
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>> For what it's worth I think copycat
> (KIP-26)
> >> > > > actually
> >> > > > > > will
> >> > > > > > >> > do
> >> > > > > > >> > > > what
> >> > > > > > >> > > > > >> you
> >> > > > > > >> > > > > >> > >> are
> >> > > > > > >> > > > > >> > >>> looking for.
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>> With regard to your point about slider, I
> >> don't
> >> > > > > > >> necessarily
> >> > > > > > >> > > > > >> disagree.
> >> > > > > > >> > > > > >> > >> But I
> >> > > > > > >> > > > > >> > >>> think getting good YARN support is quite
> >> doable
> >> > > > and I
> >> > > > > > >> think
> >> > > > > > >> > we
> >> > > > > > >> > > > can
> >> > > > > > >> > > > > >> make
> >> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> >> proposal
> >> > > > > solves
> >> > > > > > is
> >> > > > > > >> > that
> >> > > > > > >> > > > > >> > >> technically
> >> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
> >> cluster
> >> > > > > > management
> >> > > > > > >> > > systems
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> > way
> >> > > > > > >> > > > > >> > >>> things are now, you need to write an "app
> >> > master"
> >> > > > or
> >> > > > > > >> > > "framework"
> >> > > > > > >> > > > > for
> >> > > > > > >> > > > > >> > each
> >> > > > > > >> > > > > >> > >>> and they are all a little different so
> >> testing
> >> > is
> >> > > > > > really
> >> > > > > > >> > hard.
> >> > > > > > >> > > > In
> >> > > > > > >> > > > > >> the
> >> > > > > > >> > > > > >> > >>> absence of this we have been stuck with
> just
> >> > YARN
> >> > > > > which
> >> > > > > > >> has
> >> > > > > > >> > > > > >> fantastic
> >> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the
> org,
> >> but
> >> > > > zero
> >> > > > > > >> > > penetration
> >> > > > > > >> > > > > >> > >> elsewhere.
> >> > > > > > >> > > > > >> > >>> Given the huge amount of work being put in
> >> to
> >> > > > slider,
> >> > > > > > >> > > marathon,
> >> > > > > > >> > > > > aws
> >> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen
> related
> >> > > > packaging
> >> > > > > > >> > > > technologies
> >> > > > > > >> > > > > >> > people
> >> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> >> > > > > cloud-specific
> >> > > > > > >> > deploy
> >> > > > > > >> > > > > >> tools,
> >> > > > > > >> > > > > >> > >> etc)
> >> > > > > > >> > > > > >> > >>> I really think it is important to get this
> >> > right.
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>> -Jay
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> >> > Turkington
> >> > > <
> >> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> >> > > > > > >> > > > > >> > >>>
> >> > > > > > >> > > > > >> > >>>> Hi all,
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> I think the question below re does Samza
> >> > become
> >> > > a
> >> > > > > > >> > sub-project
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> Kafka
> >> > > > > > >> > > > > >> > >>>> highlights the broader point around
> >> migration.
> >> > > > Chris
> >> > > > > > >> > mentions
> >> > > > > > >> > > > > >> Samza's
> >> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1 release
> >> but
> >> > I'm
> >> > > > not
> >> > > > > > sure
> >> > > > > > >> > it
> >> > > > > > >> > > > > feels
> >> > > > > > >> > > > > >> > >> right to
> >> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> >> deprecate
> >> > > > most
> >> > > > > of
> >> > > > > > >> it.
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> From a selfish perspective I have some
> guys
> >> > who
> >> > > > have
> >> > > > > > >> > started
> >> > > > > > >> > > > > >> working
> >> > > > > > >> > > > > >> > >> with
> >> > > > > > >> > > > > >> > >>>> Samza and building some new
> >> > consumers/producers
> >> > > > was
> >> > > > > > next
> >> > > > > > >> > up.
> >> > > > > > >> > > > > Sounds
> >> > > > > > >> > > > > >> > like
> >> > > > > > >> > > > > >> > >>>> that is absolutely not the direction to
> >> go. I
> >> > > need
> >> > > > > to
> >> > > > > > >> look
> >> > > > > > >> > > into
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> > KIP
> >> > > > > > >> > > > > >> > >> in
> >> > > > > > >> > > > > >> > >>>> more detail but for me the attractiveness
> >> of
> >> > > > adding
> >> > > > > > new
> >> > > > > > >> > Samza
> >> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all
> they
> >> > were
> >> > > > > doing
> >> > > > > > was
> >> > > > > > >> > > > really
> >> > > > > > >> > > > > >> > getting
> >> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to
> avoid
> >> > > > having
> >> > > > > to
> >> > > > > > >> > worry
> >> > > > > > >> > > > > about
> >> > > > > > >> > > > > >> the
> >> > > > > > >> > > > > >> > >>>> lifecycle management of external clients.
> >> If
> >> > > there
> >> > > > > is
> >> > > > > > a
> >> > > > > > >> > > generic
> >> > > > > > >> > > > > >> Kafka
> >> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a
> new
> >> > > > connector
> >> > > > > > into
> >> > > > > > >> > and
> >> > > > > > >> > > > > have
> >> > > > > > >> > > > > >> a
> >> > > > > > >> > > > > >> > >> lot of
> >> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and
> reliability
> >> > done
> >> > > > for
> >> > > > > me
> >> > > > > > >> then
> >> > > > > > >> > > it
> >> > > > > > >> > > > > >> gives
> >> > > > > > >> > > > > >> > me
> >> > > > > > >> > > > > >> > >> all
> >> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers
> would.
> >> If
> >> > > not
> >> > > > > > then it
> >> > > > > > >> > > > > >> complicates
> >> > > > > > >> > > > > >> > my
> >> > > > > > >> > > > > >> > >>>> operational deployments.
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> Which is similar to my other question
> with
> >> the
> >> > > > > > proposal
> >> > > > > > >> --
> >> > > > > > >> > if
> >> > > > > > >> > > > we
> >> > > > > > >> > > > > >> > build a
> >> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus
> the
> >> > > > requisite
> >> > > > > > >> shims
> >> > > > > > >> > to
> >> > > > > > >> > > > > >> > integrate
> >> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former may
> >> be a
> >> > > lot
> >> > > > > more
> >> > > > > > >> work
> >> > > > > > >> > > > than
> >> > > > > > >> > > > > we
> >> > > > > > >> > > > > >> > >> think.
> >> > > > > > >> > > > > >> > >>>> We may make it much easier for a newcomer
> >> to
> >> > get
> >> > > > > > >> something
> >> > > > > > >> > > > > running
> >> > > > > > >> > > > > >> but
> >> > > > > > >> > > > > >> > >>>> having them step up and get a reliable
> >> > > production
> >> > > > > > >> > deployment
> >> > > > > > >> > > > may
> >> > > > > > >> > > > > >> still
> >> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
> >> > different
> >> > > > > > reasons
> >> > > > > > >> > than
> >> > > > > > >> > > > > >> today.
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable
> with
> >> > > making
> >> > > > > the
> >> > > > > > >> Samza
> >> > > > > > >> > > > > >> dependency
> >> > > > > > >> > > > > >> > >> on
> >> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I absolutely
> >> see
> >> > > the
> >> > > > > > >> benefits
> >> > > > > > >> > > in
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> >> > > > > > >> > > > terminologies/abstractions
> >> > > > > > >> > > > > >> that
> >> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library
> >> would
> >> > > > likely
> >> > > > > > be a
> >> > > > > > >> > very
> >> > > > > > >> > > > > nice
> >> > > > > > >> > > > > >> > tool
> >> > > > > > >> > > > > >> > >> to
> >> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have
> the
> >> > > > concerns
> >> > > > > > >> above
> >> > > > > > >> > re
> >> > > > > > >> > > > the
> >> > > > > > >> > > > > >> > >>>> operational side.
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> Garry
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> -----Original Message-----
> >> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> >> [mailto:
> >> > > > > > >> > gdfm@apache.org
> >> > > > > > >> > > ]
> >> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> >> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> >> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on
> >> > Samza
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> >> > > > > > >> > > > > >> > >>>> From outside, I have always perceived
> Samza
> >> > as a
> >> > > > > > >> computing
> >> > > > > > >> > > > layer
> >> > > > > > >> > > > > >> over
> >> > > > > > >> > > > > >> > >>>> Kafka.
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> The question, maybe a bit provocative, is
> >> > > "should
> >> > > > > > Samza
> >> > > > > > >> be
> >> > > > > > >> > a
> >> > > > > > >> > > > > >> > sub-project
> >> > > > > > >> > > > > >> > >>>> of Kafka then?"
> >> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> >> separate
> >> > > > > project
> >> > > > > > >> > with a
> >> > > > > > >> > > > > >> separate
> >> > > > > > >> > > > > >> > >>>> governance?
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> Cheers,
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> --
> >> > > > > > >> > > > > >> > >>>> Gianmarco
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> >> > > > > > yanfang724@gmail.com>
> >> > > > > > >> > > > wrote:
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka
> more
> >> > > > tightly.
> >> > > > > > >> > Because
> >> > > > > > >> > > > > Samza
> >> > > > > > >> > > > > >> de
> >> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
> >> > leverage
> >> > > > > what
> >> > > > > > >> Kafka
> >> > > > > > >> > > > has.
> >> > > > > > >> > > > > At
> >> > > > > > >> > > > > >> > the
> >> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to
> reinvent
> >> > what
> >> > > > > Samza
> >> > > > > > >> > > already
> >> > > > > > >> > > > > >> has. I
> >> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> >> > ingestion
> >> > > > and
> >> > > > > > >> > > > > transformation.
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
> >> image
> >> > > how
> >> > > > > the
> >> > > > > > >> Samza
> >> > > > > > >> > > > will
> >> > > > > > >> > > > > >> look
> >> > > > > > >> > > > > >> > >>>> like.
> >> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> >> > > difference
> >> > > > > in
> >> > > > > > >> terms
> >> > > > > > >> > > of
> >> > > > > > >> > > > > how
> >> > > > > > >> > > > > >> > >>>>> Samza should look like.
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code
> >> shows
> >> > (A
> >> > > > > > client of
> >> > > > > > >> > > > Kakfa)
> >> > > > > > >> > > > > ?
> >> > > > > > >> > > > > >> And
> >> > > > > > >> > > > > >> > >>>>> user's application code calls this
> client?
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of
> Kafka
> >> > (like
> >> > > > > what
> >> > > > > > the
> >> > > > > > >> > > code
> >> > > > > > >> > > > > >> shows),
> >> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> >> > > > > fault-tolerance?
> >> > > > > > >> Are
> >> > > > > > >> > > they
> >> > > > > > >> > > > > >> taken
> >> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> >> mechanism,
> >> > > such
> >> > > > > as
> >> > > > > > >> > "Samza
> >> > > > > > >> > > > > >> worker"
> >> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> 2. What about other features, such as
> >> > > > auto-scaling,
> >> > > > > > >> shared
> >> > > > > > >> > > > > state,
> >> > > > > > >> > > > > >> > >>>>> monitoring?
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is
> this
> >> > what
> >> > > > > Chris
> >> > > > > > >> > > > suggests?)
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from
> Kakfa
> >> > and
> >> > > > > > produce
> >> > > > > > >> to
> >> > > > > > >> > > it.
> >> > > > > > >> > > > > >> Then it
> >> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks
> like
> >> > now,
> >> > > > > > except it
> >> > > > > > >> > > does
> >> > > > > > >> > > > > not
> >> > > > > > >> > > > > >> > rely
> >> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> >> leverage
> >> > > > Kafka's
> >> > > > > > >> > metrics,
> >> > > > > > >> > > > > logs,
> >> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> Thanks,
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> Fang, Yan
> >> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang
> >> > Wang <
> >> > > > > > >> > > > > wangguoz@gmail.com
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> > >>>> wrote:
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>>> Read through the code example and it
> >> looks
> >> > > good
> >> > > > to
> >> > > > > > me.
> >> > > > > > >> A
> >> > > > > > >> > > few
> >> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> >> runnable
> >> > > like:
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> >> > > --config-factory=...
> >> > > > > > >> > > > > >> > >>>> --config-path=file://...
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for
> deploying
> >> > Samza
> >> > > > > more
> >> > > > > > as
> >> > > > > > >> > > > embedded
> >> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> >> (ignoring
> >> > > the
> >> > > > > > >> > > terminology
> >> > > > > > >> > > > > >> since
> >> > > > > > >> > > > > >> > >>>>>> it is not the
> >> > > > > > >> > > > > >> > >>>>> same
> >> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> >> MyStreamTask(configs);
> >> > > > > Thread
> >> > > > > > >> > thread
> >> > > > > > >> > > =
> >> > > > > > >> > > > > new
> >> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> I think both of these deployment modes
> >> are
> >> > > > > important
> >> > > > > > >> for
> >> > > > > > >> > > > > >> different
> >> > > > > > >> > > > > >> > >>>>>> types
> >> > > > > > >> > > > > >> > >>>>> of
> >> > > > > > >> > > > > >> > >>>>>> users. That said, I think making Samza
> >> > purely
> >> > > > > > >> standalone
> >> > > > > > >> > is
> >> > > > > > >> > > > > still
> >> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or
> library
> >> > > modes.
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> Guozhang
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
> >> Kreps
> >> > <
> >> > > > > > >> > > > jay@confluent.io>
> >> > > > > > >> > > > > >> > wrote:
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> >> example,
> >> > it
> >> > > > was
> >> > > > > > >> > supposed
> >> > > > > > >> > > > to
> >> > > > > > >> > > > > >> look
> >> > > > > > >> > > > > >> > >>>>>>> like
> >> > > > > > >> > > > > >> > >>>>>>> this:
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> >> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> >> > > > "localhost:4242");
> >> > > > > > >> > > > > >> StreamingConfig
> >> > > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> >> > > > "test-topic-2");
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > config.processor(ExampleStreamProcessor.class);
> >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> >> > StringSerializer(),
> >> > > > new
> >> > > > > > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming
> >> > > > container =
> >> > > > > > new
> >> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config);
> container.run();
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>> -Jay
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay
> >> > Kreps <
> >> > > > > > >> > > > jay@confluent.io
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > >> > >>>> wrote:
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
> >> Chris
> >> > > and
> >> > > > I
> >> > > > > > were
> >> > > > > > >> > > having
> >> > > > > > >> > > > > >> > >>>>>>>> around
> >> > > > > > >> > > > > >> > >>>>>>> whether
> >> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a
> >> kind
> >> > > of
> >> > > > > data
> >> > > > > > >> > > > ingestion
> >> > > > > > >> > > > > >> > >>>>> framework
> >> > > > > > >> > > > > >> > >>>>>>> for
> >> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to
> KIP-26
> >> > > > > "copycat").
> >> > > > > > >> This
> >> > > > > > >> > > > kind
> >> > > > > > >> > > > > of
> >> > > > > > >> > > > > >> > >>>>>> combined
> >> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and
> YARN
> >> and
> >> > > the
> >> > > > > > >> > discussion
> >> > > > > > >> > > > > >> around
> >> > > > > > >> > > > > >> > >>>>>>>> how
> >> > > > > > >> > > > > >> > >>>>> to
> >> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was, given
> >> that
> >> > > > Samza
> >> > > > > > was
> >> > > > > > >> > > > basically
> >> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific, what
> if
> >> > you
> >> > > > just
> >> > > > > > >> > embraced
> >> > > > > > >> > > > > that
> >> > > > > > >> > > > > >> > >>>>>>>> and turned it
> >> > > > > > >> > > > > >> > >>>>>> into
> >> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> >> > framework
> >> > > > and
> >> > > > > > more
> >> > > > > > >> > > like a
> >> > > > > > >> > > > > >> > >>>>>>>> third
> >> > > > > > >> > > > > >> > >>>>> Kafka
> >> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing
> consumer"
> >> > with
> >> > > > > state
> >> > > > > > >> > > > management
> >> > > > > > >> > > > > >> > >>>>>> facilities.
> >> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> >> complex
> >> > > > stream
> >> > > > > > >> > > processing
> >> > > > > > >> > > > > >> > >>>>>>>> framework
> >> > > > > > >> > > > > >> > >>>>>>> this
> >> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple
> thing,
> >> not
> >> > > > much
> >> > > > > > more
> >> > > > > > >> > > > > >> complicated
> >> > > > > > >> > > > > >> > >>>>>>>> to
> >> > > > > > >> > > > > >> > >>>>> use
> >> > > > > > >> > > > > >> > >>>>>>> or
> >> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As
> Chris
> >> > said
> >> > > > we
> >> > > > > > >> thought
> >> > > > > > >> > > > about
> >> > > > > > >> > > > > >> it
> >> > > > > > >> > > > > >> > >>>>>>>> a
> >> > > > > > >> > > > > >> > >>>>> lot
> >> > > > > > >> > > > > >> > >>>>>> of
> >> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> >> > processing
> >> > > > > > systems
> >> > > > > > >> > were
> >> > > > > > >> > > > > doing)
> >> > > > > > >> > > > > >> > >>>>> seemed
> >> > > > > > >> > > > > >> > >>>>>>> like
> >> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output
> >> data
> >> > to
> >> > > > and
> >> > > > > > from
> >> > > > > > >> > the
> >> > > > > > >> > > > > stream
> >> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually
> looked
> >> > into
> >> > > > how
> >> > > > > > that
> >> > > > > > >> > > would
> >> > > > > > >> > > > > >> > >>>>>>>> work,
> >> > > > > > >> > > > > >> > >>>>> Samza
> >> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion
> >> > > framework
> >> > > > > > for a
> >> > > > > > >> > > bunch
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> > >>>>> reasons.
> >> > > > > > >> > > > > >> > >>>>>> To
> >> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a
> pretty
> >> > > > different
> >> > > > > > >> > internal
> >> > > > > > >> > > > > data
> >> > > > > > >> > > > > >> > >>>>>>>> model
> >> > > > > > >> > > > > >> > >>>>>> and
> >> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split
> them
> >> and
> >> > > had
> >> > > > > an
> >> > > > > > api
> >> > > > > > >> > for
> >> > > > > > >> > > > > Kafka
> >> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26)
> >> and a
> >> > > > > separate
> >> > > > > > >> api
> >> > > > > > >> > > for
> >> > > > > > >> > > > > >> Kafka
> >> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> This would also allow really
> embracing
> >> the
> >> > > > same
> >> > > > > > >> > > terminology
> >> > > > > > >> > > > > and
> >> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about the
> >> > current
> >> > > > > > state is
> >> > > > > > >> > > that
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> > >>>>>>>> two
> >> > > > > > >> > > > > >> > >>>>>>> systems
> >> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology
> >> like
> >> > > > > "stream"
> >> > > > > > vs
> >> > > > > > >> > > > "topic"
> >> > > > > > >> > > > > >> and
> >> > > > > > >> > > > > >> > >>>>>>> different
> >> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means
> you
> >> > kind
> >> > > > of
> >> > > > > > have
> >> > > > > > >> to
> >> > > > > > >> > > > learn
> >> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> >> > > > > > >> > > > > >> > >>>>>>> way,
> >> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly different
> >> way,
> >> > > > then
> >> > > > > > kind
> >> > > > > > >> of
> >> > > > > > >> > > > > >> > >>>>>>>> understand
> >> > > > > > >> > > > > >> > >>>>> how
> >> > > > > > >> > > > > >> > >>>>>>> they
> >> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having
> walked
> >> a
> >> > few
> >> > > > > > people
> >> > > > > > >> > > through
> >> > > > > > >> > > > > >> this
> >> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to
> >> get.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of
> >> time
> >> > on
> >> > > > > > >> airplanes I
> >> > > > > > >> > > > > hacked
> >> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> >> incomplete
> >> > > > > > prototype
> >> > > > > > >> of
> >> > > > > > >> > > > what
> >> > > > > > >> > > > > >> > >>>>>>>> this would
> >> > > > > > >> > > > > >> > >>>>> look
> >> > > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
> >> dumped
> >> > > into
> >> > > > > > Kafka
> >> > > > > > >> as
> >> > > > > > >> > > it
> >> > > > > > >> > > > > >> > >>>>>>>> required a
> >> > > > > > >> > > > > >> > >>>>>> few
> >> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is
> >> the
> >> > > code:
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > >
> >> > > > > > >> >
> >> > > > > >
> >> > >
> >> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I
> just
> >> > > > > liberally
> >> > > > > > >> > renamed
> >> > > > > > >> > > > > >> > >>>>>>>> everything
> >> > > > > > >> > > > > >> > >>>>> to
> >> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
> >> regard
> >> > > for
> >> > > > > > >> > > > compatibility.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> To use this would be something like
> >> this:
> >> > > > > > >> > > > > >> > >>>>>>>> Properties props = new Properties();
> >> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> >> > > > > "localhost:4242");
> >> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> >> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> >> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> >> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> >> > > > > > >> > > > > >> config.processor(ExampleStreamProcessor.class);
> >> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> >> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> >> > > StringDeserializer());
> >> > > > > > >> > > > KafkaStreaming
> >> > > > > > >> > > > > >> > >>>>>> container =
> >> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> >> > container.run();
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> >> > > > SamzaContainer;
> >> > > > > > >> > > > > StreamProcessor
> >> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the class
> >> names
> >> > > in
> >> > > > a
> >> > > > > > file
> >> > > > > > >> > and
> >> > > > > > >> > > > then
> >> > > > > > >> > > > > >> > >>>>>>>> having
> >> > > > > > >> > > > > >> > >>>>>> the
> >> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
> >> > > > > instantiate
> >> > > > > > the
> >> > > > > > >> > > > > container
> >> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced
> over
> >> > > > however
> >> > > > > > many
> >> > > > > > >> > > > > instances
> >> > > > > > >> > > > > >> > >>>>>>>> of
> >> > > > > > >> > > > > >> > >>>>> this
> >> > > > > > >> > > > > >> > >>>>>>> are
> >> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an
> instance
> >> > dies,
> >> > > > new
> >> > > > > > >> tasks
> >> > > > > > >> > > are
> >> > > > > > >> > > > > >> added
> >> > > > > > >> > > > > >> > >>>>>>>> to
> >> > > > > > >> > > > > >> > >>>>> the
> >> > > > > > >> > > > > >> > >>>>>>>> existing containers without shutting
> >> them
> >> > > > down).
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for
> running
> >> > this
> >> > > > > stuff
> >> > > > > > in
> >> > > > > > >> > YARN
> >> > > > > > >> > > > via
> >> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
> >> using
> >> > > some
> >> > > > > of
> >> > > > > > >> their
> >> > > > > > >> > > > tools
> >> > > > > > >> > > > > >> > >>>>>>>> but from the
> >> > > > > > >> > > > > >> > >>>>>> point
> >> > > > > > >> > > > > >> > >>>>>>> of
> >> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these stream
> >> > > > processing
> >> > > > > > jobs
> >> > > > > > >> > are
> >> > > > > > >> > > > > just
> >> > > > > > >> > > > > >> > >>>>>> stateless
> >> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
> >> expand
> >> > and
> >> > > > > > contract
> >> > > > > > >> > at
> >> > > > > > >> > > > > will.
> >> > > > > > >> > > > > >> > >>>>>>>> There
> >> > > > > > >> > > > > >> > >>>>> is
> >> > > > > > >> > > > > >> > >>>>>>> no
> >> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code,
> it
> >> > would
> >> > > > get
> >> > > > > > >> larger
> >> > > > > > >> > > if
> >> > > > > > >> > > > we
> >> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly
> larger.
> >> We
> >> > > > really
> >> > > > > > do
> >> > > > > > >> > get a
> >> > > > > > >> > > > ton
> >> > > > > > >> > > > > >> > >>>>>>>> of
> >> > > > > > >> > > > > >> > >>>>>>> leverage
> >> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> >> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> >> > delegated
> >> > > to
> >> > > > > the
> >> > > > > > >> new
> >> > > > > > >> > > > > >> consumer.
> >> > > > > > >> > > > > >> > >>>>> This
> >> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> >> > management
> >> > > > > > strategy
> >> > > > > > >> > > > > available
> >> > > > > > >> > > > > >> > >>>>>>>> to
> >> > > > > > >> > > > > >> > >>>>>> Kafka
> >> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to Samza
> >> (and
> >> > > vice
> >> > > > > > versa)
> >> > > > > > >> > and
> >> > > > > > >> > > > > with
> >> > > > > > >> > > > > >> > >>>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>>> exact
> >> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> >> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as
> state
> >> > reuse
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
> >> > thought
> >> > > > > > >> provoking.
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> -Jay
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM,
> Chris
> >> > > > > Riccomini <
> >> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> >> > > > > > >> > > > > >> > >>>>>>>> wrote:
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with
> Samza
> >> > > > > engineers
> >> > > > > > at
> >> > > > > > >> > > > LinkedIn
> >> > > > > > >> > > > > >> > >>>>>>>>> and
> >> > > > > > >> > > > > >> > >>>>>>> Confluent
> >> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few
> observations
> >> > and
> >> > > > > would
> >> > > > > > >> like
> >> > > > > > >> > to
> >> > > > > > >> > > > > >> > >>>>>>>>> propose
> >> > > > > > >> > > > > >> > >>>>> some
> >> > > > > > >> > > > > >> > >>>>>>>>> changes.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I
> >> want to
> >> > > > call
> >> > > > > > out
> >> > > > > > >> > about
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> >> > > > > > >> > > > > >> > >>>>>> design,
> >> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some
> changes.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic
> >> > > > deployment
> >> > > > > > >> system.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
> >> SystemConsumer/SystemProducer
> >> > and
> >> > > > > > Kafka's
> >> > > > > > >> > > > consumer
> >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> >> > > > > > >> > > > > >> > >>>>> are
> >> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
> >> > problems.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are
> related,
> >> > but
> >> > > > I'll
> >> > > > > > >> > address
> >> > > > > > >> > > > them
> >> > > > > > >> > > > > >> in
> >> > > > > > >> > > > > >> > >>>>> order.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use
> of a
> >> > > > dynamic
> >> > > > > > >> > > deployment
> >> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> >> > > > > > >> > > > > >> > >>>>>> such
> >> > > > > > >> > > > > >> > >>>>>>>>> as
> >> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially
> >> built
> >> > > > > Samza,
> >> > > > > > we
> >> > > > > > >> > bet
> >> > > > > > >> > > > that
> >> > > > > > >> > > > > >> > >>>>>>>>> there
> >> > > > > > >> > > > > >> > >>>>>> would
> >> > > > > > >> > > > > >> > >>>>>>>>> be
> >> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area, and
> >> we
> >> > > could
> >> > > > > > >> support
> >> > > > > > >> > > > them,
> >> > > > > > >> > > > > >> and
> >> > > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>> rest
> >> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there are
> >> many
> >> > > > > > >> variations.
> >> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> >> > > > > > >> > > > > >> > >>>>>> many
> >> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start
> >> their
> >> > > > > > processors
> >> > > > > > >> > like
> >> > > > > > >> > > > > normal
> >> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use traditional
> >> > > > deployment
> >> > > > > > >> scripts
> >> > > > > > >> > > > such
> >> > > > > > >> > > > > as
> >> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> >> > > > > > >> > > > > >> > >>>>>> Chef,
> >> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
> >> system
> >> > > on
> >> > > > > > users
> >> > > > > > >> > makes
> >> > > > > > >> > > > the
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really
> painful
> >> for
> >> > > > first
> >> > > > > > time
> >> > > > > > >> > > > users.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement
> >> was
> >> > > also
> >> > > > a
> >> > > > > > bit
> >> > > > > > >> of
> >> > > > > > >> > a
> >> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> >> > > > > > >> > > > > >> > >>>>>> because
> >> > > > > > >> > > > > >> > >>>>>>>>> of
> >> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding
> between
> >> > the
> >> > > > > > nature of
> >> > > > > > >> > > batch
> >> > > > > > >> > > > > >> jobs
> >> > > > > > >> > > > > >> > >>>>>>>>> and
> >> > > > > > >> > > > > >> > >>>>>>> stream
> >> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> >> > > conscious
> >> > > > > > effort
> >> > > > > > >> to
> >> > > > > > >> > > > favor
> >> > > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>> Hadoop
> >> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things,
> >> since
> >> > it
> >> > > > > worked
> >> > > > > > >> and
> >> > > > > > >> > > was
> >> > > > > > >> > > > > well
> >> > > > > > >> > > > > >> > >>>>>>> understood.
> >> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that
> >> batch
> >> > > jobs
> >> > > > > > have a
> >> > > > > > >> > > > definite
> >> > > > > > >> > > > > >> > >>>>>> beginning,
> >> > > > > > >> > > > > >> > >>>>>>>>> and
> >> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs
> don't
> >> > > > > (usually).
> >> > > > > > >> This
> >> > > > > > >> > > > leads
> >> > > > > > >> > > > > to
> >> > > > > > >> > > > > >> > >>>>>>>>> a
> >> > > > > > >> > > > > >> > >>>>> much
> >> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for
> stream
> >> > > > > processors.
> >> > > > > > >> You
> >> > > > > > >> > > > > >> basically
> >> > > > > > >> > > > > >> > >>>>>>>>> just
> >> > > > > > >> > > > > >> > >>>>>>> need
> >> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> >> processor,
> >> > and
> >> > > > > start
> >> > > > > > >> it.
> >> > > > > > >> > > The
> >> > > > > > >> > > > > way
> >> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's
> no
> >> > > concept
> >> > > > > of
> >> > > > > > a
> >> > > > > > >> > > cluster
> >> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> >> > > > > > >> > > > > >> > >>>>>> add
> >> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> >> coupling
> >> > > > Samza
> >> > > > > > with
> >> > > > > > >> a
> >> > > > > > >> > > > > >> scheduler
> >> > > > > > >> > > > > >> > >>>>>>>>> is
> >> > > > > > >> > > > > >> > >>>>>> that
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
> >> handle
> >> > > > > > deployment.
> >> > > > > > >> > > This
> >> > > > > > >> > > > > >> pulls
> >> > > > > > >> > > > > >> > >>>>>>>>> in a
> >> > > > > > >> > > > > >> > >>>>>>> bunch
> >> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> >> > > distribution
> >> > > > > > (config
> >> > > > > > >> > > > > stream),
> >> > > > > > >> > > > > >> > >>>>>>>>> shell
> >> > > > > > >> > > > > >> > >>>>>>> scrips
> >> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner),
> packaging
> >> > (all
> >> > > > the
> >> > > > > > .tgz
> >> > > > > > >> > > > stuff),
> >> > > > > > >> > > > > >> etc.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
> >> > > > deployment
> >> > > > > > was
> >> > > > > > >> to
> >> > > > > > >> > > > > support
> >> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> >> > > locality,
> >> > > > > you
> >> > > > > > >> need
> >> > > > > > >> > to
> >> > > > > > >> > > > put
> >> > > > > > >> > > > > >> > >>>>>>>>> your
> >> > > > > > >> > > > > >> > >>>>>> processors
> >> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're
> processing.
> >> > Upon
> >> > > > > > further
> >> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> >> > > > > > >> > > > > >> > >>>>>>> though,
> >> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that beneficial.
> >> > There
> >> > > is
> >> > > > > > some
> >> > > > > > >> > good
> >> > > > > > >> > > > > >> > >>>>>>>>> discussion
> >> > > > > > >> > > > > >> > >>>>>> about
> >> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335.
> >> > Again,
> >> > > we
> >> > > > > > took
> >> > > > > > >> the
> >> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> >> > > > > > >> > > > > >> > >>>>>> path,
> >> > > > > > >> > > > > >> > >>>>>>>>> but
> >> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental
> differences
> >> > > > between
> >> > > > > > HDFS
> >> > > > > > >> > and
> >> > > > > > >> > > > > Kafka.
> >> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> >> > > > > > >> > > > > >> > >>>>>> has
> >> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions.
> >> This
> >> > > > leads
> >> > > > > to
> >> > > > > > >> less
> >> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
> >> > > processors
> >> > > > > on
> >> > > > > > top
> >> > > > > > >> > of
> >> > > > > > >> > > > > Kafka.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a
> crutch.
> >> > > Samza
> >> > > > > > doesn't
> >> > > > > > >> > > have
> >> > > > > > >> > > > > any
> >> > > > > > >> > > > > >> > >>>>>>>>> built
> >> > > > > > >> > > > > >> > >>>>> in
> >> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
> >> > depends
> >> > > on
> >> > > > > the
> >> > > > > > >> > > dynamic
> >> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to
> handle
> >> > > > restarts
> >> > > > > > >> when a
> >> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> >> > > > > > >> > > > > >> > >>>>>>> made
> >> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> >> standalone
> >> > > Samza
> >> > > > > > >> > container
> >> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is good,
> >> but I
> >> > > > think
> >> > > > > > that
> >> > > > > > >> > > we've
> >> > > > > > >> > > > > >> gone
> >> > > > > > >> > > > > >> > >>>>>>>>> too
> >> > > > > > >> > > > > >> > >>>>>> far
> >> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> >> > > > (SystemConsumer,
> >> > > > > > >> > > > > SystemProducer,
> >> > > > > > >> > > > > >> > >>>> etc).
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just
> about
> >> > every
> >> > > > > > >> component
> >> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> >> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> >> > > ConfigRewriter,
> >> > > > > > etc).
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> >> > forgotten,
> >> > > as
> >> > > > > > well.
> >> > > > > > >> > Some
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> > >>>>>>>>> these
> >> > > > > > >> > > > > >> > >>>>> are
> >> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not to
> >> be.
> >> > > This
> >> > > > > all
> >> > > > > > >> comes
> >> > > > > > >> > > at
> >> > > > > > >> > > > a
> >> > > > > > >> > > > > >> cost:
> >> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is
> making
> >> it
> >> > > > harder
> >> > > > > > for
> >> > > > > > >> > our
> >> > > > > > >> > > > > users
> >> > > > > > >> > > > > >> > >>>>>>>>> to
> >> > > > > > >> > > > > >> > >>>>> pick
> >> > > > > > >> > > > > >> > >>>>>> up
> >> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It
> also
> >> > makes
> >> > > > it
> >> > > > > > >> > difficult
> >> > > > > > >> > > > for
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about
> what
> >> the
> >> > > > > > >> > > characteristics
> >> > > > > > >> > > > of
> >> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> >> characteristics
> >> > > > change
> >> > > > > > >> > > depending
> >> > > > > > >> > > > on
> >> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are
> most
> >> > > visible
> >> > > > > in
> >> > > > > > the
> >> > > > > > >> > > > System
> >> > > > > > >> > > > > >> APIs.
> >> > > > > > >> > > > > >> > >>>>> What
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
> >> functional is
> >> > > > Kafka
> >> > > > > > as
> >> > > > > > >> its
> >> > > > > > >> > > > > >> > >>>>>>>>> transport
> >> > > > > > >> > > > > >> > >>>>>> layer.
> >> > > > > > >> > > > > >> > >>>>>>>>> But
> >> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use
> >> cases
> >> > > into
> >> > > > > one
> >> > > > > > >> API:
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> >> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports both
> >> of
> >> > > these
> >> > > > > use
> >> > > > > > >> > cases.
> >> > > > > > >> > > > The
> >> > > > > > >> > > > > >> > >>>>>>>>> problem
> >> > > > > > >> > > > > >> > >>>>>> is,
> >> > > > > > >> > > > > >> > >>>>>>>>> we
> >> > > > > > >> > > > > >> > >>>>>>>>> actually want different features for
> >> each
> >> > > use
> >> > > > > > case.
> >> > > > > > >> By
> >> > > > > > >> > > > > >> papering
> >> > > > > > >> > > > > >> > >>>>>>>>> over
> >> > > > > > >> > > > > >> > >>>>>>> these
> >> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a
> single
> >> > API,
> >> > > > > we've
> >> > > > > > >> > > > introduced
> >> > > > > > >> > > > > a
> >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> >> > > > > > >> > > > > >> > >>>>>>> leaky
> >> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like
> in
> >> (2)
> >> > > is
> >> > > > to
> >> > > > > > have
> >> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
> >> > offsets
> >> > > > > (like
> >> > > > > > >> > Kafka).
> >> > > > > > >> > > > > This
> >> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> >> > > > > > >> > > > > >> > >>>>> with
> >> > > > > > >> > > > > >> > >>>>>>> (1),
> >> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems have
> >> > > > different
> >> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> >> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> >> mailing
> >> > > list
> >> > > > > and
> >> > > > > > >> the
> >> > > > > > >> > > SQL
> >> > > > > > >> > > > > >> JIRAs
> >> > > > > > >> > > > > >> > >>>>> about
> >> > > > > > >> > > > > >> > >>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> >> > > replayability.
> >> > > > > > Kafka
> >> > > > > > >> > > allows
> >> > > > > > >> > > > us
> >> > > > > > >> > > > > >> to
> >> > > > > > >> > > > > >> > >>>>> rewind
> >> > > > > > >> > > > > >> > >>>>>>>>> when
> >> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other
> systems
> >> > > don't.
> >> > > > In
> >> > > > > > some
> >> > > > > > >> > > > cases,
> >> > > > > > >> > > > > >> > >>>>>>>>> systems
> >> > > > > > >> > > > > >> > >>>>>>> return
> >> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> >> > > > > > >> WikipediaSystemConsumer)
> >> > > > > > >> > > > > because
> >> > > > > > >> > > > > >> > >>>>>>>>> they
> >> > > > > > >> > > > > >> > >>>>>> have
> >> > > > > > >> > > > > >> > >>>>>>> no
> >> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example.
> Kafka
> >> > > > supports
> >> > > > > > >> > > > > partitioning,
> >> > > > > > >> > > > > >> > >>>>>>>>> but
> >> > > > > > >> > > > > >> > >>>>> many
> >> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
> >> having a
> >> > > > single
> >> > > > > > >> > > partition
> >> > > > > > >> > > > > for
> >> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other systems
> >> model
> >> > > > > > >> partitioning
> >> > > > > > >> > > > > >> > >>>> differently (e.g.
> >> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a
> >> mess.
> >> > > > > > Creating
> >> > > > > > >> > > streams
> >> > > > > > >> > > > > in
> >> > > > > > >> > > > > >> a
> >> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> >> impossible.
> >> > > As
> >> > > > is
> >> > > > > > >> > modeling
> >> > > > > > >> > > > > >> > >>>>>>>>> metadata
> >> > > > > > >> > > > > >> > >>>>> for
> >> > > > > > >> > > > > >> > >>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> >> partitions,
> >> > > > > location,
> >> > > > > > >> > etc).
> >> > > > > > >> > > > The
> >> > > > > > >> > > > > >> > >>>>>>>>> list
> >> > > > > > >> > > > > >> > >>>>> goes
> >> > > > > > >> > > > > >> > >>>>>>> on.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing
> >> Samza,
> >> > > > > Kafka's
> >> > > > > > >> > > consumer
> >> > > > > > >> > > > > and
> >> > > > > > >> > > > > >> > >>>>> producer
> >> > > > > > >> > > > > >> > >>>>>>>>> APIs
> >> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set.
> On
> >> the
> >> > > > > > >> > consumer-side,
> >> > > > > > >> > > > you
> >> > > > > > >> > > > > >> > >>>>>>>>> had two
> >> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level
> consumer,
> >> or
> >> > > the
> >> > > > > > simple
> >> > > > > > >> > > > > consumer.
> >> > > > > > >> > > > > >> > >>>>>>>>> The
> >> > > > > > >> > > > > >> > >>>>>>> problem
> >> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was
> that
> >> it
> >> > > > > > controlled
> >> > > > > > >> > your
> >> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and
> >> the
> >> > > order
> >> > > > > in
> >> > > > > > >> which
> >> > > > > > >> > > you
> >> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> >> > > > > > >> > > > > >> > >>>>> problem
> >> > > > > > >> > > > > >> > >>>>>>>>> with
> >> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not
> >> > > simple.
> >> > > > > It's
> >> > > > > > >> > basic.
> >> > > > > > >> > > > You
> >> > > > > > >> > > > > >> > >>>>>>>>> end up
> >> > > > > > >> > > > > >> > >>>>>>> having
> >> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level
> >> stuff
> >> > > > that
> >> > > > > > you
> >> > > > > > >> > > > > shouldn't.
> >> > > > > > >> > > > > >> > >>>>>>>>> We
> >> > > > > > >> > > > > >> > >>>>>> spent a
> >> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> >> > > > KafkaSystemConsumer
> >> > > > > > very
> >> > > > > > >> > > > robust.
> >> > > > > > >> > > > > >> It
> >> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some cool
> >> > > features:
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
> >> > > > > > prioritization.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> >> assignment
> >> > > to
> >> > > > > > support
> >> > > > > > >> > > > joins,
> >> > > > > > >> > > > > >> > >>>>>>>>> global
> >> > > > > > >> > > > > >> > >>>>>> state
> >> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)),
> etc.
> >> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> >> > checkpointing.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time
> is
> >> > that
> >> > > > > these
> >> > > > > > >> > > features
> >> > > > > > >> > > > > >> > >>>>>>>>> should
> >> > > > > > >> > > > > >> > >>>>>>> actually
> >> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka
> consumers
> >> > (not
> >> > > > just
> >> > > > > > >> Samza
> >> > > > > > >> > > > stream
> >> > > > > > >> > > > > >> > >>>>>> processors)
> >> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like
> joins
> >> > and
> >> > > > > > partition
> >> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> >> > > > > > >> > > > > >> > >>>>>>> Kafka
> >> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> >> > conclusion.
> >> > > > > > They're
> >> > > > > > >> > > adding
> >> > > > > > >> > > > a
> >> > > > > > >> > > > > >> ton
> >> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> >> consumer
> >> > > > > > >> > > implementation.
> >> > > > > > >> > > > > To a
> >> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> >> > > > > > >> > > > > >> > >>>>> it's
> >> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've already
> >> done
> >> > > in
> >> > > > > > Samza.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up
> taking
> >> a
> >> > > very
> >> > > > > > similar
> >> > > > > > >> > > > > approach
> >> > > > > > >> > > > > >> > >>>>>>>>> to
> >> > > > > > >> > > > > >> > >>>>>> Samza's
> >> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager
> implementation
> >> for
> >> > > > > > handling
> >> > > > > > >> > > offset
> >> > > > > > >> > > > > >> > >>>>>> checkpointing.
> >> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> >> management
> >> > > > > feature
> >> > > > > > >> > stores
> >> > > > > > >> > > > > >> offset
> >> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows
> >> you to
> >> > > > fetch
> >> > > > > > them
> >> > > > > > >> > > from
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> > >>>>>>>>> broker.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste,
> >> since
> >> > we
> >> > > > > could
> >> > > > > > >> have
> >> > > > > > >> > > > shared
> >> > > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > > >> > > > > >> > >>>>> work
> >> > > > > > >> > > > > >> > >>>>>> if
> >> > > > > > >> > > > > >> > >>>>>>>>> it
> >> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
> >> get-go.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Vision
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> >> radical
> >> > > > > > proposal.
> >> > > > > > >> > Samza
> >> > > > > > >> > > > is
> >> > > > > > >> > > > > >> > >>>>> relatively
> >> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to
> >> say
> >> > > that
> >> > > > > > we're
> >> > > > > > >> > > near a
> >> > > > > > >> > > > > 1.0
> >> > > > > > >> > > > > >> > >>>>>> release.
> >> > > > > > >> > > > > >> > >>>>>>>>> I'd
> >> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what
> >> we've
> >> > > > > learned,
> >> > > > > > and
> >> > > > > > >> > > begin
> >> > > > > > >> > > > > >> > >>>>>>>>> thinking
> >> > > > > > >> > > > > >> > >>>>>>> about
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we
> >> change if
> >> > > we
> >> > > > > were
> >> > > > > > >> > > starting
> >> > > > > > >> > > > > >> from
> >> > > > > > >> > > > > >> > >>>>>> scratch?
> >> > > > > > >> > > > > >> > >>>>>>>>> My
> >> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only*
> >> way
> >> > to
> >> > > > run
> >> > > > > > Samza
> >> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
> >> > > > > dependences
> >> > > > > > on
> >> > > > > > >> > > YARN,
> >> > > > > > >> > > > > >> Mesos,
> >> > > > > > >> > > > > >> > >>>> etc.
> >> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support
> >> only
> >> > > > Kafka
> >> > > > > > as
> >> > > > > > >> the
> >> > > > > > >> > > > > stream
> >> > > > > > >> > > > > >> > >>>>>> processing
> >> > > > > > >> > > > > >> > >>>>>>>>> layer.
> >> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics,
> logging,
> >> > > > > > >> serialization,
> >> > > > > > >> > > and
> >> > > > > > >> > > > > >> > >>>>>>>>> config
> >> > > > > > >> > > > > >> > >>>>>>> systems,
> >> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues
> that
> >> I
> >> > > > > outlined
> >> > > > > > >> > above.
> >> > > > > > >> > > It
> >> > > > > > >> > > > > >> > >>>>>>>>> should
> >> > > > > > >> > > > > >> > >>>>> also
> >> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> >> > > > dramatically.
> >> > > > > > >> > > Supporting
> >> > > > > > >> > > > > >> only
> >> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow
> >> Samza
> >> > to
> >> > > be
> >> > > > > > >> executed
> >> > > > > > >> > > on
> >> > > > > > >> > > > > YARN
> >> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> >> > > > Marathon/Aurora),
> >> > > > > or
> >> > > > > > >> most
> >> > > > > > >> > > > other
> >> > > > > > >> > > > > >> > >>>>>>>>> in-house
> >> > > > > > >> > > > > >> > >>>>>>> deployment
> >> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a lot
> >> > easier
> >> > > > for
> >> > > > > > new
> >> > > > > > >> > > users.
> >> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> >> > > > > > >> > > > > >> > >>>>>>> having
> >> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without
> YARN.
> >> > The
> >> > > > drop
> >> > > > > > in
> >> > > > > > >> > > mailing
> >> > > > > > >> > > > > >> list
> >> > > > > > >> > > > > >> > >>>>>> traffic
> >> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
> >> overdue to
> >> > > me.
> >> > > > > The
> >> > > > > > >> > > reality
> >> > > > > > >> > > > > is,
> >> > > > > > >> > > > > >> > >>>>> everyone
> >> > > > > > >> > > > > >> > >>>>>>>>> that
> >> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
> >> Kafka.
> >> > We
> >> > > > > > basically
> >> > > > > > >> > > > require
> >> > > > > > >> > > > > >> it
> >> > > > > > >> > > > > >> > >>>>>> already
> >> > > > > > >> > > > > >> > >>>>>>> in
> >> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work.
> Those
> >> > that
> >> > > > are
> >> > > > > > >> using
> >> > > > > > >> > > > other
> >> > > > > > >> > > > > >> > >>>>>>>>> systems
> >> > > > > > >> > > > > >> > >>>>>> are
> >> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into
> >> Kafka
> >> > > (1),
> >> > > > > and
> >> > > > > > >> then
> >> > > > > > >> > > > they
> >> > > > > > >> > > > > do
> >> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
> >> already
> >> > > > > > discussion (
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > >
> >> > > > > > >> >
> >> > > > > >
> >> > >
> >> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >> > > > > > >> > > > > >> > >>>>> 767
> >> > > > > > >> > > > > >> > >>>>>>>>> )
> >> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into
> Kafka
> >> > > > extremely
> >> > > > > > >> easy.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple with
> >> > Kafka,
> >> > > > we
> >> > > > > > can
> >> > > > > > >> > > > leverage
> >> > > > > > >> > > > > a
> >> > > > > > >> > > > > >> > >>>>>>>>> ton of
> >> > > > > > >> > > > > >> > >>>>>>> their
> >> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> >> maintain
> >> > > our
> >> > > > > own
> >> > > > > > >> > config,
> >> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> >> > > > > > >> > > > > >> > >>>>> etc.
> >> > > > > > >> > > > > >> > >>>>>>> We
> >> > > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries,
> and
> >> > make
> >> > > > them
> >> > > > > > >> > better.
> >> > > > > > >> > > > This
> >> > > > > > >> > > > > >> > >>>>>>>>> will
> >> > > > > > >> > > > > >> > >>>>> also
> >> > > > > > >> > > > > >> > >>>>>>>>> allow us to share the
> >> consumer/producer
> >> > > APIs,
> >> > > > > and
> >> > > > > > >> will
> >> > > > > > >> > > let
> >> > > > > > >> > > > > us
> >> > > > > > >> > > > > >> > >>>>> leverage
> >> > > > > > >> > > > > >> > >>>>>>>>> their offset management and
> partition
> >> > > > > management,
> >> > > > > > >> > rather
> >> > > > > > >> > > > > than
> >> > > > > > >> > > > > >> > >>>>>>>>> having
> >> > > > > > >> > > > > >> > >>>>>> our
> >> > > > > > >> > > > > >> > >>>>>>>>> own. All of the coordinator stream
> >> code
> >> > > would
> >> > > > > go
> >> > > > > > >> away,
> >> > > > > > >> > > as
> >> > > > > > >> > > > > >> would
> >> > > > > > >> > > > > >> > >>>>>>>>> most
> >> > > > > > >> > > > > >> > >>>>>> of
> >> > > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably
> >> have
> >> > to
> >> > > > push
> >> > > > > > some
> >> > > > > > >> > > > > partition
> >> > > > > > >> > > > > >> > >>>>>>> management
> >> > > > > > >> > > > > >> > >>>>>>>>> features into the Kafka broker, but
> >> > they're
> >> > > > > > already
> >> > > > > > >> > > moving
> >> > > > > > >> > > > > in
> >> > > > > > >> > > > > >> > >>>>>>>>> that direction with the new consumer
> >> API.
> >> > > The
> >> > > > > > >> features
> >> > > > > > >> > > we
> >> > > > > > >> > > > > have
> >> > > > > > >> > > > > >> > >>>>>>>>> for
> >> > > > > > >> > > > > >> > >>>>>> partition
> >> > > > > > >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza,
> and
> >> > seem
> >> > > > > like
> >> > > > > > >> they
> >> > > > > > >> > > > should
> >> > > > > > >> > > > > >> be
> >> > > > > > >> > > > > >> > >>>>>>>>> in
> >> > > > > > >> > > > > >> > >>>>>> Kafka
> >> > > > > > >> > > > > >> > >>>>>>>>> anyway. There will always be some
> >> niche
> >> > > > usages
> >> > > > > > which
> >> > > > > > >> > > will
> >> > > > > > >> > > > > >> > >>>>>>>>> require
> >> > > > > > >> > > > > >> > >>>>>> extra
> >> > > > > > >> > > > > >> > >>>>>>>>> care and hence full control over
> >> > partition
> >> > > > > > >> assignments
> >> > > > > > >> > > > much
> >> > > > > > >> > > > > >> > >>>>>>>>> like the
> >> > > > > > >> > > > > >> > >>>>>>> Kafka
> >> > > > > > >> > > > > >> > >>>>>>>>> low level consumer api. These would
> >> > > continue
> >> > > > to
> >> > > > > > be
> >> > > > > > >> > > > > supported.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> These items will be good for the
> Samza
> >> > > > > community.
> >> > > > > > >> > > They'll
> >> > > > > > >> > > > > make
> >> > > > > > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it
> >> easier
> >> > for
> >> > > > > > >> developers
> >> > > > > > >> > > to
> >> > > > > > >> > > > > add
> >> > > > > > >> > > > > >> > >>>>>>>>> new features.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large
> (and
> >> > > > somewhat
> >> > > > > > >> > backwards
> >> > > > > > >> > > > > >> > >>>>> incompatible
> >> > > > > > >> > > > > >> > >>>>>>>>> change). If we choose to go this
> >> route,
> >> > > it's
> >> > > > > > >> important
> >> > > > > > >> > > > that
> >> > > > > > >> > > > > we
> >> > > > > > >> > > > > >> > >>>>> openly
> >> > > > > > >> > > > > >> > >>>>>>>>> communicate how we're going to
> >> provide a
> >> > > > > > migration
> >> > > > > > >> > path
> >> > > > > > >> > > > from
> >> > > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>>> existing
> >> > > > > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make
> >> > > incompatible
> >> > > > > > >> > changes).
> >> > > > > > >> > > I
> >> > > > > > >> > > > > >> think
> >> > > > > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to
> >> > > provide a
> >> > > > > > >> wrapper
> >> > > > > > >> > to
> >> > > > > > >> > > > > allow
> >> > > > > > >> > > > > >> > >>>>>>>>> existing StreamTask implementations
> to
> >> > > > continue
> >> > > > > > >> > running
> >> > > > > > >> > > on
> >> > > > > > >> > > > > the
> >> > > > > > >> > > > > >> > >>>> new container.
> >> > > > > > >> > > > > >> > >>>>>>> It's
> >> > > > > > >> > > > > >> > >>>>>>>>> also important that we openly
> >> communicate
> >> > > > about
> >> > > > > > >> > timing,
> >> > > > > > >> > > > and
> >> > > > > > >> > > > > >> > >>>>>>>>> stages
> >> > > > > > >> > > > > >> > >>>>> of
> >> > > > > > >> > > > > >> > >>>>>>> the
> >> > > > > > >> > > > > >> > >>>>>>>>> migration.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure
> you
> >> > have
> >> > > > > > opinions.
> >> > > > > > >> > :)
> >> > > > > > >> > > > > Please
> >> > > > > > >> > > > > >> > >>>>>>>>> send
> >> > > > > > >> > > > > >> > >>>>>> your
> >> > > > > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>> Cheers,
> >> > > > > > >> > > > > >> > >>>>>>>>> Chris
> >> > > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>>
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>> --
> >> > > > > > >> > > > > >> > >>>>>> -- Guozhang
> >> > > > > > >> > > > > >> > >>>>>>
> >> > > > > > >> > > > > >> > >>>>>
> >> > > > > > >> > > > > >> > >>>>
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> > >>
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >> >
> >> > > > > > >> > > > > >>
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > > >
> >> > > > > > >> > > > >
> >> > > > > > >> > > >
> >> > > > > > >> > >
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
Just to make it explicitly clear what I am proposing, here is a version of
more detailed description:

The fourth option (in addition to what Jakob summarized) we are proposing
is:

- Recharter Samza to “stream processing as a service”

- The current Samza core (the basic transformation API w/ basic partition
and offset management build-in) will be moved to Kafka Streams (i.e. part
of Kafka) and supports “run-as-a-library”

- Deprecate the SystemConsumers and SystemProducers APIs and move them to
Copycat

- The current SQL development:

   * physical operators and a Trident-like stream API should stay in Kafka
Streams as libraries, enabling any standalone deployment to use the core
window/join functions

   * query parser/planner and execution on top of a distributed service
should stay in new Samza (i.e. “stream processing as a service”)

- Advanced features related to job scheduling/state management stays in new
Samza (i.e. “streaming processing as a service”)

   * Any advanced PartitionManager implementation that can be plugged into
Kafka Streams

   * Any auto-scaling, dynamic configuration via coordinator stream

   * Any advanced state management s.t. host-affinity etc.


Pros:

- W/ the current Samza core as Kafka Streams and move the ingestion to
Copycat, we achieved most of the goals in the initial proposal:

   * Tighter coupling w/ Kafka

   * Reuse Kafka’s build-in functionalities, such as offset manager, basic
partition distribution

   * Separation of ingestion vs transformation APIs

   * offload a lot of system-specific configuration to Kafka Streams and
Copycat (i.e. SystemFactory configure, serde configure, etc.)

   * remove YARN dependency and make standalone deployment easy. As
Guozhang mentioned, it would be really easy to start a process that
internally run Kafka Streams as library.

- By re-chartering Samza as “stream processing as a service”, we address
the concern regarding to

   * Pluggable partition management

   * Running in a distributed cluster to manage process lifecycle,
fault-tolerance, resource-allocation, etc.

   * More advanced features s.t. host-affinity, auto-scaling, and dynamic
configure changes, etc.


Regarding to the code and community organization, I think the following may
be the best:

Code:

- A Kafka sub-project Kafka Streams to hold samza-core, samza-kv-store, and
the physical operator layer as library in SQL: this would allow better
alignment w/ Kafka, in code, doc, and branding

- Retain the current Samza project just to keep

   * A pluggable explicit partition management in Kafka Streams client

   * Integration w/ cluster-management systems for advanced features:

      * host-affinity, auto-scaling,, dynamic configuration, etc.

   * It will fully depend on the Kafka Streams API and remove all support
for SystemConsumers/SystemProducers in the future

Community: (this is almost the same as what Chris proposed)

- Kafka Streams: the current Samza community should be supporting this
effort together with some Kafka members, since most of the code here will
be from samza-core, samza-kv-store, and samza-sql.

- new Samza: the current Samza community should continue serve the course
to support more advanced features to run Kafka Streams as a service.
Arguably, the new Samza framework may be used to run Copycat workers as
well, at least to manage Copycat worker’s lifecycle in a clustered
environment. Hence, it would stay as a general stream processing framework
that takes in any source and output to any destination, just the transport
system is fixed to Kafka.

On Sun, Jul 12, 2015 at 7:29 PM, Yi Pan <ni...@gmail.com> wrote:

> Hi, Chris,
>
> Thanks for sending out this concrete set of points here. I agree w/ all
> but have a slight different point view on 8).
>
> My view on this is: instead of sunset Samza as TLP, can we re-charter the
> scope of Samza to be the home for "running streaming process as a service"?
>
> My main motivation is from the following points from a long internal
> discussion in LinkedIn:
>
> - There is a clear ask for pluggable partition management, like we do in
> LinkedIn, and as Ben Kirwin has mentioned in
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCACuX-D-yJX++2GnF_1Laf10KYUVyamg7UP_Dt19v0ZNmmhbCPQ@mail.gmail.com%3E
> - There are concerns on lack of support for running stream processing in a
> cluster: lifecycle management, resource allocation, fault tolerance, etc.
> - There is a question to how to support more advanced features s.t.
> host-affinity, auto-scaling, and dynamic configuration in Samza jobs, as
> raised by Martin here:
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3C0D66EFD0-B7CD-4E4E-8B2F-2716167C313C@kleppmann.com%3E
>
> We have use cases that need to address all the above three cases and most
> of the functions are all in the current Samza project, in some flavor. We
> are all supporting to merge the samza-core functionalities into Kafka
> Streams, but there is a question where we keep these functions in the
> future. One option is to start a new project that includes these functions
> that are closely related w/ "run stream-processing as-a-service", while
> another personally more attractive option is to re-charter Samza project
> just do "run stream processing as-a-service". We can avoid the overhead of
> re-starting another community for this project. Personally, I felt that
> here are the benefits we should be getting:
>
> 1. We have already agreed mostly that Kafka Streams API would allow some
> pluggable partition management functions. Hence, the advanced partition
> management can live out-side the new Kafka Streams core w/o affecting the
> run-as-a-library model in Kafka Streams.
> 2. The integration w/ cluster management system and advanced features
> listed above stays in the same project and allow existing users enjoy
> no-impact migration to Kafka Stream as the core. That also addresses Tim's
> question on "removing the support for YARN".
> 3. A separate project for stream-processing-as-a-service also allow the
> new Kafka Streams being independent to any cluster management and just
> focusing on stream process core functions, while leaving the functions that
> requires cluster-resource and state management to a separate layer.
>
> Please feel free to comment. Thanks!
>
> On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
>> Hey all,
>>
>> I want to start by saying that I'm absolutely thrilled to be a part of
>> this
>> community. The amount of level-headed, thoughtful, educated discussion
>> that's gone on over the past ~10 days is overwhelming. Wonderful.
>>
>> It seems like discussion is waning a bit, and we've reached some
>> conclusions. There are several key emails in this threat, which I want to
>> call out:
>>
>> 1. Jakob's summary of the three potential ways forward.
>>
>>
>> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
>> 2. Julian's call out that we should be focusing on community over code.
>>
>>
>> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
>> 3. Martin's summary about the benefits of merging communities.
>>
>>
>> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
>> 4. Jakob's comments about the distinction between community and code
>> paths.
>>
>>
>> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
>>
>> I agree with the comments on all of these emails. I think Martin's summary
>> of his position aligns very closely with my own. To that end, I think we
>> should get concrete about what the proposal is, and call a vote on it.
>> Given that Jay, Martin, and I seem to be aligning fairly closely, I think
>> we should start with:
>>
>> 1. [community] Make Samza a subproject of Kafka.
>> 2. [community] Make all Samza PMC/committers committers of the subproject.
>> 3. [community] Migrate Samza's website/documentation into Kafka's.
>> 4. [code] Have the Samza community and the Kafka community start a
>> from-scratch reboot together in the new Kafka subproject. We can
>> borrow/copy &  paste significant chunks of code from Samza's code base.
>> 5. [code] The subproject would intentionally eliminate support for both
>> other streaming systems and all deployment systems.
>> 6. [code] Attempt to provide a bridge from our SystemConsumer to KIP-26
>> (copy cat)
>> 7. [code] Attempt to provide a bridge from the new subproject's processor
>> interface to our legacy StreamTask interface.
>> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
>> subproject that has a fault-tolerant container with state management.
>>
>> It's likely that (6) and (7) won't be fully drop-in. Still, the closer we
>> can get, the better it's going to be for our existing community.
>>
>> One thing that I didn't touch on with (2) is whether any Samza PMC members
>> should be rolled into Kafka PMC membership as well (though, Jay and Jakob
>> are already PMC members on both). I think that Samza's community deserves
>> a
>> voice on the PMC, so I'd propose that we roll at least a few PMC members
>> into the Kafka PMC, but I don't have a strong framework for which people
>> to
>> pick.
>>
>> Before (8), I think that Samza's TLP can continue to commit bug fixes and
>> patches as it sees fit, provided that we openly communicate that we won't
>> necessarily migrate new features to the new subproject, and that the TLP
>> will be shut down after the migration to the Kafka subproject occurs.
>>
>> Jakob, I could use your guidance here about about how to achieve this from
>> an Apache process perspective (sorry).
>>
>> * Should I just call a vote on this proposal?
>> * Should it happen on dev or private?
>> * Do committers have binding votes, or just PMC?
>>
>> Having trouble finding much detail on the Apache wikis. :(
>>
>> Cheers,
>> Chris
>>
>> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com> wrote:
>>
>> > Thanks, Jay. This argument persuaded me actually. :)
>> >
>> > Fang, Yan
>> > yanfang724@gmail.com
>> >
>> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io> wrote:
>> >
>> > > Hey Yan,
>> > >
>> > > Yeah philosophically I think the argument is that you should capture
>> the
>> > > stream in Kafka independent of the transformation. This is obviously a
>> > > Kafka-centric view point.
>> > >
>> > > Advantages of this:
>> > > - In practice I think this is what e.g. Storm people often end up
>> doing
>> > > anyway. You usually need to throttle any access to a live serving
>> > database.
>> > > - Can have multiple subscribers and they get the same thing without
>> > > additional load on the source system.
>> > > - Applications can tap into the stream if need be by subscribing.
>> > > - You can debug your transformation by tailing the Kafka topic with
>> the
>> > > console consumer
>> > > - Can tee off the same data stream for batch analysis or Lambda arch
>> > style
>> > > re-processing
>> > >
>> > > The disadvantage is that it will use Kafka resources. But the idea is
>> > > eventually you will have multiple subscribers to any data source (at
>> > least
>> > > for monitoring) so you will end up there soon enough anyway.
>> > >
>> > > Down the road the technical benefit is that I think it gives us a good
>> > path
>> > > towards end-to-end exactly once semantics from source to destination.
>> > > Basically the connectors need to support idempotence when talking to
>> > Kafka
>> > > and we need the transactional write feature in Kafka to make the
>> > > transformation atomic. This is actually pretty doable if you separate
>> > > connector=>kafka problem from the generic transformations which are
>> > always
>> > > kafka=>kafka. However I think it is quite impossible to do in a
>> > all_things
>> > > => all_things environment. Today you can say "well the semantics of
>> the
>> > > Samza APIs depend on the connectors you use" but it is actually worse
>> > then
>> > > that because the semantics actually depend on the pairing of
>> > connectors--so
>> > > not only can you probably not get a usable "exactly once" guarantee
>> > > end-to-end it can actually be quite hard to reverse engineer what
>> > property
>> > > (if any) your end-to-end flow has if you have heterogenous systems.
>> > >
>> > > -Jay
>> > >
>> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
>> wrote:
>> > >
>> > > > {quote}
>> > > > maintained in a separate repository and retaining the existing
>> > > > committership but sharing as much else as possible (website, etc)
>> > > > {quote}
>> > > >
>> > > > Overall, I agree on this idea. Now the question is more about "how
>> to
>> > do
>> > > > it".
>> > > >
>> > > > On the other hand, one thing I want to point out is that, if we
>> decide
>> > to
>> > > > go this way, how do we want to support
>> > > > otherSystem-transformation-otherSystem use case?
>> > > >
>> > > > Basically, there are four user groups here:
>> > > >
>> > > > 1. Kafka-transformation-Kafka
>> > > > 2. Kafka-transformation-otherSystem
>> > > > 3. otherSystem-transformation-Kafka
>> > > > 4. otherSystem-transformation-otherSystem
>> > > >
>> > > > For group 1, they can easily use the new Samza library to achieve.
>> For
>> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka or
>> > Kafka->
>> > > > transformation -> copyCat.
>> > > >
>> > > > The problem is for group 4. Do we want to abandon this or still
>> support
>> > > it?
>> > > > Of course, this use case can be achieved by using copyCat ->
>> > > transformation
>> > > > -> Kafka -> transformation -> copyCat, the thing is how we persuade
>> > them
>> > > to
>> > > > do this long chain. If yes, it will also be a win for Kafka too. Or
>> if
>> > > > there is no one in this community actually doing this so far, maybe
>> ok
>> > to
>> > > > not support the group 4 directly.
>> > > >
>> > > > Thanks,
>> > > >
>> > > > Fang, Yan
>> > > > yanfang724@gmail.com
>> > > >
>> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
>> wrote:
>> > > >
>> > > > > Yeah I agree with this summary. I think there are kind of two
>> > questions
>> > > > > here:
>> > > > > 1. Technically does alignment/reliance on Kafka make sense
>> > > > > 2. Branding wise (naming, website, concepts, etc) does alignment
>> with
>> > > > Kafka
>> > > > > make sense
>> > > > >
>> > > > > Personally I do think both of these things would be really
>> valuable,
>> > > and
>> > > > > would dramatically alter the trajectory of the project.
>> > > > >
>> > > > > My preference would be to see if people can mostly agree on a
>> > direction
>> > > > > rather than splintering things off. From my point of view the
>> ideal
>> > > > outcome
>> > > > > of all the options discussed would be to make Samza a closely
>> aligned
>> > > > > subproject, maintained in a separate repository and retaining the
>> > > > existing
>> > > > > committership but sharing as much else as possible (website,
>> etc). No
>> > > > idea
>> > > > > about how these things work, Jacob, you probably know more.
>> > > > >
>> > > > > No discussion amongst the Kafka folks has happened on this, but
>> > likely
>> > > we
>> > > > > should figure out what the Samza community actually wants first.
>> > > > >
>> > > > > I admit that this is a fairly radical departure from how things
>> are.
>> > > > >
>> > > > > If that doesn't fly, I think, yeah we could leave Samza as it is
>> and
>> > do
>> > > > the
>> > > > > more radical reboot inside Kafka. From my point of view that does
>> > leave
>> > > > > things in a somewhat confusing state since now there are two
>> stream
>> > > > > processing systems more or less coupled to Kafka in large part
>> made
>> > by
>> > > > the
>> > > > > same people. But, arguably that might be a cleaner way to make the
>> > > > cut-over
>> > > > > and perhaps less risky for Samza community since if it works
>> people
>> > can
>> > > > > switch and if it doesn't nothing will have changed. Dunno, how do
>> > > people
>> > > > > feel about this?
>> > > > >
>> > > > > -Jay
>> > > > >
>> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com>
>> > > wrote:
>> > > > >
>> > > > > > >  This leads me to thinking that merging projects and
>> communities
>> > > > might
>> > > > > > be a good idea: with the union of experience from both
>> communities,
>> > > we
>> > > > > will
>> > > > > > probably build a better system that is better for users.
>> > > > > > Is this what's being proposed though? Merging the projects seems
>> > like
>> > > > > > a consequence of at most one of the three directions under
>> > > discussion:
>> > > > > > 1) Samza 2.0: The Samza community relies more heavily on Kafka
>> for
>> > > > > > configuration, etc. (to a greater or lesser extent to be
>> > determined)
>> > > > > > but the Samza community would not automatically merge withe
>> Kafka
>> > > > > > community (the Phoenix/HBase example is a good one here).
>> > > > > > 2) Samza Reboot: The Samza community continues to exist with a
>> > > limited
>> > > > > > project scope, but similarly would not need to be part of the
>> Kafka
>> > > > > > community (ie given committership) to progress.  Here, maybe the
>> > > Samza
>> > > > > > team would become a subproject of Kafka (the Board frowns on
>> > > > > > subprojects at the moment, so I'm not sure if that's even
>> > feasible),
>> > > > > > but that would not be required.
>> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option the
>> > Kafka
>> > > > > > team builds its own streaming library, possibly off of Jay's
>> > > > > > prototype, which has not direct lineage to the Samza team.
>> There's
>> > > no
>> > > > > > reason for the Kafka team to bring in the Samza team.
>> > > > > >
>> > > > > > Is the Kafka community on board with this?
>> > > > > >
>> > > > > > To be clear, all three options under discussion are interesting,
>> > > > > > technically valid and likely healthy directions for the project.
>> > > > > > Also, they are not mutually exclusive.  The Samza community
>> could
>> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka community
>> went
>> > > > > > forward with 'Hey Samza!'  My points above are directed
>> entirely at
>> > > > > > the community aspect of these choices.
>> > > > > > -Jakob
>> > > > > >
>> > > > > > On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com>
>> > > wrote:
>> > > > > > > That's great.  Thanks, Jay.
>> > > > > > >
>> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io>
>> > > wrote:
>> > > > > > >
>> > > > > > >> Yeah totally agree. I think you have this issue even today,
>> > right?
>> > > > > I.e.
>> > > > > > if
>> > > > > > >> you need to make a simple config change and you're running in
>> > YARN
>> > > > > today
>> > > > > > >> you end up bouncing the job which then rebuilds state. I
>> think
>> > the
>> > > > fix
>> > > > > > is
>> > > > > > >> exactly what you described which is to have a long timeout on
>> > > > > partition
>> > > > > > >> movement for stateful jobs so that if a job is just getting
>> > > bounced,
>> > > > > and
>> > > > > > >> the cluster manager (or admin) is smart enough to restart it
>> on
>> > > the
>> > > > > same
>> > > > > > >> host when possible, it can optimistically reuse any existing
>> > state
>> > > > it
>> > > > > > finds
>> > > > > > >> on disk (if it is valid).
>> > > > > > >>
>> > > > > > >> So in this model the charter of the CM is to place processes
>> as
>> > > > > > stickily as
>> > > > > > >> possible and to restart or re-place failed processes. The
>> > charter
>> > > of
>> > > > > the
>> > > > > > >> partition management system is to control the assignment of
>> work
>> > > to
>> > > > > > these
>> > > > > > >> processes. The nice thing about this is that the work
>> > assignment,
>> > > > > > timeouts,
>> > > > > > >> behavior, configs, and code will all be the same across all
>> > > cluster
>> > > > > > >> managers.
>> > > > > > >>
>> > > > > > >> So I think that prototype would actually give you exactly
>> what
>> > you
>> > > > > want
>> > > > > > >> today for any cluster manager (or manual placement + restart
>> > > script)
>> > > > > > that
>> > > > > > >> was sticky in terms of host placement since there is already
>> a
>> > > > > > configurable
>> > > > > > >> partition movement timeout and task-by-task state reuse with
>> a
>> > > check
>> > > > > on
>> > > > > > >> state validity.
>> > > > > > >>
>> > > > > > >> -Jay
>> > > > > > >>
>> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
>> > > > roger.hoover@gmail.com
>> > > > > >
>> > > > > > >> wrote:
>> > > > > > >>
>> > > > > > >> > That would be great to let Kafka do as much heavy lifting
>> as
>> > > > > possible
>> > > > > > and
>> > > > > > >> > make it easier for other languages to implement Samza apis.
>> > > > > > >> >
>> > > > > > >> > One thing to watch out for is the interplay between Kafka's
>> > > group
>> > > > > > >> > management and the external scheduler/process manager's
>> fault
>> > > > > > tolerance.
>> > > > > > >> > If a container dies, the Kafka group membership protocol
>> will
>> > > try
>> > > > to
>> > > > > > >> assign
>> > > > > > >> > it's tasks to other containers while at the same time the
>> > > process
>> > > > > > manager
>> > > > > > >> > is trying to relaunch the container.  Without some
>> > consideration
>> > > > for
>> > > > > > this
>> > > > > > >> > (like a configurable amount of time to wait before Kafka
>> > alters
>> > > > the
>> > > > > > group
>> > > > > > >> > membership), there may be thrashing going on which is
>> > especially
>> > > > bad
>> > > > > > for
>> > > > > > >> > containers with large amounts of local state.
>> > > > > > >> >
>> > > > > > >> > Someone else pointed this out already but I thought it
>> might
>> > be
>> > > > > worth
>> > > > > > >> > calling out again.
>> > > > > > >> >
>> > > > > > >> > Cheers,
>> > > > > > >> >
>> > > > > > >> > Roger
>> > > > > > >> >
>> > > > > > >> >
>> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
>> jay@confluent.io>
>> > > > > wrote:
>> > > > > > >> >
>> > > > > > >> > > Hey Roger,
>> > > > > > >> > >
>> > > > > > >> > > I couldn't agree more. We spent a bunch of time talking
>> to
>> > > > people
>> > > > > > and
>> > > > > > >> > that
>> > > > > > >> > > is exactly the stuff we heard time and again. What makes
>> it
>> > > > hard,
>> > > > > of
>> > > > > > >> > > course, is that there is some tension between
>> compatibility
>> > > with
>> > > > > > what's
>> > > > > > >> > > there now and making things better for new users.
>> > > > > > >> > >
>> > > > > > >> > > I also strongly agree with the importance of
>> multi-language
>> > > > > > support. We
>> > > > > > >> > are
>> > > > > > >> > > talking now about Java, but for application development
>> use
>> > > > cases
>> > > > > > >> people
>> > > > > > >> > > want to work in whatever language they are using
>> elsewhere.
>> > I
>> > > > > think
>> > > > > > >> > moving
>> > > > > > >> > > to a model where Kafka itself does the group membership,
>> > > > lifecycle
>> > > > > > >> > control,
>> > > > > > >> > > and partition assignment has the advantage of putting all
>> > that
>> > > > > > complex
>> > > > > > >> > > stuff behind a clean api that the clients are already
>> going
>> > to
>> > > > be
>> > > > > > >> > > implementing for their consumer, so the added
>> functionality
>> > > for
>> > > > > > stream
>> > > > > > >> > > processing beyond a consumer becomes very minor.
>> > > > > > >> > >
>> > > > > > >> > > -Jay
>> > > > > > >> > >
>> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
>> > > > > > roger.hoover@gmail.com>
>> > > > > > >> > > wrote:
>> > > > > > >> > >
>> > > > > > >> > > > Metamorphosis...nice. :)
>> > > > > > >> > > >
>> > > > > > >> > > > This has been a great discussion.  As a user of Samza
>> > who's
>> > > > > > recently
>> > > > > > >> > > > integrated it into a relatively large organization, I
>> just
>> > > > want
>> > > > > to
>> > > > > > >> add
>> > > > > > >> > > > support to a few points already made.
>> > > > > > >> > > >
>> > > > > > >> > > > The biggest hurdles to adoption of Samza as it
>> currently
>> > > > exists
>> > > > > > that
>> > > > > > >> > I've
>> > > > > > >> > > > experienced are:
>> > > > > > >> > > > 1) YARN - YARN is overly complex in many environments
>> > where
>> > > > > Puppet
>> > > > > > >> > would
>> > > > > > >> > > do
>> > > > > > >> > > > just fine but it was the only mechanism to get fault
>> > > > tolerance.
>> > > > > > >> > > > 2) Configuration - I think I like the idea of
>> configuring
>> > > most
>> > > > > of
>> > > > > > the
>> > > > > > >> > job
>> > > > > > >> > > > in code rather than config files.  In general, I think
>> the
>> > > > goal
>> > > > > > >> should
>> > > > > > >> > be
>> > > > > > >> > > > to make it harder to make mistakes, especially of the
>> kind
>> > > > where
>> > > > > > the
>> > > > > > >> > code
>> > > > > > >> > > > expects something and the config doesn't match.  The
>> > current
>> > > > > > config
>> > > > > > >> is
>> > > > > > >> > > > quite intricate and error-prone.  For example, the
>> > > application
>> > > > > > logic
>> > > > > > >> > may
>> > > > > > >> > > > depend on bootstrapping a topic but rather than
>> asserting
>> > > that
>> > > > > in
>> > > > > > the
>> > > > > > >> > > code,
>> > > > > > >> > > > you have to rely on getting the config right.  Likewise
>> > with
>> > > > > > serdes,
>> > > > > > >> > the
>> > > > > > >> > > > Java representations produced by various serdes (JSON,
>> > Avro,
>> > > > > etc.)
>> > > > > > >> are
>> > > > > > >> > > not
>> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
>> without
>> > > > > changing
>> > > > > > >> the
>> > > > > > >> > > > code.   It would be nice for jobs to be able to assert
>> > what
>> > > > they
>> > > > > > >> expect
>> > > > > > >> > > > from their input topics in terms of partitioning.
>> This is
>> > > > > > getting a
>> > > > > > >> > > little
>> > > > > > >> > > > off topic but I was even thinking about creating a
>> "Samza
>> > > > config
>> > > > > > >> > linter"
>> > > > > > >> > > > that would sanity check a set of configs.  Especially
>> in
>> > > > > > >> organizations
>> > > > > > >> > > > where config is managed by a different team than the
>> > > > application
>> > > > > > >> > > developer,
>> > > > > > >> > > > it's very hard to get avoid config mistakes.
>> > > > > > >> > > > 3) Java/Scala centric - for many teams (especially
>> > > DevOps-type
>> > > > > > >> folks),
>> > > > > > >> > > the
>> > > > > > >> > > > pain of the Java toolchain (maven, slow builds, weak
>> > command
>> > > > > line
>> > > > > > >> > > support,
>> > > > > > >> > > > configuration over convention) really inhibits
>> > productivity.
>> > > > As
>> > > > > > more
>> > > > > > >> > and
>> > > > > > >> > > > more high-quality clients become available for Kafka, I
>> > hope
>> > > > > > they'll
>> > > > > > >> > > follow
>> > > > > > >> > > > Samza's model.  Not sure how much it affects the
>> proposals
>> > > in
>> > > > > this
>> > > > > > >> > thread
>> > > > > > >> > > > but please consider other languages in the ecosystem as
>> > > well.
>> > > > > > From
>> > > > > > >> > what
>> > > > > > >> > > > I've heard, Spark has more Python users than
>> Java/Scala.
>> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
>> > > > > > >> > > >
>> > > > > > >> > > >
>> > > > > > >> > >
>> > > > > > >> >
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
>> > > > > > >> > > > and are working on a Yeoman generator
>> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
>> > > Jython/Samza
>> > > > > > >> projects
>> > > > > > >> > to
>> > > > > > >> > > > alleviate some of the pain)
>> > > > > > >> > > >
>> > > > > > >> > > > I also want to underscore Jay's point about improving
>> the
>> > > user
>> > > > > > >> > > experience.
>> > > > > > >> > > > That's a very important factor for adoption.  I think
>> the
>> > > goal
>> > > > > > should
>> > > > > > >> > be
>> > > > > > >> > > to
>> > > > > > >> > > > make Samza as easy to get started with as something
>> like
>> > > > > Logstash.
>> > > > > > >> > > > Logstash is vastly inferior in terms of capabilities to
>> > > Samza
>> > > > > but
>> > > > > > >> it's
>> > > > > > >> > > easy
>> > > > > > >> > > > to get started and that makes a big difference.
>> > > > > > >> > > >
>> > > > > > >> > > > Cheers,
>> > > > > > >> > > >
>> > > > > > >> > > > Roger
>> > > > > > >> > > >
>> > > > > > >> > > >
>> > > > > > >> > > >
>> > > > > > >> > > >
>> > > > > > >> > > >
>> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci
>> > > > Morales <
>> > > > > > >> > > > gdfm@apache.org> wrote:
>> > > > > > >> > > >
>> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
>> Metamorphosis
>> > > is
>> > > > a
>> > > > > > clear
>> > > > > > >> > > > winner
>> > > > > > >> > > > > :)
>> > > > > > >> > > > >
>> > > > > > >> > > > > --
>> > > > > > >> > > > > Gianmarco
>> > > > > > >> > > > >
>> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
>> Morales
>> > <
>> > > > > > >> > > gdfm@apache.org
>> > > > > > >> > > > >
>> > > > > > >> > > > > wrote:
>> > > > > > >> > > > >
>> > > > > > >> > > > > > Hi,
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > @Martin, thanks for you comments.
>> > > > > > >> > > > > > Maybe I'm missing some important point, but I think
>> > > > coupling
>> > > > > > the
>> > > > > > >> > > > releases
>> > > > > > >> > > > > > is actually a *good* thing.
>> > > > > > >> > > > > > To make an example, would it be better if the MR
>> and
>> > > HDFS
>> > > > > > >> > components
>> > > > > > >> > > of
>> > > > > > >> > > > > > Hadoop had different release schedules?
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > Actually, keeping the discussion in a single place
>> > would
>> > > > > make
>> > > > > > >> > > agreeing
>> > > > > > >> > > > on
>> > > > > > >> > > > > > releases (and backwards compatibility) much
>> easier, as
>> > > > > > everybody
>> > > > > > >> > > would
>> > > > > > >> > > > be
>> > > > > > >> > > > > > responsible for the whole codebase.
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > That said, I like the idea of absorbing samza-core
>> as
>> > a
>> > > > > > >> > sub-project,
>> > > > > > >> > > > and
>> > > > > > >> > > > > > leave the fancy stuff separate.
>> > > > > > >> > > > > > It probably gives 90% of the benefits we have been
>> > > > > discussing
>> > > > > > >> here.
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > Cheers,
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > --
>> > > > > > >> > > > > > Gianmarco
>> > > > > > >> > > > > >
>> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
>> > jay.kreps@gmail.com
>> > > >
>> > > > > > wrote:
>> > > > > > >> > > > > >
>> > > > > > >> > > > > >> Hey Martin,
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> I agree coupling release schedules is a downside.
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> Definitely we can try to solve some of the
>> > integration
>> > > > > > problems
>> > > > > > >> in
>> > > > > > >> > > > > >> Confluent Platform or in other distributions. But
>> I
>> > > think
>> > > > > > this
>> > > > > > >> > ends
>> > > > > > >> > > up
>> > > > > > >> > > > > >> being really shallow. I guess I feel to really
>> get a
>> > > good
>> > > > > > user
>> > > > > > >> > > > > experience
>> > > > > > >> > > > > >> the two systems have to kind of feel like part of
>> the
>> > > > same
>> > > > > > thing
>> > > > > > >> > and
>> > > > > > >> > > > you
>> > > > > > >> > > > > >> can't really add that in later--you can put both
>> in
>> > the
>> > > > > same
>> > > > > > >> > > > > downloadable
>> > > > > > >> > > > > >> tar file but it doesn't really give a very
>> cohesive
>> > > > > feeling.
>> > > > > > I
>> > > > > > >> > agree
>> > > > > > >> > > > > that
>> > > > > > >> > > > > >> ultimately any of the project stuff is as much
>> social
>> > > and
>> > > > > > naming
>> > > > > > >> > as
>> > > > > > >> > > > > >> anything else--theoretically two totally
>> independent
>> > > > > projects
>> > > > > > >> > could
>> > > > > > >> > > > work
>> > > > > > >> > > > > >> to
>> > > > > > >> > > > > >> tightly align. In practice this seems to be quite
>> > > > difficult
>> > > > > > >> > though.
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> For the frameworks--totally agree it would be
>> good to
>> > > > > > maintain
>> > > > > > >> the
>> > > > > > >> > > > > >> framework support with the project. In some cases
>> > there
>> > > > may
>> > > > > > not
>> > > > > > >> be
>> > > > > > >> > > too
>> > > > > > >> > > > > >> much
>> > > > > > >> > > > > >> there since the integration gets lighter but I
>> think
>> > > > > whatever
>> > > > > > >> > stubs
>> > > > > > >> > > > you
>> > > > > > >> > > > > >> need should be included. So no I definitely wasn't
>> > > trying
>> > > > > to
>> > > > > > >> imply
>> > > > > > >> > > > > >> dropping
>> > > > > > >> > > > > >> support for these frameworks, just making the
>> > > integration
>> > > > > > >> lighter
>> > > > > > >> > by
>> > > > > > >> > > > > >> separating process management from partition
>> > > management.
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> You raise two good points we would have to figure
>> out
>> > > if
>> > > > we
>> > > > > > went
>> > > > > > >> > > down
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> alignment path:
>> > > > > > >> > > > > >> 1. With respect to the name, yeah I think the
>> first
>> > > > > question
>> > > > > > is
>> > > > > > >> > > > whether
>> > > > > > >> > > > > >> some "re-branding" would be worth it. If so then I
>> > > think
>> > > > we
>> > > > > > can
>> > > > > > >> > > have a
>> > > > > > >> > > > > big
>> > > > > > >> > > > > >> thread on the name. I'm definitely not set on
>> Kafka
>> > > > > > Streaming or
>> > > > > > >> > > Kafka
>> > > > > > >> > > > > >> Streams I was just using them to be kind of
>> > > > illustrative. I
>> > > > > > >> agree
>> > > > > > >> > > with
>> > > > > > >> > > > > >> your
>> > > > > > >> > > > > >> critique of these names, though I think people
>> would
>> > > get
>> > > > > the
>> > > > > > >> idea.
>> > > > > > >> > > > > >> 2. Yeah you also raise a good point about how to
>> > > "factor"
>> > > > > it.
>> > > > > > >> Here
>> > > > > > >> > > are
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> options I see (I could get enthusiastic about any
>> of
>> > > > them):
>> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
>> > > > > > >> > > > > >>    b. Two repos, retaining the current seperation
>> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api and
>> > > > samza-core
>> > > > > > is
>> > > > > > >> > > > absorbed
>> > > > > > >> > > > > >> almost like a third client
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> Cheers,
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> -Jay
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
>> > > > > > >> > > > martin@kleppmann.com>
>> > > > > > >> > > > > >> wrote:
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a few
>> > > follow-up
>> > > > > > >> > comments.
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
>> > becoming
>> > > a
>> > > > > > >> > subproject:
>> > > > > > >> > > > the
>> > > > > > >> > > > > >> > reasons you mention are good. The risk I see is
>> > that
>> > > > > > release
>> > > > > > >> > > > schedules
>> > > > > > >> > > > > >> > become coupled to each other, which can slow
>> > everyone
>> > > > > down,
>> > > > > > >> and
>> > > > > > >> > > > large
>> > > > > > >> > > > > >> > projects with many contributors are harder to
>> > manage.
>> > > > > > (Jakob,
>> > > > > > >> > can
>> > > > > > >> > > > you
>> > > > > > >> > > > > >> speak
>> > > > > > >> > > > > >> > from experience, having seen a wider range of
>> > Hadoop
>> > > > > > ecosystem
>> > > > > > >> > > > > >> projects?)
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > Some of the goals of a better unified developer
>> > > > > experience
>> > > > > > >> could
>> > > > > > >> > > > also
>> > > > > > >> > > > > be
>> > > > > > >> > > > > >> > solved by integrating Samza nicely into a Kafka
>> > > > > > distribution
>> > > > > > >> > (such
>> > > > > > >> > > > as
>> > > > > > >> > > > > >> > Confluent's). I'm not against merging projects
>> if
>> > we
>> > > > > decide
>> > > > > > >> > that's
>> > > > > > >> > > > the
>> > > > > > >> > > > > >> way
>> > > > > > >> > > > > >> > to go, just pointing out the same goals can
>> perhaps
>> > > > also
>> > > > > be
>> > > > > > >> > > achieved
>> > > > > > >> > > > > in
>> > > > > > >> > > > > >> > other ways.
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > - With regard to dropping the YARN dependency:
>> are
>> > > you
>> > > > > > >> proposing
>> > > > > > >> > > > that
>> > > > > > >> > > > > >> > Samza doesn't give any help to people wanting to
>> > run
>> > > on
>> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
>> > > > > > >> > > > > >> > So the docs would basically have a link to
>> Slider
>> > and
>> > > > > > nothing
>> > > > > > >> > > else?
>> > > > > > >> > > > Or
>> > > > > > >> > > > > >> > would we maintain integrations with a bunch of
>> > > popular
>> > > > > > >> > deployment
>> > > > > > >> > > > > >> methods
>> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts to
>> make
>> > > > Samza
>> > > > > > work
>> > > > > > >> > with
>> > > > > > >> > > > > >> Slider)?
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > I absolutely think it's a good idea to have the
>> > "as a
>> > > > > > library"
>> > > > > > >> > and
>> > > > > > >> > > > > "as a
>> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
>> people
>> > who
>> > > > > want
>> > > > > > >> them,
>> > > > > > >> > > > but I
>> > > > > > >> > > > > >> > think there should also be a low-friction path
>> for
>> > > > common
>> > > > > > "as
>> > > > > > >> a
>> > > > > > >> > > > > service"
>> > > > > > >> > > > > >> > deployment methods, for which we probably need
>> to
>> > > > > maintain
>> > > > > > >> > > > > integrations.
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to
>> me,
>> > > > > because
>> > > > > > >> Kafka
>> > > > > > >> > > is
>> > > > > > >> > > > > all
>> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
>> Transformers"
>> > > or
>> > > > > > "Kafka
>> > > > > > >> > > > Filters"
>> > > > > > >> > > > > >> > would be more apt?
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
>> (stream
>> > > > > > >> transformation
>> > > > > > >> > > > with
>> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a
>> library"
>> > > bit)
>> > > > > > could
>> > > > > > >> > > become
>> > > > > > >> > > > > >> part of
>> > > > > > >> > > > > >> > Kafka, while higher-level tools such as
>> streaming
>> > SQL
>> > > > and
>> > > > > > >> > > > integrations
>> > > > > > >> > > > > >> with
>> > > > > > >> > > > > >> > deployment frameworks remain in a separate
>> project?
>> > > In
>> > > > > > other
>> > > > > > >> > > words,
>> > > > > > >> > > > > >> Kafka
>> > > > > > >> > > > > >> > would absorb the proven, stable core of Samza,
>> > which
>> > > > > would
>> > > > > > >> > become
>> > > > > > >> > > > the
>> > > > > > >> > > > > >> > "third Kafka client" mentioned early in this
>> > thread.
>> > > > The
>> > > > > > Samza
>> > > > > > >> > > > project
>> > > > > > >> > > > > >> > would then target that third Kafka client as its
>> > base
>> > > > > API,
>> > > > > > and
>> > > > > > >> > the
>> > > > > > >> > > > > >> project
>> > > > > > >> > > > > >> > would be freed up to explore more experimental
>> new
>> > > > > > horizons.
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > Martin
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
>> > > > jay.kreps@gmail.com>
>> > > > > > >> wrote:
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > > Hey Martin,
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually
>> > don't
>> > > > > think
>> > > > > > it
>> > > > > > >> > ties
>> > > > > > >> > > > our
>> > > > > > >> > > > > >> > hands
>> > > > > > >> > > > > >> > > at all, all it does is refactor things. The
>> > > division
>> > > > of
>> > > > > > >> > > > > >> responsibility is
>> > > > > > >> > > > > >> > > that Samza core is responsible for task
>> > lifecycle,
>> > > > > state,
>> > > > > > >> and
>> > > > > > >> > > > > >> partition
>> > > > > > >> > > > > >> > > management (using the Kafka co-ordinator) but
>> it
>> > is
>> > > > NOT
>> > > > > > >> > > > responsible
>> > > > > > >> > > > > >> for
>> > > > > > >> > > > > >> > > packaging, configuration deployment or
>> execution
>> > of
>> > > > > > >> processes.
>> > > > > > >> > > The
>> > > > > > >> > > > > >> > problem
>> > > > > > >> > > > > >> > > of packaging and starting these processes is
>> > > > > > >> > > > > >> > > framework/environment-specific. This leaves
>> > > > individual
>> > > > > > >> > > frameworks
>> > > > > > >> > > > to
>> > > > > > >> > > > > >> be
>> > > > > > >> > > > > >> > as
>> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you can get
>> > > simple
>> > > > > > >> stateless
>> > > > > > >> > > > > >> support in
>> > > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app
>> > > > > framework
>> > > > > > >> > > (Slider,
>> > > > > > >> > > > > >> > Marathon,
>> > > > > > >> > > > > >> > > etc). These are well known by people and have
>> > nice
>> > > > UIs
>> > > > > > and a
>> > > > > > >> > lot
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> > > flexibility. I don't think they have node
>> > affinity
>> > > > as a
>> > > > > > >> built
>> > > > > > >> > in
>> > > > > > >> > > > > >> option
>> > > > > > >> > > > > >> > > (though I could be wrong). So if we want that
>> we
>> > > can
>> > > > > > either
>> > > > > > >> > wait
>> > > > > > >> > > > for
>> > > > > > >> > > > > >> them
>> > > > > > >> > > > > >> > > to add it or do a custom framework to add that
>> > > > feature
>> > > > > > (as
>> > > > > > >> > now).
>> > > > > > >> > > > > >> > Obviously
>> > > > > > >> > > > > >> > > if you manage things with old-school ops tools
>> > > > > > >> > (puppet/chef/etc)
>> > > > > > >> > > > you
>> > > > > > >> > > > > >> get
>> > > > > > >> > > > > >> > > locality easily. The nice thing, though, is
>> that
>> > > all
>> > > > > the
>> > > > > > >> samza
>> > > > > > >> > > > > >> "business
>> > > > > > >> > > > > >> > > logic" around partition management and fault
>> > > > tolerance
>> > > > > > is in
>> > > > > > >> > > Samza
>> > > > > > >> > > > > >> core
>> > > > > > >> > > > > >> > so
>> > > > > > >> > > > > >> > > it is shared across frameworks and the
>> framework
>> > > > > specific
>> > > > > > >> bit
>> > > > > > >> > is
>> > > > > > >> > > > > just
>> > > > > > >> > > > > >> > > whether it is smart enough to try to get the
>> same
>> > > > host
>> > > > > > when
>> > > > > > >> a
>> > > > > > >> > > job
>> > > > > > >> > > > is
>> > > > > > >> > > > > >> > > restarted.
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah I
>> think
>> > > the
>> > > > > > goal
>> > > > > > >> > would
>> > > > > > >> > > > be
>> > > > > > >> > > > > >> (a)
>> > > > > > >> > > > > >> > > actually get better alignment in user
>> experience,
>> > > and
>> > > > > (b)
>> > > > > > >> > > express
>> > > > > > >> > > > > >> this in
>> > > > > > >> > > > > >> > > the naming and project branding. Specifically:
>> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
>> > > > > > "transformation"
>> > > > > > >> api
>> > > > > > >> > > to
>> > > > > > >> > > > be
>> > > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be
>> able
>> > > to
>> > > > > > explain
>> > > > > > >> > > when
>> > > > > > >> > > > to
>> > > > > > >> > > > > >> use
>> > > > > > >> > > > > >> > > the consumer and when to use the stream
>> > processing
>> > > > > > >> > functionality
>> > > > > > >> > > > and
>> > > > > > >> > > > > >> lead
>> > > > > > >> > > > > >> > > people into that experience.
>> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2
>> (or
>> > > > > > whatever)
>> > > > > > >> > that
>> > > > > > >> > > > has
>> > > > > > >> > > > > >> both
>> > > > > > >> > > > > >> > > Kafka and the stream processing part and they
>> > > > actually
>> > > > > > work
>> > > > > > >> > > > > together.
>> > > > > > >> > > > > >> > > 3. Unify the programming experience so the
>> client
>> > > and
>> > > > > > Samza
>> > > > > > >> > api
>> > > > > > >> > > > > share
>> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > I think sub-projects keep separate committers
>> and
>> > > can
>> > > > > > have a
>> > > > > > >> > > > > separate
>> > > > > > >> > > > > >> > repo,
>> > > > > > >> > > > > >> > > but I'm actually not really sure (I can't
>> find a
>> > > > > > definition
>> > > > > > >> > of a
>> > > > > > >> > > > > >> > subproject
>> > > > > > >> > > > > >> > > in Apache).
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > Basically at a high-level you want the
>> experience
>> > > to
>> > > > > > "feel"
>> > > > > > >> > > like a
>> > > > > > >> > > > > >> single
>> > > > > > >> > > > > >> > > system, not to relatively independent things
>> that
>> > > are
>> > > > > > kind
>> > > > > > >> of
>> > > > > > >> > > > > >> awkwardly
>> > > > > > >> > > > > >> > > glued together.
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > I think if we did that they having naming or
>> > > branding
>> > > > > > like
>> > > > > > >> > > "kafka
>> > > > > > >> > > > > >> > > streaming" or "kafka streams" or something
>> like
>> > > that
>> > > > > > would
>> > > > > > >> > > > actually
>> > > > > > >> > > > > >> do a
>> > > > > > >> > > > > >> > > good job of conveying what it is. I do that
>> this
>> > > > would
>> > > > > > help
>> > > > > > >> > > > adoption
>> > > > > > >> > > > > >> > quite
>> > > > > > >> > > > > >> > > a lot as it would correctly convey that using
>> > Kafka
>> > > > > > >> Streaming
>> > > > > > >> > > with
>> > > > > > >> > > > > >> Kafka
>> > > > > > >> > > > > >> > is
>> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
>> pretty
>> > > > > heavily
>> > > > > > >> > adopted
>> > > > > > >> > > > at
>> > > > > > >> > > > > >> this
>> > > > > > >> > > > > >> > > point.
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > Fwiw we actually considered this model
>> originally
>> > > > when
>> > > > > > open
>> > > > > > >> > > > sourcing
>> > > > > > >> > > > > >> > Samza,
>> > > > > > >> > > > > >> > > however at that time Kafka was relatively
>> unknown
>> > > and
>> > > > > we
>> > > > > > >> > decided
>> > > > > > >> > > > not
>> > > > > > >> > > > > >> to
>> > > > > > >> > > > > >> > do
>> > > > > > >> > > > > >> > > it since we felt it would be limiting. From my
>> > > point
>> > > > of
>> > > > > > view
>> > > > > > >> > the
>> > > > > > >> > > > > three
>> > > > > > >> > > > > >> > > things have changed (1) Kafka is now really
>> > heavily
>> > > > > used
>> > > > > > for
>> > > > > > >> > > > stream
>> > > > > > >> > > > > >> > > processing, (2) we learned that abstracting
>> out
>> > the
>> > > > > > stream
>> > > > > > >> > well
>> > > > > > >> > > is
>> > > > > > >> > > > > >> > > basically impossible, (3) we learned it is
>> really
>> > > > hard
>> > > > > to
>> > > > > > >> keep
>> > > > > > >> > > the
>> > > > > > >> > > > > two
>> > > > > > >> > > > > >> > > things feeling like a single product.
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > -Jay
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
>> Kleppmann
>> > <
>> > > > > > >> > > > > >> martin@kleppmann.com>
>> > > > > > >> > > > > >> > > wrote:
>> > > > > > >> > > > > >> > >
>> > > > > > >> > > > > >> > >> Hi all,
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> Lots of good thoughts here.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> I agree with the general philosophy of tying
>> > Samza
>> > > > > more
>> > > > > > >> > firmly
>> > > > > > >> > > to
>> > > > > > >> > > > > >> Kafka.
>> > > > > > >> > > > > >> > >> After I spent a while looking at integrating
>> > other
>> > > > > > message
>> > > > > > >> > > > brokers
>> > > > > > >> > > > > >> (e.g.
>> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
>> > > > conclusion
>> > > > > > that
>> > > > > > >> > > > > >> > SystemConsumer
>> > > > > > >> > > > > >> > >> tacitly assumes a model so much like Kafka's
>> > that
>> > > > > pretty
>> > > > > > >> much
>> > > > > > >> > > > > nobody
>> > > > > > >> > > > > >> but
>> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
>> > perhaps
>> > > an
>> > > > > > >> > exception,
>> > > > > > >> > > > but
>> > > > > > >> > > > > >> it
>> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus,
>> > > making
>> > > > > > Samza
>> > > > > > >> > > fully
>> > > > > > >> > > > > >> > dependent
>> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
>> > system-independence
>> > > > was
>> > > > > > >> never
>> > > > > > >> > as
>> > > > > > >> > > > > real
>> > > > > > >> > > > > >> as
>> > > > > > >> > > > > >> > we
>> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of code
>> > reuse
>> > > > are
>> > > > > > >> real.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN has
>> also
>> > > > always
>> > > > > > been
>> > > > > > >> > > > > >> appealing to
>> > > > > > >> > > > > >> > >> me, for various reasons already mentioned in
>> > this
>> > > > > > thread.
>> > > > > > >> > > > Although
>> > > > > > >> > > > > >> > making
>> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
>> > > > (YARN/Mesos/AWS/etc)
>> > > > > > >> seems
>> > > > > > >> > > > > >> laudable,
>> > > > > > >> > > > > >> > I am
>> > > > > > >> > > > > >> > >> a little concerned that it will restrict us
>> to a
>> > > > > lowest
>> > > > > > >> > common
>> > > > > > >> > > > > >> > denominator.
>> > > > > > >> > > > > >> > >> For example, would host affinity (SAMZA-617)
>> > still
>> > > > be
>> > > > > > >> > possible?
>> > > > > > >> > > > For
>> > > > > > >> > > > > >> jobs
>> > > > > > >> > > > > >> > >> with large amounts of state, I think
>> SAMZA-617
>> > > would
>> > > > > be
>> > > > > > a
>> > > > > > >> big
>> > > > > > >> > > > boon,
>> > > > > > >> > > > > >> > since
>> > > > > > >> > > > > >> > >> restoring state off the changelog on every
>> > single
>> > > > > > restart
>> > > > > > >> is
>> > > > > > >> > > > > painful,
>> > > > > > >> > > > > >> > due
>> > > > > > >> > > > > >> > >> to long recovery times. It would be a shame
>> if
>> > the
>> > > > > > >> decoupling
>> > > > > > >> > > > from
>> > > > > > >> > > > > >> YARN
>> > > > > > >> > > > > >> > >> made host affinity impossible.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> Jay, a question about the proposed API for
>> > > > > > instantiating a
>> > > > > > >> > job
>> > > > > > >> > > in
>> > > > > > >> > > > > >> code
>> > > > > > >> > > > > >> > >> (rather than a properties file): when
>> > submitting a
>> > > > job
>> > > > > > to a
>> > > > > > >> > > > > cluster,
>> > > > > > >> > > > > >> is
>> > > > > > >> > > > > >> > the
>> > > > > > >> > > > > >> > >> idea that the instantiation code runs on a
>> > client
>> > > > > > >> somewhere,
>> > > > > > >> > > > which
>> > > > > > >> > > > > >> then
>> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
>> > > YARN/Mesos/AWS/etc?
>> > > > > Or
>> > > > > > >> does
>> > > > > > >> > > that
>> > > > > > >> > > > > >> code
>> > > > > > >> > > > > >> > run
>> > > > > > >> > > > > >> > >> on each container that is part of the job (in
>> > > which
>> > > > > > case,
>> > > > > > >> how
>> > > > > > >> > > > does
>> > > > > > >> > > > > >> the
>> > > > > > >> > > > > >> > job
>> > > > > > >> > > > > >> > >> submission to the cluster work)?
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel
>> right to
>> > > > make
>> > > > > a
>> > > > > > 1.0
>> > > > > > >> > > > release
>> > > > > > >> > > > > >> > with a
>> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete. So if
>> > this
>> > > > is
>> > > > > > going
>> > > > > > >> > to
>> > > > > > >> > > > > >> happen, I
>> > > > > > >> > > > > >> > >> think it would be more honest to stick with
>> 0.*
>> > > > > version
>> > > > > > >> > numbers
>> > > > > > >> > > > > until
>> > > > > > >> > > > > >> > the
>> > > > > > >> > > > > >> > >> library-ified Samza has been implemented, is
>> > > stable
>> > > > > and
>> > > > > > >> > widely
>> > > > > > >> > > > > used.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of
>> Kafka?
>> > > There
>> > > > > is
>> > > > > > >> > > precedent
>> > > > > > >> > > > > for
>> > > > > > >> > > > > >> > >> tight coupling between different Apache
>> projects
>> > > > (e.g.
>> > > > > > >> > Curator
>> > > > > > >> > > > and
>> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think
>> > > remaining
>> > > > > > >> separate
>> > > > > > >> > > > would
>> > > > > > >> > > > > >> be
>> > > > > > >> > > > > >> > ok.
>> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on Kafka,
>> there
>> > > is
>> > > > > > enough
>> > > > > > >> > > > > substance
>> > > > > > >> > > > > >> in
>> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
>> project.
>> > > An
>> > > > > > >> argument
>> > > > > > >> > in
>> > > > > > >> > > > > >> favour
>> > > > > > >> > > > > >> > of
>> > > > > > >> > > > > >> > >> merging would be if we think Kafka has a much
>> > > > stronger
>> > > > > > >> "brand
>> > > > > > >> > > > > >> presence"
>> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If
>> the
>> > > Kafka
>> > > > > > >> project
>> > > > > > >> > is
>> > > > > > >> > > > > >> willing
>> > > > > > >> > > > > >> > to
>> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of doing
>> > > > stateful
>> > > > > > >> stream
>> > > > > > >> > > > > >> > >> transformations, that would probably have
>> much
>> > the
>> > > > > same
>> > > > > > >> > effect
>> > > > > > >> > > as
>> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
>> Processors"
>> > or
>> > > > > > suchlike.
>> > > > > > >> > > Close
>> > > > > > >> > > > > >> > >> collaboration between the two projects will
>> be
>> > > > needed
>> > > > > in
>> > > > > > >> any
>> > > > > > >> > > > case.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> From a project management perspective, I
>> guess
>> > the
>> > > > > "new
>> > > > > > >> > Samza"
>> > > > > > >> > > > > would
>> > > > > > >> > > > > >> > have
>> > > > > > >> > > > > >> > >> to be developed on a branch alongside ongoing
>> > > > > > maintenance
>> > > > > > >> of
>> > > > > > >> > > the
>> > > > > > >> > > > > >> current
>> > > > > > >> > > > > >> > >> line of development? I think it would be
>> > important
>> > > > to
>> > > > > > >> > continue
>> > > > > > >> > > > > >> > supporting
>> > > > > > >> > > > > >> > >> existing users, and provide a graceful
>> migration
>> > > > path
>> > > > > to
>> > > > > > >> the
>> > > > > > >> > > new
>> > > > > > >> > > > > >> > version.
>> > > > > > >> > > > > >> > >> Leaving the current versions unsupported and
>> > > forcing
>> > > > > > people
>> > > > > > >> > to
>> > > > > > >> > > > > >> rewrite
>> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> Best,
>> > > > > > >> > > > > >> > >> Martin
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
>> > > > jay@confluent.io>
>> > > > > > >> wrote:
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >>> Hey Garry,
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy
>> to
>> > > chat
>> > > > > > more
>> > > > > > >> > about
>> > > > > > >> > > > > this
>> > > > > > >> > > > > >> if
>> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
>> > started
>> > > > with
>> > > > > > the
>> > > > > > >> > idea
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> "what
>> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
>> > ingestion
>> > > > > tool"
>> > > > > > but
>> > > > > > >> > > > > >> ultimately
>> > > > > > >> > > > > >> > we
>> > > > > > >> > > > > >> > >>> kind of came around to the idea that
>> ingestion
>> > > and
>> > > > > > >> > > > transformation
>> > > > > > >> > > > > >> had
>> > > > > > >> > > > > >> > >>> pretty different needs and coupling the two
>> > made
>> > > > > things
>> > > > > > >> > hard.
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>> For what it's worth I think copycat (KIP-26)
>> > > > actually
>> > > > > > will
>> > > > > > >> > do
>> > > > > > >> > > > what
>> > > > > > >> > > > > >> you
>> > > > > > >> > > > > >> > >> are
>> > > > > > >> > > > > >> > >>> looking for.
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>> With regard to your point about slider, I
>> don't
>> > > > > > >> necessarily
>> > > > > > >> > > > > >> disagree.
>> > > > > > >> > > > > >> > >> But I
>> > > > > > >> > > > > >> > >>> think getting good YARN support is quite
>> doable
>> > > > and I
>> > > > > > >> think
>> > > > > > >> > we
>> > > > > > >> > > > can
>> > > > > > >> > > > > >> make
>> > > > > > >> > > > > >> > >>> that work well. I think the issue this
>> proposal
>> > > > > solves
>> > > > > > is
>> > > > > > >> > that
>> > > > > > >> > > > > >> > >> technically
>> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple
>> cluster
>> > > > > > management
>> > > > > > >> > > systems
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> > way
>> > > > > > >> > > > > >> > >>> things are now, you need to write an "app
>> > master"
>> > > > or
>> > > > > > >> > > "framework"
>> > > > > > >> > > > > for
>> > > > > > >> > > > > >> > each
>> > > > > > >> > > > > >> > >>> and they are all a little different so
>> testing
>> > is
>> > > > > > really
>> > > > > > >> > hard.
>> > > > > > >> > > > In
>> > > > > > >> > > > > >> the
>> > > > > > >> > > > > >> > >>> absence of this we have been stuck with just
>> > YARN
>> > > > > which
>> > > > > > >> has
>> > > > > > >> > > > > >> fantastic
>> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the org,
>> but
>> > > > zero
>> > > > > > >> > > penetration
>> > > > > > >> > > > > >> > >> elsewhere.
>> > > > > > >> > > > > >> > >>> Given the huge amount of work being put in
>> to
>> > > > slider,
>> > > > > > >> > > marathon,
>> > > > > > >> > > > > aws
>> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen related
>> > > > packaging
>> > > > > > >> > > > technologies
>> > > > > > >> > > > > >> > people
>> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
>> > > > > cloud-specific
>> > > > > > >> > deploy
>> > > > > > >> > > > > >> tools,
>> > > > > > >> > > > > >> > >> etc)
>> > > > > > >> > > > > >> > >>> I really think it is important to get this
>> > right.
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>> -Jay
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
>> > Turkington
>> > > <
>> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
>> > > > > > >> > > > > >> > >>>
>> > > > > > >> > > > > >> > >>>> Hi all,
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> I think the question below re does Samza
>> > become
>> > > a
>> > > > > > >> > sub-project
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> Kafka
>> > > > > > >> > > > > >> > >>>> highlights the broader point around
>> migration.
>> > > > Chris
>> > > > > > >> > mentions
>> > > > > > >> > > > > >> Samza's
>> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1 release
>> but
>> > I'm
>> > > > not
>> > > > > > sure
>> > > > > > >> > it
>> > > > > > >> > > > > feels
>> > > > > > >> > > > > >> > >> right to
>> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
>> deprecate
>> > > > most
>> > > > > of
>> > > > > > >> it.
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> From a selfish perspective I have some guys
>> > who
>> > > > have
>> > > > > > >> > started
>> > > > > > >> > > > > >> working
>> > > > > > >> > > > > >> > >> with
>> > > > > > >> > > > > >> > >>>> Samza and building some new
>> > consumers/producers
>> > > > was
>> > > > > > next
>> > > > > > >> > up.
>> > > > > > >> > > > > Sounds
>> > > > > > >> > > > > >> > like
>> > > > > > >> > > > > >> > >>>> that is absolutely not the direction to
>> go. I
>> > > need
>> > > > > to
>> > > > > > >> look
>> > > > > > >> > > into
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> > KIP
>> > > > > > >> > > > > >> > >> in
>> > > > > > >> > > > > >> > >>>> more detail but for me the attractiveness
>> of
>> > > > adding
>> > > > > > new
>> > > > > > >> > Samza
>> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all they
>> > were
>> > > > > doing
>> > > > > > was
>> > > > > > >> > > > really
>> > > > > > >> > > > > >> > getting
>> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to avoid
>> > > > having
>> > > > > to
>> > > > > > >> > worry
>> > > > > > >> > > > > about
>> > > > > > >> > > > > >> the
>> > > > > > >> > > > > >> > >>>> lifecycle management of external clients.
>> If
>> > > there
>> > > > > is
>> > > > > > a
>> > > > > > >> > > generic
>> > > > > > >> > > > > >> Kafka
>> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a new
>> > > > connector
>> > > > > > into
>> > > > > > >> > and
>> > > > > > >> > > > > have
>> > > > > > >> > > > > >> a
>> > > > > > >> > > > > >> > >> lot of
>> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and reliability
>> > done
>> > > > for
>> > > > > me
>> > > > > > >> then
>> > > > > > >> > > it
>> > > > > > >> > > > > >> gives
>> > > > > > >> > > > > >> > me
>> > > > > > >> > > > > >> > >> all
>> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers would.
>> If
>> > > not
>> > > > > > then it
>> > > > > > >> > > > > >> complicates
>> > > > > > >> > > > > >> > my
>> > > > > > >> > > > > >> > >>>> operational deployments.
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> Which is similar to my other question with
>> the
>> > > > > > proposal
>> > > > > > >> --
>> > > > > > >> > if
>> > > > > > >> > > > we
>> > > > > > >> > > > > >> > build a
>> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus the
>> > > > requisite
>> > > > > > >> shims
>> > > > > > >> > to
>> > > > > > >> > > > > >> > integrate
>> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former may
>> be a
>> > > lot
>> > > > > more
>> > > > > > >> work
>> > > > > > >> > > > than
>> > > > > > >> > > > > we
>> > > > > > >> > > > > >> > >> think.
>> > > > > > >> > > > > >> > >>>> We may make it much easier for a newcomer
>> to
>> > get
>> > > > > > >> something
>> > > > > > >> > > > > running
>> > > > > > >> > > > > >> but
>> > > > > > >> > > > > >> > >>>> having them step up and get a reliable
>> > > production
>> > > > > > >> > deployment
>> > > > > > >> > > > may
>> > > > > > >> > > > > >> still
>> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
>> > different
>> > > > > > reasons
>> > > > > > >> > than
>> > > > > > >> > > > > >> today.
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with
>> > > making
>> > > > > the
>> > > > > > >> Samza
>> > > > > > >> > > > > >> dependency
>> > > > > > >> > > > > >> > >> on
>> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I absolutely
>> see
>> > > the
>> > > > > > >> benefits
>> > > > > > >> > > in
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
>> > > > > > >> > > > terminologies/abstractions
>> > > > > > >> > > > > >> that
>> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library
>> would
>> > > > likely
>> > > > > > be a
>> > > > > > >> > very
>> > > > > > >> > > > > nice
>> > > > > > >> > > > > >> > tool
>> > > > > > >> > > > > >> > >> to
>> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the
>> > > > concerns
>> > > > > > >> above
>> > > > > > >> > re
>> > > > > > >> > > > the
>> > > > > > >> > > > > >> > >>>> operational side.
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> Garry
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> -----Original Message-----
>> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
>> [mailto:
>> > > > > > >> > gdfm@apache.org
>> > > > > > >> > > ]
>> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
>> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
>> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on
>> > Samza
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
>> > > > > > >> > > > > >> > >>>> From outside, I have always perceived Samza
>> > as a
>> > > > > > >> computing
>> > > > > > >> > > > layer
>> > > > > > >> > > > > >> over
>> > > > > > >> > > > > >> > >>>> Kafka.
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> The question, maybe a bit provocative, is
>> > > "should
>> > > > > > Samza
>> > > > > > >> be
>> > > > > > >> > a
>> > > > > > >> > > > > >> > sub-project
>> > > > > > >> > > > > >> > >>>> of Kafka then?"
>> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
>> separate
>> > > > > project
>> > > > > > >> > with a
>> > > > > > >> > > > > >> separate
>> > > > > > >> > > > > >> > >>>> governance?
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> Cheers,
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> --
>> > > > > > >> > > > > >> > >>>> Gianmarco
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
>> > > > > > yanfang724@gmail.com>
>> > > > > > >> > > > wrote:
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more
>> > > > tightly.
>> > > > > > >> > Because
>> > > > > > >> > > > > Samza
>> > > > > > >> > > > > >> de
>> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
>> > leverage
>> > > > > what
>> > > > > > >> Kafka
>> > > > > > >> > > > has.
>> > > > > > >> > > > > At
>> > > > > > >> > > > > >> > the
>> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to reinvent
>> > what
>> > > > > Samza
>> > > > > > >> > > already
>> > > > > > >> > > > > >> has. I
>> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
>> > ingestion
>> > > > and
>> > > > > > >> > > > > transformation.
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
>> image
>> > > how
>> > > > > the
>> > > > > > >> Samza
>> > > > > > >> > > > will
>> > > > > > >> > > > > >> look
>> > > > > > >> > > > > >> > >>>> like.
>> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
>> > > difference
>> > > > > in
>> > > > > > >> terms
>> > > > > > >> > > of
>> > > > > > >> > > > > how
>> > > > > > >> > > > > >> > >>>>> Samza should look like.
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code
>> shows
>> > (A
>> > > > > > client of
>> > > > > > >> > > > Kakfa)
>> > > > > > >> > > > > ?
>> > > > > > >> > > > > >> And
>> > > > > > >> > > > > >> > >>>>> user's application code calls this client?
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka
>> > (like
>> > > > > what
>> > > > > > the
>> > > > > > >> > > code
>> > > > > > >> > > > > >> shows),
>> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
>> > > > > fault-tolerance?
>> > > > > > >> Are
>> > > > > > >> > > they
>> > > > > > >> > > > > >> taken
>> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
>> mechanism,
>> > > such
>> > > > > as
>> > > > > > >> > "Samza
>> > > > > > >> > > > > >> worker"
>> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> 2. What about other features, such as
>> > > > auto-scaling,
>> > > > > > >> shared
>> > > > > > >> > > > > state,
>> > > > > > >> > > > > >> > >>>>> monitoring?
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is this
>> > what
>> > > > > Chris
>> > > > > > >> > > > suggests?)
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa
>> > and
>> > > > > > produce
>> > > > > > >> to
>> > > > > > >> > > it.
>> > > > > > >> > > > > >> Then it
>> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks like
>> > now,
>> > > > > > except it
>> > > > > > >> > > does
>> > > > > > >> > > > > not
>> > > > > > >> > > > > >> > rely
>> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
>> leverage
>> > > > Kafka's
>> > > > > > >> > metrics,
>> > > > > > >> > > > > logs,
>> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> Thanks,
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> Fang, Yan
>> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang
>> > Wang <
>> > > > > > >> > > > > wangguoz@gmail.com
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> > >>>> wrote:
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>>> Read through the code example and it
>> looks
>> > > good
>> > > > to
>> > > > > > me.
>> > > > > > >> A
>> > > > > > >> > > few
>> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
>> runnable
>> > > like:
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
>> > > --config-factory=...
>> > > > > > >> > > > > >> > >>>> --config-path=file://...
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for deploying
>> > Samza
>> > > > > more
>> > > > > > as
>> > > > > > >> > > > embedded
>> > > > > > >> > > > > >> > >>>>>> libraries in user application code
>> (ignoring
>> > > the
>> > > > > > >> > > terminology
>> > > > > > >> > > > > >> since
>> > > > > > >> > > > > >> > >>>>>> it is not the
>> > > > > > >> > > > > >> > >>>>> same
>> > > > > > >> > > > > >> > >>>>>> as the prototype code):
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
>> MyStreamTask(configs);
>> > > > > Thread
>> > > > > > >> > thread
>> > > > > > >> > > =
>> > > > > > >> > > > > new
>> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> I think both of these deployment modes
>> are
>> > > > > important
>> > > > > > >> for
>> > > > > > >> > > > > >> different
>> > > > > > >> > > > > >> > >>>>>> types
>> > > > > > >> > > > > >> > >>>>> of
>> > > > > > >> > > > > >> > >>>>>> users. That said, I think making Samza
>> > purely
>> > > > > > >> standalone
>> > > > > > >> > is
>> > > > > > >> > > > > still
>> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or library
>> > > modes.
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> Guozhang
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
>> Kreps
>> > <
>> > > > > > >> > > > jay@confluent.io>
>> > > > > > >> > > > > >> > wrote:
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
>> example,
>> > it
>> > > > was
>> > > > > > >> > supposed
>> > > > > > >> > > > to
>> > > > > > >> > > > > >> look
>> > > > > > >> > > > > >> > >>>>>>> like
>> > > > > > >> > > > > >> > >>>>>>> this:
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
>> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
>> > > > "localhost:4242");
>> > > > > > >> > > > > >> StreamingConfig
>> > > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
>> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
>> > > > "test-topic-2");
>> > > > > > >> > > > > >> > >>>>>>>
>> > > config.processor(ExampleStreamProcessor.class);
>> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
>> > StringSerializer(),
>> > > > new
>> > > > > > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming
>> > > > container =
>> > > > > > new
>> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>> -Jay
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay
>> > Kreps <
>> > > > > > >> > > > jay@confluent.io
>> > > > > > >> > > > > >
>> > > > > > >> > > > > >> > >>>> wrote:
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
>> Chris
>> > > and
>> > > > I
>> > > > > > were
>> > > > > > >> > > having
>> > > > > > >> > > > > >> > >>>>>>>> around
>> > > > > > >> > > > > >> > >>>>>>> whether
>> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a
>> kind
>> > > of
>> > > > > data
>> > > > > > >> > > > ingestion
>> > > > > > >> > > > > >> > >>>>> framework
>> > > > > > >> > > > > >> > >>>>>>> for
>> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26
>> > > > > "copycat").
>> > > > > > >> This
>> > > > > > >> > > > kind
>> > > > > > >> > > > > of
>> > > > > > >> > > > > >> > >>>>>> combined
>> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and YARN
>> and
>> > > the
>> > > > > > >> > discussion
>> > > > > > >> > > > > >> around
>> > > > > > >> > > > > >> > >>>>>>>> how
>> > > > > > >> > > > > >> > >>>>> to
>> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was, given
>> that
>> > > > Samza
>> > > > > > was
>> > > > > > >> > > > basically
>> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific, what if
>> > you
>> > > > just
>> > > > > > >> > embraced
>> > > > > > >> > > > > that
>> > > > > > >> > > > > >> > >>>>>>>> and turned it
>> > > > > > >> > > > > >> > >>>>>> into
>> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
>> > framework
>> > > > and
>> > > > > > more
>> > > > > > >> > > like a
>> > > > > > >> > > > > >> > >>>>>>>> third
>> > > > > > >> > > > > >> > >>>>> Kafka
>> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing consumer"
>> > with
>> > > > > state
>> > > > > > >> > > > management
>> > > > > > >> > > > > >> > >>>>>> facilities.
>> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
>> complex
>> > > > stream
>> > > > > > >> > > processing
>> > > > > > >> > > > > >> > >>>>>>>> framework
>> > > > > > >> > > > > >> > >>>>>>> this
>> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple thing,
>> not
>> > > > much
>> > > > > > more
>> > > > > > >> > > > > >> complicated
>> > > > > > >> > > > > >> > >>>>>>>> to
>> > > > > > >> > > > > >> > >>>>> use
>> > > > > > >> > > > > >> > >>>>>>> or
>> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris
>> > said
>> > > > we
>> > > > > > >> thought
>> > > > > > >> > > > about
>> > > > > > >> > > > > >> it
>> > > > > > >> > > > > >> > >>>>>>>> a
>> > > > > > >> > > > > >> > >>>>> lot
>> > > > > > >> > > > > >> > >>>>>> of
>> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
>> > processing
>> > > > > > systems
>> > > > > > >> > were
>> > > > > > >> > > > > doing)
>> > > > > > >> > > > > >> > >>>>> seemed
>> > > > > > >> > > > > >> > >>>>>>> like
>> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output
>> data
>> > to
>> > > > and
>> > > > > > from
>> > > > > > >> > the
>> > > > > > >> > > > > stream
>> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually looked
>> > into
>> > > > how
>> > > > > > that
>> > > > > > >> > > would
>> > > > > > >> > > > > >> > >>>>>>>> work,
>> > > > > > >> > > > > >> > >>>>> Samza
>> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion
>> > > framework
>> > > > > > for a
>> > > > > > >> > > bunch
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> > >>>>> reasons.
>> > > > > > >> > > > > >> > >>>>>> To
>> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a pretty
>> > > > different
>> > > > > > >> > internal
>> > > > > > >> > > > > data
>> > > > > > >> > > > > >> > >>>>>>>> model
>> > > > > > >> > > > > >> > >>>>>> and
>> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split them
>> and
>> > > had
>> > > > > an
>> > > > > > api
>> > > > > > >> > for
>> > > > > > >> > > > > Kafka
>> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26)
>> and a
>> > > > > separate
>> > > > > > >> api
>> > > > > > >> > > for
>> > > > > > >> > > > > >> Kafka
>> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> This would also allow really embracing
>> the
>> > > > same
>> > > > > > >> > > terminology
>> > > > > > >> > > > > and
>> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about the
>> > current
>> > > > > > state is
>> > > > > > >> > > that
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> > >>>>>>>> two
>> > > > > > >> > > > > >> > >>>>>>> systems
>> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology
>> like
>> > > > > "stream"
>> > > > > > vs
>> > > > > > >> > > > "topic"
>> > > > > > >> > > > > >> and
>> > > > > > >> > > > > >> > >>>>>>> different
>> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means you
>> > kind
>> > > > of
>> > > > > > have
>> > > > > > >> to
>> > > > > > >> > > > learn
>> > > > > > >> > > > > >> > >>>>>>>> Kafka's
>> > > > > > >> > > > > >> > >>>>>>> way,
>> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly different
>> way,
>> > > > then
>> > > > > > kind
>> > > > > > >> of
>> > > > > > >> > > > > >> > >>>>>>>> understand
>> > > > > > >> > > > > >> > >>>>> how
>> > > > > > >> > > > > >> > >>>>>>> they
>> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having walked
>> a
>> > few
>> > > > > > people
>> > > > > > >> > > through
>> > > > > > >> > > > > >> this
>> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to
>> get.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of
>> time
>> > on
>> > > > > > >> airplanes I
>> > > > > > >> > > > > hacked
>> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
>> incomplete
>> > > > > > prototype
>> > > > > > >> of
>> > > > > > >> > > > what
>> > > > > > >> > > > > >> > >>>>>>>> this would
>> > > > > > >> > > > > >> > >>>>> look
>> > > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
>> dumped
>> > > into
>> > > > > > Kafka
>> > > > > > >> as
>> > > > > > >> > > it
>> > > > > > >> > > > > >> > >>>>>>>> required a
>> > > > > > >> > > > > >> > >>>>>> few
>> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is
>> the
>> > > code:
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > >
>> > > > > > >> >
>> > > > > >
>> > >
>> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I just
>> > > > > liberally
>> > > > > > >> > renamed
>> > > > > > >> > > > > >> > >>>>>>>> everything
>> > > > > > >> > > > > >> > >>>>> to
>> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
>> regard
>> > > for
>> > > > > > >> > > > compatibility.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> To use this would be something like
>> this:
>> > > > > > >> > > > > >> > >>>>>>>> Properties props = new Properties();
>> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
>> > > > > "localhost:4242");
>> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
>> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
>> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
>> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
>> > > > > > >> > > > > >> config.processor(ExampleStreamProcessor.class);
>> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
>> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
>> > > StringDeserializer());
>> > > > > > >> > > > KafkaStreaming
>> > > > > > >> > > > > >> > >>>>>> container =
>> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
>> > container.run();
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
>> > > > SamzaContainer;
>> > > > > > >> > > > > StreamProcessor
>> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the class
>> names
>> > > in
>> > > > a
>> > > > > > file
>> > > > > > >> > and
>> > > > > > >> > > > then
>> > > > > > >> > > > > >> > >>>>>>>> having
>> > > > > > >> > > > > >> > >>>>>> the
>> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
>> > > > > instantiate
>> > > > > > the
>> > > > > > >> > > > > container
>> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced over
>> > > > however
>> > > > > > many
>> > > > > > >> > > > > instances
>> > > > > > >> > > > > >> > >>>>>>>> of
>> > > > > > >> > > > > >> > >>>>> this
>> > > > > > >> > > > > >> > >>>>>>> are
>> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance
>> > dies,
>> > > > new
>> > > > > > >> tasks
>> > > > > > >> > > are
>> > > > > > >> > > > > >> added
>> > > > > > >> > > > > >> > >>>>>>>> to
>> > > > > > >> > > > > >> > >>>>> the
>> > > > > > >> > > > > >> > >>>>>>>> existing containers without shutting
>> them
>> > > > down).
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for running
>> > this
>> > > > > stuff
>> > > > > > in
>> > > > > > >> > YARN
>> > > > > > >> > > > via
>> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
>> using
>> > > some
>> > > > > of
>> > > > > > >> their
>> > > > > > >> > > > tools
>> > > > > > >> > > > > >> > >>>>>>>> but from the
>> > > > > > >> > > > > >> > >>>>>> point
>> > > > > > >> > > > > >> > >>>>>>> of
>> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these stream
>> > > > processing
>> > > > > > jobs
>> > > > > > >> > are
>> > > > > > >> > > > > just
>> > > > > > >> > > > > >> > >>>>>> stateless
>> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and
>> expand
>> > and
>> > > > > > contract
>> > > > > > >> > at
>> > > > > > >> > > > > will.
>> > > > > > >> > > > > >> > >>>>>>>> There
>> > > > > > >> > > > > >> > >>>>> is
>> > > > > > >> > > > > >> > >>>>>>> no
>> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it
>> > would
>> > > > get
>> > > > > > >> larger
>> > > > > > >> > > if
>> > > > > > >> > > > we
>> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly larger.
>> We
>> > > > really
>> > > > > > do
>> > > > > > >> > get a
>> > > > > > >> > > > ton
>> > > > > > >> > > > > >> > >>>>>>>> of
>> > > > > > >> > > > > >> > >>>>>>> leverage
>> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
>> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
>> > delegated
>> > > to
>> > > > > the
>> > > > > > >> new
>> > > > > > >> > > > > >> consumer.
>> > > > > > >> > > > > >> > >>>>> This
>> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
>> > management
>> > > > > > strategy
>> > > > > > >> > > > > available
>> > > > > > >> > > > > >> > >>>>>>>> to
>> > > > > > >> > > > > >> > >>>>>> Kafka
>> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to Samza
>> (and
>> > > vice
>> > > > > > versa)
>> > > > > > >> > and
>> > > > > > >> > > > > with
>> > > > > > >> > > > > >> > >>>>>>>> the
>> > > > > > >> > > > > >> > >>>>>>> exact
>> > > > > > >> > > > > >> > >>>>>>>>  same configs.
>> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as state
>> > reuse
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
>> > thought
>> > > > > > >> provoking.
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> -Jay
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris
>> > > > > Riccomini <
>> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
>> > > > > > >> > > > > >> > >>>>>>>> wrote:
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with Samza
>> > > > > engineers
>> > > > > > at
>> > > > > > >> > > > LinkedIn
>> > > > > > >> > > > > >> > >>>>>>>>> and
>> > > > > > >> > > > > >> > >>>>>>> Confluent
>> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few observations
>> > and
>> > > > > would
>> > > > > > >> like
>> > > > > > >> > to
>> > > > > > >> > > > > >> > >>>>>>>>> propose
>> > > > > > >> > > > > >> > >>>>> some
>> > > > > > >> > > > > >> > >>>>>>>>> changes.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I
>> want to
>> > > > call
>> > > > > > out
>> > > > > > >> > about
>> > > > > > >> > > > > >> > >>>>>>>>> Samza's
>> > > > > > >> > > > > >> > >>>>>> design,
>> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic
>> > > > deployment
>> > > > > > >> system.
>> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
>> > > > > > >> > > > > >> > >>>>>>>>> * Samza's
>> SystemConsumer/SystemProducer
>> > and
>> > > > > > Kafka's
>> > > > > > >> > > > consumer
>> > > > > > >> > > > > >> > >>>>>>>>> APIs
>> > > > > > >> > > > > >> > >>>>> are
>> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
>> > problems.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are related,
>> > but
>> > > > I'll
>> > > > > > >> > address
>> > > > > > >> > > > them
>> > > > > > >> > > > > >> in
>> > > > > > >> > > > > >> > >>>>> order.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Deployment
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a
>> > > > dynamic
>> > > > > > >> > > deployment
>> > > > > > >> > > > > >> > >>>>>>>>> scheduler
>> > > > > > >> > > > > >> > >>>>>> such
>> > > > > > >> > > > > >> > >>>>>>>>> as
>> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially
>> built
>> > > > > Samza,
>> > > > > > we
>> > > > > > >> > bet
>> > > > > > >> > > > that
>> > > > > > >> > > > > >> > >>>>>>>>> there
>> > > > > > >> > > > > >> > >>>>>> would
>> > > > > > >> > > > > >> > >>>>>>>>> be
>> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area, and
>> we
>> > > could
>> > > > > > >> support
>> > > > > > >> > > > them,
>> > > > > > >> > > > > >> and
>> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > > >> > > > > >> > >>>>>> rest
>> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there are
>> many
>> > > > > > >> variations.
>> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
>> > > > > > >> > > > > >> > >>>>>> many
>> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start
>> their
>> > > > > > processors
>> > > > > > >> > like
>> > > > > > >> > > > > normal
>> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use traditional
>> > > > deployment
>> > > > > > >> scripts
>> > > > > > >> > > > such
>> > > > > > >> > > > > as
>> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
>> > > > > > >> > > > > >> > >>>>>> Chef,
>> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
>> system
>> > > on
>> > > > > > users
>> > > > > > >> > makes
>> > > > > > >> > > > the
>> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really painful
>> for
>> > > > first
>> > > > > > time
>> > > > > > >> > > > users.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement
>> was
>> > > also
>> > > > a
>> > > > > > bit
>> > > > > > >> of
>> > > > > > >> > a
>> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
>> > > > > > >> > > > > >> > >>>>>> because
>> > > > > > >> > > > > >> > >>>>>>>>> of
>> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between
>> > the
>> > > > > > nature of
>> > > > > > >> > > batch
>> > > > > > >> > > > > >> jobs
>> > > > > > >> > > > > >> > >>>>>>>>> and
>> > > > > > >> > > > > >> > >>>>>>> stream
>> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
>> > > conscious
>> > > > > > effort
>> > > > > > >> to
>> > > > > > >> > > > favor
>> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > > >> > > > > >> > >>>>>> Hadoop
>> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things,
>> since
>> > it
>> > > > > worked
>> > > > > > >> and
>> > > > > > >> > > was
>> > > > > > >> > > > > well
>> > > > > > >> > > > > >> > >>>>>>> understood.
>> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that
>> batch
>> > > jobs
>> > > > > > have a
>> > > > > > >> > > > definite
>> > > > > > >> > > > > >> > >>>>>> beginning,
>> > > > > > >> > > > > >> > >>>>>>>>> and
>> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't
>> > > > > (usually).
>> > > > > > >> This
>> > > > > > >> > > > leads
>> > > > > > >> > > > > to
>> > > > > > >> > > > > >> > >>>>>>>>> a
>> > > > > > >> > > > > >> > >>>>> much
>> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream
>> > > > > processors.
>> > > > > > >> You
>> > > > > > >> > > > > >> basically
>> > > > > > >> > > > > >> > >>>>>>>>> just
>> > > > > > >> > > > > >> > >>>>>>> need
>> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
>> processor,
>> > and
>> > > > > start
>> > > > > > >> it.
>> > > > > > >> > > The
>> > > > > > >> > > > > way
>> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no
>> > > concept
>> > > > > of
>> > > > > > a
>> > > > > > >> > > cluster
>> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
>> > > > > > >> > > > > >> > >>>>>> add
>> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
>> coupling
>> > > > Samza
>> > > > > > with
>> > > > > > >> a
>> > > > > > >> > > > > >> scheduler
>> > > > > > >> > > > > >> > >>>>>>>>> is
>> > > > > > >> > > > > >> > >>>>>> that
>> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
>> handle
>> > > > > > deployment.
>> > > > > > >> > > This
>> > > > > > >> > > > > >> pulls
>> > > > > > >> > > > > >> > >>>>>>>>> in a
>> > > > > > >> > > > > >> > >>>>>>> bunch
>> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
>> > > distribution
>> > > > > > (config
>> > > > > > >> > > > > stream),
>> > > > > > >> > > > > >> > >>>>>>>>> shell
>> > > > > > >> > > > > >> > >>>>>>> scrips
>> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging
>> > (all
>> > > > the
>> > > > > > .tgz
>> > > > > > >> > > > stuff),
>> > > > > > >> > > > > >> etc.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
>> > > > deployment
>> > > > > > was
>> > > > > > >> to
>> > > > > > >> > > > > support
>> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
>> > > locality,
>> > > > > you
>> > > > > > >> need
>> > > > > > >> > to
>> > > > > > >> > > > put
>> > > > > > >> > > > > >> > >>>>>>>>> your
>> > > > > > >> > > > > >> > >>>>>> processors
>> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're processing.
>> > Upon
>> > > > > > further
>> > > > > > >> > > > > >> > >>>>>>>>> investigation,
>> > > > > > >> > > > > >> > >>>>>>> though,
>> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that beneficial.
>> > There
>> > > is
>> > > > > > some
>> > > > > > >> > good
>> > > > > > >> > > > > >> > >>>>>>>>> discussion
>> > > > > > >> > > > > >> > >>>>>> about
>> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335.
>> > Again,
>> > > we
>> > > > > > took
>> > > > > > >> the
>> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
>> > > > > > >> > > > > >> > >>>>>> path,
>> > > > > > >> > > > > >> > >>>>>>>>> but
>> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental differences
>> > > > between
>> > > > > > HDFS
>> > > > > > >> > and
>> > > > > > >> > > > > Kafka.
>> > > > > > >> > > > > >> > >>>>>>>>> HDFS
>> > > > > > >> > > > > >> > >>>>>> has
>> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions.
>> This
>> > > > leads
>> > > > > to
>> > > > > > >> less
>> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
>> > > processors
>> > > > > on
>> > > > > > top
>> > > > > > >> > of
>> > > > > > >> > > > > Kafka.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a crutch.
>> > > Samza
>> > > > > > doesn't
>> > > > > > >> > > have
>> > > > > > >> > > > > any
>> > > > > > >> > > > > >> > >>>>>>>>> built
>> > > > > > >> > > > > >> > >>>>> in
>> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
>> > depends
>> > > on
>> > > > > the
>> > > > > > >> > > dynamic
>> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to handle
>> > > > restarts
>> > > > > > >> when a
>> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
>> > > > > > >> > > > > >> > >>>>>>> made
>> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
>> standalone
>> > > Samza
>> > > > > > >> > container
>> > > > > > >> > > > > >> > >>>> (SAMZA-516).
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is good,
>> but I
>> > > > think
>> > > > > > that
>> > > > > > >> > > we've
>> > > > > > >> > > > > >> gone
>> > > > > > >> > > > > >> > >>>>>>>>> too
>> > > > > > >> > > > > >> > >>>>>> far
>> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
>> > > > (SystemConsumer,
>> > > > > > >> > > > > SystemProducer,
>> > > > > > >> > > > > >> > >>>> etc).
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
>> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about
>> > every
>> > > > > > >> component
>> > > > > > >> > > > > >> > >>>>> (MessageChooser,
>> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
>> > > ConfigRewriter,
>> > > > > > etc).
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
>> > forgotten,
>> > > as
>> > > > > > well.
>> > > > > > >> > Some
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> > >>>>>>>>> these
>> > > > > > >> > > > > >> > >>>>> are
>> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not to
>> be.
>> > > This
>> > > > > all
>> > > > > > >> comes
>> > > > > > >> > > at
>> > > > > > >> > > > a
>> > > > > > >> > > > > >> cost:
>> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is making
>> it
>> > > > harder
>> > > > > > for
>> > > > > > >> > our
>> > > > > > >> > > > > users
>> > > > > > >> > > > > >> > >>>>>>>>> to
>> > > > > > >> > > > > >> > >>>>> pick
>> > > > > > >> > > > > >> > >>>>>> up
>> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also
>> > makes
>> > > > it
>> > > > > > >> > difficult
>> > > > > > >> > > > for
>> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about what
>> the
>> > > > > > >> > > characteristics
>> > > > > > >> > > > of
>> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
>> characteristics
>> > > > change
>> > > > > > >> > > depending
>> > > > > > >> > > > on
>> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are most
>> > > visible
>> > > > > in
>> > > > > > the
>> > > > > > >> > > > System
>> > > > > > >> > > > > >> APIs.
>> > > > > > >> > > > > >> > >>>>> What
>> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be
>> functional is
>> > > > Kafka
>> > > > > > as
>> > > > > > >> its
>> > > > > > >> > > > > >> > >>>>>>>>> transport
>> > > > > > >> > > > > >> > >>>>>> layer.
>> > > > > > >> > > > > >> > >>>>>>>>> But
>> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use
>> cases
>> > > into
>> > > > > one
>> > > > > > >> API:
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
>> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports both
>> of
>> > > these
>> > > > > use
>> > > > > > >> > cases.
>> > > > > > >> > > > The
>> > > > > > >> > > > > >> > >>>>>>>>> problem
>> > > > > > >> > > > > >> > >>>>>> is,
>> > > > > > >> > > > > >> > >>>>>>>>> we
>> > > > > > >> > > > > >> > >>>>>>>>> actually want different features for
>> each
>> > > use
>> > > > > > case.
>> > > > > > >> By
>> > > > > > >> > > > > >> papering
>> > > > > > >> > > > > >> > >>>>>>>>> over
>> > > > > > >> > > > > >> > >>>>>>> these
>> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a single
>> > API,
>> > > > > we've
>> > > > > > >> > > > introduced
>> > > > > > >> > > > > a
>> > > > > > >> > > > > >> > >>>>>>>>> ton of
>> > > > > > >> > > > > >> > >>>>>>> leaky
>> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like in
>> (2)
>> > > is
>> > > > to
>> > > > > > have
>> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
>> > offsets
>> > > > > (like
>> > > > > > >> > Kafka).
>> > > > > > >> > > > > This
>> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
>> > > > > > >> > > > > >> > >>>>> with
>> > > > > > >> > > > > >> > >>>>>>> (1),
>> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems have
>> > > > different
>> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
>> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
>> mailing
>> > > list
>> > > > > and
>> > > > > > >> the
>> > > > > > >> > > SQL
>> > > > > > >> > > > > >> JIRAs
>> > > > > > >> > > > > >> > >>>>> about
>> > > > > > >> > > > > >> > >>>>>>> the
>> > > > > > >> > > > > >> > >>>>>>>>> need for this.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
>> > > replayability.
>> > > > > > Kafka
>> > > > > > >> > > allows
>> > > > > > >> > > > us
>> > > > > > >> > > > > >> to
>> > > > > > >> > > > > >> > >>>>> rewind
>> > > > > > >> > > > > >> > >>>>>>>>> when
>> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other systems
>> > > don't.
>> > > > In
>> > > > > > some
>> > > > > > >> > > > cases,
>> > > > > > >> > > > > >> > >>>>>>>>> systems
>> > > > > > >> > > > > >> > >>>>>>> return
>> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
>> > > > > > >> WikipediaSystemConsumer)
>> > > > > > >> > > > > because
>> > > > > > >> > > > > >> > >>>>>>>>> they
>> > > > > > >> > > > > >> > >>>>>> have
>> > > > > > >> > > > > >> > >>>>>>> no
>> > > > > > >> > > > > >> > >>>>>>>>> offsets.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka
>> > > > supports
>> > > > > > >> > > > > partitioning,
>> > > > > > >> > > > > >> > >>>>>>>>> but
>> > > > > > >> > > > > >> > >>>>> many
>> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
>> having a
>> > > > single
>> > > > > > >> > > partition
>> > > > > > >> > > > > for
>> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other systems
>> model
>> > > > > > >> partitioning
>> > > > > > >> > > > > >> > >>>> differently (e.g.
>> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a
>> mess.
>> > > > > > Creating
>> > > > > > >> > > streams
>> > > > > > >> > > > > in
>> > > > > > >> > > > > >> a
>> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
>> impossible.
>> > > As
>> > > > is
>> > > > > > >> > modeling
>> > > > > > >> > > > > >> > >>>>>>>>> metadata
>> > > > > > >> > > > > >> > >>>>> for
>> > > > > > >> > > > > >> > >>>>>>> the
>> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
>> partitions,
>> > > > > location,
>> > > > > > >> > etc).
>> > > > > > >> > > > The
>> > > > > > >> > > > > >> > >>>>>>>>> list
>> > > > > > >> > > > > >> > >>>>> goes
>> > > > > > >> > > > > >> > >>>>>>> on.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing
>> Samza,
>> > > > > Kafka's
>> > > > > > >> > > consumer
>> > > > > > >> > > > > and
>> > > > > > >> > > > > >> > >>>>> producer
>> > > > > > >> > > > > >> > >>>>>>>>> APIs
>> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On
>> the
>> > > > > > >> > consumer-side,
>> > > > > > >> > > > you
>> > > > > > >> > > > > >> > >>>>>>>>> had two
>> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level consumer,
>> or
>> > > the
>> > > > > > simple
>> > > > > > >> > > > > consumer.
>> > > > > > >> > > > > >> > >>>>>>>>> The
>> > > > > > >> > > > > >> > >>>>>>> problem
>> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was that
>> it
>> > > > > > controlled
>> > > > > > >> > your
>> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and
>> the
>> > > order
>> > > > > in
>> > > > > > >> which
>> > > > > > >> > > you
>> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
>> > > > > > >> > > > > >> > >>>>> problem
>> > > > > > >> > > > > >> > >>>>>>>>> with
>> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not
>> > > simple.
>> > > > > It's
>> > > > > > >> > basic.
>> > > > > > >> > > > You
>> > > > > > >> > > > > >> > >>>>>>>>> end up
>> > > > > > >> > > > > >> > >>>>>>> having
>> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level
>> stuff
>> > > > that
>> > > > > > you
>> > > > > > >> > > > > shouldn't.
>> > > > > > >> > > > > >> > >>>>>>>>> We
>> > > > > > >> > > > > >> > >>>>>> spent a
>> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
>> > > > KafkaSystemConsumer
>> > > > > > very
>> > > > > > >> > > > robust.
>> > > > > > >> > > > > >> It
>> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some cool
>> > > features:
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
>> > > > > > prioritization.
>> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
>> assignment
>> > > to
>> > > > > > support
>> > > > > > >> > > > joins,
>> > > > > > >> > > > > >> > >>>>>>>>> global
>> > > > > > >> > > > > >> > >>>>>> state
>> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
>> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
>> > checkpointing.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time is
>> > that
>> > > > > these
>> > > > > > >> > > features
>> > > > > > >> > > > > >> > >>>>>>>>> should
>> > > > > > >> > > > > >> > >>>>>>> actually
>> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers
>> > (not
>> > > > just
>> > > > > > >> Samza
>> > > > > > >> > > > stream
>> > > > > > >> > > > > >> > >>>>>> processors)
>> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like joins
>> > and
>> > > > > > partition
>> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
>> > > > > > >> > > > > >> > >>>>>>> Kafka
>> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
>> > conclusion.
>> > > > > > They're
>> > > > > > >> > > adding
>> > > > > > >> > > > a
>> > > > > > >> > > > > >> ton
>> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
>> consumer
>> > > > > > >> > > implementation.
>> > > > > > >> > > > > To a
>> > > > > > >> > > > > >> > >>>>>>>>> large extent,
>> > > > > > >> > > > > >> > >>>>> it's
>> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've already
>> done
>> > > in
>> > > > > > Samza.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking
>> a
>> > > very
>> > > > > > similar
>> > > > > > >> > > > > approach
>> > > > > > >> > > > > >> > >>>>>>>>> to
>> > > > > > >> > > > > >> > >>>>>> Samza's
>> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation
>> for
>> > > > > > handling
>> > > > > > >> > > offset
>> > > > > > >> > > > > >> > >>>>>> checkpointing.
>> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
>> management
>> > > > > feature
>> > > > > > >> > stores
>> > > > > > >> > > > > >> offset
>> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows
>> you to
>> > > > fetch
>> > > > > > them
>> > > > > > >> > > from
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> > >>>>>>>>> broker.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste,
>> since
>> > we
>> > > > > could
>> > > > > > >> have
>> > > > > > >> > > > shared
>> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > > >> > > > > >> > >>>>> work
>> > > > > > >> > > > > >> > >>>>>> if
>> > > > > > >> > > > > >> > >>>>>>>>> it
>> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
>> get-go.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Vision
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
>> radical
>> > > > > > proposal.
>> > > > > > >> > Samza
>> > > > > > >> > > > is
>> > > > > > >> > > > > >> > >>>>> relatively
>> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to
>> say
>> > > that
>> > > > > > we're
>> > > > > > >> > > near a
>> > > > > > >> > > > > 1.0
>> > > > > > >> > > > > >> > >>>>>> release.
>> > > > > > >> > > > > >> > >>>>>>>>> I'd
>> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what
>> we've
>> > > > > learned,
>> > > > > > and
>> > > > > > >> > > begin
>> > > > > > >> > > > > >> > >>>>>>>>> thinking
>> > > > > > >> > > > > >> > >>>>>>> about
>> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we
>> change if
>> > > we
>> > > > > were
>> > > > > > >> > > starting
>> > > > > > >> > > > > >> from
>> > > > > > >> > > > > >> > >>>>>> scratch?
>> > > > > > >> > > > > >> > >>>>>>>>> My
>> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only*
>> way
>> > to
>> > > > run
>> > > > > > Samza
>> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
>> > > > > dependences
>> > > > > > on
>> > > > > > >> > > YARN,
>> > > > > > >> > > > > >> Mesos,
>> > > > > > >> > > > > >> > >>>> etc.
>> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support
>> only
>> > > > Kafka
>> > > > > > as
>> > > > > > >> the
>> > > > > > >> > > > > stream
>> > > > > > >> > > > > >> > >>>>>> processing
>> > > > > > >> > > > > >> > >>>>>>>>> layer.
>> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
>> > > > > > >> serialization,
>> > > > > > >> > > and
>> > > > > > >> > > > > >> > >>>>>>>>> config
>> > > > > > >> > > > > >> > >>>>>>> systems,
>> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues that
>> I
>> > > > > outlined
>> > > > > > >> > above.
>> > > > > > >> > > It
>> > > > > > >> > > > > >> > >>>>>>>>> should
>> > > > > > >> > > > > >> > >>>>> also
>> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
>> > > > dramatically.
>> > > > > > >> > > Supporting
>> > > > > > >> > > > > >> only
>> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow
>> Samza
>> > to
>> > > be
>> > > > > > >> executed
>> > > > > > >> > > on
>> > > > > > >> > > > > YARN
>> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
>> > > > Marathon/Aurora),
>> > > > > or
>> > > > > > >> most
>> > > > > > >> > > > other
>> > > > > > >> > > > > >> > >>>>>>>>> in-house
>> > > > > > >> > > > > >> > >>>>>>> deployment
>> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a lot
>> > easier
>> > > > for
>> > > > > > new
>> > > > > > >> > > users.
>> > > > > > >> > > > > >> > >>>>>>>>> Imagine
>> > > > > > >> > > > > >> > >>>>>>> having
>> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN.
>> > The
>> > > > drop
>> > > > > > in
>> > > > > > >> > > mailing
>> > > > > > >> > > > > >> list
>> > > > > > >> > > > > >> > >>>>>> traffic
>> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long
>> overdue to
>> > > me.
>> > > > > The
>> > > > > > >> > > reality
>> > > > > > >> > > > > is,
>> > > > > > >> > > > > >> > >>>>> everyone
>> > > > > > >> > > > > >> > >>>>>>>>> that
>> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
>> Kafka.
>> > We
>> > > > > > basically
>> > > > > > >> > > > require
>> > > > > > >> > > > > >> it
>> > > > > > >> > > > > >> > >>>>>> already
>> > > > > > >> > > > > >> > >>>>>>> in
>> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work. Those
>> > that
>> > > > are
>> > > > > > >> using
>> > > > > > >> > > > other
>> > > > > > >> > > > > >> > >>>>>>>>> systems
>> > > > > > >> > > > > >> > >>>>>> are
>> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into
>> Kafka
>> > > (1),
>> > > > > and
>> > > > > > >> then
>> > > > > > >> > > > they
>> > > > > > >> > > > > do
>> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
>> already
>> > > > > > discussion (
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > >
>> > > > > > >> >
>> > > > > >
>> > >
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> > > > > > >> > > > > >> > >>>>> 767
>> > > > > > >> > > > > >> > >>>>>>>>> )
>> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka
>> > > > extremely
>> > > > > > >> easy.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple with
>> > Kafka,
>> > > > we
>> > > > > > can
>> > > > > > >> > > > leverage
>> > > > > > >> > > > > a
>> > > > > > >> > > > > >> > >>>>>>>>> ton of
>> > > > > > >> > > > > >> > >>>>>>> their
>> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
>> maintain
>> > > our
>> > > > > own
>> > > > > > >> > config,
>> > > > > > >> > > > > >> > >>>>>>>>> metrics,
>> > > > > > >> > > > > >> > >>>>> etc.
>> > > > > > >> > > > > >> > >>>>>>> We
>> > > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries, and
>> > make
>> > > > them
>> > > > > > >> > better.
>> > > > > > >> > > > This
>> > > > > > >> > > > > >> > >>>>>>>>> will
>> > > > > > >> > > > > >> > >>>>> also
>> > > > > > >> > > > > >> > >>>>>>>>> allow us to share the
>> consumer/producer
>> > > APIs,
>> > > > > and
>> > > > > > >> will
>> > > > > > >> > > let
>> > > > > > >> > > > > us
>> > > > > > >> > > > > >> > >>>>> leverage
>> > > > > > >> > > > > >> > >>>>>>>>> their offset management and partition
>> > > > > management,
>> > > > > > >> > rather
>> > > > > > >> > > > > than
>> > > > > > >> > > > > >> > >>>>>>>>> having
>> > > > > > >> > > > > >> > >>>>>> our
>> > > > > > >> > > > > >> > >>>>>>>>> own. All of the coordinator stream
>> code
>> > > would
>> > > > > go
>> > > > > > >> away,
>> > > > > > >> > > as
>> > > > > > >> > > > > >> would
>> > > > > > >> > > > > >> > >>>>>>>>> most
>> > > > > > >> > > > > >> > >>>>>> of
>> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably
>> have
>> > to
>> > > > push
>> > > > > > some
>> > > > > > >> > > > > partition
>> > > > > > >> > > > > >> > >>>>>>> management
>> > > > > > >> > > > > >> > >>>>>>>>> features into the Kafka broker, but
>> > they're
>> > > > > > already
>> > > > > > >> > > moving
>> > > > > > >> > > > > in
>> > > > > > >> > > > > >> > >>>>>>>>> that direction with the new consumer
>> API.
>> > > The
>> > > > > > >> features
>> > > > > > >> > > we
>> > > > > > >> > > > > have
>> > > > > > >> > > > > >> > >>>>>>>>> for
>> > > > > > >> > > > > >> > >>>>>> partition
>> > > > > > >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and
>> > seem
>> > > > > like
>> > > > > > >> they
>> > > > > > >> > > > should
>> > > > > > >> > > > > >> be
>> > > > > > >> > > > > >> > >>>>>>>>> in
>> > > > > > >> > > > > >> > >>>>>> Kafka
>> > > > > > >> > > > > >> > >>>>>>>>> anyway. There will always be some
>> niche
>> > > > usages
>> > > > > > which
>> > > > > > >> > > will
>> > > > > > >> > > > > >> > >>>>>>>>> require
>> > > > > > >> > > > > >> > >>>>>> extra
>> > > > > > >> > > > > >> > >>>>>>>>> care and hence full control over
>> > partition
>> > > > > > >> assignments
>> > > > > > >> > > > much
>> > > > > > >> > > > > >> > >>>>>>>>> like the
>> > > > > > >> > > > > >> > >>>>>>> Kafka
>> > > > > > >> > > > > >> > >>>>>>>>> low level consumer api. These would
>> > > continue
>> > > > to
>> > > > > > be
>> > > > > > >> > > > > supported.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> These items will be good for the Samza
>> > > > > community.
>> > > > > > >> > > They'll
>> > > > > > >> > > > > make
>> > > > > > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it
>> easier
>> > for
>> > > > > > >> developers
>> > > > > > >> > > to
>> > > > > > >> > > > > add
>> > > > > > >> > > > > >> > >>>>>>>>> new features.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and
>> > > > somewhat
>> > > > > > >> > backwards
>> > > > > > >> > > > > >> > >>>>> incompatible
>> > > > > > >> > > > > >> > >>>>>>>>> change). If we choose to go this
>> route,
>> > > it's
>> > > > > > >> important
>> > > > > > >> > > > that
>> > > > > > >> > > > > we
>> > > > > > >> > > > > >> > >>>>> openly
>> > > > > > >> > > > > >> > >>>>>>>>> communicate how we're going to
>> provide a
>> > > > > > migration
>> > > > > > >> > path
>> > > > > > >> > > > from
>> > > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > > >> > > > > >> > >>>>>>> existing
>> > > > > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make
>> > > incompatible
>> > > > > > >> > changes).
>> > > > > > >> > > I
>> > > > > > >> > > > > >> think
>> > > > > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to
>> > > provide a
>> > > > > > >> wrapper
>> > > > > > >> > to
>> > > > > > >> > > > > allow
>> > > > > > >> > > > > >> > >>>>>>>>> existing StreamTask implementations to
>> > > > continue
>> > > > > > >> > running
>> > > > > > >> > > on
>> > > > > > >> > > > > the
>> > > > > > >> > > > > >> > >>>> new container.
>> > > > > > >> > > > > >> > >>>>>>> It's
>> > > > > > >> > > > > >> > >>>>>>>>> also important that we openly
>> communicate
>> > > > about
>> > > > > > >> > timing,
>> > > > > > >> > > > and
>> > > > > > >> > > > > >> > >>>>>>>>> stages
>> > > > > > >> > > > > >> > >>>>> of
>> > > > > > >> > > > > >> > >>>>>>> the
>> > > > > > >> > > > > >> > >>>>>>>>> migration.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you
>> > have
>> > > > > > opinions.
>> > > > > > >> > :)
>> > > > > > >> > > > > Please
>> > > > > > >> > > > > >> > >>>>>>>>> send
>> > > > > > >> > > > > >> > >>>>>> your
>> > > > > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>> Cheers,
>> > > > > > >> > > > > >> > >>>>>>>>> Chris
>> > > > > > >> > > > > >> > >>>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>>
>> > > > > > >> > > > > >> > >>>>>>>
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>> --
>> > > > > > >> > > > > >> > >>>>>> -- Guozhang
>> > > > > > >> > > > > >> > >>>>>>
>> > > > > > >> > > > > >> > >>>>>
>> > > > > > >> > > > > >> > >>>>
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> > >>
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >> >
>> > > > > > >> > > > > >>
>> > > > > > >> > > > > >
>> > > > > > >> > > > > >
>> > > > > > >> > > > >
>> > > > > > >> > > >
>> > > > > > >> > >
>> > > > > > >> >
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
Hi, Chris,

Thanks for sending out this concrete set of points here. I agree w/ all but
have a slight different point view on 8).

My view on this is: instead of sunset Samza as TLP, can we re-charter the
scope of Samza to be the home for "running streaming process as a service"?

My main motivation is from the following points from a long internal
discussion in LinkedIn:

- There is a clear ask for pluggable partition management, like we do in
LinkedIn, and as Ben Kirwin has mentioned in
http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCACuX-D-yJX++2GnF_1Laf10KYUVyamg7UP_Dt19v0ZNmmhbCPQ@mail.gmail.com%3E
- There are concerns on lack of support for running stream processing in a
cluster: lifecycle management, resource allocation, fault tolerance, etc.
- There is a question to how to support more advanced features s.t.
host-affinity, auto-scaling, and dynamic configuration in Samza jobs, as
raised by Martin here:
http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3C0D66EFD0-B7CD-4E4E-8B2F-2716167C313C@kleppmann.com%3E

We have use cases that need to address all the above three cases and most
of the functions are all in the current Samza project, in some flavor. We
are all supporting to merge the samza-core functionalities into Kafka
Streams, but there is a question where we keep these functions in the
future. One option is to start a new project that includes these functions
that are closely related w/ "run stream-processing as-a-service", while
another personally more attractive option is to re-charter Samza project
just do "run stream processing as-a-service". We can avoid the overhead of
re-starting another community for this project. Personally, I felt that
here are the benefits we should be getting:

1. We have already agreed mostly that Kafka Streams API would allow some
pluggable partition management functions. Hence, the advanced partition
management can live out-side the new Kafka Streams core w/o affecting the
run-as-a-library model in Kafka Streams.
2. The integration w/ cluster management system and advanced features
listed above stays in the same project and allow existing users enjoy
no-impact migration to Kafka Stream as the core. That also addresses Tim's
question on "removing the support for YARN".
3. A separate project for stream-processing-as-a-service also allow the new
Kafka Streams being independent to any cluster management and just focusing
on stream process core functions, while leaving the functions that requires
cluster-resource and state management to a separate layer.

Please feel free to comment. Thanks!

On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <cr...@apache.org>
wrote:

> Hey all,
>
> I want to start by saying that I'm absolutely thrilled to be a part of this
> community. The amount of level-headed, thoughtful, educated discussion
> that's gone on over the past ~10 days is overwhelming. Wonderful.
>
> It seems like discussion is waning a bit, and we've reached some
> conclusions. There are several key emails in this threat, which I want to
> call out:
>
> 1. Jakob's summary of the three potential ways forward.
>
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> 2. Julian's call out that we should be focusing on community over code.
>
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> 3. Martin's summary about the benefits of merging communities.
>
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> 4. Jakob's comments about the distinction between community and code paths.
>
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
>
> I agree with the comments on all of these emails. I think Martin's summary
> of his position aligns very closely with my own. To that end, I think we
> should get concrete about what the proposal is, and call a vote on it.
> Given that Jay, Martin, and I seem to be aligning fairly closely, I think
> we should start with:
>
> 1. [community] Make Samza a subproject of Kafka.
> 2. [community] Make all Samza PMC/committers committers of the subproject.
> 3. [community] Migrate Samza's website/documentation into Kafka's.
> 4. [code] Have the Samza community and the Kafka community start a
> from-scratch reboot together in the new Kafka subproject. We can
> borrow/copy &  paste significant chunks of code from Samza's code base.
> 5. [code] The subproject would intentionally eliminate support for both
> other streaming systems and all deployment systems.
> 6. [code] Attempt to provide a bridge from our SystemConsumer to KIP-26
> (copy cat)
> 7. [code] Attempt to provide a bridge from the new subproject's processor
> interface to our legacy StreamTask interface.
> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> subproject that has a fault-tolerant container with state management.
>
> It's likely that (6) and (7) won't be fully drop-in. Still, the closer we
> can get, the better it's going to be for our existing community.
>
> One thing that I didn't touch on with (2) is whether any Samza PMC members
> should be rolled into Kafka PMC membership as well (though, Jay and Jakob
> are already PMC members on both). I think that Samza's community deserves a
> voice on the PMC, so I'd propose that we roll at least a few PMC members
> into the Kafka PMC, but I don't have a strong framework for which people to
> pick.
>
> Before (8), I think that Samza's TLP can continue to commit bug fixes and
> patches as it sees fit, provided that we openly communicate that we won't
> necessarily migrate new features to the new subproject, and that the TLP
> will be shut down after the migration to the Kafka subproject occurs.
>
> Jakob, I could use your guidance here about about how to achieve this from
> an Apache process perspective (sorry).
>
> * Should I just call a vote on this proposal?
> * Should it happen on dev or private?
> * Do committers have binding votes, or just PMC?
>
> Having trouble finding much detail on the Apache wikis. :(
>
> Cheers,
> Chris
>
> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com> wrote:
>
> > Thanks, Jay. This argument persuaded me actually. :)
> >
> > Fang, Yan
> > yanfang724@gmail.com
> >
> > On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Hey Yan,
> > >
> > > Yeah philosophically I think the argument is that you should capture
> the
> > > stream in Kafka independent of the transformation. This is obviously a
> > > Kafka-centric view point.
> > >
> > > Advantages of this:
> > > - In practice I think this is what e.g. Storm people often end up doing
> > > anyway. You usually need to throttle any access to a live serving
> > database.
> > > - Can have multiple subscribers and they get the same thing without
> > > additional load on the source system.
> > > - Applications can tap into the stream if need be by subscribing.
> > > - You can debug your transformation by tailing the Kafka topic with the
> > > console consumer
> > > - Can tee off the same data stream for batch analysis or Lambda arch
> > style
> > > re-processing
> > >
> > > The disadvantage is that it will use Kafka resources. But the idea is
> > > eventually you will have multiple subscribers to any data source (at
> > least
> > > for monitoring) so you will end up there soon enough anyway.
> > >
> > > Down the road the technical benefit is that I think it gives us a good
> > path
> > > towards end-to-end exactly once semantics from source to destination.
> > > Basically the connectors need to support idempotence when talking to
> > Kafka
> > > and we need the transactional write feature in Kafka to make the
> > > transformation atomic. This is actually pretty doable if you separate
> > > connector=>kafka problem from the generic transformations which are
> > always
> > > kafka=>kafka. However I think it is quite impossible to do in a
> > all_things
> > > => all_things environment. Today you can say "well the semantics of the
> > > Samza APIs depend on the connectors you use" but it is actually worse
> > then
> > > that because the semantics actually depend on the pairing of
> > connectors--so
> > > not only can you probably not get a usable "exactly once" guarantee
> > > end-to-end it can actually be quite hard to reverse engineer what
> > property
> > > (if any) your end-to-end flow has if you have heterogenous systems.
> > >
> > > -Jay
> > >
> > > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
> wrote:
> > >
> > > > {quote}
> > > > maintained in a separate repository and retaining the existing
> > > > committership but sharing as much else as possible (website, etc)
> > > > {quote}
> > > >
> > > > Overall, I agree on this idea. Now the question is more about "how to
> > do
> > > > it".
> > > >
> > > > On the other hand, one thing I want to point out is that, if we
> decide
> > to
> > > > go this way, how do we want to support
> > > > otherSystem-transformation-otherSystem use case?
> > > >
> > > > Basically, there are four user groups here:
> > > >
> > > > 1. Kafka-transformation-Kafka
> > > > 2. Kafka-transformation-otherSystem
> > > > 3. otherSystem-transformation-Kafka
> > > > 4. otherSystem-transformation-otherSystem
> > > >
> > > > For group 1, they can easily use the new Samza library to achieve.
> For
> > > > group 2 and 3, they can use copyCat -> transformation -> Kafka or
> > Kafka->
> > > > transformation -> copyCat.
> > > >
> > > > The problem is for group 4. Do we want to abandon this or still
> support
> > > it?
> > > > Of course, this use case can be achieved by using copyCat ->
> > > transformation
> > > > -> Kafka -> transformation -> copyCat, the thing is how we persuade
> > them
> > > to
> > > > do this long chain. If yes, it will also be a win for Kafka too. Or
> if
> > > > there is no one in this community actually doing this so far, maybe
> ok
> > to
> > > > not support the group 4 directly.
> > > >
> > > > Thanks,
> > > >
> > > > Fang, Yan
> > > > yanfang724@gmail.com
> > > >
> > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > > >
> > > > > Yeah I agree with this summary. I think there are kind of two
> > questions
> > > > > here:
> > > > > 1. Technically does alignment/reliance on Kafka make sense
> > > > > 2. Branding wise (naming, website, concepts, etc) does alignment
> with
> > > > Kafka
> > > > > make sense
> > > > >
> > > > > Personally I do think both of these things would be really
> valuable,
> > > and
> > > > > would dramatically alter the trajectory of the project.
> > > > >
> > > > > My preference would be to see if people can mostly agree on a
> > direction
> > > > > rather than splintering things off. From my point of view the ideal
> > > > outcome
> > > > > of all the options discussed would be to make Samza a closely
> aligned
> > > > > subproject, maintained in a separate repository and retaining the
> > > > existing
> > > > > committership but sharing as much else as possible (website, etc).
> No
> > > > idea
> > > > > about how these things work, Jacob, you probably know more.
> > > > >
> > > > > No discussion amongst the Kafka folks has happened on this, but
> > likely
> > > we
> > > > > should figure out what the Samza community actually wants first.
> > > > >
> > > > > I admit that this is a fairly radical departure from how things
> are.
> > > > >
> > > > > If that doesn't fly, I think, yeah we could leave Samza as it is
> and
> > do
> > > > the
> > > > > more radical reboot inside Kafka. From my point of view that does
> > leave
> > > > > things in a somewhat confusing state since now there are two stream
> > > > > processing systems more or less coupled to Kafka in large part made
> > by
> > > > the
> > > > > same people. But, arguably that might be a cleaner way to make the
> > > > cut-over
> > > > > and perhaps less risky for Samza community since if it works people
> > can
> > > > > switch and if it doesn't nothing will have changed. Dunno, how do
> > > people
> > > > > feel about this?
> > > > >
> > > > > -Jay
> > > > >
> > > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com>
> > > wrote:
> > > > >
> > > > > > >  This leads me to thinking that merging projects and
> communities
> > > > might
> > > > > > be a good idea: with the union of experience from both
> communities,
> > > we
> > > > > will
> > > > > > probably build a better system that is better for users.
> > > > > > Is this what's being proposed though? Merging the projects seems
> > like
> > > > > > a consequence of at most one of the three directions under
> > > discussion:
> > > > > > 1) Samza 2.0: The Samza community relies more heavily on Kafka
> for
> > > > > > configuration, etc. (to a greater or lesser extent to be
> > determined)
> > > > > > but the Samza community would not automatically merge withe Kafka
> > > > > > community (the Phoenix/HBase example is a good one here).
> > > > > > 2) Samza Reboot: The Samza community continues to exist with a
> > > limited
> > > > > > project scope, but similarly would not need to be part of the
> Kafka
> > > > > > community (ie given committership) to progress.  Here, maybe the
> > > Samza
> > > > > > team would become a subproject of Kafka (the Board frowns on
> > > > > > subprojects at the moment, so I'm not sure if that's even
> > feasible),
> > > > > > but that would not be required.
> > > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option the
> > Kafka
> > > > > > team builds its own streaming library, possibly off of Jay's
> > > > > > prototype, which has not direct lineage to the Samza team.
> There's
> > > no
> > > > > > reason for the Kafka team to bring in the Samza team.
> > > > > >
> > > > > > Is the Kafka community on board with this?
> > > > > >
> > > > > > To be clear, all three options under discussion are interesting,
> > > > > > technically valid and likely healthy directions for the project.
> > > > > > Also, they are not mutually exclusive.  The Samza community could
> > > > > > decide to pursue, say, 'Samza 2.0', while the Kafka community
> went
> > > > > > forward with 'Hey Samza!'  My points above are directed entirely
> at
> > > > > > the community aspect of these choices.
> > > > > > -Jakob
> > > > > >
> > > > > > On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com>
> > > wrote:
> > > > > > > That's great.  Thanks, Jay.
> > > > > > >
> > > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io>
> > > wrote:
> > > > > > >
> > > > > > >> Yeah totally agree. I think you have this issue even today,
> > right?
> > > > > I.e.
> > > > > > if
> > > > > > >> you need to make a simple config change and you're running in
> > YARN
> > > > > today
> > > > > > >> you end up bouncing the job which then rebuilds state. I think
> > the
> > > > fix
> > > > > > is
> > > > > > >> exactly what you described which is to have a long timeout on
> > > > > partition
> > > > > > >> movement for stateful jobs so that if a job is just getting
> > > bounced,
> > > > > and
> > > > > > >> the cluster manager (or admin) is smart enough to restart it
> on
> > > the
> > > > > same
> > > > > > >> host when possible, it can optimistically reuse any existing
> > state
> > > > it
> > > > > > finds
> > > > > > >> on disk (if it is valid).
> > > > > > >>
> > > > > > >> So in this model the charter of the CM is to place processes
> as
> > > > > > stickily as
> > > > > > >> possible and to restart or re-place failed processes. The
> > charter
> > > of
> > > > > the
> > > > > > >> partition management system is to control the assignment of
> work
> > > to
> > > > > > these
> > > > > > >> processes. The nice thing about this is that the work
> > assignment,
> > > > > > timeouts,
> > > > > > >> behavior, configs, and code will all be the same across all
> > > cluster
> > > > > > >> managers.
> > > > > > >>
> > > > > > >> So I think that prototype would actually give you exactly what
> > you
> > > > > want
> > > > > > >> today for any cluster manager (or manual placement + restart
> > > script)
> > > > > > that
> > > > > > >> was sticky in terms of host placement since there is already a
> > > > > > configurable
> > > > > > >> partition movement timeout and task-by-task state reuse with a
> > > check
> > > > > on
> > > > > > >> state validity.
> > > > > > >>
> > > > > > >> -Jay
> > > > > > >>
> > > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > > > roger.hoover@gmail.com
> > > > > >
> > > > > > >> wrote:
> > > > > > >>
> > > > > > >> > That would be great to let Kafka do as much heavy lifting as
> > > > > possible
> > > > > > and
> > > > > > >> > make it easier for other languages to implement Samza apis.
> > > > > > >> >
> > > > > > >> > One thing to watch out for is the interplay between Kafka's
> > > group
> > > > > > >> > management and the external scheduler/process manager's
> fault
> > > > > > tolerance.
> > > > > > >> > If a container dies, the Kafka group membership protocol
> will
> > > try
> > > > to
> > > > > > >> assign
> > > > > > >> > it's tasks to other containers while at the same time the
> > > process
> > > > > > manager
> > > > > > >> > is trying to relaunch the container.  Without some
> > consideration
> > > > for
> > > > > > this
> > > > > > >> > (like a configurable amount of time to wait before Kafka
> > alters
> > > > the
> > > > > > group
> > > > > > >> > membership), there may be thrashing going on which is
> > especially
> > > > bad
> > > > > > for
> > > > > > >> > containers with large amounts of local state.
> > > > > > >> >
> > > > > > >> > Someone else pointed this out already but I thought it might
> > be
> > > > > worth
> > > > > > >> > calling out again.
> > > > > > >> >
> > > > > > >> > Cheers,
> > > > > > >> >
> > > > > > >> > Roger
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> jay@confluent.io>
> > > > > wrote:
> > > > > > >> >
> > > > > > >> > > Hey Roger,
> > > > > > >> > >
> > > > > > >> > > I couldn't agree more. We spent a bunch of time talking to
> > > > people
> > > > > > and
> > > > > > >> > that
> > > > > > >> > > is exactly the stuff we heard time and again. What makes
> it
> > > > hard,
> > > > > of
> > > > > > >> > > course, is that there is some tension between
> compatibility
> > > with
> > > > > > what's
> > > > > > >> > > there now and making things better for new users.
> > > > > > >> > >
> > > > > > >> > > I also strongly agree with the importance of
> multi-language
> > > > > > support. We
> > > > > > >> > are
> > > > > > >> > > talking now about Java, but for application development
> use
> > > > cases
> > > > > > >> people
> > > > > > >> > > want to work in whatever language they are using
> elsewhere.
> > I
> > > > > think
> > > > > > >> > moving
> > > > > > >> > > to a model where Kafka itself does the group membership,
> > > > lifecycle
> > > > > > >> > control,
> > > > > > >> > > and partition assignment has the advantage of putting all
> > that
> > > > > > complex
> > > > > > >> > > stuff behind a clean api that the clients are already
> going
> > to
> > > > be
> > > > > > >> > > implementing for their consumer, so the added
> functionality
> > > for
> > > > > > stream
> > > > > > >> > > processing beyond a consumer becomes very minor.
> > > > > > >> > >
> > > > > > >> > > -Jay
> > > > > > >> > >
> > > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > > > > > roger.hoover@gmail.com>
> > > > > > >> > > wrote:
> > > > > > >> > >
> > > > > > >> > > > Metamorphosis...nice. :)
> > > > > > >> > > >
> > > > > > >> > > > This has been a great discussion.  As a user of Samza
> > who's
> > > > > > recently
> > > > > > >> > > > integrated it into a relatively large organization, I
> just
> > > > want
> > > > > to
> > > > > > >> add
> > > > > > >> > > > support to a few points already made.
> > > > > > >> > > >
> > > > > > >> > > > The biggest hurdles to adoption of Samza as it currently
> > > > exists
> > > > > > that
> > > > > > >> > I've
> > > > > > >> > > > experienced are:
> > > > > > >> > > > 1) YARN - YARN is overly complex in many environments
> > where
> > > > > Puppet
> > > > > > >> > would
> > > > > > >> > > do
> > > > > > >> > > > just fine but it was the only mechanism to get fault
> > > > tolerance.
> > > > > > >> > > > 2) Configuration - I think I like the idea of
> configuring
> > > most
> > > > > of
> > > > > > the
> > > > > > >> > job
> > > > > > >> > > > in code rather than config files.  In general, I think
> the
> > > > goal
> > > > > > >> should
> > > > > > >> > be
> > > > > > >> > > > to make it harder to make mistakes, especially of the
> kind
> > > > where
> > > > > > the
> > > > > > >> > code
> > > > > > >> > > > expects something and the config doesn't match.  The
> > current
> > > > > > config
> > > > > > >> is
> > > > > > >> > > > quite intricate and error-prone.  For example, the
> > > application
> > > > > > logic
> > > > > > >> > may
> > > > > > >> > > > depend on bootstrapping a topic but rather than
> asserting
> > > that
> > > > > in
> > > > > > the
> > > > > > >> > > code,
> > > > > > >> > > > you have to rely on getting the config right.  Likewise
> > with
> > > > > > serdes,
> > > > > > >> > the
> > > > > > >> > > > Java representations produced by various serdes (JSON,
> > Avro,
> > > > > etc.)
> > > > > > >> are
> > > > > > >> > > not
> > > > > > >> > > > equivalent so you cannot just reconfigure a serde
> without
> > > > > changing
> > > > > > >> the
> > > > > > >> > > > code.   It would be nice for jobs to be able to assert
> > what
> > > > they
> > > > > > >> expect
> > > > > > >> > > > from their input topics in terms of partitioning.  This
> is
> > > > > > getting a
> > > > > > >> > > little
> > > > > > >> > > > off topic but I was even thinking about creating a
> "Samza
> > > > config
> > > > > > >> > linter"
> > > > > > >> > > > that would sanity check a set of configs.  Especially in
> > > > > > >> organizations
> > > > > > >> > > > where config is managed by a different team than the
> > > > application
> > > > > > >> > > developer,
> > > > > > >> > > > it's very hard to get avoid config mistakes.
> > > > > > >> > > > 3) Java/Scala centric - for many teams (especially
> > > DevOps-type
> > > > > > >> folks),
> > > > > > >> > > the
> > > > > > >> > > > pain of the Java toolchain (maven, slow builds, weak
> > command
> > > > > line
> > > > > > >> > > support,
> > > > > > >> > > > configuration over convention) really inhibits
> > productivity.
> > > > As
> > > > > > more
> > > > > > >> > and
> > > > > > >> > > > more high-quality clients become available for Kafka, I
> > hope
> > > > > > they'll
> > > > > > >> > > follow
> > > > > > >> > > > Samza's model.  Not sure how much it affects the
> proposals
> > > in
> > > > > this
> > > > > > >> > thread
> > > > > > >> > > > but please consider other languages in the ecosystem as
> > > well.
> > > > > > From
> > > > > > >> > what
> > > > > > >> > > > I've heard, Spark has more Python users than Java/Scala.
> > > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > > > > >> > > > and are working on a Yeoman generator
> > > > > > >> > > > https://github.com/Quantiply/generator-rico for
> > > Jython/Samza
> > > > > > >> projects
> > > > > > >> > to
> > > > > > >> > > > alleviate some of the pain)
> > > > > > >> > > >
> > > > > > >> > > > I also want to underscore Jay's point about improving
> the
> > > user
> > > > > > >> > > experience.
> > > > > > >> > > > That's a very important factor for adoption.  I think
> the
> > > goal
> > > > > > should
> > > > > > >> > be
> > > > > > >> > > to
> > > > > > >> > > > make Samza as easy to get started with as something like
> > > > > Logstash.
> > > > > > >> > > > Logstash is vastly inferior in terms of capabilities to
> > > Samza
> > > > > but
> > > > > > >> it's
> > > > > > >> > > easy
> > > > > > >> > > > to get started and that makes a big difference.
> > > > > > >> > > >
> > > > > > >> > > > Cheers,
> > > > > > >> > > >
> > > > > > >> > > > Roger
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci
> > > > Morales <
> > > > > > >> > > > gdfm@apache.org> wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> Metamorphosis
> > > is
> > > > a
> > > > > > clear
> > > > > > >> > > > winner
> > > > > > >> > > > > :)
> > > > > > >> > > > >
> > > > > > >> > > > > --
> > > > > > >> > > > > Gianmarco
> > > > > > >> > > > >
> > > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> Morales
> > <
> > > > > > >> > > gdfm@apache.org
> > > > > > >> > > > >
> > > > > > >> > > > > wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > Hi,
> > > > > > >> > > > > >
> > > > > > >> > > > > > @Martin, thanks for you comments.
> > > > > > >> > > > > > Maybe I'm missing some important point, but I think
> > > > coupling
> > > > > > the
> > > > > > >> > > > releases
> > > > > > >> > > > > > is actually a *good* thing.
> > > > > > >> > > > > > To make an example, would it be better if the MR and
> > > HDFS
> > > > > > >> > components
> > > > > > >> > > of
> > > > > > >> > > > > > Hadoop had different release schedules?
> > > > > > >> > > > > >
> > > > > > >> > > > > > Actually, keeping the discussion in a single place
> > would
> > > > > make
> > > > > > >> > > agreeing
> > > > > > >> > > > on
> > > > > > >> > > > > > releases (and backwards compatibility) much easier,
> as
> > > > > > everybody
> > > > > > >> > > would
> > > > > > >> > > > be
> > > > > > >> > > > > > responsible for the whole codebase.
> > > > > > >> > > > > >
> > > > > > >> > > > > > That said, I like the idea of absorbing samza-core
> as
> > a
> > > > > > >> > sub-project,
> > > > > > >> > > > and
> > > > > > >> > > > > > leave the fancy stuff separate.
> > > > > > >> > > > > > It probably gives 90% of the benefits we have been
> > > > > discussing
> > > > > > >> here.
> > > > > > >> > > > > >
> > > > > > >> > > > > > Cheers,
> > > > > > >> > > > > >
> > > > > > >> > > > > > --
> > > > > > >> > > > > > Gianmarco
> > > > > > >> > > > > >
> > > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> > jay.kreps@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >> > > > > >
> > > > > > >> > > > > >> Hey Martin,
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> I agree coupling release schedules is a downside.
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> Definitely we can try to solve some of the
> > integration
> > > > > > problems
> > > > > > >> in
> > > > > > >> > > > > >> Confluent Platform or in other distributions. But I
> > > think
> > > > > > this
> > > > > > >> > ends
> > > > > > >> > > up
> > > > > > >> > > > > >> being really shallow. I guess I feel to really get
> a
> > > good
> > > > > > user
> > > > > > >> > > > > experience
> > > > > > >> > > > > >> the two systems have to kind of feel like part of
> the
> > > > same
> > > > > > thing
> > > > > > >> > and
> > > > > > >> > > > you
> > > > > > >> > > > > >> can't really add that in later--you can put both in
> > the
> > > > > same
> > > > > > >> > > > > downloadable
> > > > > > >> > > > > >> tar file but it doesn't really give a very cohesive
> > > > > feeling.
> > > > > > I
> > > > > > >> > agree
> > > > > > >> > > > > that
> > > > > > >> > > > > >> ultimately any of the project stuff is as much
> social
> > > and
> > > > > > naming
> > > > > > >> > as
> > > > > > >> > > > > >> anything else--theoretically two totally
> independent
> > > > > projects
> > > > > > >> > could
> > > > > > >> > > > work
> > > > > > >> > > > > >> to
> > > > > > >> > > > > >> tightly align. In practice this seems to be quite
> > > > difficult
> > > > > > >> > though.
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> For the frameworks--totally agree it would be good
> to
> > > > > > maintain
> > > > > > >> the
> > > > > > >> > > > > >> framework support with the project. In some cases
> > there
> > > > may
> > > > > > not
> > > > > > >> be
> > > > > > >> > > too
> > > > > > >> > > > > >> much
> > > > > > >> > > > > >> there since the integration gets lighter but I
> think
> > > > > whatever
> > > > > > >> > stubs
> > > > > > >> > > > you
> > > > > > >> > > > > >> need should be included. So no I definitely wasn't
> > > trying
> > > > > to
> > > > > > >> imply
> > > > > > >> > > > > >> dropping
> > > > > > >> > > > > >> support for these frameworks, just making the
> > > integration
> > > > > > >> lighter
> > > > > > >> > by
> > > > > > >> > > > > >> separating process management from partition
> > > management.
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> You raise two good points we would have to figure
> out
> > > if
> > > > we
> > > > > > went
> > > > > > >> > > down
> > > > > > >> > > > > the
> > > > > > >> > > > > >> alignment path:
> > > > > > >> > > > > >> 1. With respect to the name, yeah I think the first
> > > > > question
> > > > > > is
> > > > > > >> > > > whether
> > > > > > >> > > > > >> some "re-branding" would be worth it. If so then I
> > > think
> > > > we
> > > > > > can
> > > > > > >> > > have a
> > > > > > >> > > > > big
> > > > > > >> > > > > >> thread on the name. I'm definitely not set on Kafka
> > > > > > Streaming or
> > > > > > >> > > Kafka
> > > > > > >> > > > > >> Streams I was just using them to be kind of
> > > > illustrative. I
> > > > > > >> agree
> > > > > > >> > > with
> > > > > > >> > > > > >> your
> > > > > > >> > > > > >> critique of these names, though I think people
> would
> > > get
> > > > > the
> > > > > > >> idea.
> > > > > > >> > > > > >> 2. Yeah you also raise a good point about how to
> > > "factor"
> > > > > it.
> > > > > > >> Here
> > > > > > >> > > are
> > > > > > >> > > > > the
> > > > > > >> > > > > >> options I see (I could get enthusiastic about any
> of
> > > > them):
> > > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > > > > > >> > > > > >>    b. Two repos, retaining the current seperation
> > > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api and
> > > > samza-core
> > > > > > is
> > > > > > >> > > > absorbed
> > > > > > >> > > > > >> almost like a third client
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> Cheers,
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> -Jay
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> > > > > > >> > > > martin@kleppmann.com>
> > > > > > >> > > > > >> wrote:
> > > > > > >> > > > > >>
> > > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a few
> > > follow-up
> > > > > > >> > comments.
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
> > becoming
> > > a
> > > > > > >> > subproject:
> > > > > > >> > > > the
> > > > > > >> > > > > >> > reasons you mention are good. The risk I see is
> > that
> > > > > > release
> > > > > > >> > > > schedules
> > > > > > >> > > > > >> > become coupled to each other, which can slow
> > everyone
> > > > > down,
> > > > > > >> and
> > > > > > >> > > > large
> > > > > > >> > > > > >> > projects with many contributors are harder to
> > manage.
> > > > > > (Jakob,
> > > > > > >> > can
> > > > > > >> > > > you
> > > > > > >> > > > > >> speak
> > > > > > >> > > > > >> > from experience, having seen a wider range of
> > Hadoop
> > > > > > ecosystem
> > > > > > >> > > > > >> projects?)
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > Some of the goals of a better unified developer
> > > > > experience
> > > > > > >> could
> > > > > > >> > > > also
> > > > > > >> > > > > be
> > > > > > >> > > > > >> > solved by integrating Samza nicely into a Kafka
> > > > > > distribution
> > > > > > >> > (such
> > > > > > >> > > > as
> > > > > > >> > > > > >> > Confluent's). I'm not against merging projects if
> > we
> > > > > decide
> > > > > > >> > that's
> > > > > > >> > > > the
> > > > > > >> > > > > >> way
> > > > > > >> > > > > >> > to go, just pointing out the same goals can
> perhaps
> > > > also
> > > > > be
> > > > > > >> > > achieved
> > > > > > >> > > > > in
> > > > > > >> > > > > >> > other ways.
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > - With regard to dropping the YARN dependency:
> are
> > > you
> > > > > > >> proposing
> > > > > > >> > > > that
> > > > > > >> > > > > >> > Samza doesn't give any help to people wanting to
> > run
> > > on
> > > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > > > > > >> > > > > >> > So the docs would basically have a link to Slider
> > and
> > > > > > nothing
> > > > > > >> > > else?
> > > > > > >> > > > Or
> > > > > > >> > > > > >> > would we maintain integrations with a bunch of
> > > popular
> > > > > > >> > deployment
> > > > > > >> > > > > >> methods
> > > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts to
> make
> > > > Samza
> > > > > > work
> > > > > > >> > with
> > > > > > >> > > > > >> Slider)?
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > I absolutely think it's a good idea to have the
> > "as a
> > > > > > library"
> > > > > > >> > and
> > > > > > >> > > > > "as a
> > > > > > >> > > > > >> > process" (using Yi's taxonomy) options for people
> > who
> > > > > want
> > > > > > >> them,
> > > > > > >> > > > but I
> > > > > > >> > > > > >> > think there should also be a low-friction path
> for
> > > > common
> > > > > > "as
> > > > > > >> a
> > > > > > >> > > > > service"
> > > > > > >> > > > > >> > deployment methods, for which we probably need to
> > > > > maintain
> > > > > > >> > > > > integrations.
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to
> me,
> > > > > because
> > > > > > >> Kafka
> > > > > > >> > > is
> > > > > > >> > > > > all
> > > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> Transformers"
> > > or
> > > > > > "Kafka
> > > > > > >> > > > Filters"
> > > > > > >> > > > > >> > would be more apt?
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > One suggestion: perhaps the core of Samza (stream
> > > > > > >> transformation
> > > > > > >> > > > with
> > > > > > >> > > > > >> > state management -- i.e. the "Samza as a library"
> > > bit)
> > > > > > could
> > > > > > >> > > become
> > > > > > >> > > > > >> part of
> > > > > > >> > > > > >> > Kafka, while higher-level tools such as streaming
> > SQL
> > > > and
> > > > > > >> > > > integrations
> > > > > > >> > > > > >> with
> > > > > > >> > > > > >> > deployment frameworks remain in a separate
> project?
> > > In
> > > > > > other
> > > > > > >> > > words,
> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > >> > would absorb the proven, stable core of Samza,
> > which
> > > > > would
> > > > > > >> > become
> > > > > > >> > > > the
> > > > > > >> > > > > >> > "third Kafka client" mentioned early in this
> > thread.
> > > > The
> > > > > > Samza
> > > > > > >> > > > project
> > > > > > >> > > > > >> > would then target that third Kafka client as its
> > base
> > > > > API,
> > > > > > and
> > > > > > >> > the
> > > > > > >> > > > > >> project
> > > > > > >> > > > > >> > would be freed up to explore more experimental
> new
> > > > > > horizons.
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > Martin
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > > > jay.kreps@gmail.com>
> > > > > > >> wrote:
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > > Hey Martin,
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually
> > don't
> > > > > think
> > > > > > it
> > > > > > >> > ties
> > > > > > >> > > > our
> > > > > > >> > > > > >> > hands
> > > > > > >> > > > > >> > > at all, all it does is refactor things. The
> > > division
> > > > of
> > > > > > >> > > > > >> responsibility is
> > > > > > >> > > > > >> > > that Samza core is responsible for task
> > lifecycle,
> > > > > state,
> > > > > > >> and
> > > > > > >> > > > > >> partition
> > > > > > >> > > > > >> > > management (using the Kafka co-ordinator) but
> it
> > is
> > > > NOT
> > > > > > >> > > > responsible
> > > > > > >> > > > > >> for
> > > > > > >> > > > > >> > > packaging, configuration deployment or
> execution
> > of
> > > > > > >> processes.
> > > > > > >> > > The
> > > > > > >> > > > > >> > problem
> > > > > > >> > > > > >> > > of packaging and starting these processes is
> > > > > > >> > > > > >> > > framework/environment-specific. This leaves
> > > > individual
> > > > > > >> > > frameworks
> > > > > > >> > > > to
> > > > > > >> > > > > >> be
> > > > > > >> > > > > >> > as
> > > > > > >> > > > > >> > > fancy or vanilla as they like. So you can get
> > > simple
> > > > > > >> stateless
> > > > > > >> > > > > >> support in
> > > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app
> > > > > framework
> > > > > > >> > > (Slider,
> > > > > > >> > > > > >> > Marathon,
> > > > > > >> > > > > >> > > etc). These are well known by people and have
> > nice
> > > > UIs
> > > > > > and a
> > > > > > >> > lot
> > > > > > >> > > > of
> > > > > > >> > > > > >> > > flexibility. I don't think they have node
> > affinity
> > > > as a
> > > > > > >> built
> > > > > > >> > in
> > > > > > >> > > > > >> option
> > > > > > >> > > > > >> > > (though I could be wrong). So if we want that
> we
> > > can
> > > > > > either
> > > > > > >> > wait
> > > > > > >> > > > for
> > > > > > >> > > > > >> them
> > > > > > >> > > > > >> > > to add it or do a custom framework to add that
> > > > feature
> > > > > > (as
> > > > > > >> > now).
> > > > > > >> > > > > >> > Obviously
> > > > > > >> > > > > >> > > if you manage things with old-school ops tools
> > > > > > >> > (puppet/chef/etc)
> > > > > > >> > > > you
> > > > > > >> > > > > >> get
> > > > > > >> > > > > >> > > locality easily. The nice thing, though, is
> that
> > > all
> > > > > the
> > > > > > >> samza
> > > > > > >> > > > > >> "business
> > > > > > >> > > > > >> > > logic" around partition management and fault
> > > > tolerance
> > > > > > is in
> > > > > > >> > > Samza
> > > > > > >> > > > > >> core
> > > > > > >> > > > > >> > so
> > > > > > >> > > > > >> > > it is shared across frameworks and the
> framework
> > > > > specific
> > > > > > >> bit
> > > > > > >> > is
> > > > > > >> > > > > just
> > > > > > >> > > > > >> > > whether it is smart enough to try to get the
> same
> > > > host
> > > > > > when
> > > > > > >> a
> > > > > > >> > > job
> > > > > > >> > > > is
> > > > > > >> > > > > >> > > restarted.
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah I
> think
> > > the
> > > > > > goal
> > > > > > >> > would
> > > > > > >> > > > be
> > > > > > >> > > > > >> (a)
> > > > > > >> > > > > >> > > actually get better alignment in user
> experience,
> > > and
> > > > > (b)
> > > > > > >> > > express
> > > > > > >> > > > > >> this in
> > > > > > >> > > > > >> > > the naming and project branding. Specifically:
> > > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> > > > > > "transformation"
> > > > > > >> api
> > > > > > >> > > to
> > > > > > >> > > > be
> > > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be
> able
> > > to
> > > > > > explain
> > > > > > >> > > when
> > > > > > >> > > > to
> > > > > > >> > > > > >> use
> > > > > > >> > > > > >> > > the consumer and when to use the stream
> > processing
> > > > > > >> > functionality
> > > > > > >> > > > and
> > > > > > >> > > > > >> lead
> > > > > > >> > > > > >> > > people into that experience.
> > > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2
> (or
> > > > > > whatever)
> > > > > > >> > that
> > > > > > >> > > > has
> > > > > > >> > > > > >> both
> > > > > > >> > > > > >> > > Kafka and the stream processing part and they
> > > > actually
> > > > > > work
> > > > > > >> > > > > together.
> > > > > > >> > > > > >> > > 3. Unify the programming experience so the
> client
> > > and
> > > > > > Samza
> > > > > > >> > api
> > > > > > >> > > > > share
> > > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > I think sub-projects keep separate committers
> and
> > > can
> > > > > > have a
> > > > > > >> > > > > separate
> > > > > > >> > > > > >> > repo,
> > > > > > >> > > > > >> > > but I'm actually not really sure (I can't find
> a
> > > > > > definition
> > > > > > >> > of a
> > > > > > >> > > > > >> > subproject
> > > > > > >> > > > > >> > > in Apache).
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > Basically at a high-level you want the
> experience
> > > to
> > > > > > "feel"
> > > > > > >> > > like a
> > > > > > >> > > > > >> single
> > > > > > >> > > > > >> > > system, not to relatively independent things
> that
> > > are
> > > > > > kind
> > > > > > >> of
> > > > > > >> > > > > >> awkwardly
> > > > > > >> > > > > >> > > glued together.
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > I think if we did that they having naming or
> > > branding
> > > > > > like
> > > > > > >> > > "kafka
> > > > > > >> > > > > >> > > streaming" or "kafka streams" or something like
> > > that
> > > > > > would
> > > > > > >> > > > actually
> > > > > > >> > > > > >> do a
> > > > > > >> > > > > >> > > good job of conveying what it is. I do that
> this
> > > > would
> > > > > > help
> > > > > > >> > > > adoption
> > > > > > >> > > > > >> > quite
> > > > > > >> > > > > >> > > a lot as it would correctly convey that using
> > Kafka
> > > > > > >> Streaming
> > > > > > >> > > with
> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > >> > is
> > > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
> pretty
> > > > > heavily
> > > > > > >> > adopted
> > > > > > >> > > > at
> > > > > > >> > > > > >> this
> > > > > > >> > > > > >> > > point.
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > Fwiw we actually considered this model
> originally
> > > > when
> > > > > > open
> > > > > > >> > > > sourcing
> > > > > > >> > > > > >> > Samza,
> > > > > > >> > > > > >> > > however at that time Kafka was relatively
> unknown
> > > and
> > > > > we
> > > > > > >> > decided
> > > > > > >> > > > not
> > > > > > >> > > > > >> to
> > > > > > >> > > > > >> > do
> > > > > > >> > > > > >> > > it since we felt it would be limiting. From my
> > > point
> > > > of
> > > > > > view
> > > > > > >> > the
> > > > > > >> > > > > three
> > > > > > >> > > > > >> > > things have changed (1) Kafka is now really
> > heavily
> > > > > used
> > > > > > for
> > > > > > >> > > > stream
> > > > > > >> > > > > >> > > processing, (2) we learned that abstracting out
> > the
> > > > > > stream
> > > > > > >> > well
> > > > > > >> > > is
> > > > > > >> > > > > >> > > basically impossible, (3) we learned it is
> really
> > > > hard
> > > > > to
> > > > > > >> keep
> > > > > > >> > > the
> > > > > > >> > > > > two
> > > > > > >> > > > > >> > > things feeling like a single product.
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > -Jay
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> Kleppmann
> > <
> > > > > > >> > > > > >> martin@kleppmann.com>
> > > > > > >> > > > > >> > > wrote:
> > > > > > >> > > > > >> > >
> > > > > > >> > > > > >> > >> Hi all,
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> Lots of good thoughts here.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> I agree with the general philosophy of tying
> > Samza
> > > > > more
> > > > > > >> > firmly
> > > > > > >> > > to
> > > > > > >> > > > > >> Kafka.
> > > > > > >> > > > > >> > >> After I spent a while looking at integrating
> > other
> > > > > > message
> > > > > > >> > > > brokers
> > > > > > >> > > > > >> (e.g.
> > > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
> > > > conclusion
> > > > > > that
> > > > > > >> > > > > >> > SystemConsumer
> > > > > > >> > > > > >> > >> tacitly assumes a model so much like Kafka's
> > that
> > > > > pretty
> > > > > > >> much
> > > > > > >> > > > > nobody
> > > > > > >> > > > > >> but
> > > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
> > perhaps
> > > an
> > > > > > >> > exception,
> > > > > > >> > > > but
> > > > > > >> > > > > >> it
> > > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus,
> > > making
> > > > > > Samza
> > > > > > >> > > fully
> > > > > > >> > > > > >> > dependent
> > > > > > >> > > > > >> > >> on Kafka acknowledges that the
> > system-independence
> > > > was
> > > > > > >> never
> > > > > > >> > as
> > > > > > >> > > > > real
> > > > > > >> > > > > >> as
> > > > > > >> > > > > >> > we
> > > > > > >> > > > > >> > >> perhaps made it out to be. The gains of code
> > reuse
> > > > are
> > > > > > >> real.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN has
> also
> > > > always
> > > > > > been
> > > > > > >> > > > > >> appealing to
> > > > > > >> > > > > >> > >> me, for various reasons already mentioned in
> > this
> > > > > > thread.
> > > > > > >> > > > Although
> > > > > > >> > > > > >> > making
> > > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > > > (YARN/Mesos/AWS/etc)
> > > > > > >> seems
> > > > > > >> > > > > >> laudable,
> > > > > > >> > > > > >> > I am
> > > > > > >> > > > > >> > >> a little concerned that it will restrict us
> to a
> > > > > lowest
> > > > > > >> > common
> > > > > > >> > > > > >> > denominator.
> > > > > > >> > > > > >> > >> For example, would host affinity (SAMZA-617)
> > still
> > > > be
> > > > > > >> > possible?
> > > > > > >> > > > For
> > > > > > >> > > > > >> jobs
> > > > > > >> > > > > >> > >> with large amounts of state, I think SAMZA-617
> > > would
> > > > > be
> > > > > > a
> > > > > > >> big
> > > > > > >> > > > boon,
> > > > > > >> > > > > >> > since
> > > > > > >> > > > > >> > >> restoring state off the changelog on every
> > single
> > > > > > restart
> > > > > > >> is
> > > > > > >> > > > > painful,
> > > > > > >> > > > > >> > due
> > > > > > >> > > > > >> > >> to long recovery times. It would be a shame if
> > the
> > > > > > >> decoupling
> > > > > > >> > > > from
> > > > > > >> > > > > >> YARN
> > > > > > >> > > > > >> > >> made host affinity impossible.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> Jay, a question about the proposed API for
> > > > > > instantiating a
> > > > > > >> > job
> > > > > > >> > > in
> > > > > > >> > > > > >> code
> > > > > > >> > > > > >> > >> (rather than a properties file): when
> > submitting a
> > > > job
> > > > > > to a
> > > > > > >> > > > > cluster,
> > > > > > >> > > > > >> is
> > > > > > >> > > > > >> > the
> > > > > > >> > > > > >> > >> idea that the instantiation code runs on a
> > client
> > > > > > >> somewhere,
> > > > > > >> > > > which
> > > > > > >> > > > > >> then
> > > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > > YARN/Mesos/AWS/etc?
> > > > > Or
> > > > > > >> does
> > > > > > >> > > that
> > > > > > >> > > > > >> code
> > > > > > >> > > > > >> > run
> > > > > > >> > > > > >> > >> on each container that is part of the job (in
> > > which
> > > > > > case,
> > > > > > >> how
> > > > > > >> > > > does
> > > > > > >> > > > > >> the
> > > > > > >> > > > > >> > job
> > > > > > >> > > > > >> > >> submission to the cluster work)?
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel right
> to
> > > > make
> > > > > a
> > > > > > 1.0
> > > > > > >> > > > release
> > > > > > >> > > > > >> > with a
> > > > > > >> > > > > >> > >> plan for it to be immediately obsolete. So if
> > this
> > > > is
> > > > > > going
> > > > > > >> > to
> > > > > > >> > > > > >> happen, I
> > > > > > >> > > > > >> > >> think it would be more honest to stick with
> 0.*
> > > > > version
> > > > > > >> > numbers
> > > > > > >> > > > > until
> > > > > > >> > > > > >> > the
> > > > > > >> > > > > >> > >> library-ified Samza has been implemented, is
> > > stable
> > > > > and
> > > > > > >> > widely
> > > > > > >> > > > > used.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> Should the new Samza be a subproject of Kafka?
> > > There
> > > > > is
> > > > > > >> > > precedent
> > > > > > >> > > > > for
> > > > > > >> > > > > >> > >> tight coupling between different Apache
> projects
> > > > (e.g.
> > > > > > >> > Curator
> > > > > > >> > > > and
> > > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think
> > > remaining
> > > > > > >> separate
> > > > > > >> > > > would
> > > > > > >> > > > > >> be
> > > > > > >> > > > > >> > ok.
> > > > > > >> > > > > >> > >> Even if Samza is fully dependent on Kafka,
> there
> > > is
> > > > > > enough
> > > > > > >> > > > > substance
> > > > > > >> > > > > >> in
> > > > > > >> > > > > >> > >> Samza that it warrants being a separate
> project.
> > > An
> > > > > > >> argument
> > > > > > >> > in
> > > > > > >> > > > > >> favour
> > > > > > >> > > > > >> > of
> > > > > > >> > > > > >> > >> merging would be if we think Kafka has a much
> > > > stronger
> > > > > > >> "brand
> > > > > > >> > > > > >> presence"
> > > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If the
> > > Kafka
> > > > > > >> project
> > > > > > >> > is
> > > > > > >> > > > > >> willing
> > > > > > >> > > > > >> > to
> > > > > > >> > > > > >> > >> endorse Samza as the "official" way of doing
> > > > stateful
> > > > > > >> stream
> > > > > > >> > > > > >> > >> transformations, that would probably have much
> > the
> > > > > same
> > > > > > >> > effect
> > > > > > >> > > as
> > > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream Processors"
> > or
> > > > > > suchlike.
> > > > > > >> > > Close
> > > > > > >> > > > > >> > >> collaboration between the two projects will be
> > > > needed
> > > > > in
> > > > > > >> any
> > > > > > >> > > > case.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> From a project management perspective, I guess
> > the
> > > > > "new
> > > > > > >> > Samza"
> > > > > > >> > > > > would
> > > > > > >> > > > > >> > have
> > > > > > >> > > > > >> > >> to be developed on a branch alongside ongoing
> > > > > > maintenance
> > > > > > >> of
> > > > > > >> > > the
> > > > > > >> > > > > >> current
> > > > > > >> > > > > >> > >> line of development? I think it would be
> > important
> > > > to
> > > > > > >> > continue
> > > > > > >> > > > > >> > supporting
> > > > > > >> > > > > >> > >> existing users, and provide a graceful
> migration
> > > > path
> > > > > to
> > > > > > >> the
> > > > > > >> > > new
> > > > > > >> > > > > >> > version.
> > > > > > >> > > > > >> > >> Leaving the current versions unsupported and
> > > forcing
> > > > > > people
> > > > > > >> > to
> > > > > > >> > > > > >> rewrite
> > > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> Best,
> > > > > > >> > > > > >> > >> Martin
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > > > jay@confluent.io>
> > > > > > >> wrote:
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >>> Hey Garry,
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy
> to
> > > chat
> > > > > > more
> > > > > > >> > about
> > > > > > >> > > > > this
> > > > > > >> > > > > >> if
> > > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
> > started
> > > > with
> > > > > > the
> > > > > > >> > idea
> > > > > > >> > > > of
> > > > > > >> > > > > >> "what
> > > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> > ingestion
> > > > > tool"
> > > > > > but
> > > > > > >> > > > > >> ultimately
> > > > > > >> > > > > >> > we
> > > > > > >> > > > > >> > >>> kind of came around to the idea that
> ingestion
> > > and
> > > > > > >> > > > transformation
> > > > > > >> > > > > >> had
> > > > > > >> > > > > >> > >>> pretty different needs and coupling the two
> > made
> > > > > things
> > > > > > >> > hard.
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>> For what it's worth I think copycat (KIP-26)
> > > > actually
> > > > > > will
> > > > > > >> > do
> > > > > > >> > > > what
> > > > > > >> > > > > >> you
> > > > > > >> > > > > >> > >> are
> > > > > > >> > > > > >> > >>> looking for.
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>> With regard to your point about slider, I
> don't
> > > > > > >> necessarily
> > > > > > >> > > > > >> disagree.
> > > > > > >> > > > > >> > >> But I
> > > > > > >> > > > > >> > >>> think getting good YARN support is quite
> doable
> > > > and I
> > > > > > >> think
> > > > > > >> > we
> > > > > > >> > > > can
> > > > > > >> > > > > >> make
> > > > > > >> > > > > >> > >>> that work well. I think the issue this
> proposal
> > > > > solves
> > > > > > is
> > > > > > >> > that
> > > > > > >> > > > > >> > >> technically
> > > > > > >> > > > > >> > >>> it is pretty hard to support multiple cluster
> > > > > > management
> > > > > > >> > > systems
> > > > > > >> > > > > the
> > > > > > >> > > > > >> > way
> > > > > > >> > > > > >> > >>> things are now, you need to write an "app
> > master"
> > > > or
> > > > > > >> > > "framework"
> > > > > > >> > > > > for
> > > > > > >> > > > > >> > each
> > > > > > >> > > > > >> > >>> and they are all a little different so
> testing
> > is
> > > > > > really
> > > > > > >> > hard.
> > > > > > >> > > > In
> > > > > > >> > > > > >> the
> > > > > > >> > > > > >> > >>> absence of this we have been stuck with just
> > YARN
> > > > > which
> > > > > > >> has
> > > > > > >> > > > > >> fantastic
> > > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the org,
> but
> > > > zero
> > > > > > >> > > penetration
> > > > > > >> > > > > >> > >> elsewhere.
> > > > > > >> > > > > >> > >>> Given the huge amount of work being put in to
> > > > slider,
> > > > > > >> > > marathon,
> > > > > > >> > > > > aws
> > > > > > >> > > > > >> > >>> tooling, not to mention the umpteen related
> > > > packaging
> > > > > > >> > > > technologies
> > > > > > >> > > > > >> > people
> > > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> > > > > cloud-specific
> > > > > > >> > deploy
> > > > > > >> > > > > >> tools,
> > > > > > >> > > > > >> > >> etc)
> > > > > > >> > > > > >> > >>> I really think it is important to get this
> > right.
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>> -Jay
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> > Turkington
> > > <
> > > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > > > > > >> > > > > >> > >>>
> > > > > > >> > > > > >> > >>>> Hi all,
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> I think the question below re does Samza
> > become
> > > a
> > > > > > >> > sub-project
> > > > > > >> > > > of
> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > >> > >>>> highlights the broader point around
> migration.
> > > > Chris
> > > > > > >> > mentions
> > > > > > >> > > > > >> Samza's
> > > > > > >> > > > > >> > >>>> maturity is heading towards a v1 release but
> > I'm
> > > > not
> > > > > > sure
> > > > > > >> > it
> > > > > > >> > > > > feels
> > > > > > >> > > > > >> > >> right to
> > > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> deprecate
> > > > most
> > > > > of
> > > > > > >> it.
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> From a selfish perspective I have some guys
> > who
> > > > have
> > > > > > >> > started
> > > > > > >> > > > > >> working
> > > > > > >> > > > > >> > >> with
> > > > > > >> > > > > >> > >>>> Samza and building some new
> > consumers/producers
> > > > was
> > > > > > next
> > > > > > >> > up.
> > > > > > >> > > > > Sounds
> > > > > > >> > > > > >> > like
> > > > > > >> > > > > >> > >>>> that is absolutely not the direction to go.
> I
> > > need
> > > > > to
> > > > > > >> look
> > > > > > >> > > into
> > > > > > >> > > > > the
> > > > > > >> > > > > >> > KIP
> > > > > > >> > > > > >> > >> in
> > > > > > >> > > > > >> > >>>> more detail but for me the attractiveness of
> > > > adding
> > > > > > new
> > > > > > >> > Samza
> > > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all they
> > were
> > > > > doing
> > > > > > was
> > > > > > >> > > > really
> > > > > > >> > > > > >> > getting
> > > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to avoid
> > > > having
> > > > > to
> > > > > > >> > worry
> > > > > > >> > > > > about
> > > > > > >> > > > > >> the
> > > > > > >> > > > > >> > >>>> lifecycle management of external clients. If
> > > there
> > > > > is
> > > > > > a
> > > > > > >> > > generic
> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a new
> > > > connector
> > > > > > into
> > > > > > >> > and
> > > > > > >> > > > > have
> > > > > > >> > > > > >> a
> > > > > > >> > > > > >> > >> lot of
> > > > > > >> > > > > >> > >>>> the heavy lifting re scale and reliability
> > done
> > > > for
> > > > > me
> > > > > > >> then
> > > > > > >> > > it
> > > > > > >> > > > > >> gives
> > > > > > >> > > > > >> > me
> > > > > > >> > > > > >> > >> all
> > > > > > >> > > > > >> > >>>> the pushing new consumers/producers would.
> If
> > > not
> > > > > > then it
> > > > > > >> > > > > >> complicates
> > > > > > >> > > > > >> > my
> > > > > > >> > > > > >> > >>>> operational deployments.
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> Which is similar to my other question with
> the
> > > > > > proposal
> > > > > > >> --
> > > > > > >> > if
> > > > > > >> > > > we
> > > > > > >> > > > > >> > build a
> > > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus the
> > > > requisite
> > > > > > >> shims
> > > > > > >> > to
> > > > > > >> > > > > >> > integrate
> > > > > > >> > > > > >> > >>>> with Slider etc I suspect the former may be
> a
> > > lot
> > > > > more
> > > > > > >> work
> > > > > > >> > > > than
> > > > > > >> > > > > we
> > > > > > >> > > > > >> > >> think.
> > > > > > >> > > > > >> > >>>> We may make it much easier for a newcomer to
> > get
> > > > > > >> something
> > > > > > >> > > > > running
> > > > > > >> > > > > >> but
> > > > > > >> > > > > >> > >>>> having them step up and get a reliable
> > > production
> > > > > > >> > deployment
> > > > > > >> > > > may
> > > > > > >> > > > > >> still
> > > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
> > different
> > > > > > reasons
> > > > > > >> > than
> > > > > > >> > > > > >> today.
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with
> > > making
> > > > > the
> > > > > > >> Samza
> > > > > > >> > > > > >> dependency
> > > > > > >> > > > > >> > >> on
> > > > > > >> > > > > >> > >>>> Kafka much more explicit and I absolutely
> see
> > > the
> > > > > > >> benefits
> > > > > > >> > > in
> > > > > > >> > > > > the
> > > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > > > > > >> > > > terminologies/abstractions
> > > > > > >> > > > > >> that
> > > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library would
> > > > likely
> > > > > > be a
> > > > > > >> > very
> > > > > > >> > > > > nice
> > > > > > >> > > > > >> > tool
> > > > > > >> > > > > >> > >> to
> > > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the
> > > > concerns
> > > > > > >> above
> > > > > > >> > re
> > > > > > >> > > > the
> > > > > > >> > > > > >> > >>>> operational side.
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> Garry
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> -----Original Message-----
> > > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> [mailto:
> > > > > > >> > gdfm@apache.org
> > > > > > >> > > ]
> > > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on
> > Samza
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > > > > > >> > > > > >> > >>>> From outside, I have always perceived Samza
> > as a
> > > > > > >> computing
> > > > > > >> > > > layer
> > > > > > >> > > > > >> over
> > > > > > >> > > > > >> > >>>> Kafka.
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> The question, maybe a bit provocative, is
> > > "should
> > > > > > Samza
> > > > > > >> be
> > > > > > >> > a
> > > > > > >> > > > > >> > sub-project
> > > > > > >> > > > > >> > >>>> of Kafka then?"
> > > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> separate
> > > > > project
> > > > > > >> > with a
> > > > > > >> > > > > >> separate
> > > > > > >> > > > > >> > >>>> governance?
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> Cheers,
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> --
> > > > > > >> > > > > >> > >>>> Gianmarco
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > > > > > yanfang724@gmail.com>
> > > > > > >> > > > wrote:
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more
> > > > tightly.
> > > > > > >> > Because
> > > > > > >> > > > > Samza
> > > > > > >> > > > > >> de
> > > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
> > leverage
> > > > > what
> > > > > > >> Kafka
> > > > > > >> > > > has.
> > > > > > >> > > > > At
> > > > > > >> > > > > >> > the
> > > > > > >> > > > > >> > >>>>> same time, Kafka does not need to reinvent
> > what
> > > > > Samza
> > > > > > >> > > already
> > > > > > >> > > > > >> has. I
> > > > > > >> > > > > >> > >>>>> also like the idea of separating the
> > ingestion
> > > > and
> > > > > > >> > > > > transformation.
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
> image
> > > how
> > > > > the
> > > > > > >> Samza
> > > > > > >> > > > will
> > > > > > >> > > > > >> look
> > > > > > >> > > > > >> > >>>> like.
> > > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> > > difference
> > > > > in
> > > > > > >> terms
> > > > > > >> > > of
> > > > > > >> > > > > how
> > > > > > >> > > > > >> > >>>>> Samza should look like.
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code shows
> > (A
> > > > > > client of
> > > > > > >> > > > Kakfa)
> > > > > > >> > > > > ?
> > > > > > >> > > > > >> And
> > > > > > >> > > > > >> > >>>>> user's application code calls this client?
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka
> > (like
> > > > > what
> > > > > > the
> > > > > > >> > > code
> > > > > > >> > > > > >> shows),
> > > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > > > > fault-tolerance?
> > > > > > >> Are
> > > > > > >> > > they
> > > > > > >> > > > > >> taken
> > > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> mechanism,
> > > such
> > > > > as
> > > > > > >> > "Samza
> > > > > > >> > > > > >> worker"
> > > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> 2. What about other features, such as
> > > > auto-scaling,
> > > > > > >> shared
> > > > > > >> > > > > state,
> > > > > > >> > > > > >> > >>>>> monitoring?
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is this
> > what
> > > > > Chris
> > > > > > >> > > > suggests?)
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa
> > and
> > > > > > produce
> > > > > > >> to
> > > > > > >> > > it.
> > > > > > >> > > > > >> Then it
> > > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks like
> > now,
> > > > > > except it
> > > > > > >> > > does
> > > > > > >> > > > > not
> > > > > > >> > > > > >> > rely
> > > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it leverage
> > > > Kafka's
> > > > > > >> > metrics,
> > > > > > >> > > > > logs,
> > > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> Thanks,
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> Fang, Yan
> > > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang
> > Wang <
> > > > > > >> > > > > wangguoz@gmail.com
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> > >>>> wrote:
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>>> Read through the code example and it looks
> > > good
> > > > to
> > > > > > me.
> > > > > > >> A
> > > > > > >> > > few
> > > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable runnable
> > > like:
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > > --config-factory=...
> > > > > > >> > > > > >> > >>>> --config-path=file://...
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> And this proposal advocate for deploying
> > Samza
> > > > > more
> > > > > > as
> > > > > > >> > > > embedded
> > > > > > >> > > > > >> > >>>>>> libraries in user application code
> (ignoring
> > > the
> > > > > > >> > > terminology
> > > > > > >> > > > > >> since
> > > > > > >> > > > > >> > >>>>>> it is not the
> > > > > > >> > > > > >> > >>>>> same
> > > > > > >> > > > > >> > >>>>>> as the prototype code):
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> StreamTask task = new
> MyStreamTask(configs);
> > > > > Thread
> > > > > > >> > thread
> > > > > > >> > > =
> > > > > > >> > > > > new
> > > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> I think both of these deployment modes are
> > > > > important
> > > > > > >> for
> > > > > > >> > > > > >> different
> > > > > > >> > > > > >> > >>>>>> types
> > > > > > >> > > > > >> > >>>>> of
> > > > > > >> > > > > >> > >>>>>> users. That said, I think making Samza
> > purely
> > > > > > >> standalone
> > > > > > >> > is
> > > > > > >> > > > > still
> > > > > > >> > > > > >> > >>>>>> sufficient for either runnable or library
> > > modes.
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> Guozhang
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
> Kreps
> > <
> > > > > > >> > > > jay@confluent.io>
> > > > > > >> > > > > >> > wrote:
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> example,
> > it
> > > > was
> > > > > > >> > supposed
> > > > > > >> > > > to
> > > > > > >> > > > > >> look
> > > > > > >> > > > > >> > >>>>>>> like
> > > > > > >> > > > > >> > >>>>>>> this:
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> > > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > > > "localhost:4242");
> > > > > > >> > > > > >> StreamingConfig
> > > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > > "test-topic-2");
> > > > > > >> > > > > >> > >>>>>>>
> > > config.processor(ExampleStreamProcessor.class);
> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > StringSerializer(),
> > > > new
> > > > > > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming
> > > > container =
> > > > > > new
> > > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>> -Jay
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay
> > Kreps <
> > > > > > >> > > > jay@confluent.io
> > > > > > >> > > > > >
> > > > > > >> > > > > >> > >>>> wrote:
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
> Chris
> > > and
> > > > I
> > > > > > were
> > > > > > >> > > having
> > > > > > >> > > > > >> > >>>>>>>> around
> > > > > > >> > > > > >> > >>>>>>> whether
> > > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a
> kind
> > > of
> > > > > data
> > > > > > >> > > > ingestion
> > > > > > >> > > > > >> > >>>>> framework
> > > > > > >> > > > > >> > >>>>>>> for
> > > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26
> > > > > "copycat").
> > > > > > >> This
> > > > > > >> > > > kind
> > > > > > >> > > > > of
> > > > > > >> > > > > >> > >>>>>> combined
> > > > > > >> > > > > >> > >>>>>>>> with complaints around config and YARN
> and
> > > the
> > > > > > >> > discussion
> > > > > > >> > > > > >> around
> > > > > > >> > > > > >> > >>>>>>>> how
> > > > > > >> > > > > >> > >>>>> to
> > > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> So the thought experiment was, given
> that
> > > > Samza
> > > > > > was
> > > > > > >> > > > basically
> > > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific, what if
> > you
> > > > just
> > > > > > >> > embraced
> > > > > > >> > > > > that
> > > > > > >> > > > > >> > >>>>>>>> and turned it
> > > > > > >> > > > > >> > >>>>>> into
> > > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> > framework
> > > > and
> > > > > > more
> > > > > > >> > > like a
> > > > > > >> > > > > >> > >>>>>>>> third
> > > > > > >> > > > > >> > >>>>> Kafka
> > > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing consumer"
> > with
> > > > > state
> > > > > > >> > > > management
> > > > > > >> > > > > >> > >>>>>> facilities.
> > > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> complex
> > > > stream
> > > > > > >> > > processing
> > > > > > >> > > > > >> > >>>>>>>> framework
> > > > > > >> > > > > >> > >>>>>>> this
> > > > > > >> > > > > >> > >>>>>>>> would actually be a very simple thing,
> not
> > > > much
> > > > > > more
> > > > > > >> > > > > >> complicated
> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > > >> > > > > >> > >>>>> use
> > > > > > >> > > > > >> > >>>>>>> or
> > > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris
> > said
> > > > we
> > > > > > >> thought
> > > > > > >> > > > about
> > > > > > >> > > > > >> it
> > > > > > >> > > > > >> > >>>>>>>> a
> > > > > > >> > > > > >> > >>>>> lot
> > > > > > >> > > > > >> > >>>>>> of
> > > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> > processing
> > > > > > systems
> > > > > > >> > were
> > > > > > >> > > > > doing)
> > > > > > >> > > > > >> > >>>>> seemed
> > > > > > >> > > > > >> > >>>>>>> like
> > > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output data
> > to
> > > > and
> > > > > > from
> > > > > > >> > the
> > > > > > >> > > > > stream
> > > > > > >> > > > > >> > >>>>>>>> processing. But when we actually looked
> > into
> > > > how
> > > > > > that
> > > > > > >> > > would
> > > > > > >> > > > > >> > >>>>>>>> work,
> > > > > > >> > > > > >> > >>>>> Samza
> > > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion
> > > framework
> > > > > > for a
> > > > > > >> > > bunch
> > > > > > >> > > > of
> > > > > > >> > > > > >> > >>>>> reasons.
> > > > > > >> > > > > >> > >>>>>> To
> > > > > > >> > > > > >> > >>>>>>>> really do that right you need a pretty
> > > > different
> > > > > > >> > internal
> > > > > > >> > > > > data
> > > > > > >> > > > > >> > >>>>>>>> model
> > > > > > >> > > > > >> > >>>>>> and
> > > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split them
> and
> > > had
> > > > > an
> > > > > > api
> > > > > > >> > for
> > > > > > >> > > > > Kafka
> > > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and
> a
> > > > > separate
> > > > > > >> api
> > > > > > >> > > for
> > > > > > >> > > > > >> Kafka
> > > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> This would also allow really embracing
> the
> > > > same
> > > > > > >> > > terminology
> > > > > > >> > > > > and
> > > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about the
> > current
> > > > > > state is
> > > > > > >> > > that
> > > > > > >> > > > > the
> > > > > > >> > > > > >> > >>>>>>>> two
> > > > > > >> > > > > >> > >>>>>>> systems
> > > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like
> > > > > "stream"
> > > > > > vs
> > > > > > >> > > > "topic"
> > > > > > >> > > > > >> and
> > > > > > >> > > > > >> > >>>>>>> different
> > > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means you
> > kind
> > > > of
> > > > > > have
> > > > > > >> to
> > > > > > >> > > > learn
> > > > > > >> > > > > >> > >>>>>>>> Kafka's
> > > > > > >> > > > > >> > >>>>>>> way,
> > > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly different
> way,
> > > > then
> > > > > > kind
> > > > > > >> of
> > > > > > >> > > > > >> > >>>>>>>> understand
> > > > > > >> > > > > >> > >>>>> how
> > > > > > >> > > > > >> > >>>>>>> they
> > > > > > >> > > > > >> > >>>>>>>> map to each other, which having walked a
> > few
> > > > > > people
> > > > > > >> > > through
> > > > > > >> > > > > >> this
> > > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of time
> > on
> > > > > > >> airplanes I
> > > > > > >> > > > > hacked
> > > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> incomplete
> > > > > > prototype
> > > > > > >> of
> > > > > > >> > > > what
> > > > > > >> > > > > >> > >>>>>>>> this would
> > > > > > >> > > > > >> > >>>>> look
> > > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
> dumped
> > > into
> > > > > > Kafka
> > > > > > >> as
> > > > > > >> > > it
> > > > > > >> > > > > >> > >>>>>>>> required a
> > > > > > >> > > > > >> > >>>>>> few
> > > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is the
> > > code:
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> >
> > > > > > >> > > > >
> > > > > > >> >
> > > > > >
> > > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I just
> > > > > liberally
> > > > > > >> > renamed
> > > > > > >> > > > > >> > >>>>>>>> everything
> > > > > > >> > > > > >> > >>>>> to
> > > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
> regard
> > > for
> > > > > > >> > > > compatibility.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> To use this would be something like
> this:
> > > > > > >> > > > > >> > >>>>>>>> Properties props = new Properties();
> > > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > > > > "localhost:4242");
> > > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > > > > > >> > > > > >> config.processor(ExampleStreamProcessor.class);
> > > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > > StringDeserializer());
> > > > > > >> > > > KafkaStreaming
> > > > > > >> > > > > >> > >>>>>> container =
> > > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> > container.run();
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > > > SamzaContainer;
> > > > > > >> > > > > StreamProcessor
> > > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> So rather than putting all the class
> names
> > > in
> > > > a
> > > > > > file
> > > > > > >> > and
> > > > > > >> > > > then
> > > > > > >> > > > > >> > >>>>>>>> having
> > > > > > >> > > > > >> > >>>>>> the
> > > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
> > > > > instantiate
> > > > > > the
> > > > > > >> > > > > container
> > > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced over
> > > > however
> > > > > > many
> > > > > > >> > > > > instances
> > > > > > >> > > > > >> > >>>>>>>> of
> > > > > > >> > > > > >> > >>>>> this
> > > > > > >> > > > > >> > >>>>>>> are
> > > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance
> > dies,
> > > > new
> > > > > > >> tasks
> > > > > > >> > > are
> > > > > > >> > > > > >> added
> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > > >> > > > > >> > >>>>> the
> > > > > > >> > > > > >> > >>>>>>>> existing containers without shutting
> them
> > > > down).
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> We would provide some glue for running
> > this
> > > > > stuff
> > > > > > in
> > > > > > >> > YARN
> > > > > > >> > > > via
> > > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
> using
> > > some
> > > > > of
> > > > > > >> their
> > > > > > >> > > > tools
> > > > > > >> > > > > >> > >>>>>>>> but from the
> > > > > > >> > > > > >> > >>>>>> point
> > > > > > >> > > > > >> > >>>>>>> of
> > > > > > >> > > > > >> > >>>>>>>> view of these frameworks these stream
> > > > processing
> > > > > > jobs
> > > > > > >> > are
> > > > > > >> > > > > just
> > > > > > >> > > > > >> > >>>>>> stateless
> > > > > > >> > > > > >> > >>>>>>>> services that can come and go and expand
> > and
> > > > > > contract
> > > > > > >> > at
> > > > > > >> > > > > will.
> > > > > > >> > > > > >> > >>>>>>>> There
> > > > > > >> > > > > >> > >>>>> is
> > > > > > >> > > > > >> > >>>>>>> no
> > > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it
> > would
> > > > get
> > > > > > >> larger
> > > > > > >> > > if
> > > > > > >> > > > we
> > > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly larger.
> We
> > > > really
> > > > > > do
> > > > > > >> > get a
> > > > > > >> > > > ton
> > > > > > >> > > > > >> > >>>>>>>> of
> > > > > > >> > > > > >> > >>>>>>> leverage
> > > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> > delegated
> > > to
> > > > > the
> > > > > > >> new
> > > > > > >> > > > > >> consumer.
> > > > > > >> > > > > >> > >>>>> This
> > > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> > management
> > > > > > strategy
> > > > > > >> > > > > available
> > > > > > >> > > > > >> > >>>>>>>> to
> > > > > > >> > > > > >> > >>>>>> Kafka
> > > > > > >> > > > > >> > >>>>>>>>  consumer is also available to Samza
> (and
> > > vice
> > > > > > versa)
> > > > > > >> > and
> > > > > > >> > > > > with
> > > > > > >> > > > > >> > >>>>>>>> the
> > > > > > >> > > > > >> > >>>>>>> exact
> > > > > > >> > > > > >> > >>>>>>>>  same configs.
> > > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as state
> > reuse
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
> > thought
> > > > > > >> provoking.
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> -Jay
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris
> > > > > Riccomini <
> > > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > > > > > >> > > > > >> > >>>>>>>> wrote:
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with Samza
> > > > > engineers
> > > > > > at
> > > > > > >> > > > LinkedIn
> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > > >> > > > > >> > >>>>>>> Confluent
> > > > > > >> > > > > >> > >>>>>>>>> and we came up with a few observations
> > and
> > > > > would
> > > > > > >> like
> > > > > > >> > to
> > > > > > >> > > > > >> > >>>>>>>>> propose
> > > > > > >> > > > > >> > >>>>> some
> > > > > > >> > > > > >> > >>>>>>>>> changes.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I want
> to
> > > > call
> > > > > > out
> > > > > > >> > about
> > > > > > >> > > > > >> > >>>>>>>>> Samza's
> > > > > > >> > > > > >> > >>>>>> design,
> > > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic
> > > > deployment
> > > > > > >> system.
> > > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > > > > >> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer
> > and
> > > > > > Kafka's
> > > > > > >> > > > consumer
> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > > > >> > > > > >> > >>>>> are
> > > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
> > problems.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> All three of these issues are related,
> > but
> > > > I'll
> > > > > > >> > address
> > > > > > >> > > > them
> > > > > > >> > > > > >> in
> > > > > > >> > > > > >> > >>>>> order.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Deployment
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a
> > > > dynamic
> > > > > > >> > > deployment
> > > > > > >> > > > > >> > >>>>>>>>> scheduler
> > > > > > >> > > > > >> > >>>>>> such
> > > > > > >> > > > > >> > >>>>>>>>> as
> > > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially
> built
> > > > > Samza,
> > > > > > we
> > > > > > >> > bet
> > > > > > >> > > > that
> > > > > > >> > > > > >> > >>>>>>>>> there
> > > > > > >> > > > > >> > >>>>>> would
> > > > > > >> > > > > >> > >>>>>>>>> be
> > > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area, and we
> > > could
> > > > > > >> support
> > > > > > >> > > > them,
> > > > > > >> > > > > >> and
> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > > >> > > > > >> > >>>>>> rest
> > > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there are
> many
> > > > > > >> variations.
> > > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > > > > > >> > > > > >> > >>>>>> many
> > > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start their
> > > > > > processors
> > > > > > >> > like
> > > > > > >> > > > > normal
> > > > > > >> > > > > >> > >>>>>>>>> Java processes, and use traditional
> > > > deployment
> > > > > > >> scripts
> > > > > > >> > > > such
> > > > > > >> > > > > as
> > > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > > > > > >> > > > > >> > >>>>>> Chef,
> > > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
> system
> > > on
> > > > > > users
> > > > > > >> > makes
> > > > > > >> > > > the
> > > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really painful
> for
> > > > first
> > > > > > time
> > > > > > >> > > > users.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was
> > > also
> > > > a
> > > > > > bit
> > > > > > >> of
> > > > > > >> > a
> > > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > > > > > >> > > > > >> > >>>>>> because
> > > > > > >> > > > > >> > >>>>>>>>> of
> > > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between
> > the
> > > > > > nature of
> > > > > > >> > > batch
> > > > > > >> > > > > >> jobs
> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > > >> > > > > >> > >>>>>>> stream
> > > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> > > conscious
> > > > > > effort
> > > > > > >> to
> > > > > > >> > > > favor
> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > > >> > > > > >> > >>>>>> Hadoop
> > > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since
> > it
> > > > > worked
> > > > > > >> and
> > > > > > >> > > was
> > > > > > >> > > > > well
> > > > > > >> > > > > >> > >>>>>>> understood.
> > > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that batch
> > > jobs
> > > > > > have a
> > > > > > >> > > > definite
> > > > > > >> > > > > >> > >>>>>> beginning,
> > > > > > >> > > > > >> > >>>>>>>>> and
> > > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't
> > > > > (usually).
> > > > > > >> This
> > > > > > >> > > > leads
> > > > > > >> > > > > to
> > > > > > >> > > > > >> > >>>>>>>>> a
> > > > > > >> > > > > >> > >>>>> much
> > > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream
> > > > > processors.
> > > > > > >> You
> > > > > > >> > > > > >> basically
> > > > > > >> > > > > >> > >>>>>>>>> just
> > > > > > >> > > > > >> > >>>>>>> need
> > > > > > >> > > > > >> > >>>>>>>>> to find a place to start the processor,
> > and
> > > > > start
> > > > > > >> it.
> > > > > > >> > > The
> > > > > > >> > > > > way
> > > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no
> > > concept
> > > > > of
> > > > > > a
> > > > > > >> > > cluster
> > > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > > > > > >> > > > > >> > >>>>>> add
> > > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> coupling
> > > > Samza
> > > > > > with
> > > > > > >> a
> > > > > > >> > > > > >> scheduler
> > > > > > >> > > > > >> > >>>>>>>>> is
> > > > > > >> > > > > >> > >>>>>> that
> > > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
> handle
> > > > > > deployment.
> > > > > > >> > > This
> > > > > > >> > > > > >> pulls
> > > > > > >> > > > > >> > >>>>>>>>> in a
> > > > > > >> > > > > >> > >>>>>>> bunch
> > > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> > > distribution
> > > > > > (config
> > > > > > >> > > > > stream),
> > > > > > >> > > > > >> > >>>>>>>>> shell
> > > > > > >> > > > > >> > >>>>>>> scrips
> > > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging
> > (all
> > > > the
> > > > > > .tgz
> > > > > > >> > > > stuff),
> > > > > > >> > > > > >> etc.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
> > > > deployment
> > > > > > was
> > > > > > >> to
> > > > > > >> > > > > support
> > > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> > > locality,
> > > > > you
> > > > > > >> need
> > > > > > >> > to
> > > > > > >> > > > put
> > > > > > >> > > > > >> > >>>>>>>>> your
> > > > > > >> > > > > >> > >>>>>> processors
> > > > > > >> > > > > >> > >>>>>>>>> close to the data they're processing.
> > Upon
> > > > > > further
> > > > > > >> > > > > >> > >>>>>>>>> investigation,
> > > > > > >> > > > > >> > >>>>>>> though,
> > > > > > >> > > > > >> > >>>>>>>>> this feature is not that beneficial.
> > There
> > > is
> > > > > > some
> > > > > > >> > good
> > > > > > >> > > > > >> > >>>>>>>>> discussion
> > > > > > >> > > > > >> > >>>>>> about
> > > > > > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335.
> > Again,
> > > we
> > > > > > took
> > > > > > >> the
> > > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > > > > > >> > > > > >> > >>>>>> path,
> > > > > > >> > > > > >> > >>>>>>>>> but
> > > > > > >> > > > > >> > >>>>>>>>> there are some fundamental differences
> > > > between
> > > > > > HDFS
> > > > > > >> > and
> > > > > > >> > > > > Kafka.
> > > > > > >> > > > > >> > >>>>>>>>> HDFS
> > > > > > >> > > > > >> > >>>>>> has
> > > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions.
> This
> > > > leads
> > > > > to
> > > > > > >> less
> > > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
> > > processors
> > > > > on
> > > > > > top
> > > > > > >> > of
> > > > > > >> > > > > Kafka.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a crutch.
> > > Samza
> > > > > > doesn't
> > > > > > >> > > have
> > > > > > >> > > > > any
> > > > > > >> > > > > >> > >>>>>>>>> built
> > > > > > >> > > > > >> > >>>>> in
> > > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
> > depends
> > > on
> > > > > the
> > > > > > >> > > dynamic
> > > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to handle
> > > > restarts
> > > > > > >> when a
> > > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > > > > > >> > > > > >> > >>>>>>> made
> > > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a standalone
> > > Samza
> > > > > > >> > container
> > > > > > >> > > > > >> > >>>> (SAMZA-516).
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is good,
> but I
> > > > think
> > > > > > that
> > > > > > >> > > we've
> > > > > > >> > > > > >> gone
> > > > > > >> > > > > >> > >>>>>>>>> too
> > > > > > >> > > > > >> > >>>>>> far
> > > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > > > (SystemConsumer,
> > > > > > >> > > > > SystemProducer,
> > > > > > >> > > > > >> > >>>> etc).
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about
> > every
> > > > > > >> component
> > > > > > >> > > > > >> > >>>>> (MessageChooser,
> > > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> > > ConfigRewriter,
> > > > > > etc).
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> > forgotten,
> > > as
> > > > > > well.
> > > > > > >> > Some
> > > > > > >> > > > of
> > > > > > >> > > > > >> > >>>>>>>>> these
> > > > > > >> > > > > >> > >>>>> are
> > > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not to be.
> > > This
> > > > > all
> > > > > > >> comes
> > > > > > >> > > at
> > > > > > >> > > > a
> > > > > > >> > > > > >> cost:
> > > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is making
> it
> > > > harder
> > > > > > for
> > > > > > >> > our
> > > > > > >> > > > > users
> > > > > > >> > > > > >> > >>>>>>>>> to
> > > > > > >> > > > > >> > >>>>> pick
> > > > > > >> > > > > >> > >>>>>> up
> > > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also
> > makes
> > > > it
> > > > > > >> > difficult
> > > > > > >> > > > for
> > > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about what
> the
> > > > > > >> > > characteristics
> > > > > > >> > > > of
> > > > > > >> > > > > >> > >>>>>>>>> the container (since the
> characteristics
> > > > change
> > > > > > >> > > depending
> > > > > > >> > > > on
> > > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are most
> > > visible
> > > > > in
> > > > > > the
> > > > > > >> > > > System
> > > > > > >> > > > > >> APIs.
> > > > > > >> > > > > >> > >>>>> What
> > > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be functional
> is
> > > > Kafka
> > > > > > as
> > > > > > >> its
> > > > > > >> > > > > >> > >>>>>>>>> transport
> > > > > > >> > > > > >> > >>>>>> layer.
> > > > > > >> > > > > >> > >>>>>>>>> But
> > > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases
> > > into
> > > > > one
> > > > > > >> API:
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> The current System API supports both of
> > > these
> > > > > use
> > > > > > >> > cases.
> > > > > > >> > > > The
> > > > > > >> > > > > >> > >>>>>>>>> problem
> > > > > > >> > > > > >> > >>>>>> is,
> > > > > > >> > > > > >> > >>>>>>>>> we
> > > > > > >> > > > > >> > >>>>>>>>> actually want different features for
> each
> > > use
> > > > > > case.
> > > > > > >> By
> > > > > > >> > > > > >> papering
> > > > > > >> > > > > >> > >>>>>>>>> over
> > > > > > >> > > > > >> > >>>>>>> these
> > > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a single
> > API,
> > > > > we've
> > > > > > >> > > > introduced
> > > > > > >> > > > > a
> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > > > >> > > > > >> > >>>>>>> leaky
> > > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like in
> (2)
> > > is
> > > > to
> > > > > > have
> > > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
> > offsets
> > > > > (like
> > > > > > >> > Kafka).
> > > > > > >> > > > > This
> > > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > > > > > >> > > > > >> > >>>>> with
> > > > > > >> > > > > >> > >>>>>>> (1),
> > > > > > >> > > > > >> > >>>>>>>>> though, since different systems have
> > > > different
> > > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> mailing
> > > list
> > > > > and
> > > > > > >> the
> > > > > > >> > > SQL
> > > > > > >> > > > > >> JIRAs
> > > > > > >> > > > > >> > >>>>> about
> > > > > > >> > > > > >> > >>>>>>> the
> > > > > > >> > > > > >> > >>>>>>>>> need for this.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> > > replayability.
> > > > > > Kafka
> > > > > > >> > > allows
> > > > > > >> > > > us
> > > > > > >> > > > > >> to
> > > > > > >> > > > > >> > >>>>> rewind
> > > > > > >> > > > > >> > >>>>>>>>> when
> > > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other systems
> > > don't.
> > > > In
> > > > > > some
> > > > > > >> > > > cases,
> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > > > > >> > > > > >> > >>>>>>> return
> > > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > > > > > >> WikipediaSystemConsumer)
> > > > > > >> > > > > because
> > > > > > >> > > > > >> > >>>>>>>>> they
> > > > > > >> > > > > >> > >>>>>> have
> > > > > > >> > > > > >> > >>>>>>> no
> > > > > > >> > > > > >> > >>>>>>>>> offsets.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka
> > > > supports
> > > > > > >> > > > > partitioning,
> > > > > > >> > > > > >> > >>>>>>>>> but
> > > > > > >> > > > > >> > >>>>> many
> > > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by having
> a
> > > > single
> > > > > > >> > > partition
> > > > > > >> > > > > for
> > > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other systems
> model
> > > > > > >> partitioning
> > > > > > >> > > > > >> > >>>> differently (e.g.
> > > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a
> mess.
> > > > > > Creating
> > > > > > >> > > streams
> > > > > > >> > > > > in
> > > > > > >> > > > > >> a
> > > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> impossible.
> > > As
> > > > is
> > > > > > >> > modeling
> > > > > > >> > > > > >> > >>>>>>>>> metadata
> > > > > > >> > > > > >> > >>>>> for
> > > > > > >> > > > > >> > >>>>>>> the
> > > > > > >> > > > > >> > >>>>>>>>> system (replication factor, partitions,
> > > > > location,
> > > > > > >> > etc).
> > > > > > >> > > > The
> > > > > > >> > > > > >> > >>>>>>>>> list
> > > > > > >> > > > > >> > >>>>> goes
> > > > > > >> > > > > >> > >>>>>>> on.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing
> Samza,
> > > > > Kafka's
> > > > > > >> > > consumer
> > > > > > >> > > > > and
> > > > > > >> > > > > >> > >>>>> producer
> > > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On
> the
> > > > > > >> > consumer-side,
> > > > > > >> > > > you
> > > > > > >> > > > > >> > >>>>>>>>> had two
> > > > > > >> > > > > >> > >>>>>>>>> options: use the high level consumer,
> or
> > > the
> > > > > > simple
> > > > > > >> > > > > consumer.
> > > > > > >> > > > > >> > >>>>>>>>> The
> > > > > > >> > > > > >> > >>>>>>> problem
> > > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was that
> it
> > > > > > controlled
> > > > > > >> > your
> > > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and the
> > > order
> > > > > in
> > > > > > >> which
> > > > > > >> > > you
> > > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > > > > > >> > > > > >> > >>>>> problem
> > > > > > >> > > > > >> > >>>>>>>>> with
> > > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not
> > > simple.
> > > > > It's
> > > > > > >> > basic.
> > > > > > >> > > > You
> > > > > > >> > > > > >> > >>>>>>>>> end up
> > > > > > >> > > > > >> > >>>>>>> having
> > > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level
> stuff
> > > > that
> > > > > > you
> > > > > > >> > > > > shouldn't.
> > > > > > >> > > > > >> > >>>>>>>>> We
> > > > > > >> > > > > >> > >>>>>> spent a
> > > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > > > KafkaSystemConsumer
> > > > > > very
> > > > > > >> > > > robust.
> > > > > > >> > > > > >> It
> > > > > > >> > > > > >> > >>>>>>>>> also allows us to support some cool
> > > features:
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
> > > > > > prioritization.
> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> assignment
> > > to
> > > > > > support
> > > > > > >> > > > joins,
> > > > > > >> > > > > >> > >>>>>>>>> global
> > > > > > >> > > > > >> > >>>>>> state
> > > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> > checkpointing.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time is
> > that
> > > > > these
> > > > > > >> > > features
> > > > > > >> > > > > >> > >>>>>>>>> should
> > > > > > >> > > > > >> > >>>>>>> actually
> > > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers
> > (not
> > > > just
> > > > > > >> Samza
> > > > > > >> > > > stream
> > > > > > >> > > > > >> > >>>>>> processors)
> > > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like joins
> > and
> > > > > > partition
> > > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > > > > > >> > > > > >> > >>>>>>> Kafka
> > > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> > conclusion.
> > > > > > They're
> > > > > > >> > > adding
> > > > > > >> > > > a
> > > > > > >> > > > > >> ton
> > > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> consumer
> > > > > > >> > > implementation.
> > > > > > >> > > > > To a
> > > > > > >> > > > > >> > >>>>>>>>> large extent,
> > > > > > >> > > > > >> > >>>>> it's
> > > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've already
> done
> > > in
> > > > > > Samza.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a
> > > very
> > > > > > similar
> > > > > > >> > > > > approach
> > > > > > >> > > > > >> > >>>>>>>>> to
> > > > > > >> > > > > >> > >>>>>> Samza's
> > > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation
> for
> > > > > > handling
> > > > > > >> > > offset
> > > > > > >> > > > > >> > >>>>>> checkpointing.
> > > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> management
> > > > > feature
> > > > > > >> > stores
> > > > > > >> > > > > >> offset
> > > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you
> to
> > > > fetch
> > > > > > them
> > > > > > >> > > from
> > > > > > >> > > > > the
> > > > > > >> > > > > >> > >>>>>>>>> broker.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since
> > we
> > > > > could
> > > > > > >> have
> > > > > > >> > > > shared
> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > > >> > > > > >> > >>>>> work
> > > > > > >> > > > > >> > >>>>>> if
> > > > > > >> > > > > >> > >>>>>>>>> it
> > > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Vision
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> radical
> > > > > > proposal.
> > > > > > >> > Samza
> > > > > > >> > > > is
> > > > > > >> > > > > >> > >>>>> relatively
> > > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to
> say
> > > that
> > > > > > we're
> > > > > > >> > > near a
> > > > > > >> > > > > 1.0
> > > > > > >> > > > > >> > >>>>>> release.
> > > > > > >> > > > > >> > >>>>>>>>> I'd
> > > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what we've
> > > > > learned,
> > > > > > and
> > > > > > >> > > begin
> > > > > > >> > > > > >> > >>>>>>>>> thinking
> > > > > > >> > > > > >> > >>>>>>> about
> > > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change
> if
> > > we
> > > > > were
> > > > > > >> > > starting
> > > > > > >> > > > > >> from
> > > > > > >> > > > > >> > >>>>>> scratch?
> > > > > > >> > > > > >> > >>>>>>>>> My
> > > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way
> > to
> > > > run
> > > > > > Samza
> > > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
> > > > > dependences
> > > > > > on
> > > > > > >> > > YARN,
> > > > > > >> > > > > >> Mesos,
> > > > > > >> > > > > >> > >>>> etc.
> > > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support
> only
> > > > Kafka
> > > > > > as
> > > > > > >> the
> > > > > > >> > > > > stream
> > > > > > >> > > > > >> > >>>>>> processing
> > > > > > >> > > > > >> > >>>>>>>>> layer.
> > > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
> > > > > > >> serialization,
> > > > > > >> > > and
> > > > > > >> > > > > >> > >>>>>>>>> config
> > > > > > >> > > > > >> > >>>>>>> systems,
> > > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues that I
> > > > > outlined
> > > > > > >> > above.
> > > > > > >> > > It
> > > > > > >> > > > > >> > >>>>>>>>> should
> > > > > > >> > > > > >> > >>>>> also
> > > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> > > > dramatically.
> > > > > > >> > > Supporting
> > > > > > >> > > > > >> only
> > > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow Samza
> > to
> > > be
> > > > > > >> executed
> > > > > > >> > > on
> > > > > > >> > > > > YARN
> > > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > > > Marathon/Aurora),
> > > > > or
> > > > > > >> most
> > > > > > >> > > > other
> > > > > > >> > > > > >> > >>>>>>>>> in-house
> > > > > > >> > > > > >> > >>>>>>> deployment
> > > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a lot
> > easier
> > > > for
> > > > > > new
> > > > > > >> > > users.
> > > > > > >> > > > > >> > >>>>>>>>> Imagine
> > > > > > >> > > > > >> > >>>>>>> having
> > > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN.
> > The
> > > > drop
> > > > > > in
> > > > > > >> > > mailing
> > > > > > >> > > > > >> list
> > > > > > >> > > > > >> > >>>>>> traffic
> > > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue
> to
> > > me.
> > > > > The
> > > > > > >> > > reality
> > > > > > >> > > > > is,
> > > > > > >> > > > > >> > >>>>> everyone
> > > > > > >> > > > > >> > >>>>>>>>> that
> > > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka.
> > We
> > > > > > basically
> > > > > > >> > > > require
> > > > > > >> > > > > >> it
> > > > > > >> > > > > >> > >>>>>> already
> > > > > > >> > > > > >> > >>>>>>> in
> > > > > > >> > > > > >> > >>>>>>>>> order for most features to work. Those
> > that
> > > > are
> > > > > > >> using
> > > > > > >> > > > other
> > > > > > >> > > > > >> > >>>>>>>>> systems
> > > > > > >> > > > > >> > >>>>>> are
> > > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into
> Kafka
> > > (1),
> > > > > and
> > > > > > >> then
> > > > > > >> > > > they
> > > > > > >> > > > > do
> > > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is already
> > > > > > discussion (
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> >
> > > > > > >> > > > >
> > > > > > >> >
> > > > > >
> > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > > > >> > > > > >> > >>>>> 767
> > > > > > >> > > > > >> > >>>>>>>>> )
> > > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka
> > > > extremely
> > > > > > >> easy.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple with
> > Kafka,
> > > > we
> > > > > > can
> > > > > > >> > > > leverage
> > > > > > >> > > > > a
> > > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > > > >> > > > > >> > >>>>>>> their
> > > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> maintain
> > > our
> > > > > own
> > > > > > >> > config,
> > > > > > >> > > > > >> > >>>>>>>>> metrics,
> > > > > > >> > > > > >> > >>>>> etc.
> > > > > > >> > > > > >> > >>>>>>> We
> > > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries, and
> > make
> > > > them
> > > > > > >> > better.
> > > > > > >> > > > This
> > > > > > >> > > > > >> > >>>>>>>>> will
> > > > > > >> > > > > >> > >>>>> also
> > > > > > >> > > > > >> > >>>>>>>>> allow us to share the consumer/producer
> > > APIs,
> > > > > and
> > > > > > >> will
> > > > > > >> > > let
> > > > > > >> > > > > us
> > > > > > >> > > > > >> > >>>>> leverage
> > > > > > >> > > > > >> > >>>>>>>>> their offset management and partition
> > > > > management,
> > > > > > >> > rather
> > > > > > >> > > > > than
> > > > > > >> > > > > >> > >>>>>>>>> having
> > > > > > >> > > > > >> > >>>>>> our
> > > > > > >> > > > > >> > >>>>>>>>> own. All of the coordinator stream code
> > > would
> > > > > go
> > > > > > >> away,
> > > > > > >> > > as
> > > > > > >> > > > > >> would
> > > > > > >> > > > > >> > >>>>>>>>> most
> > > > > > >> > > > > >> > >>>>>> of
> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have
> > to
> > > > push
> > > > > > some
> > > > > > >> > > > > partition
> > > > > > >> > > > > >> > >>>>>>> management
> > > > > > >> > > > > >> > >>>>>>>>> features into the Kafka broker, but
> > they're
> > > > > > already
> > > > > > >> > > moving
> > > > > > >> > > > > in
> > > > > > >> > > > > >> > >>>>>>>>> that direction with the new consumer
> API.
> > > The
> > > > > > >> features
> > > > > > >> > > we
> > > > > > >> > > > > have
> > > > > > >> > > > > >> > >>>>>>>>> for
> > > > > > >> > > > > >> > >>>>>> partition
> > > > > > >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and
> > seem
> > > > > like
> > > > > > >> they
> > > > > > >> > > > should
> > > > > > >> > > > > >> be
> > > > > > >> > > > > >> > >>>>>>>>> in
> > > > > > >> > > > > >> > >>>>>> Kafka
> > > > > > >> > > > > >> > >>>>>>>>> anyway. There will always be some niche
> > > > usages
> > > > > > which
> > > > > > >> > > will
> > > > > > >> > > > > >> > >>>>>>>>> require
> > > > > > >> > > > > >> > >>>>>> extra
> > > > > > >> > > > > >> > >>>>>>>>> care and hence full control over
> > partition
> > > > > > >> assignments
> > > > > > >> > > > much
> > > > > > >> > > > > >> > >>>>>>>>> like the
> > > > > > >> > > > > >> > >>>>>>> Kafka
> > > > > > >> > > > > >> > >>>>>>>>> low level consumer api. These would
> > > continue
> > > > to
> > > > > > be
> > > > > > >> > > > > supported.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> These items will be good for the Samza
> > > > > community.
> > > > > > >> > > They'll
> > > > > > >> > > > > make
> > > > > > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it easier
> > for
> > > > > > >> developers
> > > > > > >> > > to
> > > > > > >> > > > > add
> > > > > > >> > > > > >> > >>>>>>>>> new features.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and
> > > > somewhat
> > > > > > >> > backwards
> > > > > > >> > > > > >> > >>>>> incompatible
> > > > > > >> > > > > >> > >>>>>>>>> change). If we choose to go this route,
> > > it's
> > > > > > >> important
> > > > > > >> > > > that
> > > > > > >> > > > > we
> > > > > > >> > > > > >> > >>>>> openly
> > > > > > >> > > > > >> > >>>>>>>>> communicate how we're going to provide
> a
> > > > > > migration
> > > > > > >> > path
> > > > > > >> > > > from
> > > > > > >> > > > > >> > >>>>>>>>> the
> > > > > > >> > > > > >> > >>>>>>> existing
> > > > > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make
> > > incompatible
> > > > > > >> > changes).
> > > > > > >> > > I
> > > > > > >> > > > > >> think
> > > > > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to
> > > provide a
> > > > > > >> wrapper
> > > > > > >> > to
> > > > > > >> > > > > allow
> > > > > > >> > > > > >> > >>>>>>>>> existing StreamTask implementations to
> > > > continue
> > > > > > >> > running
> > > > > > >> > > on
> > > > > > >> > > > > the
> > > > > > >> > > > > >> > >>>> new container.
> > > > > > >> > > > > >> > >>>>>>> It's
> > > > > > >> > > > > >> > >>>>>>>>> also important that we openly
> communicate
> > > > about
> > > > > > >> > timing,
> > > > > > >> > > > and
> > > > > > >> > > > > >> > >>>>>>>>> stages
> > > > > > >> > > > > >> > >>>>> of
> > > > > > >> > > > > >> > >>>>>>> the
> > > > > > >> > > > > >> > >>>>>>>>> migration.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you
> > have
> > > > > > opinions.
> > > > > > >> > :)
> > > > > > >> > > > > Please
> > > > > > >> > > > > >> > >>>>>>>>> send
> > > > > > >> > > > > >> > >>>>>> your
> > > > > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>> Cheers,
> > > > > > >> > > > > >> > >>>>>>>>> Chris
> > > > > > >> > > > > >> > >>>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>>
> > > > > > >> > > > > >> > >>>>>>>
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>> --
> > > > > > >> > > > > >> > >>>>>> -- Guozhang
> > > > > > >> > > > > >> > >>>>>>
> > > > > > >> > > > > >> > >>>>>
> > > > > > >> > > > > >> > >>>>
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> > >>
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >> >
> > > > > > >> > > > > >>
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Garrett Barton <ga...@gmail.com>.
​Hey all, just want to chime in before it too late.  Been following samza
for a long time, and using it in production for the past 6 months or so.

 In no particular order the things I like most about Samza are:

- Yarn support, resiliency of my deployment is paramount.  This is why I
use Samza over Flume.
- Kafka Support​, I do a lot of work with Kafka so tight integration with
Kafka is awesome.
- Single transform per job vs entire topology like Storm.  Made life easier
to manage, upgrade and process streams/transformations independent of each
other.


Overall I really like where ya'll wanna go with better integration with
Kafka.  Where the project lives I don't have much thoughts on, I think
there is a great community here that has been extremely helpful and its
self sustaining which is awesome.

Getting a more out of the box and easy to setup version of Samza would be
cool, yes its a bit tedious to have to run the very well documented set of
instructions to startup Yarn locally to test with.  That could be
improved.  I would still very much like to see container support remain
within Samza, just make the core be able to run as a library and build on
top of it.  If I had to go one place to get Samza-new-core, then somewhere
else to find a version that replaces samza-yarn I think that would be an
overall loss for the folks doing business with Samza on yarn today.

 I'd much rather see both Yarn and Docker supported as the defacto scaling
solutions on top of the baseline library version. The Library version would
I imagine would only ever run as a single instance, lots of people can get
by just fine with a single thread chewing streams.

Oh, one thing that could get better is the configs, not enough coupling to
tell when one has utterly bolluxed the config other than funky behavior
when trying to run.  No idea on a fix, more in code, some kind of builder
class? Dunno.

Thank you all who got Samza as far as it is today, its a powerful tool and
in my current organization we hope to move the majority of our workflows
into it in the next 6 months.  So this thread is being followed very
closely from our futures perspective.



On Sun, Jul 12, 2015 at 8:58 PM, Chris Riccomini <cr...@apache.org>
wrote:

> That was meant to be "thread" not "threat". lol. :)
>
> On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
> > Hey all,
> >
> > I want to start by saying that I'm absolutely thrilled to be a part of
> > this community. The amount of level-headed, thoughtful, educated
> discussion
> > that's gone on over the past ~10 days is overwhelming. Wonderful.
> >
> > It seems like discussion is waning a bit, and we've reached some
> > conclusions. There are several key emails in this threat, which I want to
> > call out:
> >
> > 1. Jakob's summary of the three potential ways forward.
> >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> > 2. Julian's call out that we should be focusing on community over code.
> >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> > 3. Martin's summary about the benefits of merging communities.
> >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> > 4. Jakob's comments about the distinction between community and code
> paths.
> >
> >
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
> >
> > I agree with the comments on all of these emails. I think Martin's
> summary
> > of his position aligns very closely with my own. To that end, I think we
> > should get concrete about what the proposal is, and call a vote on it.
> > Given that Jay, Martin, and I seem to be aligning fairly closely, I think
> > we should start with:
> >
> > 1. [community] Make Samza a subproject of Kafka.
> > 2. [community] Make all Samza PMC/committers committers of the
> subproject.
> > 3. [community] Migrate Samza's website/documentation into Kafka's.
> > 4. [code] Have the Samza community and the Kafka community start a
> > from-scratch reboot together in the new Kafka subproject. We can
> > borrow/copy &  paste significant chunks of code from Samza's code base.
> > 5. [code] The subproject would intentionally eliminate support for both
> > other streaming systems and all deployment systems.
> > 6. [code] Attempt to provide a bridge from our SystemConsumer to KIP-26
> > (copy cat)
> > 7. [code] Attempt to provide a bridge from the new subproject's processor
> > interface to our legacy StreamTask interface.
> > 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> > subproject that has a fault-tolerant container with state management.
> >
> > It's likely that (6) and (7) won't be fully drop-in. Still, the closer we
> > can get, the better it's going to be for our existing community.
> >
> > One thing that I didn't touch on with (2) is whether any Samza PMC
> members
> > should be rolled into Kafka PMC membership as well (though, Jay and Jakob
> > are already PMC members on both). I think that Samza's community
> deserves a
> > voice on the PMC, so I'd propose that we roll at least a few PMC members
> > into the Kafka PMC, but I don't have a strong framework for which people
> to
> > pick.
> >
> > Before (8), I think that Samza's TLP can continue to commit bug fixes and
> > patches as it sees fit, provided that we openly communicate that we won't
> > necessarily migrate new features to the new subproject, and that the TLP
> > will be shut down after the migration to the Kafka subproject occurs.
> >
> > Jakob, I could use your guidance here about about how to achieve this
> from
> > an Apache process perspective (sorry).
> >
> > * Should I just call a vote on this proposal?
> > * Should it happen on dev or private?
> > * Do committers have binding votes, or just PMC?
> >
> > Having trouble finding much detail on the Apache wikis. :(
> >
> > Cheers,
> > Chris
> >
> > On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com> wrote:
> >
> >> Thanks, Jay. This argument persuaded me actually. :)
> >>
> >> Fang, Yan
> >> yanfang724@gmail.com
> >>
> >> On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> >>
> >> > Hey Yan,
> >> >
> >> > Yeah philosophically I think the argument is that you should capture
> the
> >> > stream in Kafka independent of the transformation. This is obviously a
> >> > Kafka-centric view point.
> >> >
> >> > Advantages of this:
> >> > - In practice I think this is what e.g. Storm people often end up
> doing
> >> > anyway. You usually need to throttle any access to a live serving
> >> database.
> >> > - Can have multiple subscribers and they get the same thing without
> >> > additional load on the source system.
> >> > - Applications can tap into the stream if need be by subscribing.
> >> > - You can debug your transformation by tailing the Kafka topic with
> the
> >> > console consumer
> >> > - Can tee off the same data stream for batch analysis or Lambda arch
> >> style
> >> > re-processing
> >> >
> >> > The disadvantage is that it will use Kafka resources. But the idea is
> >> > eventually you will have multiple subscribers to any data source (at
> >> least
> >> > for monitoring) so you will end up there soon enough anyway.
> >> >
> >> > Down the road the technical benefit is that I think it gives us a good
> >> path
> >> > towards end-to-end exactly once semantics from source to destination.
> >> > Basically the connectors need to support idempotence when talking to
> >> Kafka
> >> > and we need the transactional write feature in Kafka to make the
> >> > transformation atomic. This is actually pretty doable if you separate
> >> > connector=>kafka problem from the generic transformations which are
> >> always
> >> > kafka=>kafka. However I think it is quite impossible to do in a
> >> all_things
> >> > => all_things environment. Today you can say "well the semantics of
> the
> >> > Samza APIs depend on the connectors you use" but it is actually worse
> >> then
> >> > that because the semantics actually depend on the pairing of
> >> connectors--so
> >> > not only can you probably not get a usable "exactly once" guarantee
> >> > end-to-end it can actually be quite hard to reverse engineer what
> >> property
> >> > (if any) your end-to-end flow has if you have heterogenous systems.
> >> >
> >> > -Jay
> >> >
> >> > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com>
> wrote:
> >> >
> >> > > {quote}
> >> > > maintained in a separate repository and retaining the existing
> >> > > committership but sharing as much else as possible (website, etc)
> >> > > {quote}
> >> > >
> >> > > Overall, I agree on this idea. Now the question is more about "how
> to
> >> do
> >> > > it".
> >> > >
> >> > > On the other hand, one thing I want to point out is that, if we
> >> decide to
> >> > > go this way, how do we want to support
> >> > > otherSystem-transformation-otherSystem use case?
> >> > >
> >> > > Basically, there are four user groups here:
> >> > >
> >> > > 1. Kafka-transformation-Kafka
> >> > > 2. Kafka-transformation-otherSystem
> >> > > 3. otherSystem-transformation-Kafka
> >> > > 4. otherSystem-transformation-otherSystem
> >> > >
> >> > > For group 1, they can easily use the new Samza library to achieve.
> For
> >> > > group 2 and 3, they can use copyCat -> transformation -> Kafka or
> >> Kafka->
> >> > > transformation -> copyCat.
> >> > >
> >> > > The problem is for group 4. Do we want to abandon this or still
> >> support
> >> > it?
> >> > > Of course, this use case can be achieved by using copyCat ->
> >> > transformation
> >> > > -> Kafka -> transformation -> copyCat, the thing is how we persuade
> >> them
> >> > to
> >> > > do this long chain. If yes, it will also be a win for Kafka too. Or
> if
> >> > > there is no one in this community actually doing this so far, maybe
> >> ok to
> >> > > not support the group 4 directly.
> >> > >
> >> > > Thanks,
> >> > >
> >> > > Fang, Yan
> >> > > yanfang724@gmail.com
> >> > >
> >> > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> >> > >
> >> > > > Yeah I agree with this summary. I think there are kind of two
> >> questions
> >> > > > here:
> >> > > > 1. Technically does alignment/reliance on Kafka make sense
> >> > > > 2. Branding wise (naming, website, concepts, etc) does alignment
> >> with
> >> > > Kafka
> >> > > > make sense
> >> > > >
> >> > > > Personally I do think both of these things would be really
> valuable,
> >> > and
> >> > > > would dramatically alter the trajectory of the project.
> >> > > >
> >> > > > My preference would be to see if people can mostly agree on a
> >> direction
> >> > > > rather than splintering things off. From my point of view the
> ideal
> >> > > outcome
> >> > > > of all the options discussed would be to make Samza a closely
> >> aligned
> >> > > > subproject, maintained in a separate repository and retaining the
> >> > > existing
> >> > > > committership but sharing as much else as possible (website, etc).
> >> No
> >> > > idea
> >> > > > about how these things work, Jacob, you probably know more.
> >> > > >
> >> > > > No discussion amongst the Kafka folks has happened on this, but
> >> likely
> >> > we
> >> > > > should figure out what the Samza community actually wants first.
> >> > > >
> >> > > > I admit that this is a fairly radical departure from how things
> are.
> >> > > >
> >> > > > If that doesn't fly, I think, yeah we could leave Samza as it is
> >> and do
> >> > > the
> >> > > > more radical reboot inside Kafka. From my point of view that does
> >> leave
> >> > > > things in a somewhat confusing state since now there are two
> stream
> >> > > > processing systems more or less coupled to Kafka in large part
> made
> >> by
> >> > > the
> >> > > > same people. But, arguably that might be a cleaner way to make the
> >> > > cut-over
> >> > > > and perhaps less risky for Samza community since if it works
> people
> >> can
> >> > > > switch and if it doesn't nothing will have changed. Dunno, how do
> >> > people
> >> > > > feel about this?
> >> > > >
> >> > > > -Jay
> >> > > >
> >> > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com>
> >> > wrote:
> >> > > >
> >> > > > > >  This leads me to thinking that merging projects and
> communities
> >> > > might
> >> > > > > be a good idea: with the union of experience from both
> >> communities,
> >> > we
> >> > > > will
> >> > > > > probably build a better system that is better for users.
> >> > > > > Is this what's being proposed though? Merging the projects seems
> >> like
> >> > > > > a consequence of at most one of the three directions under
> >> > discussion:
> >> > > > > 1) Samza 2.0: The Samza community relies more heavily on Kafka
> for
> >> > > > > configuration, etc. (to a greater or lesser extent to be
> >> determined)
> >> > > > > but the Samza community would not automatically merge withe
> Kafka
> >> > > > > community (the Phoenix/HBase example is a good one here).
> >> > > > > 2) Samza Reboot: The Samza community continues to exist with a
> >> > limited
> >> > > > > project scope, but similarly would not need to be part of the
> >> Kafka
> >> > > > > community (ie given committership) to progress.  Here, maybe the
> >> > Samza
> >> > > > > team would become a subproject of Kafka (the Board frowns on
> >> > > > > subprojects at the moment, so I'm not sure if that's even
> >> feasible),
> >> > > > > but that would not be required.
> >> > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option the
> >> Kafka
> >> > > > > team builds its own streaming library, possibly off of Jay's
> >> > > > > prototype, which has not direct lineage to the Samza team.
> >> There's
> >> > no
> >> > > > > reason for the Kafka team to bring in the Samza team.
> >> > > > >
> >> > > > > Is the Kafka community on board with this?
> >> > > > >
> >> > > > > To be clear, all three options under discussion are interesting,
> >> > > > > technically valid and likely healthy directions for the project.
> >> > > > > Also, they are not mutually exclusive.  The Samza community
> could
> >> > > > > decide to pursue, say, 'Samza 2.0', while the Kafka community
> went
> >> > > > > forward with 'Hey Samza!'  My points above are directed entirely
> >> at
> >> > > > > the community aspect of these choices.
> >> > > > > -Jakob
> >> > > > >
> >> > > > > On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com>
> >> > wrote:
> >> > > > > > That's great.  Thanks, Jay.
> >> > > > > >
> >> > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io>
> >> > wrote:
> >> > > > > >
> >> > > > > >> Yeah totally agree. I think you have this issue even today,
> >> right?
> >> > > > I.e.
> >> > > > > if
> >> > > > > >> you need to make a simple config change and you're running in
> >> YARN
> >> > > > today
> >> > > > > >> you end up bouncing the job which then rebuilds state. I
> think
> >> the
> >> > > fix
> >> > > > > is
> >> > > > > >> exactly what you described which is to have a long timeout on
> >> > > > partition
> >> > > > > >> movement for stateful jobs so that if a job is just getting
> >> > bounced,
> >> > > > and
> >> > > > > >> the cluster manager (or admin) is smart enough to restart it
> on
> >> > the
> >> > > > same
> >> > > > > >> host when possible, it can optimistically reuse any existing
> >> state
> >> > > it
> >> > > > > finds
> >> > > > > >> on disk (if it is valid).
> >> > > > > >>
> >> > > > > >> So in this model the charter of the CM is to place processes
> as
> >> > > > > stickily as
> >> > > > > >> possible and to restart or re-place failed processes. The
> >> charter
> >> > of
> >> > > > the
> >> > > > > >> partition management system is to control the assignment of
> >> work
> >> > to
> >> > > > > these
> >> > > > > >> processes. The nice thing about this is that the work
> >> assignment,
> >> > > > > timeouts,
> >> > > > > >> behavior, configs, and code will all be the same across all
> >> > cluster
> >> > > > > >> managers.
> >> > > > > >>
> >> > > > > >> So I think that prototype would actually give you exactly
> what
> >> you
> >> > > > want
> >> > > > > >> today for any cluster manager (or manual placement + restart
> >> > script)
> >> > > > > that
> >> > > > > >> was sticky in terms of host placement since there is already
> a
> >> > > > > configurable
> >> > > > > >> partition movement timeout and task-by-task state reuse with
> a
> >> > check
> >> > > > on
> >> > > > > >> state validity.
> >> > > > > >>
> >> > > > > >> -Jay
> >> > > > > >>
> >> > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> >> > > roger.hoover@gmail.com
> >> > > > >
> >> > > > > >> wrote:
> >> > > > > >>
> >> > > > > >> > That would be great to let Kafka do as much heavy lifting
> as
> >> > > > possible
> >> > > > > and
> >> > > > > >> > make it easier for other languages to implement Samza apis.
> >> > > > > >> >
> >> > > > > >> > One thing to watch out for is the interplay between Kafka's
> >> > group
> >> > > > > >> > management and the external scheduler/process manager's
> fault
> >> > > > > tolerance.
> >> > > > > >> > If a container dies, the Kafka group membership protocol
> will
> >> > try
> >> > > to
> >> > > > > >> assign
> >> > > > > >> > it's tasks to other containers while at the same time the
> >> > process
> >> > > > > manager
> >> > > > > >> > is trying to relaunch the container.  Without some
> >> consideration
> >> > > for
> >> > > > > this
> >> > > > > >> > (like a configurable amount of time to wait before Kafka
> >> alters
> >> > > the
> >> > > > > group
> >> > > > > >> > membership), there may be thrashing going on which is
> >> especially
> >> > > bad
> >> > > > > for
> >> > > > > >> > containers with large amounts of local state.
> >> > > > > >> >
> >> > > > > >> > Someone else pointed this out already but I thought it
> might
> >> be
> >> > > > worth
> >> > > > > >> > calling out again.
> >> > > > > >> >
> >> > > > > >> > Cheers,
> >> > > > > >> >
> >> > > > > >> > Roger
> >> > > > > >> >
> >> > > > > >> >
> >> > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <
> jay@confluent.io
> >> >
> >> > > > wrote:
> >> > > > > >> >
> >> > > > > >> > > Hey Roger,
> >> > > > > >> > >
> >> > > > > >> > > I couldn't agree more. We spent a bunch of time talking
> to
> >> > > people
> >> > > > > and
> >> > > > > >> > that
> >> > > > > >> > > is exactly the stuff we heard time and again. What makes
> it
> >> > > hard,
> >> > > > of
> >> > > > > >> > > course, is that there is some tension between
> compatibility
> >> > with
> >> > > > > what's
> >> > > > > >> > > there now and making things better for new users.
> >> > > > > >> > >
> >> > > > > >> > > I also strongly agree with the importance of
> multi-language
> >> > > > > support. We
> >> > > > > >> > are
> >> > > > > >> > > talking now about Java, but for application development
> use
> >> > > cases
> >> > > > > >> people
> >> > > > > >> > > want to work in whatever language they are using
> >> elsewhere. I
> >> > > > think
> >> > > > > >> > moving
> >> > > > > >> > > to a model where Kafka itself does the group membership,
> >> > > lifecycle
> >> > > > > >> > control,
> >> > > > > >> > > and partition assignment has the advantage of putting all
> >> that
> >> > > > > complex
> >> > > > > >> > > stuff behind a clean api that the clients are already
> >> going to
> >> > > be
> >> > > > > >> > > implementing for their consumer, so the added
> functionality
> >> > for
> >> > > > > stream
> >> > > > > >> > > processing beyond a consumer becomes very minor.
> >> > > > > >> > >
> >> > > > > >> > > -Jay
> >> > > > > >> > >
> >> > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> >> > > > > roger.hoover@gmail.com>
> >> > > > > >> > > wrote:
> >> > > > > >> > >
> >> > > > > >> > > > Metamorphosis...nice. :)
> >> > > > > >> > > >
> >> > > > > >> > > > This has been a great discussion.  As a user of Samza
> >> who's
> >> > > > > recently
> >> > > > > >> > > > integrated it into a relatively large organization, I
> >> just
> >> > > want
> >> > > > to
> >> > > > > >> add
> >> > > > > >> > > > support to a few points already made.
> >> > > > > >> > > >
> >> > > > > >> > > > The biggest hurdles to adoption of Samza as it
> currently
> >> > > exists
> >> > > > > that
> >> > > > > >> > I've
> >> > > > > >> > > > experienced are:
> >> > > > > >> > > > 1) YARN - YARN is overly complex in many environments
> >> where
> >> > > > Puppet
> >> > > > > >> > would
> >> > > > > >> > > do
> >> > > > > >> > > > just fine but it was the only mechanism to get fault
> >> > > tolerance.
> >> > > > > >> > > > 2) Configuration - I think I like the idea of
> configuring
> >> > most
> >> > > > of
> >> > > > > the
> >> > > > > >> > job
> >> > > > > >> > > > in code rather than config files.  In general, I think
> >> the
> >> > > goal
> >> > > > > >> should
> >> > > > > >> > be
> >> > > > > >> > > > to make it harder to make mistakes, especially of the
> >> kind
> >> > > where
> >> > > > > the
> >> > > > > >> > code
> >> > > > > >> > > > expects something and the config doesn't match.  The
> >> current
> >> > > > > config
> >> > > > > >> is
> >> > > > > >> > > > quite intricate and error-prone.  For example, the
> >> > application
> >> > > > > logic
> >> > > > > >> > may
> >> > > > > >> > > > depend on bootstrapping a topic but rather than
> asserting
> >> > that
> >> > > > in
> >> > > > > the
> >> > > > > >> > > code,
> >> > > > > >> > > > you have to rely on getting the config right.  Likewise
> >> with
> >> > > > > serdes,
> >> > > > > >> > the
> >> > > > > >> > > > Java representations produced by various serdes (JSON,
> >> Avro,
> >> > > > etc.)
> >> > > > > >> are
> >> > > > > >> > > not
> >> > > > > >> > > > equivalent so you cannot just reconfigure a serde
> without
> >> > > > changing
> >> > > > > >> the
> >> > > > > >> > > > code.   It would be nice for jobs to be able to assert
> >> what
> >> > > they
> >> > > > > >> expect
> >> > > > > >> > > > from their input topics in terms of partitioning.  This
> >> is
> >> > > > > getting a
> >> > > > > >> > > little
> >> > > > > >> > > > off topic but I was even thinking about creating a
> "Samza
> >> > > config
> >> > > > > >> > linter"
> >> > > > > >> > > > that would sanity check a set of configs.  Especially
> in
> >> > > > > >> organizations
> >> > > > > >> > > > where config is managed by a different team than the
> >> > > application
> >> > > > > >> > > developer,
> >> > > > > >> > > > it's very hard to get avoid config mistakes.
> >> > > > > >> > > > 3) Java/Scala centric - for many teams (especially
> >> > DevOps-type
> >> > > > > >> folks),
> >> > > > > >> > > the
> >> > > > > >> > > > pain of the Java toolchain (maven, slow builds, weak
> >> command
> >> > > > line
> >> > > > > >> > > support,
> >> > > > > >> > > > configuration over convention) really inhibits
> >> productivity.
> >> > > As
> >> > > > > more
> >> > > > > >> > and
> >> > > > > >> > > > more high-quality clients become available for Kafka, I
> >> hope
> >> > > > > they'll
> >> > > > > >> > > follow
> >> > > > > >> > > > Samza's model.  Not sure how much it affects the
> >> proposals
> >> > in
> >> > > > this
> >> > > > > >> > thread
> >> > > > > >> > > > but please consider other languages in the ecosystem as
> >> > well.
> >> > > > > From
> >> > > > > >> > what
> >> > > > > >> > > > I've heard, Spark has more Python users than
> Java/Scala.
> >> > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> >> > > > > >> > > >
> >> > > > > >> > > >
> >> > > > > >> > >
> >> > > > > >> >
> >> > > > > >>
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> >> > > > > >> > > > and are working on a Yeoman generator
> >> > > > > >> > > > https://github.com/Quantiply/generator-rico for
> >> > Jython/Samza
> >> > > > > >> projects
> >> > > > > >> > to
> >> > > > > >> > > > alleviate some of the pain)
> >> > > > > >> > > >
> >> > > > > >> > > > I also want to underscore Jay's point about improving
> the
> >> > user
> >> > > > > >> > > experience.
> >> > > > > >> > > > That's a very important factor for adoption.  I think
> the
> >> > goal
> >> > > > > should
> >> > > > > >> > be
> >> > > > > >> > > to
> >> > > > > >> > > > make Samza as easy to get started with as something
> like
> >> > > > Logstash.
> >> > > > > >> > > > Logstash is vastly inferior in terms of capabilities to
> >> > Samza
> >> > > > but
> >> > > > > >> it's
> >> > > > > >> > > easy
> >> > > > > >> > > > to get started and that makes a big difference.
> >> > > > > >> > > >
> >> > > > > >> > > > Cheers,
> >> > > > > >> > > >
> >> > > > > >> > > > Roger
> >> > > > > >> > > >
> >> > > > > >> > > >
> >> > > > > >> > > >
> >> > > > > >> > > >
> >> > > > > >> > > >
> >> > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci
> >> > > Morales <
> >> > > > > >> > > > gdfm@apache.org> wrote:
> >> > > > > >> > > >
> >> > > > > >> > > > > Forgot to add. On the naming issues, Kafka
> >> Metamorphosis
> >> > is
> >> > > a
> >> > > > > clear
> >> > > > > >> > > > winner
> >> > > > > >> > > > > :)
> >> > > > > >> > > > >
> >> > > > > >> > > > > --
> >> > > > > >> > > > > Gianmarco
> >> > > > > >> > > > >
> >> > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
> >> Morales <
> >> > > > > >> > > gdfm@apache.org
> >> > > > > >> > > > >
> >> > > > > >> > > > > wrote:
> >> > > > > >> > > > >
> >> > > > > >> > > > > > Hi,
> >> > > > > >> > > > > >
> >> > > > > >> > > > > > @Martin, thanks for you comments.
> >> > > > > >> > > > > > Maybe I'm missing some important point, but I think
> >> > > coupling
> >> > > > > the
> >> > > > > >> > > > releases
> >> > > > > >> > > > > > is actually a *good* thing.
> >> > > > > >> > > > > > To make an example, would it be better if the MR
> and
> >> > HDFS
> >> > > > > >> > components
> >> > > > > >> > > of
> >> > > > > >> > > > > > Hadoop had different release schedules?
> >> > > > > >> > > > > >
> >> > > > > >> > > > > > Actually, keeping the discussion in a single place
> >> would
> >> > > > make
> >> > > > > >> > > agreeing
> >> > > > > >> > > > on
> >> > > > > >> > > > > > releases (and backwards compatibility) much easier,
> >> as
> >> > > > > everybody
> >> > > > > >> > > would
> >> > > > > >> > > > be
> >> > > > > >> > > > > > responsible for the whole codebase.
> >> > > > > >> > > > > >
> >> > > > > >> > > > > > That said, I like the idea of absorbing samza-core
> >> as a
> >> > > > > >> > sub-project,
> >> > > > > >> > > > and
> >> > > > > >> > > > > > leave the fancy stuff separate.
> >> > > > > >> > > > > > It probably gives 90% of the benefits we have been
> >> > > > discussing
> >> > > > > >> here.
> >> > > > > >> > > > > >
> >> > > > > >> > > > > > Cheers,
> >> > > > > >> > > > > >
> >> > > > > >> > > > > > --
> >> > > > > >> > > > > > Gianmarco
> >> > > > > >> > > > > >
> >> > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> >> jay.kreps@gmail.com
> >> > >
> >> > > > > wrote:
> >> > > > > >> > > > > >
> >> > > > > >> > > > > >> Hey Martin,
> >> > > > > >> > > > > >>
> >> > > > > >> > > > > >> I agree coupling release schedules is a downside.
> >> > > > > >> > > > > >>
> >> > > > > >> > > > > >> Definitely we can try to solve some of the
> >> integration
> >> > > > > problems
> >> > > > > >> in
> >> > > > > >> > > > > >> Confluent Platform or in other distributions. But
> I
> >> > think
> >> > > > > this
> >> > > > > >> > ends
> >> > > > > >> > > up
> >> > > > > >> > > > > >> being really shallow. I guess I feel to really
> get a
> >> > good
> >> > > > > user
> >> > > > > >> > > > > experience
> >> > > > > >> > > > > >> the two systems have to kind of feel like part of
> >> the
> >> > > same
> >> > > > > thing
> >> > > > > >> > and
> >> > > > > >> > > > you
> >> > > > > >> > > > > >> can't really add that in later--you can put both
> in
> >> the
> >> > > > same
> >> > > > > >> > > > > downloadable
> >> > > > > >> > > > > >> tar file but it doesn't really give a very
> cohesive
> >> > > > feeling.
> >> > > > > I
> >> > > > > >> > agree
> >> > > > > >> > > > > that
> >> > > > > >> > > > > >> ultimately any of the project stuff is as much
> >> social
> >> > and
> >> > > > > naming
> >> > > > > >> > as
> >> > > > > >> > > > > >> anything else--theoretically two totally
> independent
> >> > > > projects
> >> > > > > >> > could
> >> > > > > >> > > > work
> >> > > > > >> > > > > >> to
> >> > > > > >> > > > > >> tightly align. In practice this seems to be quite
> >> > > difficult
> >> > > > > >> > though.
> >> > > > > >> > > > > >>
> >> > > > > >> > > > > >> For the frameworks--totally agree it would be good
> >> to
> >> > > > > maintain
> >> > > > > >> the
> >> > > > > >> > > > > >> framework support with the project. In some cases
> >> there
> >> > > may
> >> > > > > not
> >> > > > > >> be
> >> > > > > >> > > too
> >> > > > > >> > > > > >> much
> >> > > > > >> > > > > >> there since the integration gets lighter but I
> think
> >> > > > whatever
> >> > > > > >> > stubs
> >> > > > > >> > > > you
> >> > > > > >> > > > > >> need should be included. So no I definitely wasn't
> >> > trying
> >> > > > to
> >> > > > > >> imply
> >> > > > > >> > > > > >> dropping
> >> > > > > >> > > > > >> support for these frameworks, just making the
> >> > integration
> >> > > > > >> lighter
> >> > > > > >> > by
> >> > > > > >> > > > > >> separating process management from partition
> >> > management.
> >> > > > > >> > > > > >>
> >> > > > > >> > > > > >> You raise two good points we would have to figure
> >> out
> >> > if
> >> > > we
> >> > > > > went
> >> > > > > >> > > down
> >> > > > > >> > > > > the
> >> > > > > >> > > > > >> alignment path:
> >> > > > > >> > > > > >> 1. With respect to the name, yeah I think the
> first
> >> > > > question
> >> > > > > is
> >> > > > > >> > > > whether
> >> > > > > >> > > > > >> some "re-branding" would be worth it. If so then I
> >> > think
> >> > > we
> >> > > > > can
> >> > > > > >> > > have a
> >> > > > > >> > > > > big
> >> > > > > >> > > > > >> thread on the name. I'm definitely not set on
> Kafka
> >> > > > > Streaming or
> >> > > > > >> > > Kafka
> >> > > > > >> > > > > >> Streams I was just using them to be kind of
> >> > > illustrative. I
> >> > > > > >> agree
> >> > > > > >> > > with
> >> > > > > >> > > > > >> your
> >> > > > > >> > > > > >> critique of these names, though I think people
> would
> >> > get
> >> > > > the
> >> > > > > >> idea.
> >> > > > > >> > > > > >> 2. Yeah you also raise a good point about how to
> >> > "factor"
> >> > > > it.
> >> > > > > >> Here
> >> > > > > >> > > are
> >> > > > > >> > > > > the
> >> > > > > >> > > > > >> options I see (I could get enthusiastic about any
> of
> >> > > them):
> >> > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> >> > > > > >> > > > > >>    b. Two repos, retaining the current seperation
> >> > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api and
> >> > > samza-core
> >> > > > > is
> >> > > > > >> > > > absorbed
> >> > > > > >> > > > > >> almost like a third client
> >> > > > > >> > > > > >>
> >> > > > > >> > > > > >> Cheers,
> >> > > > > >> > > > > >>
> >> > > > > >> > > > > >> -Jay
> >> > > > > >> > > > > >>
> >> > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> >> > > > > >> > > > martin@kleppmann.com>
> >> > > > > >> > > > > >> wrote:
> >> > > > > >> > > > > >>
> >> > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a few
> >> > follow-up
> >> > > > > >> > comments.
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
> >> becoming
> >> > a
> >> > > > > >> > subproject:
> >> > > > > >> > > > the
> >> > > > > >> > > > > >> > reasons you mention are good. The risk I see is
> >> that
> >> > > > > release
> >> > > > > >> > > > schedules
> >> > > > > >> > > > > >> > become coupled to each other, which can slow
> >> everyone
> >> > > > down,
> >> > > > > >> and
> >> > > > > >> > > > large
> >> > > > > >> > > > > >> > projects with many contributors are harder to
> >> manage.
> >> > > > > (Jakob,
> >> > > > > >> > can
> >> > > > > >> > > > you
> >> > > > > >> > > > > >> speak
> >> > > > > >> > > > > >> > from experience, having seen a wider range of
> >> Hadoop
> >> > > > > ecosystem
> >> > > > > >> > > > > >> projects?)
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> > Some of the goals of a better unified developer
> >> > > > experience
> >> > > > > >> could
> >> > > > > >> > > > also
> >> > > > > >> > > > > be
> >> > > > > >> > > > > >> > solved by integrating Samza nicely into a Kafka
> >> > > > > distribution
> >> > > > > >> > (such
> >> > > > > >> > > > as
> >> > > > > >> > > > > >> > Confluent's). I'm not against merging projects
> if
> >> we
> >> > > > decide
> >> > > > > >> > that's
> >> > > > > >> > > > the
> >> > > > > >> > > > > >> way
> >> > > > > >> > > > > >> > to go, just pointing out the same goals can
> >> perhaps
> >> > > also
> >> > > > be
> >> > > > > >> > > achieved
> >> > > > > >> > > > > in
> >> > > > > >> > > > > >> > other ways.
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> > - With regard to dropping the YARN dependency:
> are
> >> > you
> >> > > > > >> proposing
> >> > > > > >> > > > that
> >> > > > > >> > > > > >> > Samza doesn't give any help to people wanting to
> >> run
> >> > on
> >> > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> >> > > > > >> > > > > >> > So the docs would basically have a link to
> Slider
> >> and
> >> > > > > nothing
> >> > > > > >> > > else?
> >> > > > > >> > > > Or
> >> > > > > >> > > > > >> > would we maintain integrations with a bunch of
> >> > popular
> >> > > > > >> > deployment
> >> > > > > >> > > > > >> methods
> >> > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts to
> make
> >> > > Samza
> >> > > > > work
> >> > > > > >> > with
> >> > > > > >> > > > > >> Slider)?
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> > I absolutely think it's a good idea to have the
> >> "as a
> >> > > > > library"
> >> > > > > >> > and
> >> > > > > >> > > > > "as a
> >> > > > > >> > > > > >> > process" (using Yi's taxonomy) options for
> people
> >> who
> >> > > > want
> >> > > > > >> them,
> >> > > > > >> > > > but I
> >> > > > > >> > > > > >> > think there should also be a low-friction path
> for
> >> > > common
> >> > > > > "as
> >> > > > > >> a
> >> > > > > >> > > > > service"
> >> > > > > >> > > > > >> > deployment methods, for which we probably need
> to
> >> > > > maintain
> >> > > > > >> > > > > integrations.
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to
> me,
> >> > > > because
> >> > > > > >> Kafka
> >> > > > > >> > > is
> >> > > > > >> > > > > all
> >> > > > > >> > > > > >> > about streams already. Perhaps "Kafka
> >> Transformers"
> >> > or
> >> > > > > "Kafka
> >> > > > > >> > > > Filters"
> >> > > > > >> > > > > >> > would be more apt?
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> > One suggestion: perhaps the core of Samza
> (stream
> >> > > > > >> transformation
> >> > > > > >> > > > with
> >> > > > > >> > > > > >> > state management -- i.e. the "Samza as a
> library"
> >> > bit)
> >> > > > > could
> >> > > > > >> > > become
> >> > > > > >> > > > > >> part of
> >> > > > > >> > > > > >> > Kafka, while higher-level tools such as
> streaming
> >> SQL
> >> > > and
> >> > > > > >> > > > integrations
> >> > > > > >> > > > > >> with
> >> > > > > >> > > > > >> > deployment frameworks remain in a separate
> >> project?
> >> > In
> >> > > > > other
> >> > > > > >> > > words,
> >> > > > > >> > > > > >> Kafka
> >> > > > > >> > > > > >> > would absorb the proven, stable core of Samza,
> >> which
> >> > > > would
> >> > > > > >> > become
> >> > > > > >> > > > the
> >> > > > > >> > > > > >> > "third Kafka client" mentioned early in this
> >> thread.
> >> > > The
> >> > > > > Samza
> >> > > > > >> > > > project
> >> > > > > >> > > > > >> > would then target that third Kafka client as its
> >> base
> >> > > > API,
> >> > > > > and
> >> > > > > >> > the
> >> > > > > >> > > > > >> project
> >> > > > > >> > > > > >> > would be freed up to explore more experimental
> new
> >> > > > > horizons.
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> > Martin
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> >> > > jay.kreps@gmail.com>
> >> > > > > >> wrote:
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> > > Hey Martin,
> >> > > > > >> > > > > >> > >
> >> > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually
> >> don't
> >> > > > think
> >> > > > > it
> >> > > > > >> > ties
> >> > > > > >> > > > our
> >> > > > > >> > > > > >> > hands
> >> > > > > >> > > > > >> > > at all, all it does is refactor things. The
> >> > division
> >> > > of
> >> > > > > >> > > > > >> responsibility is
> >> > > > > >> > > > > >> > > that Samza core is responsible for task
> >> lifecycle,
> >> > > > state,
> >> > > > > >> and
> >> > > > > >> > > > > >> partition
> >> > > > > >> > > > > >> > > management (using the Kafka co-ordinator) but
> >> it is
> >> > > NOT
> >> > > > > >> > > > responsible
> >> > > > > >> > > > > >> for
> >> > > > > >> > > > > >> > > packaging, configuration deployment or
> >> execution of
> >> > > > > >> processes.
> >> > > > > >> > > The
> >> > > > > >> > > > > >> > problem
> >> > > > > >> > > > > >> > > of packaging and starting these processes is
> >> > > > > >> > > > > >> > > framework/environment-specific. This leaves
> >> > > individual
> >> > > > > >> > > frameworks
> >> > > > > >> > > > to
> >> > > > > >> > > > > >> be
> >> > > > > >> > > > > >> > as
> >> > > > > >> > > > > >> > > fancy or vanilla as they like. So you can get
> >> > simple
> >> > > > > >> stateless
> >> > > > > >> > > > > >> support in
> >> > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app
> >> > > > framework
> >> > > > > >> > > (Slider,
> >> > > > > >> > > > > >> > Marathon,
> >> > > > > >> > > > > >> > > etc). These are well known by people and have
> >> nice
> >> > > UIs
> >> > > > > and a
> >> > > > > >> > lot
> >> > > > > >> > > > of
> >> > > > > >> > > > > >> > > flexibility. I don't think they have node
> >> affinity
> >> > > as a
> >> > > > > >> built
> >> > > > > >> > in
> >> > > > > >> > > > > >> option
> >> > > > > >> > > > > >> > > (though I could be wrong). So if we want that
> we
> >> > can
> >> > > > > either
> >> > > > > >> > wait
> >> > > > > >> > > > for
> >> > > > > >> > > > > >> them
> >> > > > > >> > > > > >> > > to add it or do a custom framework to add that
> >> > > feature
> >> > > > > (as
> >> > > > > >> > now).
> >> > > > > >> > > > > >> > Obviously
> >> > > > > >> > > > > >> > > if you manage things with old-school ops tools
> >> > > > > >> > (puppet/chef/etc)
> >> > > > > >> > > > you
> >> > > > > >> > > > > >> get
> >> > > > > >> > > > > >> > > locality easily. The nice thing, though, is
> that
> >> > all
> >> > > > the
> >> > > > > >> samza
> >> > > > > >> > > > > >> "business
> >> > > > > >> > > > > >> > > logic" around partition management and fault
> >> > > tolerance
> >> > > > > is in
> >> > > > > >> > > Samza
> >> > > > > >> > > > > >> core
> >> > > > > >> > > > > >> > so
> >> > > > > >> > > > > >> > > it is shared across frameworks and the
> framework
> >> > > > specific
> >> > > > > >> bit
> >> > > > > >> > is
> >> > > > > >> > > > > just
> >> > > > > >> > > > > >> > > whether it is smart enough to try to get the
> >> same
> >> > > host
> >> > > > > when
> >> > > > > >> a
> >> > > > > >> > > job
> >> > > > > >> > > > is
> >> > > > > >> > > > > >> > > restarted.
> >> > > > > >> > > > > >> > >
> >> > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah I
> >> think
> >> > the
> >> > > > > goal
> >> > > > > >> > would
> >> > > > > >> > > > be
> >> > > > > >> > > > > >> (a)
> >> > > > > >> > > > > >> > > actually get better alignment in user
> >> experience,
> >> > and
> >> > > > (b)
> >> > > > > >> > > express
> >> > > > > >> > > > > >> this in
> >> > > > > >> > > > > >> > > the naming and project branding. Specifically:
> >> > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> >> > > > > "transformation"
> >> > > > > >> api
> >> > > > > >> > > to
> >> > > > > >> > > > be
> >> > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be
> >> able
> >> > to
> >> > > > > explain
> >> > > > > >> > > when
> >> > > > > >> > > > to
> >> > > > > >> > > > > >> use
> >> > > > > >> > > > > >> > > the consumer and when to use the stream
> >> processing
> >> > > > > >> > functionality
> >> > > > > >> > > > and
> >> > > > > >> > > > > >> lead
> >> > > > > >> > > > > >> > > people into that experience.
> >> > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2
> (or
> >> > > > > whatever)
> >> > > > > >> > that
> >> > > > > >> > > > has
> >> > > > > >> > > > > >> both
> >> > > > > >> > > > > >> > > Kafka and the stream processing part and they
> >> > > actually
> >> > > > > work
> >> > > > > >> > > > > together.
> >> > > > > >> > > > > >> > > 3. Unify the programming experience so the
> >> client
> >> > and
> >> > > > > Samza
> >> > > > > >> > api
> >> > > > > >> > > > > share
> >> > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> >> > > > > >> > > > > >> > >
> >> > > > > >> > > > > >> > > I think sub-projects keep separate committers
> >> and
> >> > can
> >> > > > > have a
> >> > > > > >> > > > > separate
> >> > > > > >> > > > > >> > repo,
> >> > > > > >> > > > > >> > > but I'm actually not really sure (I can't
> find a
> >> > > > > definition
> >> > > > > >> > of a
> >> > > > > >> > > > > >> > subproject
> >> > > > > >> > > > > >> > > in Apache).
> >> > > > > >> > > > > >> > >
> >> > > > > >> > > > > >> > > Basically at a high-level you want the
> >> experience
> >> > to
> >> > > > > "feel"
> >> > > > > >> > > like a
> >> > > > > >> > > > > >> single
> >> > > > > >> > > > > >> > > system, not to relatively independent things
> >> that
> >> > are
> >> > > > > kind
> >> > > > > >> of
> >> > > > > >> > > > > >> awkwardly
> >> > > > > >> > > > > >> > > glued together.
> >> > > > > >> > > > > >> > >
> >> > > > > >> > > > > >> > > I think if we did that they having naming or
> >> > branding
> >> > > > > like
> >> > > > > >> > > "kafka
> >> > > > > >> > > > > >> > > streaming" or "kafka streams" or something
> like
> >> > that
> >> > > > > would
> >> > > > > >> > > > actually
> >> > > > > >> > > > > >> do a
> >> > > > > >> > > > > >> > > good job of conveying what it is. I do that
> this
> >> > > would
> >> > > > > help
> >> > > > > >> > > > adoption
> >> > > > > >> > > > > >> > quite
> >> > > > > >> > > > > >> > > a lot as it would correctly convey that using
> >> Kafka
> >> > > > > >> Streaming
> >> > > > > >> > > with
> >> > > > > >> > > > > >> Kafka
> >> > > > > >> > > > > >> > is
> >> > > > > >> > > > > >> > > a fairly seamless experience and Kafka is
> pretty
> >> > > > heavily
> >> > > > > >> > adopted
> >> > > > > >> > > > at
> >> > > > > >> > > > > >> this
> >> > > > > >> > > > > >> > > point.
> >> > > > > >> > > > > >> > >
> >> > > > > >> > > > > >> > > Fwiw we actually considered this model
> >> originally
> >> > > when
> >> > > > > open
> >> > > > > >> > > > sourcing
> >> > > > > >> > > > > >> > Samza,
> >> > > > > >> > > > > >> > > however at that time Kafka was relatively
> >> unknown
> >> > and
> >> > > > we
> >> > > > > >> > decided
> >> > > > > >> > > > not
> >> > > > > >> > > > > >> to
> >> > > > > >> > > > > >> > do
> >> > > > > >> > > > > >> > > it since we felt it would be limiting. From my
> >> > point
> >> > > of
> >> > > > > view
> >> > > > > >> > the
> >> > > > > >> > > > > three
> >> > > > > >> > > > > >> > > things have changed (1) Kafka is now really
> >> heavily
> >> > > > used
> >> > > > > for
> >> > > > > >> > > > stream
> >> > > > > >> > > > > >> > > processing, (2) we learned that abstracting
> out
> >> the
> >> > > > > stream
> >> > > > > >> > well
> >> > > > > >> > > is
> >> > > > > >> > > > > >> > > basically impossible, (3) we learned it is
> >> really
> >> > > hard
> >> > > > to
> >> > > > > >> keep
> >> > > > > >> > > the
> >> > > > > >> > > > > two
> >> > > > > >> > > > > >> > > things feeling like a single product.
> >> > > > > >> > > > > >> > >
> >> > > > > >> > > > > >> > > -Jay
> >> > > > > >> > > > > >> > >
> >> > > > > >> > > > > >> > >
> >> > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
> >> Kleppmann <
> >> > > > > >> > > > > >> martin@kleppmann.com>
> >> > > > > >> > > > > >> > > wrote:
> >> > > > > >> > > > > >> > >
> >> > > > > >> > > > > >> > >> Hi all,
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> > >> Lots of good thoughts here.
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> > >> I agree with the general philosophy of tying
> >> Samza
> >> > > > more
> >> > > > > >> > firmly
> >> > > > > >> > > to
> >> > > > > >> > > > > >> Kafka.
> >> > > > > >> > > > > >> > >> After I spent a while looking at integrating
> >> other
> >> > > > > message
> >> > > > > >> > > > brokers
> >> > > > > >> > > > > >> (e.g.
> >> > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
> >> > > conclusion
> >> > > > > that
> >> > > > > >> > > > > >> > SystemConsumer
> >> > > > > >> > > > > >> > >> tacitly assumes a model so much like Kafka's
> >> that
> >> > > > pretty
> >> > > > > >> much
> >> > > > > >> > > > > nobody
> >> > > > > >> > > > > >> but
> >> > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
> >> perhaps
> >> > an
> >> > > > > >> > exception,
> >> > > > > >> > > > but
> >> > > > > >> > > > > >> it
> >> > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus,
> >> > making
> >> > > > > Samza
> >> > > > > >> > > fully
> >> > > > > >> > > > > >> > dependent
> >> > > > > >> > > > > >> > >> on Kafka acknowledges that the
> >> system-independence
> >> > > was
> >> > > > > >> never
> >> > > > > >> > as
> >> > > > > >> > > > > real
> >> > > > > >> > > > > >> as
> >> > > > > >> > > > > >> > we
> >> > > > > >> > > > > >> > >> perhaps made it out to be. The gains of code
> >> reuse
> >> > > are
> >> > > > > >> real.
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN has
> also
> >> > > always
> >> > > > > been
> >> > > > > >> > > > > >> appealing to
> >> > > > > >> > > > > >> > >> me, for various reasons already mentioned in
> >> this
> >> > > > > thread.
> >> > > > > >> > > > Although
> >> > > > > >> > > > > >> > making
> >> > > > > >> > > > > >> > >> Samza jobs deployable on anything
> >> > > (YARN/Mesos/AWS/etc)
> >> > > > > >> seems
> >> > > > > >> > > > > >> laudable,
> >> > > > > >> > > > > >> > I am
> >> > > > > >> > > > > >> > >> a little concerned that it will restrict us
> to
> >> a
> >> > > > lowest
> >> > > > > >> > common
> >> > > > > >> > > > > >> > denominator.
> >> > > > > >> > > > > >> > >> For example, would host affinity (SAMZA-617)
> >> still
> >> > > be
> >> > > > > >> > possible?
> >> > > > > >> > > > For
> >> > > > > >> > > > > >> jobs
> >> > > > > >> > > > > >> > >> with large amounts of state, I think
> SAMZA-617
> >> > would
> >> > > > be
> >> > > > > a
> >> > > > > >> big
> >> > > > > >> > > > boon,
> >> > > > > >> > > > > >> > since
> >> > > > > >> > > > > >> > >> restoring state off the changelog on every
> >> single
> >> > > > > restart
> >> > > > > >> is
> >> > > > > >> > > > > painful,
> >> > > > > >> > > > > >> > due
> >> > > > > >> > > > > >> > >> to long recovery times. It would be a shame
> if
> >> the
> >> > > > > >> decoupling
> >> > > > > >> > > > from
> >> > > > > >> > > > > >> YARN
> >> > > > > >> > > > > >> > >> made host affinity impossible.
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> > >> Jay, a question about the proposed API for
> >> > > > > instantiating a
> >> > > > > >> > job
> >> > > > > >> > > in
> >> > > > > >> > > > > >> code
> >> > > > > >> > > > > >> > >> (rather than a properties file): when
> >> submitting a
> >> > > job
> >> > > > > to a
> >> > > > > >> > > > > cluster,
> >> > > > > >> > > > > >> is
> >> > > > > >> > > > > >> > the
> >> > > > > >> > > > > >> > >> idea that the instantiation code runs on a
> >> client
> >> > > > > >> somewhere,
> >> > > > > >> > > > which
> >> > > > > >> > > > > >> then
> >> > > > > >> > > > > >> > >> pokes the necessary endpoints on
> >> > YARN/Mesos/AWS/etc?
> >> > > > Or
> >> > > > > >> does
> >> > > > > >> > > that
> >> > > > > >> > > > > >> code
> >> > > > > >> > > > > >> > run
> >> > > > > >> > > > > >> > >> on each container that is part of the job (in
> >> > which
> >> > > > > case,
> >> > > > > >> how
> >> > > > > >> > > > does
> >> > > > > >> > > > > >> the
> >> > > > > >> > > > > >> > job
> >> > > > > >> > > > > >> > >> submission to the cluster work)?
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel right
> >> to
> >> > > make
> >> > > > a
> >> > > > > 1.0
> >> > > > > >> > > > release
> >> > > > > >> > > > > >> > with a
> >> > > > > >> > > > > >> > >> plan for it to be immediately obsolete. So if
> >> this
> >> > > is
> >> > > > > going
> >> > > > > >> > to
> >> > > > > >> > > > > >> happen, I
> >> > > > > >> > > > > >> > >> think it would be more honest to stick with
> 0.*
> >> > > > version
> >> > > > > >> > numbers
> >> > > > > >> > > > > until
> >> > > > > >> > > > > >> > the
> >> > > > > >> > > > > >> > >> library-ified Samza has been implemented, is
> >> > stable
> >> > > > and
> >> > > > > >> > widely
> >> > > > > >> > > > > used.
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> > >> Should the new Samza be a subproject of
> Kafka?
> >> > There
> >> > > > is
> >> > > > > >> > > precedent
> >> > > > > >> > > > > for
> >> > > > > >> > > > > >> > >> tight coupling between different Apache
> >> projects
> >> > > (e.g.
> >> > > > > >> > Curator
> >> > > > > >> > > > and
> >> > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think
> >> > remaining
> >> > > > > >> separate
> >> > > > > >> > > > would
> >> > > > > >> > > > > >> be
> >> > > > > >> > > > > >> > ok.
> >> > > > > >> > > > > >> > >> Even if Samza is fully dependent on Kafka,
> >> there
> >> > is
> >> > > > > enough
> >> > > > > >> > > > > substance
> >> > > > > >> > > > > >> in
> >> > > > > >> > > > > >> > >> Samza that it warrants being a separate
> >> project.
> >> > An
> >> > > > > >> argument
> >> > > > > >> > in
> >> > > > > >> > > > > >> favour
> >> > > > > >> > > > > >> > of
> >> > > > > >> > > > > >> > >> merging would be if we think Kafka has a much
> >> > > stronger
> >> > > > > >> "brand
> >> > > > > >> > > > > >> presence"
> >> > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If
> the
> >> > Kafka
> >> > > > > >> project
> >> > > > > >> > is
> >> > > > > >> > > > > >> willing
> >> > > > > >> > > > > >> > to
> >> > > > > >> > > > > >> > >> endorse Samza as the "official" way of doing
> >> > > stateful
> >> > > > > >> stream
> >> > > > > >> > > > > >> > >> transformations, that would probably have
> much
> >> the
> >> > > > same
> >> > > > > >> > effect
> >> > > > > >> > > as
> >> > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream
> Processors"
> >> or
> >> > > > > suchlike.
> >> > > > > >> > > Close
> >> > > > > >> > > > > >> > >> collaboration between the two projects will
> be
> >> > > needed
> >> > > > in
> >> > > > > >> any
> >> > > > > >> > > > case.
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> > >> From a project management perspective, I
> guess
> >> the
> >> > > > "new
> >> > > > > >> > Samza"
> >> > > > > >> > > > > would
> >> > > > > >> > > > > >> > have
> >> > > > > >> > > > > >> > >> to be developed on a branch alongside ongoing
> >> > > > > maintenance
> >> > > > > >> of
> >> > > > > >> > > the
> >> > > > > >> > > > > >> current
> >> > > > > >> > > > > >> > >> line of development? I think it would be
> >> important
> >> > > to
> >> > > > > >> > continue
> >> > > > > >> > > > > >> > supporting
> >> > > > > >> > > > > >> > >> existing users, and provide a graceful
> >> migration
> >> > > path
> >> > > > to
> >> > > > > >> the
> >> > > > > >> > > new
> >> > > > > >> > > > > >> > version.
> >> > > > > >> > > > > >> > >> Leaving the current versions unsupported and
> >> > forcing
> >> > > > > people
> >> > > > > >> > to
> >> > > > > >> > > > > >> rewrite
> >> > > > > >> > > > > >> > >> their jobs would send a bad signal.
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> > >> Best,
> >> > > > > >> > > > > >> > >> Martin
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> >> > > jay@confluent.io>
> >> > > > > >> wrote:
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> > >>> Hey Garry,
> >> > > > > >> > > > > >> > >>>
> >> > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy
> to
> >> > chat
> >> > > > > more
> >> > > > > >> > about
> >> > > > > >> > > > > this
> >> > > > > >> > > > > >> if
> >> > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
> >> started
> >> > > with
> >> > > > > the
> >> > > > > >> > idea
> >> > > > > >> > > > of
> >> > > > > >> > > > > >> "what
> >> > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> >> ingestion
> >> > > > tool"
> >> > > > > but
> >> > > > > >> > > > > >> ultimately
> >> > > > > >> > > > > >> > we
> >> > > > > >> > > > > >> > >>> kind of came around to the idea that
> ingestion
> >> > and
> >> > > > > >> > > > transformation
> >> > > > > >> > > > > >> had
> >> > > > > >> > > > > >> > >>> pretty different needs and coupling the two
> >> made
> >> > > > things
> >> > > > > >> > hard.
> >> > > > > >> > > > > >> > >>>
> >> > > > > >> > > > > >> > >>> For what it's worth I think copycat (KIP-26)
> >> > > actually
> >> > > > > will
> >> > > > > >> > do
> >> > > > > >> > > > what
> >> > > > > >> > > > > >> you
> >> > > > > >> > > > > >> > >> are
> >> > > > > >> > > > > >> > >>> looking for.
> >> > > > > >> > > > > >> > >>>
> >> > > > > >> > > > > >> > >>> With regard to your point about slider, I
> >> don't
> >> > > > > >> necessarily
> >> > > > > >> > > > > >> disagree.
> >> > > > > >> > > > > >> > >> But I
> >> > > > > >> > > > > >> > >>> think getting good YARN support is quite
> >> doable
> >> > > and I
> >> > > > > >> think
> >> > > > > >> > we
> >> > > > > >> > > > can
> >> > > > > >> > > > > >> make
> >> > > > > >> > > > > >> > >>> that work well. I think the issue this
> >> proposal
> >> > > > solves
> >> > > > > is
> >> > > > > >> > that
> >> > > > > >> > > > > >> > >> technically
> >> > > > > >> > > > > >> > >>> it is pretty hard to support multiple
> cluster
> >> > > > > management
> >> > > > > >> > > systems
> >> > > > > >> > > > > the
> >> > > > > >> > > > > >> > way
> >> > > > > >> > > > > >> > >>> things are now, you need to write an "app
> >> master"
> >> > > or
> >> > > > > >> > > "framework"
> >> > > > > >> > > > > for
> >> > > > > >> > > > > >> > each
> >> > > > > >> > > > > >> > >>> and they are all a little different so
> >> testing is
> >> > > > > really
> >> > > > > >> > hard.
> >> > > > > >> > > > In
> >> > > > > >> > > > > >> the
> >> > > > > >> > > > > >> > >>> absence of this we have been stuck with just
> >> YARN
> >> > > > which
> >> > > > > >> has
> >> > > > > >> > > > > >> fantastic
> >> > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the org,
> >> but
> >> > > zero
> >> > > > > >> > > penetration
> >> > > > > >> > > > > >> > >> elsewhere.
> >> > > > > >> > > > > >> > >>> Given the huge amount of work being put in
> to
> >> > > slider,
> >> > > > > >> > > marathon,
> >> > > > > >> > > > > aws
> >> > > > > >> > > > > >> > >>> tooling, not to mention the umpteen related
> >> > > packaging
> >> > > > > >> > > > technologies
> >> > > > > >> > > > > >> > people
> >> > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> >> > > > cloud-specific
> >> > > > > >> > deploy
> >> > > > > >> > > > > >> tools,
> >> > > > > >> > > > > >> > >> etc)
> >> > > > > >> > > > > >> > >>> I really think it is important to get this
> >> right.
> >> > > > > >> > > > > >> > >>>
> >> > > > > >> > > > > >> > >>> -Jay
> >> > > > > >> > > > > >> > >>>
> >> > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> >> Turkington
> >> > <
> >> > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> >> > > > > >> > > > > >> > >>>
> >> > > > > >> > > > > >> > >>>> Hi all,
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>> I think the question below re does Samza
> >> become
> >> > a
> >> > > > > >> > sub-project
> >> > > > > >> > > > of
> >> > > > > >> > > > > >> Kafka
> >> > > > > >> > > > > >> > >>>> highlights the broader point around
> >> migration.
> >> > > Chris
> >> > > > > >> > mentions
> >> > > > > >> > > > > >> Samza's
> >> > > > > >> > > > > >> > >>>> maturity is heading towards a v1 release
> but
> >> I'm
> >> > > not
> >> > > > > sure
> >> > > > > >> > it
> >> > > > > >> > > > > feels
> >> > > > > >> > > > > >> > >> right to
> >> > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
> >> deprecate
> >> > > most
> >> > > > of
> >> > > > > >> it.
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>> From a selfish perspective I have some guys
> >> who
> >> > > have
> >> > > > > >> > started
> >> > > > > >> > > > > >> working
> >> > > > > >> > > > > >> > >> with
> >> > > > > >> > > > > >> > >>>> Samza and building some new
> >> consumers/producers
> >> > > was
> >> > > > > next
> >> > > > > >> > up.
> >> > > > > >> > > > > Sounds
> >> > > > > >> > > > > >> > like
> >> > > > > >> > > > > >> > >>>> that is absolutely not the direction to
> go. I
> >> > need
> >> > > > to
> >> > > > > >> look
> >> > > > > >> > > into
> >> > > > > >> > > > > the
> >> > > > > >> > > > > >> > KIP
> >> > > > > >> > > > > >> > >> in
> >> > > > > >> > > > > >> > >>>> more detail but for me the attractiveness
> of
> >> > > adding
> >> > > > > new
> >> > > > > >> > Samza
> >> > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all they
> >> were
> >> > > > doing
> >> > > > > was
> >> > > > > >> > > > really
> >> > > > > >> > > > > >> > getting
> >> > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to avoid
> >> > > having
> >> > > > to
> >> > > > > >> > worry
> >> > > > > >> > > > > about
> >> > > > > >> > > > > >> the
> >> > > > > >> > > > > >> > >>>> lifecycle management of external clients.
> If
> >> > there
> >> > > > is
> >> > > > > a
> >> > > > > >> > > generic
> >> > > > > >> > > > > >> Kafka
> >> > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a new
> >> > > connector
> >> > > > > into
> >> > > > > >> > and
> >> > > > > >> > > > > have
> >> > > > > >> > > > > >> a
> >> > > > > >> > > > > >> > >> lot of
> >> > > > > >> > > > > >> > >>>> the heavy lifting re scale and reliability
> >> done
> >> > > for
> >> > > > me
> >> > > > > >> then
> >> > > > > >> > > it
> >> > > > > >> > > > > >> gives
> >> > > > > >> > > > > >> > me
> >> > > > > >> > > > > >> > >> all
> >> > > > > >> > > > > >> > >>>> the pushing new consumers/producers would.
> If
> >> > not
> >> > > > > then it
> >> > > > > >> > > > > >> complicates
> >> > > > > >> > > > > >> > my
> >> > > > > >> > > > > >> > >>>> operational deployments.
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>> Which is similar to my other question with
> >> the
> >> > > > > proposal
> >> > > > > >> --
> >> > > > > >> > if
> >> > > > > >> > > > we
> >> > > > > >> > > > > >> > build a
> >> > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus the
> >> > > requisite
> >> > > > > >> shims
> >> > > > > >> > to
> >> > > > > >> > > > > >> > integrate
> >> > > > > >> > > > > >> > >>>> with Slider etc I suspect the former may
> be a
> >> > lot
> >> > > > more
> >> > > > > >> work
> >> > > > > >> > > > than
> >> > > > > >> > > > > we
> >> > > > > >> > > > > >> > >> think.
> >> > > > > >> > > > > >> > >>>> We may make it much easier for a newcomer
> to
> >> get
> >> > > > > >> something
> >> > > > > >> > > > > running
> >> > > > > >> > > > > >> but
> >> > > > > >> > > > > >> > >>>> having them step up and get a reliable
> >> > production
> >> > > > > >> > deployment
> >> > > > > >> > > > may
> >> > > > > >> > > > > >> still
> >> > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
> >> different
> >> > > > > reasons
> >> > > > > >> > than
> >> > > > > >> > > > > >> today.
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with
> >> > making
> >> > > > the
> >> > > > > >> Samza
> >> > > > > >> > > > > >> dependency
> >> > > > > >> > > > > >> > >> on
> >> > > > > >> > > > > >> > >>>> Kafka much more explicit and I absolutely
> see
> >> > the
> >> > > > > >> benefits
> >> > > > > >> > > in
> >> > > > > >> > > > > the
> >> > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> >> > > > > >> > > > terminologies/abstractions
> >> > > > > >> > > > > >> that
> >> > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library
> would
> >> > > likely
> >> > > > > be a
> >> > > > > >> > very
> >> > > > > >> > > > > nice
> >> > > > > >> > > > > >> > tool
> >> > > > > >> > > > > >> > >> to
> >> > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the
> >> > > concerns
> >> > > > > >> above
> >> > > > > >> > re
> >> > > > > >> > > > the
> >> > > > > >> > > > > >> > >>>> operational side.
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>> Garry
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>> -----Original Message-----
> >> > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales
> [mailto:
> >> > > > > >> > gdfm@apache.org
> >> > > > > >> > > ]
> >> > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> >> > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> >> > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on
> >> Samza
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>> Very interesting thoughts.
> >> > > > > >> > > > > >> > >>>> From outside, I have always perceived Samza
> >> as a
> >> > > > > >> computing
> >> > > > > >> > > > layer
> >> > > > > >> > > > > >> over
> >> > > > > >> > > > > >> > >>>> Kafka.
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>> The question, maybe a bit provocative, is
> >> > "should
> >> > > > > Samza
> >> > > > > >> be
> >> > > > > >> > a
> >> > > > > >> > > > > >> > sub-project
> >> > > > > >> > > > > >> > >>>> of Kafka then?"
> >> > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
> >> separate
> >> > > > project
> >> > > > > >> > with a
> >> > > > > >> > > > > >> separate
> >> > > > > >> > > > > >> > >>>> governance?
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>> Cheers,
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>> --
> >> > > > > >> > > > > >> > >>>> Gianmarco
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> >> > > > > yanfang724@gmail.com>
> >> > > > > >> > > > wrote:
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more
> >> > > tightly.
> >> > > > > >> > Because
> >> > > > > >> > > > > Samza
> >> > > > > >> > > > > >> de
> >> > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
> >> leverage
> >> > > > what
> >> > > > > >> Kafka
> >> > > > > >> > > > has.
> >> > > > > >> > > > > At
> >> > > > > >> > > > > >> > the
> >> > > > > >> > > > > >> > >>>>> same time, Kafka does not need to reinvent
> >> what
> >> > > > Samza
> >> > > > > >> > > already
> >> > > > > >> > > > > >> has. I
> >> > > > > >> > > > > >> > >>>>> also like the idea of separating the
> >> ingestion
> >> > > and
> >> > > > > >> > > > > transformation.
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>> But it is a little difficult for me to
> image
> >> > how
> >> > > > the
> >> > > > > >> Samza
> >> > > > > >> > > > will
> >> > > > > >> > > > > >> look
> >> > > > > >> > > > > >> > >>>> like.
> >> > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> >> > difference
> >> > > > in
> >> > > > > >> terms
> >> > > > > >> > > of
> >> > > > > >> > > > > how
> >> > > > > >> > > > > >> > >>>>> Samza should look like.
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code
> shows
> >> (A
> >> > > > > client of
> >> > > > > >> > > > Kakfa)
> >> > > > > >> > > > > ?
> >> > > > > >> > > > > >> And
> >> > > > > >> > > > > >> > >>>>> user's application code calls this client?
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka
> >> (like
> >> > > > what
> >> > > > > the
> >> > > > > >> > > code
> >> > > > > >> > > > > >> shows),
> >> > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> >> > > > fault-tolerance?
> >> > > > > >> Are
> >> > > > > >> > > they
> >> > > > > >> > > > > >> taken
> >> > > > > >> > > > > >> > >>>>> care by the Kafka broker or other
> mechanism,
> >> > such
> >> > > > as
> >> > > > > >> > "Samza
> >> > > > > >> > > > > >> worker"
> >> > > > > >> > > > > >> > >>>>> (just make up the name) ?
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>> 2. What about other features, such as
> >> > > auto-scaling,
> >> > > > > >> shared
> >> > > > > >> > > > > state,
> >> > > > > >> > > > > >> > >>>>> monitoring?
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is this
> >> what
> >> > > > Chris
> >> > > > > >> > > > suggests?)
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa
> >> and
> >> > > > > produce
> >> > > > > >> to
> >> > > > > >> > > it.
> >> > > > > >> > > > > >> Then it
> >> > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks like
> >> now,
> >> > > > > except it
> >> > > > > >> > > does
> >> > > > > >> > > > > not
> >> > > > > >> > > > > >> > rely
> >> > > > > >> > > > > >> > >>>>> on Yarn anymore.
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it
> leverage
> >> > > Kafka's
> >> > > > > >> > metrics,
> >> > > > > >> > > > > logs,
> >> > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>> Thanks,
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>> Fang, Yan
> >> > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang
> >> Wang <
> >> > > > > >> > > > > wangguoz@gmail.com
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> > >>>> wrote:
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>>> Read through the code example and it
> looks
> >> > good
> >> > > to
> >> > > > > me.
> >> > > > > >> A
> >> > > > > >> > > few
> >> > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable
> runnable
> >> > like:
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> >> > --config-factory=...
> >> > > > > >> > > > > >> > >>>> --config-path=file://...
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>> And this proposal advocate for deploying
> >> Samza
> >> > > > more
> >> > > > > as
> >> > > > > >> > > > embedded
> >> > > > > >> > > > > >> > >>>>>> libraries in user application code
> >> (ignoring
> >> > the
> >> > > > > >> > > terminology
> >> > > > > >> > > > > >> since
> >> > > > > >> > > > > >> > >>>>>> it is not the
> >> > > > > >> > > > > >> > >>>>> same
> >> > > > > >> > > > > >> > >>>>>> as the prototype code):
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>> StreamTask task = new
> >> MyStreamTask(configs);
> >> > > > Thread
> >> > > > > >> > thread
> >> > > > > >> > > =
> >> > > > > >> > > > > new
> >> > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>> I think both of these deployment modes
> are
> >> > > > important
> >> > > > > >> for
> >> > > > > >> > > > > >> different
> >> > > > > >> > > > > >> > >>>>>> types
> >> > > > > >> > > > > >> > >>>>> of
> >> > > > > >> > > > > >> > >>>>>> users. That said, I think making Samza
> >> purely
> >> > > > > >> standalone
> >> > > > > >> > is
> >> > > > > >> > > > > still
> >> > > > > >> > > > > >> > >>>>>> sufficient for either runnable or library
> >> > modes.
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>> Guozhang
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
> >> Kreps <
> >> > > > > >> > > > jay@confluent.io>
> >> > > > > >> > > > > >> > wrote:
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
> >> example, it
> >> > > was
> >> > > > > >> > supposed
> >> > > > > >> > > > to
> >> > > > > >> > > > > >> look
> >> > > > > >> > > > > >> > >>>>>>> like
> >> > > > > >> > > > > >> > >>>>>>> this:
> >> > > > > >> > > > > >> > >>>>>>>
> >> > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> >> > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> >> > > "localhost:4242");
> >> > > > > >> > > > > >> StreamingConfig
> >> > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> >> > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> >> > > "test-topic-2");
> >> > > > > >> > > > > >> > >>>>>>>
> >> > config.processor(ExampleStreamProcessor.class);
> >> > > > > >> > > > > >> > >>>>>>> config.serialization(new
> >> StringSerializer(),
> >> > > new
> >> > > > > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming
> >> > > container =
> >> > > > > new
> >> > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
> >> > > > > >> > > > > >> > >>>>>>>
> >> > > > > >> > > > > >> > >>>>>>> -Jay
> >> > > > > >> > > > > >> > >>>>>>>
> >> > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay
> >> Kreps <
> >> > > > > >> > > > jay@confluent.io
> >> > > > > >> > > > > >
> >> > > > > >> > > > > >> > >>>> wrote:
> >> > > > > >> > > > > >> > >>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> Hey guys,
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> This came out of some conversations
> Chris
> >> > and
> >> > > I
> >> > > > > were
> >> > > > > >> > > having
> >> > > > > >> > > > > >> > >>>>>>>> around
> >> > > > > >> > > > > >> > >>>>>>> whether
> >> > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a
> >> kind
> >> > of
> >> > > > data
> >> > > > > >> > > > ingestion
> >> > > > > >> > > > > >> > >>>>> framework
> >> > > > > >> > > > > >> > >>>>>>> for
> >> > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26
> >> > > > "copycat").
> >> > > > > >> This
> >> > > > > >> > > > kind
> >> > > > > >> > > > > of
> >> > > > > >> > > > > >> > >>>>>> combined
> >> > > > > >> > > > > >> > >>>>>>>> with complaints around config and YARN
> >> and
> >> > the
> >> > > > > >> > discussion
> >> > > > > >> > > > > >> around
> >> > > > > >> > > > > >> > >>>>>>>> how
> >> > > > > >> > > > > >> > >>>>> to
> >> > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> So the thought experiment was, given
> that
> >> > > Samza
> >> > > > > was
> >> > > > > >> > > > basically
> >> > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific, what if
> >> you
> >> > > just
> >> > > > > >> > embraced
> >> > > > > >> > > > > that
> >> > > > > >> > > > > >> > >>>>>>>> and turned it
> >> > > > > >> > > > > >> > >>>>>> into
> >> > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> >> framework
> >> > > and
> >> > > > > more
> >> > > > > >> > > like a
> >> > > > > >> > > > > >> > >>>>>>>> third
> >> > > > > >> > > > > >> > >>>>> Kafka
> >> > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing consumer"
> >> with
> >> > > > state
> >> > > > > >> > > > management
> >> > > > > >> > > > > >> > >>>>>> facilities.
> >> > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a
> complex
> >> > > stream
> >> > > > > >> > > processing
> >> > > > > >> > > > > >> > >>>>>>>> framework
> >> > > > > >> > > > > >> > >>>>>>> this
> >> > > > > >> > > > > >> > >>>>>>>> would actually be a very simple thing,
> >> not
> >> > > much
> >> > > > > more
> >> > > > > >> > > > > >> complicated
> >> > > > > >> > > > > >> > >>>>>>>> to
> >> > > > > >> > > > > >> > >>>>> use
> >> > > > > >> > > > > >> > >>>>>>> or
> >> > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris
> >> said
> >> > > we
> >> > > > > >> thought
> >> > > > > >> > > > about
> >> > > > > >> > > > > >> it
> >> > > > > >> > > > > >> > >>>>>>>> a
> >> > > > > >> > > > > >> > >>>>> lot
> >> > > > > >> > > > > >> > >>>>>> of
> >> > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> >> processing
> >> > > > > systems
> >> > > > > >> > were
> >> > > > > >> > > > > doing)
> >> > > > > >> > > > > >> > >>>>> seemed
> >> > > > > >> > > > > >> > >>>>>>> like
> >> > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output
> data
> >> to
> >> > > and
> >> > > > > from
> >> > > > > >> > the
> >> > > > > >> > > > > stream
> >> > > > > >> > > > > >> > >>>>>>>> processing. But when we actually looked
> >> into
> >> > > how
> >> > > > > that
> >> > > > > >> > > would
> >> > > > > >> > > > > >> > >>>>>>>> work,
> >> > > > > >> > > > > >> > >>>>> Samza
> >> > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion
> >> > framework
> >> > > > > for a
> >> > > > > >> > > bunch
> >> > > > > >> > > > of
> >> > > > > >> > > > > >> > >>>>> reasons.
> >> > > > > >> > > > > >> > >>>>>> To
> >> > > > > >> > > > > >> > >>>>>>>> really do that right you need a pretty
> >> > > different
> >> > > > > >> > internal
> >> > > > > >> > > > > data
> >> > > > > >> > > > > >> > >>>>>>>> model
> >> > > > > >> > > > > >> > >>>>>> and
> >> > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split them
> >> and
> >> > had
> >> > > > an
> >> > > > > api
> >> > > > > >> > for
> >> > > > > >> > > > > Kafka
> >> > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26)
> and a
> >> > > > separate
> >> > > > > >> api
> >> > > > > >> > > for
> >> > > > > >> > > > > >> Kafka
> >> > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> This would also allow really embracing
> >> the
> >> > > same
> >> > > > > >> > > terminology
> >> > > > > >> > > > > and
> >> > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about the
> >> current
> >> > > > > state is
> >> > > > > >> > > that
> >> > > > > >> > > > > the
> >> > > > > >> > > > > >> > >>>>>>>> two
> >> > > > > >> > > > > >> > >>>>>>> systems
> >> > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology
> like
> >> > > > "stream"
> >> > > > > vs
> >> > > > > >> > > > "topic"
> >> > > > > >> > > > > >> and
> >> > > > > >> > > > > >> > >>>>>>> different
> >> > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means you
> >> kind
> >> > > of
> >> > > > > have
> >> > > > > >> to
> >> > > > > >> > > > learn
> >> > > > > >> > > > > >> > >>>>>>>> Kafka's
> >> > > > > >> > > > > >> > >>>>>>> way,
> >> > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly different
> >> way,
> >> > > then
> >> > > > > kind
> >> > > > > >> of
> >> > > > > >> > > > > >> > >>>>>>>> understand
> >> > > > > >> > > > > >> > >>>>> how
> >> > > > > >> > > > > >> > >>>>>>> they
> >> > > > > >> > > > > >> > >>>>>>>> map to each other, which having walked
> a
> >> few
> >> > > > > people
> >> > > > > >> > > through
> >> > > > > >> > > > > >> this
> >> > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to
> get.
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of
> time
> >> on
> >> > > > > >> airplanes I
> >> > > > > >> > > > > hacked
> >> > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
> >> incomplete
> >> > > > > prototype
> >> > > > > >> of
> >> > > > > >> > > > what
> >> > > > > >> > > > > >> > >>>>>>>> this would
> >> > > > > >> > > > > >> > >>>>> look
> >> > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously
> dumped
> >> > into
> >> > > > > Kafka
> >> > > > > >> as
> >> > > > > >> > > it
> >> > > > > >> > > > > >> > >>>>>>>> required a
> >> > > > > >> > > > > >> > >>>>>> few
> >> > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is
> the
> >> > code:
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > >
> >> > > > > >> >
> >> > > > >
> >> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >> > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I just
> >> > > > liberally
> >> > > > > >> > renamed
> >> > > > > >> > > > > >> > >>>>>>>> everything
> >> > > > > >> > > > > >> > >>>>> to
> >> > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no
> regard
> >> > for
> >> > > > > >> > > > compatibility.
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> To use this would be something like
> this:
> >> > > > > >> > > > > >> > >>>>>>>> Properties props = new Properties();
> >> > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> >> > > > "localhost:4242");
> >> > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> >> > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> >> > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> >> > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> >> > > > > >> > > > > >> config.processor(ExampleStreamProcessor.class);
> >> > > > > >> > > > > >> > >>>>>>> config.serialization(new
> >> > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> >> > StringDeserializer());
> >> > > > > >> > > > KafkaStreaming
> >> > > > > >> > > > > >> > >>>>>> container =
> >> > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> >> container.run();
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> >> > > SamzaContainer;
> >> > > > > >> > > > > StreamProcessor
> >> > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> So rather than putting all the class
> >> names
> >> > in
> >> > > a
> >> > > > > file
> >> > > > > >> > and
> >> > > > > >> > > > then
> >> > > > > >> > > > > >> > >>>>>>>> having
> >> > > > > >> > > > > >> > >>>>>> the
> >> > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
> >> > > > instantiate
> >> > > > > the
> >> > > > > >> > > > > container
> >> > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced over
> >> > > however
> >> > > > > many
> >> > > > > >> > > > > instances
> >> > > > > >> > > > > >> > >>>>>>>> of
> >> > > > > >> > > > > >> > >>>>> this
> >> > > > > >> > > > > >> > >>>>>>> are
> >> > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance
> >> dies,
> >> > > new
> >> > > > > >> tasks
> >> > > > > >> > > are
> >> > > > > >> > > > > >> added
> >> > > > > >> > > > > >> > >>>>>>>> to
> >> > > > > >> > > > > >> > >>>>> the
> >> > > > > >> > > > > >> > >>>>>>>> existing containers without shutting
> them
> >> > > down).
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> We would provide some glue for running
> >> this
> >> > > > stuff
> >> > > > > in
> >> > > > > >> > YARN
> >> > > > > >> > > > via
> >> > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS
> using
> >> > some
> >> > > > of
> >> > > > > >> their
> >> > > > > >> > > > tools
> >> > > > > >> > > > > >> > >>>>>>>> but from the
> >> > > > > >> > > > > >> > >>>>>> point
> >> > > > > >> > > > > >> > >>>>>>> of
> >> > > > > >> > > > > >> > >>>>>>>> view of these frameworks these stream
> >> > > processing
> >> > > > > jobs
> >> > > > > >> > are
> >> > > > > >> > > > > just
> >> > > > > >> > > > > >> > >>>>>> stateless
> >> > > > > >> > > > > >> > >>>>>>>> services that can come and go and
> expand
> >> and
> >> > > > > contract
> >> > > > > >> > at
> >> > > > > >> > > > > will.
> >> > > > > >> > > > > >> > >>>>>>>> There
> >> > > > > >> > > > > >> > >>>>> is
> >> > > > > >> > > > > >> > >>>>>>> no
> >> > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it
> >> would
> >> > > get
> >> > > > > >> larger
> >> > > > > >> > > if
> >> > > > > >> > > > we
> >> > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly larger.
> We
> >> > > really
> >> > > > > do
> >> > > > > >> > get a
> >> > > > > >> > > > ton
> >> > > > > >> > > > > >> > >>>>>>>> of
> >> > > > > >> > > > > >> > >>>>>>> leverage
> >> > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> >> > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> >> delegated
> >> > to
> >> > > > the
> >> > > > > >> new
> >> > > > > >> > > > > >> consumer.
> >> > > > > >> > > > > >> > >>>>> This
> >> > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> >> management
> >> > > > > strategy
> >> > > > > >> > > > > available
> >> > > > > >> > > > > >> > >>>>>>>> to
> >> > > > > >> > > > > >> > >>>>>> Kafka
> >> > > > > >> > > > > >> > >>>>>>>>  consumer is also available to Samza
> (and
> >> > vice
> >> > > > > versa)
> >> > > > > >> > and
> >> > > > > >> > > > > with
> >> > > > > >> > > > > >> > >>>>>>>> the
> >> > > > > >> > > > > >> > >>>>>>> exact
> >> > > > > >> > > > > >> > >>>>>>>>  same configs.
> >> > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as state
> >> reuse
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
> >> thought
> >> > > > > >> provoking.
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> -Jay
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris
> >> > > > Riccomini <
> >> > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> >> > > > > >> > > > > >> > >>>>>>>> wrote:
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Hey all,
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with Samza
> >> > > > engineers
> >> > > > > at
> >> > > > > >> > > > LinkedIn
> >> > > > > >> > > > > >> > >>>>>>>>> and
> >> > > > > >> > > > > >> > >>>>>>> Confluent
> >> > > > > >> > > > > >> > >>>>>>>>> and we came up with a few observations
> >> and
> >> > > > would
> >> > > > > >> like
> >> > > > > >> > to
> >> > > > > >> > > > > >> > >>>>>>>>> propose
> >> > > > > >> > > > > >> > >>>>> some
> >> > > > > >> > > > > >> > >>>>>>>>> changes.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I want
> >> to
> >> > > call
> >> > > > > out
> >> > > > > >> > about
> >> > > > > >> > > > > >> > >>>>>>>>> Samza's
> >> > > > > >> > > > > >> > >>>>>> design,
> >> > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic
> >> > > deployment
> >> > > > > >> system.
> >> > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> >> > > > > >> > > > > >> > >>>>>>>>> * Samza's
> SystemConsumer/SystemProducer
> >> and
> >> > > > > Kafka's
> >> > > > > >> > > > consumer
> >> > > > > >> > > > > >> > >>>>>>>>> APIs
> >> > > > > >> > > > > >> > >>>>> are
> >> > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
> >> problems.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> All three of these issues are related,
> >> but
> >> > > I'll
> >> > > > > >> > address
> >> > > > > >> > > > them
> >> > > > > >> > > > > >> in
> >> > > > > >> > > > > >> > >>>>> order.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Deployment
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a
> >> > > dynamic
> >> > > > > >> > > deployment
> >> > > > > >> > > > > >> > >>>>>>>>> scheduler
> >> > > > > >> > > > > >> > >>>>>> such
> >> > > > > >> > > > > >> > >>>>>>>>> as
> >> > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially
> >> built
> >> > > > Samza,
> >> > > > > we
> >> > > > > >> > bet
> >> > > > > >> > > > that
> >> > > > > >> > > > > >> > >>>>>>>>> there
> >> > > > > >> > > > > >> > >>>>>> would
> >> > > > > >> > > > > >> > >>>>>>>>> be
> >> > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area, and
> we
> >> > could
> >> > > > > >> support
> >> > > > > >> > > > them,
> >> > > > > >> > > > > >> and
> >> > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > >> > > > > >> > >>>>>> rest
> >> > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there are
> >> many
> >> > > > > >> variations.
> >> > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> >> > > > > >> > > > > >> > >>>>>> many
> >> > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start
> their
> >> > > > > processors
> >> > > > > >> > like
> >> > > > > >> > > > > normal
> >> > > > > >> > > > > >> > >>>>>>>>> Java processes, and use traditional
> >> > > deployment
> >> > > > > >> scripts
> >> > > > > >> > > > such
> >> > > > > >> > > > > as
> >> > > > > >> > > > > >> > >>>>>>>>> Fabric,
> >> > > > > >> > > > > >> > >>>>>> Chef,
> >> > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
> >> system
> >> > on
> >> > > > > users
> >> > > > > >> > makes
> >> > > > > >> > > > the
> >> > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really painful
> >> for
> >> > > first
> >> > > > > time
> >> > > > > >> > > > users.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement
> was
> >> > also
> >> > > a
> >> > > > > bit
> >> > > > > >> of
> >> > > > > >> > a
> >> > > > > >> > > > > >> > >>>>>>>>> mis-fire
> >> > > > > >> > > > > >> > >>>>>> because
> >> > > > > >> > > > > >> > >>>>>>>>> of
> >> > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between
> >> the
> >> > > > > nature of
> >> > > > > >> > > batch
> >> > > > > >> > > > > >> jobs
> >> > > > > >> > > > > >> > >>>>>>>>> and
> >> > > > > >> > > > > >> > >>>>>>> stream
> >> > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> >> > conscious
> >> > > > > effort
> >> > > > > >> to
> >> > > > > >> > > > favor
> >> > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > >> > > > > >> > >>>>>> Hadoop
> >> > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things,
> since
> >> it
> >> > > > worked
> >> > > > > >> and
> >> > > > > >> > > was
> >> > > > > >> > > > > well
> >> > > > > >> > > > > >> > >>>>>>> understood.
> >> > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that
> batch
> >> > jobs
> >> > > > > have a
> >> > > > > >> > > > definite
> >> > > > > >> > > > > >> > >>>>>> beginning,
> >> > > > > >> > > > > >> > >>>>>>>>> and
> >> > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't
> >> > > > (usually).
> >> > > > > >> This
> >> > > > > >> > > > leads
> >> > > > > >> > > > > to
> >> > > > > >> > > > > >> > >>>>>>>>> a
> >> > > > > >> > > > > >> > >>>>> much
> >> > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream
> >> > > > processors.
> >> > > > > >> You
> >> > > > > >> > > > > >> basically
> >> > > > > >> > > > > >> > >>>>>>>>> just
> >> > > > > >> > > > > >> > >>>>>>> need
> >> > > > > >> > > > > >> > >>>>>>>>> to find a place to start the
> processor,
> >> and
> >> > > > start
> >> > > > > >> it.
> >> > > > > >> > > The
> >> > > > > >> > > > > way
> >> > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no
> >> > concept
> >> > > > of
> >> > > > > a
> >> > > > > >> > > cluster
> >> > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> >> > > > > >> > > > > >> > >>>>>> add
> >> > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with
> coupling
> >> > > Samza
> >> > > > > with
> >> > > > > >> a
> >> > > > > >> > > > > >> scheduler
> >> > > > > >> > > > > >> > >>>>>>>>> is
> >> > > > > >> > > > > >> > >>>>>> that
> >> > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to
> handle
> >> > > > > deployment.
> >> > > > > >> > > This
> >> > > > > >> > > > > >> pulls
> >> > > > > >> > > > > >> > >>>>>>>>> in a
> >> > > > > >> > > > > >> > >>>>>>> bunch
> >> > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> >> > distribution
> >> > > > > (config
> >> > > > > >> > > > > stream),
> >> > > > > >> > > > > >> > >>>>>>>>> shell
> >> > > > > >> > > > > >> > >>>>>>> scrips
> >> > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging
> >> (all
> >> > > the
> >> > > > > .tgz
> >> > > > > >> > > > stuff),
> >> > > > > >> > > > > >> etc.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
> >> > > deployment
> >> > > > > was
> >> > > > > >> to
> >> > > > > >> > > > > support
> >> > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> >> > locality,
> >> > > > you
> >> > > > > >> need
> >> > > > > >> > to
> >> > > > > >> > > > put
> >> > > > > >> > > > > >> > >>>>>>>>> your
> >> > > > > >> > > > > >> > >>>>>> processors
> >> > > > > >> > > > > >> > >>>>>>>>> close to the data they're processing.
> >> Upon
> >> > > > > further
> >> > > > > >> > > > > >> > >>>>>>>>> investigation,
> >> > > > > >> > > > > >> > >>>>>>> though,
> >> > > > > >> > > > > >> > >>>>>>>>> this feature is not that beneficial.
> >> There
> >> > is
> >> > > > > some
> >> > > > > >> > good
> >> > > > > >> > > > > >> > >>>>>>>>> discussion
> >> > > > > >> > > > > >> > >>>>>> about
> >> > > > > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335.
> >> Again,
> >> > we
> >> > > > > took
> >> > > > > >> the
> >> > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> >> > > > > >> > > > > >> > >>>>>> path,
> >> > > > > >> > > > > >> > >>>>>>>>> but
> >> > > > > >> > > > > >> > >>>>>>>>> there are some fundamental differences
> >> > > between
> >> > > > > HDFS
> >> > > > > >> > and
> >> > > > > >> > > > > Kafka.
> >> > > > > >> > > > > >> > >>>>>>>>> HDFS
> >> > > > > >> > > > > >> > >>>>>> has
> >> > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions.
> This
> >> > > leads
> >> > > > to
> >> > > > > >> less
> >> > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
> >> > processors
> >> > > > on
> >> > > > > top
> >> > > > > >> > of
> >> > > > > >> > > > > Kafka.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a crutch.
> >> > Samza
> >> > > > > doesn't
> >> > > > > >> > > have
> >> > > > > >> > > > > any
> >> > > > > >> > > > > >> > >>>>>>>>> built
> >> > > > > >> > > > > >> > >>>>> in
> >> > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
> >> depends
> >> > on
> >> > > > the
> >> > > > > >> > > dynamic
> >> > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to handle
> >> > > restarts
> >> > > > > >> when a
> >> > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> >> > > > > >> > > > > >> > >>>>>>> made
> >> > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a
> standalone
> >> > Samza
> >> > > > > >> > container
> >> > > > > >> > > > > >> > >>>> (SAMZA-516).
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Pluggability
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is good,
> but
> >> I
> >> > > think
> >> > > > > that
> >> > > > > >> > > we've
> >> > > > > >> > > > > >> gone
> >> > > > > >> > > > > >> > >>>>>>>>> too
> >> > > > > >> > > > > >> > >>>>>> far
> >> > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> >> > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> >> > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> >> > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> >> > > (SystemConsumer,
> >> > > > > >> > > > > SystemProducer,
> >> > > > > >> > > > > >> > >>>> etc).
> >> > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> >> > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> >> > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about
> >> every
> >> > > > > >> component
> >> > > > > >> > > > > >> > >>>>> (MessageChooser,
> >> > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> >> > ConfigRewriter,
> >> > > > > etc).
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> >> forgotten,
> >> > as
> >> > > > > well.
> >> > > > > >> > Some
> >> > > > > >> > > > of
> >> > > > > >> > > > > >> > >>>>>>>>> these
> >> > > > > >> > > > > >> > >>>>> are
> >> > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not to
> be.
> >> > This
> >> > > > all
> >> > > > > >> comes
> >> > > > > >> > > at
> >> > > > > >> > > > a
> >> > > > > >> > > > > >> cost:
> >> > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is making
> it
> >> > > harder
> >> > > > > for
> >> > > > > >> > our
> >> > > > > >> > > > > users
> >> > > > > >> > > > > >> > >>>>>>>>> to
> >> > > > > >> > > > > >> > >>>>> pick
> >> > > > > >> > > > > >> > >>>>>> up
> >> > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also
> >> makes
> >> > > it
> >> > > > > >> > difficult
> >> > > > > >> > > > for
> >> > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about what
> >> the
> >> > > > > >> > > characteristics
> >> > > > > >> > > > of
> >> > > > > >> > > > > >> > >>>>>>>>> the container (since the
> characteristics
> >> > > change
> >> > > > > >> > > depending
> >> > > > > >> > > > on
> >> > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are most
> >> > visible
> >> > > > in
> >> > > > > the
> >> > > > > >> > > > System
> >> > > > > >> > > > > >> APIs.
> >> > > > > >> > > > > >> > >>>>> What
> >> > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be functional
> >> is
> >> > > Kafka
> >> > > > > as
> >> > > > > >> its
> >> > > > > >> > > > > >> > >>>>>>>>> transport
> >> > > > > >> > > > > >> > >>>>>> layer.
> >> > > > > >> > > > > >> > >>>>>>>>> But
> >> > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use
> cases
> >> > into
> >> > > > one
> >> > > > > >> API:
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> >> > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> The current System API supports both
> of
> >> > these
> >> > > > use
> >> > > > > >> > cases.
> >> > > > > >> > > > The
> >> > > > > >> > > > > >> > >>>>>>>>> problem
> >> > > > > >> > > > > >> > >>>>>> is,
> >> > > > > >> > > > > >> > >>>>>>>>> we
> >> > > > > >> > > > > >> > >>>>>>>>> actually want different features for
> >> each
> >> > use
> >> > > > > case.
> >> > > > > >> By
> >> > > > > >> > > > > >> papering
> >> > > > > >> > > > > >> > >>>>>>>>> over
> >> > > > > >> > > > > >> > >>>>>>> these
> >> > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a single
> >> API,
> >> > > > we've
> >> > > > > >> > > > introduced
> >> > > > > >> > > > > a
> >> > > > > >> > > > > >> > >>>>>>>>> ton of
> >> > > > > >> > > > > >> > >>>>>>> leaky
> >> > > > > >> > > > > >> > >>>>>>>>> abstractions.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like in
> >> (2)
> >> > is
> >> > > to
> >> > > > > have
> >> > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
> >> offsets
> >> > > > (like
> >> > > > > >> > Kafka).
> >> > > > > >> > > > > This
> >> > > > > >> > > > > >> > >>>>>>>>> would be at odds
> >> > > > > >> > > > > >> > >>>>> with
> >> > > > > >> > > > > >> > >>>>>>> (1),
> >> > > > > >> > > > > >> > >>>>>>>>> though, since different systems have
> >> > > different
> >> > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> >> > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the
> mailing
> >> > list
> >> > > > and
> >> > > > > >> the
> >> > > > > >> > > SQL
> >> > > > > >> > > > > >> JIRAs
> >> > > > > >> > > > > >> > >>>>> about
> >> > > > > >> > > > > >> > >>>>>>> the
> >> > > > > >> > > > > >> > >>>>>>>>> need for this.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> >> > replayability.
> >> > > > > Kafka
> >> > > > > >> > > allows
> >> > > > > >> > > > us
> >> > > > > >> > > > > >> to
> >> > > > > >> > > > > >> > >>>>> rewind
> >> > > > > >> > > > > >> > >>>>>>>>> when
> >> > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other systems
> >> > don't.
> >> > > In
> >> > > > > some
> >> > > > > >> > > > cases,
> >> > > > > >> > > > > >> > >>>>>>>>> systems
> >> > > > > >> > > > > >> > >>>>>>> return
> >> > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> >> > > > > >> WikipediaSystemConsumer)
> >> > > > > >> > > > > because
> >> > > > > >> > > > > >> > >>>>>>>>> they
> >> > > > > >> > > > > >> > >>>>>> have
> >> > > > > >> > > > > >> > >>>>>>> no
> >> > > > > >> > > > > >> > >>>>>>>>> offsets.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka
> >> > > supports
> >> > > > > >> > > > > partitioning,
> >> > > > > >> > > > > >> > >>>>>>>>> but
> >> > > > > >> > > > > >> > >>>>> many
> >> > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by
> having a
> >> > > single
> >> > > > > >> > > partition
> >> > > > > >> > > > > for
> >> > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other systems
> >> model
> >> > > > > >> partitioning
> >> > > > > >> > > > > >> > >>>> differently (e.g.
> >> > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a
> >> mess.
> >> > > > > Creating
> >> > > > > >> > > streams
> >> > > > > >> > > > > in
> >> > > > > >> > > > > >> a
> >> > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
> >> impossible.
> >> > As
> >> > > is
> >> > > > > >> > modeling
> >> > > > > >> > > > > >> > >>>>>>>>> metadata
> >> > > > > >> > > > > >> > >>>>> for
> >> > > > > >> > > > > >> > >>>>>>> the
> >> > > > > >> > > > > >> > >>>>>>>>> system (replication factor,
> partitions,
> >> > > > location,
> >> > > > > >> > etc).
> >> > > > > >> > > > The
> >> > > > > >> > > > > >> > >>>>>>>>> list
> >> > > > > >> > > > > >> > >>>>> goes
> >> > > > > >> > > > > >> > >>>>>>> on.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing
> Samza,
> >> > > > Kafka's
> >> > > > > >> > > consumer
> >> > > > > >> > > > > and
> >> > > > > >> > > > > >> > >>>>> producer
> >> > > > > >> > > > > >> > >>>>>>>>> APIs
> >> > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On
> >> the
> >> > > > > >> > consumer-side,
> >> > > > > >> > > > you
> >> > > > > >> > > > > >> > >>>>>>>>> had two
> >> > > > > >> > > > > >> > >>>>>>>>> options: use the high level consumer,
> or
> >> > the
> >> > > > > simple
> >> > > > > >> > > > > consumer.
> >> > > > > >> > > > > >> > >>>>>>>>> The
> >> > > > > >> > > > > >> > >>>>>>> problem
> >> > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was that
> it
> >> > > > > controlled
> >> > > > > >> > your
> >> > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and
> the
> >> > order
> >> > > > in
> >> > > > > >> which
> >> > > > > >> > > you
> >> > > > > >> > > > > >> > >>>>>>>>> received messages. The
> >> > > > > >> > > > > >> > >>>>> problem
> >> > > > > >> > > > > >> > >>>>>>>>> with
> >> > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not
> >> > simple.
> >> > > > It's
> >> > > > > >> > basic.
> >> > > > > >> > > > You
> >> > > > > >> > > > > >> > >>>>>>>>> end up
> >> > > > > >> > > > > >> > >>>>>>> having
> >> > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level
> >> stuff
> >> > > that
> >> > > > > you
> >> > > > > >> > > > > shouldn't.
> >> > > > > >> > > > > >> > >>>>>>>>> We
> >> > > > > >> > > > > >> > >>>>>> spent a
> >> > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> >> > > KafkaSystemConsumer
> >> > > > > very
> >> > > > > >> > > > robust.
> >> > > > > >> > > > > >> It
> >> > > > > >> > > > > >> > >>>>>>>>> also allows us to support some cool
> >> > features:
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
> >> > > > > prioritization.
> >> > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
> >> assignment
> >> > to
> >> > > > > support
> >> > > > > >> > > > joins,
> >> > > > > >> > > > > >> > >>>>>>>>> global
> >> > > > > >> > > > > >> > >>>>>> state
> >> > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> >> > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> >> checkpointing.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time is
> >> that
> >> > > > these
> >> > > > > >> > > features
> >> > > > > >> > > > > >> > >>>>>>>>> should
> >> > > > > >> > > > > >> > >>>>>>> actually
> >> > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers
> >> (not
> >> > > just
> >> > > > > >> Samza
> >> > > > > >> > > > stream
> >> > > > > >> > > > > >> > >>>>>> processors)
> >> > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like joins
> >> and
> >> > > > > partition
> >> > > > > >> > > > > >> > >>>>>>>>> assignment. The
> >> > > > > >> > > > > >> > >>>>>>> Kafka
> >> > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> >> conclusion.
> >> > > > > They're
> >> > > > > >> > > adding
> >> > > > > >> > > > a
> >> > > > > >> > > > > >> ton
> >> > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
> >> consumer
> >> > > > > >> > > implementation.
> >> > > > > >> > > > > To a
> >> > > > > >> > > > > >> > >>>>>>>>> large extent,
> >> > > > > >> > > > > >> > >>>>> it's
> >> > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've already
> >> done
> >> > in
> >> > > > > Samza.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking
> a
> >> > very
> >> > > > > similar
> >> > > > > >> > > > > approach
> >> > > > > >> > > > > >> > >>>>>>>>> to
> >> > > > > >> > > > > >> > >>>>>> Samza's
> >> > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation
> >> for
> >> > > > > handling
> >> > > > > >> > > offset
> >> > > > > >> > > > > >> > >>>>>> checkpointing.
> >> > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
> >> management
> >> > > > feature
> >> > > > > >> > stores
> >> > > > > >> > > > > >> offset
> >> > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you
> >> to
> >> > > fetch
> >> > > > > them
> >> > > > > >> > > from
> >> > > > > >> > > > > the
> >> > > > > >> > > > > >> > >>>>>>>>> broker.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste,
> since
> >> we
> >> > > > could
> >> > > > > >> have
> >> > > > > >> > > > shared
> >> > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > >> > > > > >> > >>>>> work
> >> > > > > >> > > > > >> > >>>>>> if
> >> > > > > >> > > > > >> > >>>>>>>>> it
> >> > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the
> get-go.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Vision
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather
> radical
> >> > > > > proposal.
> >> > > > > >> > Samza
> >> > > > > >> > > > is
> >> > > > > >> > > > > >> > >>>>> relatively
> >> > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to
> say
> >> > that
> >> > > > > we're
> >> > > > > >> > > near a
> >> > > > > >> > > > > 1.0
> >> > > > > >> > > > > >> > >>>>>> release.
> >> > > > > >> > > > > >> > >>>>>>>>> I'd
> >> > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what
> we've
> >> > > > learned,
> >> > > > > and
> >> > > > > >> > > begin
> >> > > > > >> > > > > >> > >>>>>>>>> thinking
> >> > > > > >> > > > > >> > >>>>>>> about
> >> > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change
> >> if
> >> > we
> >> > > > were
> >> > > > > >> > > starting
> >> > > > > >> > > > > >> from
> >> > > > > >> > > > > >> > >>>>>> scratch?
> >> > > > > >> > > > > >> > >>>>>>>>> My
> >> > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only*
> way
> >> to
> >> > > run
> >> > > > > Samza
> >> > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
> >> > > > dependences
> >> > > > > on
> >> > > > > >> > > YARN,
> >> > > > > >> > > > > >> Mesos,
> >> > > > > >> > > > > >> > >>>> etc.
> >> > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support
> >> only
> >> > > Kafka
> >> > > > > as
> >> > > > > >> the
> >> > > > > >> > > > > stream
> >> > > > > >> > > > > >> > >>>>>> processing
> >> > > > > >> > > > > >> > >>>>>>>>> layer.
> >> > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
> >> > > > > >> serialization,
> >> > > > > >> > > and
> >> > > > > >> > > > > >> > >>>>>>>>> config
> >> > > > > >> > > > > >> > >>>>>>> systems,
> >> > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues that
> I
> >> > > > outlined
> >> > > > > >> > above.
> >> > > > > >> > > It
> >> > > > > >> > > > > >> > >>>>>>>>> should
> >> > > > > >> > > > > >> > >>>>> also
> >> > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> >> > > dramatically.
> >> > > > > >> > > Supporting
> >> > > > > >> > > > > >> only
> >> > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow
> Samza
> >> to
> >> > be
> >> > > > > >> executed
> >> > > > > >> > > on
> >> > > > > >> > > > > YARN
> >> > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> >> > > Marathon/Aurora),
> >> > > > or
> >> > > > > >> most
> >> > > > > >> > > > other
> >> > > > > >> > > > > >> > >>>>>>>>> in-house
> >> > > > > >> > > > > >> > >>>>>>> deployment
> >> > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a lot
> >> easier
> >> > > for
> >> > > > > new
> >> > > > > >> > > users.
> >> > > > > >> > > > > >> > >>>>>>>>> Imagine
> >> > > > > >> > > > > >> > >>>>>>> having
> >> > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN.
> >> The
> >> > > drop
> >> > > > > in
> >> > > > > >> > > mailing
> >> > > > > >> > > > > >> list
> >> > > > > >> > > > > >> > >>>>>> traffic
> >> > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue
> >> to
> >> > me.
> >> > > > The
> >> > > > > >> > > reality
> >> > > > > >> > > > > is,
> >> > > > > >> > > > > >> > >>>>> everyone
> >> > > > > >> > > > > >> > >>>>>>>>> that
> >> > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with
> Kafka.
> >> We
> >> > > > > basically
> >> > > > > >> > > > require
> >> > > > > >> > > > > >> it
> >> > > > > >> > > > > >> > >>>>>> already
> >> > > > > >> > > > > >> > >>>>>>> in
> >> > > > > >> > > > > >> > >>>>>>>>> order for most features to work. Those
> >> that
> >> > > are
> >> > > > > >> using
> >> > > > > >> > > > other
> >> > > > > >> > > > > >> > >>>>>>>>> systems
> >> > > > > >> > > > > >> > >>>>>> are
> >> > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into
> Kafka
> >> > (1),
> >> > > > and
> >> > > > > >> then
> >> > > > > >> > > > they
> >> > > > > >> > > > > do
> >> > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is
> already
> >> > > > > discussion (
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > >
> >> > > > > >> >
> >> > > > >
> >> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >> > > > > >> > > > > >> > >>>>> 767
> >> > > > > >> > > > > >> > >>>>>>>>> )
> >> > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka
> >> > > extremely
> >> > > > > >> easy.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple with
> >> Kafka,
> >> > > we
> >> > > > > can
> >> > > > > >> > > > leverage
> >> > > > > >> > > > > a
> >> > > > > >> > > > > >> > >>>>>>>>> ton of
> >> > > > > >> > > > > >> > >>>>>>> their
> >> > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to
> maintain
> >> > our
> >> > > > own
> >> > > > > >> > config,
> >> > > > > >> > > > > >> > >>>>>>>>> metrics,
> >> > > > > >> > > > > >> > >>>>> etc.
> >> > > > > >> > > > > >> > >>>>>>> We
> >> > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries, and
> >> make
> >> > > them
> >> > > > > >> > better.
> >> > > > > >> > > > This
> >> > > > > >> > > > > >> > >>>>>>>>> will
> >> > > > > >> > > > > >> > >>>>> also
> >> > > > > >> > > > > >> > >>>>>>>>> allow us to share the
> consumer/producer
> >> > APIs,
> >> > > > and
> >> > > > > >> will
> >> > > > > >> > > let
> >> > > > > >> > > > > us
> >> > > > > >> > > > > >> > >>>>> leverage
> >> > > > > >> > > > > >> > >>>>>>>>> their offset management and partition
> >> > > > management,
> >> > > > > >> > rather
> >> > > > > >> > > > > than
> >> > > > > >> > > > > >> > >>>>>>>>> having
> >> > > > > >> > > > > >> > >>>>>> our
> >> > > > > >> > > > > >> > >>>>>>>>> own. All of the coordinator stream
> code
> >> > would
> >> > > > go
> >> > > > > >> away,
> >> > > > > >> > > as
> >> > > > > >> > > > > >> would
> >> > > > > >> > > > > >> > >>>>>>>>> most
> >> > > > > >> > > > > >> > >>>>>> of
> >> > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably
> have
> >> to
> >> > > push
> >> > > > > some
> >> > > > > >> > > > > partition
> >> > > > > >> > > > > >> > >>>>>>> management
> >> > > > > >> > > > > >> > >>>>>>>>> features into the Kafka broker, but
> >> they're
> >> > > > > already
> >> > > > > >> > > moving
> >> > > > > >> > > > > in
> >> > > > > >> > > > > >> > >>>>>>>>> that direction with the new consumer
> >> API.
> >> > The
> >> > > > > >> features
> >> > > > > >> > > we
> >> > > > > >> > > > > have
> >> > > > > >> > > > > >> > >>>>>>>>> for
> >> > > > > >> > > > > >> > >>>>>> partition
> >> > > > > >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and
> >> seem
> >> > > > like
> >> > > > > >> they
> >> > > > > >> > > > should
> >> > > > > >> > > > > >> be
> >> > > > > >> > > > > >> > >>>>>>>>> in
> >> > > > > >> > > > > >> > >>>>>> Kafka
> >> > > > > >> > > > > >> > >>>>>>>>> anyway. There will always be some
> niche
> >> > > usages
> >> > > > > which
> >> > > > > >> > > will
> >> > > > > >> > > > > >> > >>>>>>>>> require
> >> > > > > >> > > > > >> > >>>>>> extra
> >> > > > > >> > > > > >> > >>>>>>>>> care and hence full control over
> >> partition
> >> > > > > >> assignments
> >> > > > > >> > > > much
> >> > > > > >> > > > > >> > >>>>>>>>> like the
> >> > > > > >> > > > > >> > >>>>>>> Kafka
> >> > > > > >> > > > > >> > >>>>>>>>> low level consumer api. These would
> >> > continue
> >> > > to
> >> > > > > be
> >> > > > > >> > > > > supported.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> These items will be good for the Samza
> >> > > > community.
> >> > > > > >> > > They'll
> >> > > > > >> > > > > make
> >> > > > > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it
> easier
> >> for
> >> > > > > >> developers
> >> > > > > >> > > to
> >> > > > > >> > > > > add
> >> > > > > >> > > > > >> > >>>>>>>>> new features.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and
> >> > > somewhat
> >> > > > > >> > backwards
> >> > > > > >> > > > > >> > >>>>> incompatible
> >> > > > > >> > > > > >> > >>>>>>>>> change). If we choose to go this
> route,
> >> > it's
> >> > > > > >> important
> >> > > > > >> > > > that
> >> > > > > >> > > > > we
> >> > > > > >> > > > > >> > >>>>> openly
> >> > > > > >> > > > > >> > >>>>>>>>> communicate how we're going to
> provide a
> >> > > > > migration
> >> > > > > >> > path
> >> > > > > >> > > > from
> >> > > > > >> > > > > >> > >>>>>>>>> the
> >> > > > > >> > > > > >> > >>>>>>> existing
> >> > > > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make
> >> > incompatible
> >> > > > > >> > changes).
> >> > > > > >> > > I
> >> > > > > >> > > > > >> think
> >> > > > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to
> >> > provide a
> >> > > > > >> wrapper
> >> > > > > >> > to
> >> > > > > >> > > > > allow
> >> > > > > >> > > > > >> > >>>>>>>>> existing StreamTask implementations to
> >> > > continue
> >> > > > > >> > running
> >> > > > > >> > > on
> >> > > > > >> > > > > the
> >> > > > > >> > > > > >> > >>>> new container.
> >> > > > > >> > > > > >> > >>>>>>> It's
> >> > > > > >> > > > > >> > >>>>>>>>> also important that we openly
> >> communicate
> >> > > about
> >> > > > > >> > timing,
> >> > > > > >> > > > and
> >> > > > > >> > > > > >> > >>>>>>>>> stages
> >> > > > > >> > > > > >> > >>>>> of
> >> > > > > >> > > > > >> > >>>>>>> the
> >> > > > > >> > > > > >> > >>>>>>>>> migration.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you
> >> have
> >> > > > > opinions.
> >> > > > > >> > :)
> >> > > > > >> > > > > Please
> >> > > > > >> > > > > >> > >>>>>>>>> send
> >> > > > > >> > > > > >> > >>>>>> your
> >> > > > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>> Cheers,
> >> > > > > >> > > > > >> > >>>>>>>>> Chris
> >> > > > > >> > > > > >> > >>>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>>
> >> > > > > >> > > > > >> > >>>>>>>
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>> --
> >> > > > > >> > > > > >> > >>>>>> -- Guozhang
> >> > > > > >> > > > > >> > >>>>>>
> >> > > > > >> > > > > >> > >>>>>
> >> > > > > >> > > > > >> > >>>>
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> > >>
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >> >
> >> > > > > >> > > > > >>
> >> > > > > >> > > > > >
> >> > > > > >> > > > > >
> >> > > > > >> > > > >
> >> > > > > >> > > >
> >> > > > > >> > >
> >> > > > > >> >
> >> > > > > >>
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Chris Riccomini <cr...@apache.org>.
That was meant to be "thread" not "threat". lol. :)

On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini <cr...@apache.org>
wrote:

> Hey all,
>
> I want to start by saying that I'm absolutely thrilled to be a part of
> this community. The amount of level-headed, thoughtful, educated discussion
> that's gone on over the past ~10 days is overwhelming. Wonderful.
>
> It seems like discussion is waning a bit, and we've reached some
> conclusions. There are several key emails in this threat, which I want to
> call out:
>
> 1. Jakob's summary of the three potential ways forward.
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
> 2. Julian's call out that we should be focusing on community over code.
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
> 3. Martin's summary about the benefits of merging communities.
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
> 4. Jakob's comments about the distinction between community and code paths.
>
> http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E
>
> I agree with the comments on all of these emails. I think Martin's summary
> of his position aligns very closely with my own. To that end, I think we
> should get concrete about what the proposal is, and call a vote on it.
> Given that Jay, Martin, and I seem to be aligning fairly closely, I think
> we should start with:
>
> 1. [community] Make Samza a subproject of Kafka.
> 2. [community] Make all Samza PMC/committers committers of the subproject.
> 3. [community] Migrate Samza's website/documentation into Kafka's.
> 4. [code] Have the Samza community and the Kafka community start a
> from-scratch reboot together in the new Kafka subproject. We can
> borrow/copy &  paste significant chunks of code from Samza's code base.
> 5. [code] The subproject would intentionally eliminate support for both
> other streaming systems and all deployment systems.
> 6. [code] Attempt to provide a bridge from our SystemConsumer to KIP-26
> (copy cat)
> 7. [code] Attempt to provide a bridge from the new subproject's processor
> interface to our legacy StreamTask interface.
> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> subproject that has a fault-tolerant container with state management.
>
> It's likely that (6) and (7) won't be fully drop-in. Still, the closer we
> can get, the better it's going to be for our existing community.
>
> One thing that I didn't touch on with (2) is whether any Samza PMC members
> should be rolled into Kafka PMC membership as well (though, Jay and Jakob
> are already PMC members on both). I think that Samza's community deserves a
> voice on the PMC, so I'd propose that we roll at least a few PMC members
> into the Kafka PMC, but I don't have a strong framework for which people to
> pick.
>
> Before (8), I think that Samza's TLP can continue to commit bug fixes and
> patches as it sees fit, provided that we openly communicate that we won't
> necessarily migrate new features to the new subproject, and that the TLP
> will be shut down after the migration to the Kafka subproject occurs.
>
> Jakob, I could use your guidance here about about how to achieve this from
> an Apache process perspective (sorry).
>
> * Should I just call a vote on this proposal?
> * Should it happen on dev or private?
> * Do committers have binding votes, or just PMC?
>
> Having trouble finding much detail on the Apache wikis. :(
>
> Cheers,
> Chris
>
> On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com> wrote:
>
>> Thanks, Jay. This argument persuaded me actually. :)
>>
>> Fang, Yan
>> yanfang724@gmail.com
>>
>> On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io> wrote:
>>
>> > Hey Yan,
>> >
>> > Yeah philosophically I think the argument is that you should capture the
>> > stream in Kafka independent of the transformation. This is obviously a
>> > Kafka-centric view point.
>> >
>> > Advantages of this:
>> > - In practice I think this is what e.g. Storm people often end up doing
>> > anyway. You usually need to throttle any access to a live serving
>> database.
>> > - Can have multiple subscribers and they get the same thing without
>> > additional load on the source system.
>> > - Applications can tap into the stream if need be by subscribing.
>> > - You can debug your transformation by tailing the Kafka topic with the
>> > console consumer
>> > - Can tee off the same data stream for batch analysis or Lambda arch
>> style
>> > re-processing
>> >
>> > The disadvantage is that it will use Kafka resources. But the idea is
>> > eventually you will have multiple subscribers to any data source (at
>> least
>> > for monitoring) so you will end up there soon enough anyway.
>> >
>> > Down the road the technical benefit is that I think it gives us a good
>> path
>> > towards end-to-end exactly once semantics from source to destination.
>> > Basically the connectors need to support idempotence when talking to
>> Kafka
>> > and we need the transactional write feature in Kafka to make the
>> > transformation atomic. This is actually pretty doable if you separate
>> > connector=>kafka problem from the generic transformations which are
>> always
>> > kafka=>kafka. However I think it is quite impossible to do in a
>> all_things
>> > => all_things environment. Today you can say "well the semantics of the
>> > Samza APIs depend on the connectors you use" but it is actually worse
>> then
>> > that because the semantics actually depend on the pairing of
>> connectors--so
>> > not only can you probably not get a usable "exactly once" guarantee
>> > end-to-end it can actually be quite hard to reverse engineer what
>> property
>> > (if any) your end-to-end flow has if you have heterogenous systems.
>> >
>> > -Jay
>> >
>> > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com> wrote:
>> >
>> > > {quote}
>> > > maintained in a separate repository and retaining the existing
>> > > committership but sharing as much else as possible (website, etc)
>> > > {quote}
>> > >
>> > > Overall, I agree on this idea. Now the question is more about "how to
>> do
>> > > it".
>> > >
>> > > On the other hand, one thing I want to point out is that, if we
>> decide to
>> > > go this way, how do we want to support
>> > > otherSystem-transformation-otherSystem use case?
>> > >
>> > > Basically, there are four user groups here:
>> > >
>> > > 1. Kafka-transformation-Kafka
>> > > 2. Kafka-transformation-otherSystem
>> > > 3. otherSystem-transformation-Kafka
>> > > 4. otherSystem-transformation-otherSystem
>> > >
>> > > For group 1, they can easily use the new Samza library to achieve. For
>> > > group 2 and 3, they can use copyCat -> transformation -> Kafka or
>> Kafka->
>> > > transformation -> copyCat.
>> > >
>> > > The problem is for group 4. Do we want to abandon this or still
>> support
>> > it?
>> > > Of course, this use case can be achieved by using copyCat ->
>> > transformation
>> > > -> Kafka -> transformation -> copyCat, the thing is how we persuade
>> them
>> > to
>> > > do this long chain. If yes, it will also be a win for Kafka too. Or if
>> > > there is no one in this community actually doing this so far, maybe
>> ok to
>> > > not support the group 4 directly.
>> > >
>> > > Thanks,
>> > >
>> > > Fang, Yan
>> > > yanfang724@gmail.com
>> > >
>> > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io> wrote:
>> > >
>> > > > Yeah I agree with this summary. I think there are kind of two
>> questions
>> > > > here:
>> > > > 1. Technically does alignment/reliance on Kafka make sense
>> > > > 2. Branding wise (naming, website, concepts, etc) does alignment
>> with
>> > > Kafka
>> > > > make sense
>> > > >
>> > > > Personally I do think both of these things would be really valuable,
>> > and
>> > > > would dramatically alter the trajectory of the project.
>> > > >
>> > > > My preference would be to see if people can mostly agree on a
>> direction
>> > > > rather than splintering things off. From my point of view the ideal
>> > > outcome
>> > > > of all the options discussed would be to make Samza a closely
>> aligned
>> > > > subproject, maintained in a separate repository and retaining the
>> > > existing
>> > > > committership but sharing as much else as possible (website, etc).
>> No
>> > > idea
>> > > > about how these things work, Jacob, you probably know more.
>> > > >
>> > > > No discussion amongst the Kafka folks has happened on this, but
>> likely
>> > we
>> > > > should figure out what the Samza community actually wants first.
>> > > >
>> > > > I admit that this is a fairly radical departure from how things are.
>> > > >
>> > > > If that doesn't fly, I think, yeah we could leave Samza as it is
>> and do
>> > > the
>> > > > more radical reboot inside Kafka. From my point of view that does
>> leave
>> > > > things in a somewhat confusing state since now there are two stream
>> > > > processing systems more or less coupled to Kafka in large part made
>> by
>> > > the
>> > > > same people. But, arguably that might be a cleaner way to make the
>> > > cut-over
>> > > > and perhaps less risky for Samza community since if it works people
>> can
>> > > > switch and if it doesn't nothing will have changed. Dunno, how do
>> > people
>> > > > feel about this?
>> > > >
>> > > > -Jay
>> > > >
>> > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com>
>> > wrote:
>> > > >
>> > > > > >  This leads me to thinking that merging projects and communities
>> > > might
>> > > > > be a good idea: with the union of experience from both
>> communities,
>> > we
>> > > > will
>> > > > > probably build a better system that is better for users.
>> > > > > Is this what's being proposed though? Merging the projects seems
>> like
>> > > > > a consequence of at most one of the three directions under
>> > discussion:
>> > > > > 1) Samza 2.0: The Samza community relies more heavily on Kafka for
>> > > > > configuration, etc. (to a greater or lesser extent to be
>> determined)
>> > > > > but the Samza community would not automatically merge withe Kafka
>> > > > > community (the Phoenix/HBase example is a good one here).
>> > > > > 2) Samza Reboot: The Samza community continues to exist with a
>> > limited
>> > > > > project scope, but similarly would not need to be part of the
>> Kafka
>> > > > > community (ie given committership) to progress.  Here, maybe the
>> > Samza
>> > > > > team would become a subproject of Kafka (the Board frowns on
>> > > > > subprojects at the moment, so I'm not sure if that's even
>> feasible),
>> > > > > but that would not be required.
>> > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option the
>> Kafka
>> > > > > team builds its own streaming library, possibly off of Jay's
>> > > > > prototype, which has not direct lineage to the Samza team.
>> There's
>> > no
>> > > > > reason for the Kafka team to bring in the Samza team.
>> > > > >
>> > > > > Is the Kafka community on board with this?
>> > > > >
>> > > > > To be clear, all three options under discussion are interesting,
>> > > > > technically valid and likely healthy directions for the project.
>> > > > > Also, they are not mutually exclusive.  The Samza community could
>> > > > > decide to pursue, say, 'Samza 2.0', while the Kafka community went
>> > > > > forward with 'Hey Samza!'  My points above are directed entirely
>> at
>> > > > > the community aspect of these choices.
>> > > > > -Jakob
>> > > > >
>> > > > > On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com>
>> > wrote:
>> > > > > > That's great.  Thanks, Jay.
>> > > > > >
>> > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io>
>> > wrote:
>> > > > > >
>> > > > > >> Yeah totally agree. I think you have this issue even today,
>> right?
>> > > > I.e.
>> > > > > if
>> > > > > >> you need to make a simple config change and you're running in
>> YARN
>> > > > today
>> > > > > >> you end up bouncing the job which then rebuilds state. I think
>> the
>> > > fix
>> > > > > is
>> > > > > >> exactly what you described which is to have a long timeout on
>> > > > partition
>> > > > > >> movement for stateful jobs so that if a job is just getting
>> > bounced,
>> > > > and
>> > > > > >> the cluster manager (or admin) is smart enough to restart it on
>> > the
>> > > > same
>> > > > > >> host when possible, it can optimistically reuse any existing
>> state
>> > > it
>> > > > > finds
>> > > > > >> on disk (if it is valid).
>> > > > > >>
>> > > > > >> So in this model the charter of the CM is to place processes as
>> > > > > stickily as
>> > > > > >> possible and to restart or re-place failed processes. The
>> charter
>> > of
>> > > > the
>> > > > > >> partition management system is to control the assignment of
>> work
>> > to
>> > > > > these
>> > > > > >> processes. The nice thing about this is that the work
>> assignment,
>> > > > > timeouts,
>> > > > > >> behavior, configs, and code will all be the same across all
>> > cluster
>> > > > > >> managers.
>> > > > > >>
>> > > > > >> So I think that prototype would actually give you exactly what
>> you
>> > > > want
>> > > > > >> today for any cluster manager (or manual placement + restart
>> > script)
>> > > > > that
>> > > > > >> was sticky in terms of host placement since there is already a
>> > > > > configurable
>> > > > > >> partition movement timeout and task-by-task state reuse with a
>> > check
>> > > > on
>> > > > > >> state validity.
>> > > > > >>
>> > > > > >> -Jay
>> > > > > >>
>> > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
>> > > roger.hoover@gmail.com
>> > > > >
>> > > > > >> wrote:
>> > > > > >>
>> > > > > >> > That would be great to let Kafka do as much heavy lifting as
>> > > > possible
>> > > > > and
>> > > > > >> > make it easier for other languages to implement Samza apis.
>> > > > > >> >
>> > > > > >> > One thing to watch out for is the interplay between Kafka's
>> > group
>> > > > > >> > management and the external scheduler/process manager's fault
>> > > > > tolerance.
>> > > > > >> > If a container dies, the Kafka group membership protocol will
>> > try
>> > > to
>> > > > > >> assign
>> > > > > >> > it's tasks to other containers while at the same time the
>> > process
>> > > > > manager
>> > > > > >> > is trying to relaunch the container.  Without some
>> consideration
>> > > for
>> > > > > this
>> > > > > >> > (like a configurable amount of time to wait before Kafka
>> alters
>> > > the
>> > > > > group
>> > > > > >> > membership), there may be thrashing going on which is
>> especially
>> > > bad
>> > > > > for
>> > > > > >> > containers with large amounts of local state.
>> > > > > >> >
>> > > > > >> > Someone else pointed this out already but I thought it might
>> be
>> > > > worth
>> > > > > >> > calling out again.
>> > > > > >> >
>> > > > > >> > Cheers,
>> > > > > >> >
>> > > > > >> > Roger
>> > > > > >> >
>> > > > > >> >
>> > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <jay@confluent.io
>> >
>> > > > wrote:
>> > > > > >> >
>> > > > > >> > > Hey Roger,
>> > > > > >> > >
>> > > > > >> > > I couldn't agree more. We spent a bunch of time talking to
>> > > people
>> > > > > and
>> > > > > >> > that
>> > > > > >> > > is exactly the stuff we heard time and again. What makes it
>> > > hard,
>> > > > of
>> > > > > >> > > course, is that there is some tension between compatibility
>> > with
>> > > > > what's
>> > > > > >> > > there now and making things better for new users.
>> > > > > >> > >
>> > > > > >> > > I also strongly agree with the importance of multi-language
>> > > > > support. We
>> > > > > >> > are
>> > > > > >> > > talking now about Java, but for application development use
>> > > cases
>> > > > > >> people
>> > > > > >> > > want to work in whatever language they are using
>> elsewhere. I
>> > > > think
>> > > > > >> > moving
>> > > > > >> > > to a model where Kafka itself does the group membership,
>> > > lifecycle
>> > > > > >> > control,
>> > > > > >> > > and partition assignment has the advantage of putting all
>> that
>> > > > > complex
>> > > > > >> > > stuff behind a clean api that the clients are already
>> going to
>> > > be
>> > > > > >> > > implementing for their consumer, so the added functionality
>> > for
>> > > > > stream
>> > > > > >> > > processing beyond a consumer becomes very minor.
>> > > > > >> > >
>> > > > > >> > > -Jay
>> > > > > >> > >
>> > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
>> > > > > roger.hoover@gmail.com>
>> > > > > >> > > wrote:
>> > > > > >> > >
>> > > > > >> > > > Metamorphosis...nice. :)
>> > > > > >> > > >
>> > > > > >> > > > This has been a great discussion.  As a user of Samza
>> who's
>> > > > > recently
>> > > > > >> > > > integrated it into a relatively large organization, I
>> just
>> > > want
>> > > > to
>> > > > > >> add
>> > > > > >> > > > support to a few points already made.
>> > > > > >> > > >
>> > > > > >> > > > The biggest hurdles to adoption of Samza as it currently
>> > > exists
>> > > > > that
>> > > > > >> > I've
>> > > > > >> > > > experienced are:
>> > > > > >> > > > 1) YARN - YARN is overly complex in many environments
>> where
>> > > > Puppet
>> > > > > >> > would
>> > > > > >> > > do
>> > > > > >> > > > just fine but it was the only mechanism to get fault
>> > > tolerance.
>> > > > > >> > > > 2) Configuration - I think I like the idea of configuring
>> > most
>> > > > of
>> > > > > the
>> > > > > >> > job
>> > > > > >> > > > in code rather than config files.  In general, I think
>> the
>> > > goal
>> > > > > >> should
>> > > > > >> > be
>> > > > > >> > > > to make it harder to make mistakes, especially of the
>> kind
>> > > where
>> > > > > the
>> > > > > >> > code
>> > > > > >> > > > expects something and the config doesn't match.  The
>> current
>> > > > > config
>> > > > > >> is
>> > > > > >> > > > quite intricate and error-prone.  For example, the
>> > application
>> > > > > logic
>> > > > > >> > may
>> > > > > >> > > > depend on bootstrapping a topic but rather than asserting
>> > that
>> > > > in
>> > > > > the
>> > > > > >> > > code,
>> > > > > >> > > > you have to rely on getting the config right.  Likewise
>> with
>> > > > > serdes,
>> > > > > >> > the
>> > > > > >> > > > Java representations produced by various serdes (JSON,
>> Avro,
>> > > > etc.)
>> > > > > >> are
>> > > > > >> > > not
>> > > > > >> > > > equivalent so you cannot just reconfigure a serde without
>> > > > changing
>> > > > > >> the
>> > > > > >> > > > code.   It would be nice for jobs to be able to assert
>> what
>> > > they
>> > > > > >> expect
>> > > > > >> > > > from their input topics in terms of partitioning.  This
>> is
>> > > > > getting a
>> > > > > >> > > little
>> > > > > >> > > > off topic but I was even thinking about creating a "Samza
>> > > config
>> > > > > >> > linter"
>> > > > > >> > > > that would sanity check a set of configs.  Especially in
>> > > > > >> organizations
>> > > > > >> > > > where config is managed by a different team than the
>> > > application
>> > > > > >> > > developer,
>> > > > > >> > > > it's very hard to get avoid config mistakes.
>> > > > > >> > > > 3) Java/Scala centric - for many teams (especially
>> > DevOps-type
>> > > > > >> folks),
>> > > > > >> > > the
>> > > > > >> > > > pain of the Java toolchain (maven, slow builds, weak
>> command
>> > > > line
>> > > > > >> > > support,
>> > > > > >> > > > configuration over convention) really inhibits
>> productivity.
>> > > As
>> > > > > more
>> > > > > >> > and
>> > > > > >> > > > more high-quality clients become available for Kafka, I
>> hope
>> > > > > they'll
>> > > > > >> > > follow
>> > > > > >> > > > Samza's model.  Not sure how much it affects the
>> proposals
>> > in
>> > > > this
>> > > > > >> > thread
>> > > > > >> > > > but please consider other languages in the ecosystem as
>> > well.
>> > > > > From
>> > > > > >> > what
>> > > > > >> > > > I've heard, Spark has more Python users than Java/Scala.
>> > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
>> > > > > >> > > >
>> > > > > >> > > >
>> > > > > >> > >
>> > > > > >> >
>> > > > > >>
>> > > > >
>> > > >
>> > >
>> >
>> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
>> > > > > >> > > > and are working on a Yeoman generator
>> > > > > >> > > > https://github.com/Quantiply/generator-rico for
>> > Jython/Samza
>> > > > > >> projects
>> > > > > >> > to
>> > > > > >> > > > alleviate some of the pain)
>> > > > > >> > > >
>> > > > > >> > > > I also want to underscore Jay's point about improving the
>> > user
>> > > > > >> > > experience.
>> > > > > >> > > > That's a very important factor for adoption.  I think the
>> > goal
>> > > > > should
>> > > > > >> > be
>> > > > > >> > > to
>> > > > > >> > > > make Samza as easy to get started with as something like
>> > > > Logstash.
>> > > > > >> > > > Logstash is vastly inferior in terms of capabilities to
>> > Samza
>> > > > but
>> > > > > >> it's
>> > > > > >> > > easy
>> > > > > >> > > > to get started and that makes a big difference.
>> > > > > >> > > >
>> > > > > >> > > > Cheers,
>> > > > > >> > > >
>> > > > > >> > > > Roger
>> > > > > >> > > >
>> > > > > >> > > >
>> > > > > >> > > >
>> > > > > >> > > >
>> > > > > >> > > >
>> > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci
>> > > Morales <
>> > > > > >> > > > gdfm@apache.org> wrote:
>> > > > > >> > > >
>> > > > > >> > > > > Forgot to add. On the naming issues, Kafka
>> Metamorphosis
>> > is
>> > > a
>> > > > > clear
>> > > > > >> > > > winner
>> > > > > >> > > > > :)
>> > > > > >> > > > >
>> > > > > >> > > > > --
>> > > > > >> > > > > Gianmarco
>> > > > > >> > > > >
>> > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci
>> Morales <
>> > > > > >> > > gdfm@apache.org
>> > > > > >> > > > >
>> > > > > >> > > > > wrote:
>> > > > > >> > > > >
>> > > > > >> > > > > > Hi,
>> > > > > >> > > > > >
>> > > > > >> > > > > > @Martin, thanks for you comments.
>> > > > > >> > > > > > Maybe I'm missing some important point, but I think
>> > > coupling
>> > > > > the
>> > > > > >> > > > releases
>> > > > > >> > > > > > is actually a *good* thing.
>> > > > > >> > > > > > To make an example, would it be better if the MR and
>> > HDFS
>> > > > > >> > components
>> > > > > >> > > of
>> > > > > >> > > > > > Hadoop had different release schedules?
>> > > > > >> > > > > >
>> > > > > >> > > > > > Actually, keeping the discussion in a single place
>> would
>> > > > make
>> > > > > >> > > agreeing
>> > > > > >> > > > on
>> > > > > >> > > > > > releases (and backwards compatibility) much easier,
>> as
>> > > > > everybody
>> > > > > >> > > would
>> > > > > >> > > > be
>> > > > > >> > > > > > responsible for the whole codebase.
>> > > > > >> > > > > >
>> > > > > >> > > > > > That said, I like the idea of absorbing samza-core
>> as a
>> > > > > >> > sub-project,
>> > > > > >> > > > and
>> > > > > >> > > > > > leave the fancy stuff separate.
>> > > > > >> > > > > > It probably gives 90% of the benefits we have been
>> > > > discussing
>> > > > > >> here.
>> > > > > >> > > > > >
>> > > > > >> > > > > > Cheers,
>> > > > > >> > > > > >
>> > > > > >> > > > > > --
>> > > > > >> > > > > > Gianmarco
>> > > > > >> > > > > >
>> > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
>> jay.kreps@gmail.com
>> > >
>> > > > > wrote:
>> > > > > >> > > > > >
>> > > > > >> > > > > >> Hey Martin,
>> > > > > >> > > > > >>
>> > > > > >> > > > > >> I agree coupling release schedules is a downside.
>> > > > > >> > > > > >>
>> > > > > >> > > > > >> Definitely we can try to solve some of the
>> integration
>> > > > > problems
>> > > > > >> in
>> > > > > >> > > > > >> Confluent Platform or in other distributions. But I
>> > think
>> > > > > this
>> > > > > >> > ends
>> > > > > >> > > up
>> > > > > >> > > > > >> being really shallow. I guess I feel to really get a
>> > good
>> > > > > user
>> > > > > >> > > > > experience
>> > > > > >> > > > > >> the two systems have to kind of feel like part of
>> the
>> > > same
>> > > > > thing
>> > > > > >> > and
>> > > > > >> > > > you
>> > > > > >> > > > > >> can't really add that in later--you can put both in
>> the
>> > > > same
>> > > > > >> > > > > downloadable
>> > > > > >> > > > > >> tar file but it doesn't really give a very cohesive
>> > > > feeling.
>> > > > > I
>> > > > > >> > agree
>> > > > > >> > > > > that
>> > > > > >> > > > > >> ultimately any of the project stuff is as much
>> social
>> > and
>> > > > > naming
>> > > > > >> > as
>> > > > > >> > > > > >> anything else--theoretically two totally independent
>> > > > projects
>> > > > > >> > could
>> > > > > >> > > > work
>> > > > > >> > > > > >> to
>> > > > > >> > > > > >> tightly align. In practice this seems to be quite
>> > > difficult
>> > > > > >> > though.
>> > > > > >> > > > > >>
>> > > > > >> > > > > >> For the frameworks--totally agree it would be good
>> to
>> > > > > maintain
>> > > > > >> the
>> > > > > >> > > > > >> framework support with the project. In some cases
>> there
>> > > may
>> > > > > not
>> > > > > >> be
>> > > > > >> > > too
>> > > > > >> > > > > >> much
>> > > > > >> > > > > >> there since the integration gets lighter but I think
>> > > > whatever
>> > > > > >> > stubs
>> > > > > >> > > > you
>> > > > > >> > > > > >> need should be included. So no I definitely wasn't
>> > trying
>> > > > to
>> > > > > >> imply
>> > > > > >> > > > > >> dropping
>> > > > > >> > > > > >> support for these frameworks, just making the
>> > integration
>> > > > > >> lighter
>> > > > > >> > by
>> > > > > >> > > > > >> separating process management from partition
>> > management.
>> > > > > >> > > > > >>
>> > > > > >> > > > > >> You raise two good points we would have to figure
>> out
>> > if
>> > > we
>> > > > > went
>> > > > > >> > > down
>> > > > > >> > > > > the
>> > > > > >> > > > > >> alignment path:
>> > > > > >> > > > > >> 1. With respect to the name, yeah I think the first
>> > > > question
>> > > > > is
>> > > > > >> > > > whether
>> > > > > >> > > > > >> some "re-branding" would be worth it. If so then I
>> > think
>> > > we
>> > > > > can
>> > > > > >> > > have a
>> > > > > >> > > > > big
>> > > > > >> > > > > >> thread on the name. I'm definitely not set on Kafka
>> > > > > Streaming or
>> > > > > >> > > Kafka
>> > > > > >> > > > > >> Streams I was just using them to be kind of
>> > > illustrative. I
>> > > > > >> agree
>> > > > > >> > > with
>> > > > > >> > > > > >> your
>> > > > > >> > > > > >> critique of these names, though I think people would
>> > get
>> > > > the
>> > > > > >> idea.
>> > > > > >> > > > > >> 2. Yeah you also raise a good point about how to
>> > "factor"
>> > > > it.
>> > > > > >> Here
>> > > > > >> > > are
>> > > > > >> > > > > the
>> > > > > >> > > > > >> options I see (I could get enthusiastic about any of
>> > > them):
>> > > > > >> > > > > >>    a. One repo for both Kafka and Samza
>> > > > > >> > > > > >>    b. Two repos, retaining the current seperation
>> > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api and
>> > > samza-core
>> > > > > is
>> > > > > >> > > > absorbed
>> > > > > >> > > > > >> almost like a third client
>> > > > > >> > > > > >>
>> > > > > >> > > > > >> Cheers,
>> > > > > >> > > > > >>
>> > > > > >> > > > > >> -Jay
>> > > > > >> > > > > >>
>> > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
>> > > > > >> > > > martin@kleppmann.com>
>> > > > > >> > > > > >> wrote:
>> > > > > >> > > > > >>
>> > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a few
>> > follow-up
>> > > > > >> > comments.
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
>> becoming
>> > a
>> > > > > >> > subproject:
>> > > > > >> > > > the
>> > > > > >> > > > > >> > reasons you mention are good. The risk I see is
>> that
>> > > > > release
>> > > > > >> > > > schedules
>> > > > > >> > > > > >> > become coupled to each other, which can slow
>> everyone
>> > > > down,
>> > > > > >> and
>> > > > > >> > > > large
>> > > > > >> > > > > >> > projects with many contributors are harder to
>> manage.
>> > > > > (Jakob,
>> > > > > >> > can
>> > > > > >> > > > you
>> > > > > >> > > > > >> speak
>> > > > > >> > > > > >> > from experience, having seen a wider range of
>> Hadoop
>> > > > > ecosystem
>> > > > > >> > > > > >> projects?)
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> > Some of the goals of a better unified developer
>> > > > experience
>> > > > > >> could
>> > > > > >> > > > also
>> > > > > >> > > > > be
>> > > > > >> > > > > >> > solved by integrating Samza nicely into a Kafka
>> > > > > distribution
>> > > > > >> > (such
>> > > > > >> > > > as
>> > > > > >> > > > > >> > Confluent's). I'm not against merging projects if
>> we
>> > > > decide
>> > > > > >> > that's
>> > > > > >> > > > the
>> > > > > >> > > > > >> way
>> > > > > >> > > > > >> > to go, just pointing out the same goals can
>> perhaps
>> > > also
>> > > > be
>> > > > > >> > > achieved
>> > > > > >> > > > > in
>> > > > > >> > > > > >> > other ways.
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> > - With regard to dropping the YARN dependency: are
>> > you
>> > > > > >> proposing
>> > > > > >> > > > that
>> > > > > >> > > > > >> > Samza doesn't give any help to people wanting to
>> run
>> > on
>> > > > > >> > > > > >> YARN/Mesos/AWS/etc?
>> > > > > >> > > > > >> > So the docs would basically have a link to Slider
>> and
>> > > > > nothing
>> > > > > >> > > else?
>> > > > > >> > > > Or
>> > > > > >> > > > > >> > would we maintain integrations with a bunch of
>> > popular
>> > > > > >> > deployment
>> > > > > >> > > > > >> methods
>> > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts to make
>> > > Samza
>> > > > > work
>> > > > > >> > with
>> > > > > >> > > > > >> Slider)?
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> > I absolutely think it's a good idea to have the
>> "as a
>> > > > > library"
>> > > > > >> > and
>> > > > > >> > > > > "as a
>> > > > > >> > > > > >> > process" (using Yi's taxonomy) options for people
>> who
>> > > > want
>> > > > > >> them,
>> > > > > >> > > > but I
>> > > > > >> > > > > >> > think there should also be a low-friction path for
>> > > common
>> > > > > "as
>> > > > > >> a
>> > > > > >> > > > > service"
>> > > > > >> > > > > >> > deployment methods, for which we probably need to
>> > > > maintain
>> > > > > >> > > > > integrations.
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to me,
>> > > > because
>> > > > > >> Kafka
>> > > > > >> > > is
>> > > > > >> > > > > all
>> > > > > >> > > > > >> > about streams already. Perhaps "Kafka
>> Transformers"
>> > or
>> > > > > "Kafka
>> > > > > >> > > > Filters"
>> > > > > >> > > > > >> > would be more apt?
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> > One suggestion: perhaps the core of Samza (stream
>> > > > > >> transformation
>> > > > > >> > > > with
>> > > > > >> > > > > >> > state management -- i.e. the "Samza as a library"
>> > bit)
>> > > > > could
>> > > > > >> > > become
>> > > > > >> > > > > >> part of
>> > > > > >> > > > > >> > Kafka, while higher-level tools such as streaming
>> SQL
>> > > and
>> > > > > >> > > > integrations
>> > > > > >> > > > > >> with
>> > > > > >> > > > > >> > deployment frameworks remain in a separate
>> project?
>> > In
>> > > > > other
>> > > > > >> > > words,
>> > > > > >> > > > > >> Kafka
>> > > > > >> > > > > >> > would absorb the proven, stable core of Samza,
>> which
>> > > > would
>> > > > > >> > become
>> > > > > >> > > > the
>> > > > > >> > > > > >> > "third Kafka client" mentioned early in this
>> thread.
>> > > The
>> > > > > Samza
>> > > > > >> > > > project
>> > > > > >> > > > > >> > would then target that third Kafka client as its
>> base
>> > > > API,
>> > > > > and
>> > > > > >> > the
>> > > > > >> > > > > >> project
>> > > > > >> > > > > >> > would be freed up to explore more experimental new
>> > > > > horizons.
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> > Martin
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
>> > > jay.kreps@gmail.com>
>> > > > > >> wrote:
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> > > Hey Martin,
>> > > > > >> > > > > >> > >
>> > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually
>> don't
>> > > > think
>> > > > > it
>> > > > > >> > ties
>> > > > > >> > > > our
>> > > > > >> > > > > >> > hands
>> > > > > >> > > > > >> > > at all, all it does is refactor things. The
>> > division
>> > > of
>> > > > > >> > > > > >> responsibility is
>> > > > > >> > > > > >> > > that Samza core is responsible for task
>> lifecycle,
>> > > > state,
>> > > > > >> and
>> > > > > >> > > > > >> partition
>> > > > > >> > > > > >> > > management (using the Kafka co-ordinator) but
>> it is
>> > > NOT
>> > > > > >> > > > responsible
>> > > > > >> > > > > >> for
>> > > > > >> > > > > >> > > packaging, configuration deployment or
>> execution of
>> > > > > >> processes.
>> > > > > >> > > The
>> > > > > >> > > > > >> > problem
>> > > > > >> > > > > >> > > of packaging and starting these processes is
>> > > > > >> > > > > >> > > framework/environment-specific. This leaves
>> > > individual
>> > > > > >> > > frameworks
>> > > > > >> > > > to
>> > > > > >> > > > > >> be
>> > > > > >> > > > > >> > as
>> > > > > >> > > > > >> > > fancy or vanilla as they like. So you can get
>> > simple
>> > > > > >> stateless
>> > > > > >> > > > > >> support in
>> > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app
>> > > > framework
>> > > > > >> > > (Slider,
>> > > > > >> > > > > >> > Marathon,
>> > > > > >> > > > > >> > > etc). These are well known by people and have
>> nice
>> > > UIs
>> > > > > and a
>> > > > > >> > lot
>> > > > > >> > > > of
>> > > > > >> > > > > >> > > flexibility. I don't think they have node
>> affinity
>> > > as a
>> > > > > >> built
>> > > > > >> > in
>> > > > > >> > > > > >> option
>> > > > > >> > > > > >> > > (though I could be wrong). So if we want that we
>> > can
>> > > > > either
>> > > > > >> > wait
>> > > > > >> > > > for
>> > > > > >> > > > > >> them
>> > > > > >> > > > > >> > > to add it or do a custom framework to add that
>> > > feature
>> > > > > (as
>> > > > > >> > now).
>> > > > > >> > > > > >> > Obviously
>> > > > > >> > > > > >> > > if you manage things with old-school ops tools
>> > > > > >> > (puppet/chef/etc)
>> > > > > >> > > > you
>> > > > > >> > > > > >> get
>> > > > > >> > > > > >> > > locality easily. The nice thing, though, is that
>> > all
>> > > > the
>> > > > > >> samza
>> > > > > >> > > > > >> "business
>> > > > > >> > > > > >> > > logic" around partition management and fault
>> > > tolerance
>> > > > > is in
>> > > > > >> > > Samza
>> > > > > >> > > > > >> core
>> > > > > >> > > > > >> > so
>> > > > > >> > > > > >> > > it is shared across frameworks and the framework
>> > > > specific
>> > > > > >> bit
>> > > > > >> > is
>> > > > > >> > > > > just
>> > > > > >> > > > > >> > > whether it is smart enough to try to get the
>> same
>> > > host
>> > > > > when
>> > > > > >> a
>> > > > > >> > > job
>> > > > > >> > > > is
>> > > > > >> > > > > >> > > restarted.
>> > > > > >> > > > > >> > >
>> > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah I
>> think
>> > the
>> > > > > goal
>> > > > > >> > would
>> > > > > >> > > > be
>> > > > > >> > > > > >> (a)
>> > > > > >> > > > > >> > > actually get better alignment in user
>> experience,
>> > and
>> > > > (b)
>> > > > > >> > > express
>> > > > > >> > > > > >> this in
>> > > > > >> > > > > >> > > the naming and project branding. Specifically:
>> > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
>> > > > > "transformation"
>> > > > > >> api
>> > > > > >> > > to
>> > > > > >> > > > be
>> > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be
>> able
>> > to
>> > > > > explain
>> > > > > >> > > when
>> > > > > >> > > > to
>> > > > > >> > > > > >> use
>> > > > > >> > > > > >> > > the consumer and when to use the stream
>> processing
>> > > > > >> > functionality
>> > > > > >> > > > and
>> > > > > >> > > > > >> lead
>> > > > > >> > > > > >> > > people into that experience.
>> > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or
>> > > > > whatever)
>> > > > > >> > that
>> > > > > >> > > > has
>> > > > > >> > > > > >> both
>> > > > > >> > > > > >> > > Kafka and the stream processing part and they
>> > > actually
>> > > > > work
>> > > > > >> > > > > together.
>> > > > > >> > > > > >> > > 3. Unify the programming experience so the
>> client
>> > and
>> > > > > Samza
>> > > > > >> > api
>> > > > > >> > > > > share
>> > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
>> > > > > >> > > > > >> > >
>> > > > > >> > > > > >> > > I think sub-projects keep separate committers
>> and
>> > can
>> > > > > have a
>> > > > > >> > > > > separate
>> > > > > >> > > > > >> > repo,
>> > > > > >> > > > > >> > > but I'm actually not really sure (I can't find a
>> > > > > definition
>> > > > > >> > of a
>> > > > > >> > > > > >> > subproject
>> > > > > >> > > > > >> > > in Apache).
>> > > > > >> > > > > >> > >
>> > > > > >> > > > > >> > > Basically at a high-level you want the
>> experience
>> > to
>> > > > > "feel"
>> > > > > >> > > like a
>> > > > > >> > > > > >> single
>> > > > > >> > > > > >> > > system, not to relatively independent things
>> that
>> > are
>> > > > > kind
>> > > > > >> of
>> > > > > >> > > > > >> awkwardly
>> > > > > >> > > > > >> > > glued together.
>> > > > > >> > > > > >> > >
>> > > > > >> > > > > >> > > I think if we did that they having naming or
>> > branding
>> > > > > like
>> > > > > >> > > "kafka
>> > > > > >> > > > > >> > > streaming" or "kafka streams" or something like
>> > that
>> > > > > would
>> > > > > >> > > > actually
>> > > > > >> > > > > >> do a
>> > > > > >> > > > > >> > > good job of conveying what it is. I do that this
>> > > would
>> > > > > help
>> > > > > >> > > > adoption
>> > > > > >> > > > > >> > quite
>> > > > > >> > > > > >> > > a lot as it would correctly convey that using
>> Kafka
>> > > > > >> Streaming
>> > > > > >> > > with
>> > > > > >> > > > > >> Kafka
>> > > > > >> > > > > >> > is
>> > > > > >> > > > > >> > > a fairly seamless experience and Kafka is pretty
>> > > > heavily
>> > > > > >> > adopted
>> > > > > >> > > > at
>> > > > > >> > > > > >> this
>> > > > > >> > > > > >> > > point.
>> > > > > >> > > > > >> > >
>> > > > > >> > > > > >> > > Fwiw we actually considered this model
>> originally
>> > > when
>> > > > > open
>> > > > > >> > > > sourcing
>> > > > > >> > > > > >> > Samza,
>> > > > > >> > > > > >> > > however at that time Kafka was relatively
>> unknown
>> > and
>> > > > we
>> > > > > >> > decided
>> > > > > >> > > > not
>> > > > > >> > > > > >> to
>> > > > > >> > > > > >> > do
>> > > > > >> > > > > >> > > it since we felt it would be limiting. From my
>> > point
>> > > of
>> > > > > view
>> > > > > >> > the
>> > > > > >> > > > > three
>> > > > > >> > > > > >> > > things have changed (1) Kafka is now really
>> heavily
>> > > > used
>> > > > > for
>> > > > > >> > > > stream
>> > > > > >> > > > > >> > > processing, (2) we learned that abstracting out
>> the
>> > > > > stream
>> > > > > >> > well
>> > > > > >> > > is
>> > > > > >> > > > > >> > > basically impossible, (3) we learned it is
>> really
>> > > hard
>> > > > to
>> > > > > >> keep
>> > > > > >> > > the
>> > > > > >> > > > > two
>> > > > > >> > > > > >> > > things feeling like a single product.
>> > > > > >> > > > > >> > >
>> > > > > >> > > > > >> > > -Jay
>> > > > > >> > > > > >> > >
>> > > > > >> > > > > >> > >
>> > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin
>> Kleppmann <
>> > > > > >> > > > > >> martin@kleppmann.com>
>> > > > > >> > > > > >> > > wrote:
>> > > > > >> > > > > >> > >
>> > > > > >> > > > > >> > >> Hi all,
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> > >> Lots of good thoughts here.
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> > >> I agree with the general philosophy of tying
>> Samza
>> > > > more
>> > > > > >> > firmly
>> > > > > >> > > to
>> > > > > >> > > > > >> Kafka.
>> > > > > >> > > > > >> > >> After I spent a while looking at integrating
>> other
>> > > > > message
>> > > > > >> > > > brokers
>> > > > > >> > > > > >> (e.g.
>> > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
>> > > conclusion
>> > > > > that
>> > > > > >> > > > > >> > SystemConsumer
>> > > > > >> > > > > >> > >> tacitly assumes a model so much like Kafka's
>> that
>> > > > pretty
>> > > > > >> much
>> > > > > >> > > > > nobody
>> > > > > >> > > > > >> but
>> > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
>> perhaps
>> > an
>> > > > > >> > exception,
>> > > > > >> > > > but
>> > > > > >> > > > > >> it
>> > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus,
>> > making
>> > > > > Samza
>> > > > > >> > > fully
>> > > > > >> > > > > >> > dependent
>> > > > > >> > > > > >> > >> on Kafka acknowledges that the
>> system-independence
>> > > was
>> > > > > >> never
>> > > > > >> > as
>> > > > > >> > > > > real
>> > > > > >> > > > > >> as
>> > > > > >> > > > > >> > we
>> > > > > >> > > > > >> > >> perhaps made it out to be. The gains of code
>> reuse
>> > > are
>> > > > > >> real.
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN has also
>> > > always
>> > > > > been
>> > > > > >> > > > > >> appealing to
>> > > > > >> > > > > >> > >> me, for various reasons already mentioned in
>> this
>> > > > > thread.
>> > > > > >> > > > Although
>> > > > > >> > > > > >> > making
>> > > > > >> > > > > >> > >> Samza jobs deployable on anything
>> > > (YARN/Mesos/AWS/etc)
>> > > > > >> seems
>> > > > > >> > > > > >> laudable,
>> > > > > >> > > > > >> > I am
>> > > > > >> > > > > >> > >> a little concerned that it will restrict us to
>> a
>> > > > lowest
>> > > > > >> > common
>> > > > > >> > > > > >> > denominator.
>> > > > > >> > > > > >> > >> For example, would host affinity (SAMZA-617)
>> still
>> > > be
>> > > > > >> > possible?
>> > > > > >> > > > For
>> > > > > >> > > > > >> jobs
>> > > > > >> > > > > >> > >> with large amounts of state, I think SAMZA-617
>> > would
>> > > > be
>> > > > > a
>> > > > > >> big
>> > > > > >> > > > boon,
>> > > > > >> > > > > >> > since
>> > > > > >> > > > > >> > >> restoring state off the changelog on every
>> single
>> > > > > restart
>> > > > > >> is
>> > > > > >> > > > > painful,
>> > > > > >> > > > > >> > due
>> > > > > >> > > > > >> > >> to long recovery times. It would be a shame if
>> the
>> > > > > >> decoupling
>> > > > > >> > > > from
>> > > > > >> > > > > >> YARN
>> > > > > >> > > > > >> > >> made host affinity impossible.
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> > >> Jay, a question about the proposed API for
>> > > > > instantiating a
>> > > > > >> > job
>> > > > > >> > > in
>> > > > > >> > > > > >> code
>> > > > > >> > > > > >> > >> (rather than a properties file): when
>> submitting a
>> > > job
>> > > > > to a
>> > > > > >> > > > > cluster,
>> > > > > >> > > > > >> is
>> > > > > >> > > > > >> > the
>> > > > > >> > > > > >> > >> idea that the instantiation code runs on a
>> client
>> > > > > >> somewhere,
>> > > > > >> > > > which
>> > > > > >> > > > > >> then
>> > > > > >> > > > > >> > >> pokes the necessary endpoints on
>> > YARN/Mesos/AWS/etc?
>> > > > Or
>> > > > > >> does
>> > > > > >> > > that
>> > > > > >> > > > > >> code
>> > > > > >> > > > > >> > run
>> > > > > >> > > > > >> > >> on each container that is part of the job (in
>> > which
>> > > > > case,
>> > > > > >> how
>> > > > > >> > > > does
>> > > > > >> > > > > >> the
>> > > > > >> > > > > >> > job
>> > > > > >> > > > > >> > >> submission to the cluster work)?
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel right
>> to
>> > > make
>> > > > a
>> > > > > 1.0
>> > > > > >> > > > release
>> > > > > >> > > > > >> > with a
>> > > > > >> > > > > >> > >> plan for it to be immediately obsolete. So if
>> this
>> > > is
>> > > > > going
>> > > > > >> > to
>> > > > > >> > > > > >> happen, I
>> > > > > >> > > > > >> > >> think it would be more honest to stick with 0.*
>> > > > version
>> > > > > >> > numbers
>> > > > > >> > > > > until
>> > > > > >> > > > > >> > the
>> > > > > >> > > > > >> > >> library-ified Samza has been implemented, is
>> > stable
>> > > > and
>> > > > > >> > widely
>> > > > > >> > > > > used.
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> > >> Should the new Samza be a subproject of Kafka?
>> > There
>> > > > is
>> > > > > >> > > precedent
>> > > > > >> > > > > for
>> > > > > >> > > > > >> > >> tight coupling between different Apache
>> projects
>> > > (e.g.
>> > > > > >> > Curator
>> > > > > >> > > > and
>> > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think
>> > remaining
>> > > > > >> separate
>> > > > > >> > > > would
>> > > > > >> > > > > >> be
>> > > > > >> > > > > >> > ok.
>> > > > > >> > > > > >> > >> Even if Samza is fully dependent on Kafka,
>> there
>> > is
>> > > > > enough
>> > > > > >> > > > > substance
>> > > > > >> > > > > >> in
>> > > > > >> > > > > >> > >> Samza that it warrants being a separate
>> project.
>> > An
>> > > > > >> argument
>> > > > > >> > in
>> > > > > >> > > > > >> favour
>> > > > > >> > > > > >> > of
>> > > > > >> > > > > >> > >> merging would be if we think Kafka has a much
>> > > stronger
>> > > > > >> "brand
>> > > > > >> > > > > >> presence"
>> > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If the
>> > Kafka
>> > > > > >> project
>> > > > > >> > is
>> > > > > >> > > > > >> willing
>> > > > > >> > > > > >> > to
>> > > > > >> > > > > >> > >> endorse Samza as the "official" way of doing
>> > > stateful
>> > > > > >> stream
>> > > > > >> > > > > >> > >> transformations, that would probably have much
>> the
>> > > > same
>> > > > > >> > effect
>> > > > > >> > > as
>> > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream Processors"
>> or
>> > > > > suchlike.
>> > > > > >> > > Close
>> > > > > >> > > > > >> > >> collaboration between the two projects will be
>> > > needed
>> > > > in
>> > > > > >> any
>> > > > > >> > > > case.
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> > >> From a project management perspective, I guess
>> the
>> > > > "new
>> > > > > >> > Samza"
>> > > > > >> > > > > would
>> > > > > >> > > > > >> > have
>> > > > > >> > > > > >> > >> to be developed on a branch alongside ongoing
>> > > > > maintenance
>> > > > > >> of
>> > > > > >> > > the
>> > > > > >> > > > > >> current
>> > > > > >> > > > > >> > >> line of development? I think it would be
>> important
>> > > to
>> > > > > >> > continue
>> > > > > >> > > > > >> > supporting
>> > > > > >> > > > > >> > >> existing users, and provide a graceful
>> migration
>> > > path
>> > > > to
>> > > > > >> the
>> > > > > >> > > new
>> > > > > >> > > > > >> > version.
>> > > > > >> > > > > >> > >> Leaving the current versions unsupported and
>> > forcing
>> > > > > people
>> > > > > >> > to
>> > > > > >> > > > > >> rewrite
>> > > > > >> > > > > >> > >> their jobs would send a bad signal.
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> > >> Best,
>> > > > > >> > > > > >> > >> Martin
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
>> > > jay@confluent.io>
>> > > > > >> wrote:
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> > >>> Hey Garry,
>> > > > > >> > > > > >> > >>>
>> > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy to
>> > chat
>> > > > > more
>> > > > > >> > about
>> > > > > >> > > > > this
>> > > > > >> > > > > >> if
>> > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
>> started
>> > > with
>> > > > > the
>> > > > > >> > idea
>> > > > > >> > > > of
>> > > > > >> > > > > >> "what
>> > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
>> ingestion
>> > > > tool"
>> > > > > but
>> > > > > >> > > > > >> ultimately
>> > > > > >> > > > > >> > we
>> > > > > >> > > > > >> > >>> kind of came around to the idea that ingestion
>> > and
>> > > > > >> > > > transformation
>> > > > > >> > > > > >> had
>> > > > > >> > > > > >> > >>> pretty different needs and coupling the two
>> made
>> > > > things
>> > > > > >> > hard.
>> > > > > >> > > > > >> > >>>
>> > > > > >> > > > > >> > >>> For what it's worth I think copycat (KIP-26)
>> > > actually
>> > > > > will
>> > > > > >> > do
>> > > > > >> > > > what
>> > > > > >> > > > > >> you
>> > > > > >> > > > > >> > >> are
>> > > > > >> > > > > >> > >>> looking for.
>> > > > > >> > > > > >> > >>>
>> > > > > >> > > > > >> > >>> With regard to your point about slider, I
>> don't
>> > > > > >> necessarily
>> > > > > >> > > > > >> disagree.
>> > > > > >> > > > > >> > >> But I
>> > > > > >> > > > > >> > >>> think getting good YARN support is quite
>> doable
>> > > and I
>> > > > > >> think
>> > > > > >> > we
>> > > > > >> > > > can
>> > > > > >> > > > > >> make
>> > > > > >> > > > > >> > >>> that work well. I think the issue this
>> proposal
>> > > > solves
>> > > > > is
>> > > > > >> > that
>> > > > > >> > > > > >> > >> technically
>> > > > > >> > > > > >> > >>> it is pretty hard to support multiple cluster
>> > > > > management
>> > > > > >> > > systems
>> > > > > >> > > > > the
>> > > > > >> > > > > >> > way
>> > > > > >> > > > > >> > >>> things are now, you need to write an "app
>> master"
>> > > or
>> > > > > >> > > "framework"
>> > > > > >> > > > > for
>> > > > > >> > > > > >> > each
>> > > > > >> > > > > >> > >>> and they are all a little different so
>> testing is
>> > > > > really
>> > > > > >> > hard.
>> > > > > >> > > > In
>> > > > > >> > > > > >> the
>> > > > > >> > > > > >> > >>> absence of this we have been stuck with just
>> YARN
>> > > > which
>> > > > > >> has
>> > > > > >> > > > > >> fantastic
>> > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the org,
>> but
>> > > zero
>> > > > > >> > > penetration
>> > > > > >> > > > > >> > >> elsewhere.
>> > > > > >> > > > > >> > >>> Given the huge amount of work being put in to
>> > > slider,
>> > > > > >> > > marathon,
>> > > > > >> > > > > aws
>> > > > > >> > > > > >> > >>> tooling, not to mention the umpteen related
>> > > packaging
>> > > > > >> > > > technologies
>> > > > > >> > > > > >> > people
>> > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
>> > > > cloud-specific
>> > > > > >> > deploy
>> > > > > >> > > > > >> tools,
>> > > > > >> > > > > >> > >> etc)
>> > > > > >> > > > > >> > >>> I really think it is important to get this
>> right.
>> > > > > >> > > > > >> > >>>
>> > > > > >> > > > > >> > >>> -Jay
>> > > > > >> > > > > >> > >>>
>> > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
>> Turkington
>> > <
>> > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
>> > > > > >> > > > > >> > >>>
>> > > > > >> > > > > >> > >>>> Hi all,
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>> I think the question below re does Samza
>> become
>> > a
>> > > > > >> > sub-project
>> > > > > >> > > > of
>> > > > > >> > > > > >> Kafka
>> > > > > >> > > > > >> > >>>> highlights the broader point around
>> migration.
>> > > Chris
>> > > > > >> > mentions
>> > > > > >> > > > > >> Samza's
>> > > > > >> > > > > >> > >>>> maturity is heading towards a v1 release but
>> I'm
>> > > not
>> > > > > sure
>> > > > > >> > it
>> > > > > >> > > > > feels
>> > > > > >> > > > > >> > >> right to
>> > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to
>> deprecate
>> > > most
>> > > > of
>> > > > > >> it.
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>> From a selfish perspective I have some guys
>> who
>> > > have
>> > > > > >> > started
>> > > > > >> > > > > >> working
>> > > > > >> > > > > >> > >> with
>> > > > > >> > > > > >> > >>>> Samza and building some new
>> consumers/producers
>> > > was
>> > > > > next
>> > > > > >> > up.
>> > > > > >> > > > > Sounds
>> > > > > >> > > > > >> > like
>> > > > > >> > > > > >> > >>>> that is absolutely not the direction to go. I
>> > need
>> > > > to
>> > > > > >> look
>> > > > > >> > > into
>> > > > > >> > > > > the
>> > > > > >> > > > > >> > KIP
>> > > > > >> > > > > >> > >> in
>> > > > > >> > > > > >> > >>>> more detail but for me the attractiveness of
>> > > adding
>> > > > > new
>> > > > > >> > Samza
>> > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all they
>> were
>> > > > doing
>> > > > > was
>> > > > > >> > > > really
>> > > > > >> > > > > >> > getting
>> > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to avoid
>> > > having
>> > > > to
>> > > > > >> > worry
>> > > > > >> > > > > about
>> > > > > >> > > > > >> the
>> > > > > >> > > > > >> > >>>> lifecycle management of external clients. If
>> > there
>> > > > is
>> > > > > a
>> > > > > >> > > generic
>> > > > > >> > > > > >> Kafka
>> > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a new
>> > > connector
>> > > > > into
>> > > > > >> > and
>> > > > > >> > > > > have
>> > > > > >> > > > > >> a
>> > > > > >> > > > > >> > >> lot of
>> > > > > >> > > > > >> > >>>> the heavy lifting re scale and reliability
>> done
>> > > for
>> > > > me
>> > > > > >> then
>> > > > > >> > > it
>> > > > > >> > > > > >> gives
>> > > > > >> > > > > >> > me
>> > > > > >> > > > > >> > >> all
>> > > > > >> > > > > >> > >>>> the pushing new consumers/producers would. If
>> > not
>> > > > > then it
>> > > > > >> > > > > >> complicates
>> > > > > >> > > > > >> > my
>> > > > > >> > > > > >> > >>>> operational deployments.
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>> Which is similar to my other question with
>> the
>> > > > > proposal
>> > > > > >> --
>> > > > > >> > if
>> > > > > >> > > > we
>> > > > > >> > > > > >> > build a
>> > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus the
>> > > requisite
>> > > > > >> shims
>> > > > > >> > to
>> > > > > >> > > > > >> > integrate
>> > > > > >> > > > > >> > >>>> with Slider etc I suspect the former may be a
>> > lot
>> > > > more
>> > > > > >> work
>> > > > > >> > > > than
>> > > > > >> > > > > we
>> > > > > >> > > > > >> > >> think.
>> > > > > >> > > > > >> > >>>> We may make it much easier for a newcomer to
>> get
>> > > > > >> something
>> > > > > >> > > > > running
>> > > > > >> > > > > >> but
>> > > > > >> > > > > >> > >>>> having them step up and get a reliable
>> > production
>> > > > > >> > deployment
>> > > > > >> > > > may
>> > > > > >> > > > > >> still
>> > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
>> different
>> > > > > reasons
>> > > > > >> > than
>> > > > > >> > > > > >> today.
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with
>> > making
>> > > > the
>> > > > > >> Samza
>> > > > > >> > > > > >> dependency
>> > > > > >> > > > > >> > >> on
>> > > > > >> > > > > >> > >>>> Kafka much more explicit and I absolutely see
>> > the
>> > > > > >> benefits
>> > > > > >> > > in
>> > > > > >> > > > > the
>> > > > > >> > > > > >> > >>>> reduction of duplication and clashing
>> > > > > >> > > > terminologies/abstractions
>> > > > > >> > > > > >> that
>> > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library would
>> > > likely
>> > > > > be a
>> > > > > >> > very
>> > > > > >> > > > > nice
>> > > > > >> > > > > >> > tool
>> > > > > >> > > > > >> > >> to
>> > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the
>> > > concerns
>> > > > > >> above
>> > > > > >> > re
>> > > > > >> > > > the
>> > > > > >> > > > > >> > >>>> operational side.
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>> Garry
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>> -----Original Message-----
>> > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
>> > > > > >> > gdfm@apache.org
>> > > > > >> > > ]
>> > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
>> > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
>> > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on
>> Samza
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>> Very interesting thoughts.
>> > > > > >> > > > > >> > >>>> From outside, I have always perceived Samza
>> as a
>> > > > > >> computing
>> > > > > >> > > > layer
>> > > > > >> > > > > >> over
>> > > > > >> > > > > >> > >>>> Kafka.
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>> The question, maybe a bit provocative, is
>> > "should
>> > > > > Samza
>> > > > > >> be
>> > > > > >> > a
>> > > > > >> > > > > >> > sub-project
>> > > > > >> > > > > >> > >>>> of Kafka then?"
>> > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a
>> separate
>> > > > project
>> > > > > >> > with a
>> > > > > >> > > > > >> separate
>> > > > > >> > > > > >> > >>>> governance?
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>> Cheers,
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>> --
>> > > > > >> > > > > >> > >>>> Gianmarco
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
>> > > > > yanfang724@gmail.com>
>> > > > > >> > > > wrote:
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more
>> > > tightly.
>> > > > > >> > Because
>> > > > > >> > > > > Samza
>> > > > > >> > > > > >> de
>> > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
>> leverage
>> > > > what
>> > > > > >> Kafka
>> > > > > >> > > > has.
>> > > > > >> > > > > At
>> > > > > >> > > > > >> > the
>> > > > > >> > > > > >> > >>>>> same time, Kafka does not need to reinvent
>> what
>> > > > Samza
>> > > > > >> > > already
>> > > > > >> > > > > >> has. I
>> > > > > >> > > > > >> > >>>>> also like the idea of separating the
>> ingestion
>> > > and
>> > > > > >> > > > > transformation.
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>> But it is a little difficult for me to image
>> > how
>> > > > the
>> > > > > >> Samza
>> > > > > >> > > > will
>> > > > > >> > > > > >> look
>> > > > > >> > > > > >> > >>>> like.
>> > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
>> > difference
>> > > > in
>> > > > > >> terms
>> > > > > >> > > of
>> > > > > >> > > > > how
>> > > > > >> > > > > >> > >>>>> Samza should look like.
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code shows
>> (A
>> > > > > client of
>> > > > > >> > > > Kakfa)
>> > > > > >> > > > > ?
>> > > > > >> > > > > >> And
>> > > > > >> > > > > >> > >>>>> user's application code calls this client?
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka
>> (like
>> > > > what
>> > > > > the
>> > > > > >> > > code
>> > > > > >> > > > > >> shows),
>> > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
>> > > > fault-tolerance?
>> > > > > >> Are
>> > > > > >> > > they
>> > > > > >> > > > > >> taken
>> > > > > >> > > > > >> > >>>>> care by the Kafka broker or other mechanism,
>> > such
>> > > > as
>> > > > > >> > "Samza
>> > > > > >> > > > > >> worker"
>> > > > > >> > > > > >> > >>>>> (just make up the name) ?
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>> 2. What about other features, such as
>> > > auto-scaling,
>> > > > > >> shared
>> > > > > >> > > > > state,
>> > > > > >> > > > > >> > >>>>> monitoring?
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is this
>> what
>> > > > Chris
>> > > > > >> > > > suggests?)
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa
>> and
>> > > > > produce
>> > > > > >> to
>> > > > > >> > > it.
>> > > > > >> > > > > >> Then it
>> > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks like
>> now,
>> > > > > except it
>> > > > > >> > > does
>> > > > > >> > > > > not
>> > > > > >> > > > > >> > rely
>> > > > > >> > > > > >> > >>>>> on Yarn anymore.
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it leverage
>> > > Kafka's
>> > > > > >> > metrics,
>> > > > > >> > > > > logs,
>> > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>> Thanks,
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>> Fang, Yan
>> > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang
>> Wang <
>> > > > > >> > > > > wangguoz@gmail.com
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> > >>>> wrote:
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>>> Read through the code example and it looks
>> > good
>> > > to
>> > > > > me.
>> > > > > >> A
>> > > > > >> > > few
>> > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable runnable
>> > like:
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
>> > --config-factory=...
>> > > > > >> > > > > >> > >>>> --config-path=file://...
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>> And this proposal advocate for deploying
>> Samza
>> > > > more
>> > > > > as
>> > > > > >> > > > embedded
>> > > > > >> > > > > >> > >>>>>> libraries in user application code
>> (ignoring
>> > the
>> > > > > >> > > terminology
>> > > > > >> > > > > >> since
>> > > > > >> > > > > >> > >>>>>> it is not the
>> > > > > >> > > > > >> > >>>>> same
>> > > > > >> > > > > >> > >>>>>> as the prototype code):
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>> StreamTask task = new
>> MyStreamTask(configs);
>> > > > Thread
>> > > > > >> > thread
>> > > > > >> > > =
>> > > > > >> > > > > new
>> > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>> I think both of these deployment modes are
>> > > > important
>> > > > > >> for
>> > > > > >> > > > > >> different
>> > > > > >> > > > > >> > >>>>>> types
>> > > > > >> > > > > >> > >>>>> of
>> > > > > >> > > > > >> > >>>>>> users. That said, I think making Samza
>> purely
>> > > > > >> standalone
>> > > > > >> > is
>> > > > > >> > > > > still
>> > > > > >> > > > > >> > >>>>>> sufficient for either runnable or library
>> > modes.
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>> Guozhang
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay
>> Kreps <
>> > > > > >> > > > jay@confluent.io>
>> > > > > >> > > > > >> > wrote:
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code
>> example, it
>> > > was
>> > > > > >> > supposed
>> > > > > >> > > > to
>> > > > > >> > > > > >> look
>> > > > > >> > > > > >> > >>>>>>> like
>> > > > > >> > > > > >> > >>>>>>> this:
>> > > > > >> > > > > >> > >>>>>>>
>> > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
>> > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
>> > > "localhost:4242");
>> > > > > >> > > > > >> StreamingConfig
>> > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
>> > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
>> > > "test-topic-2");
>> > > > > >> > > > > >> > >>>>>>>
>> > config.processor(ExampleStreamProcessor.class);
>> > > > > >> > > > > >> > >>>>>>> config.serialization(new
>> StringSerializer(),
>> > > new
>> > > > > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming
>> > > container =
>> > > > > new
>> > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
>> > > > > >> > > > > >> > >>>>>>>
>> > > > > >> > > > > >> > >>>>>>> -Jay
>> > > > > >> > > > > >> > >>>>>>>
>> > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay
>> Kreps <
>> > > > > >> > > > jay@confluent.io
>> > > > > >> > > > > >
>> > > > > >> > > > > >> > >>>> wrote:
>> > > > > >> > > > > >> > >>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> Hey guys,
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> This came out of some conversations Chris
>> > and
>> > > I
>> > > > > were
>> > > > > >> > > having
>> > > > > >> > > > > >> > >>>>>>>> around
>> > > > > >> > > > > >> > >>>>>>> whether
>> > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a
>> kind
>> > of
>> > > > data
>> > > > > >> > > > ingestion
>> > > > > >> > > > > >> > >>>>> framework
>> > > > > >> > > > > >> > >>>>>>> for
>> > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26
>> > > > "copycat").
>> > > > > >> This
>> > > > > >> > > > kind
>> > > > > >> > > > > of
>> > > > > >> > > > > >> > >>>>>> combined
>> > > > > >> > > > > >> > >>>>>>>> with complaints around config and YARN
>> and
>> > the
>> > > > > >> > discussion
>> > > > > >> > > > > >> around
>> > > > > >> > > > > >> > >>>>>>>> how
>> > > > > >> > > > > >> > >>>>> to
>> > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> So the thought experiment was, given that
>> > > Samza
>> > > > > was
>> > > > > >> > > > basically
>> > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific, what if
>> you
>> > > just
>> > > > > >> > embraced
>> > > > > >> > > > > that
>> > > > > >> > > > > >> > >>>>>>>> and turned it
>> > > > > >> > > > > >> > >>>>>> into
>> > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
>> framework
>> > > and
>> > > > > more
>> > > > > >> > > like a
>> > > > > >> > > > > >> > >>>>>>>> third
>> > > > > >> > > > > >> > >>>>> Kafka
>> > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing consumer"
>> with
>> > > > state
>> > > > > >> > > > management
>> > > > > >> > > > > >> > >>>>>> facilities.
>> > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a complex
>> > > stream
>> > > > > >> > > processing
>> > > > > >> > > > > >> > >>>>>>>> framework
>> > > > > >> > > > > >> > >>>>>>> this
>> > > > > >> > > > > >> > >>>>>>>> would actually be a very simple thing,
>> not
>> > > much
>> > > > > more
>> > > > > >> > > > > >> complicated
>> > > > > >> > > > > >> > >>>>>>>> to
>> > > > > >> > > > > >> > >>>>> use
>> > > > > >> > > > > >> > >>>>>>> or
>> > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris
>> said
>> > > we
>> > > > > >> thought
>> > > > > >> > > > about
>> > > > > >> > > > > >> it
>> > > > > >> > > > > >> > >>>>>>>> a
>> > > > > >> > > > > >> > >>>>> lot
>> > > > > >> > > > > >> > >>>>>> of
>> > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
>> processing
>> > > > > systems
>> > > > > >> > were
>> > > > > >> > > > > doing)
>> > > > > >> > > > > >> > >>>>> seemed
>> > > > > >> > > > > >> > >>>>>>> like
>> > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output data
>> to
>> > > and
>> > > > > from
>> > > > > >> > the
>> > > > > >> > > > > stream
>> > > > > >> > > > > >> > >>>>>>>> processing. But when we actually looked
>> into
>> > > how
>> > > > > that
>> > > > > >> > > would
>> > > > > >> > > > > >> > >>>>>>>> work,
>> > > > > >> > > > > >> > >>>>> Samza
>> > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion
>> > framework
>> > > > > for a
>> > > > > >> > > bunch
>> > > > > >> > > > of
>> > > > > >> > > > > >> > >>>>> reasons.
>> > > > > >> > > > > >> > >>>>>> To
>> > > > > >> > > > > >> > >>>>>>>> really do that right you need a pretty
>> > > different
>> > > > > >> > internal
>> > > > > >> > > > > data
>> > > > > >> > > > > >> > >>>>>>>> model
>> > > > > >> > > > > >> > >>>>>> and
>> > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split them
>> and
>> > had
>> > > > an
>> > > > > api
>> > > > > >> > for
>> > > > > >> > > > > Kafka
>> > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a
>> > > > separate
>> > > > > >> api
>> > > > > >> > > for
>> > > > > >> > > > > >> Kafka
>> > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> This would also allow really embracing
>> the
>> > > same
>> > > > > >> > > terminology
>> > > > > >> > > > > and
>> > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about the
>> current
>> > > > > state is
>> > > > > >> > > that
>> > > > > >> > > > > the
>> > > > > >> > > > > >> > >>>>>>>> two
>> > > > > >> > > > > >> > >>>>>>> systems
>> > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like
>> > > > "stream"
>> > > > > vs
>> > > > > >> > > > "topic"
>> > > > > >> > > > > >> and
>> > > > > >> > > > > >> > >>>>>>> different
>> > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means you
>> kind
>> > > of
>> > > > > have
>> > > > > >> to
>> > > > > >> > > > learn
>> > > > > >> > > > > >> > >>>>>>>> Kafka's
>> > > > > >> > > > > >> > >>>>>>> way,
>> > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly different
>> way,
>> > > then
>> > > > > kind
>> > > > > >> of
>> > > > > >> > > > > >> > >>>>>>>> understand
>> > > > > >> > > > > >> > >>>>> how
>> > > > > >> > > > > >> > >>>>>>> they
>> > > > > >> > > > > >> > >>>>>>>> map to each other, which having walked a
>> few
>> > > > > people
>> > > > > >> > > through
>> > > > > >> > > > > >> this
>> > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of time
>> on
>> > > > > >> airplanes I
>> > > > > >> > > > > hacked
>> > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat
>> incomplete
>> > > > > prototype
>> > > > > >> of
>> > > > > >> > > > what
>> > > > > >> > > > > >> > >>>>>>>> this would
>> > > > > >> > > > > >> > >>>>> look
>> > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously dumped
>> > into
>> > > > > Kafka
>> > > > > >> as
>> > > > > >> > > it
>> > > > > >> > > > > >> > >>>>>>>> required a
>> > > > > >> > > > > >> > >>>>>> few
>> > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is the
>> > code:
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> >
>> > > > > >> > > > >
>> > > > > >> >
>> > > > >
>> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I just
>> > > > liberally
>> > > > > >> > renamed
>> > > > > >> > > > > >> > >>>>>>>> everything
>> > > > > >> > > > > >> > >>>>> to
>> > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no regard
>> > for
>> > > > > >> > > > compatibility.
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> To use this would be something like this:
>> > > > > >> > > > > >> > >>>>>>>> Properties props = new Properties();
>> > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
>> > > > "localhost:4242");
>> > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
>> > > > > >> > > > > >> > >>>>> StreamingConfig(props);
>> > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
>> > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
>> > > > > >> > > > > >> config.processor(ExampleStreamProcessor.class);
>> > > > > >> > > > > >> > >>>>>>> config.serialization(new
>> > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
>> > StringDeserializer());
>> > > > > >> > > > KafkaStreaming
>> > > > > >> > > > > >> > >>>>>> container =
>> > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
>> container.run();
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
>> > > SamzaContainer;
>> > > > > >> > > > > StreamProcessor
>> > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> So rather than putting all the class
>> names
>> > in
>> > > a
>> > > > > file
>> > > > > >> > and
>> > > > > >> > > > then
>> > > > > >> > > > > >> > >>>>>>>> having
>> > > > > >> > > > > >> > >>>>>> the
>> > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
>> > > > instantiate
>> > > > > the
>> > > > > >> > > > > container
>> > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced over
>> > > however
>> > > > > many
>> > > > > >> > > > > instances
>> > > > > >> > > > > >> > >>>>>>>> of
>> > > > > >> > > > > >> > >>>>> this
>> > > > > >> > > > > >> > >>>>>>> are
>> > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance
>> dies,
>> > > new
>> > > > > >> tasks
>> > > > > >> > > are
>> > > > > >> > > > > >> added
>> > > > > >> > > > > >> > >>>>>>>> to
>> > > > > >> > > > > >> > >>>>> the
>> > > > > >> > > > > >> > >>>>>>>> existing containers without shutting them
>> > > down).
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> We would provide some glue for running
>> this
>> > > > stuff
>> > > > > in
>> > > > > >> > YARN
>> > > > > >> > > > via
>> > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using
>> > some
>> > > > of
>> > > > > >> their
>> > > > > >> > > > tools
>> > > > > >> > > > > >> > >>>>>>>> but from the
>> > > > > >> > > > > >> > >>>>>> point
>> > > > > >> > > > > >> > >>>>>>> of
>> > > > > >> > > > > >> > >>>>>>>> view of these frameworks these stream
>> > > processing
>> > > > > jobs
>> > > > > >> > are
>> > > > > >> > > > > just
>> > > > > >> > > > > >> > >>>>>> stateless
>> > > > > >> > > > > >> > >>>>>>>> services that can come and go and expand
>> and
>> > > > > contract
>> > > > > >> > at
>> > > > > >> > > > > will.
>> > > > > >> > > > > >> > >>>>>>>> There
>> > > > > >> > > > > >> > >>>>> is
>> > > > > >> > > > > >> > >>>>>>> no
>> > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it
>> would
>> > > get
>> > > > > >> larger
>> > > > > >> > > if
>> > > > > >> > > > we
>> > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly larger. We
>> > > really
>> > > > > do
>> > > > > >> > get a
>> > > > > >> > > > ton
>> > > > > >> > > > > >> > >>>>>>>> of
>> > > > > >> > > > > >> > >>>>>>> leverage
>> > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
>> > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
>> delegated
>> > to
>> > > > the
>> > > > > >> new
>> > > > > >> > > > > >> consumer.
>> > > > > >> > > > > >> > >>>>> This
>> > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
>> management
>> > > > > strategy
>> > > > > >> > > > > available
>> > > > > >> > > > > >> > >>>>>>>> to
>> > > > > >> > > > > >> > >>>>>> Kafka
>> > > > > >> > > > > >> > >>>>>>>>  consumer is also available to Samza (and
>> > vice
>> > > > > versa)
>> > > > > >> > and
>> > > > > >> > > > > with
>> > > > > >> > > > > >> > >>>>>>>> the
>> > > > > >> > > > > >> > >>>>>>> exact
>> > > > > >> > > > > >> > >>>>>>>>  same configs.
>> > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as state
>> reuse
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
>> thought
>> > > > > >> provoking.
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> -Jay
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris
>> > > > Riccomini <
>> > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
>> > > > > >> > > > > >> > >>>>>>>> wrote:
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Hey all,
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with Samza
>> > > > engineers
>> > > > > at
>> > > > > >> > > > LinkedIn
>> > > > > >> > > > > >> > >>>>>>>>> and
>> > > > > >> > > > > >> > >>>>>>> Confluent
>> > > > > >> > > > > >> > >>>>>>>>> and we came up with a few observations
>> and
>> > > > would
>> > > > > >> like
>> > > > > >> > to
>> > > > > >> > > > > >> > >>>>>>>>> propose
>> > > > > >> > > > > >> > >>>>> some
>> > > > > >> > > > > >> > >>>>>>>>> changes.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I want
>> to
>> > > call
>> > > > > out
>> > > > > >> > about
>> > > > > >> > > > > >> > >>>>>>>>> Samza's
>> > > > > >> > > > > >> > >>>>>> design,
>> > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic
>> > > deployment
>> > > > > >> system.
>> > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
>> > > > > >> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer
>> and
>> > > > > Kafka's
>> > > > > >> > > > consumer
>> > > > > >> > > > > >> > >>>>>>>>> APIs
>> > > > > >> > > > > >> > >>>>> are
>> > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
>> problems.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> All three of these issues are related,
>> but
>> > > I'll
>> > > > > >> > address
>> > > > > >> > > > them
>> > > > > >> > > > > >> in
>> > > > > >> > > > > >> > >>>>> order.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Deployment
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a
>> > > dynamic
>> > > > > >> > > deployment
>> > > > > >> > > > > >> > >>>>>>>>> scheduler
>> > > > > >> > > > > >> > >>>>>> such
>> > > > > >> > > > > >> > >>>>>>>>> as
>> > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially
>> built
>> > > > Samza,
>> > > > > we
>> > > > > >> > bet
>> > > > > >> > > > that
>> > > > > >> > > > > >> > >>>>>>>>> there
>> > > > > >> > > > > >> > >>>>>> would
>> > > > > >> > > > > >> > >>>>>>>>> be
>> > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area, and we
>> > could
>> > > > > >> support
>> > > > > >> > > > them,
>> > > > > >> > > > > >> and
>> > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > >> > > > > >> > >>>>>> rest
>> > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there are
>> many
>> > > > > >> variations.
>> > > > > >> > > > > >> > >>>>>>>>> Furthermore,
>> > > > > >> > > > > >> > >>>>>> many
>> > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start their
>> > > > > processors
>> > > > > >> > like
>> > > > > >> > > > > normal
>> > > > > >> > > > > >> > >>>>>>>>> Java processes, and use traditional
>> > > deployment
>> > > > > >> scripts
>> > > > > >> > > > such
>> > > > > >> > > > > as
>> > > > > >> > > > > >> > >>>>>>>>> Fabric,
>> > > > > >> > > > > >> > >>>>>> Chef,
>> > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment
>> system
>> > on
>> > > > > users
>> > > > > >> > makes
>> > > > > >> > > > the
>> > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really painful
>> for
>> > > first
>> > > > > time
>> > > > > >> > > > users.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was
>> > also
>> > > a
>> > > > > bit
>> > > > > >> of
>> > > > > >> > a
>> > > > > >> > > > > >> > >>>>>>>>> mis-fire
>> > > > > >> > > > > >> > >>>>>> because
>> > > > > >> > > > > >> > >>>>>>>>> of
>> > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between
>> the
>> > > > > nature of
>> > > > > >> > > batch
>> > > > > >> > > > > >> jobs
>> > > > > >> > > > > >> > >>>>>>>>> and
>> > > > > >> > > > > >> > >>>>>>> stream
>> > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
>> > conscious
>> > > > > effort
>> > > > > >> to
>> > > > > >> > > > favor
>> > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > >> > > > > >> > >>>>>> Hadoop
>> > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since
>> it
>> > > > worked
>> > > > > >> and
>> > > > > >> > > was
>> > > > > >> > > > > well
>> > > > > >> > > > > >> > >>>>>>> understood.
>> > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that batch
>> > jobs
>> > > > > have a
>> > > > > >> > > > definite
>> > > > > >> > > > > >> > >>>>>> beginning,
>> > > > > >> > > > > >> > >>>>>>>>> and
>> > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't
>> > > > (usually).
>> > > > > >> This
>> > > > > >> > > > leads
>> > > > > >> > > > > to
>> > > > > >> > > > > >> > >>>>>>>>> a
>> > > > > >> > > > > >> > >>>>> much
>> > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream
>> > > > processors.
>> > > > > >> You
>> > > > > >> > > > > >> basically
>> > > > > >> > > > > >> > >>>>>>>>> just
>> > > > > >> > > > > >> > >>>>>>> need
>> > > > > >> > > > > >> > >>>>>>>>> to find a place to start the processor,
>> and
>> > > > start
>> > > > > >> it.
>> > > > > >> > > The
>> > > > > >> > > > > way
>> > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no
>> > concept
>> > > > of
>> > > > > a
>> > > > > >> > > cluster
>> > > > > >> > > > > >> > >>>>>>>>> being "full". We always
>> > > > > >> > > > > >> > >>>>>> add
>> > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with coupling
>> > > Samza
>> > > > > with
>> > > > > >> a
>> > > > > >> > > > > >> scheduler
>> > > > > >> > > > > >> > >>>>>>>>> is
>> > > > > >> > > > > >> > >>>>>> that
>> > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to handle
>> > > > > deployment.
>> > > > > >> > > This
>> > > > > >> > > > > >> pulls
>> > > > > >> > > > > >> > >>>>>>>>> in a
>> > > > > >> > > > > >> > >>>>>>> bunch
>> > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
>> > distribution
>> > > > > (config
>> > > > > >> > > > > stream),
>> > > > > >> > > > > >> > >>>>>>>>> shell
>> > > > > >> > > > > >> > >>>>>>> scrips
>> > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging
>> (all
>> > > the
>> > > > > .tgz
>> > > > > >> > > > stuff),
>> > > > > >> > > > > >> etc.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
>> > > deployment
>> > > > > was
>> > > > > >> to
>> > > > > >> > > > > support
>> > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
>> > locality,
>> > > > you
>> > > > > >> need
>> > > > > >> > to
>> > > > > >> > > > put
>> > > > > >> > > > > >> > >>>>>>>>> your
>> > > > > >> > > > > >> > >>>>>> processors
>> > > > > >> > > > > >> > >>>>>>>>> close to the data they're processing.
>> Upon
>> > > > > further
>> > > > > >> > > > > >> > >>>>>>>>> investigation,
>> > > > > >> > > > > >> > >>>>>>> though,
>> > > > > >> > > > > >> > >>>>>>>>> this feature is not that beneficial.
>> There
>> > is
>> > > > > some
>> > > > > >> > good
>> > > > > >> > > > > >> > >>>>>>>>> discussion
>> > > > > >> > > > > >> > >>>>>> about
>> > > > > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335.
>> Again,
>> > we
>> > > > > took
>> > > > > >> the
>> > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
>> > > > > >> > > > > >> > >>>>>> path,
>> > > > > >> > > > > >> > >>>>>>>>> but
>> > > > > >> > > > > >> > >>>>>>>>> there are some fundamental differences
>> > > between
>> > > > > HDFS
>> > > > > >> > and
>> > > > > >> > > > > Kafka.
>> > > > > >> > > > > >> > >>>>>>>>> HDFS
>> > > > > >> > > > > >> > >>>>>> has
>> > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions. This
>> > > leads
>> > > > to
>> > > > > >> less
>> > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
>> > processors
>> > > > on
>> > > > > top
>> > > > > >> > of
>> > > > > >> > > > > Kafka.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a crutch.
>> > Samza
>> > > > > doesn't
>> > > > > >> > > have
>> > > > > >> > > > > any
>> > > > > >> > > > > >> > >>>>>>>>> built
>> > > > > >> > > > > >> > >>>>> in
>> > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
>> depends
>> > on
>> > > > the
>> > > > > >> > > dynamic
>> > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to handle
>> > > restarts
>> > > > > >> when a
>> > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
>> > > > > >> > > > > >> > >>>>>>> made
>> > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a standalone
>> > Samza
>> > > > > >> > container
>> > > > > >> > > > > >> > >>>> (SAMZA-516).
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Pluggability
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is good, but
>> I
>> > > think
>> > > > > that
>> > > > > >> > > we've
>> > > > > >> > > > > >> gone
>> > > > > >> > > > > >> > >>>>>>>>> too
>> > > > > >> > > > > >> > >>>>>> far
>> > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
>> > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
>> > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
>> > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
>> > > (SystemConsumer,
>> > > > > >> > > > > SystemProducer,
>> > > > > >> > > > > >> > >>>> etc).
>> > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
>> > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
>> > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about
>> every
>> > > > > >> component
>> > > > > >> > > > > >> > >>>>> (MessageChooser,
>> > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
>> > ConfigRewriter,
>> > > > > etc).
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
>> forgotten,
>> > as
>> > > > > well.
>> > > > > >> > Some
>> > > > > >> > > > of
>> > > > > >> > > > > >> > >>>>>>>>> these
>> > > > > >> > > > > >> > >>>>> are
>> > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not to be.
>> > This
>> > > > all
>> > > > > >> comes
>> > > > > >> > > at
>> > > > > >> > > > a
>> > > > > >> > > > > >> cost:
>> > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is making it
>> > > harder
>> > > > > for
>> > > > > >> > our
>> > > > > >> > > > > users
>> > > > > >> > > > > >> > >>>>>>>>> to
>> > > > > >> > > > > >> > >>>>> pick
>> > > > > >> > > > > >> > >>>>>> up
>> > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also
>> makes
>> > > it
>> > > > > >> > difficult
>> > > > > >> > > > for
>> > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about what
>> the
>> > > > > >> > > characteristics
>> > > > > >> > > > of
>> > > > > >> > > > > >> > >>>>>>>>> the container (since the characteristics
>> > > change
>> > > > > >> > > depending
>> > > > > >> > > > on
>> > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are most
>> > visible
>> > > > in
>> > > > > the
>> > > > > >> > > > System
>> > > > > >> > > > > >> APIs.
>> > > > > >> > > > > >> > >>>>> What
>> > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be functional
>> is
>> > > Kafka
>> > > > > as
>> > > > > >> its
>> > > > > >> > > > > >> > >>>>>>>>> transport
>> > > > > >> > > > > >> > >>>>>> layer.
>> > > > > >> > > > > >> > >>>>>>>>> But
>> > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases
>> > into
>> > > > one
>> > > > > >> API:
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
>> > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> The current System API supports both of
>> > these
>> > > > use
>> > > > > >> > cases.
>> > > > > >> > > > The
>> > > > > >> > > > > >> > >>>>>>>>> problem
>> > > > > >> > > > > >> > >>>>>> is,
>> > > > > >> > > > > >> > >>>>>>>>> we
>> > > > > >> > > > > >> > >>>>>>>>> actually want different features for
>> each
>> > use
>> > > > > case.
>> > > > > >> By
>> > > > > >> > > > > >> papering
>> > > > > >> > > > > >> > >>>>>>>>> over
>> > > > > >> > > > > >> > >>>>>>> these
>> > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a single
>> API,
>> > > > we've
>> > > > > >> > > > introduced
>> > > > > >> > > > > a
>> > > > > >> > > > > >> > >>>>>>>>> ton of
>> > > > > >> > > > > >> > >>>>>>> leaky
>> > > > > >> > > > > >> > >>>>>>>>> abstractions.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like in
>> (2)
>> > is
>> > > to
>> > > > > have
>> > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
>> offsets
>> > > > (like
>> > > > > >> > Kafka).
>> > > > > >> > > > > This
>> > > > > >> > > > > >> > >>>>>>>>> would be at odds
>> > > > > >> > > > > >> > >>>>> with
>> > > > > >> > > > > >> > >>>>>>> (1),
>> > > > > >> > > > > >> > >>>>>>>>> though, since different systems have
>> > > different
>> > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
>> > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the mailing
>> > list
>> > > > and
>> > > > > >> the
>> > > > > >> > > SQL
>> > > > > >> > > > > >> JIRAs
>> > > > > >> > > > > >> > >>>>> about
>> > > > > >> > > > > >> > >>>>>>> the
>> > > > > >> > > > > >> > >>>>>>>>> need for this.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
>> > replayability.
>> > > > > Kafka
>> > > > > >> > > allows
>> > > > > >> > > > us
>> > > > > >> > > > > >> to
>> > > > > >> > > > > >> > >>>>> rewind
>> > > > > >> > > > > >> > >>>>>>>>> when
>> > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other systems
>> > don't.
>> > > In
>> > > > > some
>> > > > > >> > > > cases,
>> > > > > >> > > > > >> > >>>>>>>>> systems
>> > > > > >> > > > > >> > >>>>>>> return
>> > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
>> > > > > >> WikipediaSystemConsumer)
>> > > > > >> > > > > because
>> > > > > >> > > > > >> > >>>>>>>>> they
>> > > > > >> > > > > >> > >>>>>> have
>> > > > > >> > > > > >> > >>>>>>> no
>> > > > > >> > > > > >> > >>>>>>>>> offsets.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka
>> > > supports
>> > > > > >> > > > > partitioning,
>> > > > > >> > > > > >> > >>>>>>>>> but
>> > > > > >> > > > > >> > >>>>> many
>> > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by having a
>> > > single
>> > > > > >> > > partition
>> > > > > >> > > > > for
>> > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other systems
>> model
>> > > > > >> partitioning
>> > > > > >> > > > > >> > >>>> differently (e.g.
>> > > > > >> > > > > >> > >>>>>>>>> Kinesis).
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a
>> mess.
>> > > > > Creating
>> > > > > >> > > streams
>> > > > > >> > > > > in
>> > > > > >> > > > > >> a
>> > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost
>> impossible.
>> > As
>> > > is
>> > > > > >> > modeling
>> > > > > >> > > > > >> > >>>>>>>>> metadata
>> > > > > >> > > > > >> > >>>>> for
>> > > > > >> > > > > >> > >>>>>>> the
>> > > > > >> > > > > >> > >>>>>>>>> system (replication factor, partitions,
>> > > > location,
>> > > > > >> > etc).
>> > > > > >> > > > The
>> > > > > >> > > > > >> > >>>>>>>>> list
>> > > > > >> > > > > >> > >>>>> goes
>> > > > > >> > > > > >> > >>>>>>> on.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Duplicate work
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing Samza,
>> > > > Kafka's
>> > > > > >> > > consumer
>> > > > > >> > > > > and
>> > > > > >> > > > > >> > >>>>> producer
>> > > > > >> > > > > >> > >>>>>>>>> APIs
>> > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On
>> the
>> > > > > >> > consumer-side,
>> > > > > >> > > > you
>> > > > > >> > > > > >> > >>>>>>>>> had two
>> > > > > >> > > > > >> > >>>>>>>>> options: use the high level consumer, or
>> > the
>> > > > > simple
>> > > > > >> > > > > consumer.
>> > > > > >> > > > > >> > >>>>>>>>> The
>> > > > > >> > > > > >> > >>>>>>> problem
>> > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was that it
>> > > > > controlled
>> > > > > >> > your
>> > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and the
>> > order
>> > > > in
>> > > > > >> which
>> > > > > >> > > you
>> > > > > >> > > > > >> > >>>>>>>>> received messages. The
>> > > > > >> > > > > >> > >>>>> problem
>> > > > > >> > > > > >> > >>>>>>>>> with
>> > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not
>> > simple.
>> > > > It's
>> > > > > >> > basic.
>> > > > > >> > > > You
>> > > > > >> > > > > >> > >>>>>>>>> end up
>> > > > > >> > > > > >> > >>>>>>> having
>> > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level
>> stuff
>> > > that
>> > > > > you
>> > > > > >> > > > > shouldn't.
>> > > > > >> > > > > >> > >>>>>>>>> We
>> > > > > >> > > > > >> > >>>>>> spent a
>> > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
>> > > KafkaSystemConsumer
>> > > > > very
>> > > > > >> > > > robust.
>> > > > > >> > > > > >> It
>> > > > > >> > > > > >> > >>>>>>>>> also allows us to support some cool
>> > features:
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
>> > > > > prioritization.
>> > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition
>> assignment
>> > to
>> > > > > support
>> > > > > >> > > > joins,
>> > > > > >> > > > > >> > >>>>>>>>> global
>> > > > > >> > > > > >> > >>>>>> state
>> > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
>> > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
>> checkpointing.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time is
>> that
>> > > > these
>> > > > > >> > > features
>> > > > > >> > > > > >> > >>>>>>>>> should
>> > > > > >> > > > > >> > >>>>>>> actually
>> > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers
>> (not
>> > > just
>> > > > > >> Samza
>> > > > > >> > > > stream
>> > > > > >> > > > > >> > >>>>>> processors)
>> > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like joins
>> and
>> > > > > partition
>> > > > > >> > > > > >> > >>>>>>>>> assignment. The
>> > > > > >> > > > > >> > >>>>>>> Kafka
>> > > > > >> > > > > >> > >>>>>>>>> community has come to the same
>> conclusion.
>> > > > > They're
>> > > > > >> > > adding
>> > > > > >> > > > a
>> > > > > >> > > > > >> ton
>> > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka
>> consumer
>> > > > > >> > > implementation.
>> > > > > >> > > > > To a
>> > > > > >> > > > > >> > >>>>>>>>> large extent,
>> > > > > >> > > > > >> > >>>>> it's
>> > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've already
>> done
>> > in
>> > > > > Samza.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a
>> > very
>> > > > > similar
>> > > > > >> > > > > approach
>> > > > > >> > > > > >> > >>>>>>>>> to
>> > > > > >> > > > > >> > >>>>>> Samza's
>> > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation
>> for
>> > > > > handling
>> > > > > >> > > offset
>> > > > > >> > > > > >> > >>>>>> checkpointing.
>> > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset
>> management
>> > > > feature
>> > > > > >> > stores
>> > > > > >> > > > > >> offset
>> > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you
>> to
>> > > fetch
>> > > > > them
>> > > > > >> > > from
>> > > > > >> > > > > the
>> > > > > >> > > > > >> > >>>>>>>>> broker.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since
>> we
>> > > > could
>> > > > > >> have
>> > > > > >> > > > shared
>> > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > >> > > > > >> > >>>>> work
>> > > > > >> > > > > >> > >>>>>> if
>> > > > > >> > > > > >> > >>>>>>>>> it
>> > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Vision
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather radical
>> > > > > proposal.
>> > > > > >> > Samza
>> > > > > >> > > > is
>> > > > > >> > > > > >> > >>>>> relatively
>> > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to say
>> > that
>> > > > > we're
>> > > > > >> > > near a
>> > > > > >> > > > > 1.0
>> > > > > >> > > > > >> > >>>>>> release.
>> > > > > >> > > > > >> > >>>>>>>>> I'd
>> > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what we've
>> > > > learned,
>> > > > > and
>> > > > > >> > > begin
>> > > > > >> > > > > >> > >>>>>>>>> thinking
>> > > > > >> > > > > >> > >>>>>>> about
>> > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change
>> if
>> > we
>> > > > were
>> > > > > >> > > starting
>> > > > > >> > > > > >> from
>> > > > > >> > > > > >> > >>>>>> scratch?
>> > > > > >> > > > > >> > >>>>>>>>> My
>> > > > > >> > > > > >> > >>>>>>>>> proposal is to:
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way
>> to
>> > > run
>> > > > > Samza
>> > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
>> > > > dependences
>> > > > > on
>> > > > > >> > > YARN,
>> > > > > >> > > > > >> Mesos,
>> > > > > >> > > > > >> > >>>> etc.
>> > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support
>> only
>> > > Kafka
>> > > > > as
>> > > > > >> the
>> > > > > >> > > > > stream
>> > > > > >> > > > > >> > >>>>>> processing
>> > > > > >> > > > > >> > >>>>>>>>> layer.
>> > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
>> > > > > >> serialization,
>> > > > > >> > > and
>> > > > > >> > > > > >> > >>>>>>>>> config
>> > > > > >> > > > > >> > >>>>>>> systems,
>> > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues that I
>> > > > outlined
>> > > > > >> > above.
>> > > > > >> > > It
>> > > > > >> > > > > >> > >>>>>>>>> should
>> > > > > >> > > > > >> > >>>>> also
>> > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
>> > > dramatically.
>> > > > > >> > > Supporting
>> > > > > >> > > > > >> only
>> > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow Samza
>> to
>> > be
>> > > > > >> executed
>> > > > > >> > > on
>> > > > > >> > > > > YARN
>> > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
>> > > Marathon/Aurora),
>> > > > or
>> > > > > >> most
>> > > > > >> > > > other
>> > > > > >> > > > > >> > >>>>>>>>> in-house
>> > > > > >> > > > > >> > >>>>>>> deployment
>> > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a lot
>> easier
>> > > for
>> > > > > new
>> > > > > >> > > users.
>> > > > > >> > > > > >> > >>>>>>>>> Imagine
>> > > > > >> > > > > >> > >>>>>>> having
>> > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN.
>> The
>> > > drop
>> > > > > in
>> > > > > >> > > mailing
>> > > > > >> > > > > >> list
>> > > > > >> > > > > >> > >>>>>> traffic
>> > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue
>> to
>> > me.
>> > > > The
>> > > > > >> > > reality
>> > > > > >> > > > > is,
>> > > > > >> > > > > >> > >>>>> everyone
>> > > > > >> > > > > >> > >>>>>>>>> that
>> > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka.
>> We
>> > > > > basically
>> > > > > >> > > > require
>> > > > > >> > > > > >> it
>> > > > > >> > > > > >> > >>>>>> already
>> > > > > >> > > > > >> > >>>>>>> in
>> > > > > >> > > > > >> > >>>>>>>>> order for most features to work. Those
>> that
>> > > are
>> > > > > >> using
>> > > > > >> > > > other
>> > > > > >> > > > > >> > >>>>>>>>> systems
>> > > > > >> > > > > >> > >>>>>> are
>> > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into Kafka
>> > (1),
>> > > > and
>> > > > > >> then
>> > > > > >> > > > they
>> > > > > >> > > > > do
>> > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is already
>> > > > > discussion (
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> >
>> > > > > >> > > > >
>> > > > > >> >
>> > > > >
>> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> > > > > >> > > > > >> > >>>>> 767
>> > > > > >> > > > > >> > >>>>>>>>> )
>> > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka
>> > > extremely
>> > > > > >> easy.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple with
>> Kafka,
>> > > we
>> > > > > can
>> > > > > >> > > > leverage
>> > > > > >> > > > > a
>> > > > > >> > > > > >> > >>>>>>>>> ton of
>> > > > > >> > > > > >> > >>>>>>> their
>> > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to maintain
>> > our
>> > > > own
>> > > > > >> > config,
>> > > > > >> > > > > >> > >>>>>>>>> metrics,
>> > > > > >> > > > > >> > >>>>> etc.
>> > > > > >> > > > > >> > >>>>>>> We
>> > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries, and
>> make
>> > > them
>> > > > > >> > better.
>> > > > > >> > > > This
>> > > > > >> > > > > >> > >>>>>>>>> will
>> > > > > >> > > > > >> > >>>>> also
>> > > > > >> > > > > >> > >>>>>>>>> allow us to share the consumer/producer
>> > APIs,
>> > > > and
>> > > > > >> will
>> > > > > >> > > let
>> > > > > >> > > > > us
>> > > > > >> > > > > >> > >>>>> leverage
>> > > > > >> > > > > >> > >>>>>>>>> their offset management and partition
>> > > > management,
>> > > > > >> > rather
>> > > > > >> > > > > than
>> > > > > >> > > > > >> > >>>>>>>>> having
>> > > > > >> > > > > >> > >>>>>> our
>> > > > > >> > > > > >> > >>>>>>>>> own. All of the coordinator stream code
>> > would
>> > > > go
>> > > > > >> away,
>> > > > > >> > > as
>> > > > > >> > > > > >> would
>> > > > > >> > > > > >> > >>>>>>>>> most
>> > > > > >> > > > > >> > >>>>>> of
>> > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have
>> to
>> > > push
>> > > > > some
>> > > > > >> > > > > partition
>> > > > > >> > > > > >> > >>>>>>> management
>> > > > > >> > > > > >> > >>>>>>>>> features into the Kafka broker, but
>> they're
>> > > > > already
>> > > > > >> > > moving
>> > > > > >> > > > > in
>> > > > > >> > > > > >> > >>>>>>>>> that direction with the new consumer
>> API.
>> > The
>> > > > > >> features
>> > > > > >> > > we
>> > > > > >> > > > > have
>> > > > > >> > > > > >> > >>>>>>>>> for
>> > > > > >> > > > > >> > >>>>>> partition
>> > > > > >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and
>> seem
>> > > > like
>> > > > > >> they
>> > > > > >> > > > should
>> > > > > >> > > > > >> be
>> > > > > >> > > > > >> > >>>>>>>>> in
>> > > > > >> > > > > >> > >>>>>> Kafka
>> > > > > >> > > > > >> > >>>>>>>>> anyway. There will always be some niche
>> > > usages
>> > > > > which
>> > > > > >> > > will
>> > > > > >> > > > > >> > >>>>>>>>> require
>> > > > > >> > > > > >> > >>>>>> extra
>> > > > > >> > > > > >> > >>>>>>>>> care and hence full control over
>> partition
>> > > > > >> assignments
>> > > > > >> > > > much
>> > > > > >> > > > > >> > >>>>>>>>> like the
>> > > > > >> > > > > >> > >>>>>>> Kafka
>> > > > > >> > > > > >> > >>>>>>>>> low level consumer api. These would
>> > continue
>> > > to
>> > > > > be
>> > > > > >> > > > > supported.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> These items will be good for the Samza
>> > > > community.
>> > > > > >> > > They'll
>> > > > > >> > > > > make
>> > > > > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it easier
>> for
>> > > > > >> developers
>> > > > > >> > > to
>> > > > > >> > > > > add
>> > > > > >> > > > > >> > >>>>>>>>> new features.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and
>> > > somewhat
>> > > > > >> > backwards
>> > > > > >> > > > > >> > >>>>> incompatible
>> > > > > >> > > > > >> > >>>>>>>>> change). If we choose to go this route,
>> > it's
>> > > > > >> important
>> > > > > >> > > > that
>> > > > > >> > > > > we
>> > > > > >> > > > > >> > >>>>> openly
>> > > > > >> > > > > >> > >>>>>>>>> communicate how we're going to provide a
>> > > > > migration
>> > > > > >> > path
>> > > > > >> > > > from
>> > > > > >> > > > > >> > >>>>>>>>> the
>> > > > > >> > > > > >> > >>>>>>> existing
>> > > > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make
>> > incompatible
>> > > > > >> > changes).
>> > > > > >> > > I
>> > > > > >> > > > > >> think
>> > > > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to
>> > provide a
>> > > > > >> wrapper
>> > > > > >> > to
>> > > > > >> > > > > allow
>> > > > > >> > > > > >> > >>>>>>>>> existing StreamTask implementations to
>> > > continue
>> > > > > >> > running
>> > > > > >> > > on
>> > > > > >> > > > > the
>> > > > > >> > > > > >> > >>>> new container.
>> > > > > >> > > > > >> > >>>>>>> It's
>> > > > > >> > > > > >> > >>>>>>>>> also important that we openly
>> communicate
>> > > about
>> > > > > >> > timing,
>> > > > > >> > > > and
>> > > > > >> > > > > >> > >>>>>>>>> stages
>> > > > > >> > > > > >> > >>>>> of
>> > > > > >> > > > > >> > >>>>>>> the
>> > > > > >> > > > > >> > >>>>>>>>> migration.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you
>> have
>> > > > > opinions.
>> > > > > >> > :)
>> > > > > >> > > > > Please
>> > > > > >> > > > > >> > >>>>>>>>> send
>> > > > > >> > > > > >> > >>>>>> your
>> > > > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>> Cheers,
>> > > > > >> > > > > >> > >>>>>>>>> Chris
>> > > > > >> > > > > >> > >>>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>>
>> > > > > >> > > > > >> > >>>>>>>
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>> --
>> > > > > >> > > > > >> > >>>>>> -- Guozhang
>> > > > > >> > > > > >> > >>>>>>
>> > > > > >> > > > > >> > >>>>>
>> > > > > >> > > > > >> > >>>>
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> > >>
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >> >
>> > > > > >> > > > > >>
>> > > > > >> > > > > >
>> > > > > >> > > > > >
>> > > > > >> > > > >
>> > > > > >> > > >
>> > > > > >> > >
>> > > > > >> >
>> > > > > >>
>> > > > >
>> > > >
>> > >
>> >
>>
>
>

Re: Thoughts and obesrvations on Samza

Posted by Jakob Homan <jg...@gmail.com>.
I'm afraid I don't agree that we're anywhere near coming to a
consensus, or even that we're all agreeing on what we're discussing.
(I do totally agree that the discussion itself has been awesome both
in tone and content, though).

As Tim brought up and I mentioned, the Board is not big on subprojects
right now, for a lot of the reasons that flowed from Chris' points.
What role would the current Samza PMC have in Kafka? What role would
the Kafka PMC have over the Samza code?  Why would some members of the
Samza PMC be rolled into the Kafka, but not others?  These types of
questions are where the whole Community Over Code ethos comes from;
it's better to have a happy community than the absolute, subjective
best bit of code in the repo.  As a member of both communities, I can
say that he Kafka and Samza cultures and communities are significantly
different.  For example, Kafka has very, very strict procedures for
code contributions.  Samza does not.  One might be better than the
other, but again, it's down to community and asking the Samza
community to integrate the Kafka approach is a bigger issue than
asking for another project to add some code to its repo.

The Board will care about the communities, not the code and most of
this discussion has been nearly entirely focused on the code.

Additionally, except for Jay (and myself, but I'm pretty Kafka
inactive), there has been no input from the Kafka community.  Even if
we did have full agreement on the Samza side for Option C ("Hey,
Samza! FYI, Kafka does streaming now!"), the Kafka community has no
need to agree or participate.

Personally, my preference would be for a Samza 2.0 approach.  There
are a lot of lessons learned in the project so far and, with a
willingness to break APIs, we could improve dramatically in terms of
ease of use, supported execution environments and support for other
types of input and output methods.  It may be that the community
splits in this regard, with some contributing to a new streaming
library in Kafka and others contributing to a continuation of the
current Samza approach.  From an ASF approach, this would be a
perfectly acceptable outcome because, again, the communities would be
quiet harmonious.

-Jakob

On 12 July 2015 at 17:54, Chris Riccomini <cr...@apache.org> wrote:
> Given that Jay, Martin, and I seem to be aligning fairly closely, I think
> we should start with:
>
> 1. [community] Make Samza a subproject of Kafka.
> 2. [community] Make all Samza PMC/committers committers of the subproject.
> 3. [community] Migrate Samza's website/documentation into Kafka's.
> 4. [code] Have the Samza community and the Kafka community start a
> from-scratch reboot together in the new Kafka subproject. We can
> borrow/copy &  paste significant chunks of code from Samza's code base.
> 5. [code] The subproject would intentionally eliminate support for both
> other streaming systems and all deployment systems.
> 6. [code] Attempt to provide a bridge from our SystemConsumer to KIP-26
> (copy cat)
> 7. [code] Attempt to provide a bridge from the new subproject's processor
> interface to our legacy StreamTask interface.
> 8. [code/community] Sunset Samza as a TLP when we have a working Kafka
> subproject that has a fault-tolerant container with state management.
>
> It's likely that (6) and (7) won't be fully drop-in. Still, the closer we
> can get, the better it's going to be for our existing community.
>
> One thing that I didn't touch on with (2) is whether any Samza PMC members
> should be rolled into Kafka PMC membership as well (though, Jay and Jakob
> are already PMC members on both). I think that Samza's community deserves a
> voice on the PMC, so I'd propose that we roll at least a few PMC members
> into the Kafka PMC, but I don't have a strong framework for which people to
> pick.
>
> Before (8), I think that Samza's TLP can continue to commit bug fixes and
> patches as it sees fit, provided that we openly communicate that we won't
> necessarily migrate new features to the new subproject, and that the TLP
> will be shut down after the migration to the Kafka subproject occurs.
>
> Jakob, I could use your guidance here about about how to achieve this from
> an Apache process perspective (sorry).
>
> * Should I just call a vote on this proposal?
> * Should it happen on dev or private?
> * Do committers have binding votes, or just PMC?
>
> Having trouble finding much detail on the Apache wikis. :(

Re: Thoughts and obesrvations on Samza

Posted by Chris Riccomini <cr...@apache.org>.
Hey all,

I want to start by saying that I'm absolutely thrilled to be a part of this
community. The amount of level-headed, thoughtful, educated discussion
that's gone on over the past ~10 days is overwhelming. Wonderful.

It seems like discussion is waning a bit, and we've reached some
conclusions. There are several key emails in this threat, which I want to
call out:

1. Jakob's summary of the three potential ways forward.

http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVu-hxdBfyQ4qm3LDC55cUQbPdmbe4zGzTOOatYF1Pz43A%40mail.gmail.com%3E
2. Julian's call out that we should be focusing on community over code.

http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCAPSgeESZ_7bVFbwN%2Bzqi5MH%3D4CWu9MZUSanKg0-1woMqt55Fvg%40mail.gmail.com%3E
3. Martin's summary about the benefits of merging communities.

http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CBFB866B6-D9D8-4578-93C0-FFAEB1DF00FC%40kleppmann.com%3E
4. Jakob's comments about the distinction between community and code paths.

http://mail-archives.apache.org/mod_mbox/samza-dev/201507.mbox/%3CCADiKvVtWPjHLLDsmxvz9KggVA5DfBi-nUvfqB6QdA-du%2B_a9Ng%40mail.gmail.com%3E

I agree with the comments on all of these emails. I think Martin's summary
of his position aligns very closely with my own. To that end, I think we
should get concrete about what the proposal is, and call a vote on it.
Given that Jay, Martin, and I seem to be aligning fairly closely, I think
we should start with:

1. [community] Make Samza a subproject of Kafka.
2. [community] Make all Samza PMC/committers committers of the subproject.
3. [community] Migrate Samza's website/documentation into Kafka's.
4. [code] Have the Samza community and the Kafka community start a
from-scratch reboot together in the new Kafka subproject. We can
borrow/copy &  paste significant chunks of code from Samza's code base.
5. [code] The subproject would intentionally eliminate support for both
other streaming systems and all deployment systems.
6. [code] Attempt to provide a bridge from our SystemConsumer to KIP-26
(copy cat)
7. [code] Attempt to provide a bridge from the new subproject's processor
interface to our legacy StreamTask interface.
8. [code/community] Sunset Samza as a TLP when we have a working Kafka
subproject that has a fault-tolerant container with state management.

It's likely that (6) and (7) won't be fully drop-in. Still, the closer we
can get, the better it's going to be for our existing community.

One thing that I didn't touch on with (2) is whether any Samza PMC members
should be rolled into Kafka PMC membership as well (though, Jay and Jakob
are already PMC members on both). I think that Samza's community deserves a
voice on the PMC, so I'd propose that we roll at least a few PMC members
into the Kafka PMC, but I don't have a strong framework for which people to
pick.

Before (8), I think that Samza's TLP can continue to commit bug fixes and
patches as it sees fit, provided that we openly communicate that we won't
necessarily migrate new features to the new subproject, and that the TLP
will be shut down after the migration to the Kafka subproject occurs.

Jakob, I could use your guidance here about about how to achieve this from
an Apache process perspective (sorry).

* Should I just call a vote on this proposal?
* Should it happen on dev or private?
* Do committers have binding votes, or just PMC?

Having trouble finding much detail on the Apache wikis. :(

Cheers,
Chris

On Fri, Jul 10, 2015 at 2:38 PM, Yan Fang <ya...@gmail.com> wrote:

> Thanks, Jay. This argument persuaded me actually. :)
>
> Fang, Yan
> yanfang724@gmail.com
>
> On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey Yan,
> >
> > Yeah philosophically I think the argument is that you should capture the
> > stream in Kafka independent of the transformation. This is obviously a
> > Kafka-centric view point.
> >
> > Advantages of this:
> > - In practice I think this is what e.g. Storm people often end up doing
> > anyway. You usually need to throttle any access to a live serving
> database.
> > - Can have multiple subscribers and they get the same thing without
> > additional load on the source system.
> > - Applications can tap into the stream if need be by subscribing.
> > - You can debug your transformation by tailing the Kafka topic with the
> > console consumer
> > - Can tee off the same data stream for batch analysis or Lambda arch
> style
> > re-processing
> >
> > The disadvantage is that it will use Kafka resources. But the idea is
> > eventually you will have multiple subscribers to any data source (at
> least
> > for monitoring) so you will end up there soon enough anyway.
> >
> > Down the road the technical benefit is that I think it gives us a good
> path
> > towards end-to-end exactly once semantics from source to destination.
> > Basically the connectors need to support idempotence when talking to
> Kafka
> > and we need the transactional write feature in Kafka to make the
> > transformation atomic. This is actually pretty doable if you separate
> > connector=>kafka problem from the generic transformations which are
> always
> > kafka=>kafka. However I think it is quite impossible to do in a
> all_things
> > => all_things environment. Today you can say "well the semantics of the
> > Samza APIs depend on the connectors you use" but it is actually worse
> then
> > that because the semantics actually depend on the pairing of
> connectors--so
> > not only can you probably not get a usable "exactly once" guarantee
> > end-to-end it can actually be quite hard to reverse engineer what
> property
> > (if any) your end-to-end flow has if you have heterogenous systems.
> >
> > -Jay
> >
> > On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com> wrote:
> >
> > > {quote}
> > > maintained in a separate repository and retaining the existing
> > > committership but sharing as much else as possible (website, etc)
> > > {quote}
> > >
> > > Overall, I agree on this idea. Now the question is more about "how to
> do
> > > it".
> > >
> > > On the other hand, one thing I want to point out is that, if we decide
> to
> > > go this way, how do we want to support
> > > otherSystem-transformation-otherSystem use case?
> > >
> > > Basically, there are four user groups here:
> > >
> > > 1. Kafka-transformation-Kafka
> > > 2. Kafka-transformation-otherSystem
> > > 3. otherSystem-transformation-Kafka
> > > 4. otherSystem-transformation-otherSystem
> > >
> > > For group 1, they can easily use the new Samza library to achieve. For
> > > group 2 and 3, they can use copyCat -> transformation -> Kafka or
> Kafka->
> > > transformation -> copyCat.
> > >
> > > The problem is for group 4. Do we want to abandon this or still support
> > it?
> > > Of course, this use case can be achieved by using copyCat ->
> > transformation
> > > -> Kafka -> transformation -> copyCat, the thing is how we persuade
> them
> > to
> > > do this long chain. If yes, it will also be a win for Kafka too. Or if
> > > there is no one in this community actually doing this so far, maybe ok
> to
> > > not support the group 4 directly.
> > >
> > > Thanks,
> > >
> > > Fang, Yan
> > > yanfang724@gmail.com
> > >
> > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Yeah I agree with this summary. I think there are kind of two
> questions
> > > > here:
> > > > 1. Technically does alignment/reliance on Kafka make sense
> > > > 2. Branding wise (naming, website, concepts, etc) does alignment with
> > > Kafka
> > > > make sense
> > > >
> > > > Personally I do think both of these things would be really valuable,
> > and
> > > > would dramatically alter the trajectory of the project.
> > > >
> > > > My preference would be to see if people can mostly agree on a
> direction
> > > > rather than splintering things off. From my point of view the ideal
> > > outcome
> > > > of all the options discussed would be to make Samza a closely aligned
> > > > subproject, maintained in a separate repository and retaining the
> > > existing
> > > > committership but sharing as much else as possible (website, etc). No
> > > idea
> > > > about how these things work, Jacob, you probably know more.
> > > >
> > > > No discussion amongst the Kafka folks has happened on this, but
> likely
> > we
> > > > should figure out what the Samza community actually wants first.
> > > >
> > > > I admit that this is a fairly radical departure from how things are.
> > > >
> > > > If that doesn't fly, I think, yeah we could leave Samza as it is and
> do
> > > the
> > > > more radical reboot inside Kafka. From my point of view that does
> leave
> > > > things in a somewhat confusing state since now there are two stream
> > > > processing systems more or less coupled to Kafka in large part made
> by
> > > the
> > > > same people. But, arguably that might be a cleaner way to make the
> > > cut-over
> > > > and perhaps less risky for Samza community since if it works people
> can
> > > > switch and if it doesn't nothing will have changed. Dunno, how do
> > people
> > > > feel about this?
> > > >
> > > > -Jay
> > > >
> > > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com>
> > wrote:
> > > >
> > > > > >  This leads me to thinking that merging projects and communities
> > > might
> > > > > be a good idea: with the union of experience from both communities,
> > we
> > > > will
> > > > > probably build a better system that is better for users.
> > > > > Is this what's being proposed though? Merging the projects seems
> like
> > > > > a consequence of at most one of the three directions under
> > discussion:
> > > > > 1) Samza 2.0: The Samza community relies more heavily on Kafka for
> > > > > configuration, etc. (to a greater or lesser extent to be
> determined)
> > > > > but the Samza community would not automatically merge withe Kafka
> > > > > community (the Phoenix/HBase example is a good one here).
> > > > > 2) Samza Reboot: The Samza community continues to exist with a
> > limited
> > > > > project scope, but similarly would not need to be part of the Kafka
> > > > > community (ie given committership) to progress.  Here, maybe the
> > Samza
> > > > > team would become a subproject of Kafka (the Board frowns on
> > > > > subprojects at the moment, so I'm not sure if that's even
> feasible),
> > > > > but that would not be required.
> > > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option the
> Kafka
> > > > > team builds its own streaming library, possibly off of Jay's
> > > > > prototype, which has not direct lineage to the Samza team.  There's
> > no
> > > > > reason for the Kafka team to bring in the Samza team.
> > > > >
> > > > > Is the Kafka community on board with this?
> > > > >
> > > > > To be clear, all three options under discussion are interesting,
> > > > > technically valid and likely healthy directions for the project.
> > > > > Also, they are not mutually exclusive.  The Samza community could
> > > > > decide to pursue, say, 'Samza 2.0', while the Kafka community went
> > > > > forward with 'Hey Samza!'  My points above are directed entirely at
> > > > > the community aspect of these choices.
> > > > > -Jakob
> > > > >
> > > > > On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com>
> > wrote:
> > > > > > That's great.  Thanks, Jay.
> > > > > >
> > > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io>
> > wrote:
> > > > > >
> > > > > >> Yeah totally agree. I think you have this issue even today,
> right?
> > > > I.e.
> > > > > if
> > > > > >> you need to make a simple config change and you're running in
> YARN
> > > > today
> > > > > >> you end up bouncing the job which then rebuilds state. I think
> the
> > > fix
> > > > > is
> > > > > >> exactly what you described which is to have a long timeout on
> > > > partition
> > > > > >> movement for stateful jobs so that if a job is just getting
> > bounced,
> > > > and
> > > > > >> the cluster manager (or admin) is smart enough to restart it on
> > the
> > > > same
> > > > > >> host when possible, it can optimistically reuse any existing
> state
> > > it
> > > > > finds
> > > > > >> on disk (if it is valid).
> > > > > >>
> > > > > >> So in this model the charter of the CM is to place processes as
> > > > > stickily as
> > > > > >> possible and to restart or re-place failed processes. The
> charter
> > of
> > > > the
> > > > > >> partition management system is to control the assignment of work
> > to
> > > > > these
> > > > > >> processes. The nice thing about this is that the work
> assignment,
> > > > > timeouts,
> > > > > >> behavior, configs, and code will all be the same across all
> > cluster
> > > > > >> managers.
> > > > > >>
> > > > > >> So I think that prototype would actually give you exactly what
> you
> > > > want
> > > > > >> today for any cluster manager (or manual placement + restart
> > script)
> > > > > that
> > > > > >> was sticky in terms of host placement since there is already a
> > > > > configurable
> > > > > >> partition movement timeout and task-by-task state reuse with a
> > check
> > > > on
> > > > > >> state validity.
> > > > > >>
> > > > > >> -Jay
> > > > > >>
> > > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > > roger.hoover@gmail.com
> > > > >
> > > > > >> wrote:
> > > > > >>
> > > > > >> > That would be great to let Kafka do as much heavy lifting as
> > > > possible
> > > > > and
> > > > > >> > make it easier for other languages to implement Samza apis.
> > > > > >> >
> > > > > >> > One thing to watch out for is the interplay between Kafka's
> > group
> > > > > >> > management and the external scheduler/process manager's fault
> > > > > tolerance.
> > > > > >> > If a container dies, the Kafka group membership protocol will
> > try
> > > to
> > > > > >> assign
> > > > > >> > it's tasks to other containers while at the same time the
> > process
> > > > > manager
> > > > > >> > is trying to relaunch the container.  Without some
> consideration
> > > for
> > > > > this
> > > > > >> > (like a configurable amount of time to wait before Kafka
> alters
> > > the
> > > > > group
> > > > > >> > membership), there may be thrashing going on which is
> especially
> > > bad
> > > > > for
> > > > > >> > containers with large amounts of local state.
> > > > > >> >
> > > > > >> > Someone else pointed this out already but I thought it might
> be
> > > > worth
> > > > > >> > calling out again.
> > > > > >> >
> > > > > >> > Cheers,
> > > > > >> >
> > > > > >> > Roger
> > > > > >> >
> > > > > >> >
> > > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <ja...@confluent.io>
> > > > wrote:
> > > > > >> >
> > > > > >> > > Hey Roger,
> > > > > >> > >
> > > > > >> > > I couldn't agree more. We spent a bunch of time talking to
> > > people
> > > > > and
> > > > > >> > that
> > > > > >> > > is exactly the stuff we heard time and again. What makes it
> > > hard,
> > > > of
> > > > > >> > > course, is that there is some tension between compatibility
> > with
> > > > > what's
> > > > > >> > > there now and making things better for new users.
> > > > > >> > >
> > > > > >> > > I also strongly agree with the importance of multi-language
> > > > > support. We
> > > > > >> > are
> > > > > >> > > talking now about Java, but for application development use
> > > cases
> > > > > >> people
> > > > > >> > > want to work in whatever language they are using elsewhere.
> I
> > > > think
> > > > > >> > moving
> > > > > >> > > to a model where Kafka itself does the group membership,
> > > lifecycle
> > > > > >> > control,
> > > > > >> > > and partition assignment has the advantage of putting all
> that
> > > > > complex
> > > > > >> > > stuff behind a clean api that the clients are already going
> to
> > > be
> > > > > >> > > implementing for their consumer, so the added functionality
> > for
> > > > > stream
> > > > > >> > > processing beyond a consumer becomes very minor.
> > > > > >> > >
> > > > > >> > > -Jay
> > > > > >> > >
> > > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > > > > roger.hoover@gmail.com>
> > > > > >> > > wrote:
> > > > > >> > >
> > > > > >> > > > Metamorphosis...nice. :)
> > > > > >> > > >
> > > > > >> > > > This has been a great discussion.  As a user of Samza
> who's
> > > > > recently
> > > > > >> > > > integrated it into a relatively large organization, I just
> > > want
> > > > to
> > > > > >> add
> > > > > >> > > > support to a few points already made.
> > > > > >> > > >
> > > > > >> > > > The biggest hurdles to adoption of Samza as it currently
> > > exists
> > > > > that
> > > > > >> > I've
> > > > > >> > > > experienced are:
> > > > > >> > > > 1) YARN - YARN is overly complex in many environments
> where
> > > > Puppet
> > > > > >> > would
> > > > > >> > > do
> > > > > >> > > > just fine but it was the only mechanism to get fault
> > > tolerance.
> > > > > >> > > > 2) Configuration - I think I like the idea of configuring
> > most
> > > > of
> > > > > the
> > > > > >> > job
> > > > > >> > > > in code rather than config files.  In general, I think the
> > > goal
> > > > > >> should
> > > > > >> > be
> > > > > >> > > > to make it harder to make mistakes, especially of the kind
> > > where
> > > > > the
> > > > > >> > code
> > > > > >> > > > expects something and the config doesn't match.  The
> current
> > > > > config
> > > > > >> is
> > > > > >> > > > quite intricate and error-prone.  For example, the
> > application
> > > > > logic
> > > > > >> > may
> > > > > >> > > > depend on bootstrapping a topic but rather than asserting
> > that
> > > > in
> > > > > the
> > > > > >> > > code,
> > > > > >> > > > you have to rely on getting the config right.  Likewise
> with
> > > > > serdes,
> > > > > >> > the
> > > > > >> > > > Java representations produced by various serdes (JSON,
> Avro,
> > > > etc.)
> > > > > >> are
> > > > > >> > > not
> > > > > >> > > > equivalent so you cannot just reconfigure a serde without
> > > > changing
> > > > > >> the
> > > > > >> > > > code.   It would be nice for jobs to be able to assert
> what
> > > they
> > > > > >> expect
> > > > > >> > > > from their input topics in terms of partitioning.  This is
> > > > > getting a
> > > > > >> > > little
> > > > > >> > > > off topic but I was even thinking about creating a "Samza
> > > config
> > > > > >> > linter"
> > > > > >> > > > that would sanity check a set of configs.  Especially in
> > > > > >> organizations
> > > > > >> > > > where config is managed by a different team than the
> > > application
> > > > > >> > > developer,
> > > > > >> > > > it's very hard to get avoid config mistakes.
> > > > > >> > > > 3) Java/Scala centric - for many teams (especially
> > DevOps-type
> > > > > >> folks),
> > > > > >> > > the
> > > > > >> > > > pain of the Java toolchain (maven, slow builds, weak
> command
> > > > line
> > > > > >> > > support,
> > > > > >> > > > configuration over convention) really inhibits
> productivity.
> > > As
> > > > > more
> > > > > >> > and
> > > > > >> > > > more high-quality clients become available for Kafka, I
> hope
> > > > > they'll
> > > > > >> > > follow
> > > > > >> > > > Samza's model.  Not sure how much it affects the proposals
> > in
> > > > this
> > > > > >> > thread
> > > > > >> > > > but please consider other languages in the ecosystem as
> > well.
> > > > > From
> > > > > >> > what
> > > > > >> > > > I've heard, Spark has more Python users than Java/Scala.
> > > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > > > >> > > > and are working on a Yeoman generator
> > > > > >> > > > https://github.com/Quantiply/generator-rico for
> > Jython/Samza
> > > > > >> projects
> > > > > >> > to
> > > > > >> > > > alleviate some of the pain)
> > > > > >> > > >
> > > > > >> > > > I also want to underscore Jay's point about improving the
> > user
> > > > > >> > > experience.
> > > > > >> > > > That's a very important factor for adoption.  I think the
> > goal
> > > > > should
> > > > > >> > be
> > > > > >> > > to
> > > > > >> > > > make Samza as easy to get started with as something like
> > > > Logstash.
> > > > > >> > > > Logstash is vastly inferior in terms of capabilities to
> > Samza
> > > > but
> > > > > >> it's
> > > > > >> > > easy
> > > > > >> > > > to get started and that makes a big difference.
> > > > > >> > > >
> > > > > >> > > > Cheers,
> > > > > >> > > >
> > > > > >> > > > Roger
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci
> > > Morales <
> > > > > >> > > > gdfm@apache.org> wrote:
> > > > > >> > > >
> > > > > >> > > > > Forgot to add. On the naming issues, Kafka Metamorphosis
> > is
> > > a
> > > > > clear
> > > > > >> > > > winner
> > > > > >> > > > > :)
> > > > > >> > > > >
> > > > > >> > > > > --
> > > > > >> > > > > Gianmarco
> > > > > >> > > > >
> > > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales
> <
> > > > > >> > > gdfm@apache.org
> > > > > >> > > > >
> > > > > >> > > > > wrote:
> > > > > >> > > > >
> > > > > >> > > > > > Hi,
> > > > > >> > > > > >
> > > > > >> > > > > > @Martin, thanks for you comments.
> > > > > >> > > > > > Maybe I'm missing some important point, but I think
> > > coupling
> > > > > the
> > > > > >> > > > releases
> > > > > >> > > > > > is actually a *good* thing.
> > > > > >> > > > > > To make an example, would it be better if the MR and
> > HDFS
> > > > > >> > components
> > > > > >> > > of
> > > > > >> > > > > > Hadoop had different release schedules?
> > > > > >> > > > > >
> > > > > >> > > > > > Actually, keeping the discussion in a single place
> would
> > > > make
> > > > > >> > > agreeing
> > > > > >> > > > on
> > > > > >> > > > > > releases (and backwards compatibility) much easier, as
> > > > > everybody
> > > > > >> > > would
> > > > > >> > > > be
> > > > > >> > > > > > responsible for the whole codebase.
> > > > > >> > > > > >
> > > > > >> > > > > > That said, I like the idea of absorbing samza-core as
> a
> > > > > >> > sub-project,
> > > > > >> > > > and
> > > > > >> > > > > > leave the fancy stuff separate.
> > > > > >> > > > > > It probably gives 90% of the benefits we have been
> > > > discussing
> > > > > >> here.
> > > > > >> > > > > >
> > > > > >> > > > > > Cheers,
> > > > > >> > > > > >
> > > > > >> > > > > > --
> > > > > >> > > > > > Gianmarco
> > > > > >> > > > > >
> > > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <
> jay.kreps@gmail.com
> > >
> > > > > wrote:
> > > > > >> > > > > >
> > > > > >> > > > > >> Hey Martin,
> > > > > >> > > > > >>
> > > > > >> > > > > >> I agree coupling release schedules is a downside.
> > > > > >> > > > > >>
> > > > > >> > > > > >> Definitely we can try to solve some of the
> integration
> > > > > problems
> > > > > >> in
> > > > > >> > > > > >> Confluent Platform or in other distributions. But I
> > think
> > > > > this
> > > > > >> > ends
> > > > > >> > > up
> > > > > >> > > > > >> being really shallow. I guess I feel to really get a
> > good
> > > > > user
> > > > > >> > > > > experience
> > > > > >> > > > > >> the two systems have to kind of feel like part of the
> > > same
> > > > > thing
> > > > > >> > and
> > > > > >> > > > you
> > > > > >> > > > > >> can't really add that in later--you can put both in
> the
> > > > same
> > > > > >> > > > > downloadable
> > > > > >> > > > > >> tar file but it doesn't really give a very cohesive
> > > > feeling.
> > > > > I
> > > > > >> > agree
> > > > > >> > > > > that
> > > > > >> > > > > >> ultimately any of the project stuff is as much social
> > and
> > > > > naming
> > > > > >> > as
> > > > > >> > > > > >> anything else--theoretically two totally independent
> > > > projects
> > > > > >> > could
> > > > > >> > > > work
> > > > > >> > > > > >> to
> > > > > >> > > > > >> tightly align. In practice this seems to be quite
> > > difficult
> > > > > >> > though.
> > > > > >> > > > > >>
> > > > > >> > > > > >> For the frameworks--totally agree it would be good to
> > > > > maintain
> > > > > >> the
> > > > > >> > > > > >> framework support with the project. In some cases
> there
> > > may
> > > > > not
> > > > > >> be
> > > > > >> > > too
> > > > > >> > > > > >> much
> > > > > >> > > > > >> there since the integration gets lighter but I think
> > > > whatever
> > > > > >> > stubs
> > > > > >> > > > you
> > > > > >> > > > > >> need should be included. So no I definitely wasn't
> > trying
> > > > to
> > > > > >> imply
> > > > > >> > > > > >> dropping
> > > > > >> > > > > >> support for these frameworks, just making the
> > integration
> > > > > >> lighter
> > > > > >> > by
> > > > > >> > > > > >> separating process management from partition
> > management.
> > > > > >> > > > > >>
> > > > > >> > > > > >> You raise two good points we would have to figure out
> > if
> > > we
> > > > > went
> > > > > >> > > down
> > > > > >> > > > > the
> > > > > >> > > > > >> alignment path:
> > > > > >> > > > > >> 1. With respect to the name, yeah I think the first
> > > > question
> > > > > is
> > > > > >> > > > whether
> > > > > >> > > > > >> some "re-branding" would be worth it. If so then I
> > think
> > > we
> > > > > can
> > > > > >> > > have a
> > > > > >> > > > > big
> > > > > >> > > > > >> thread on the name. I'm definitely not set on Kafka
> > > > > Streaming or
> > > > > >> > > Kafka
> > > > > >> > > > > >> Streams I was just using them to be kind of
> > > illustrative. I
> > > > > >> agree
> > > > > >> > > with
> > > > > >> > > > > >> your
> > > > > >> > > > > >> critique of these names, though I think people would
> > get
> > > > the
> > > > > >> idea.
> > > > > >> > > > > >> 2. Yeah you also raise a good point about how to
> > "factor"
> > > > it.
> > > > > >> Here
> > > > > >> > > are
> > > > > >> > > > > the
> > > > > >> > > > > >> options I see (I could get enthusiastic about any of
> > > them):
> > > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > > > > >> > > > > >>    b. Two repos, retaining the current seperation
> > > > > >> > > > > >>    c. Two repos, the equivalent of samza-api and
> > > samza-core
> > > > > is
> > > > > >> > > > absorbed
> > > > > >> > > > > >> almost like a third client
> > > > > >> > > > > >>
> > > > > >> > > > > >> Cheers,
> > > > > >> > > > > >>
> > > > > >> > > > > >> -Jay
> > > > > >> > > > > >>
> > > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> > > > > >> > > > martin@kleppmann.com>
> > > > > >> > > > > >> wrote:
> > > > > >> > > > > >>
> > > > > >> > > > > >> > Ok, thanks for the clarifications. Just a few
> > follow-up
> > > > > >> > comments.
> > > > > >> > > > > >> >
> > > > > >> > > > > >> > - I see the appeal of merging with Kafka or
> becoming
> > a
> > > > > >> > subproject:
> > > > > >> > > > the
> > > > > >> > > > > >> > reasons you mention are good. The risk I see is
> that
> > > > > release
> > > > > >> > > > schedules
> > > > > >> > > > > >> > become coupled to each other, which can slow
> everyone
> > > > down,
> > > > > >> and
> > > > > >> > > > large
> > > > > >> > > > > >> > projects with many contributors are harder to
> manage.
> > > > > (Jakob,
> > > > > >> > can
> > > > > >> > > > you
> > > > > >> > > > > >> speak
> > > > > >> > > > > >> > from experience, having seen a wider range of
> Hadoop
> > > > > ecosystem
> > > > > >> > > > > >> projects?)
> > > > > >> > > > > >> >
> > > > > >> > > > > >> > Some of the goals of a better unified developer
> > > > experience
> > > > > >> could
> > > > > >> > > > also
> > > > > >> > > > > be
> > > > > >> > > > > >> > solved by integrating Samza nicely into a Kafka
> > > > > distribution
> > > > > >> > (such
> > > > > >> > > > as
> > > > > >> > > > > >> > Confluent's). I'm not against merging projects if
> we
> > > > decide
> > > > > >> > that's
> > > > > >> > > > the
> > > > > >> > > > > >> way
> > > > > >> > > > > >> > to go, just pointing out the same goals can perhaps
> > > also
> > > > be
> > > > > >> > > achieved
> > > > > >> > > > > in
> > > > > >> > > > > >> > other ways.
> > > > > >> > > > > >> >
> > > > > >> > > > > >> > - With regard to dropping the YARN dependency: are
> > you
> > > > > >> proposing
> > > > > >> > > > that
> > > > > >> > > > > >> > Samza doesn't give any help to people wanting to
> run
> > on
> > > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > > > > >> > > > > >> > So the docs would basically have a link to Slider
> and
> > > > > nothing
> > > > > >> > > else?
> > > > > >> > > > Or
> > > > > >> > > > > >> > would we maintain integrations with a bunch of
> > popular
> > > > > >> > deployment
> > > > > >> > > > > >> methods
> > > > > >> > > > > >> > (e.g. the necessary glue and shell scripts to make
> > > Samza
> > > > > work
> > > > > >> > with
> > > > > >> > > > > >> Slider)?
> > > > > >> > > > > >> >
> > > > > >> > > > > >> > I absolutely think it's a good idea to have the
> "as a
> > > > > library"
> > > > > >> > and
> > > > > >> > > > > "as a
> > > > > >> > > > > >> > process" (using Yi's taxonomy) options for people
> who
> > > > want
> > > > > >> them,
> > > > > >> > > > but I
> > > > > >> > > > > >> > think there should also be a low-friction path for
> > > common
> > > > > "as
> > > > > >> a
> > > > > >> > > > > service"
> > > > > >> > > > > >> > deployment methods, for which we probably need to
> > > > maintain
> > > > > >> > > > > integrations.
> > > > > >> > > > > >> >
> > > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to me,
> > > > because
> > > > > >> Kafka
> > > > > >> > > is
> > > > > >> > > > > all
> > > > > >> > > > > >> > about streams already. Perhaps "Kafka Transformers"
> > or
> > > > > "Kafka
> > > > > >> > > > Filters"
> > > > > >> > > > > >> > would be more apt?
> > > > > >> > > > > >> >
> > > > > >> > > > > >> > One suggestion: perhaps the core of Samza (stream
> > > > > >> transformation
> > > > > >> > > > with
> > > > > >> > > > > >> > state management -- i.e. the "Samza as a library"
> > bit)
> > > > > could
> > > > > >> > > become
> > > > > >> > > > > >> part of
> > > > > >> > > > > >> > Kafka, while higher-level tools such as streaming
> SQL
> > > and
> > > > > >> > > > integrations
> > > > > >> > > > > >> with
> > > > > >> > > > > >> > deployment frameworks remain in a separate project?
> > In
> > > > > other
> > > > > >> > > words,
> > > > > >> > > > > >> Kafka
> > > > > >> > > > > >> > would absorb the proven, stable core of Samza,
> which
> > > > would
> > > > > >> > become
> > > > > >> > > > the
> > > > > >> > > > > >> > "third Kafka client" mentioned early in this
> thread.
> > > The
> > > > > Samza
> > > > > >> > > > project
> > > > > >> > > > > >> > would then target that third Kafka client as its
> base
> > > > API,
> > > > > and
> > > > > >> > the
> > > > > >> > > > > >> project
> > > > > >> > > > > >> > would be freed up to explore more experimental new
> > > > > horizons.
> > > > > >> > > > > >> >
> > > > > >> > > > > >> > Martin
> > > > > >> > > > > >> >
> > > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > > jay.kreps@gmail.com>
> > > > > >> wrote:
> > > > > >> > > > > >> >
> > > > > >> > > > > >> > > Hey Martin,
> > > > > >> > > > > >> > >
> > > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually
> don't
> > > > think
> > > > > it
> > > > > >> > ties
> > > > > >> > > > our
> > > > > >> > > > > >> > hands
> > > > > >> > > > > >> > > at all, all it does is refactor things. The
> > division
> > > of
> > > > > >> > > > > >> responsibility is
> > > > > >> > > > > >> > > that Samza core is responsible for task
> lifecycle,
> > > > state,
> > > > > >> and
> > > > > >> > > > > >> partition
> > > > > >> > > > > >> > > management (using the Kafka co-ordinator) but it
> is
> > > NOT
> > > > > >> > > > responsible
> > > > > >> > > > > >> for
> > > > > >> > > > > >> > > packaging, configuration deployment or execution
> of
> > > > > >> processes.
> > > > > >> > > The
> > > > > >> > > > > >> > problem
> > > > > >> > > > > >> > > of packaging and starting these processes is
> > > > > >> > > > > >> > > framework/environment-specific. This leaves
> > > individual
> > > > > >> > > frameworks
> > > > > >> > > > to
> > > > > >> > > > > >> be
> > > > > >> > > > > >> > as
> > > > > >> > > > > >> > > fancy or vanilla as they like. So you can get
> > simple
> > > > > >> stateless
> > > > > >> > > > > >> support in
> > > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app
> > > > framework
> > > > > >> > > (Slider,
> > > > > >> > > > > >> > Marathon,
> > > > > >> > > > > >> > > etc). These are well known by people and have
> nice
> > > UIs
> > > > > and a
> > > > > >> > lot
> > > > > >> > > > of
> > > > > >> > > > > >> > > flexibility. I don't think they have node
> affinity
> > > as a
> > > > > >> built
> > > > > >> > in
> > > > > >> > > > > >> option
> > > > > >> > > > > >> > > (though I could be wrong). So if we want that we
> > can
> > > > > either
> > > > > >> > wait
> > > > > >> > > > for
> > > > > >> > > > > >> them
> > > > > >> > > > > >> > > to add it or do a custom framework to add that
> > > feature
> > > > > (as
> > > > > >> > now).
> > > > > >> > > > > >> > Obviously
> > > > > >> > > > > >> > > if you manage things with old-school ops tools
> > > > > >> > (puppet/chef/etc)
> > > > > >> > > > you
> > > > > >> > > > > >> get
> > > > > >> > > > > >> > > locality easily. The nice thing, though, is that
> > all
> > > > the
> > > > > >> samza
> > > > > >> > > > > >> "business
> > > > > >> > > > > >> > > logic" around partition management and fault
> > > tolerance
> > > > > is in
> > > > > >> > > Samza
> > > > > >> > > > > >> core
> > > > > >> > > > > >> > so
> > > > > >> > > > > >> > > it is shared across frameworks and the framework
> > > > specific
> > > > > >> bit
> > > > > >> > is
> > > > > >> > > > > just
> > > > > >> > > > > >> > > whether it is smart enough to try to get the same
> > > host
> > > > > when
> > > > > >> a
> > > > > >> > > job
> > > > > >> > > > is
> > > > > >> > > > > >> > > restarted.
> > > > > >> > > > > >> > >
> > > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah I think
> > the
> > > > > goal
> > > > > >> > would
> > > > > >> > > > be
> > > > > >> > > > > >> (a)
> > > > > >> > > > > >> > > actually get better alignment in user experience,
> > and
> > > > (b)
> > > > > >> > > express
> > > > > >> > > > > >> this in
> > > > > >> > > > > >> > > the naming and project branding. Specifically:
> > > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> > > > > "transformation"
> > > > > >> api
> > > > > >> > > to
> > > > > >> > > > be
> > > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be able
> > to
> > > > > explain
> > > > > >> > > when
> > > > > >> > > > to
> > > > > >> > > > > >> use
> > > > > >> > > > > >> > > the consumer and when to use the stream
> processing
> > > > > >> > functionality
> > > > > >> > > > and
> > > > > >> > > > > >> lead
> > > > > >> > > > > >> > > people into that experience.
> > > > > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or
> > > > > whatever)
> > > > > >> > that
> > > > > >> > > > has
> > > > > >> > > > > >> both
> > > > > >> > > > > >> > > Kafka and the stream processing part and they
> > > actually
> > > > > work
> > > > > >> > > > > together.
> > > > > >> > > > > >> > > 3. Unify the programming experience so the client
> > and
> > > > > Samza
> > > > > >> > api
> > > > > >> > > > > share
> > > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > > > > >> > > > > >> > >
> > > > > >> > > > > >> > > I think sub-projects keep separate committers and
> > can
> > > > > have a
> > > > > >> > > > > separate
> > > > > >> > > > > >> > repo,
> > > > > >> > > > > >> > > but I'm actually not really sure (I can't find a
> > > > > definition
> > > > > >> > of a
> > > > > >> > > > > >> > subproject
> > > > > >> > > > > >> > > in Apache).
> > > > > >> > > > > >> > >
> > > > > >> > > > > >> > > Basically at a high-level you want the experience
> > to
> > > > > "feel"
> > > > > >> > > like a
> > > > > >> > > > > >> single
> > > > > >> > > > > >> > > system, not to relatively independent things that
> > are
> > > > > kind
> > > > > >> of
> > > > > >> > > > > >> awkwardly
> > > > > >> > > > > >> > > glued together.
> > > > > >> > > > > >> > >
> > > > > >> > > > > >> > > I think if we did that they having naming or
> > branding
> > > > > like
> > > > > >> > > "kafka
> > > > > >> > > > > >> > > streaming" or "kafka streams" or something like
> > that
> > > > > would
> > > > > >> > > > actually
> > > > > >> > > > > >> do a
> > > > > >> > > > > >> > > good job of conveying what it is. I do that this
> > > would
> > > > > help
> > > > > >> > > > adoption
> > > > > >> > > > > >> > quite
> > > > > >> > > > > >> > > a lot as it would correctly convey that using
> Kafka
> > > > > >> Streaming
> > > > > >> > > with
> > > > > >> > > > > >> Kafka
> > > > > >> > > > > >> > is
> > > > > >> > > > > >> > > a fairly seamless experience and Kafka is pretty
> > > > heavily
> > > > > >> > adopted
> > > > > >> > > > at
> > > > > >> > > > > >> this
> > > > > >> > > > > >> > > point.
> > > > > >> > > > > >> > >
> > > > > >> > > > > >> > > Fwiw we actually considered this model originally
> > > when
> > > > > open
> > > > > >> > > > sourcing
> > > > > >> > > > > >> > Samza,
> > > > > >> > > > > >> > > however at that time Kafka was relatively unknown
> > and
> > > > we
> > > > > >> > decided
> > > > > >> > > > not
> > > > > >> > > > > >> to
> > > > > >> > > > > >> > do
> > > > > >> > > > > >> > > it since we felt it would be limiting. From my
> > point
> > > of
> > > > > view
> > > > > >> > the
> > > > > >> > > > > three
> > > > > >> > > > > >> > > things have changed (1) Kafka is now really
> heavily
> > > > used
> > > > > for
> > > > > >> > > > stream
> > > > > >> > > > > >> > > processing, (2) we learned that abstracting out
> the
> > > > > stream
> > > > > >> > well
> > > > > >> > > is
> > > > > >> > > > > >> > > basically impossible, (3) we learned it is really
> > > hard
> > > > to
> > > > > >> keep
> > > > > >> > > the
> > > > > >> > > > > two
> > > > > >> > > > > >> > > things feeling like a single product.
> > > > > >> > > > > >> > >
> > > > > >> > > > > >> > > -Jay
> > > > > >> > > > > >> > >
> > > > > >> > > > > >> > >
> > > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann
> <
> > > > > >> > > > > >> martin@kleppmann.com>
> > > > > >> > > > > >> > > wrote:
> > > > > >> > > > > >> > >
> > > > > >> > > > > >> > >> Hi all,
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> > >> Lots of good thoughts here.
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> > >> I agree with the general philosophy of tying
> Samza
> > > > more
> > > > > >> > firmly
> > > > > >> > > to
> > > > > >> > > > > >> Kafka.
> > > > > >> > > > > >> > >> After I spent a while looking at integrating
> other
> > > > > message
> > > > > >> > > > brokers
> > > > > >> > > > > >> (e.g.
> > > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
> > > conclusion
> > > > > that
> > > > > >> > > > > >> > SystemConsumer
> > > > > >> > > > > >> > >> tacitly assumes a model so much like Kafka's
> that
> > > > pretty
> > > > > >> much
> > > > > >> > > > > nobody
> > > > > >> > > > > >> but
> > > > > >> > > > > >> > >> Kafka actually implements it. (Databus is
> perhaps
> > an
> > > > > >> > exception,
> > > > > >> > > > but
> > > > > >> > > > > >> it
> > > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus,
> > making
> > > > > Samza
> > > > > >> > > fully
> > > > > >> > > > > >> > dependent
> > > > > >> > > > > >> > >> on Kafka acknowledges that the
> system-independence
> > > was
> > > > > >> never
> > > > > >> > as
> > > > > >> > > > > real
> > > > > >> > > > > >> as
> > > > > >> > > > > >> > we
> > > > > >> > > > > >> > >> perhaps made it out to be. The gains of code
> reuse
> > > are
> > > > > >> real.
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> > >> The idea of decoupling Samza from YARN has also
> > > always
> > > > > been
> > > > > >> > > > > >> appealing to
> > > > > >> > > > > >> > >> me, for various reasons already mentioned in
> this
> > > > > thread.
> > > > > >> > > > Although
> > > > > >> > > > > >> > making
> > > > > >> > > > > >> > >> Samza jobs deployable on anything
> > > (YARN/Mesos/AWS/etc)
> > > > > >> seems
> > > > > >> > > > > >> laudable,
> > > > > >> > > > > >> > I am
> > > > > >> > > > > >> > >> a little concerned that it will restrict us to a
> > > > lowest
> > > > > >> > common
> > > > > >> > > > > >> > denominator.
> > > > > >> > > > > >> > >> For example, would host affinity (SAMZA-617)
> still
> > > be
> > > > > >> > possible?
> > > > > >> > > > For
> > > > > >> > > > > >> jobs
> > > > > >> > > > > >> > >> with large amounts of state, I think SAMZA-617
> > would
> > > > be
> > > > > a
> > > > > >> big
> > > > > >> > > > boon,
> > > > > >> > > > > >> > since
> > > > > >> > > > > >> > >> restoring state off the changelog on every
> single
> > > > > restart
> > > > > >> is
> > > > > >> > > > > painful,
> > > > > >> > > > > >> > due
> > > > > >> > > > > >> > >> to long recovery times. It would be a shame if
> the
> > > > > >> decoupling
> > > > > >> > > > from
> > > > > >> > > > > >> YARN
> > > > > >> > > > > >> > >> made host affinity impossible.
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> > >> Jay, a question about the proposed API for
> > > > > instantiating a
> > > > > >> > job
> > > > > >> > > in
> > > > > >> > > > > >> code
> > > > > >> > > > > >> > >> (rather than a properties file): when
> submitting a
> > > job
> > > > > to a
> > > > > >> > > > > cluster,
> > > > > >> > > > > >> is
> > > > > >> > > > > >> > the
> > > > > >> > > > > >> > >> idea that the instantiation code runs on a
> client
> > > > > >> somewhere,
> > > > > >> > > > which
> > > > > >> > > > > >> then
> > > > > >> > > > > >> > >> pokes the necessary endpoints on
> > YARN/Mesos/AWS/etc?
> > > > Or
> > > > > >> does
> > > > > >> > > that
> > > > > >> > > > > >> code
> > > > > >> > > > > >> > run
> > > > > >> > > > > >> > >> on each container that is part of the job (in
> > which
> > > > > case,
> > > > > >> how
> > > > > >> > > > does
> > > > > >> > > > > >> the
> > > > > >> > > > > >> > job
> > > > > >> > > > > >> > >> submission to the cluster work)?
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> > >> I agree with Garry that it doesn't feel right to
> > > make
> > > > a
> > > > > 1.0
> > > > > >> > > > release
> > > > > >> > > > > >> > with a
> > > > > >> > > > > >> > >> plan for it to be immediately obsolete. So if
> this
> > > is
> > > > > going
> > > > > >> > to
> > > > > >> > > > > >> happen, I
> > > > > >> > > > > >> > >> think it would be more honest to stick with 0.*
> > > > version
> > > > > >> > numbers
> > > > > >> > > > > until
> > > > > >> > > > > >> > the
> > > > > >> > > > > >> > >> library-ified Samza has been implemented, is
> > stable
> > > > and
> > > > > >> > widely
> > > > > >> > > > > used.
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> > >> Should the new Samza be a subproject of Kafka?
> > There
> > > > is
> > > > > >> > > precedent
> > > > > >> > > > > for
> > > > > >> > > > > >> > >> tight coupling between different Apache projects
> > > (e.g.
> > > > > >> > Curator
> > > > > >> > > > and
> > > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think
> > remaining
> > > > > >> separate
> > > > > >> > > > would
> > > > > >> > > > > >> be
> > > > > >> > > > > >> > ok.
> > > > > >> > > > > >> > >> Even if Samza is fully dependent on Kafka, there
> > is
> > > > > enough
> > > > > >> > > > > substance
> > > > > >> > > > > >> in
> > > > > >> > > > > >> > >> Samza that it warrants being a separate project.
> > An
> > > > > >> argument
> > > > > >> > in
> > > > > >> > > > > >> favour
> > > > > >> > > > > >> > of
> > > > > >> > > > > >> > >> merging would be if we think Kafka has a much
> > > stronger
> > > > > >> "brand
> > > > > >> > > > > >> presence"
> > > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If the
> > Kafka
> > > > > >> project
> > > > > >> > is
> > > > > >> > > > > >> willing
> > > > > >> > > > > >> > to
> > > > > >> > > > > >> > >> endorse Samza as the "official" way of doing
> > > stateful
> > > > > >> stream
> > > > > >> > > > > >> > >> transformations, that would probably have much
> the
> > > > same
> > > > > >> > effect
> > > > > >> > > as
> > > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream Processors"
> or
> > > > > suchlike.
> > > > > >> > > Close
> > > > > >> > > > > >> > >> collaboration between the two projects will be
> > > needed
> > > > in
> > > > > >> any
> > > > > >> > > > case.
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> > >> From a project management perspective, I guess
> the
> > > > "new
> > > > > >> > Samza"
> > > > > >> > > > > would
> > > > > >> > > > > >> > have
> > > > > >> > > > > >> > >> to be developed on a branch alongside ongoing
> > > > > maintenance
> > > > > >> of
> > > > > >> > > the
> > > > > >> > > > > >> current
> > > > > >> > > > > >> > >> line of development? I think it would be
> important
> > > to
> > > > > >> > continue
> > > > > >> > > > > >> > supporting
> > > > > >> > > > > >> > >> existing users, and provide a graceful migration
> > > path
> > > > to
> > > > > >> the
> > > > > >> > > new
> > > > > >> > > > > >> > version.
> > > > > >> > > > > >> > >> Leaving the current versions unsupported and
> > forcing
> > > > > people
> > > > > >> > to
> > > > > >> > > > > >> rewrite
> > > > > >> > > > > >> > >> their jobs would send a bad signal.
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> > >> Best,
> > > > > >> > > > > >> > >> Martin
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > > jay@confluent.io>
> > > > > >> wrote:
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> > >>> Hey Garry,
> > > > > >> > > > > >> > >>>
> > > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy to
> > chat
> > > > > more
> > > > > >> > about
> > > > > >> > > > > this
> > > > > >> > > > > >> if
> > > > > >> > > > > >> > >>> you'd be interested. I think Chris and I
> started
> > > with
> > > > > the
> > > > > >> > idea
> > > > > >> > > > of
> > > > > >> > > > > >> "what
> > > > > >> > > > > >> > >>> would it take to make Samza a kick-ass
> ingestion
> > > > tool"
> > > > > but
> > > > > >> > > > > >> ultimately
> > > > > >> > > > > >> > we
> > > > > >> > > > > >> > >>> kind of came around to the idea that ingestion
> > and
> > > > > >> > > > transformation
> > > > > >> > > > > >> had
> > > > > >> > > > > >> > >>> pretty different needs and coupling the two
> made
> > > > things
> > > > > >> > hard.
> > > > > >> > > > > >> > >>>
> > > > > >> > > > > >> > >>> For what it's worth I think copycat (KIP-26)
> > > actually
> > > > > will
> > > > > >> > do
> > > > > >> > > > what
> > > > > >> > > > > >> you
> > > > > >> > > > > >> > >> are
> > > > > >> > > > > >> > >>> looking for.
> > > > > >> > > > > >> > >>>
> > > > > >> > > > > >> > >>> With regard to your point about slider, I don't
> > > > > >> necessarily
> > > > > >> > > > > >> disagree.
> > > > > >> > > > > >> > >> But I
> > > > > >> > > > > >> > >>> think getting good YARN support is quite doable
> > > and I
> > > > > >> think
> > > > > >> > we
> > > > > >> > > > can
> > > > > >> > > > > >> make
> > > > > >> > > > > >> > >>> that work well. I think the issue this proposal
> > > > solves
> > > > > is
> > > > > >> > that
> > > > > >> > > > > >> > >> technically
> > > > > >> > > > > >> > >>> it is pretty hard to support multiple cluster
> > > > > management
> > > > > >> > > systems
> > > > > >> > > > > the
> > > > > >> > > > > >> > way
> > > > > >> > > > > >> > >>> things are now, you need to write an "app
> master"
> > > or
> > > > > >> > > "framework"
> > > > > >> > > > > for
> > > > > >> > > > > >> > each
> > > > > >> > > > > >> > >>> and they are all a little different so testing
> is
> > > > > really
> > > > > >> > hard.
> > > > > >> > > > In
> > > > > >> > > > > >> the
> > > > > >> > > > > >> > >>> absence of this we have been stuck with just
> YARN
> > > > which
> > > > > >> has
> > > > > >> > > > > >> fantastic
> > > > > >> > > > > >> > >>> penetration in the Hadoopy part of the org, but
> > > zero
> > > > > >> > > penetration
> > > > > >> > > > > >> > >> elsewhere.
> > > > > >> > > > > >> > >>> Given the huge amount of work being put in to
> > > slider,
> > > > > >> > > marathon,
> > > > > >> > > > > aws
> > > > > >> > > > > >> > >>> tooling, not to mention the umpteen related
> > > packaging
> > > > > >> > > > technologies
> > > > > >> > > > > >> > people
> > > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> > > > cloud-specific
> > > > > >> > deploy
> > > > > >> > > > > >> tools,
> > > > > >> > > > > >> > >> etc)
> > > > > >> > > > > >> > >>> I really think it is important to get this
> right.
> > > > > >> > > > > >> > >>>
> > > > > >> > > > > >> > >>> -Jay
> > > > > >> > > > > >> > >>>
> > > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry
> Turkington
> > <
> > > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > > > > >> > > > > >> > >>>
> > > > > >> > > > > >> > >>>> Hi all,
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>> I think the question below re does Samza
> become
> > a
> > > > > >> > sub-project
> > > > > >> > > > of
> > > > > >> > > > > >> Kafka
> > > > > >> > > > > >> > >>>> highlights the broader point around migration.
> > > Chris
> > > > > >> > mentions
> > > > > >> > > > > >> Samza's
> > > > > >> > > > > >> > >>>> maturity is heading towards a v1 release but
> I'm
> > > not
> > > > > sure
> > > > > >> > it
> > > > > >> > > > > feels
> > > > > >> > > > > >> > >> right to
> > > > > >> > > > > >> > >>>> launch a v1 then immediately plan to deprecate
> > > most
> > > > of
> > > > > >> it.
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>> From a selfish perspective I have some guys
> who
> > > have
> > > > > >> > started
> > > > > >> > > > > >> working
> > > > > >> > > > > >> > >> with
> > > > > >> > > > > >> > >>>> Samza and building some new
> consumers/producers
> > > was
> > > > > next
> > > > > >> > up.
> > > > > >> > > > > Sounds
> > > > > >> > > > > >> > like
> > > > > >> > > > > >> > >>>> that is absolutely not the direction to go. I
> > need
> > > > to
> > > > > >> look
> > > > > >> > > into
> > > > > >> > > > > the
> > > > > >> > > > > >> > KIP
> > > > > >> > > > > >> > >> in
> > > > > >> > > > > >> > >>>> more detail but for me the attractiveness of
> > > adding
> > > > > new
> > > > > >> > Samza
> > > > > >> > > > > >> > >>>> consumer/producers -- even if yes all they
> were
> > > > doing
> > > > > was
> > > > > >> > > > really
> > > > > >> > > > > >> > getting
> > > > > >> > > > > >> > >>>> data into and out of Kafka --  was to avoid
> > > having
> > > > to
> > > > > >> > worry
> > > > > >> > > > > about
> > > > > >> > > > > >> the
> > > > > >> > > > > >> > >>>> lifecycle management of external clients. If
> > there
> > > > is
> > > > > a
> > > > > >> > > generic
> > > > > >> > > > > >> Kafka
> > > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a new
> > > connector
> > > > > into
> > > > > >> > and
> > > > > >> > > > > have
> > > > > >> > > > > >> a
> > > > > >> > > > > >> > >> lot of
> > > > > >> > > > > >> > >>>> the heavy lifting re scale and reliability
> done
> > > for
> > > > me
> > > > > >> then
> > > > > >> > > it
> > > > > >> > > > > >> gives
> > > > > >> > > > > >> > me
> > > > > >> > > > > >> > >> all
> > > > > >> > > > > >> > >>>> the pushing new consumers/producers would. If
> > not
> > > > > then it
> > > > > >> > > > > >> complicates
> > > > > >> > > > > >> > my
> > > > > >> > > > > >> > >>>> operational deployments.
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>> Which is similar to my other question with the
> > > > > proposal
> > > > > >> --
> > > > > >> > if
> > > > > >> > > > we
> > > > > >> > > > > >> > build a
> > > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus the
> > > requisite
> > > > > >> shims
> > > > > >> > to
> > > > > >> > > > > >> > integrate
> > > > > >> > > > > >> > >>>> with Slider etc I suspect the former may be a
> > lot
> > > > more
> > > > > >> work
> > > > > >> > > > than
> > > > > >> > > > > we
> > > > > >> > > > > >> > >> think.
> > > > > >> > > > > >> > >>>> We may make it much easier for a newcomer to
> get
> > > > > >> something
> > > > > >> > > > > running
> > > > > >> > > > > >> but
> > > > > >> > > > > >> > >>>> having them step up and get a reliable
> > production
> > > > > >> > deployment
> > > > > >> > > > may
> > > > > >> > > > > >> still
> > > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for
> different
> > > > > reasons
> > > > > >> > than
> > > > > >> > > > > >> today.
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with
> > making
> > > > the
> > > > > >> Samza
> > > > > >> > > > > >> dependency
> > > > > >> > > > > >> > >> on
> > > > > >> > > > > >> > >>>> Kafka much more explicit and I absolutely see
> > the
> > > > > >> benefits
> > > > > >> > > in
> > > > > >> > > > > the
> > > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > > > > >> > > > terminologies/abstractions
> > > > > >> > > > > >> that
> > > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library would
> > > likely
> > > > > be a
> > > > > >> > very
> > > > > >> > > > > nice
> > > > > >> > > > > >> > tool
> > > > > >> > > > > >> > >> to
> > > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the
> > > concerns
> > > > > >> above
> > > > > >> > re
> > > > > >> > > > the
> > > > > >> > > > > >> > >>>> operational side.
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>> Garry
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>> -----Original Message-----
> > > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
> > > > > >> > gdfm@apache.org
> > > > > >> > > ]
> > > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on
> Samza
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>> Very interesting thoughts.
> > > > > >> > > > > >> > >>>> From outside, I have always perceived Samza
> as a
> > > > > >> computing
> > > > > >> > > > layer
> > > > > >> > > > > >> over
> > > > > >> > > > > >> > >>>> Kafka.
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>> The question, maybe a bit provocative, is
> > "should
> > > > > Samza
> > > > > >> be
> > > > > >> > a
> > > > > >> > > > > >> > sub-project
> > > > > >> > > > > >> > >>>> of Kafka then?"
> > > > > >> > > > > >> > >>>> Or does it make sense to keep it as a separate
> > > > project
> > > > > >> > with a
> > > > > >> > > > > >> separate
> > > > > >> > > > > >> > >>>> governance?
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>> Cheers,
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>> --
> > > > > >> > > > > >> > >>>> Gianmarco
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > > > > yanfang724@gmail.com>
> > > > > >> > > > wrote:
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more
> > > tightly.
> > > > > >> > Because
> > > > > >> > > > > Samza
> > > > > >> > > > > >> de
> > > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should
> leverage
> > > > what
> > > > > >> Kafka
> > > > > >> > > > has.
> > > > > >> > > > > At
> > > > > >> > > > > >> > the
> > > > > >> > > > > >> > >>>>> same time, Kafka does not need to reinvent
> what
> > > > Samza
> > > > > >> > > already
> > > > > >> > > > > >> has. I
> > > > > >> > > > > >> > >>>>> also like the idea of separating the
> ingestion
> > > and
> > > > > >> > > > > transformation.
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>> But it is a little difficult for me to image
> > how
> > > > the
> > > > > >> Samza
> > > > > >> > > > will
> > > > > >> > > > > >> look
> > > > > >> > > > > >> > >>>> like.
> > > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> > difference
> > > > in
> > > > > >> terms
> > > > > >> > > of
> > > > > >> > > > > how
> > > > > >> > > > > >> > >>>>> Samza should look like.
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code shows
> (A
> > > > > client of
> > > > > >> > > > Kakfa)
> > > > > >> > > > > ?
> > > > > >> > > > > >> And
> > > > > >> > > > > >> > >>>>> user's application code calls this client?
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka
> (like
> > > > what
> > > > > the
> > > > > >> > > code
> > > > > >> > > > > >> shows),
> > > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > > > fault-tolerance?
> > > > > >> Are
> > > > > >> > > they
> > > > > >> > > > > >> taken
> > > > > >> > > > > >> > >>>>> care by the Kafka broker or other mechanism,
> > such
> > > > as
> > > > > >> > "Samza
> > > > > >> > > > > >> worker"
> > > > > >> > > > > >> > >>>>> (just make up the name) ?
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>> 2. What about other features, such as
> > > auto-scaling,
> > > > > >> shared
> > > > > >> > > > > state,
> > > > > >> > > > > >> > >>>>> monitoring?
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is this
> what
> > > > Chris
> > > > > >> > > > suggests?)
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa
> and
> > > > > produce
> > > > > >> to
> > > > > >> > > it.
> > > > > >> > > > > >> Then it
> > > > > >> > > > > >> > >>>>> becomes the same as what Samza looks like
> now,
> > > > > except it
> > > > > >> > > does
> > > > > >> > > > > not
> > > > > >> > > > > >> > rely
> > > > > >> > > > > >> > >>>>> on Yarn anymore.
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it leverage
> > > Kafka's
> > > > > >> > metrics,
> > > > > >> > > > > logs,
> > > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>> Thanks,
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>> Fang, Yan
> > > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang
> Wang <
> > > > > >> > > > > wangguoz@gmail.com
> > > > > >> > > > > >> >
> > > > > >> > > > > >> > >>>> wrote:
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>>> Read through the code example and it looks
> > good
> > > to
> > > > > me.
> > > > > >> A
> > > > > >> > > few
> > > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>> Today Samza deploys as executable runnable
> > like:
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> > --config-factory=...
> > > > > >> > > > > >> > >>>> --config-path=file://...
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>> And this proposal advocate for deploying
> Samza
> > > > more
> > > > > as
> > > > > >> > > > embedded
> > > > > >> > > > > >> > >>>>>> libraries in user application code (ignoring
> > the
> > > > > >> > > terminology
> > > > > >> > > > > >> since
> > > > > >> > > > > >> > >>>>>> it is not the
> > > > > >> > > > > >> > >>>>> same
> > > > > >> > > > > >> > >>>>>> as the prototype code):
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>> StreamTask task = new MyStreamTask(configs);
> > > > Thread
> > > > > >> > thread
> > > > > >> > > =
> > > > > >> > > > > new
> > > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>> I think both of these deployment modes are
> > > > important
> > > > > >> for
> > > > > >> > > > > >> different
> > > > > >> > > > > >> > >>>>>> types
> > > > > >> > > > > >> > >>>>> of
> > > > > >> > > > > >> > >>>>>> users. That said, I think making Samza
> purely
> > > > > >> standalone
> > > > > >> > is
> > > > > >> > > > > still
> > > > > >> > > > > >> > >>>>>> sufficient for either runnable or library
> > modes.
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>> Guozhang
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps
> <
> > > > > >> > > > jay@confluent.io>
> > > > > >> > > > > >> > wrote:
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code example,
> it
> > > was
> > > > > >> > supposed
> > > > > >> > > > to
> > > > > >> > > > > >> look
> > > > > >> > > > > >> > >>>>>>> like
> > > > > >> > > > > >> > >>>>>>> this:
> > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> > > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > > "localhost:4242");
> > > > > >> > > > > >> StreamingConfig
> > > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > "test-topic-2");
> > > > > >> > > > > >> > >>>>>>>
> > config.processor(ExampleStreamProcessor.class);
> > > > > >> > > > > >> > >>>>>>> config.serialization(new
> StringSerializer(),
> > > new
> > > > > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming
> > > container =
> > > > > new
> > > > > >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
> > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > >> > >>>>>>> -Jay
> > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay
> Kreps <
> > > > > >> > > > jay@confluent.io
> > > > > >> > > > > >
> > > > > >> > > > > >> > >>>> wrote:
> > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > >> > >>>>>>>> Hey guys,
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> This came out of some conversations Chris
> > and
> > > I
> > > > > were
> > > > > >> > > having
> > > > > >> > > > > >> > >>>>>>>> around
> > > > > >> > > > > >> > >>>>>>> whether
> > > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a kind
> > of
> > > > data
> > > > > >> > > > ingestion
> > > > > >> > > > > >> > >>>>> framework
> > > > > >> > > > > >> > >>>>>>> for
> > > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26
> > > > "copycat").
> > > > > >> This
> > > > > >> > > > kind
> > > > > >> > > > > of
> > > > > >> > > > > >> > >>>>>> combined
> > > > > >> > > > > >> > >>>>>>>> with complaints around config and YARN and
> > the
> > > > > >> > discussion
> > > > > >> > > > > >> around
> > > > > >> > > > > >> > >>>>>>>> how
> > > > > >> > > > > >> > >>>>> to
> > > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> So the thought experiment was, given that
> > > Samza
> > > > > was
> > > > > >> > > > basically
> > > > > >> > > > > >> > >>>>>>>> already totally Kafka specific, what if
> you
> > > just
> > > > > >> > embraced
> > > > > >> > > > > that
> > > > > >> > > > > >> > >>>>>>>> and turned it
> > > > > >> > > > > >> > >>>>>> into
> > > > > >> > > > > >> > >>>>>>>> something less like a heavyweight
> framework
> > > and
> > > > > more
> > > > > >> > > like a
> > > > > >> > > > > >> > >>>>>>>> third
> > > > > >> > > > > >> > >>>>> Kafka
> > > > > >> > > > > >> > >>>>>>>> client--a kind of "producing consumer"
> with
> > > > state
> > > > > >> > > > management
> > > > > >> > > > > >> > >>>>>> facilities.
> > > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a complex
> > > stream
> > > > > >> > > processing
> > > > > >> > > > > >> > >>>>>>>> framework
> > > > > >> > > > > >> > >>>>>>> this
> > > > > >> > > > > >> > >>>>>>>> would actually be a very simple thing, not
> > > much
> > > > > more
> > > > > >> > > > > >> complicated
> > > > > >> > > > > >> > >>>>>>>> to
> > > > > >> > > > > >> > >>>>> use
> > > > > >> > > > > >> > >>>>>>> or
> > > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris
> said
> > > we
> > > > > >> thought
> > > > > >> > > > about
> > > > > >> > > > > >> it
> > > > > >> > > > > >> > >>>>>>>> a
> > > > > >> > > > > >> > >>>>> lot
> > > > > >> > > > > >> > >>>>>> of
> > > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream
> processing
> > > > > systems
> > > > > >> > were
> > > > > >> > > > > doing)
> > > > > >> > > > > >> > >>>>> seemed
> > > > > >> > > > > >> > >>>>>>> like
> > > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output data
> to
> > > and
> > > > > from
> > > > > >> > the
> > > > > >> > > > > stream
> > > > > >> > > > > >> > >>>>>>>> processing. But when we actually looked
> into
> > > how
> > > > > that
> > > > > >> > > would
> > > > > >> > > > > >> > >>>>>>>> work,
> > > > > >> > > > > >> > >>>>> Samza
> > > > > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion
> > framework
> > > > > for a
> > > > > >> > > bunch
> > > > > >> > > > of
> > > > > >> > > > > >> > >>>>> reasons.
> > > > > >> > > > > >> > >>>>>> To
> > > > > >> > > > > >> > >>>>>>>> really do that right you need a pretty
> > > different
> > > > > >> > internal
> > > > > >> > > > > data
> > > > > >> > > > > >> > >>>>>>>> model
> > > > > >> > > > > >> > >>>>>> and
> > > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split them and
> > had
> > > > an
> > > > > api
> > > > > >> > for
> > > > > >> > > > > Kafka
> > > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a
> > > > separate
> > > > > >> api
> > > > > >> > > for
> > > > > >> > > > > >> Kafka
> > > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> This would also allow really embracing the
> > > same
> > > > > >> > > terminology
> > > > > >> > > > > and
> > > > > >> > > > > >> > >>>>>>>> conventions. One complaint about the
> current
> > > > > state is
> > > > > >> > > that
> > > > > >> > > > > the
> > > > > >> > > > > >> > >>>>>>>> two
> > > > > >> > > > > >> > >>>>>>> systems
> > > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like
> > > > "stream"
> > > > > vs
> > > > > >> > > > "topic"
> > > > > >> > > > > >> and
> > > > > >> > > > > >> > >>>>>>> different
> > > > > >> > > > > >> > >>>>>>>> config and monitoring systems means you
> kind
> > > of
> > > > > have
> > > > > >> to
> > > > > >> > > > learn
> > > > > >> > > > > >> > >>>>>>>> Kafka's
> > > > > >> > > > > >> > >>>>>>> way,
> > > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly different way,
> > > then
> > > > > kind
> > > > > >> of
> > > > > >> > > > > >> > >>>>>>>> understand
> > > > > >> > > > > >> > >>>>> how
> > > > > >> > > > > >> > >>>>>>> they
> > > > > >> > > > > >> > >>>>>>>> map to each other, which having walked a
> few
> > > > > people
> > > > > >> > > through
> > > > > >> > > > > >> this
> > > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of time
> on
> > > > > >> airplanes I
> > > > > >> > > > > hacked
> > > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat incomplete
> > > > > prototype
> > > > > >> of
> > > > > >> > > > what
> > > > > >> > > > > >> > >>>>>>>> this would
> > > > > >> > > > > >> > >>>>> look
> > > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously dumped
> > into
> > > > > Kafka
> > > > > >> as
> > > > > >> > > it
> > > > > >> > > > > >> > >>>>>>>> required a
> > > > > >> > > > > >> > >>>>>> few
> > > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is the
> > code:
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> >
> > > > > >> > > > >
> > > > > >> >
> > > > >
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I just
> > > > liberally
> > > > > >> > renamed
> > > > > >> > > > > >> > >>>>>>>> everything
> > > > > >> > > > > >> > >>>>> to
> > > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no regard
> > for
> > > > > >> > > > compatibility.
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> To use this would be something like this:
> > > > > >> > > > > >> > >>>>>>>> Properties props = new Properties();
> > > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > > > "localhost:4242");
> > > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > > > > >> > > > > >> config.processor(ExampleStreamProcessor.class);
> > > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> > StringDeserializer());
> > > > > >> > > > KafkaStreaming
> > > > > >> > > > > >> > >>>>>> container =
> > > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config);
> container.run();
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > > SamzaContainer;
> > > > > >> > > > > StreamProcessor
> > > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> So rather than putting all the class names
> > in
> > > a
> > > > > file
> > > > > >> > and
> > > > > >> > > > then
> > > > > >> > > > > >> > >>>>>>>> having
> > > > > >> > > > > >> > >>>>>> the
> > > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
> > > > instantiate
> > > > > the
> > > > > >> > > > > container
> > > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced over
> > > however
> > > > > many
> > > > > >> > > > > instances
> > > > > >> > > > > >> > >>>>>>>> of
> > > > > >> > > > > >> > >>>>> this
> > > > > >> > > > > >> > >>>>>>> are
> > > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance
> dies,
> > > new
> > > > > >> tasks
> > > > > >> > > are
> > > > > >> > > > > >> added
> > > > > >> > > > > >> > >>>>>>>> to
> > > > > >> > > > > >> > >>>>> the
> > > > > >> > > > > >> > >>>>>>>> existing containers without shutting them
> > > down).
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> We would provide some glue for running
> this
> > > > stuff
> > > > > in
> > > > > >> > YARN
> > > > > >> > > > via
> > > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using
> > some
> > > > of
> > > > > >> their
> > > > > >> > > > tools
> > > > > >> > > > > >> > >>>>>>>> but from the
> > > > > >> > > > > >> > >>>>>> point
> > > > > >> > > > > >> > >>>>>>> of
> > > > > >> > > > > >> > >>>>>>>> view of these frameworks these stream
> > > processing
> > > > > jobs
> > > > > >> > are
> > > > > >> > > > > just
> > > > > >> > > > > >> > >>>>>> stateless
> > > > > >> > > > > >> > >>>>>>>> services that can come and go and expand
> and
> > > > > contract
> > > > > >> > at
> > > > > >> > > > > will.
> > > > > >> > > > > >> > >>>>>>>> There
> > > > > >> > > > > >> > >>>>> is
> > > > > >> > > > > >> > >>>>>>> no
> > > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it
> would
> > > get
> > > > > >> larger
> > > > > >> > > if
> > > > > >> > > > we
> > > > > >> > > > > >> > >>>>>>>>  productionized but not vastly larger. We
> > > really
> > > > > do
> > > > > >> > get a
> > > > > >> > > > ton
> > > > > >> > > > > >> > >>>>>>>> of
> > > > > >> > > > > >> > >>>>>>> leverage
> > > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully
> delegated
> > to
> > > > the
> > > > > >> new
> > > > > >> > > > > >> consumer.
> > > > > >> > > > > >> > >>>>> This
> > > > > >> > > > > >> > >>>>>>>>  is nice since now any partition
> management
> > > > > strategy
> > > > > >> > > > > available
> > > > > >> > > > > >> > >>>>>>>> to
> > > > > >> > > > > >> > >>>>>> Kafka
> > > > > >> > > > > >> > >>>>>>>>  consumer is also available to Samza (and
> > vice
> > > > > versa)
> > > > > >> > and
> > > > > >> > > > > with
> > > > > >> > > > > >> > >>>>>>>> the
> > > > > >> > > > > >> > >>>>>>> exact
> > > > > >> > > > > >> > >>>>>>>>  same configs.
> > > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as state
> reuse
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is
> thought
> > > > > >> provoking.
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> -Jay
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris
> > > > Riccomini <
> > > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > > > > >> > > > > >> > >>>>>>>> wrote:
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Hey all,
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> I have had some discussions with Samza
> > > > engineers
> > > > > at
> > > > > >> > > > LinkedIn
> > > > > >> > > > > >> > >>>>>>>>> and
> > > > > >> > > > > >> > >>>>>>> Confluent
> > > > > >> > > > > >> > >>>>>>>>> and we came up with a few observations
> and
> > > > would
> > > > > >> like
> > > > > >> > to
> > > > > >> > > > > >> > >>>>>>>>> propose
> > > > > >> > > > > >> > >>>>> some
> > > > > >> > > > > >> > >>>>>>>>> changes.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> We've observed some things that I want to
> > > call
> > > > > out
> > > > > >> > about
> > > > > >> > > > > >> > >>>>>>>>> Samza's
> > > > > >> > > > > >> > >>>>>> design,
> > > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic
> > > deployment
> > > > > >> system.
> > > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > > > >> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer
> and
> > > > > Kafka's
> > > > > >> > > > consumer
> > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > > >> > > > > >> > >>>>> are
> > > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same
> problems.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> All three of these issues are related,
> but
> > > I'll
> > > > > >> > address
> > > > > >> > > > them
> > > > > >> > > > > >> in
> > > > > >> > > > > >> > >>>>> order.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Deployment
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a
> > > dynamic
> > > > > >> > > deployment
> > > > > >> > > > > >> > >>>>>>>>> scheduler
> > > > > >> > > > > >> > >>>>>> such
> > > > > >> > > > > >> > >>>>>>>>> as
> > > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built
> > > > Samza,
> > > > > we
> > > > > >> > bet
> > > > > >> > > > that
> > > > > >> > > > > >> > >>>>>>>>> there
> > > > > >> > > > > >> > >>>>>> would
> > > > > >> > > > > >> > >>>>>>>>> be
> > > > > >> > > > > >> > >>>>>>>>> one or two winners in this area, and we
> > could
> > > > > >> support
> > > > > >> > > > them,
> > > > > >> > > > > >> and
> > > > > >> > > > > >> > >>>>>>>>> the
> > > > > >> > > > > >> > >>>>>> rest
> > > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there are many
> > > > > >> variations.
> > > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > > > > >> > > > > >> > >>>>>> many
> > > > > >> > > > > >> > >>>>>>>>> people still prefer to just start their
> > > > > processors
> > > > > >> > like
> > > > > >> > > > > normal
> > > > > >> > > > > >> > >>>>>>>>> Java processes, and use traditional
> > > deployment
> > > > > >> scripts
> > > > > >> > > > such
> > > > > >> > > > > as
> > > > > >> > > > > >> > >>>>>>>>> Fabric,
> > > > > >> > > > > >> > >>>>>> Chef,
> > > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system
> > on
> > > > > users
> > > > > >> > makes
> > > > > >> > > > the
> > > > > >> > > > > >> > >>>>>>>>> Samza start-up process really painful for
> > > first
> > > > > time
> > > > > >> > > > users.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was
> > also
> > > a
> > > > > bit
> > > > > >> of
> > > > > >> > a
> > > > > >> > > > > >> > >>>>>>>>> mis-fire
> > > > > >> > > > > >> > >>>>>> because
> > > > > >> > > > > >> > >>>>>>>>> of
> > > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between
> the
> > > > > nature of
> > > > > >> > > batch
> > > > > >> > > > > >> jobs
> > > > > >> > > > > >> > >>>>>>>>> and
> > > > > >> > > > > >> > >>>>>>> stream
> > > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> > conscious
> > > > > effort
> > > > > >> to
> > > > > >> > > > favor
> > > > > >> > > > > >> > >>>>>>>>> the
> > > > > >> > > > > >> > >>>>>> Hadoop
> > > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since
> it
> > > > worked
> > > > > >> and
> > > > > >> > > was
> > > > > >> > > > > well
> > > > > >> > > > > >> > >>>>>>> understood.
> > > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that batch
> > jobs
> > > > > have a
> > > > > >> > > > definite
> > > > > >> > > > > >> > >>>>>> beginning,
> > > > > >> > > > > >> > >>>>>>>>> and
> > > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't
> > > > (usually).
> > > > > >> This
> > > > > >> > > > leads
> > > > > >> > > > > to
> > > > > >> > > > > >> > >>>>>>>>> a
> > > > > >> > > > > >> > >>>>> much
> > > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream
> > > > processors.
> > > > > >> You
> > > > > >> > > > > >> basically
> > > > > >> > > > > >> > >>>>>>>>> just
> > > > > >> > > > > >> > >>>>>>> need
> > > > > >> > > > > >> > >>>>>>>>> to find a place to start the processor,
> and
> > > > start
> > > > > >> it.
> > > > > >> > > The
> > > > > >> > > > > way
> > > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no
> > concept
> > > > of
> > > > > a
> > > > > >> > > cluster
> > > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > > > > >> > > > > >> > >>>>>> add
> > > > > >> > > > > >> > >>>>>>>>> more machines. The problem with coupling
> > > Samza
> > > > > with
> > > > > >> a
> > > > > >> > > > > >> scheduler
> > > > > >> > > > > >> > >>>>>>>>> is
> > > > > >> > > > > >> > >>>>>> that
> > > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to handle
> > > > > deployment.
> > > > > >> > > This
> > > > > >> > > > > >> pulls
> > > > > >> > > > > >> > >>>>>>>>> in a
> > > > > >> > > > > >> > >>>>>>> bunch
> > > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> > distribution
> > > > > (config
> > > > > >> > > > > stream),
> > > > > >> > > > > >> > >>>>>>>>> shell
> > > > > >> > > > > >> > >>>>>>> scrips
> > > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging
> (all
> > > the
> > > > > .tgz
> > > > > >> > > > stuff),
> > > > > >> > > > > >> etc.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
> > > deployment
> > > > > was
> > > > > >> to
> > > > > >> > > > > support
> > > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> > locality,
> > > > you
> > > > > >> need
> > > > > >> > to
> > > > > >> > > > put
> > > > > >> > > > > >> > >>>>>>>>> your
> > > > > >> > > > > >> > >>>>>> processors
> > > > > >> > > > > >> > >>>>>>>>> close to the data they're processing.
> Upon
> > > > > further
> > > > > >> > > > > >> > >>>>>>>>> investigation,
> > > > > >> > > > > >> > >>>>>>> though,
> > > > > >> > > > > >> > >>>>>>>>> this feature is not that beneficial.
> There
> > is
> > > > > some
> > > > > >> > good
> > > > > >> > > > > >> > >>>>>>>>> discussion
> > > > > >> > > > > >> > >>>>>> about
> > > > > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335.
> Again,
> > we
> > > > > took
> > > > > >> the
> > > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > > > > >> > > > > >> > >>>>>> path,
> > > > > >> > > > > >> > >>>>>>>>> but
> > > > > >> > > > > >> > >>>>>>>>> there are some fundamental differences
> > > between
> > > > > HDFS
> > > > > >> > and
> > > > > >> > > > > Kafka.
> > > > > >> > > > > >> > >>>>>>>>> HDFS
> > > > > >> > > > > >> > >>>>>> has
> > > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions. This
> > > leads
> > > > to
> > > > > >> less
> > > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
> > processors
> > > > on
> > > > > top
> > > > > >> > of
> > > > > >> > > > > Kafka.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> This feature is also used as a crutch.
> > Samza
> > > > > doesn't
> > > > > >> > > have
> > > > > >> > > > > any
> > > > > >> > > > > >> > >>>>>>>>> built
> > > > > >> > > > > >> > >>>>> in
> > > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it
> depends
> > on
> > > > the
> > > > > >> > > dynamic
> > > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to handle
> > > restarts
> > > > > >> when a
> > > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > > > > >> > > > > >> > >>>>>>> made
> > > > > >> > > > > >> > >>>>>>>>> it very difficult to write a standalone
> > Samza
> > > > > >> > container
> > > > > >> > > > > >> > >>>> (SAMZA-516).
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Pluggability
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is good, but I
> > > think
> > > > > that
> > > > > >> > > we've
> > > > > >> > > > > >> gone
> > > > > >> > > > > >> > >>>>>>>>> too
> > > > > >> > > > > >> > >>>>>> far
> > > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > > (SystemConsumer,
> > > > > >> > > > > SystemProducer,
> > > > > >> > > > > >> > >>>> etc).
> > > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about
> every
> > > > > >> component
> > > > > >> > > > > >> > >>>>> (MessageChooser,
> > > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> > ConfigRewriter,
> > > > > etc).
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> There's probably more that I've
> forgotten,
> > as
> > > > > well.
> > > > > >> > Some
> > > > > >> > > > of
> > > > > >> > > > > >> > >>>>>>>>> these
> > > > > >> > > > > >> > >>>>> are
> > > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not to be.
> > This
> > > > all
> > > > > >> comes
> > > > > >> > > at
> > > > > >> > > > a
> > > > > >> > > > > >> cost:
> > > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is making it
> > > harder
> > > > > for
> > > > > >> > our
> > > > > >> > > > > users
> > > > > >> > > > > >> > >>>>>>>>> to
> > > > > >> > > > > >> > >>>>> pick
> > > > > >> > > > > >> > >>>>>> up
> > > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also
> makes
> > > it
> > > > > >> > difficult
> > > > > >> > > > for
> > > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about what the
> > > > > >> > > characteristics
> > > > > >> > > > of
> > > > > >> > > > > >> > >>>>>>>>> the container (since the characteristics
> > > change
> > > > > >> > > depending
> > > > > >> > > > on
> > > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are most
> > visible
> > > > in
> > > > > the
> > > > > >> > > > System
> > > > > >> > > > > >> APIs.
> > > > > >> > > > > >> > >>>>> What
> > > > > >> > > > > >> > >>>>>>>>> Samza really requires to be functional is
> > > Kafka
> > > > > as
> > > > > >> its
> > > > > >> > > > > >> > >>>>>>>>> transport
> > > > > >> > > > > >> > >>>>>> layer.
> > > > > >> > > > > >> > >>>>>>>>> But
> > > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases
> > into
> > > > one
> > > > > >> API:
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> The current System API supports both of
> > these
> > > > use
> > > > > >> > cases.
> > > > > >> > > > The
> > > > > >> > > > > >> > >>>>>>>>> problem
> > > > > >> > > > > >> > >>>>>> is,
> > > > > >> > > > > >> > >>>>>>>>> we
> > > > > >> > > > > >> > >>>>>>>>> actually want different features for each
> > use
> > > > > case.
> > > > > >> By
> > > > > >> > > > > >> papering
> > > > > >> > > > > >> > >>>>>>>>> over
> > > > > >> > > > > >> > >>>>>>> these
> > > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a single
> API,
> > > > we've
> > > > > >> > > > introduced
> > > > > >> > > > > a
> > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > > >> > > > > >> > >>>>>>> leaky
> > > > > >> > > > > >> > >>>>>>>>> abstractions.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like in (2)
> > is
> > > to
> > > > > have
> > > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for
> offsets
> > > > (like
> > > > > >> > Kafka).
> > > > > >> > > > > This
> > > > > >> > > > > >> > >>>>>>>>> would be at odds
> > > > > >> > > > > >> > >>>>> with
> > > > > >> > > > > >> > >>>>>>> (1),
> > > > > >> > > > > >> > >>>>>>>>> though, since different systems have
> > > different
> > > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > > > >> > > > > >> > >>>>>>>>> There was discussion both on the mailing
> > list
> > > > and
> > > > > >> the
> > > > > >> > > SQL
> > > > > >> > > > > >> JIRAs
> > > > > >> > > > > >> > >>>>> about
> > > > > >> > > > > >> > >>>>>>> the
> > > > > >> > > > > >> > >>>>>>>>> need for this.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> > replayability.
> > > > > Kafka
> > > > > >> > > allows
> > > > > >> > > > us
> > > > > >> > > > > >> to
> > > > > >> > > > > >> > >>>>> rewind
> > > > > >> > > > > >> > >>>>>>>>> when
> > > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other systems
> > don't.
> > > In
> > > > > some
> > > > > >> > > > cases,
> > > > > >> > > > > >> > >>>>>>>>> systems
> > > > > >> > > > > >> > >>>>>>> return
> > > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > > > > >> WikipediaSystemConsumer)
> > > > > >> > > > > because
> > > > > >> > > > > >> > >>>>>>>>> they
> > > > > >> > > > > >> > >>>>>> have
> > > > > >> > > > > >> > >>>>>>> no
> > > > > >> > > > > >> > >>>>>>>>> offsets.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka
> > > supports
> > > > > >> > > > > partitioning,
> > > > > >> > > > > >> > >>>>>>>>> but
> > > > > >> > > > > >> > >>>>> many
> > > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by having a
> > > single
> > > > > >> > > partition
> > > > > >> > > > > for
> > > > > >> > > > > >> > >>>>>>>>> those systems. Still, other systems model
> > > > > >> partitioning
> > > > > >> > > > > >> > >>>> differently (e.g.
> > > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a mess.
> > > > > Creating
> > > > > >> > > streams
> > > > > >> > > > > in
> > > > > >> > > > > >> a
> > > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost impossible.
> > As
> > > is
> > > > > >> > modeling
> > > > > >> > > > > >> > >>>>>>>>> metadata
> > > > > >> > > > > >> > >>>>> for
> > > > > >> > > > > >> > >>>>>>> the
> > > > > >> > > > > >> > >>>>>>>>> system (replication factor, partitions,
> > > > location,
> > > > > >> > etc).
> > > > > >> > > > The
> > > > > >> > > > > >> > >>>>>>>>> list
> > > > > >> > > > > >> > >>>>> goes
> > > > > >> > > > > >> > >>>>>>> on.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> At the time that we began writing Samza,
> > > > Kafka's
> > > > > >> > > consumer
> > > > > >> > > > > and
> > > > > >> > > > > >> > >>>>> producer
> > > > > >> > > > > >> > >>>>>>>>> APIs
> > > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On the
> > > > > >> > consumer-side,
> > > > > >> > > > you
> > > > > >> > > > > >> > >>>>>>>>> had two
> > > > > >> > > > > >> > >>>>>>>>> options: use the high level consumer, or
> > the
> > > > > simple
> > > > > >> > > > > consumer.
> > > > > >> > > > > >> > >>>>>>>>> The
> > > > > >> > > > > >> > >>>>>>> problem
> > > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was that it
> > > > > controlled
> > > > > >> > your
> > > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and the
> > order
> > > > in
> > > > > >> which
> > > > > >> > > you
> > > > > >> > > > > >> > >>>>>>>>> received messages. The
> > > > > >> > > > > >> > >>>>> problem
> > > > > >> > > > > >> > >>>>>>>>> with
> > > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not
> > simple.
> > > > It's
> > > > > >> > basic.
> > > > > >> > > > You
> > > > > >> > > > > >> > >>>>>>>>> end up
> > > > > >> > > > > >> > >>>>>>> having
> > > > > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level stuff
> > > that
> > > > > you
> > > > > >> > > > > shouldn't.
> > > > > >> > > > > >> > >>>>>>>>> We
> > > > > >> > > > > >> > >>>>>> spent a
> > > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > > KafkaSystemConsumer
> > > > > very
> > > > > >> > > > robust.
> > > > > >> > > > > >> It
> > > > > >> > > > > >> > >>>>>>>>> also allows us to support some cool
> > features:
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
> > > > > prioritization.
> > > > > >> > > > > >> > >>>>>>>>> * Tight control over partition assignment
> > to
> > > > > support
> > > > > >> > > > joins,
> > > > > >> > > > > >> > >>>>>>>>> global
> > > > > >> > > > > >> > >>>>>> state
> > > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > > > > >> > > > > >> > >>>>>>>>> * Tight control over offset
> checkpointing.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time is
> that
> > > > these
> > > > > >> > > features
> > > > > >> > > > > >> > >>>>>>>>> should
> > > > > >> > > > > >> > >>>>>>> actually
> > > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers
> (not
> > > just
> > > > > >> Samza
> > > > > >> > > > stream
> > > > > >> > > > > >> > >>>>>> processors)
> > > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like joins
> and
> > > > > partition
> > > > > >> > > > > >> > >>>>>>>>> assignment. The
> > > > > >> > > > > >> > >>>>>>> Kafka
> > > > > >> > > > > >> > >>>>>>>>> community has come to the same
> conclusion.
> > > > > They're
> > > > > >> > > adding
> > > > > >> > > > a
> > > > > >> > > > > >> ton
> > > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka consumer
> > > > > >> > > implementation.
> > > > > >> > > > > To a
> > > > > >> > > > > >> > >>>>>>>>> large extent,
> > > > > >> > > > > >> > >>>>> it's
> > > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've already done
> > in
> > > > > Samza.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a
> > very
> > > > > similar
> > > > > >> > > > > approach
> > > > > >> > > > > >> > >>>>>>>>> to
> > > > > >> > > > > >> > >>>>>> Samza's
> > > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation for
> > > > > handling
> > > > > >> > > offset
> > > > > >> > > > > >> > >>>>>> checkpointing.
> > > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset management
> > > > feature
> > > > > >> > stores
> > > > > >> > > > > >> offset
> > > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you to
> > > fetch
> > > > > them
> > > > > >> > > from
> > > > > >> > > > > the
> > > > > >> > > > > >> > >>>>>>>>> broker.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since
> we
> > > > could
> > > > > >> have
> > > > > >> > > > shared
> > > > > >> > > > > >> > >>>>>>>>> the
> > > > > >> > > > > >> > >>>>> work
> > > > > >> > > > > >> > >>>>>> if
> > > > > >> > > > > >> > >>>>>>>>> it
> > > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Vision
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather radical
> > > > > proposal.
> > > > > >> > Samza
> > > > > >> > > > is
> > > > > >> > > > > >> > >>>>> relatively
> > > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to say
> > that
> > > > > we're
> > > > > >> > > near a
> > > > > >> > > > > 1.0
> > > > > >> > > > > >> > >>>>>> release.
> > > > > >> > > > > >> > >>>>>>>>> I'd
> > > > > >> > > > > >> > >>>>>>>>> like to propose that we take what we've
> > > > learned,
> > > > > and
> > > > > >> > > begin
> > > > > >> > > > > >> > >>>>>>>>> thinking
> > > > > >> > > > > >> > >>>>>>> about
> > > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change if
> > we
> > > > were
> > > > > >> > > starting
> > > > > >> > > > > >> from
> > > > > >> > > > > >> > >>>>>> scratch?
> > > > > >> > > > > >> > >>>>>>>>> My
> > > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way
> to
> > > run
> > > > > Samza
> > > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
> > > > dependences
> > > > > on
> > > > > >> > > YARN,
> > > > > >> > > > > >> Mesos,
> > > > > >> > > > > >> > >>>> etc.
> > > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support only
> > > Kafka
> > > > > as
> > > > > >> the
> > > > > >> > > > > stream
> > > > > >> > > > > >> > >>>>>> processing
> > > > > >> > > > > >> > >>>>>>>>> layer.
> > > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
> > > > > >> serialization,
> > > > > >> > > and
> > > > > >> > > > > >> > >>>>>>>>> config
> > > > > >> > > > > >> > >>>>>>> systems,
> > > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues that I
> > > > outlined
> > > > > >> > above.
> > > > > >> > > It
> > > > > >> > > > > >> > >>>>>>>>> should
> > > > > >> > > > > >> > >>>>> also
> > > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> > > dramatically.
> > > > > >> > > Supporting
> > > > > >> > > > > >> only
> > > > > >> > > > > >> > >>>>>>>>> a standalone container will allow Samza
> to
> > be
> > > > > >> executed
> > > > > >> > > on
> > > > > >> > > > > YARN
> > > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > > Marathon/Aurora),
> > > > or
> > > > > >> most
> > > > > >> > > > other
> > > > > >> > > > > >> > >>>>>>>>> in-house
> > > > > >> > > > > >> > >>>>>>> deployment
> > > > > >> > > > > >> > >>>>>>>>> systems. This should make life a lot
> easier
> > > for
> > > > > new
> > > > > >> > > users.
> > > > > >> > > > > >> > >>>>>>>>> Imagine
> > > > > >> > > > > >> > >>>>>>> having
> > > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN.
> The
> > > drop
> > > > > in
> > > > > >> > > mailing
> > > > > >> > > > > >> list
> > > > > >> > > > > >> > >>>>>> traffic
> > > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue to
> > me.
> > > > The
> > > > > >> > > reality
> > > > > >> > > > > is,
> > > > > >> > > > > >> > >>>>> everyone
> > > > > >> > > > > >> > >>>>>>>>> that
> > > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka.
> We
> > > > > basically
> > > > > >> > > > require
> > > > > >> > > > > >> it
> > > > > >> > > > > >> > >>>>>> already
> > > > > >> > > > > >> > >>>>>>> in
> > > > > >> > > > > >> > >>>>>>>>> order for most features to work. Those
> that
> > > are
> > > > > >> using
> > > > > >> > > > other
> > > > > >> > > > > >> > >>>>>>>>> systems
> > > > > >> > > > > >> > >>>>>> are
> > > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into Kafka
> > (1),
> > > > and
> > > > > >> then
> > > > > >> > > > they
> > > > > >> > > > > do
> > > > > >> > > > > >> > >>>>>>>>> the processing on top. There is already
> > > > > discussion (
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> >
> > > > > >> > > > >
> > > > > >> >
> > > > >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > > >> > > > > >> > >>>>> 767
> > > > > >> > > > > >> > >>>>>>>>> )
> > > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka
> > > extremely
> > > > > >> easy.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple with
> Kafka,
> > > we
> > > > > can
> > > > > >> > > > leverage
> > > > > >> > > > > a
> > > > > >> > > > > >> > >>>>>>>>> ton of
> > > > > >> > > > > >> > >>>>>>> their
> > > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to maintain
> > our
> > > > own
> > > > > >> > config,
> > > > > >> > > > > >> > >>>>>>>>> metrics,
> > > > > >> > > > > >> > >>>>> etc.
> > > > > >> > > > > >> > >>>>>>> We
> > > > > >> > > > > >> > >>>>>>>>> can all share the same libraries, and
> make
> > > them
> > > > > >> > better.
> > > > > >> > > > This
> > > > > >> > > > > >> > >>>>>>>>> will
> > > > > >> > > > > >> > >>>>> also
> > > > > >> > > > > >> > >>>>>>>>> allow us to share the consumer/producer
> > APIs,
> > > > and
> > > > > >> will
> > > > > >> > > let
> > > > > >> > > > > us
> > > > > >> > > > > >> > >>>>> leverage
> > > > > >> > > > > >> > >>>>>>>>> their offset management and partition
> > > > management,
> > > > > >> > rather
> > > > > >> > > > > than
> > > > > >> > > > > >> > >>>>>>>>> having
> > > > > >> > > > > >> > >>>>>> our
> > > > > >> > > > > >> > >>>>>>>>> own. All of the coordinator stream code
> > would
> > > > go
> > > > > >> away,
> > > > > >> > > as
> > > > > >> > > > > >> would
> > > > > >> > > > > >> > >>>>>>>>> most
> > > > > >> > > > > >> > >>>>>> of
> > > > > >> > > > > >> > >>>>>>>>> the
> > > > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have
> to
> > > push
> > > > > some
> > > > > >> > > > > partition
> > > > > >> > > > > >> > >>>>>>> management
> > > > > >> > > > > >> > >>>>>>>>> features into the Kafka broker, but
> they're
> > > > > already
> > > > > >> > > moving
> > > > > >> > > > > in
> > > > > >> > > > > >> > >>>>>>>>> that direction with the new consumer API.
> > The
> > > > > >> features
> > > > > >> > > we
> > > > > >> > > > > have
> > > > > >> > > > > >> > >>>>>>>>> for
> > > > > >> > > > > >> > >>>>>> partition
> > > > > >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and
> seem
> > > > like
> > > > > >> they
> > > > > >> > > > should
> > > > > >> > > > > >> be
> > > > > >> > > > > >> > >>>>>>>>> in
> > > > > >> > > > > >> > >>>>>> Kafka
> > > > > >> > > > > >> > >>>>>>>>> anyway. There will always be some niche
> > > usages
> > > > > which
> > > > > >> > > will
> > > > > >> > > > > >> > >>>>>>>>> require
> > > > > >> > > > > >> > >>>>>> extra
> > > > > >> > > > > >> > >>>>>>>>> care and hence full control over
> partition
> > > > > >> assignments
> > > > > >> > > > much
> > > > > >> > > > > >> > >>>>>>>>> like the
> > > > > >> > > > > >> > >>>>>>> Kafka
> > > > > >> > > > > >> > >>>>>>>>> low level consumer api. These would
> > continue
> > > to
> > > > > be
> > > > > >> > > > > supported.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> These items will be good for the Samza
> > > > community.
> > > > > >> > > They'll
> > > > > >> > > > > make
> > > > > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it easier
> for
> > > > > >> developers
> > > > > >> > > to
> > > > > >> > > > > add
> > > > > >> > > > > >> > >>>>>>>>> new features.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and
> > > somewhat
> > > > > >> > backwards
> > > > > >> > > > > >> > >>>>> incompatible
> > > > > >> > > > > >> > >>>>>>>>> change). If we choose to go this route,
> > it's
> > > > > >> important
> > > > > >> > > > that
> > > > > >> > > > > we
> > > > > >> > > > > >> > >>>>> openly
> > > > > >> > > > > >> > >>>>>>>>> communicate how we're going to provide a
> > > > > migration
> > > > > >> > path
> > > > > >> > > > from
> > > > > >> > > > > >> > >>>>>>>>> the
> > > > > >> > > > > >> > >>>>>>> existing
> > > > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make
> > incompatible
> > > > > >> > changes).
> > > > > >> > > I
> > > > > >> > > > > >> think
> > > > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to
> > provide a
> > > > > >> wrapper
> > > > > >> > to
> > > > > >> > > > > allow
> > > > > >> > > > > >> > >>>>>>>>> existing StreamTask implementations to
> > > continue
> > > > > >> > running
> > > > > >> > > on
> > > > > >> > > > > the
> > > > > >> > > > > >> > >>>> new container.
> > > > > >> > > > > >> > >>>>>>> It's
> > > > > >> > > > > >> > >>>>>>>>> also important that we openly communicate
> > > about
> > > > > >> > timing,
> > > > > >> > > > and
> > > > > >> > > > > >> > >>>>>>>>> stages
> > > > > >> > > > > >> > >>>>> of
> > > > > >> > > > > >> > >>>>>>> the
> > > > > >> > > > > >> > >>>>>>>>> migration.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you
> have
> > > > > opinions.
> > > > > >> > :)
> > > > > >> > > > > Please
> > > > > >> > > > > >> > >>>>>>>>> send
> > > > > >> > > > > >> > >>>>>> your
> > > > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>> Cheers,
> > > > > >> > > > > >> > >>>>>>>>> Chris
> > > > > >> > > > > >> > >>>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>>
> > > > > >> > > > > >> > >>>>>>>
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>> --
> > > > > >> > > > > >> > >>>>>> -- Guozhang
> > > > > >> > > > > >> > >>>>>>
> > > > > >> > > > > >> > >>>>>
> > > > > >> > > > > >> > >>>>
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> > >>
> > > > > >> > > > > >> >
> > > > > >> > > > > >> >
> > > > > >> > > > > >> >
> > > > > >> > > > > >>
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yan Fang <ya...@gmail.com>.
Thanks, Jay. This argument persuaded me actually. :)

Fang, Yan
yanfang724@gmail.com

On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps <ja...@confluent.io> wrote:

> Hey Yan,
>
> Yeah philosophically I think the argument is that you should capture the
> stream in Kafka independent of the transformation. This is obviously a
> Kafka-centric view point.
>
> Advantages of this:
> - In practice I think this is what e.g. Storm people often end up doing
> anyway. You usually need to throttle any access to a live serving database.
> - Can have multiple subscribers and they get the same thing without
> additional load on the source system.
> - Applications can tap into the stream if need be by subscribing.
> - You can debug your transformation by tailing the Kafka topic with the
> console consumer
> - Can tee off the same data stream for batch analysis or Lambda arch style
> re-processing
>
> The disadvantage is that it will use Kafka resources. But the idea is
> eventually you will have multiple subscribers to any data source (at least
> for monitoring) so you will end up there soon enough anyway.
>
> Down the road the technical benefit is that I think it gives us a good path
> towards end-to-end exactly once semantics from source to destination.
> Basically the connectors need to support idempotence when talking to Kafka
> and we need the transactional write feature in Kafka to make the
> transformation atomic. This is actually pretty doable if you separate
> connector=>kafka problem from the generic transformations which are always
> kafka=>kafka. However I think it is quite impossible to do in a all_things
> => all_things environment. Today you can say "well the semantics of the
> Samza APIs depend on the connectors you use" but it is actually worse then
> that because the semantics actually depend on the pairing of connectors--so
> not only can you probably not get a usable "exactly once" guarantee
> end-to-end it can actually be quite hard to reverse engineer what property
> (if any) your end-to-end flow has if you have heterogenous systems.
>
> -Jay
>
> On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com> wrote:
>
> > {quote}
> > maintained in a separate repository and retaining the existing
> > committership but sharing as much else as possible (website, etc)
> > {quote}
> >
> > Overall, I agree on this idea. Now the question is more about "how to do
> > it".
> >
> > On the other hand, one thing I want to point out is that, if we decide to
> > go this way, how do we want to support
> > otherSystem-transformation-otherSystem use case?
> >
> > Basically, there are four user groups here:
> >
> > 1. Kafka-transformation-Kafka
> > 2. Kafka-transformation-otherSystem
> > 3. otherSystem-transformation-Kafka
> > 4. otherSystem-transformation-otherSystem
> >
> > For group 1, they can easily use the new Samza library to achieve. For
> > group 2 and 3, they can use copyCat -> transformation -> Kafka or Kafka->
> > transformation -> copyCat.
> >
> > The problem is for group 4. Do we want to abandon this or still support
> it?
> > Of course, this use case can be achieved by using copyCat ->
> transformation
> > -> Kafka -> transformation -> copyCat, the thing is how we persuade them
> to
> > do this long chain. If yes, it will also be a win for Kafka too. Or if
> > there is no one in this community actually doing this so far, maybe ok to
> > not support the group 4 directly.
> >
> > Thanks,
> >
> > Fang, Yan
> > yanfang724@gmail.com
> >
> > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Yeah I agree with this summary. I think there are kind of two questions
> > > here:
> > > 1. Technically does alignment/reliance on Kafka make sense
> > > 2. Branding wise (naming, website, concepts, etc) does alignment with
> > Kafka
> > > make sense
> > >
> > > Personally I do think both of these things would be really valuable,
> and
> > > would dramatically alter the trajectory of the project.
> > >
> > > My preference would be to see if people can mostly agree on a direction
> > > rather than splintering things off. From my point of view the ideal
> > outcome
> > > of all the options discussed would be to make Samza a closely aligned
> > > subproject, maintained in a separate repository and retaining the
> > existing
> > > committership but sharing as much else as possible (website, etc). No
> > idea
> > > about how these things work, Jacob, you probably know more.
> > >
> > > No discussion amongst the Kafka folks has happened on this, but likely
> we
> > > should figure out what the Samza community actually wants first.
> > >
> > > I admit that this is a fairly radical departure from how things are.
> > >
> > > If that doesn't fly, I think, yeah we could leave Samza as it is and do
> > the
> > > more radical reboot inside Kafka. From my point of view that does leave
> > > things in a somewhat confusing state since now there are two stream
> > > processing systems more or less coupled to Kafka in large part made by
> > the
> > > same people. But, arguably that might be a cleaner way to make the
> > cut-over
> > > and perhaps less risky for Samza community since if it works people can
> > > switch and if it doesn't nothing will have changed. Dunno, how do
> people
> > > feel about this?
> > >
> > > -Jay
> > >
> > > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com>
> wrote:
> > >
> > > > >  This leads me to thinking that merging projects and communities
> > might
> > > > be a good idea: with the union of experience from both communities,
> we
> > > will
> > > > probably build a better system that is better for users.
> > > > Is this what's being proposed though? Merging the projects seems like
> > > > a consequence of at most one of the three directions under
> discussion:
> > > > 1) Samza 2.0: The Samza community relies more heavily on Kafka for
> > > > configuration, etc. (to a greater or lesser extent to be determined)
> > > > but the Samza community would not automatically merge withe Kafka
> > > > community (the Phoenix/HBase example is a good one here).
> > > > 2) Samza Reboot: The Samza community continues to exist with a
> limited
> > > > project scope, but similarly would not need to be part of the Kafka
> > > > community (ie given committership) to progress.  Here, maybe the
> Samza
> > > > team would become a subproject of Kafka (the Board frowns on
> > > > subprojects at the moment, so I'm not sure if that's even feasible),
> > > > but that would not be required.
> > > > 3) Hey Samza! FYI, Kafka does streaming now: In this option the Kafka
> > > > team builds its own streaming library, possibly off of Jay's
> > > > prototype, which has not direct lineage to the Samza team.  There's
> no
> > > > reason for the Kafka team to bring in the Samza team.
> > > >
> > > > Is the Kafka community on board with this?
> > > >
> > > > To be clear, all three options under discussion are interesting,
> > > > technically valid and likely healthy directions for the project.
> > > > Also, they are not mutually exclusive.  The Samza community could
> > > > decide to pursue, say, 'Samza 2.0', while the Kafka community went
> > > > forward with 'Hey Samza!'  My points above are directed entirely at
> > > > the community aspect of these choices.
> > > > -Jakob
> > > >
> > > > On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com>
> wrote:
> > > > > That's great.  Thanks, Jay.
> > > > >
> > > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io>
> wrote:
> > > > >
> > > > >> Yeah totally agree. I think you have this issue even today, right?
> > > I.e.
> > > > if
> > > > >> you need to make a simple config change and you're running in YARN
> > > today
> > > > >> you end up bouncing the job which then rebuilds state. I think the
> > fix
> > > > is
> > > > >> exactly what you described which is to have a long timeout on
> > > partition
> > > > >> movement for stateful jobs so that if a job is just getting
> bounced,
> > > and
> > > > >> the cluster manager (or admin) is smart enough to restart it on
> the
> > > same
> > > > >> host when possible, it can optimistically reuse any existing state
> > it
> > > > finds
> > > > >> on disk (if it is valid).
> > > > >>
> > > > >> So in this model the charter of the CM is to place processes as
> > > > stickily as
> > > > >> possible and to restart or re-place failed processes. The charter
> of
> > > the
> > > > >> partition management system is to control the assignment of work
> to
> > > > these
> > > > >> processes. The nice thing about this is that the work assignment,
> > > > timeouts,
> > > > >> behavior, configs, and code will all be the same across all
> cluster
> > > > >> managers.
> > > > >>
> > > > >> So I think that prototype would actually give you exactly what you
> > > want
> > > > >> today for any cluster manager (or manual placement + restart
> script)
> > > > that
> > > > >> was sticky in terms of host placement since there is already a
> > > > configurable
> > > > >> partition movement timeout and task-by-task state reuse with a
> check
> > > on
> > > > >> state validity.
> > > > >>
> > > > >> -Jay
> > > > >>
> > > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> > roger.hoover@gmail.com
> > > >
> > > > >> wrote:
> > > > >>
> > > > >> > That would be great to let Kafka do as much heavy lifting as
> > > possible
> > > > and
> > > > >> > make it easier for other languages to implement Samza apis.
> > > > >> >
> > > > >> > One thing to watch out for is the interplay between Kafka's
> group
> > > > >> > management and the external scheduler/process manager's fault
> > > > tolerance.
> > > > >> > If a container dies, the Kafka group membership protocol will
> try
> > to
> > > > >> assign
> > > > >> > it's tasks to other containers while at the same time the
> process
> > > > manager
> > > > >> > is trying to relaunch the container.  Without some consideration
> > for
> > > > this
> > > > >> > (like a configurable amount of time to wait before Kafka alters
> > the
> > > > group
> > > > >> > membership), there may be thrashing going on which is especially
> > bad
> > > > for
> > > > >> > containers with large amounts of local state.
> > > > >> >
> > > > >> > Someone else pointed this out already but I thought it might be
> > > worth
> > > > >> > calling out again.
> > > > >> >
> > > > >> > Cheers,
> > > > >> >
> > > > >> > Roger
> > > > >> >
> > > > >> >
> > > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <ja...@confluent.io>
> > > wrote:
> > > > >> >
> > > > >> > > Hey Roger,
> > > > >> > >
> > > > >> > > I couldn't agree more. We spent a bunch of time talking to
> > people
> > > > and
> > > > >> > that
> > > > >> > > is exactly the stuff we heard time and again. What makes it
> > hard,
> > > of
> > > > >> > > course, is that there is some tension between compatibility
> with
> > > > what's
> > > > >> > > there now and making things better for new users.
> > > > >> > >
> > > > >> > > I also strongly agree with the importance of multi-language
> > > > support. We
> > > > >> > are
> > > > >> > > talking now about Java, but for application development use
> > cases
> > > > >> people
> > > > >> > > want to work in whatever language they are using elsewhere. I
> > > think
> > > > >> > moving
> > > > >> > > to a model where Kafka itself does the group membership,
> > lifecycle
> > > > >> > control,
> > > > >> > > and partition assignment has the advantage of putting all that
> > > > complex
> > > > >> > > stuff behind a clean api that the clients are already going to
> > be
> > > > >> > > implementing for their consumer, so the added functionality
> for
> > > > stream
> > > > >> > > processing beyond a consumer becomes very minor.
> > > > >> > >
> > > > >> > > -Jay
> > > > >> > >
> > > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > > > roger.hoover@gmail.com>
> > > > >> > > wrote:
> > > > >> > >
> > > > >> > > > Metamorphosis...nice. :)
> > > > >> > > >
> > > > >> > > > This has been a great discussion.  As a user of Samza who's
> > > > recently
> > > > >> > > > integrated it into a relatively large organization, I just
> > want
> > > to
> > > > >> add
> > > > >> > > > support to a few points already made.
> > > > >> > > >
> > > > >> > > > The biggest hurdles to adoption of Samza as it currently
> > exists
> > > > that
> > > > >> > I've
> > > > >> > > > experienced are:
> > > > >> > > > 1) YARN - YARN is overly complex in many environments where
> > > Puppet
> > > > >> > would
> > > > >> > > do
> > > > >> > > > just fine but it was the only mechanism to get fault
> > tolerance.
> > > > >> > > > 2) Configuration - I think I like the idea of configuring
> most
> > > of
> > > > the
> > > > >> > job
> > > > >> > > > in code rather than config files.  In general, I think the
> > goal
> > > > >> should
> > > > >> > be
> > > > >> > > > to make it harder to make mistakes, especially of the kind
> > where
> > > > the
> > > > >> > code
> > > > >> > > > expects something and the config doesn't match.  The current
> > > > config
> > > > >> is
> > > > >> > > > quite intricate and error-prone.  For example, the
> application
> > > > logic
> > > > >> > may
> > > > >> > > > depend on bootstrapping a topic but rather than asserting
> that
> > > in
> > > > the
> > > > >> > > code,
> > > > >> > > > you have to rely on getting the config right.  Likewise with
> > > > serdes,
> > > > >> > the
> > > > >> > > > Java representations produced by various serdes (JSON, Avro,
> > > etc.)
> > > > >> are
> > > > >> > > not
> > > > >> > > > equivalent so you cannot just reconfigure a serde without
> > > changing
> > > > >> the
> > > > >> > > > code.   It would be nice for jobs to be able to assert what
> > they
> > > > >> expect
> > > > >> > > > from their input topics in terms of partitioning.  This is
> > > > getting a
> > > > >> > > little
> > > > >> > > > off topic but I was even thinking about creating a "Samza
> > config
> > > > >> > linter"
> > > > >> > > > that would sanity check a set of configs.  Especially in
> > > > >> organizations
> > > > >> > > > where config is managed by a different team than the
> > application
> > > > >> > > developer,
> > > > >> > > > it's very hard to get avoid config mistakes.
> > > > >> > > > 3) Java/Scala centric - for many teams (especially
> DevOps-type
> > > > >> folks),
> > > > >> > > the
> > > > >> > > > pain of the Java toolchain (maven, slow builds, weak command
> > > line
> > > > >> > > support,
> > > > >> > > > configuration over convention) really inhibits productivity.
> > As
> > > > more
> > > > >> > and
> > > > >> > > > more high-quality clients become available for Kafka, I hope
> > > > they'll
> > > > >> > > follow
> > > > >> > > > Samza's model.  Not sure how much it affects the proposals
> in
> > > this
> > > > >> > thread
> > > > >> > > > but please consider other languages in the ecosystem as
> well.
> > > > From
> > > > >> > what
> > > > >> > > > I've heard, Spark has more Python users than Java/Scala.
> > > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > > > >> > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > > >> > > > and are working on a Yeoman generator
> > > > >> > > > https://github.com/Quantiply/generator-rico for
> Jython/Samza
> > > > >> projects
> > > > >> > to
> > > > >> > > > alleviate some of the pain)
> > > > >> > > >
> > > > >> > > > I also want to underscore Jay's point about improving the
> user
> > > > >> > > experience.
> > > > >> > > > That's a very important factor for adoption.  I think the
> goal
> > > > should
> > > > >> > be
> > > > >> > > to
> > > > >> > > > make Samza as easy to get started with as something like
> > > Logstash.
> > > > >> > > > Logstash is vastly inferior in terms of capabilities to
> Samza
> > > but
> > > > >> it's
> > > > >> > > easy
> > > > >> > > > to get started and that makes a big difference.
> > > > >> > > >
> > > > >> > > > Cheers,
> > > > >> > > >
> > > > >> > > > Roger
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci
> > Morales <
> > > > >> > > > gdfm@apache.org> wrote:
> > > > >> > > >
> > > > >> > > > > Forgot to add. On the naming issues, Kafka Metamorphosis
> is
> > a
> > > > clear
> > > > >> > > > winner
> > > > >> > > > > :)
> > > > >> > > > >
> > > > >> > > > > --
> > > > >> > > > > Gianmarco
> > > > >> > > > >
> > > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
> > > > >> > > gdfm@apache.org
> > > > >> > > > >
> > > > >> > > > > wrote:
> > > > >> > > > >
> > > > >> > > > > > Hi,
> > > > >> > > > > >
> > > > >> > > > > > @Martin, thanks for you comments.
> > > > >> > > > > > Maybe I'm missing some important point, but I think
> > coupling
> > > > the
> > > > >> > > > releases
> > > > >> > > > > > is actually a *good* thing.
> > > > >> > > > > > To make an example, would it be better if the MR and
> HDFS
> > > > >> > components
> > > > >> > > of
> > > > >> > > > > > Hadoop had different release schedules?
> > > > >> > > > > >
> > > > >> > > > > > Actually, keeping the discussion in a single place would
> > > make
> > > > >> > > agreeing
> > > > >> > > > on
> > > > >> > > > > > releases (and backwards compatibility) much easier, as
> > > > everybody
> > > > >> > > would
> > > > >> > > > be
> > > > >> > > > > > responsible for the whole codebase.
> > > > >> > > > > >
> > > > >> > > > > > That said, I like the idea of absorbing samza-core as a
> > > > >> > sub-project,
> > > > >> > > > and
> > > > >> > > > > > leave the fancy stuff separate.
> > > > >> > > > > > It probably gives 90% of the benefits we have been
> > > discussing
> > > > >> here.
> > > > >> > > > > >
> > > > >> > > > > > Cheers,
> > > > >> > > > > >
> > > > >> > > > > > --
> > > > >> > > > > > Gianmarco
> > > > >> > > > > >
> > > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <jay.kreps@gmail.com
> >
> > > > wrote:
> > > > >> > > > > >
> > > > >> > > > > >> Hey Martin,
> > > > >> > > > > >>
> > > > >> > > > > >> I agree coupling release schedules is a downside.
> > > > >> > > > > >>
> > > > >> > > > > >> Definitely we can try to solve some of the integration
> > > > problems
> > > > >> in
> > > > >> > > > > >> Confluent Platform or in other distributions. But I
> think
> > > > this
> > > > >> > ends
> > > > >> > > up
> > > > >> > > > > >> being really shallow. I guess I feel to really get a
> good
> > > > user
> > > > >> > > > > experience
> > > > >> > > > > >> the two systems have to kind of feel like part of the
> > same
> > > > thing
> > > > >> > and
> > > > >> > > > you
> > > > >> > > > > >> can't really add that in later--you can put both in the
> > > same
> > > > >> > > > > downloadable
> > > > >> > > > > >> tar file but it doesn't really give a very cohesive
> > > feeling.
> > > > I
> > > > >> > agree
> > > > >> > > > > that
> > > > >> > > > > >> ultimately any of the project stuff is as much social
> and
> > > > naming
> > > > >> > as
> > > > >> > > > > >> anything else--theoretically two totally independent
> > > projects
> > > > >> > could
> > > > >> > > > work
> > > > >> > > > > >> to
> > > > >> > > > > >> tightly align. In practice this seems to be quite
> > difficult
> > > > >> > though.
> > > > >> > > > > >>
> > > > >> > > > > >> For the frameworks--totally agree it would be good to
> > > > maintain
> > > > >> the
> > > > >> > > > > >> framework support with the project. In some cases there
> > may
> > > > not
> > > > >> be
> > > > >> > > too
> > > > >> > > > > >> much
> > > > >> > > > > >> there since the integration gets lighter but I think
> > > whatever
> > > > >> > stubs
> > > > >> > > > you
> > > > >> > > > > >> need should be included. So no I definitely wasn't
> trying
> > > to
> > > > >> imply
> > > > >> > > > > >> dropping
> > > > >> > > > > >> support for these frameworks, just making the
> integration
> > > > >> lighter
> > > > >> > by
> > > > >> > > > > >> separating process management from partition
> management.
> > > > >> > > > > >>
> > > > >> > > > > >> You raise two good points we would have to figure out
> if
> > we
> > > > went
> > > > >> > > down
> > > > >> > > > > the
> > > > >> > > > > >> alignment path:
> > > > >> > > > > >> 1. With respect to the name, yeah I think the first
> > > question
> > > > is
> > > > >> > > > whether
> > > > >> > > > > >> some "re-branding" would be worth it. If so then I
> think
> > we
> > > > can
> > > > >> > > have a
> > > > >> > > > > big
> > > > >> > > > > >> thread on the name. I'm definitely not set on Kafka
> > > > Streaming or
> > > > >> > > Kafka
> > > > >> > > > > >> Streams I was just using them to be kind of
> > illustrative. I
> > > > >> agree
> > > > >> > > with
> > > > >> > > > > >> your
> > > > >> > > > > >> critique of these names, though I think people would
> get
> > > the
> > > > >> idea.
> > > > >> > > > > >> 2. Yeah you also raise a good point about how to
> "factor"
> > > it.
> > > > >> Here
> > > > >> > > are
> > > > >> > > > > the
> > > > >> > > > > >> options I see (I could get enthusiastic about any of
> > them):
> > > > >> > > > > >>    a. One repo for both Kafka and Samza
> > > > >> > > > > >>    b. Two repos, retaining the current seperation
> > > > >> > > > > >>    c. Two repos, the equivalent of samza-api and
> > samza-core
> > > > is
> > > > >> > > > absorbed
> > > > >> > > > > >> almost like a third client
> > > > >> > > > > >>
> > > > >> > > > > >> Cheers,
> > > > >> > > > > >>
> > > > >> > > > > >> -Jay
> > > > >> > > > > >>
> > > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> > > > >> > > > martin@kleppmann.com>
> > > > >> > > > > >> wrote:
> > > > >> > > > > >>
> > > > >> > > > > >> > Ok, thanks for the clarifications. Just a few
> follow-up
> > > > >> > comments.
> > > > >> > > > > >> >
> > > > >> > > > > >> > - I see the appeal of merging with Kafka or becoming
> a
> > > > >> > subproject:
> > > > >> > > > the
> > > > >> > > > > >> > reasons you mention are good. The risk I see is that
> > > > release
> > > > >> > > > schedules
> > > > >> > > > > >> > become coupled to each other, which can slow everyone
> > > down,
> > > > >> and
> > > > >> > > > large
> > > > >> > > > > >> > projects with many contributors are harder to manage.
> > > > (Jakob,
> > > > >> > can
> > > > >> > > > you
> > > > >> > > > > >> speak
> > > > >> > > > > >> > from experience, having seen a wider range of Hadoop
> > > > ecosystem
> > > > >> > > > > >> projects?)
> > > > >> > > > > >> >
> > > > >> > > > > >> > Some of the goals of a better unified developer
> > > experience
> > > > >> could
> > > > >> > > > also
> > > > >> > > > > be
> > > > >> > > > > >> > solved by integrating Samza nicely into a Kafka
> > > > distribution
> > > > >> > (such
> > > > >> > > > as
> > > > >> > > > > >> > Confluent's). I'm not against merging projects if we
> > > decide
> > > > >> > that's
> > > > >> > > > the
> > > > >> > > > > >> way
> > > > >> > > > > >> > to go, just pointing out the same goals can perhaps
> > also
> > > be
> > > > >> > > achieved
> > > > >> > > > > in
> > > > >> > > > > >> > other ways.
> > > > >> > > > > >> >
> > > > >> > > > > >> > - With regard to dropping the YARN dependency: are
> you
> > > > >> proposing
> > > > >> > > > that
> > > > >> > > > > >> > Samza doesn't give any help to people wanting to run
> on
> > > > >> > > > > >> YARN/Mesos/AWS/etc?
> > > > >> > > > > >> > So the docs would basically have a link to Slider and
> > > > nothing
> > > > >> > > else?
> > > > >> > > > Or
> > > > >> > > > > >> > would we maintain integrations with a bunch of
> popular
> > > > >> > deployment
> > > > >> > > > > >> methods
> > > > >> > > > > >> > (e.g. the necessary glue and shell scripts to make
> > Samza
> > > > work
> > > > >> > with
> > > > >> > > > > >> Slider)?
> > > > >> > > > > >> >
> > > > >> > > > > >> > I absolutely think it's a good idea to have the "as a
> > > > library"
> > > > >> > and
> > > > >> > > > > "as a
> > > > >> > > > > >> > process" (using Yi's taxonomy) options for people who
> > > want
> > > > >> them,
> > > > >> > > > but I
> > > > >> > > > > >> > think there should also be a low-friction path for
> > common
> > > > "as
> > > > >> a
> > > > >> > > > > service"
> > > > >> > > > > >> > deployment methods, for which we probably need to
> > > maintain
> > > > >> > > > > integrations.
> > > > >> > > > > >> >
> > > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to me,
> > > because
> > > > >> Kafka
> > > > >> > > is
> > > > >> > > > > all
> > > > >> > > > > >> > about streams already. Perhaps "Kafka Transformers"
> or
> > > > "Kafka
> > > > >> > > > Filters"
> > > > >> > > > > >> > would be more apt?
> > > > >> > > > > >> >
> > > > >> > > > > >> > One suggestion: perhaps the core of Samza (stream
> > > > >> transformation
> > > > >> > > > with
> > > > >> > > > > >> > state management -- i.e. the "Samza as a library"
> bit)
> > > > could
> > > > >> > > become
> > > > >> > > > > >> part of
> > > > >> > > > > >> > Kafka, while higher-level tools such as streaming SQL
> > and
> > > > >> > > > integrations
> > > > >> > > > > >> with
> > > > >> > > > > >> > deployment frameworks remain in a separate project?
> In
> > > > other
> > > > >> > > words,
> > > > >> > > > > >> Kafka
> > > > >> > > > > >> > would absorb the proven, stable core of Samza, which
> > > would
> > > > >> > become
> > > > >> > > > the
> > > > >> > > > > >> > "third Kafka client" mentioned early in this thread.
> > The
> > > > Samza
> > > > >> > > > project
> > > > >> > > > > >> > would then target that third Kafka client as its base
> > > API,
> > > > and
> > > > >> > the
> > > > >> > > > > >> project
> > > > >> > > > > >> > would be freed up to explore more experimental new
> > > > horizons.
> > > > >> > > > > >> >
> > > > >> > > > > >> > Martin
> > > > >> > > > > >> >
> > > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> > jay.kreps@gmail.com>
> > > > >> wrote:
> > > > >> > > > > >> >
> > > > >> > > > > >> > > Hey Martin,
> > > > >> > > > > >> > >
> > > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually don't
> > > think
> > > > it
> > > > >> > ties
> > > > >> > > > our
> > > > >> > > > > >> > hands
> > > > >> > > > > >> > > at all, all it does is refactor things. The
> division
> > of
> > > > >> > > > > >> responsibility is
> > > > >> > > > > >> > > that Samza core is responsible for task lifecycle,
> > > state,
> > > > >> and
> > > > >> > > > > >> partition
> > > > >> > > > > >> > > management (using the Kafka co-ordinator) but it is
> > NOT
> > > > >> > > > responsible
> > > > >> > > > > >> for
> > > > >> > > > > >> > > packaging, configuration deployment or execution of
> > > > >> processes.
> > > > >> > > The
> > > > >> > > > > >> > problem
> > > > >> > > > > >> > > of packaging and starting these processes is
> > > > >> > > > > >> > > framework/environment-specific. This leaves
> > individual
> > > > >> > > frameworks
> > > > >> > > > to
> > > > >> > > > > >> be
> > > > >> > > > > >> > as
> > > > >> > > > > >> > > fancy or vanilla as they like. So you can get
> simple
> > > > >> stateless
> > > > >> > > > > >> support in
> > > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app
> > > framework
> > > > >> > > (Slider,
> > > > >> > > > > >> > Marathon,
> > > > >> > > > > >> > > etc). These are well known by people and have nice
> > UIs
> > > > and a
> > > > >> > lot
> > > > >> > > > of
> > > > >> > > > > >> > > flexibility. I don't think they have node affinity
> > as a
> > > > >> built
> > > > >> > in
> > > > >> > > > > >> option
> > > > >> > > > > >> > > (though I could be wrong). So if we want that we
> can
> > > > either
> > > > >> > wait
> > > > >> > > > for
> > > > >> > > > > >> them
> > > > >> > > > > >> > > to add it or do a custom framework to add that
> > feature
> > > > (as
> > > > >> > now).
> > > > >> > > > > >> > Obviously
> > > > >> > > > > >> > > if you manage things with old-school ops tools
> > > > >> > (puppet/chef/etc)
> > > > >> > > > you
> > > > >> > > > > >> get
> > > > >> > > > > >> > > locality easily. The nice thing, though, is that
> all
> > > the
> > > > >> samza
> > > > >> > > > > >> "business
> > > > >> > > > > >> > > logic" around partition management and fault
> > tolerance
> > > > is in
> > > > >> > > Samza
> > > > >> > > > > >> core
> > > > >> > > > > >> > so
> > > > >> > > > > >> > > it is shared across frameworks and the framework
> > > specific
> > > > >> bit
> > > > >> > is
> > > > >> > > > > just
> > > > >> > > > > >> > > whether it is smart enough to try to get the same
> > host
> > > > when
> > > > >> a
> > > > >> > > job
> > > > >> > > > is
> > > > >> > > > > >> > > restarted.
> > > > >> > > > > >> > >
> > > > >> > > > > >> > > With respect to the Kafka-alignment, yeah I think
> the
> > > > goal
> > > > >> > would
> > > > >> > > > be
> > > > >> > > > > >> (a)
> > > > >> > > > > >> > > actually get better alignment in user experience,
> and
> > > (b)
> > > > >> > > express
> > > > >> > > > > >> this in
> > > > >> > > > > >> > > the naming and project branding. Specifically:
> > > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> > > > "transformation"
> > > > >> api
> > > > >> > > to
> > > > >> > > > be
> > > > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be able
> to
> > > > explain
> > > > >> > > when
> > > > >> > > > to
> > > > >> > > > > >> use
> > > > >> > > > > >> > > the consumer and when to use the stream processing
> > > > >> > functionality
> > > > >> > > > and
> > > > >> > > > > >> lead
> > > > >> > > > > >> > > people into that experience.
> > > > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or
> > > > whatever)
> > > > >> > that
> > > > >> > > > has
> > > > >> > > > > >> both
> > > > >> > > > > >> > > Kafka and the stream processing part and they
> > actually
> > > > work
> > > > >> > > > > together.
> > > > >> > > > > >> > > 3. Unify the programming experience so the client
> and
> > > > Samza
> > > > >> > api
> > > > >> > > > > share
> > > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > > > >> > > > > >> > >
> > > > >> > > > > >> > > I think sub-projects keep separate committers and
> can
> > > > have a
> > > > >> > > > > separate
> > > > >> > > > > >> > repo,
> > > > >> > > > > >> > > but I'm actually not really sure (I can't find a
> > > > definition
> > > > >> > of a
> > > > >> > > > > >> > subproject
> > > > >> > > > > >> > > in Apache).
> > > > >> > > > > >> > >
> > > > >> > > > > >> > > Basically at a high-level you want the experience
> to
> > > > "feel"
> > > > >> > > like a
> > > > >> > > > > >> single
> > > > >> > > > > >> > > system, not to relatively independent things that
> are
> > > > kind
> > > > >> of
> > > > >> > > > > >> awkwardly
> > > > >> > > > > >> > > glued together.
> > > > >> > > > > >> > >
> > > > >> > > > > >> > > I think if we did that they having naming or
> branding
> > > > like
> > > > >> > > "kafka
> > > > >> > > > > >> > > streaming" or "kafka streams" or something like
> that
> > > > would
> > > > >> > > > actually
> > > > >> > > > > >> do a
> > > > >> > > > > >> > > good job of conveying what it is. I do that this
> > would
> > > > help
> > > > >> > > > adoption
> > > > >> > > > > >> > quite
> > > > >> > > > > >> > > a lot as it would correctly convey that using Kafka
> > > > >> Streaming
> > > > >> > > with
> > > > >> > > > > >> Kafka
> > > > >> > > > > >> > is
> > > > >> > > > > >> > > a fairly seamless experience and Kafka is pretty
> > > heavily
> > > > >> > adopted
> > > > >> > > > at
> > > > >> > > > > >> this
> > > > >> > > > > >> > > point.
> > > > >> > > > > >> > >
> > > > >> > > > > >> > > Fwiw we actually considered this model originally
> > when
> > > > open
> > > > >> > > > sourcing
> > > > >> > > > > >> > Samza,
> > > > >> > > > > >> > > however at that time Kafka was relatively unknown
> and
> > > we
> > > > >> > decided
> > > > >> > > > not
> > > > >> > > > > >> to
> > > > >> > > > > >> > do
> > > > >> > > > > >> > > it since we felt it would be limiting. From my
> point
> > of
> > > > view
> > > > >> > the
> > > > >> > > > > three
> > > > >> > > > > >> > > things have changed (1) Kafka is now really heavily
> > > used
> > > > for
> > > > >> > > > stream
> > > > >> > > > > >> > > processing, (2) we learned that abstracting out the
> > > > stream
> > > > >> > well
> > > > >> > > is
> > > > >> > > > > >> > > basically impossible, (3) we learned it is really
> > hard
> > > to
> > > > >> keep
> > > > >> > > the
> > > > >> > > > > two
> > > > >> > > > > >> > > things feeling like a single product.
> > > > >> > > > > >> > >
> > > > >> > > > > >> > > -Jay
> > > > >> > > > > >> > >
> > > > >> > > > > >> > >
> > > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> > > > >> > > > > >> martin@kleppmann.com>
> > > > >> > > > > >> > > wrote:
> > > > >> > > > > >> > >
> > > > >> > > > > >> > >> Hi all,
> > > > >> > > > > >> > >>
> > > > >> > > > > >> > >> Lots of good thoughts here.
> > > > >> > > > > >> > >>
> > > > >> > > > > >> > >> I agree with the general philosophy of tying Samza
> > > more
> > > > >> > firmly
> > > > >> > > to
> > > > >> > > > > >> Kafka.
> > > > >> > > > > >> > >> After I spent a while looking at integrating other
> > > > message
> > > > >> > > > brokers
> > > > >> > > > > >> (e.g.
> > > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
> > conclusion
> > > > that
> > > > >> > > > > >> > SystemConsumer
> > > > >> > > > > >> > >> tacitly assumes a model so much like Kafka's that
> > > pretty
> > > > >> much
> > > > >> > > > > nobody
> > > > >> > > > > >> but
> > > > >> > > > > >> > >> Kafka actually implements it. (Databus is perhaps
> an
> > > > >> > exception,
> > > > >> > > > but
> > > > >> > > > > >> it
> > > > >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus,
> making
> > > > Samza
> > > > >> > > fully
> > > > >> > > > > >> > dependent
> > > > >> > > > > >> > >> on Kafka acknowledges that the system-independence
> > was
> > > > >> never
> > > > >> > as
> > > > >> > > > > real
> > > > >> > > > > >> as
> > > > >> > > > > >> > we
> > > > >> > > > > >> > >> perhaps made it out to be. The gains of code reuse
> > are
> > > > >> real.
> > > > >> > > > > >> > >>
> > > > >> > > > > >> > >> The idea of decoupling Samza from YARN has also
> > always
> > > > been
> > > > >> > > > > >> appealing to
> > > > >> > > > > >> > >> me, for various reasons already mentioned in this
> > > > thread.
> > > > >> > > > Although
> > > > >> > > > > >> > making
> > > > >> > > > > >> > >> Samza jobs deployable on anything
> > (YARN/Mesos/AWS/etc)
> > > > >> seems
> > > > >> > > > > >> laudable,
> > > > >> > > > > >> > I am
> > > > >> > > > > >> > >> a little concerned that it will restrict us to a
> > > lowest
> > > > >> > common
> > > > >> > > > > >> > denominator.
> > > > >> > > > > >> > >> For example, would host affinity (SAMZA-617) still
> > be
> > > > >> > possible?
> > > > >> > > > For
> > > > >> > > > > >> jobs
> > > > >> > > > > >> > >> with large amounts of state, I think SAMZA-617
> would
> > > be
> > > > a
> > > > >> big
> > > > >> > > > boon,
> > > > >> > > > > >> > since
> > > > >> > > > > >> > >> restoring state off the changelog on every single
> > > > restart
> > > > >> is
> > > > >> > > > > painful,
> > > > >> > > > > >> > due
> > > > >> > > > > >> > >> to long recovery times. It would be a shame if the
> > > > >> decoupling
> > > > >> > > > from
> > > > >> > > > > >> YARN
> > > > >> > > > > >> > >> made host affinity impossible.
> > > > >> > > > > >> > >>
> > > > >> > > > > >> > >> Jay, a question about the proposed API for
> > > > instantiating a
> > > > >> > job
> > > > >> > > in
> > > > >> > > > > >> code
> > > > >> > > > > >> > >> (rather than a properties file): when submitting a
> > job
> > > > to a
> > > > >> > > > > cluster,
> > > > >> > > > > >> is
> > > > >> > > > > >> > the
> > > > >> > > > > >> > >> idea that the instantiation code runs on a client
> > > > >> somewhere,
> > > > >> > > > which
> > > > >> > > > > >> then
> > > > >> > > > > >> > >> pokes the necessary endpoints on
> YARN/Mesos/AWS/etc?
> > > Or
> > > > >> does
> > > > >> > > that
> > > > >> > > > > >> code
> > > > >> > > > > >> > run
> > > > >> > > > > >> > >> on each container that is part of the job (in
> which
> > > > case,
> > > > >> how
> > > > >> > > > does
> > > > >> > > > > >> the
> > > > >> > > > > >> > job
> > > > >> > > > > >> > >> submission to the cluster work)?
> > > > >> > > > > >> > >>
> > > > >> > > > > >> > >> I agree with Garry that it doesn't feel right to
> > make
> > > a
> > > > 1.0
> > > > >> > > > release
> > > > >> > > > > >> > with a
> > > > >> > > > > >> > >> plan for it to be immediately obsolete. So if this
> > is
> > > > going
> > > > >> > to
> > > > >> > > > > >> happen, I
> > > > >> > > > > >> > >> think it would be more honest to stick with 0.*
> > > version
> > > > >> > numbers
> > > > >> > > > > until
> > > > >> > > > > >> > the
> > > > >> > > > > >> > >> library-ified Samza has been implemented, is
> stable
> > > and
> > > > >> > widely
> > > > >> > > > > used.
> > > > >> > > > > >> > >>
> > > > >> > > > > >> > >> Should the new Samza be a subproject of Kafka?
> There
> > > is
> > > > >> > > precedent
> > > > >> > > > > for
> > > > >> > > > > >> > >> tight coupling between different Apache projects
> > (e.g.
> > > > >> > Curator
> > > > >> > > > and
> > > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think
> remaining
> > > > >> separate
> > > > >> > > > would
> > > > >> > > > > >> be
> > > > >> > > > > >> > ok.
> > > > >> > > > > >> > >> Even if Samza is fully dependent on Kafka, there
> is
> > > > enough
> > > > >> > > > > substance
> > > > >> > > > > >> in
> > > > >> > > > > >> > >> Samza that it warrants being a separate project.
> An
> > > > >> argument
> > > > >> > in
> > > > >> > > > > >> favour
> > > > >> > > > > >> > of
> > > > >> > > > > >> > >> merging would be if we think Kafka has a much
> > stronger
> > > > >> "brand
> > > > >> > > > > >> presence"
> > > > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If the
> Kafka
> > > > >> project
> > > > >> > is
> > > > >> > > > > >> willing
> > > > >> > > > > >> > to
> > > > >> > > > > >> > >> endorse Samza as the "official" way of doing
> > stateful
> > > > >> stream
> > > > >> > > > > >> > >> transformations, that would probably have much the
> > > same
> > > > >> > effect
> > > > >> > > as
> > > > >> > > > > >> > >> re-branding Samza as "Kafka Stream Processors" or
> > > > suchlike.
> > > > >> > > Close
> > > > >> > > > > >> > >> collaboration between the two projects will be
> > needed
> > > in
> > > > >> any
> > > > >> > > > case.
> > > > >> > > > > >> > >>
> > > > >> > > > > >> > >> From a project management perspective, I guess the
> > > "new
> > > > >> > Samza"
> > > > >> > > > > would
> > > > >> > > > > >> > have
> > > > >> > > > > >> > >> to be developed on a branch alongside ongoing
> > > > maintenance
> > > > >> of
> > > > >> > > the
> > > > >> > > > > >> current
> > > > >> > > > > >> > >> line of development? I think it would be important
> > to
> > > > >> > continue
> > > > >> > > > > >> > supporting
> > > > >> > > > > >> > >> existing users, and provide a graceful migration
> > path
> > > to
> > > > >> the
> > > > >> > > new
> > > > >> > > > > >> > version.
> > > > >> > > > > >> > >> Leaving the current versions unsupported and
> forcing
> > > > people
> > > > >> > to
> > > > >> > > > > >> rewrite
> > > > >> > > > > >> > >> their jobs would send a bad signal.
> > > > >> > > > > >> > >>
> > > > >> > > > > >> > >> Best,
> > > > >> > > > > >> > >> Martin
> > > > >> > > > > >> > >>
> > > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> > jay@confluent.io>
> > > > >> wrote:
> > > > >> > > > > >> > >>
> > > > >> > > > > >> > >>> Hey Garry,
> > > > >> > > > > >> > >>>
> > > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy to
> chat
> > > > more
> > > > >> > about
> > > > >> > > > > this
> > > > >> > > > > >> if
> > > > >> > > > > >> > >>> you'd be interested. I think Chris and I started
> > with
> > > > the
> > > > >> > idea
> > > > >> > > > of
> > > > >> > > > > >> "what
> > > > >> > > > > >> > >>> would it take to make Samza a kick-ass ingestion
> > > tool"
> > > > but
> > > > >> > > > > >> ultimately
> > > > >> > > > > >> > we
> > > > >> > > > > >> > >>> kind of came around to the idea that ingestion
> and
> > > > >> > > > transformation
> > > > >> > > > > >> had
> > > > >> > > > > >> > >>> pretty different needs and coupling the two made
> > > things
> > > > >> > hard.
> > > > >> > > > > >> > >>>
> > > > >> > > > > >> > >>> For what it's worth I think copycat (KIP-26)
> > actually
> > > > will
> > > > >> > do
> > > > >> > > > what
> > > > >> > > > > >> you
> > > > >> > > > > >> > >> are
> > > > >> > > > > >> > >>> looking for.
> > > > >> > > > > >> > >>>
> > > > >> > > > > >> > >>> With regard to your point about slider, I don't
> > > > >> necessarily
> > > > >> > > > > >> disagree.
> > > > >> > > > > >> > >> But I
> > > > >> > > > > >> > >>> think getting good YARN support is quite doable
> > and I
> > > > >> think
> > > > >> > we
> > > > >> > > > can
> > > > >> > > > > >> make
> > > > >> > > > > >> > >>> that work well. I think the issue this proposal
> > > solves
> > > > is
> > > > >> > that
> > > > >> > > > > >> > >> technically
> > > > >> > > > > >> > >>> it is pretty hard to support multiple cluster
> > > > management
> > > > >> > > systems
> > > > >> > > > > the
> > > > >> > > > > >> > way
> > > > >> > > > > >> > >>> things are now, you need to write an "app master"
> > or
> > > > >> > > "framework"
> > > > >> > > > > for
> > > > >> > > > > >> > each
> > > > >> > > > > >> > >>> and they are all a little different so testing is
> > > > really
> > > > >> > hard.
> > > > >> > > > In
> > > > >> > > > > >> the
> > > > >> > > > > >> > >>> absence of this we have been stuck with just YARN
> > > which
> > > > >> has
> > > > >> > > > > >> fantastic
> > > > >> > > > > >> > >>> penetration in the Hadoopy part of the org, but
> > zero
> > > > >> > > penetration
> > > > >> > > > > >> > >> elsewhere.
> > > > >> > > > > >> > >>> Given the huge amount of work being put in to
> > slider,
> > > > >> > > marathon,
> > > > >> > > > > aws
> > > > >> > > > > >> > >>> tooling, not to mention the umpteen related
> > packaging
> > > > >> > > > technologies
> > > > >> > > > > >> > people
> > > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> > > cloud-specific
> > > > >> > deploy
> > > > >> > > > > >> tools,
> > > > >> > > > > >> > >> etc)
> > > > >> > > > > >> > >>> I really think it is important to get this right.
> > > > >> > > > > >> > >>>
> > > > >> > > > > >> > >>> -Jay
> > > > >> > > > > >> > >>>
> > > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington
> <
> > > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > > > >> > > > > >> > >>>
> > > > >> > > > > >> > >>>> Hi all,
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>> I think the question below re does Samza become
> a
> > > > >> > sub-project
> > > > >> > > > of
> > > > >> > > > > >> Kafka
> > > > >> > > > > >> > >>>> highlights the broader point around migration.
> > Chris
> > > > >> > mentions
> > > > >> > > > > >> Samza's
> > > > >> > > > > >> > >>>> maturity is heading towards a v1 release but I'm
> > not
> > > > sure
> > > > >> > it
> > > > >> > > > > feels
> > > > >> > > > > >> > >> right to
> > > > >> > > > > >> > >>>> launch a v1 then immediately plan to deprecate
> > most
> > > of
> > > > >> it.
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>> From a selfish perspective I have some guys who
> > have
> > > > >> > started
> > > > >> > > > > >> working
> > > > >> > > > > >> > >> with
> > > > >> > > > > >> > >>>> Samza and building some new consumers/producers
> > was
> > > > next
> > > > >> > up.
> > > > >> > > > > Sounds
> > > > >> > > > > >> > like
> > > > >> > > > > >> > >>>> that is absolutely not the direction to go. I
> need
> > > to
> > > > >> look
> > > > >> > > into
> > > > >> > > > > the
> > > > >> > > > > >> > KIP
> > > > >> > > > > >> > >> in
> > > > >> > > > > >> > >>>> more detail but for me the attractiveness of
> > adding
> > > > new
> > > > >> > Samza
> > > > >> > > > > >> > >>>> consumer/producers -- even if yes all they were
> > > doing
> > > > was
> > > > >> > > > really
> > > > >> > > > > >> > getting
> > > > >> > > > > >> > >>>> data into and out of Kafka --  was to avoid
> > having
> > > to
> > > > >> > worry
> > > > >> > > > > about
> > > > >> > > > > >> the
> > > > >> > > > > >> > >>>> lifecycle management of external clients. If
> there
> > > is
> > > > a
> > > > >> > > generic
> > > > >> > > > > >> Kafka
> > > > >> > > > > >> > >>>> ingress/egress layer that I can plug a new
> > connector
> > > > into
> > > > >> > and
> > > > >> > > > > have
> > > > >> > > > > >> a
> > > > >> > > > > >> > >> lot of
> > > > >> > > > > >> > >>>> the heavy lifting re scale and reliability done
> > for
> > > me
> > > > >> then
> > > > >> > > it
> > > > >> > > > > >> gives
> > > > >> > > > > >> > me
> > > > >> > > > > >> > >> all
> > > > >> > > > > >> > >>>> the pushing new consumers/producers would. If
> not
> > > > then it
> > > > >> > > > > >> complicates
> > > > >> > > > > >> > my
> > > > >> > > > > >> > >>>> operational deployments.
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>> Which is similar to my other question with the
> > > > proposal
> > > > >> --
> > > > >> > if
> > > > >> > > > we
> > > > >> > > > > >> > build a
> > > > >> > > > > >> > >>>> fully available/stand-alone Samza plus the
> > requisite
> > > > >> shims
> > > > >> > to
> > > > >> > > > > >> > integrate
> > > > >> > > > > >> > >>>> with Slider etc I suspect the former may be a
> lot
> > > more
> > > > >> work
> > > > >> > > > than
> > > > >> > > > > we
> > > > >> > > > > >> > >> think.
> > > > >> > > > > >> > >>>> We may make it much easier for a newcomer to get
> > > > >> something
> > > > >> > > > > running
> > > > >> > > > > >> but
> > > > >> > > > > >> > >>>> having them step up and get a reliable
> production
> > > > >> > deployment
> > > > >> > > > may
> > > > >> > > > > >> still
> > > > >> > > > > >> > >>>> dominate mailing list  traffic, if for different
> > > > reasons
> > > > >> > than
> > > > >> > > > > >> today.
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with
> making
> > > the
> > > > >> Samza
> > > > >> > > > > >> dependency
> > > > >> > > > > >> > >> on
> > > > >> > > > > >> > >>>> Kafka much more explicit and I absolutely see
> the
> > > > >> benefits
> > > > >> > > in
> > > > >> > > > > the
> > > > >> > > > > >> > >>>> reduction of duplication and clashing
> > > > >> > > > terminologies/abstractions
> > > > >> > > > > >> that
> > > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library would
> > likely
> > > > be a
> > > > >> > very
> > > > >> > > > > nice
> > > > >> > > > > >> > tool
> > > > >> > > > > >> > >> to
> > > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the
> > concerns
> > > > >> above
> > > > >> > re
> > > > >> > > > the
> > > > >> > > > > >> > >>>> operational side.
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>> Garry
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>> -----Original Message-----
> > > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
> > > > >> > gdfm@apache.org
> > > > >> > > ]
> > > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>> Very interesting thoughts.
> > > > >> > > > > >> > >>>> From outside, I have always perceived Samza as a
> > > > >> computing
> > > > >> > > > layer
> > > > >> > > > > >> over
> > > > >> > > > > >> > >>>> Kafka.
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>> The question, maybe a bit provocative, is
> "should
> > > > Samza
> > > > >> be
> > > > >> > a
> > > > >> > > > > >> > sub-project
> > > > >> > > > > >> > >>>> of Kafka then?"
> > > > >> > > > > >> > >>>> Or does it make sense to keep it as a separate
> > > project
> > > > >> > with a
> > > > >> > > > > >> separate
> > > > >> > > > > >> > >>>> governance?
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>> Cheers,
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>> --
> > > > >> > > > > >> > >>>> Gianmarco
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > > > yanfang724@gmail.com>
> > > > >> > > > wrote:
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more
> > tightly.
> > > > >> > Because
> > > > >> > > > > Samza
> > > > >> > > > > >> de
> > > > >> > > > > >> > >>>>> facto is based on Kafka, and it should leverage
> > > what
> > > > >> Kafka
> > > > >> > > > has.
> > > > >> > > > > At
> > > > >> > > > > >> > the
> > > > >> > > > > >> > >>>>> same time, Kafka does not need to reinvent what
> > > Samza
> > > > >> > > already
> > > > >> > > > > >> has. I
> > > > >> > > > > >> > >>>>> also like the idea of separating the ingestion
> > and
> > > > >> > > > > transformation.
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>> But it is a little difficult for me to image
> how
> > > the
> > > > >> Samza
> > > > >> > > > will
> > > > >> > > > > >> look
> > > > >> > > > > >> > >>>> like.
> > > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little
> difference
> > > in
> > > > >> terms
> > > > >> > > of
> > > > >> > > > > how
> > > > >> > > > > >> > >>>>> Samza should look like.
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>> *** Will it look like what Jay's code shows (A
> > > > client of
> > > > >> > > > Kakfa)
> > > > >> > > > > ?
> > > > >> > > > > >> And
> > > > >> > > > > >> > >>>>> user's application code calls this client?
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka (like
> > > what
> > > > the
> > > > >> > > code
> > > > >> > > > > >> shows),
> > > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > > fault-tolerance?
> > > > >> Are
> > > > >> > > they
> > > > >> > > > > >> taken
> > > > >> > > > > >> > >>>>> care by the Kafka broker or other mechanism,
> such
> > > as
> > > > >> > "Samza
> > > > >> > > > > >> worker"
> > > > >> > > > > >> > >>>>> (just make up the name) ?
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>> 2. What about other features, such as
> > auto-scaling,
> > > > >> shared
> > > > >> > > > > state,
> > > > >> > > > > >> > >>>>> monitoring?
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is this what
> > > Chris
> > > > >> > > > suggests?)
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa and
> > > > produce
> > > > >> to
> > > > >> > > it.
> > > > >> > > > > >> Then it
> > > > >> > > > > >> > >>>>> becomes the same as what Samza looks like now,
> > > > except it
> > > > >> > > does
> > > > >> > > > > not
> > > > >> > > > > >> > rely
> > > > >> > > > > >> > >>>>> on Yarn anymore.
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>> 2. if it is standalone, how can it leverage
> > Kafka's
> > > > >> > metrics,
> > > > >> > > > > logs,
> > > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>> Thanks,
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>> Fang, Yan
> > > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > > > >> > > > > wangguoz@gmail.com
> > > > >> > > > > >> >
> > > > >> > > > > >> > >>>> wrote:
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>>> Read through the code example and it looks
> good
> > to
> > > > me.
> > > > >> A
> > > > >> > > few
> > > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>> Today Samza deploys as executable runnable
> like:
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh
> --config-factory=...
> > > > >> > > > > >> > >>>> --config-path=file://...
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>> And this proposal advocate for deploying Samza
> > > more
> > > > as
> > > > >> > > > embedded
> > > > >> > > > > >> > >>>>>> libraries in user application code (ignoring
> the
> > > > >> > > terminology
> > > > >> > > > > >> since
> > > > >> > > > > >> > >>>>>> it is not the
> > > > >> > > > > >> > >>>>> same
> > > > >> > > > > >> > >>>>>> as the prototype code):
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>> StreamTask task = new MyStreamTask(configs);
> > > Thread
> > > > >> > thread
> > > > >> > > =
> > > > >> > > > > new
> > > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>> I think both of these deployment modes are
> > > important
> > > > >> for
> > > > >> > > > > >> different
> > > > >> > > > > >> > >>>>>> types
> > > > >> > > > > >> > >>>>> of
> > > > >> > > > > >> > >>>>>> users. That said, I think making Samza purely
> > > > >> standalone
> > > > >> > is
> > > > >> > > > > still
> > > > >> > > > > >> > >>>>>> sufficient for either runnable or library
> modes.
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>> Guozhang
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> > > > >> > > > jay@confluent.io>
> > > > >> > > > > >> > wrote:
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code example, it
> > was
> > > > >> > supposed
> > > > >> > > > to
> > > > >> > > > > >> look
> > > > >> > > > > >> > >>>>>>> like
> > > > >> > > > > >> > >>>>>>> this:
> > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> > > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> > "localhost:4242");
> > > > >> > > > > >> StreamingConfig
> > > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > "test-topic-2");
> > > > >> > > > > >> > >>>>>>>
> config.processor(ExampleStreamProcessor.class);
> > > > >> > > > > >> > >>>>>>> config.serialization(new StringSerializer(),
> > new
> > > > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming
> > container =
> > > > new
> > > > >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
> > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > >> > >>>>>>> -Jay
> > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> > > > >> > > > jay@confluent.io
> > > > >> > > > > >
> > > > >> > > > > >> > >>>> wrote:
> > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > >> > >>>>>>>> Hey guys,
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> This came out of some conversations Chris
> and
> > I
> > > > were
> > > > >> > > having
> > > > >> > > > > >> > >>>>>>>> around
> > > > >> > > > > >> > >>>>>>> whether
> > > > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a kind
> of
> > > data
> > > > >> > > > ingestion
> > > > >> > > > > >> > >>>>> framework
> > > > >> > > > > >> > >>>>>>> for
> > > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26
> > > "copycat").
> > > > >> This
> > > > >> > > > kind
> > > > >> > > > > of
> > > > >> > > > > >> > >>>>>> combined
> > > > >> > > > > >> > >>>>>>>> with complaints around config and YARN and
> the
> > > > >> > discussion
> > > > >> > > > > >> around
> > > > >> > > > > >> > >>>>>>>> how
> > > > >> > > > > >> > >>>>> to
> > > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> So the thought experiment was, given that
> > Samza
> > > > was
> > > > >> > > > basically
> > > > >> > > > > >> > >>>>>>>> already totally Kafka specific, what if you
> > just
> > > > >> > embraced
> > > > >> > > > > that
> > > > >> > > > > >> > >>>>>>>> and turned it
> > > > >> > > > > >> > >>>>>> into
> > > > >> > > > > >> > >>>>>>>> something less like a heavyweight framework
> > and
> > > > more
> > > > >> > > like a
> > > > >> > > > > >> > >>>>>>>> third
> > > > >> > > > > >> > >>>>> Kafka
> > > > >> > > > > >> > >>>>>>>> client--a kind of "producing consumer" with
> > > state
> > > > >> > > > management
> > > > >> > > > > >> > >>>>>> facilities.
> > > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a complex
> > stream
> > > > >> > > processing
> > > > >> > > > > >> > >>>>>>>> framework
> > > > >> > > > > >> > >>>>>>> this
> > > > >> > > > > >> > >>>>>>>> would actually be a very simple thing, not
> > much
> > > > more
> > > > >> > > > > >> complicated
> > > > >> > > > > >> > >>>>>>>> to
> > > > >> > > > > >> > >>>>> use
> > > > >> > > > > >> > >>>>>>> or
> > > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris said
> > we
> > > > >> thought
> > > > >> > > > about
> > > > >> > > > > >> it
> > > > >> > > > > >> > >>>>>>>> a
> > > > >> > > > > >> > >>>>> lot
> > > > >> > > > > >> > >>>>>> of
> > > > >> > > > > >> > >>>>>>>> what Samza (and the other stream processing
> > > > systems
> > > > >> > were
> > > > >> > > > > doing)
> > > > >> > > > > >> > >>>>> seemed
> > > > >> > > > > >> > >>>>>>> like
> > > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output data to
> > and
> > > > from
> > > > >> > the
> > > > >> > > > > stream
> > > > >> > > > > >> > >>>>>>>> processing. But when we actually looked into
> > how
> > > > that
> > > > >> > > would
> > > > >> > > > > >> > >>>>>>>> work,
> > > > >> > > > > >> > >>>>> Samza
> > > > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion
> framework
> > > > for a
> > > > >> > > bunch
> > > > >> > > > of
> > > > >> > > > > >> > >>>>> reasons.
> > > > >> > > > > >> > >>>>>> To
> > > > >> > > > > >> > >>>>>>>> really do that right you need a pretty
> > different
> > > > >> > internal
> > > > >> > > > > data
> > > > >> > > > > >> > >>>>>>>> model
> > > > >> > > > > >> > >>>>>> and
> > > > >> > > > > >> > >>>>>>>> set of apis. So what if you split them and
> had
> > > an
> > > > api
> > > > >> > for
> > > > >> > > > > Kafka
> > > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a
> > > separate
> > > > >> api
> > > > >> > > for
> > > > >> > > > > >> Kafka
> > > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> This would also allow really embracing the
> > same
> > > > >> > > terminology
> > > > >> > > > > and
> > > > >> > > > > >> > >>>>>>>> conventions. One complaint about the current
> > > > state is
> > > > >> > > that
> > > > >> > > > > the
> > > > >> > > > > >> > >>>>>>>> two
> > > > >> > > > > >> > >>>>>>> systems
> > > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like
> > > "stream"
> > > > vs
> > > > >> > > > "topic"
> > > > >> > > > > >> and
> > > > >> > > > > >> > >>>>>>> different
> > > > >> > > > > >> > >>>>>>>> config and monitoring systems means you kind
> > of
> > > > have
> > > > >> to
> > > > >> > > > learn
> > > > >> > > > > >> > >>>>>>>> Kafka's
> > > > >> > > > > >> > >>>>>>> way,
> > > > >> > > > > >> > >>>>>>>> then learn Samza's slightly different way,
> > then
> > > > kind
> > > > >> of
> > > > >> > > > > >> > >>>>>>>> understand
> > > > >> > > > > >> > >>>>> how
> > > > >> > > > > >> > >>>>>>> they
> > > > >> > > > > >> > >>>>>>>> map to each other, which having walked a few
> > > > people
> > > > >> > > through
> > > > >> > > > > >> this
> > > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of time on
> > > > >> airplanes I
> > > > >> > > > > hacked
> > > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat incomplete
> > > > prototype
> > > > >> of
> > > > >> > > > what
> > > > >> > > > > >> > >>>>>>>> this would
> > > > >> > > > > >> > >>>>> look
> > > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously dumped
> into
> > > > Kafka
> > > > >> as
> > > > >> > > it
> > > > >> > > > > >> > >>>>>>>> required a
> > > > >> > > > > >> > >>>>>> few
> > > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is the
> code:
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> >
> > > > >> > > > >
> > > > >> >
> > > >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I just
> > > liberally
> > > > >> > renamed
> > > > >> > > > > >> > >>>>>>>> everything
> > > > >> > > > > >> > >>>>> to
> > > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no regard
> for
> > > > >> > > > compatibility.
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> To use this would be something like this:
> > > > >> > > > > >> > >>>>>>>> Properties props = new Properties();
> > > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > > "localhost:4242");
> > > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > > > >> > > > > >> > >>>>> StreamingConfig(props);
> > > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > > > >> > > > > >> config.processor(ExampleStreamProcessor.class);
> > > > >> > > > > >> > >>>>>>> config.serialization(new
> > > > >> > > > > >> > >>>>>>>> StringSerializer(), new
> StringDeserializer());
> > > > >> > > > KafkaStreaming
> > > > >> > > > > >> > >>>>>> container =
> > > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> > SamzaContainer;
> > > > >> > > > > StreamProcessor
> > > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> So rather than putting all the class names
> in
> > a
> > > > file
> > > > >> > and
> > > > >> > > > then
> > > > >> > > > > >> > >>>>>>>> having
> > > > >> > > > > >> > >>>>>> the
> > > > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
> > > instantiate
> > > > the
> > > > >> > > > > container
> > > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced over
> > however
> > > > many
> > > > >> > > > > instances
> > > > >> > > > > >> > >>>>>>>> of
> > > > >> > > > > >> > >>>>> this
> > > > >> > > > > >> > >>>>>>> are
> > > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance dies,
> > new
> > > > >> tasks
> > > > >> > > are
> > > > >> > > > > >> added
> > > > >> > > > > >> > >>>>>>>> to
> > > > >> > > > > >> > >>>>> the
> > > > >> > > > > >> > >>>>>>>> existing containers without shutting them
> > down).
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> We would provide some glue for running this
> > > stuff
> > > > in
> > > > >> > YARN
> > > > >> > > > via
> > > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using
> some
> > > of
> > > > >> their
> > > > >> > > > tools
> > > > >> > > > > >> > >>>>>>>> but from the
> > > > >> > > > > >> > >>>>>> point
> > > > >> > > > > >> > >>>>>>> of
> > > > >> > > > > >> > >>>>>>>> view of these frameworks these stream
> > processing
> > > > jobs
> > > > >> > are
> > > > >> > > > > just
> > > > >> > > > > >> > >>>>>> stateless
> > > > >> > > > > >> > >>>>>>>> services that can come and go and expand and
> > > > contract
> > > > >> > at
> > > > >> > > > > will.
> > > > >> > > > > >> > >>>>>>>> There
> > > > >> > > > > >> > >>>>> is
> > > > >> > > > > >> > >>>>>>> no
> > > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would
> > get
> > > > >> larger
> > > > >> > > if
> > > > >> > > > we
> > > > >> > > > > >> > >>>>>>>>  productionized but not vastly larger. We
> > really
> > > > do
> > > > >> > get a
> > > > >> > > > ton
> > > > >> > > > > >> > >>>>>>>> of
> > > > >> > > > > >> > >>>>>>> leverage
> > > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > > > >> > > > > >> > >>>>>>>>  2. Partition management is fully delegated
> to
> > > the
> > > > >> new
> > > > >> > > > > >> consumer.
> > > > >> > > > > >> > >>>>> This
> > > > >> > > > > >> > >>>>>>>>  is nice since now any partition management
> > > > strategy
> > > > >> > > > > available
> > > > >> > > > > >> > >>>>>>>> to
> > > > >> > > > > >> > >>>>>> Kafka
> > > > >> > > > > >> > >>>>>>>>  consumer is also available to Samza (and
> vice
> > > > versa)
> > > > >> > and
> > > > >> > > > > with
> > > > >> > > > > >> > >>>>>>>> the
> > > > >> > > > > >> > >>>>>>> exact
> > > > >> > > > > >> > >>>>>>>>  same configs.
> > > > >> > > > > >> > >>>>>>>>  3. It supports state as well as state reuse
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is thought
> > > > >> provoking.
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> -Jay
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris
> > > Riccomini <
> > > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > > > >> > > > > >> > >>>>>>>> wrote:
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Hey all,
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> I have had some discussions with Samza
> > > engineers
> > > > at
> > > > >> > > > LinkedIn
> > > > >> > > > > >> > >>>>>>>>> and
> > > > >> > > > > >> > >>>>>>> Confluent
> > > > >> > > > > >> > >>>>>>>>> and we came up with a few observations and
> > > would
> > > > >> like
> > > > >> > to
> > > > >> > > > > >> > >>>>>>>>> propose
> > > > >> > > > > >> > >>>>> some
> > > > >> > > > > >> > >>>>>>>>> changes.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> We've observed some things that I want to
> > call
> > > > out
> > > > >> > about
> > > > >> > > > > >> > >>>>>>>>> Samza's
> > > > >> > > > > >> > >>>>>> design,
> > > > >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic
> > deployment
> > > > >> system.
> > > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > > >> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and
> > > > Kafka's
> > > > >> > > > consumer
> > > > >> > > > > >> > >>>>>>>>> APIs
> > > > >> > > > > >> > >>>>> are
> > > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same problems.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> All three of these issues are related, but
> > I'll
> > > > >> > address
> > > > >> > > > them
> > > > >> > > > > >> in
> > > > >> > > > > >> > >>>>> order.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Deployment
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a
> > dynamic
> > > > >> > > deployment
> > > > >> > > > > >> > >>>>>>>>> scheduler
> > > > >> > > > > >> > >>>>>> such
> > > > >> > > > > >> > >>>>>>>>> as
> > > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built
> > > Samza,
> > > > we
> > > > >> > bet
> > > > >> > > > that
> > > > >> > > > > >> > >>>>>>>>> there
> > > > >> > > > > >> > >>>>>> would
> > > > >> > > > > >> > >>>>>>>>> be
> > > > >> > > > > >> > >>>>>>>>> one or two winners in this area, and we
> could
> > > > >> support
> > > > >> > > > them,
> > > > >> > > > > >> and
> > > > >> > > > > >> > >>>>>>>>> the
> > > > >> > > > > >> > >>>>>> rest
> > > > >> > > > > >> > >>>>>>>>> would go away. In reality, there are many
> > > > >> variations.
> > > > >> > > > > >> > >>>>>>>>> Furthermore,
> > > > >> > > > > >> > >>>>>> many
> > > > >> > > > > >> > >>>>>>>>> people still prefer to just start their
> > > > processors
> > > > >> > like
> > > > >> > > > > normal
> > > > >> > > > > >> > >>>>>>>>> Java processes, and use traditional
> > deployment
> > > > >> scripts
> > > > >> > > > such
> > > > >> > > > > as
> > > > >> > > > > >> > >>>>>>>>> Fabric,
> > > > >> > > > > >> > >>>>>> Chef,
> > > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system
> on
> > > > users
> > > > >> > makes
> > > > >> > > > the
> > > > >> > > > > >> > >>>>>>>>> Samza start-up process really painful for
> > first
> > > > time
> > > > >> > > > users.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was
> also
> > a
> > > > bit
> > > > >> of
> > > > >> > a
> > > > >> > > > > >> > >>>>>>>>> mis-fire
> > > > >> > > > > >> > >>>>>> because
> > > > >> > > > > >> > >>>>>>>>> of
> > > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between the
> > > > nature of
> > > > >> > > batch
> > > > >> > > > > >> jobs
> > > > >> > > > > >> > >>>>>>>>> and
> > > > >> > > > > >> > >>>>>>> stream
> > > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made
> conscious
> > > > effort
> > > > >> to
> > > > >> > > > favor
> > > > >> > > > > >> > >>>>>>>>> the
> > > > >> > > > > >> > >>>>>> Hadoop
> > > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it
> > > worked
> > > > >> and
> > > > >> > > was
> > > > >> > > > > well
> > > > >> > > > > >> > >>>>>>> understood.
> > > > >> > > > > >> > >>>>>>>>> One thing that we missed was that batch
> jobs
> > > > have a
> > > > >> > > > definite
> > > > >> > > > > >> > >>>>>> beginning,
> > > > >> > > > > >> > >>>>>>>>> and
> > > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't
> > > (usually).
> > > > >> This
> > > > >> > > > leads
> > > > >> > > > > to
> > > > >> > > > > >> > >>>>>>>>> a
> > > > >> > > > > >> > >>>>> much
> > > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream
> > > processors.
> > > > >> You
> > > > >> > > > > >> basically
> > > > >> > > > > >> > >>>>>>>>> just
> > > > >> > > > > >> > >>>>>>> need
> > > > >> > > > > >> > >>>>>>>>> to find a place to start the processor, and
> > > start
> > > > >> it.
> > > > >> > > The
> > > > >> > > > > way
> > > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no
> concept
> > > of
> > > > a
> > > > >> > > cluster
> > > > >> > > > > >> > >>>>>>>>> being "full". We always
> > > > >> > > > > >> > >>>>>> add
> > > > >> > > > > >> > >>>>>>>>> more machines. The problem with coupling
> > Samza
> > > > with
> > > > >> a
> > > > >> > > > > >> scheduler
> > > > >> > > > > >> > >>>>>>>>> is
> > > > >> > > > > >> > >>>>>> that
> > > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to handle
> > > > deployment.
> > > > >> > > This
> > > > >> > > > > >> pulls
> > > > >> > > > > >> > >>>>>>>>> in a
> > > > >> > > > > >> > >>>>>>> bunch
> > > > >> > > > > >> > >>>>>>>>> of things such as configuration
> distribution
> > > > (config
> > > > >> > > > > stream),
> > > > >> > > > > >> > >>>>>>>>> shell
> > > > >> > > > > >> > >>>>>>> scrips
> > > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all
> > the
> > > > .tgz
> > > > >> > > > stuff),
> > > > >> > > > > >> etc.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
> > deployment
> > > > was
> > > > >> to
> > > > >> > > > > support
> > > > >> > > > > >> > >>>>>>>>> data locality. If you want to have
> locality,
> > > you
> > > > >> need
> > > > >> > to
> > > > >> > > > put
> > > > >> > > > > >> > >>>>>>>>> your
> > > > >> > > > > >> > >>>>>> processors
> > > > >> > > > > >> > >>>>>>>>> close to the data they're processing. Upon
> > > > further
> > > > >> > > > > >> > >>>>>>>>> investigation,
> > > > >> > > > > >> > >>>>>>> though,
> > > > >> > > > > >> > >>>>>>>>> this feature is not that beneficial. There
> is
> > > > some
> > > > >> > good
> > > > >> > > > > >> > >>>>>>>>> discussion
> > > > >> > > > > >> > >>>>>> about
> > > > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335. Again,
> we
> > > > took
> > > > >> the
> > > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > > > >> > > > > >> > >>>>>> path,
> > > > >> > > > > >> > >>>>>>>>> but
> > > > >> > > > > >> > >>>>>>>>> there are some fundamental differences
> > between
> > > > HDFS
> > > > >> > and
> > > > >> > > > > Kafka.
> > > > >> > > > > >> > >>>>>>>>> HDFS
> > > > >> > > > > >> > >>>>>> has
> > > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions. This
> > leads
> > > to
> > > > >> less
> > > > >> > > > > >> > >>>>>>>>> optimization potential with stream
> processors
> > > on
> > > > top
> > > > >> > of
> > > > >> > > > > Kafka.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> This feature is also used as a crutch.
> Samza
> > > > doesn't
> > > > >> > > have
> > > > >> > > > > any
> > > > >> > > > > >> > >>>>>>>>> built
> > > > >> > > > > >> > >>>>> in
> > > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends
> on
> > > the
> > > > >> > > dynamic
> > > > >> > > > > >> > >>>>>>>>> deployment scheduling system to handle
> > restarts
> > > > >> when a
> > > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > > > >> > > > > >> > >>>>>>> made
> > > > >> > > > > >> > >>>>>>>>> it very difficult to write a standalone
> Samza
> > > > >> > container
> > > > >> > > > > >> > >>>> (SAMZA-516).
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Pluggability
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> In some cases pluggability is good, but I
> > think
> > > > that
> > > > >> > > we've
> > > > >> > > > > >> gone
> > > > >> > > > > >> > >>>>>>>>> too
> > > > >> > > > > >> > >>>>>> far
> > > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> > (SystemConsumer,
> > > > >> > > > > SystemProducer,
> > > > >> > > > > >> > >>>> etc).
> > > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about every
> > > > >> component
> > > > >> > > > > >> > >>>>> (MessageChooser,
> > > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper,
> ConfigRewriter,
> > > > etc).
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> There's probably more that I've forgotten,
> as
> > > > well.
> > > > >> > Some
> > > > >> > > > of
> > > > >> > > > > >> > >>>>>>>>> these
> > > > >> > > > > >> > >>>>> are
> > > > >> > > > > >> > >>>>>>>>> useful, but some have proven not to be.
> This
> > > all
> > > > >> comes
> > > > >> > > at
> > > > >> > > > a
> > > > >> > > > > >> cost:
> > > > >> > > > > >> > >>>>>>>>> complexity. This complexity is making it
> > harder
> > > > for
> > > > >> > our
> > > > >> > > > > users
> > > > >> > > > > >> > >>>>>>>>> to
> > > > >> > > > > >> > >>>>> pick
> > > > >> > > > > >> > >>>>>> up
> > > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also makes
> > it
> > > > >> > difficult
> > > > >> > > > for
> > > > >> > > > > >> > >>>>>>>>> Samza developers to reason about what the
> > > > >> > > characteristics
> > > > >> > > > of
> > > > >> > > > > >> > >>>>>>>>> the container (since the characteristics
> > change
> > > > >> > > depending
> > > > >> > > > on
> > > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> The issues with pluggability are most
> visible
> > > in
> > > > the
> > > > >> > > > System
> > > > >> > > > > >> APIs.
> > > > >> > > > > >> > >>>>> What
> > > > >> > > > > >> > >>>>>>>>> Samza really requires to be functional is
> > Kafka
> > > > as
> > > > >> its
> > > > >> > > > > >> > >>>>>>>>> transport
> > > > >> > > > > >> > >>>>>> layer.
> > > > >> > > > > >> > >>>>>>>>> But
> > > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases
> into
> > > one
> > > > >> API:
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> The current System API supports both of
> these
> > > use
> > > > >> > cases.
> > > > >> > > > The
> > > > >> > > > > >> > >>>>>>>>> problem
> > > > >> > > > > >> > >>>>>> is,
> > > > >> > > > > >> > >>>>>>>>> we
> > > > >> > > > > >> > >>>>>>>>> actually want different features for each
> use
> > > > case.
> > > > >> By
> > > > >> > > > > >> papering
> > > > >> > > > > >> > >>>>>>>>> over
> > > > >> > > > > >> > >>>>>>> these
> > > > >> > > > > >> > >>>>>>>>> two use cases, and providing a single API,
> > > we've
> > > > >> > > > introduced
> > > > >> > > > > a
> > > > >> > > > > >> > >>>>>>>>> ton of
> > > > >> > > > > >> > >>>>>>> leaky
> > > > >> > > > > >> > >>>>>>>>> abstractions.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> For example, what we'd really like in (2)
> is
> > to
> > > > have
> > > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for offsets
> > > (like
> > > > >> > Kafka).
> > > > >> > > > > This
> > > > >> > > > > >> > >>>>>>>>> would be at odds
> > > > >> > > > > >> > >>>>> with
> > > > >> > > > > >> > >>>>>>> (1),
> > > > >> > > > > >> > >>>>>>>>> though, since different systems have
> > different
> > > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > > >> > > > > >> > >>>>>>>>> There was discussion both on the mailing
> list
> > > and
> > > > >> the
> > > > >> > > SQL
> > > > >> > > > > >> JIRAs
> > > > >> > > > > >> > >>>>> about
> > > > >> > > > > >> > >>>>>>> the
> > > > >> > > > > >> > >>>>>>>>> need for this.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> The same thing holds true for
> replayability.
> > > > Kafka
> > > > >> > > allows
> > > > >> > > > us
> > > > >> > > > > >> to
> > > > >> > > > > >> > >>>>> rewind
> > > > >> > > > > >> > >>>>>>>>> when
> > > > >> > > > > >> > >>>>>>>>> we have a failure. Many other systems
> don't.
> > In
> > > > some
> > > > >> > > > cases,
> > > > >> > > > > >> > >>>>>>>>> systems
> > > > >> > > > > >> > >>>>>>> return
> > > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > > > >> WikipediaSystemConsumer)
> > > > >> > > > > because
> > > > >> > > > > >> > >>>>>>>>> they
> > > > >> > > > > >> > >>>>>> have
> > > > >> > > > > >> > >>>>>>> no
> > > > >> > > > > >> > >>>>>>>>> offsets.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka
> > supports
> > > > >> > > > > partitioning,
> > > > >> > > > > >> > >>>>>>>>> but
> > > > >> > > > > >> > >>>>> many
> > > > >> > > > > >> > >>>>>>>>> systems don't. We model this by having a
> > single
> > > > >> > > partition
> > > > >> > > > > for
> > > > >> > > > > >> > >>>>>>>>> those systems. Still, other systems model
> > > > >> partitioning
> > > > >> > > > > >> > >>>> differently (e.g.
> > > > >> > > > > >> > >>>>>>>>> Kinesis).
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a mess.
> > > > Creating
> > > > >> > > streams
> > > > >> > > > > in
> > > > >> > > > > >> a
> > > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost impossible.
> As
> > is
> > > > >> > modeling
> > > > >> > > > > >> > >>>>>>>>> metadata
> > > > >> > > > > >> > >>>>> for
> > > > >> > > > > >> > >>>>>>> the
> > > > >> > > > > >> > >>>>>>>>> system (replication factor, partitions,
> > > location,
> > > > >> > etc).
> > > > >> > > > The
> > > > >> > > > > >> > >>>>>>>>> list
> > > > >> > > > > >> > >>>>> goes
> > > > >> > > > > >> > >>>>>>> on.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Duplicate work
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> At the time that we began writing Samza,
> > > Kafka's
> > > > >> > > consumer
> > > > >> > > > > and
> > > > >> > > > > >> > >>>>> producer
> > > > >> > > > > >> > >>>>>>>>> APIs
> > > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On the
> > > > >> > consumer-side,
> > > > >> > > > you
> > > > >> > > > > >> > >>>>>>>>> had two
> > > > >> > > > > >> > >>>>>>>>> options: use the high level consumer, or
> the
> > > > simple
> > > > >> > > > > consumer.
> > > > >> > > > > >> > >>>>>>>>> The
> > > > >> > > > > >> > >>>>>>> problem
> > > > >> > > > > >> > >>>>>>>>> with the high-level consumer was that it
> > > > controlled
> > > > >> > your
> > > > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and the
> order
> > > in
> > > > >> which
> > > > >> > > you
> > > > >> > > > > >> > >>>>>>>>> received messages. The
> > > > >> > > > > >> > >>>>> problem
> > > > >> > > > > >> > >>>>>>>>> with
> > > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not
> simple.
> > > It's
> > > > >> > basic.
> > > > >> > > > You
> > > > >> > > > > >> > >>>>>>>>> end up
> > > > >> > > > > >> > >>>>>>> having
> > > > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level stuff
> > that
> > > > you
> > > > >> > > > > shouldn't.
> > > > >> > > > > >> > >>>>>>>>> We
> > > > >> > > > > >> > >>>>>> spent a
> > > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> > KafkaSystemConsumer
> > > > very
> > > > >> > > > robust.
> > > > >> > > > > >> It
> > > > >> > > > > >> > >>>>>>>>> also allows us to support some cool
> features:
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
> > > > prioritization.
> > > > >> > > > > >> > >>>>>>>>> * Tight control over partition assignment
> to
> > > > support
> > > > >> > > > joins,
> > > > >> > > > > >> > >>>>>>>>> global
> > > > >> > > > > >> > >>>>>> state
> > > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > > > >> > > > > >> > >>>>>>>>> * Tight control over offset checkpointing.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time is that
> > > these
> > > > >> > > features
> > > > >> > > > > >> > >>>>>>>>> should
> > > > >> > > > > >> > >>>>>>> actually
> > > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not
> > just
> > > > >> Samza
> > > > >> > > > stream
> > > > >> > > > > >> > >>>>>> processors)
> > > > >> > > > > >> > >>>>>>>>> end up wanting to do things like joins and
> > > > partition
> > > > >> > > > > >> > >>>>>>>>> assignment. The
> > > > >> > > > > >> > >>>>>>> Kafka
> > > > >> > > > > >> > >>>>>>>>> community has come to the same conclusion.
> > > > They're
> > > > >> > > adding
> > > > >> > > > a
> > > > >> > > > > >> ton
> > > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka consumer
> > > > >> > > implementation.
> > > > >> > > > > To a
> > > > >> > > > > >> > >>>>>>>>> large extent,
> > > > >> > > > > >> > >>>>> it's
> > > > >> > > > > >> > >>>>>>>>> duplicate work to what we've already done
> in
> > > > Samza.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a
> very
> > > > similar
> > > > >> > > > > approach
> > > > >> > > > > >> > >>>>>>>>> to
> > > > >> > > > > >> > >>>>>> Samza's
> > > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation for
> > > > handling
> > > > >> > > offset
> > > > >> > > > > >> > >>>>>> checkpointing.
> > > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset management
> > > feature
> > > > >> > stores
> > > > >> > > > > >> offset
> > > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you to
> > fetch
> > > > them
> > > > >> > > from
> > > > >> > > > > the
> > > > >> > > > > >> > >>>>>>>>> broker.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since we
> > > could
> > > > >> have
> > > > >> > > > shared
> > > > >> > > > > >> > >>>>>>>>> the
> > > > >> > > > > >> > >>>>> work
> > > > >> > > > > >> > >>>>>> if
> > > > >> > > > > >> > >>>>>>>>> it
> > > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Vision
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather radical
> > > > proposal.
> > > > >> > Samza
> > > > >> > > > is
> > > > >> > > > > >> > >>>>> relatively
> > > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to say
> that
> > > > we're
> > > > >> > > near a
> > > > >> > > > > 1.0
> > > > >> > > > > >> > >>>>>> release.
> > > > >> > > > > >> > >>>>>>>>> I'd
> > > > >> > > > > >> > >>>>>>>>> like to propose that we take what we've
> > > learned,
> > > > and
> > > > >> > > begin
> > > > >> > > > > >> > >>>>>>>>> thinking
> > > > >> > > > > >> > >>>>>>> about
> > > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change if
> we
> > > were
> > > > >> > > starting
> > > > >> > > > > >> from
> > > > >> > > > > >> > >>>>>> scratch?
> > > > >> > > > > >> > >>>>>>>>> My
> > > > >> > > > > >> > >>>>>>>>> proposal is to:
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to
> > run
> > > > Samza
> > > > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
> > > dependences
> > > > on
> > > > >> > > YARN,
> > > > >> > > > > >> Mesos,
> > > > >> > > > > >> > >>>> etc.
> > > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support only
> > Kafka
> > > > as
> > > > >> the
> > > > >> > > > > stream
> > > > >> > > > > >> > >>>>>> processing
> > > > >> > > > > >> > >>>>>>>>> layer.
> > > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
> > > > >> serialization,
> > > > >> > > and
> > > > >> > > > > >> > >>>>>>>>> config
> > > > >> > > > > >> > >>>>>>> systems,
> > > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> This would fix all of the issues that I
> > > outlined
> > > > >> > above.
> > > > >> > > It
> > > > >> > > > > >> > >>>>>>>>> should
> > > > >> > > > > >> > >>>>> also
> > > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> > dramatically.
> > > > >> > > Supporting
> > > > >> > > > > >> only
> > > > >> > > > > >> > >>>>>>>>> a standalone container will allow Samza to
> be
> > > > >> executed
> > > > >> > > on
> > > > >> > > > > YARN
> > > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> > Marathon/Aurora),
> > > or
> > > > >> most
> > > > >> > > > other
> > > > >> > > > > >> > >>>>>>>>> in-house
> > > > >> > > > > >> > >>>>>>> deployment
> > > > >> > > > > >> > >>>>>>>>> systems. This should make life a lot easier
> > for
> > > > new
> > > > >> > > users.
> > > > >> > > > > >> > >>>>>>>>> Imagine
> > > > >> > > > > >> > >>>>>>> having
> > > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN. The
> > drop
> > > > in
> > > > >> > > mailing
> > > > >> > > > > >> list
> > > > >> > > > > >> > >>>>>> traffic
> > > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue to
> me.
> > > The
> > > > >> > > reality
> > > > >> > > > > is,
> > > > >> > > > > >> > >>>>> everyone
> > > > >> > > > > >> > >>>>>>>>> that
> > > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We
> > > > basically
> > > > >> > > > require
> > > > >> > > > > >> it
> > > > >> > > > > >> > >>>>>> already
> > > > >> > > > > >> > >>>>>>> in
> > > > >> > > > > >> > >>>>>>>>> order for most features to work. Those that
> > are
> > > > >> using
> > > > >> > > > other
> > > > >> > > > > >> > >>>>>>>>> systems
> > > > >> > > > > >> > >>>>>> are
> > > > >> > > > > >> > >>>>>>>>> generally using it for ingest into Kafka
> (1),
> > > and
> > > > >> then
> > > > >> > > > they
> > > > >> > > > > do
> > > > >> > > > > >> > >>>>>>>>> the processing on top. There is already
> > > > discussion (
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> >
> > > > >> > > > >
> > > > >> >
> > > >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > >> > > > > >> > >>>>> 767
> > > > >> > > > > >> > >>>>>>>>> )
> > > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka
> > extremely
> > > > >> easy.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Once we make the call to couple with Kafka,
> > we
> > > > can
> > > > >> > > > leverage
> > > > >> > > > > a
> > > > >> > > > > >> > >>>>>>>>> ton of
> > > > >> > > > > >> > >>>>>>> their
> > > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to maintain
> our
> > > own
> > > > >> > config,
> > > > >> > > > > >> > >>>>>>>>> metrics,
> > > > >> > > > > >> > >>>>> etc.
> > > > >> > > > > >> > >>>>>>> We
> > > > >> > > > > >> > >>>>>>>>> can all share the same libraries, and make
> > them
> > > > >> > better.
> > > > >> > > > This
> > > > >> > > > > >> > >>>>>>>>> will
> > > > >> > > > > >> > >>>>> also
> > > > >> > > > > >> > >>>>>>>>> allow us to share the consumer/producer
> APIs,
> > > and
> > > > >> will
> > > > >> > > let
> > > > >> > > > > us
> > > > >> > > > > >> > >>>>> leverage
> > > > >> > > > > >> > >>>>>>>>> their offset management and partition
> > > management,
> > > > >> > rather
> > > > >> > > > > than
> > > > >> > > > > >> > >>>>>>>>> having
> > > > >> > > > > >> > >>>>>> our
> > > > >> > > > > >> > >>>>>>>>> own. All of the coordinator stream code
> would
> > > go
> > > > >> away,
> > > > >> > > as
> > > > >> > > > > >> would
> > > > >> > > > > >> > >>>>>>>>> most
> > > > >> > > > > >> > >>>>>> of
> > > > >> > > > > >> > >>>>>>>>> the
> > > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to
> > push
> > > > some
> > > > >> > > > > partition
> > > > >> > > > > >> > >>>>>>> management
> > > > >> > > > > >> > >>>>>>>>> features into the Kafka broker, but they're
> > > > already
> > > > >> > > moving
> > > > >> > > > > in
> > > > >> > > > > >> > >>>>>>>>> that direction with the new consumer API.
> The
> > > > >> features
> > > > >> > > we
> > > > >> > > > > have
> > > > >> > > > > >> > >>>>>>>>> for
> > > > >> > > > > >> > >>>>>> partition
> > > > >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and seem
> > > like
> > > > >> they
> > > > >> > > > should
> > > > >> > > > > >> be
> > > > >> > > > > >> > >>>>>>>>> in
> > > > >> > > > > >> > >>>>>> Kafka
> > > > >> > > > > >> > >>>>>>>>> anyway. There will always be some niche
> > usages
> > > > which
> > > > >> > > will
> > > > >> > > > > >> > >>>>>>>>> require
> > > > >> > > > > >> > >>>>>> extra
> > > > >> > > > > >> > >>>>>>>>> care and hence full control over partition
> > > > >> assignments
> > > > >> > > > much
> > > > >> > > > > >> > >>>>>>>>> like the
> > > > >> > > > > >> > >>>>>>> Kafka
> > > > >> > > > > >> > >>>>>>>>> low level consumer api. These would
> continue
> > to
> > > > be
> > > > >> > > > > supported.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> These items will be good for the Samza
> > > community.
> > > > >> > > They'll
> > > > >> > > > > make
> > > > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it easier for
> > > > >> developers
> > > > >> > > to
> > > > >> > > > > add
> > > > >> > > > > >> > >>>>>>>>> new features.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and
> > somewhat
> > > > >> > backwards
> > > > >> > > > > >> > >>>>> incompatible
> > > > >> > > > > >> > >>>>>>>>> change). If we choose to go this route,
> it's
> > > > >> important
> > > > >> > > > that
> > > > >> > > > > we
> > > > >> > > > > >> > >>>>> openly
> > > > >> > > > > >> > >>>>>>>>> communicate how we're going to provide a
> > > > migration
> > > > >> > path
> > > > >> > > > from
> > > > >> > > > > >> > >>>>>>>>> the
> > > > >> > > > > >> > >>>>>>> existing
> > > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make
> incompatible
> > > > >> > changes).
> > > > >> > > I
> > > > >> > > > > >> think
> > > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to
> provide a
> > > > >> wrapper
> > > > >> > to
> > > > >> > > > > allow
> > > > >> > > > > >> > >>>>>>>>> existing StreamTask implementations to
> > continue
> > > > >> > running
> > > > >> > > on
> > > > >> > > > > the
> > > > >> > > > > >> > >>>> new container.
> > > > >> > > > > >> > >>>>>>> It's
> > > > >> > > > > >> > >>>>>>>>> also important that we openly communicate
> > about
> > > > >> > timing,
> > > > >> > > > and
> > > > >> > > > > >> > >>>>>>>>> stages
> > > > >> > > > > >> > >>>>> of
> > > > >> > > > > >> > >>>>>>> the
> > > > >> > > > > >> > >>>>>>>>> migration.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you have
> > > > opinions.
> > > > >> > :)
> > > > >> > > > > Please
> > > > >> > > > > >> > >>>>>>>>> send
> > > > >> > > > > >> > >>>>>> your
> > > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>> Cheers,
> > > > >> > > > > >> > >>>>>>>>> Chris
> > > > >> > > > > >> > >>>>>>>>>
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>>
> > > > >> > > > > >> > >>>>>>>
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>> --
> > > > >> > > > > >> > >>>>>> -- Guozhang
> > > > >> > > > > >> > >>>>>>
> > > > >> > > > > >> > >>>>>
> > > > >> > > > > >> > >>>>
> > > > >> > > > > >> > >>
> > > > >> > > > > >> > >>
> > > > >> > > > > >> >
> > > > >> > > > > >> >
> > > > >> > > > > >> >
> > > > >> > > > > >>
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

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

Yeah philosophically I think the argument is that you should capture the
stream in Kafka independent of the transformation. This is obviously a
Kafka-centric view point.

Advantages of this:
- In practice I think this is what e.g. Storm people often end up doing
anyway. You usually need to throttle any access to a live serving database.
- Can have multiple subscribers and they get the same thing without
additional load on the source system.
- Applications can tap into the stream if need be by subscribing.
- You can debug your transformation by tailing the Kafka topic with the
console consumer
- Can tee off the same data stream for batch analysis or Lambda arch style
re-processing

The disadvantage is that it will use Kafka resources. But the idea is
eventually you will have multiple subscribers to any data source (at least
for monitoring) so you will end up there soon enough anyway.

Down the road the technical benefit is that I think it gives us a good path
towards end-to-end exactly once semantics from source to destination.
Basically the connectors need to support idempotence when talking to Kafka
and we need the transactional write feature in Kafka to make the
transformation atomic. This is actually pretty doable if you separate
connector=>kafka problem from the generic transformations which are always
kafka=>kafka. However I think it is quite impossible to do in a all_things
=> all_things environment. Today you can say "well the semantics of the
Samza APIs depend on the connectors you use" but it is actually worse then
that because the semantics actually depend on the pairing of connectors--so
not only can you probably not get a usable "exactly once" guarantee
end-to-end it can actually be quite hard to reverse engineer what property
(if any) your end-to-end flow has if you have heterogenous systems.

-Jay

On Fri, Jul 10, 2015 at 2:00 PM, Yan Fang <ya...@gmail.com> wrote:

> {quote}
> maintained in a separate repository and retaining the existing
> committership but sharing as much else as possible (website, etc)
> {quote}
>
> Overall, I agree on this idea. Now the question is more about "how to do
> it".
>
> On the other hand, one thing I want to point out is that, if we decide to
> go this way, how do we want to support
> otherSystem-transformation-otherSystem use case?
>
> Basically, there are four user groups here:
>
> 1. Kafka-transformation-Kafka
> 2. Kafka-transformation-otherSystem
> 3. otherSystem-transformation-Kafka
> 4. otherSystem-transformation-otherSystem
>
> For group 1, they can easily use the new Samza library to achieve. For
> group 2 and 3, they can use copyCat -> transformation -> Kafka or Kafka->
> transformation -> copyCat.
>
> The problem is for group 4. Do we want to abandon this or still support it?
> Of course, this use case can be achieved by using copyCat -> transformation
> -> Kafka -> transformation -> copyCat, the thing is how we persuade them to
> do this long chain. If yes, it will also be a win for Kafka too. Or if
> there is no one in this community actually doing this so far, maybe ok to
> not support the group 4 directly.
>
> Thanks,
>
> Fang, Yan
> yanfang724@gmail.com
>
> On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Yeah I agree with this summary. I think there are kind of two questions
> > here:
> > 1. Technically does alignment/reliance on Kafka make sense
> > 2. Branding wise (naming, website, concepts, etc) does alignment with
> Kafka
> > make sense
> >
> > Personally I do think both of these things would be really valuable, and
> > would dramatically alter the trajectory of the project.
> >
> > My preference would be to see if people can mostly agree on a direction
> > rather than splintering things off. From my point of view the ideal
> outcome
> > of all the options discussed would be to make Samza a closely aligned
> > subproject, maintained in a separate repository and retaining the
> existing
> > committership but sharing as much else as possible (website, etc). No
> idea
> > about how these things work, Jacob, you probably know more.
> >
> > No discussion amongst the Kafka folks has happened on this, but likely we
> > should figure out what the Samza community actually wants first.
> >
> > I admit that this is a fairly radical departure from how things are.
> >
> > If that doesn't fly, I think, yeah we could leave Samza as it is and do
> the
> > more radical reboot inside Kafka. From my point of view that does leave
> > things in a somewhat confusing state since now there are two stream
> > processing systems more or less coupled to Kafka in large part made by
> the
> > same people. But, arguably that might be a cleaner way to make the
> cut-over
> > and perhaps less risky for Samza community since if it works people can
> > switch and if it doesn't nothing will have changed. Dunno, how do people
> > feel about this?
> >
> > -Jay
> >
> > On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com> wrote:
> >
> > > >  This leads me to thinking that merging projects and communities
> might
> > > be a good idea: with the union of experience from both communities, we
> > will
> > > probably build a better system that is better for users.
> > > Is this what's being proposed though? Merging the projects seems like
> > > a consequence of at most one of the three directions under discussion:
> > > 1) Samza 2.0: The Samza community relies more heavily on Kafka for
> > > configuration, etc. (to a greater or lesser extent to be determined)
> > > but the Samza community would not automatically merge withe Kafka
> > > community (the Phoenix/HBase example is a good one here).
> > > 2) Samza Reboot: The Samza community continues to exist with a limited
> > > project scope, but similarly would not need to be part of the Kafka
> > > community (ie given committership) to progress.  Here, maybe the Samza
> > > team would become a subproject of Kafka (the Board frowns on
> > > subprojects at the moment, so I'm not sure if that's even feasible),
> > > but that would not be required.
> > > 3) Hey Samza! FYI, Kafka does streaming now: In this option the Kafka
> > > team builds its own streaming library, possibly off of Jay's
> > > prototype, which has not direct lineage to the Samza team.  There's no
> > > reason for the Kafka team to bring in the Samza team.
> > >
> > > Is the Kafka community on board with this?
> > >
> > > To be clear, all three options under discussion are interesting,
> > > technically valid and likely healthy directions for the project.
> > > Also, they are not mutually exclusive.  The Samza community could
> > > decide to pursue, say, 'Samza 2.0', while the Kafka community went
> > > forward with 'Hey Samza!'  My points above are directed entirely at
> > > the community aspect of these choices.
> > > -Jakob
> > >
> > > On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com> wrote:
> > > > That's great.  Thanks, Jay.
> > > >
> > > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io> wrote:
> > > >
> > > >> Yeah totally agree. I think you have this issue even today, right?
> > I.e.
> > > if
> > > >> you need to make a simple config change and you're running in YARN
> > today
> > > >> you end up bouncing the job which then rebuilds state. I think the
> fix
> > > is
> > > >> exactly what you described which is to have a long timeout on
> > partition
> > > >> movement for stateful jobs so that if a job is just getting bounced,
> > and
> > > >> the cluster manager (or admin) is smart enough to restart it on the
> > same
> > > >> host when possible, it can optimistically reuse any existing state
> it
> > > finds
> > > >> on disk (if it is valid).
> > > >>
> > > >> So in this model the charter of the CM is to place processes as
> > > stickily as
> > > >> possible and to restart or re-place failed processes. The charter of
> > the
> > > >> partition management system is to control the assignment of work to
> > > these
> > > >> processes. The nice thing about this is that the work assignment,
> > > timeouts,
> > > >> behavior, configs, and code will all be the same across all cluster
> > > >> managers.
> > > >>
> > > >> So I think that prototype would actually give you exactly what you
> > want
> > > >> today for any cluster manager (or manual placement + restart script)
> > > that
> > > >> was sticky in terms of host placement since there is already a
> > > configurable
> > > >> partition movement timeout and task-by-task state reuse with a check
> > on
> > > >> state validity.
> > > >>
> > > >> -Jay
> > > >>
> > > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <
> roger.hoover@gmail.com
> > >
> > > >> wrote:
> > > >>
> > > >> > That would be great to let Kafka do as much heavy lifting as
> > possible
> > > and
> > > >> > make it easier for other languages to implement Samza apis.
> > > >> >
> > > >> > One thing to watch out for is the interplay between Kafka's group
> > > >> > management and the external scheduler/process manager's fault
> > > tolerance.
> > > >> > If a container dies, the Kafka group membership protocol will try
> to
> > > >> assign
> > > >> > it's tasks to other containers while at the same time the process
> > > manager
> > > >> > is trying to relaunch the container.  Without some consideration
> for
> > > this
> > > >> > (like a configurable amount of time to wait before Kafka alters
> the
> > > group
> > > >> > membership), there may be thrashing going on which is especially
> bad
> > > for
> > > >> > containers with large amounts of local state.
> > > >> >
> > > >> > Someone else pointed this out already but I thought it might be
> > worth
> > > >> > calling out again.
> > > >> >
> > > >> > Cheers,
> > > >> >
> > > >> > Roger
> > > >> >
> > > >> >
> > > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <ja...@confluent.io>
> > wrote:
> > > >> >
> > > >> > > Hey Roger,
> > > >> > >
> > > >> > > I couldn't agree more. We spent a bunch of time talking to
> people
> > > and
> > > >> > that
> > > >> > > is exactly the stuff we heard time and again. What makes it
> hard,
> > of
> > > >> > > course, is that there is some tension between compatibility with
> > > what's
> > > >> > > there now and making things better for new users.
> > > >> > >
> > > >> > > I also strongly agree with the importance of multi-language
> > > support. We
> > > >> > are
> > > >> > > talking now about Java, but for application development use
> cases
> > > >> people
> > > >> > > want to work in whatever language they are using elsewhere. I
> > think
> > > >> > moving
> > > >> > > to a model where Kafka itself does the group membership,
> lifecycle
> > > >> > control,
> > > >> > > and partition assignment has the advantage of putting all that
> > > complex
> > > >> > > stuff behind a clean api that the clients are already going to
> be
> > > >> > > implementing for their consumer, so the added functionality for
> > > stream
> > > >> > > processing beyond a consumer becomes very minor.
> > > >> > >
> > > >> > > -Jay
> > > >> > >
> > > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > > roger.hoover@gmail.com>
> > > >> > > wrote:
> > > >> > >
> > > >> > > > Metamorphosis...nice. :)
> > > >> > > >
> > > >> > > > This has been a great discussion.  As a user of Samza who's
> > > recently
> > > >> > > > integrated it into a relatively large organization, I just
> want
> > to
> > > >> add
> > > >> > > > support to a few points already made.
> > > >> > > >
> > > >> > > > The biggest hurdles to adoption of Samza as it currently
> exists
> > > that
> > > >> > I've
> > > >> > > > experienced are:
> > > >> > > > 1) YARN - YARN is overly complex in many environments where
> > Puppet
> > > >> > would
> > > >> > > do
> > > >> > > > just fine but it was the only mechanism to get fault
> tolerance.
> > > >> > > > 2) Configuration - I think I like the idea of configuring most
> > of
> > > the
> > > >> > job
> > > >> > > > in code rather than config files.  In general, I think the
> goal
> > > >> should
> > > >> > be
> > > >> > > > to make it harder to make mistakes, especially of the kind
> where
> > > the
> > > >> > code
> > > >> > > > expects something and the config doesn't match.  The current
> > > config
> > > >> is
> > > >> > > > quite intricate and error-prone.  For example, the application
> > > logic
> > > >> > may
> > > >> > > > depend on bootstrapping a topic but rather than asserting that
> > in
> > > the
> > > >> > > code,
> > > >> > > > you have to rely on getting the config right.  Likewise with
> > > serdes,
> > > >> > the
> > > >> > > > Java representations produced by various serdes (JSON, Avro,
> > etc.)
> > > >> are
> > > >> > > not
> > > >> > > > equivalent so you cannot just reconfigure a serde without
> > changing
> > > >> the
> > > >> > > > code.   It would be nice for jobs to be able to assert what
> they
> > > >> expect
> > > >> > > > from their input topics in terms of partitioning.  This is
> > > getting a
> > > >> > > little
> > > >> > > > off topic but I was even thinking about creating a "Samza
> config
> > > >> > linter"
> > > >> > > > that would sanity check a set of configs.  Especially in
> > > >> organizations
> > > >> > > > where config is managed by a different team than the
> application
> > > >> > > developer,
> > > >> > > > it's very hard to get avoid config mistakes.
> > > >> > > > 3) Java/Scala centric - for many teams (especially DevOps-type
> > > >> folks),
> > > >> > > the
> > > >> > > > pain of the Java toolchain (maven, slow builds, weak command
> > line
> > > >> > > support,
> > > >> > > > configuration over convention) really inhibits productivity.
> As
> > > more
> > > >> > and
> > > >> > > > more high-quality clients become available for Kafka, I hope
> > > they'll
> > > >> > > follow
> > > >> > > > Samza's model.  Not sure how much it affects the proposals in
> > this
> > > >> > thread
> > > >> > > > but please consider other languages in the ecosystem as well.
> > > From
> > > >> > what
> > > >> > > > I've heard, Spark has more Python users than Java/Scala.
> > > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > > >> > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > >> > > > and are working on a Yeoman generator
> > > >> > > > https://github.com/Quantiply/generator-rico for Jython/Samza
> > > >> projects
> > > >> > to
> > > >> > > > alleviate some of the pain)
> > > >> > > >
> > > >> > > > I also want to underscore Jay's point about improving the user
> > > >> > > experience.
> > > >> > > > That's a very important factor for adoption.  I think the goal
> > > should
> > > >> > be
> > > >> > > to
> > > >> > > > make Samza as easy to get started with as something like
> > Logstash.
> > > >> > > > Logstash is vastly inferior in terms of capabilities to Samza
> > but
> > > >> it's
> > > >> > > easy
> > > >> > > > to get started and that makes a big difference.
> > > >> > > >
> > > >> > > > Cheers,
> > > >> > > >
> > > >> > > > Roger
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci
> Morales <
> > > >> > > > gdfm@apache.org> wrote:
> > > >> > > >
> > > >> > > > > Forgot to add. On the naming issues, Kafka Metamorphosis is
> a
> > > clear
> > > >> > > > winner
> > > >> > > > > :)
> > > >> > > > >
> > > >> > > > > --
> > > >> > > > > Gianmarco
> > > >> > > > >
> > > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
> > > >> > > gdfm@apache.org
> > > >> > > > >
> > > >> > > > > wrote:
> > > >> > > > >
> > > >> > > > > > Hi,
> > > >> > > > > >
> > > >> > > > > > @Martin, thanks for you comments.
> > > >> > > > > > Maybe I'm missing some important point, but I think
> coupling
> > > the
> > > >> > > > releases
> > > >> > > > > > is actually a *good* thing.
> > > >> > > > > > To make an example, would it be better if the MR and HDFS
> > > >> > components
> > > >> > > of
> > > >> > > > > > Hadoop had different release schedules?
> > > >> > > > > >
> > > >> > > > > > Actually, keeping the discussion in a single place would
> > make
> > > >> > > agreeing
> > > >> > > > on
> > > >> > > > > > releases (and backwards compatibility) much easier, as
> > > everybody
> > > >> > > would
> > > >> > > > be
> > > >> > > > > > responsible for the whole codebase.
> > > >> > > > > >
> > > >> > > > > > That said, I like the idea of absorbing samza-core as a
> > > >> > sub-project,
> > > >> > > > and
> > > >> > > > > > leave the fancy stuff separate.
> > > >> > > > > > It probably gives 90% of the benefits we have been
> > discussing
> > > >> here.
> > > >> > > > > >
> > > >> > > > > > Cheers,
> > > >> > > > > >
> > > >> > > > > > --
> > > >> > > > > > Gianmarco
> > > >> > > > > >
> > > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > >> > > > > >
> > > >> > > > > >> Hey Martin,
> > > >> > > > > >>
> > > >> > > > > >> I agree coupling release schedules is a downside.
> > > >> > > > > >>
> > > >> > > > > >> Definitely we can try to solve some of the integration
> > > problems
> > > >> in
> > > >> > > > > >> Confluent Platform or in other distributions. But I think
> > > this
> > > >> > ends
> > > >> > > up
> > > >> > > > > >> being really shallow. I guess I feel to really get a good
> > > user
> > > >> > > > > experience
> > > >> > > > > >> the two systems have to kind of feel like part of the
> same
> > > thing
> > > >> > and
> > > >> > > > you
> > > >> > > > > >> can't really add that in later--you can put both in the
> > same
> > > >> > > > > downloadable
> > > >> > > > > >> tar file but it doesn't really give a very cohesive
> > feeling.
> > > I
> > > >> > agree
> > > >> > > > > that
> > > >> > > > > >> ultimately any of the project stuff is as much social and
> > > naming
> > > >> > as
> > > >> > > > > >> anything else--theoretically two totally independent
> > projects
> > > >> > could
> > > >> > > > work
> > > >> > > > > >> to
> > > >> > > > > >> tightly align. In practice this seems to be quite
> difficult
> > > >> > though.
> > > >> > > > > >>
> > > >> > > > > >> For the frameworks--totally agree it would be good to
> > > maintain
> > > >> the
> > > >> > > > > >> framework support with the project. In some cases there
> may
> > > not
> > > >> be
> > > >> > > too
> > > >> > > > > >> much
> > > >> > > > > >> there since the integration gets lighter but I think
> > whatever
> > > >> > stubs
> > > >> > > > you
> > > >> > > > > >> need should be included. So no I definitely wasn't trying
> > to
> > > >> imply
> > > >> > > > > >> dropping
> > > >> > > > > >> support for these frameworks, just making the integration
> > > >> lighter
> > > >> > by
> > > >> > > > > >> separating process management from partition management.
> > > >> > > > > >>
> > > >> > > > > >> You raise two good points we would have to figure out if
> we
> > > went
> > > >> > > down
> > > >> > > > > the
> > > >> > > > > >> alignment path:
> > > >> > > > > >> 1. With respect to the name, yeah I think the first
> > question
> > > is
> > > >> > > > whether
> > > >> > > > > >> some "re-branding" would be worth it. If so then I think
> we
> > > can
> > > >> > > have a
> > > >> > > > > big
> > > >> > > > > >> thread on the name. I'm definitely not set on Kafka
> > > Streaming or
> > > >> > > Kafka
> > > >> > > > > >> Streams I was just using them to be kind of
> illustrative. I
> > > >> agree
> > > >> > > with
> > > >> > > > > >> your
> > > >> > > > > >> critique of these names, though I think people would get
> > the
> > > >> idea.
> > > >> > > > > >> 2. Yeah you also raise a good point about how to "factor"
> > it.
> > > >> Here
> > > >> > > are
> > > >> > > > > the
> > > >> > > > > >> options I see (I could get enthusiastic about any of
> them):
> > > >> > > > > >>    a. One repo for both Kafka and Samza
> > > >> > > > > >>    b. Two repos, retaining the current seperation
> > > >> > > > > >>    c. Two repos, the equivalent of samza-api and
> samza-core
> > > is
> > > >> > > > absorbed
> > > >> > > > > >> almost like a third client
> > > >> > > > > >>
> > > >> > > > > >> Cheers,
> > > >> > > > > >>
> > > >> > > > > >> -Jay
> > > >> > > > > >>
> > > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> > > >> > > > martin@kleppmann.com>
> > > >> > > > > >> wrote:
> > > >> > > > > >>
> > > >> > > > > >> > Ok, thanks for the clarifications. Just a few follow-up
> > > >> > comments.
> > > >> > > > > >> >
> > > >> > > > > >> > - I see the appeal of merging with Kafka or becoming a
> > > >> > subproject:
> > > >> > > > the
> > > >> > > > > >> > reasons you mention are good. The risk I see is that
> > > release
> > > >> > > > schedules
> > > >> > > > > >> > become coupled to each other, which can slow everyone
> > down,
> > > >> and
> > > >> > > > large
> > > >> > > > > >> > projects with many contributors are harder to manage.
> > > (Jakob,
> > > >> > can
> > > >> > > > you
> > > >> > > > > >> speak
> > > >> > > > > >> > from experience, having seen a wider range of Hadoop
> > > ecosystem
> > > >> > > > > >> projects?)
> > > >> > > > > >> >
> > > >> > > > > >> > Some of the goals of a better unified developer
> > experience
> > > >> could
> > > >> > > > also
> > > >> > > > > be
> > > >> > > > > >> > solved by integrating Samza nicely into a Kafka
> > > distribution
> > > >> > (such
> > > >> > > > as
> > > >> > > > > >> > Confluent's). I'm not against merging projects if we
> > decide
> > > >> > that's
> > > >> > > > the
> > > >> > > > > >> way
> > > >> > > > > >> > to go, just pointing out the same goals can perhaps
> also
> > be
> > > >> > > achieved
> > > >> > > > > in
> > > >> > > > > >> > other ways.
> > > >> > > > > >> >
> > > >> > > > > >> > - With regard to dropping the YARN dependency: are you
> > > >> proposing
> > > >> > > > that
> > > >> > > > > >> > Samza doesn't give any help to people wanting to run on
> > > >> > > > > >> YARN/Mesos/AWS/etc?
> > > >> > > > > >> > So the docs would basically have a link to Slider and
> > > nothing
> > > >> > > else?
> > > >> > > > Or
> > > >> > > > > >> > would we maintain integrations with a bunch of popular
> > > >> > deployment
> > > >> > > > > >> methods
> > > >> > > > > >> > (e.g. the necessary glue and shell scripts to make
> Samza
> > > work
> > > >> > with
> > > >> > > > > >> Slider)?
> > > >> > > > > >> >
> > > >> > > > > >> > I absolutely think it's a good idea to have the "as a
> > > library"
> > > >> > and
> > > >> > > > > "as a
> > > >> > > > > >> > process" (using Yi's taxonomy) options for people who
> > want
> > > >> them,
> > > >> > > > but I
> > > >> > > > > >> > think there should also be a low-friction path for
> common
> > > "as
> > > >> a
> > > >> > > > > service"
> > > >> > > > > >> > deployment methods, for which we probably need to
> > maintain
> > > >> > > > > integrations.
> > > >> > > > > >> >
> > > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to me,
> > because
> > > >> Kafka
> > > >> > > is
> > > >> > > > > all
> > > >> > > > > >> > about streams already. Perhaps "Kafka Transformers" or
> > > "Kafka
> > > >> > > > Filters"
> > > >> > > > > >> > would be more apt?
> > > >> > > > > >> >
> > > >> > > > > >> > One suggestion: perhaps the core of Samza (stream
> > > >> transformation
> > > >> > > > with
> > > >> > > > > >> > state management -- i.e. the "Samza as a library" bit)
> > > could
> > > >> > > become
> > > >> > > > > >> part of
> > > >> > > > > >> > Kafka, while higher-level tools such as streaming SQL
> and
> > > >> > > > integrations
> > > >> > > > > >> with
> > > >> > > > > >> > deployment frameworks remain in a separate project? In
> > > other
> > > >> > > words,
> > > >> > > > > >> Kafka
> > > >> > > > > >> > would absorb the proven, stable core of Samza, which
> > would
> > > >> > become
> > > >> > > > the
> > > >> > > > > >> > "third Kafka client" mentioned early in this thread.
> The
> > > Samza
> > > >> > > > project
> > > >> > > > > >> > would then target that third Kafka client as its base
> > API,
> > > and
> > > >> > the
> > > >> > > > > >> project
> > > >> > > > > >> > would be freed up to explore more experimental new
> > > horizons.
> > > >> > > > > >> >
> > > >> > > > > >> > Martin
> > > >> > > > > >> >
> > > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <
> jay.kreps@gmail.com>
> > > >> wrote:
> > > >> > > > > >> >
> > > >> > > > > >> > > Hey Martin,
> > > >> > > > > >> > >
> > > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually don't
> > think
> > > it
> > > >> > ties
> > > >> > > > our
> > > >> > > > > >> > hands
> > > >> > > > > >> > > at all, all it does is refactor things. The division
> of
> > > >> > > > > >> responsibility is
> > > >> > > > > >> > > that Samza core is responsible for task lifecycle,
> > state,
> > > >> and
> > > >> > > > > >> partition
> > > >> > > > > >> > > management (using the Kafka co-ordinator) but it is
> NOT
> > > >> > > > responsible
> > > >> > > > > >> for
> > > >> > > > > >> > > packaging, configuration deployment or execution of
> > > >> processes.
> > > >> > > The
> > > >> > > > > >> > problem
> > > >> > > > > >> > > of packaging and starting these processes is
> > > >> > > > > >> > > framework/environment-specific. This leaves
> individual
> > > >> > > frameworks
> > > >> > > > to
> > > >> > > > > >> be
> > > >> > > > > >> > as
> > > >> > > > > >> > > fancy or vanilla as they like. So you can get simple
> > > >> stateless
> > > >> > > > > >> support in
> > > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app
> > framework
> > > >> > > (Slider,
> > > >> > > > > >> > Marathon,
> > > >> > > > > >> > > etc). These are well known by people and have nice
> UIs
> > > and a
> > > >> > lot
> > > >> > > > of
> > > >> > > > > >> > > flexibility. I don't think they have node affinity
> as a
> > > >> built
> > > >> > in
> > > >> > > > > >> option
> > > >> > > > > >> > > (though I could be wrong). So if we want that we can
> > > either
> > > >> > wait
> > > >> > > > for
> > > >> > > > > >> them
> > > >> > > > > >> > > to add it or do a custom framework to add that
> feature
> > > (as
> > > >> > now).
> > > >> > > > > >> > Obviously
> > > >> > > > > >> > > if you manage things with old-school ops tools
> > > >> > (puppet/chef/etc)
> > > >> > > > you
> > > >> > > > > >> get
> > > >> > > > > >> > > locality easily. The nice thing, though, is that all
> > the
> > > >> samza
> > > >> > > > > >> "business
> > > >> > > > > >> > > logic" around partition management and fault
> tolerance
> > > is in
> > > >> > > Samza
> > > >> > > > > >> core
> > > >> > > > > >> > so
> > > >> > > > > >> > > it is shared across frameworks and the framework
> > specific
> > > >> bit
> > > >> > is
> > > >> > > > > just
> > > >> > > > > >> > > whether it is smart enough to try to get the same
> host
> > > when
> > > >> a
> > > >> > > job
> > > >> > > > is
> > > >> > > > > >> > > restarted.
> > > >> > > > > >> > >
> > > >> > > > > >> > > With respect to the Kafka-alignment, yeah I think the
> > > goal
> > > >> > would
> > > >> > > > be
> > > >> > > > > >> (a)
> > > >> > > > > >> > > actually get better alignment in user experience, and
> > (b)
> > > >> > > express
> > > >> > > > > >> this in
> > > >> > > > > >> > > the naming and project branding. Specifically:
> > > >> > > > > >> > > 1. Website/docs, it would be nice for the
> > > "transformation"
> > > >> api
> > > >> > > to
> > > >> > > > be
> > > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be able to
> > > explain
> > > >> > > when
> > > >> > > > to
> > > >> > > > > >> use
> > > >> > > > > >> > > the consumer and when to use the stream processing
> > > >> > functionality
> > > >> > > > and
> > > >> > > > > >> lead
> > > >> > > > > >> > > people into that experience.
> > > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or
> > > whatever)
> > > >> > that
> > > >> > > > has
> > > >> > > > > >> both
> > > >> > > > > >> > > Kafka and the stream processing part and they
> actually
> > > work
> > > >> > > > > together.
> > > >> > > > > >> > > 3. Unify the programming experience so the client and
> > > Samza
> > > >> > api
> > > >> > > > > share
> > > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > > >> > > > > >> > >
> > > >> > > > > >> > > I think sub-projects keep separate committers and can
> > > have a
> > > >> > > > > separate
> > > >> > > > > >> > repo,
> > > >> > > > > >> > > but I'm actually not really sure (I can't find a
> > > definition
> > > >> > of a
> > > >> > > > > >> > subproject
> > > >> > > > > >> > > in Apache).
> > > >> > > > > >> > >
> > > >> > > > > >> > > Basically at a high-level you want the experience to
> > > "feel"
> > > >> > > like a
> > > >> > > > > >> single
> > > >> > > > > >> > > system, not to relatively independent things that are
> > > kind
> > > >> of
> > > >> > > > > >> awkwardly
> > > >> > > > > >> > > glued together.
> > > >> > > > > >> > >
> > > >> > > > > >> > > I think if we did that they having naming or branding
> > > like
> > > >> > > "kafka
> > > >> > > > > >> > > streaming" or "kafka streams" or something like that
> > > would
> > > >> > > > actually
> > > >> > > > > >> do a
> > > >> > > > > >> > > good job of conveying what it is. I do that this
> would
> > > help
> > > >> > > > adoption
> > > >> > > > > >> > quite
> > > >> > > > > >> > > a lot as it would correctly convey that using Kafka
> > > >> Streaming
> > > >> > > with
> > > >> > > > > >> Kafka
> > > >> > > > > >> > is
> > > >> > > > > >> > > a fairly seamless experience and Kafka is pretty
> > heavily
> > > >> > adopted
> > > >> > > > at
> > > >> > > > > >> this
> > > >> > > > > >> > > point.
> > > >> > > > > >> > >
> > > >> > > > > >> > > Fwiw we actually considered this model originally
> when
> > > open
> > > >> > > > sourcing
> > > >> > > > > >> > Samza,
> > > >> > > > > >> > > however at that time Kafka was relatively unknown and
> > we
> > > >> > decided
> > > >> > > > not
> > > >> > > > > >> to
> > > >> > > > > >> > do
> > > >> > > > > >> > > it since we felt it would be limiting. From my point
> of
> > > view
> > > >> > the
> > > >> > > > > three
> > > >> > > > > >> > > things have changed (1) Kafka is now really heavily
> > used
> > > for
> > > >> > > > stream
> > > >> > > > > >> > > processing, (2) we learned that abstracting out the
> > > stream
> > > >> > well
> > > >> > > is
> > > >> > > > > >> > > basically impossible, (3) we learned it is really
> hard
> > to
> > > >> keep
> > > >> > > the
> > > >> > > > > two
> > > >> > > > > >> > > things feeling like a single product.
> > > >> > > > > >> > >
> > > >> > > > > >> > > -Jay
> > > >> > > > > >> > >
> > > >> > > > > >> > >
> > > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> > > >> > > > > >> martin@kleppmann.com>
> > > >> > > > > >> > > wrote:
> > > >> > > > > >> > >
> > > >> > > > > >> > >> Hi all,
> > > >> > > > > >> > >>
> > > >> > > > > >> > >> Lots of good thoughts here.
> > > >> > > > > >> > >>
> > > >> > > > > >> > >> I agree with the general philosophy of tying Samza
> > more
> > > >> > firmly
> > > >> > > to
> > > >> > > > > >> Kafka.
> > > >> > > > > >> > >> After I spent a while looking at integrating other
> > > message
> > > >> > > > brokers
> > > >> > > > > >> (e.g.
> > > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the
> conclusion
> > > that
> > > >> > > > > >> > SystemConsumer
> > > >> > > > > >> > >> tacitly assumes a model so much like Kafka's that
> > pretty
> > > >> much
> > > >> > > > > nobody
> > > >> > > > > >> but
> > > >> > > > > >> > >> Kafka actually implements it. (Databus is perhaps an
> > > >> > exception,
> > > >> > > > but
> > > >> > > > > >> it
> > > >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus, making
> > > Samza
> > > >> > > fully
> > > >> > > > > >> > dependent
> > > >> > > > > >> > >> on Kafka acknowledges that the system-independence
> was
> > > >> never
> > > >> > as
> > > >> > > > > real
> > > >> > > > > >> as
> > > >> > > > > >> > we
> > > >> > > > > >> > >> perhaps made it out to be. The gains of code reuse
> are
> > > >> real.
> > > >> > > > > >> > >>
> > > >> > > > > >> > >> The idea of decoupling Samza from YARN has also
> always
> > > been
> > > >> > > > > >> appealing to
> > > >> > > > > >> > >> me, for various reasons already mentioned in this
> > > thread.
> > > >> > > > Although
> > > >> > > > > >> > making
> > > >> > > > > >> > >> Samza jobs deployable on anything
> (YARN/Mesos/AWS/etc)
> > > >> seems
> > > >> > > > > >> laudable,
> > > >> > > > > >> > I am
> > > >> > > > > >> > >> a little concerned that it will restrict us to a
> > lowest
> > > >> > common
> > > >> > > > > >> > denominator.
> > > >> > > > > >> > >> For example, would host affinity (SAMZA-617) still
> be
> > > >> > possible?
> > > >> > > > For
> > > >> > > > > >> jobs
> > > >> > > > > >> > >> with large amounts of state, I think SAMZA-617 would
> > be
> > > a
> > > >> big
> > > >> > > > boon,
> > > >> > > > > >> > since
> > > >> > > > > >> > >> restoring state off the changelog on every single
> > > restart
> > > >> is
> > > >> > > > > painful,
> > > >> > > > > >> > due
> > > >> > > > > >> > >> to long recovery times. It would be a shame if the
> > > >> decoupling
> > > >> > > > from
> > > >> > > > > >> YARN
> > > >> > > > > >> > >> made host affinity impossible.
> > > >> > > > > >> > >>
> > > >> > > > > >> > >> Jay, a question about the proposed API for
> > > instantiating a
> > > >> > job
> > > >> > > in
> > > >> > > > > >> code
> > > >> > > > > >> > >> (rather than a properties file): when submitting a
> job
> > > to a
> > > >> > > > > cluster,
> > > >> > > > > >> is
> > > >> > > > > >> > the
> > > >> > > > > >> > >> idea that the instantiation code runs on a client
> > > >> somewhere,
> > > >> > > > which
> > > >> > > > > >> then
> > > >> > > > > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc?
> > Or
> > > >> does
> > > >> > > that
> > > >> > > > > >> code
> > > >> > > > > >> > run
> > > >> > > > > >> > >> on each container that is part of the job (in which
> > > case,
> > > >> how
> > > >> > > > does
> > > >> > > > > >> the
> > > >> > > > > >> > job
> > > >> > > > > >> > >> submission to the cluster work)?
> > > >> > > > > >> > >>
> > > >> > > > > >> > >> I agree with Garry that it doesn't feel right to
> make
> > a
> > > 1.0
> > > >> > > > release
> > > >> > > > > >> > with a
> > > >> > > > > >> > >> plan for it to be immediately obsolete. So if this
> is
> > > going
> > > >> > to
> > > >> > > > > >> happen, I
> > > >> > > > > >> > >> think it would be more honest to stick with 0.*
> > version
> > > >> > numbers
> > > >> > > > > until
> > > >> > > > > >> > the
> > > >> > > > > >> > >> library-ified Samza has been implemented, is stable
> > and
> > > >> > widely
> > > >> > > > > used.
> > > >> > > > > >> > >>
> > > >> > > > > >> > >> Should the new Samza be a subproject of Kafka? There
> > is
> > > >> > > precedent
> > > >> > > > > for
> > > >> > > > > >> > >> tight coupling between different Apache projects
> (e.g.
> > > >> > Curator
> > > >> > > > and
> > > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think remaining
> > > >> separate
> > > >> > > > would
> > > >> > > > > >> be
> > > >> > > > > >> > ok.
> > > >> > > > > >> > >> Even if Samza is fully dependent on Kafka, there is
> > > enough
> > > >> > > > > substance
> > > >> > > > > >> in
> > > >> > > > > >> > >> Samza that it warrants being a separate project. An
> > > >> argument
> > > >> > in
> > > >> > > > > >> favour
> > > >> > > > > >> > of
> > > >> > > > > >> > >> merging would be if we think Kafka has a much
> stronger
> > > >> "brand
> > > >> > > > > >> presence"
> > > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If the Kafka
> > > >> project
> > > >> > is
> > > >> > > > > >> willing
> > > >> > > > > >> > to
> > > >> > > > > >> > >> endorse Samza as the "official" way of doing
> stateful
> > > >> stream
> > > >> > > > > >> > >> transformations, that would probably have much the
> > same
> > > >> > effect
> > > >> > > as
> > > >> > > > > >> > >> re-branding Samza as "Kafka Stream Processors" or
> > > suchlike.
> > > >> > > Close
> > > >> > > > > >> > >> collaboration between the two projects will be
> needed
> > in
> > > >> any
> > > >> > > > case.
> > > >> > > > > >> > >>
> > > >> > > > > >> > >> From a project management perspective, I guess the
> > "new
> > > >> > Samza"
> > > >> > > > > would
> > > >> > > > > >> > have
> > > >> > > > > >> > >> to be developed on a branch alongside ongoing
> > > maintenance
> > > >> of
> > > >> > > the
> > > >> > > > > >> current
> > > >> > > > > >> > >> line of development? I think it would be important
> to
> > > >> > continue
> > > >> > > > > >> > supporting
> > > >> > > > > >> > >> existing users, and provide a graceful migration
> path
> > to
> > > >> the
> > > >> > > new
> > > >> > > > > >> > version.
> > > >> > > > > >> > >> Leaving the current versions unsupported and forcing
> > > people
> > > >> > to
> > > >> > > > > >> rewrite
> > > >> > > > > >> > >> their jobs would send a bad signal.
> > > >> > > > > >> > >>
> > > >> > > > > >> > >> Best,
> > > >> > > > > >> > >> Martin
> > > >> > > > > >> > >>
> > > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <
> jay@confluent.io>
> > > >> wrote:
> > > >> > > > > >> > >>
> > > >> > > > > >> > >>> Hey Garry,
> > > >> > > > > >> > >>>
> > > >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy to chat
> > > more
> > > >> > about
> > > >> > > > > this
> > > >> > > > > >> if
> > > >> > > > > >> > >>> you'd be interested. I think Chris and I started
> with
> > > the
> > > >> > idea
> > > >> > > > of
> > > >> > > > > >> "what
> > > >> > > > > >> > >>> would it take to make Samza a kick-ass ingestion
> > tool"
> > > but
> > > >> > > > > >> ultimately
> > > >> > > > > >> > we
> > > >> > > > > >> > >>> kind of came around to the idea that ingestion and
> > > >> > > > transformation
> > > >> > > > > >> had
> > > >> > > > > >> > >>> pretty different needs and coupling the two made
> > things
> > > >> > hard.
> > > >> > > > > >> > >>>
> > > >> > > > > >> > >>> For what it's worth I think copycat (KIP-26)
> actually
> > > will
> > > >> > do
> > > >> > > > what
> > > >> > > > > >> you
> > > >> > > > > >> > >> are
> > > >> > > > > >> > >>> looking for.
> > > >> > > > > >> > >>>
> > > >> > > > > >> > >>> With regard to your point about slider, I don't
> > > >> necessarily
> > > >> > > > > >> disagree.
> > > >> > > > > >> > >> But I
> > > >> > > > > >> > >>> think getting good YARN support is quite doable
> and I
> > > >> think
> > > >> > we
> > > >> > > > can
> > > >> > > > > >> make
> > > >> > > > > >> > >>> that work well. I think the issue this proposal
> > solves
> > > is
> > > >> > that
> > > >> > > > > >> > >> technically
> > > >> > > > > >> > >>> it is pretty hard to support multiple cluster
> > > management
> > > >> > > systems
> > > >> > > > > the
> > > >> > > > > >> > way
> > > >> > > > > >> > >>> things are now, you need to write an "app master"
> or
> > > >> > > "framework"
> > > >> > > > > for
> > > >> > > > > >> > each
> > > >> > > > > >> > >>> and they are all a little different so testing is
> > > really
> > > >> > hard.
> > > >> > > > In
> > > >> > > > > >> the
> > > >> > > > > >> > >>> absence of this we have been stuck with just YARN
> > which
> > > >> has
> > > >> > > > > >> fantastic
> > > >> > > > > >> > >>> penetration in the Hadoopy part of the org, but
> zero
> > > >> > > penetration
> > > >> > > > > >> > >> elsewhere.
> > > >> > > > > >> > >>> Given the huge amount of work being put in to
> slider,
> > > >> > > marathon,
> > > >> > > > > aws
> > > >> > > > > >> > >>> tooling, not to mention the umpteen related
> packaging
> > > >> > > > technologies
> > > >> > > > > >> > people
> > > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> > cloud-specific
> > > >> > deploy
> > > >> > > > > >> tools,
> > > >> > > > > >> > >> etc)
> > > >> > > > > >> > >>> I really think it is important to get this right.
> > > >> > > > > >> > >>>
> > > >> > > > > >> > >>> -Jay
> > > >> > > > > >> > >>>
> > > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > > >> > > > > >> > >>>
> > > >> > > > > >> > >>>> Hi all,
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>> I think the question below re does Samza become a
> > > >> > sub-project
> > > >> > > > of
> > > >> > > > > >> Kafka
> > > >> > > > > >> > >>>> highlights the broader point around migration.
> Chris
> > > >> > mentions
> > > >> > > > > >> Samza's
> > > >> > > > > >> > >>>> maturity is heading towards a v1 release but I'm
> not
> > > sure
> > > >> > it
> > > >> > > > > feels
> > > >> > > > > >> > >> right to
> > > >> > > > > >> > >>>> launch a v1 then immediately plan to deprecate
> most
> > of
> > > >> it.
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>> From a selfish perspective I have some guys who
> have
> > > >> > started
> > > >> > > > > >> working
> > > >> > > > > >> > >> with
> > > >> > > > > >> > >>>> Samza and building some new consumers/producers
> was
> > > next
> > > >> > up.
> > > >> > > > > Sounds
> > > >> > > > > >> > like
> > > >> > > > > >> > >>>> that is absolutely not the direction to go. I need
> > to
> > > >> look
> > > >> > > into
> > > >> > > > > the
> > > >> > > > > >> > KIP
> > > >> > > > > >> > >> in
> > > >> > > > > >> > >>>> more detail but for me the attractiveness of
> adding
> > > new
> > > >> > Samza
> > > >> > > > > >> > >>>> consumer/producers -- even if yes all they were
> > doing
> > > was
> > > >> > > > really
> > > >> > > > > >> > getting
> > > >> > > > > >> > >>>> data into and out of Kafka --  was to avoid
> having
> > to
> > > >> > worry
> > > >> > > > > about
> > > >> > > > > >> the
> > > >> > > > > >> > >>>> lifecycle management of external clients. If there
> > is
> > > a
> > > >> > > generic
> > > >> > > > > >> Kafka
> > > >> > > > > >> > >>>> ingress/egress layer that I can plug a new
> connector
> > > into
> > > >> > and
> > > >> > > > > have
> > > >> > > > > >> a
> > > >> > > > > >> > >> lot of
> > > >> > > > > >> > >>>> the heavy lifting re scale and reliability done
> for
> > me
> > > >> then
> > > >> > > it
> > > >> > > > > >> gives
> > > >> > > > > >> > me
> > > >> > > > > >> > >> all
> > > >> > > > > >> > >>>> the pushing new consumers/producers would. If not
> > > then it
> > > >> > > > > >> complicates
> > > >> > > > > >> > my
> > > >> > > > > >> > >>>> operational deployments.
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>> Which is similar to my other question with the
> > > proposal
> > > >> --
> > > >> > if
> > > >> > > > we
> > > >> > > > > >> > build a
> > > >> > > > > >> > >>>> fully available/stand-alone Samza plus the
> requisite
> > > >> shims
> > > >> > to
> > > >> > > > > >> > integrate
> > > >> > > > > >> > >>>> with Slider etc I suspect the former may be a lot
> > more
> > > >> work
> > > >> > > > than
> > > >> > > > > we
> > > >> > > > > >> > >> think.
> > > >> > > > > >> > >>>> We may make it much easier for a newcomer to get
> > > >> something
> > > >> > > > > running
> > > >> > > > > >> but
> > > >> > > > > >> > >>>> having them step up and get a reliable production
> > > >> > deployment
> > > >> > > > may
> > > >> > > > > >> still
> > > >> > > > > >> > >>>> dominate mailing list  traffic, if for different
> > > reasons
> > > >> > than
> > > >> > > > > >> today.
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with making
> > the
> > > >> Samza
> > > >> > > > > >> dependency
> > > >> > > > > >> > >> on
> > > >> > > > > >> > >>>> Kafka much more explicit and I absolutely see the
> > > >> benefits
> > > >> > > in
> > > >> > > > > the
> > > >> > > > > >> > >>>> reduction of duplication and clashing
> > > >> > > > terminologies/abstractions
> > > >> > > > > >> that
> > > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library would
> likely
> > > be a
> > > >> > very
> > > >> > > > > nice
> > > >> > > > > >> > tool
> > > >> > > > > >> > >> to
> > > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the
> concerns
> > > >> above
> > > >> > re
> > > >> > > > the
> > > >> > > > > >> > >>>> operational side.
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>> Garry
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>> -----Original Message-----
> > > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
> > > >> > gdfm@apache.org
> > > >> > > ]
> > > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > >> > > > > >> > >>>> To: dev@samza.apache.org
> > > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>> Very interesting thoughts.
> > > >> > > > > >> > >>>> From outside, I have always perceived Samza as a
> > > >> computing
> > > >> > > > layer
> > > >> > > > > >> over
> > > >> > > > > >> > >>>> Kafka.
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>> The question, maybe a bit provocative, is "should
> > > Samza
> > > >> be
> > > >> > a
> > > >> > > > > >> > sub-project
> > > >> > > > > >> > >>>> of Kafka then?"
> > > >> > > > > >> > >>>> Or does it make sense to keep it as a separate
> > project
> > > >> > with a
> > > >> > > > > >> separate
> > > >> > > > > >> > >>>> governance?
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>> Cheers,
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>> --
> > > >> > > > > >> > >>>> Gianmarco
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > > yanfang724@gmail.com>
> > > >> > > > wrote:
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more
> tightly.
> > > >> > Because
> > > >> > > > > Samza
> > > >> > > > > >> de
> > > >> > > > > >> > >>>>> facto is based on Kafka, and it should leverage
> > what
> > > >> Kafka
> > > >> > > > has.
> > > >> > > > > At
> > > >> > > > > >> > the
> > > >> > > > > >> > >>>>> same time, Kafka does not need to reinvent what
> > Samza
> > > >> > > already
> > > >> > > > > >> has. I
> > > >> > > > > >> > >>>>> also like the idea of separating the ingestion
> and
> > > >> > > > > transformation.
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>> But it is a little difficult for me to image how
> > the
> > > >> Samza
> > > >> > > > will
> > > >> > > > > >> look
> > > >> > > > > >> > >>>> like.
> > > >> > > > > >> > >>>>> And I feel Chris and Jay have a little difference
> > in
> > > >> terms
> > > >> > > of
> > > >> > > > > how
> > > >> > > > > >> > >>>>> Samza should look like.
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>> *** Will it look like what Jay's code shows (A
> > > client of
> > > >> > > > Kakfa)
> > > >> > > > > ?
> > > >> > > > > >> And
> > > >> > > > > >> > >>>>> user's application code calls this client?
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka (like
> > what
> > > the
> > > >> > > code
> > > >> > > > > >> shows),
> > > >> > > > > >> > >>>>> how do we implement auto-balance and
> > fault-tolerance?
> > > >> Are
> > > >> > > they
> > > >> > > > > >> taken
> > > >> > > > > >> > >>>>> care by the Kafka broker or other mechanism, such
> > as
> > > >> > "Samza
> > > >> > > > > >> worker"
> > > >> > > > > >> > >>>>> (just make up the name) ?
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>> 2. What about other features, such as
> auto-scaling,
> > > >> shared
> > > >> > > > > state,
> > > >> > > > > >> > >>>>> monitoring?
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>> *** If we have Samza standalone, (is this what
> > Chris
> > > >> > > > suggests?)
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa and
> > > produce
> > > >> to
> > > >> > > it.
> > > >> > > > > >> Then it
> > > >> > > > > >> > >>>>> becomes the same as what Samza looks like now,
> > > except it
> > > >> > > does
> > > >> > > > > not
> > > >> > > > > >> > rely
> > > >> > > > > >> > >>>>> on Yarn anymore.
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>> 2. if it is standalone, how can it leverage
> Kafka's
> > > >> > metrics,
> > > >> > > > > logs,
> > > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>> Thanks,
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>> Fang, Yan
> > > >> > > > > >> > >>>>> yanfang724@gmail.com
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > > >> > > > > wangguoz@gmail.com
> > > >> > > > > >> >
> > > >> > > > > >> > >>>> wrote:
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>>> Read through the code example and it looks good
> to
> > > me.
> > > >> A
> > > >> > > few
> > > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>> Today Samza deploys as executable runnable like:
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> > > >> > > > > >> > >>>> --config-path=file://...
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>> And this proposal advocate for deploying Samza
> > more
> > > as
> > > >> > > > embedded
> > > >> > > > > >> > >>>>>> libraries in user application code (ignoring the
> > > >> > > terminology
> > > >> > > > > >> since
> > > >> > > > > >> > >>>>>> it is not the
> > > >> > > > > >> > >>>>> same
> > > >> > > > > >> > >>>>>> as the prototype code):
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>> StreamTask task = new MyStreamTask(configs);
> > Thread
> > > >> > thread
> > > >> > > =
> > > >> > > > > new
> > > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>> I think both of these deployment modes are
> > important
> > > >> for
> > > >> > > > > >> different
> > > >> > > > > >> > >>>>>> types
> > > >> > > > > >> > >>>>> of
> > > >> > > > > >> > >>>>>> users. That said, I think making Samza purely
> > > >> standalone
> > > >> > is
> > > >> > > > > still
> > > >> > > > > >> > >>>>>> sufficient for either runnable or library modes.
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>> Guozhang
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> > > >> > > > jay@confluent.io>
> > > >> > > > > >> > wrote:
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>>> Looks like gmail mangled the code example, it
> was
> > > >> > supposed
> > > >> > > > to
> > > >> > > > > >> look
> > > >> > > > > >> > >>>>>>> like
> > > >> > > > > >> > >>>>>>> this:
> > > >> > > > > >> > >>>>>>>
> > > >> > > > > >> > >>>>>>> Properties props = new Properties();
> > > >> > > > > >> > >>>>>>> props.put("bootstrap.servers",
> "localhost:4242");
> > > >> > > > > >> StreamingConfig
> > > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> "test-topic-2");
> > > >> > > > > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> > > >> > > > > >> > >>>>>>> config.serialization(new StringSerializer(),
> new
> > > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming
> container =
> > > new
> > > >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
> > > >> > > > > >> > >>>>>>>
> > > >> > > > > >> > >>>>>>> -Jay
> > > >> > > > > >> > >>>>>>>
> > > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> > > >> > > > jay@confluent.io
> > > >> > > > > >
> > > >> > > > > >> > >>>> wrote:
> > > >> > > > > >> > >>>>>>>
> > > >> > > > > >> > >>>>>>>> Hey guys,
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> This came out of some conversations Chris and
> I
> > > were
> > > >> > > having
> > > >> > > > > >> > >>>>>>>> around
> > > >> > > > > >> > >>>>>>> whether
> > > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a kind of
> > data
> > > >> > > > ingestion
> > > >> > > > > >> > >>>>> framework
> > > >> > > > > >> > >>>>>>> for
> > > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26
> > "copycat").
> > > >> This
> > > >> > > > kind
> > > >> > > > > of
> > > >> > > > > >> > >>>>>> combined
> > > >> > > > > >> > >>>>>>>> with complaints around config and YARN and the
> > > >> > discussion
> > > >> > > > > >> around
> > > >> > > > > >> > >>>>>>>> how
> > > >> > > > > >> > >>>>> to
> > > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> So the thought experiment was, given that
> Samza
> > > was
> > > >> > > > basically
> > > >> > > > > >> > >>>>>>>> already totally Kafka specific, what if you
> just
> > > >> > embraced
> > > >> > > > > that
> > > >> > > > > >> > >>>>>>>> and turned it
> > > >> > > > > >> > >>>>>> into
> > > >> > > > > >> > >>>>>>>> something less like a heavyweight framework
> and
> > > more
> > > >> > > like a
> > > >> > > > > >> > >>>>>>>> third
> > > >> > > > > >> > >>>>> Kafka
> > > >> > > > > >> > >>>>>>>> client--a kind of "producing consumer" with
> > state
> > > >> > > > management
> > > >> > > > > >> > >>>>>> facilities.
> > > >> > > > > >> > >>>>>>>> Basically a library. Instead of a complex
> stream
> > > >> > > processing
> > > >> > > > > >> > >>>>>>>> framework
> > > >> > > > > >> > >>>>>>> this
> > > >> > > > > >> > >>>>>>>> would actually be a very simple thing, not
> much
> > > more
> > > >> > > > > >> complicated
> > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > >> > >>>>> use
> > > >> > > > > >> > >>>>>>> or
> > > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris said
> we
> > > >> thought
> > > >> > > > about
> > > >> > > > > >> it
> > > >> > > > > >> > >>>>>>>> a
> > > >> > > > > >> > >>>>> lot
> > > >> > > > > >> > >>>>>> of
> > > >> > > > > >> > >>>>>>>> what Samza (and the other stream processing
> > > systems
> > > >> > were
> > > >> > > > > doing)
> > > >> > > > > >> > >>>>> seemed
> > > >> > > > > >> > >>>>>>> like
> > > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> Of course you need to ingest/output data to
> and
> > > from
> > > >> > the
> > > >> > > > > stream
> > > >> > > > > >> > >>>>>>>> processing. But when we actually looked into
> how
> > > that
> > > >> > > would
> > > >> > > > > >> > >>>>>>>> work,
> > > >> > > > > >> > >>>>> Samza
> > > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion framework
> > > for a
> > > >> > > bunch
> > > >> > > > of
> > > >> > > > > >> > >>>>> reasons.
> > > >> > > > > >> > >>>>>> To
> > > >> > > > > >> > >>>>>>>> really do that right you need a pretty
> different
> > > >> > internal
> > > >> > > > > data
> > > >> > > > > >> > >>>>>>>> model
> > > >> > > > > >> > >>>>>> and
> > > >> > > > > >> > >>>>>>>> set of apis. So what if you split them and had
> > an
> > > api
> > > >> > for
> > > >> > > > > Kafka
> > > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a
> > separate
> > > >> api
> > > >> > > for
> > > >> > > > > >> Kafka
> > > >> > > > > >> > >>>>>>>> transformation (Samza).
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> This would also allow really embracing the
> same
> > > >> > > terminology
> > > >> > > > > and
> > > >> > > > > >> > >>>>>>>> conventions. One complaint about the current
> > > state is
> > > >> > > that
> > > >> > > > > the
> > > >> > > > > >> > >>>>>>>> two
> > > >> > > > > >> > >>>>>>> systems
> > > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like
> > "stream"
> > > vs
> > > >> > > > "topic"
> > > >> > > > > >> and
> > > >> > > > > >> > >>>>>>> different
> > > >> > > > > >> > >>>>>>>> config and monitoring systems means you kind
> of
> > > have
> > > >> to
> > > >> > > > learn
> > > >> > > > > >> > >>>>>>>> Kafka's
> > > >> > > > > >> > >>>>>>> way,
> > > >> > > > > >> > >>>>>>>> then learn Samza's slightly different way,
> then
> > > kind
> > > >> of
> > > >> > > > > >> > >>>>>>>> understand
> > > >> > > > > >> > >>>>> how
> > > >> > > > > >> > >>>>>>> they
> > > >> > > > > >> > >>>>>>>> map to each other, which having walked a few
> > > people
> > > >> > > through
> > > >> > > > > >> this
> > > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> Since I have been spending a lot of time on
> > > >> airplanes I
> > > >> > > > > hacked
> > > >> > > > > >> > >>>>>>>> up an ernest but still somewhat incomplete
> > > prototype
> > > >> of
> > > >> > > > what
> > > >> > > > > >> > >>>>>>>> this would
> > > >> > > > > >> > >>>>> look
> > > >> > > > > >> > >>>>>>>> like. This is just unceremoniously dumped into
> > > Kafka
> > > >> as
> > > >> > > it
> > > >> > > > > >> > >>>>>>>> required a
> > > >> > > > > >> > >>>>>> few
> > > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is the code:
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> >
> > > >> > > > >
> > > >> >
> > > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> For the purpose of the prototype I just
> > liberally
> > > >> > renamed
> > > >> > > > > >> > >>>>>>>> everything
> > > >> > > > > >> > >>>>> to
> > > >> > > > > >> > >>>>>>>> try to align it with Kafka with no regard for
> > > >> > > > compatibility.
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> To use this would be something like this:
> > > >> > > > > >> > >>>>>>>> Properties props = new Properties();
> > > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> > "localhost:4242");
> > > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > > >> > > > > >> > >>>>> StreamingConfig(props);
> > > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > >> > > > > >> > >>>>>>>> "test-topic-2");
> > > >> > > > > >> config.processor(ExampleStreamProcessor.class);
> > > >> > > > > >> > >>>>>>> config.serialization(new
> > > >> > > > > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
> > > >> > > > KafkaStreaming
> > > >> > > > > >> > >>>>>> container =
> > > >> > > > > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the
> SamzaContainer;
> > > >> > > > > StreamProcessor
> > > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> So rather than putting all the class names in
> a
> > > file
> > > >> > and
> > > >> > > > then
> > > >> > > > > >> > >>>>>>>> having
> > > >> > > > > >> > >>>>>> the
> > > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
> > instantiate
> > > the
> > > >> > > > > container
> > > >> > > > > >> > >>>>>>>> programmatically. Work is balanced over
> however
> > > many
> > > >> > > > > instances
> > > >> > > > > >> > >>>>>>>> of
> > > >> > > > > >> > >>>>> this
> > > >> > > > > >> > >>>>>>> are
> > > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance dies,
> new
> > > >> tasks
> > > >> > > are
> > > >> > > > > >> added
> > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > >> > >>>>> the
> > > >> > > > > >> > >>>>>>>> existing containers without shutting them
> down).
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> We would provide some glue for running this
> > stuff
> > > in
> > > >> > YARN
> > > >> > > > via
> > > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some
> > of
> > > >> their
> > > >> > > > tools
> > > >> > > > > >> > >>>>>>>> but from the
> > > >> > > > > >> > >>>>>> point
> > > >> > > > > >> > >>>>>>> of
> > > >> > > > > >> > >>>>>>>> view of these frameworks these stream
> processing
> > > jobs
> > > >> > are
> > > >> > > > > just
> > > >> > > > > >> > >>>>>> stateless
> > > >> > > > > >> > >>>>>>>> services that can come and go and expand and
> > > contract
> > > >> > at
> > > >> > > > > will.
> > > >> > > > > >> > >>>>>>>> There
> > > >> > > > > >> > >>>>> is
> > > >> > > > > >> > >>>>>>> no
> > > >> > > > > >> > >>>>>>>> more custom scheduler.
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would
> get
> > > >> larger
> > > >> > > if
> > > >> > > > we
> > > >> > > > > >> > >>>>>>>>  productionized but not vastly larger. We
> really
> > > do
> > > >> > get a
> > > >> > > > ton
> > > >> > > > > >> > >>>>>>>> of
> > > >> > > > > >> > >>>>>>> leverage
> > > >> > > > > >> > >>>>>>>>  out of Kafka.
> > > >> > > > > >> > >>>>>>>>  2. Partition management is fully delegated to
> > the
> > > >> new
> > > >> > > > > >> consumer.
> > > >> > > > > >> > >>>>> This
> > > >> > > > > >> > >>>>>>>>  is nice since now any partition management
> > > strategy
> > > >> > > > > available
> > > >> > > > > >> > >>>>>>>> to
> > > >> > > > > >> > >>>>>> Kafka
> > > >> > > > > >> > >>>>>>>>  consumer is also available to Samza (and vice
> > > versa)
> > > >> > and
> > > >> > > > > with
> > > >> > > > > >> > >>>>>>>> the
> > > >> > > > > >> > >>>>>>> exact
> > > >> > > > > >> > >>>>>>>>  same configs.
> > > >> > > > > >> > >>>>>>>>  3. It supports state as well as state reuse
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is thought
> > > >> provoking.
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> -Jay
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris
> > Riccomini <
> > > >> > > > > >> > >>>>>> criccomini@apache.org>
> > > >> > > > > >> > >>>>>>>> wrote:
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Hey all,
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> I have had some discussions with Samza
> > engineers
> > > at
> > > >> > > > LinkedIn
> > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > >> > >>>>>>> Confluent
> > > >> > > > > >> > >>>>>>>>> and we came up with a few observations and
> > would
> > > >> like
> > > >> > to
> > > >> > > > > >> > >>>>>>>>> propose
> > > >> > > > > >> > >>>>> some
> > > >> > > > > >> > >>>>>>>>> changes.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> We've observed some things that I want to
> call
> > > out
> > > >> > about
> > > >> > > > > >> > >>>>>>>>> Samza's
> > > >> > > > > >> > >>>>>> design,
> > > >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic
> deployment
> > > >> system.
> > > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > >> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and
> > > Kafka's
> > > >> > > > consumer
> > > >> > > > > >> > >>>>>>>>> APIs
> > > >> > > > > >> > >>>>> are
> > > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same problems.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> All three of these issues are related, but
> I'll
> > > >> > address
> > > >> > > > them
> > > >> > > > > >> in
> > > >> > > > > >> > >>>>> order.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Deployment
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a
> dynamic
> > > >> > > deployment
> > > >> > > > > >> > >>>>>>>>> scheduler
> > > >> > > > > >> > >>>>>> such
> > > >> > > > > >> > >>>>>>>>> as
> > > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built
> > Samza,
> > > we
> > > >> > bet
> > > >> > > > that
> > > >> > > > > >> > >>>>>>>>> there
> > > >> > > > > >> > >>>>>> would
> > > >> > > > > >> > >>>>>>>>> be
> > > >> > > > > >> > >>>>>>>>> one or two winners in this area, and we could
> > > >> support
> > > >> > > > them,
> > > >> > > > > >> and
> > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > >> > >>>>>> rest
> > > >> > > > > >> > >>>>>>>>> would go away. In reality, there are many
> > > >> variations.
> > > >> > > > > >> > >>>>>>>>> Furthermore,
> > > >> > > > > >> > >>>>>> many
> > > >> > > > > >> > >>>>>>>>> people still prefer to just start their
> > > processors
> > > >> > like
> > > >> > > > > normal
> > > >> > > > > >> > >>>>>>>>> Java processes, and use traditional
> deployment
> > > >> scripts
> > > >> > > > such
> > > >> > > > > as
> > > >> > > > > >> > >>>>>>>>> Fabric,
> > > >> > > > > >> > >>>>>> Chef,
> > > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on
> > > users
> > > >> > makes
> > > >> > > > the
> > > >> > > > > >> > >>>>>>>>> Samza start-up process really painful for
> first
> > > time
> > > >> > > > users.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was also
> a
> > > bit
> > > >> of
> > > >> > a
> > > >> > > > > >> > >>>>>>>>> mis-fire
> > > >> > > > > >> > >>>>>> because
> > > >> > > > > >> > >>>>>>>>> of
> > > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between the
> > > nature of
> > > >> > > batch
> > > >> > > > > >> jobs
> > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > >> > >>>>>>> stream
> > > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made conscious
> > > effort
> > > >> to
> > > >> > > > favor
> > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > >> > >>>>>> Hadoop
> > > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it
> > worked
> > > >> and
> > > >> > > was
> > > >> > > > > well
> > > >> > > > > >> > >>>>>>> understood.
> > > >> > > > > >> > >>>>>>>>> One thing that we missed was that batch jobs
> > > have a
> > > >> > > > definite
> > > >> > > > > >> > >>>>>> beginning,
> > > >> > > > > >> > >>>>>>>>> and
> > > >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't
> > (usually).
> > > >> This
> > > >> > > > leads
> > > >> > > > > to
> > > >> > > > > >> > >>>>>>>>> a
> > > >> > > > > >> > >>>>> much
> > > >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream
> > processors.
> > > >> You
> > > >> > > > > >> basically
> > > >> > > > > >> > >>>>>>>>> just
> > > >> > > > > >> > >>>>>>> need
> > > >> > > > > >> > >>>>>>>>> to find a place to start the processor, and
> > start
> > > >> it.
> > > >> > > The
> > > >> > > > > way
> > > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept
> > of
> > > a
> > > >> > > cluster
> > > >> > > > > >> > >>>>>>>>> being "full". We always
> > > >> > > > > >> > >>>>>> add
> > > >> > > > > >> > >>>>>>>>> more machines. The problem with coupling
> Samza
> > > with
> > > >> a
> > > >> > > > > >> scheduler
> > > >> > > > > >> > >>>>>>>>> is
> > > >> > > > > >> > >>>>>> that
> > > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to handle
> > > deployment.
> > > >> > > This
> > > >> > > > > >> pulls
> > > >> > > > > >> > >>>>>>>>> in a
> > > >> > > > > >> > >>>>>>> bunch
> > > >> > > > > >> > >>>>>>>>> of things such as configuration distribution
> > > (config
> > > >> > > > > stream),
> > > >> > > > > >> > >>>>>>>>> shell
> > > >> > > > > >> > >>>>>>> scrips
> > > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all
> the
> > > .tgz
> > > >> > > > stuff),
> > > >> > > > > >> etc.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic
> deployment
> > > was
> > > >> to
> > > >> > > > > support
> > > >> > > > > >> > >>>>>>>>> data locality. If you want to have locality,
> > you
> > > >> need
> > > >> > to
> > > >> > > > put
> > > >> > > > > >> > >>>>>>>>> your
> > > >> > > > > >> > >>>>>> processors
> > > >> > > > > >> > >>>>>>>>> close to the data they're processing. Upon
> > > further
> > > >> > > > > >> > >>>>>>>>> investigation,
> > > >> > > > > >> > >>>>>>> though,
> > > >> > > > > >> > >>>>>>>>> this feature is not that beneficial. There is
> > > some
> > > >> > good
> > > >> > > > > >> > >>>>>>>>> discussion
> > > >> > > > > >> > >>>>>> about
> > > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we
> > > took
> > > >> the
> > > >> > > > > >> > >>>>>>>>> Map/Reduce
> > > >> > > > > >> > >>>>>> path,
> > > >> > > > > >> > >>>>>>>>> but
> > > >> > > > > >> > >>>>>>>>> there are some fundamental differences
> between
> > > HDFS
> > > >> > and
> > > >> > > > > Kafka.
> > > >> > > > > >> > >>>>>>>>> HDFS
> > > >> > > > > >> > >>>>>> has
> > > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions. This
> leads
> > to
> > > >> less
> > > >> > > > > >> > >>>>>>>>> optimization potential with stream processors
> > on
> > > top
> > > >> > of
> > > >> > > > > Kafka.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> This feature is also used as a crutch. Samza
> > > doesn't
> > > >> > > have
> > > >> > > > > any
> > > >> > > > > >> > >>>>>>>>> built
> > > >> > > > > >> > >>>>> in
> > > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on
> > the
> > > >> > > dynamic
> > > >> > > > > >> > >>>>>>>>> deployment scheduling system to handle
> restarts
> > > >> when a
> > > >> > > > > >> > >>>>>>>>> processor dies. This has
> > > >> > > > > >> > >>>>>>> made
> > > >> > > > > >> > >>>>>>>>> it very difficult to write a standalone Samza
> > > >> > container
> > > >> > > > > >> > >>>> (SAMZA-516).
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Pluggability
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> In some cases pluggability is good, but I
> think
> > > that
> > > >> > > we've
> > > >> > > > > >> gone
> > > >> > > > > >> > >>>>>>>>> too
> > > >> > > > > >> > >>>>>> far
> > > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems
> (SystemConsumer,
> > > >> > > > > SystemProducer,
> > > >> > > > > >> > >>>> etc).
> > > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about every
> > > >> component
> > > >> > > > > >> > >>>>> (MessageChooser,
> > > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter,
> > > etc).
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> There's probably more that I've forgotten, as
> > > well.
> > > >> > Some
> > > >> > > > of
> > > >> > > > > >> > >>>>>>>>> these
> > > >> > > > > >> > >>>>> are
> > > >> > > > > >> > >>>>>>>>> useful, but some have proven not to be. This
> > all
> > > >> comes
> > > >> > > at
> > > >> > > > a
> > > >> > > > > >> cost:
> > > >> > > > > >> > >>>>>>>>> complexity. This complexity is making it
> harder
> > > for
> > > >> > our
> > > >> > > > > users
> > > >> > > > > >> > >>>>>>>>> to
> > > >> > > > > >> > >>>>> pick
> > > >> > > > > >> > >>>>>> up
> > > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also makes
> it
> > > >> > difficult
> > > >> > > > for
> > > >> > > > > >> > >>>>>>>>> Samza developers to reason about what the
> > > >> > > characteristics
> > > >> > > > of
> > > >> > > > > >> > >>>>>>>>> the container (since the characteristics
> change
> > > >> > > depending
> > > >> > > > on
> > > >> > > > > >> > >>>>>>>>> which plugins are use).
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> The issues with pluggability are most visible
> > in
> > > the
> > > >> > > > System
> > > >> > > > > >> APIs.
> > > >> > > > > >> > >>>>> What
> > > >> > > > > >> > >>>>>>>>> Samza really requires to be functional is
> Kafka
> > > as
> > > >> its
> > > >> > > > > >> > >>>>>>>>> transport
> > > >> > > > > >> > >>>>>> layer.
> > > >> > > > > >> > >>>>>>>>> But
> > > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases into
> > one
> > > >> API:
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> The current System API supports both of these
> > use
> > > >> > cases.
> > > >> > > > The
> > > >> > > > > >> > >>>>>>>>> problem
> > > >> > > > > >> > >>>>>> is,
> > > >> > > > > >> > >>>>>>>>> we
> > > >> > > > > >> > >>>>>>>>> actually want different features for each use
> > > case.
> > > >> By
> > > >> > > > > >> papering
> > > >> > > > > >> > >>>>>>>>> over
> > > >> > > > > >> > >>>>>>> these
> > > >> > > > > >> > >>>>>>>>> two use cases, and providing a single API,
> > we've
> > > >> > > > introduced
> > > >> > > > > a
> > > >> > > > > >> > >>>>>>>>> ton of
> > > >> > > > > >> > >>>>>>> leaky
> > > >> > > > > >> > >>>>>>>>> abstractions.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> For example, what we'd really like in (2) is
> to
> > > have
> > > >> > > > > >> > >>>>>>>>> monotonically increasing longs for offsets
> > (like
> > > >> > Kafka).
> > > >> > > > > This
> > > >> > > > > >> > >>>>>>>>> would be at odds
> > > >> > > > > >> > >>>>> with
> > > >> > > > > >> > >>>>>>> (1),
> > > >> > > > > >> > >>>>>>>>> though, since different systems have
> different
> > > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > >> > > > > >> > >>>>>>>>> There was discussion both on the mailing list
> > and
> > > >> the
> > > >> > > SQL
> > > >> > > > > >> JIRAs
> > > >> > > > > >> > >>>>> about
> > > >> > > > > >> > >>>>>>> the
> > > >> > > > > >> > >>>>>>>>> need for this.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> The same thing holds true for replayability.
> > > Kafka
> > > >> > > allows
> > > >> > > > us
> > > >> > > > > >> to
> > > >> > > > > >> > >>>>> rewind
> > > >> > > > > >> > >>>>>>>>> when
> > > >> > > > > >> > >>>>>>>>> we have a failure. Many other systems don't.
> In
> > > some
> > > >> > > > cases,
> > > >> > > > > >> > >>>>>>>>> systems
> > > >> > > > > >> > >>>>>>> return
> > > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > > >> WikipediaSystemConsumer)
> > > >> > > > > because
> > > >> > > > > >> > >>>>>>>>> they
> > > >> > > > > >> > >>>>>> have
> > > >> > > > > >> > >>>>>>> no
> > > >> > > > > >> > >>>>>>>>> offsets.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka
> supports
> > > >> > > > > partitioning,
> > > >> > > > > >> > >>>>>>>>> but
> > > >> > > > > >> > >>>>> many
> > > >> > > > > >> > >>>>>>>>> systems don't. We model this by having a
> single
> > > >> > > partition
> > > >> > > > > for
> > > >> > > > > >> > >>>>>>>>> those systems. Still, other systems model
> > > >> partitioning
> > > >> > > > > >> > >>>> differently (e.g.
> > > >> > > > > >> > >>>>>>>>> Kinesis).
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a mess.
> > > Creating
> > > >> > > streams
> > > >> > > > > in
> > > >> > > > > >> a
> > > >> > > > > >> > >>>>>>>>> system-agnostic way is almost impossible. As
> is
> > > >> > modeling
> > > >> > > > > >> > >>>>>>>>> metadata
> > > >> > > > > >> > >>>>> for
> > > >> > > > > >> > >>>>>>> the
> > > >> > > > > >> > >>>>>>>>> system (replication factor, partitions,
> > location,
> > > >> > etc).
> > > >> > > > The
> > > >> > > > > >> > >>>>>>>>> list
> > > >> > > > > >> > >>>>> goes
> > > >> > > > > >> > >>>>>>> on.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Duplicate work
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> At the time that we began writing Samza,
> > Kafka's
> > > >> > > consumer
> > > >> > > > > and
> > > >> > > > > >> > >>>>> producer
> > > >> > > > > >> > >>>>>>>>> APIs
> > > >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On the
> > > >> > consumer-side,
> > > >> > > > you
> > > >> > > > > >> > >>>>>>>>> had two
> > > >> > > > > >> > >>>>>>>>> options: use the high level consumer, or the
> > > simple
> > > >> > > > > consumer.
> > > >> > > > > >> > >>>>>>>>> The
> > > >> > > > > >> > >>>>>>> problem
> > > >> > > > > >> > >>>>>>>>> with the high-level consumer was that it
> > > controlled
> > > >> > your
> > > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and the order
> > in
> > > >> which
> > > >> > > you
> > > >> > > > > >> > >>>>>>>>> received messages. The
> > > >> > > > > >> > >>>>> problem
> > > >> > > > > >> > >>>>>>>>> with
> > > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not simple.
> > It's
> > > >> > basic.
> > > >> > > > You
> > > >> > > > > >> > >>>>>>>>> end up
> > > >> > > > > >> > >>>>>>> having
> > > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level stuff
> that
> > > you
> > > >> > > > > shouldn't.
> > > >> > > > > >> > >>>>>>>>> We
> > > >> > > > > >> > >>>>>> spent a
> > > >> > > > > >> > >>>>>>>>> lot of time to make Samza's
> KafkaSystemConsumer
> > > very
> > > >> > > > robust.
> > > >> > > > > >> It
> > > >> > > > > >> > >>>>>>>>> also allows us to support some cool features:
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
> > > prioritization.
> > > >> > > > > >> > >>>>>>>>> * Tight control over partition assignment to
> > > support
> > > >> > > > joins,
> > > >> > > > > >> > >>>>>>>>> global
> > > >> > > > > >> > >>>>>> state
> > > >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > > >> > > > > >> > >>>>>>>>> * Tight control over offset checkpointing.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> What we didn't realize at the time is that
> > these
> > > >> > > features
> > > >> > > > > >> > >>>>>>>>> should
> > > >> > > > > >> > >>>>>>> actually
> > > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not
> just
> > > >> Samza
> > > >> > > > stream
> > > >> > > > > >> > >>>>>> processors)
> > > >> > > > > >> > >>>>>>>>> end up wanting to do things like joins and
> > > partition
> > > >> > > > > >> > >>>>>>>>> assignment. The
> > > >> > > > > >> > >>>>>>> Kafka
> > > >> > > > > >> > >>>>>>>>> community has come to the same conclusion.
> > > They're
> > > >> > > adding
> > > >> > > > a
> > > >> > > > > >> ton
> > > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka consumer
> > > >> > > implementation.
> > > >> > > > > To a
> > > >> > > > > >> > >>>>>>>>> large extent,
> > > >> > > > > >> > >>>>> it's
> > > >> > > > > >> > >>>>>>>>> duplicate work to what we've already done in
> > > Samza.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a very
> > > similar
> > > >> > > > > approach
> > > >> > > > > >> > >>>>>>>>> to
> > > >> > > > > >> > >>>>>> Samza's
> > > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation for
> > > handling
> > > >> > > offset
> > > >> > > > > >> > >>>>>> checkpointing.
> > > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset management
> > feature
> > > >> > stores
> > > >> > > > > >> offset
> > > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you to
> fetch
> > > them
> > > >> > > from
> > > >> > > > > the
> > > >> > > > > >> > >>>>>>>>> broker.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since we
> > could
> > > >> have
> > > >> > > > shared
> > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > >> > >>>>> work
> > > >> > > > > >> > >>>>>> if
> > > >> > > > > >> > >>>>>>>>> it
> > > >> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Vision
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> All of this leads me to a rather radical
> > > proposal.
> > > >> > Samza
> > > >> > > > is
> > > >> > > > > >> > >>>>> relatively
> > > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to say that
> > > we're
> > > >> > > near a
> > > >> > > > > 1.0
> > > >> > > > > >> > >>>>>> release.
> > > >> > > > > >> > >>>>>>>>> I'd
> > > >> > > > > >> > >>>>>>>>> like to propose that we take what we've
> > learned,
> > > and
> > > >> > > begin
> > > >> > > > > >> > >>>>>>>>> thinking
> > > >> > > > > >> > >>>>>>> about
> > > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we
> > were
> > > >> > > starting
> > > >> > > > > >> from
> > > >> > > > > >> > >>>>>> scratch?
> > > >> > > > > >> > >>>>>>>>> My
> > > >> > > > > >> > >>>>>>>>> proposal is to:
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to
> run
> > > Samza
> > > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
> > dependences
> > > on
> > > >> > > YARN,
> > > >> > > > > >> Mesos,
> > > >> > > > > >> > >>>> etc.
> > > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support only
> Kafka
> > > as
> > > >> the
> > > >> > > > > stream
> > > >> > > > > >> > >>>>>> processing
> > > >> > > > > >> > >>>>>>>>> layer.
> > > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
> > > >> serialization,
> > > >> > > and
> > > >> > > > > >> > >>>>>>>>> config
> > > >> > > > > >> > >>>>>>> systems,
> > > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> This would fix all of the issues that I
> > outlined
> > > >> > above.
> > > >> > > It
> > > >> > > > > >> > >>>>>>>>> should
> > > >> > > > > >> > >>>>> also
> > > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty
> dramatically.
> > > >> > > Supporting
> > > >> > > > > >> only
> > > >> > > > > >> > >>>>>>>>> a standalone container will allow Samza to be
> > > >> executed
> > > >> > > on
> > > >> > > > > YARN
> > > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using
> Marathon/Aurora),
> > or
> > > >> most
> > > >> > > > other
> > > >> > > > > >> > >>>>>>>>> in-house
> > > >> > > > > >> > >>>>>>> deployment
> > > >> > > > > >> > >>>>>>>>> systems. This should make life a lot easier
> for
> > > new
> > > >> > > users.
> > > >> > > > > >> > >>>>>>>>> Imagine
> > > >> > > > > >> > >>>>>>> having
> > > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN. The
> drop
> > > in
> > > >> > > mailing
> > > >> > > > > >> list
> > > >> > > > > >> > >>>>>> traffic
> > > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me.
> > The
> > > >> > > reality
> > > >> > > > > is,
> > > >> > > > > >> > >>>>> everyone
> > > >> > > > > >> > >>>>>>>>> that
> > > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We
> > > basically
> > > >> > > > require
> > > >> > > > > >> it
> > > >> > > > > >> > >>>>>> already
> > > >> > > > > >> > >>>>>>> in
> > > >> > > > > >> > >>>>>>>>> order for most features to work. Those that
> are
> > > >> using
> > > >> > > > other
> > > >> > > > > >> > >>>>>>>>> systems
> > > >> > > > > >> > >>>>>> are
> > > >> > > > > >> > >>>>>>>>> generally using it for ingest into Kafka (1),
> > and
> > > >> then
> > > >> > > > they
> > > >> > > > > do
> > > >> > > > > >> > >>>>>>>>> the processing on top. There is already
> > > discussion (
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> >
> > > >> > > > >
> > > >> >
> > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > >> > > > > >> > >>>>> 767
> > > >> > > > > >> > >>>>>>>>> )
> > > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka
> extremely
> > > >> easy.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Once we make the call to couple with Kafka,
> we
> > > can
> > > >> > > > leverage
> > > >> > > > > a
> > > >> > > > > >> > >>>>>>>>> ton of
> > > >> > > > > >> > >>>>>>> their
> > > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to maintain our
> > own
> > > >> > config,
> > > >> > > > > >> > >>>>>>>>> metrics,
> > > >> > > > > >> > >>>>> etc.
> > > >> > > > > >> > >>>>>>> We
> > > >> > > > > >> > >>>>>>>>> can all share the same libraries, and make
> them
> > > >> > better.
> > > >> > > > This
> > > >> > > > > >> > >>>>>>>>> will
> > > >> > > > > >> > >>>>> also
> > > >> > > > > >> > >>>>>>>>> allow us to share the consumer/producer APIs,
> > and
> > > >> will
> > > >> > > let
> > > >> > > > > us
> > > >> > > > > >> > >>>>> leverage
> > > >> > > > > >> > >>>>>>>>> their offset management and partition
> > management,
> > > >> > rather
> > > >> > > > > than
> > > >> > > > > >> > >>>>>>>>> having
> > > >> > > > > >> > >>>>>> our
> > > >> > > > > >> > >>>>>>>>> own. All of the coordinator stream code would
> > go
> > > >> away,
> > > >> > > as
> > > >> > > > > >> would
> > > >> > > > > >> > >>>>>>>>> most
> > > >> > > > > >> > >>>>>> of
> > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to
> push
> > > some
> > > >> > > > > partition
> > > >> > > > > >> > >>>>>>> management
> > > >> > > > > >> > >>>>>>>>> features into the Kafka broker, but they're
> > > already
> > > >> > > moving
> > > >> > > > > in
> > > >> > > > > >> > >>>>>>>>> that direction with the new consumer API. The
> > > >> features
> > > >> > > we
> > > >> > > > > have
> > > >> > > > > >> > >>>>>>>>> for
> > > >> > > > > >> > >>>>>> partition
> > > >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and seem
> > like
> > > >> they
> > > >> > > > should
> > > >> > > > > >> be
> > > >> > > > > >> > >>>>>>>>> in
> > > >> > > > > >> > >>>>>> Kafka
> > > >> > > > > >> > >>>>>>>>> anyway. There will always be some niche
> usages
> > > which
> > > >> > > will
> > > >> > > > > >> > >>>>>>>>> require
> > > >> > > > > >> > >>>>>> extra
> > > >> > > > > >> > >>>>>>>>> care and hence full control over partition
> > > >> assignments
> > > >> > > > much
> > > >> > > > > >> > >>>>>>>>> like the
> > > >> > > > > >> > >>>>>>> Kafka
> > > >> > > > > >> > >>>>>>>>> low level consumer api. These would continue
> to
> > > be
> > > >> > > > > supported.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> These items will be good for the Samza
> > community.
> > > >> > > They'll
> > > >> > > > > make
> > > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it easier for
> > > >> developers
> > > >> > > to
> > > >> > > > > add
> > > >> > > > > >> > >>>>>>>>> new features.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and
> somewhat
> > > >> > backwards
> > > >> > > > > >> > >>>>> incompatible
> > > >> > > > > >> > >>>>>>>>> change). If we choose to go this route, it's
> > > >> important
> > > >> > > > that
> > > >> > > > > we
> > > >> > > > > >> > >>>>> openly
> > > >> > > > > >> > >>>>>>>>> communicate how we're going to provide a
> > > migration
> > > >> > path
> > > >> > > > from
> > > >> > > > > >> > >>>>>>>>> the
> > > >> > > > > >> > >>>>>>> existing
> > > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make incompatible
> > > >> > changes).
> > > >> > > I
> > > >> > > > > >> think
> > > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to provide a
> > > >> wrapper
> > > >> > to
> > > >> > > > > allow
> > > >> > > > > >> > >>>>>>>>> existing StreamTask implementations to
> continue
> > > >> > running
> > > >> > > on
> > > >> > > > > the
> > > >> > > > > >> > >>>> new container.
> > > >> > > > > >> > >>>>>>> It's
> > > >> > > > > >> > >>>>>>>>> also important that we openly communicate
> about
> > > >> > timing,
> > > >> > > > and
> > > >> > > > > >> > >>>>>>>>> stages
> > > >> > > > > >> > >>>>> of
> > > >> > > > > >> > >>>>>>> the
> > > >> > > > > >> > >>>>>>>>> migration.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you have
> > > opinions.
> > > >> > :)
> > > >> > > > > Please
> > > >> > > > > >> > >>>>>>>>> send
> > > >> > > > > >> > >>>>>> your
> > > >> > > > > >> > >>>>>>>>> thoughts and feedback.
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>> Cheers,
> > > >> > > > > >> > >>>>>>>>> Chris
> > > >> > > > > >> > >>>>>>>>>
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>>
> > > >> > > > > >> > >>>>>>>
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>> --
> > > >> > > > > >> > >>>>>> -- Guozhang
> > > >> > > > > >> > >>>>>>
> > > >> > > > > >> > >>>>>
> > > >> > > > > >> > >>>>
> > > >> > > > > >> > >>
> > > >> > > > > >> > >>
> > > >> > > > > >> >
> > > >> > > > > >> >
> > > >> > > > > >> >
> > > >> > > > > >>
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yan Fang <ya...@gmail.com>.
{quote}
maintained in a separate repository and retaining the existing
committership but sharing as much else as possible (website, etc)
{quote}

Overall, I agree on this idea. Now the question is more about "how to do
it".

On the other hand, one thing I want to point out is that, if we decide to
go this way, how do we want to support
otherSystem-transformation-otherSystem use case?

Basically, there are four user groups here:

1. Kafka-transformation-Kafka
2. Kafka-transformation-otherSystem
3. otherSystem-transformation-Kafka
4. otherSystem-transformation-otherSystem

For group 1, they can easily use the new Samza library to achieve. For
group 2 and 3, they can use copyCat -> transformation -> Kafka or Kafka->
transformation -> copyCat.

The problem is for group 4. Do we want to abandon this or still support it?
Of course, this use case can be achieved by using copyCat -> transformation
-> Kafka -> transformation -> copyCat, the thing is how we persuade them to
do this long chain. If yes, it will also be a win for Kafka too. Or if
there is no one in this community actually doing this so far, maybe ok to
not support the group 4 directly.

Thanks,

Fang, Yan
yanfang724@gmail.com

On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps <ja...@confluent.io> wrote:

> Yeah I agree with this summary. I think there are kind of two questions
> here:
> 1. Technically does alignment/reliance on Kafka make sense
> 2. Branding wise (naming, website, concepts, etc) does alignment with Kafka
> make sense
>
> Personally I do think both of these things would be really valuable, and
> would dramatically alter the trajectory of the project.
>
> My preference would be to see if people can mostly agree on a direction
> rather than splintering things off. From my point of view the ideal outcome
> of all the options discussed would be to make Samza a closely aligned
> subproject, maintained in a separate repository and retaining the existing
> committership but sharing as much else as possible (website, etc). No idea
> about how these things work, Jacob, you probably know more.
>
> No discussion amongst the Kafka folks has happened on this, but likely we
> should figure out what the Samza community actually wants first.
>
> I admit that this is a fairly radical departure from how things are.
>
> If that doesn't fly, I think, yeah we could leave Samza as it is and do the
> more radical reboot inside Kafka. From my point of view that does leave
> things in a somewhat confusing state since now there are two stream
> processing systems more or less coupled to Kafka in large part made by the
> same people. But, arguably that might be a cleaner way to make the cut-over
> and perhaps less risky for Samza community since if it works people can
> switch and if it doesn't nothing will have changed. Dunno, how do people
> feel about this?
>
> -Jay
>
> On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com> wrote:
>
> > >  This leads me to thinking that merging projects and communities might
> > be a good idea: with the union of experience from both communities, we
> will
> > probably build a better system that is better for users.
> > Is this what's being proposed though? Merging the projects seems like
> > a consequence of at most one of the three directions under discussion:
> > 1) Samza 2.0: The Samza community relies more heavily on Kafka for
> > configuration, etc. (to a greater or lesser extent to be determined)
> > but the Samza community would not automatically merge withe Kafka
> > community (the Phoenix/HBase example is a good one here).
> > 2) Samza Reboot: The Samza community continues to exist with a limited
> > project scope, but similarly would not need to be part of the Kafka
> > community (ie given committership) to progress.  Here, maybe the Samza
> > team would become a subproject of Kafka (the Board frowns on
> > subprojects at the moment, so I'm not sure if that's even feasible),
> > but that would not be required.
> > 3) Hey Samza! FYI, Kafka does streaming now: In this option the Kafka
> > team builds its own streaming library, possibly off of Jay's
> > prototype, which has not direct lineage to the Samza team.  There's no
> > reason for the Kafka team to bring in the Samza team.
> >
> > Is the Kafka community on board with this?
> >
> > To be clear, all three options under discussion are interesting,
> > technically valid and likely healthy directions for the project.
> > Also, they are not mutually exclusive.  The Samza community could
> > decide to pursue, say, 'Samza 2.0', while the Kafka community went
> > forward with 'Hey Samza!'  My points above are directed entirely at
> > the community aspect of these choices.
> > -Jakob
> >
> > On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com> wrote:
> > > That's great.  Thanks, Jay.
> > >
> > > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > >> Yeah totally agree. I think you have this issue even today, right?
> I.e.
> > if
> > >> you need to make a simple config change and you're running in YARN
> today
> > >> you end up bouncing the job which then rebuilds state. I think the fix
> > is
> > >> exactly what you described which is to have a long timeout on
> partition
> > >> movement for stateful jobs so that if a job is just getting bounced,
> and
> > >> the cluster manager (or admin) is smart enough to restart it on the
> same
> > >> host when possible, it can optimistically reuse any existing state it
> > finds
> > >> on disk (if it is valid).
> > >>
> > >> So in this model the charter of the CM is to place processes as
> > stickily as
> > >> possible and to restart or re-place failed processes. The charter of
> the
> > >> partition management system is to control the assignment of work to
> > these
> > >> processes. The nice thing about this is that the work assignment,
> > timeouts,
> > >> behavior, configs, and code will all be the same across all cluster
> > >> managers.
> > >>
> > >> So I think that prototype would actually give you exactly what you
> want
> > >> today for any cluster manager (or manual placement + restart script)
> > that
> > >> was sticky in terms of host placement since there is already a
> > configurable
> > >> partition movement timeout and task-by-task state reuse with a check
> on
> > >> state validity.
> > >>
> > >> -Jay
> > >>
> > >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <roger.hoover@gmail.com
> >
> > >> wrote:
> > >>
> > >> > That would be great to let Kafka do as much heavy lifting as
> possible
> > and
> > >> > make it easier for other languages to implement Samza apis.
> > >> >
> > >> > One thing to watch out for is the interplay between Kafka's group
> > >> > management and the external scheduler/process manager's fault
> > tolerance.
> > >> > If a container dies, the Kafka group membership protocol will try to
> > >> assign
> > >> > it's tasks to other containers while at the same time the process
> > manager
> > >> > is trying to relaunch the container.  Without some consideration for
> > this
> > >> > (like a configurable amount of time to wait before Kafka alters the
> > group
> > >> > membership), there may be thrashing going on which is especially bad
> > for
> > >> > containers with large amounts of local state.
> > >> >
> > >> > Someone else pointed this out already but I thought it might be
> worth
> > >> > calling out again.
> > >> >
> > >> > Cheers,
> > >> >
> > >> > Roger
> > >> >
> > >> >
> > >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <ja...@confluent.io>
> wrote:
> > >> >
> > >> > > Hey Roger,
> > >> > >
> > >> > > I couldn't agree more. We spent a bunch of time talking to people
> > and
> > >> > that
> > >> > > is exactly the stuff we heard time and again. What makes it hard,
> of
> > >> > > course, is that there is some tension between compatibility with
> > what's
> > >> > > there now and making things better for new users.
> > >> > >
> > >> > > I also strongly agree with the importance of multi-language
> > support. We
> > >> > are
> > >> > > talking now about Java, but for application development use cases
> > >> people
> > >> > > want to work in whatever language they are using elsewhere. I
> think
> > >> > moving
> > >> > > to a model where Kafka itself does the group membership, lifecycle
> > >> > control,
> > >> > > and partition assignment has the advantage of putting all that
> > complex
> > >> > > stuff behind a clean api that the clients are already going to be
> > >> > > implementing for their consumer, so the added functionality for
> > stream
> > >> > > processing beyond a consumer becomes very minor.
> > >> > >
> > >> > > -Jay
> > >> > >
> > >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> > roger.hoover@gmail.com>
> > >> > > wrote:
> > >> > >
> > >> > > > Metamorphosis...nice. :)
> > >> > > >
> > >> > > > This has been a great discussion.  As a user of Samza who's
> > recently
> > >> > > > integrated it into a relatively large organization, I just want
> to
> > >> add
> > >> > > > support to a few points already made.
> > >> > > >
> > >> > > > The biggest hurdles to adoption of Samza as it currently exists
> > that
> > >> > I've
> > >> > > > experienced are:
> > >> > > > 1) YARN - YARN is overly complex in many environments where
> Puppet
> > >> > would
> > >> > > do
> > >> > > > just fine but it was the only mechanism to get fault tolerance.
> > >> > > > 2) Configuration - I think I like the idea of configuring most
> of
> > the
> > >> > job
> > >> > > > in code rather than config files.  In general, I think the goal
> > >> should
> > >> > be
> > >> > > > to make it harder to make mistakes, especially of the kind where
> > the
> > >> > code
> > >> > > > expects something and the config doesn't match.  The current
> > config
> > >> is
> > >> > > > quite intricate and error-prone.  For example, the application
> > logic
> > >> > may
> > >> > > > depend on bootstrapping a topic but rather than asserting that
> in
> > the
> > >> > > code,
> > >> > > > you have to rely on getting the config right.  Likewise with
> > serdes,
> > >> > the
> > >> > > > Java representations produced by various serdes (JSON, Avro,
> etc.)
> > >> are
> > >> > > not
> > >> > > > equivalent so you cannot just reconfigure a serde without
> changing
> > >> the
> > >> > > > code.   It would be nice for jobs to be able to assert what they
> > >> expect
> > >> > > > from their input topics in terms of partitioning.  This is
> > getting a
> > >> > > little
> > >> > > > off topic but I was even thinking about creating a "Samza config
> > >> > linter"
> > >> > > > that would sanity check a set of configs.  Especially in
> > >> organizations
> > >> > > > where config is managed by a different team than the application
> > >> > > developer,
> > >> > > > it's very hard to get avoid config mistakes.
> > >> > > > 3) Java/Scala centric - for many teams (especially DevOps-type
> > >> folks),
> > >> > > the
> > >> > > > pain of the Java toolchain (maven, slow builds, weak command
> line
> > >> > > support,
> > >> > > > configuration over convention) really inhibits productivity.  As
> > more
> > >> > and
> > >> > > > more high-quality clients become available for Kafka, I hope
> > they'll
> > >> > > follow
> > >> > > > Samza's model.  Not sure how much it affects the proposals in
> this
> > >> > thread
> > >> > > > but please consider other languages in the ecosystem as well.
> > From
> > >> > what
> > >> > > > I've heard, Spark has more Python users than Java/Scala.
> > >> > > > (FYI, we added a Jython wrapper for the Samza API
> > >> > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > >> > > > and are working on a Yeoman generator
> > >> > > > https://github.com/Quantiply/generator-rico for Jython/Samza
> > >> projects
> > >> > to
> > >> > > > alleviate some of the pain)
> > >> > > >
> > >> > > > I also want to underscore Jay's point about improving the user
> > >> > > experience.
> > >> > > > That's a very important factor for adoption.  I think the goal
> > should
> > >> > be
> > >> > > to
> > >> > > > make Samza as easy to get started with as something like
> Logstash.
> > >> > > > Logstash is vastly inferior in terms of capabilities to Samza
> but
> > >> it's
> > >> > > easy
> > >> > > > to get started and that makes a big difference.
> > >> > > >
> > >> > > > Cheers,
> > >> > > >
> > >> > > > Roger
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
> > >> > > > gdfm@apache.org> wrote:
> > >> > > >
> > >> > > > > Forgot to add. On the naming issues, Kafka Metamorphosis is a
> > clear
> > >> > > > winner
> > >> > > > > :)
> > >> > > > >
> > >> > > > > --
> > >> > > > > Gianmarco
> > >> > > > >
> > >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
> > >> > > gdfm@apache.org
> > >> > > > >
> > >> > > > > wrote:
> > >> > > > >
> > >> > > > > > Hi,
> > >> > > > > >
> > >> > > > > > @Martin, thanks for you comments.
> > >> > > > > > Maybe I'm missing some important point, but I think coupling
> > the
> > >> > > > releases
> > >> > > > > > is actually a *good* thing.
> > >> > > > > > To make an example, would it be better if the MR and HDFS
> > >> > components
> > >> > > of
> > >> > > > > > Hadoop had different release schedules?
> > >> > > > > >
> > >> > > > > > Actually, keeping the discussion in a single place would
> make
> > >> > > agreeing
> > >> > > > on
> > >> > > > > > releases (and backwards compatibility) much easier, as
> > everybody
> > >> > > would
> > >> > > > be
> > >> > > > > > responsible for the whole codebase.
> > >> > > > > >
> > >> > > > > > That said, I like the idea of absorbing samza-core as a
> > >> > sub-project,
> > >> > > > and
> > >> > > > > > leave the fancy stuff separate.
> > >> > > > > > It probably gives 90% of the benefits we have been
> discussing
> > >> here.
> > >> > > > > >
> > >> > > > > > Cheers,
> > >> > > > > >
> > >> > > > > > --
> > >> > > > > > Gianmarco
> > >> > > > > >
> > >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com>
> > wrote:
> > >> > > > > >
> > >> > > > > >> Hey Martin,
> > >> > > > > >>
> > >> > > > > >> I agree coupling release schedules is a downside.
> > >> > > > > >>
> > >> > > > > >> Definitely we can try to solve some of the integration
> > problems
> > >> in
> > >> > > > > >> Confluent Platform or in other distributions. But I think
> > this
> > >> > ends
> > >> > > up
> > >> > > > > >> being really shallow. I guess I feel to really get a good
> > user
> > >> > > > > experience
> > >> > > > > >> the two systems have to kind of feel like part of the same
> > thing
> > >> > and
> > >> > > > you
> > >> > > > > >> can't really add that in later--you can put both in the
> same
> > >> > > > > downloadable
> > >> > > > > >> tar file but it doesn't really give a very cohesive
> feeling.
> > I
> > >> > agree
> > >> > > > > that
> > >> > > > > >> ultimately any of the project stuff is as much social and
> > naming
> > >> > as
> > >> > > > > >> anything else--theoretically two totally independent
> projects
> > >> > could
> > >> > > > work
> > >> > > > > >> to
> > >> > > > > >> tightly align. In practice this seems to be quite difficult
> > >> > though.
> > >> > > > > >>
> > >> > > > > >> For the frameworks--totally agree it would be good to
> > maintain
> > >> the
> > >> > > > > >> framework support with the project. In some cases there may
> > not
> > >> be
> > >> > > too
> > >> > > > > >> much
> > >> > > > > >> there since the integration gets lighter but I think
> whatever
> > >> > stubs
> > >> > > > you
> > >> > > > > >> need should be included. So no I definitely wasn't trying
> to
> > >> imply
> > >> > > > > >> dropping
> > >> > > > > >> support for these frameworks, just making the integration
> > >> lighter
> > >> > by
> > >> > > > > >> separating process management from partition management.
> > >> > > > > >>
> > >> > > > > >> You raise two good points we would have to figure out if we
> > went
> > >> > > down
> > >> > > > > the
> > >> > > > > >> alignment path:
> > >> > > > > >> 1. With respect to the name, yeah I think the first
> question
> > is
> > >> > > > whether
> > >> > > > > >> some "re-branding" would be worth it. If so then I think we
> > can
> > >> > > have a
> > >> > > > > big
> > >> > > > > >> thread on the name. I'm definitely not set on Kafka
> > Streaming or
> > >> > > Kafka
> > >> > > > > >> Streams I was just using them to be kind of illustrative. I
> > >> agree
> > >> > > with
> > >> > > > > >> your
> > >> > > > > >> critique of these names, though I think people would get
> the
> > >> idea.
> > >> > > > > >> 2. Yeah you also raise a good point about how to "factor"
> it.
> > >> Here
> > >> > > are
> > >> > > > > the
> > >> > > > > >> options I see (I could get enthusiastic about any of them):
> > >> > > > > >>    a. One repo for both Kafka and Samza
> > >> > > > > >>    b. Two repos, retaining the current seperation
> > >> > > > > >>    c. Two repos, the equivalent of samza-api and samza-core
> > is
> > >> > > > absorbed
> > >> > > > > >> almost like a third client
> > >> > > > > >>
> > >> > > > > >> Cheers,
> > >> > > > > >>
> > >> > > > > >> -Jay
> > >> > > > > >>
> > >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> > >> > > > martin@kleppmann.com>
> > >> > > > > >> wrote:
> > >> > > > > >>
> > >> > > > > >> > Ok, thanks for the clarifications. Just a few follow-up
> > >> > comments.
> > >> > > > > >> >
> > >> > > > > >> > - I see the appeal of merging with Kafka or becoming a
> > >> > subproject:
> > >> > > > the
> > >> > > > > >> > reasons you mention are good. The risk I see is that
> > release
> > >> > > > schedules
> > >> > > > > >> > become coupled to each other, which can slow everyone
> down,
> > >> and
> > >> > > > large
> > >> > > > > >> > projects with many contributors are harder to manage.
> > (Jakob,
> > >> > can
> > >> > > > you
> > >> > > > > >> speak
> > >> > > > > >> > from experience, having seen a wider range of Hadoop
> > ecosystem
> > >> > > > > >> projects?)
> > >> > > > > >> >
> > >> > > > > >> > Some of the goals of a better unified developer
> experience
> > >> could
> > >> > > > also
> > >> > > > > be
> > >> > > > > >> > solved by integrating Samza nicely into a Kafka
> > distribution
> > >> > (such
> > >> > > > as
> > >> > > > > >> > Confluent's). I'm not against merging projects if we
> decide
> > >> > that's
> > >> > > > the
> > >> > > > > >> way
> > >> > > > > >> > to go, just pointing out the same goals can perhaps also
> be
> > >> > > achieved
> > >> > > > > in
> > >> > > > > >> > other ways.
> > >> > > > > >> >
> > >> > > > > >> > - With regard to dropping the YARN dependency: are you
> > >> proposing
> > >> > > > that
> > >> > > > > >> > Samza doesn't give any help to people wanting to run on
> > >> > > > > >> YARN/Mesos/AWS/etc?
> > >> > > > > >> > So the docs would basically have a link to Slider and
> > nothing
> > >> > > else?
> > >> > > > Or
> > >> > > > > >> > would we maintain integrations with a bunch of popular
> > >> > deployment
> > >> > > > > >> methods
> > >> > > > > >> > (e.g. the necessary glue and shell scripts to make Samza
> > work
> > >> > with
> > >> > > > > >> Slider)?
> > >> > > > > >> >
> > >> > > > > >> > I absolutely think it's a good idea to have the "as a
> > library"
> > >> > and
> > >> > > > > "as a
> > >> > > > > >> > process" (using Yi's taxonomy) options for people who
> want
> > >> them,
> > >> > > > but I
> > >> > > > > >> > think there should also be a low-friction path for common
> > "as
> > >> a
> > >> > > > > service"
> > >> > > > > >> > deployment methods, for which we probably need to
> maintain
> > >> > > > > integrations.
> > >> > > > > >> >
> > >> > > > > >> > - Project naming: "Kafka Streams" seems odd to me,
> because
> > >> Kafka
> > >> > > is
> > >> > > > > all
> > >> > > > > >> > about streams already. Perhaps "Kafka Transformers" or
> > "Kafka
> > >> > > > Filters"
> > >> > > > > >> > would be more apt?
> > >> > > > > >> >
> > >> > > > > >> > One suggestion: perhaps the core of Samza (stream
> > >> transformation
> > >> > > > with
> > >> > > > > >> > state management -- i.e. the "Samza as a library" bit)
> > could
> > >> > > become
> > >> > > > > >> part of
> > >> > > > > >> > Kafka, while higher-level tools such as streaming SQL and
> > >> > > > integrations
> > >> > > > > >> with
> > >> > > > > >> > deployment frameworks remain in a separate project? In
> > other
> > >> > > words,
> > >> > > > > >> Kafka
> > >> > > > > >> > would absorb the proven, stable core of Samza, which
> would
> > >> > become
> > >> > > > the
> > >> > > > > >> > "third Kafka client" mentioned early in this thread. The
> > Samza
> > >> > > > project
> > >> > > > > >> > would then target that third Kafka client as its base
> API,
> > and
> > >> > the
> > >> > > > > >> project
> > >> > > > > >> > would be freed up to explore more experimental new
> > horizons.
> > >> > > > > >> >
> > >> > > > > >> > Martin
> > >> > > > > >> >
> > >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com>
> > >> wrote:
> > >> > > > > >> >
> > >> > > > > >> > > Hey Martin,
> > >> > > > > >> > >
> > >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually don't
> think
> > it
> > >> > ties
> > >> > > > our
> > >> > > > > >> > hands
> > >> > > > > >> > > at all, all it does is refactor things. The division of
> > >> > > > > >> responsibility is
> > >> > > > > >> > > that Samza core is responsible for task lifecycle,
> state,
> > >> and
> > >> > > > > >> partition
> > >> > > > > >> > > management (using the Kafka co-ordinator) but it is NOT
> > >> > > > responsible
> > >> > > > > >> for
> > >> > > > > >> > > packaging, configuration deployment or execution of
> > >> processes.
> > >> > > The
> > >> > > > > >> > problem
> > >> > > > > >> > > of packaging and starting these processes is
> > >> > > > > >> > > framework/environment-specific. This leaves individual
> > >> > > frameworks
> > >> > > > to
> > >> > > > > >> be
> > >> > > > > >> > as
> > >> > > > > >> > > fancy or vanilla as they like. So you can get simple
> > >> stateless
> > >> > > > > >> support in
> > >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app
> framework
> > >> > > (Slider,
> > >> > > > > >> > Marathon,
> > >> > > > > >> > > etc). These are well known by people and have nice UIs
> > and a
> > >> > lot
> > >> > > > of
> > >> > > > > >> > > flexibility. I don't think they have node affinity as a
> > >> built
> > >> > in
> > >> > > > > >> option
> > >> > > > > >> > > (though I could be wrong). So if we want that we can
> > either
> > >> > wait
> > >> > > > for
> > >> > > > > >> them
> > >> > > > > >> > > to add it or do a custom framework to add that feature
> > (as
> > >> > now).
> > >> > > > > >> > Obviously
> > >> > > > > >> > > if you manage things with old-school ops tools
> > >> > (puppet/chef/etc)
> > >> > > > you
> > >> > > > > >> get
> > >> > > > > >> > > locality easily. The nice thing, though, is that all
> the
> > >> samza
> > >> > > > > >> "business
> > >> > > > > >> > > logic" around partition management and fault tolerance
> > is in
> > >> > > Samza
> > >> > > > > >> core
> > >> > > > > >> > so
> > >> > > > > >> > > it is shared across frameworks and the framework
> specific
> > >> bit
> > >> > is
> > >> > > > > just
> > >> > > > > >> > > whether it is smart enough to try to get the same host
> > when
> > >> a
> > >> > > job
> > >> > > > is
> > >> > > > > >> > > restarted.
> > >> > > > > >> > >
> > >> > > > > >> > > With respect to the Kafka-alignment, yeah I think the
> > goal
> > >> > would
> > >> > > > be
> > >> > > > > >> (a)
> > >> > > > > >> > > actually get better alignment in user experience, and
> (b)
> > >> > > express
> > >> > > > > >> this in
> > >> > > > > >> > > the naming and project branding. Specifically:
> > >> > > > > >> > > 1. Website/docs, it would be nice for the
> > "transformation"
> > >> api
> > >> > > to
> > >> > > > be
> > >> > > > > >> > > discoverable in the main Kafka docs--i.e. be able to
> > explain
> > >> > > when
> > >> > > > to
> > >> > > > > >> use
> > >> > > > > >> > > the consumer and when to use the stream processing
> > >> > functionality
> > >> > > > and
> > >> > > > > >> lead
> > >> > > > > >> > > people into that experience.
> > >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or
> > whatever)
> > >> > that
> > >> > > > has
> > >> > > > > >> both
> > >> > > > > >> > > Kafka and the stream processing part and they actually
> > work
> > >> > > > > together.
> > >> > > > > >> > > 3. Unify the programming experience so the client and
> > Samza
> > >> > api
> > >> > > > > share
> > >> > > > > >> > > config/monitoring/naming/packaging/etc.
> > >> > > > > >> > >
> > >> > > > > >> > > I think sub-projects keep separate committers and can
> > have a
> > >> > > > > separate
> > >> > > > > >> > repo,
> > >> > > > > >> > > but I'm actually not really sure (I can't find a
> > definition
> > >> > of a
> > >> > > > > >> > subproject
> > >> > > > > >> > > in Apache).
> > >> > > > > >> > >
> > >> > > > > >> > > Basically at a high-level you want the experience to
> > "feel"
> > >> > > like a
> > >> > > > > >> single
> > >> > > > > >> > > system, not to relatively independent things that are
> > kind
> > >> of
> > >> > > > > >> awkwardly
> > >> > > > > >> > > glued together.
> > >> > > > > >> > >
> > >> > > > > >> > > I think if we did that they having naming or branding
> > like
> > >> > > "kafka
> > >> > > > > >> > > streaming" or "kafka streams" or something like that
> > would
> > >> > > > actually
> > >> > > > > >> do a
> > >> > > > > >> > > good job of conveying what it is. I do that this would
> > help
> > >> > > > adoption
> > >> > > > > >> > quite
> > >> > > > > >> > > a lot as it would correctly convey that using Kafka
> > >> Streaming
> > >> > > with
> > >> > > > > >> Kafka
> > >> > > > > >> > is
> > >> > > > > >> > > a fairly seamless experience and Kafka is pretty
> heavily
> > >> > adopted
> > >> > > > at
> > >> > > > > >> this
> > >> > > > > >> > > point.
> > >> > > > > >> > >
> > >> > > > > >> > > Fwiw we actually considered this model originally when
> > open
> > >> > > > sourcing
> > >> > > > > >> > Samza,
> > >> > > > > >> > > however at that time Kafka was relatively unknown and
> we
> > >> > decided
> > >> > > > not
> > >> > > > > >> to
> > >> > > > > >> > do
> > >> > > > > >> > > it since we felt it would be limiting. From my point of
> > view
> > >> > the
> > >> > > > > three
> > >> > > > > >> > > things have changed (1) Kafka is now really heavily
> used
> > for
> > >> > > > stream
> > >> > > > > >> > > processing, (2) we learned that abstracting out the
> > stream
> > >> > well
> > >> > > is
> > >> > > > > >> > > basically impossible, (3) we learned it is really hard
> to
> > >> keep
> > >> > > the
> > >> > > > > two
> > >> > > > > >> > > things feeling like a single product.
> > >> > > > > >> > >
> > >> > > > > >> > > -Jay
> > >> > > > > >> > >
> > >> > > > > >> > >
> > >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> > >> > > > > >> martin@kleppmann.com>
> > >> > > > > >> > > wrote:
> > >> > > > > >> > >
> > >> > > > > >> > >> Hi all,
> > >> > > > > >> > >>
> > >> > > > > >> > >> Lots of good thoughts here.
> > >> > > > > >> > >>
> > >> > > > > >> > >> I agree with the general philosophy of tying Samza
> more
> > >> > firmly
> > >> > > to
> > >> > > > > >> Kafka.
> > >> > > > > >> > >> After I spent a while looking at integrating other
> > message
> > >> > > > brokers
> > >> > > > > >> (e.g.
> > >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the conclusion
> > that
> > >> > > > > >> > SystemConsumer
> > >> > > > > >> > >> tacitly assumes a model so much like Kafka's that
> pretty
> > >> much
> > >> > > > > nobody
> > >> > > > > >> but
> > >> > > > > >> > >> Kafka actually implements it. (Databus is perhaps an
> > >> > exception,
> > >> > > > but
> > >> > > > > >> it
> > >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus, making
> > Samza
> > >> > > fully
> > >> > > > > >> > dependent
> > >> > > > > >> > >> on Kafka acknowledges that the system-independence was
> > >> never
> > >> > as
> > >> > > > > real
> > >> > > > > >> as
> > >> > > > > >> > we
> > >> > > > > >> > >> perhaps made it out to be. The gains of code reuse are
> > >> real.
> > >> > > > > >> > >>
> > >> > > > > >> > >> The idea of decoupling Samza from YARN has also always
> > been
> > >> > > > > >> appealing to
> > >> > > > > >> > >> me, for various reasons already mentioned in this
> > thread.
> > >> > > > Although
> > >> > > > > >> > making
> > >> > > > > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc)
> > >> seems
> > >> > > > > >> laudable,
> > >> > > > > >> > I am
> > >> > > > > >> > >> a little concerned that it will restrict us to a
> lowest
> > >> > common
> > >> > > > > >> > denominator.
> > >> > > > > >> > >> For example, would host affinity (SAMZA-617) still be
> > >> > possible?
> > >> > > > For
> > >> > > > > >> jobs
> > >> > > > > >> > >> with large amounts of state, I think SAMZA-617 would
> be
> > a
> > >> big
> > >> > > > boon,
> > >> > > > > >> > since
> > >> > > > > >> > >> restoring state off the changelog on every single
> > restart
> > >> is
> > >> > > > > painful,
> > >> > > > > >> > due
> > >> > > > > >> > >> to long recovery times. It would be a shame if the
> > >> decoupling
> > >> > > > from
> > >> > > > > >> YARN
> > >> > > > > >> > >> made host affinity impossible.
> > >> > > > > >> > >>
> > >> > > > > >> > >> Jay, a question about the proposed API for
> > instantiating a
> > >> > job
> > >> > > in
> > >> > > > > >> code
> > >> > > > > >> > >> (rather than a properties file): when submitting a job
> > to a
> > >> > > > > cluster,
> > >> > > > > >> is
> > >> > > > > >> > the
> > >> > > > > >> > >> idea that the instantiation code runs on a client
> > >> somewhere,
> > >> > > > which
> > >> > > > > >> then
> > >> > > > > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc?
> Or
> > >> does
> > >> > > that
> > >> > > > > >> code
> > >> > > > > >> > run
> > >> > > > > >> > >> on each container that is part of the job (in which
> > case,
> > >> how
> > >> > > > does
> > >> > > > > >> the
> > >> > > > > >> > job
> > >> > > > > >> > >> submission to the cluster work)?
> > >> > > > > >> > >>
> > >> > > > > >> > >> I agree with Garry that it doesn't feel right to make
> a
> > 1.0
> > >> > > > release
> > >> > > > > >> > with a
> > >> > > > > >> > >> plan for it to be immediately obsolete. So if this is
> > going
> > >> > to
> > >> > > > > >> happen, I
> > >> > > > > >> > >> think it would be more honest to stick with 0.*
> version
> > >> > numbers
> > >> > > > > until
> > >> > > > > >> > the
> > >> > > > > >> > >> library-ified Samza has been implemented, is stable
> and
> > >> > widely
> > >> > > > > used.
> > >> > > > > >> > >>
> > >> > > > > >> > >> Should the new Samza be a subproject of Kafka? There
> is
> > >> > > precedent
> > >> > > > > for
> > >> > > > > >> > >> tight coupling between different Apache projects (e.g.
> > >> > Curator
> > >> > > > and
> > >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think remaining
> > >> separate
> > >> > > > would
> > >> > > > > >> be
> > >> > > > > >> > ok.
> > >> > > > > >> > >> Even if Samza is fully dependent on Kafka, there is
> > enough
> > >> > > > > substance
> > >> > > > > >> in
> > >> > > > > >> > >> Samza that it warrants being a separate project. An
> > >> argument
> > >> > in
> > >> > > > > >> favour
> > >> > > > > >> > of
> > >> > > > > >> > >> merging would be if we think Kafka has a much stronger
> > >> "brand
> > >> > > > > >> presence"
> > >> > > > > >> > >> than Samza; I'm ambivalent on that one. If the Kafka
> > >> project
> > >> > is
> > >> > > > > >> willing
> > >> > > > > >> > to
> > >> > > > > >> > >> endorse Samza as the "official" way of doing stateful
> > >> stream
> > >> > > > > >> > >> transformations, that would probably have much the
> same
> > >> > effect
> > >> > > as
> > >> > > > > >> > >> re-branding Samza as "Kafka Stream Processors" or
> > suchlike.
> > >> > > Close
> > >> > > > > >> > >> collaboration between the two projects will be needed
> in
> > >> any
> > >> > > > case.
> > >> > > > > >> > >>
> > >> > > > > >> > >> From a project management perspective, I guess the
> "new
> > >> > Samza"
> > >> > > > > would
> > >> > > > > >> > have
> > >> > > > > >> > >> to be developed on a branch alongside ongoing
> > maintenance
> > >> of
> > >> > > the
> > >> > > > > >> current
> > >> > > > > >> > >> line of development? I think it would be important to
> > >> > continue
> > >> > > > > >> > supporting
> > >> > > > > >> > >> existing users, and provide a graceful migration path
> to
> > >> the
> > >> > > new
> > >> > > > > >> > version.
> > >> > > > > >> > >> Leaving the current versions unsupported and forcing
> > people
> > >> > to
> > >> > > > > >> rewrite
> > >> > > > > >> > >> their jobs would send a bad signal.
> > >> > > > > >> > >>
> > >> > > > > >> > >> Best,
> > >> > > > > >> > >> Martin
> > >> > > > > >> > >>
> > >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io>
> > >> wrote:
> > >> > > > > >> > >>
> > >> > > > > >> > >>> Hey Garry,
> > >> > > > > >> > >>>
> > >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy to chat
> > more
> > >> > about
> > >> > > > > this
> > >> > > > > >> if
> > >> > > > > >> > >>> you'd be interested. I think Chris and I started with
> > the
> > >> > idea
> > >> > > > of
> > >> > > > > >> "what
> > >> > > > > >> > >>> would it take to make Samza a kick-ass ingestion
> tool"
> > but
> > >> > > > > >> ultimately
> > >> > > > > >> > we
> > >> > > > > >> > >>> kind of came around to the idea that ingestion and
> > >> > > > transformation
> > >> > > > > >> had
> > >> > > > > >> > >>> pretty different needs and coupling the two made
> things
> > >> > hard.
> > >> > > > > >> > >>>
> > >> > > > > >> > >>> For what it's worth I think copycat (KIP-26) actually
> > will
> > >> > do
> > >> > > > what
> > >> > > > > >> you
> > >> > > > > >> > >> are
> > >> > > > > >> > >>> looking for.
> > >> > > > > >> > >>>
> > >> > > > > >> > >>> With regard to your point about slider, I don't
> > >> necessarily
> > >> > > > > >> disagree.
> > >> > > > > >> > >> But I
> > >> > > > > >> > >>> think getting good YARN support is quite doable and I
> > >> think
> > >> > we
> > >> > > > can
> > >> > > > > >> make
> > >> > > > > >> > >>> that work well. I think the issue this proposal
> solves
> > is
> > >> > that
> > >> > > > > >> > >> technically
> > >> > > > > >> > >>> it is pretty hard to support multiple cluster
> > management
> > >> > > systems
> > >> > > > > the
> > >> > > > > >> > way
> > >> > > > > >> > >>> things are now, you need to write an "app master" or
> > >> > > "framework"
> > >> > > > > for
> > >> > > > > >> > each
> > >> > > > > >> > >>> and they are all a little different so testing is
> > really
> > >> > hard.
> > >> > > > In
> > >> > > > > >> the
> > >> > > > > >> > >>> absence of this we have been stuck with just YARN
> which
> > >> has
> > >> > > > > >> fantastic
> > >> > > > > >> > >>> penetration in the Hadoopy part of the org, but zero
> > >> > > penetration
> > >> > > > > >> > >> elsewhere.
> > >> > > > > >> > >>> Given the huge amount of work being put in to slider,
> > >> > > marathon,
> > >> > > > > aws
> > >> > > > > >> > >>> tooling, not to mention the umpteen related packaging
> > >> > > > technologies
> > >> > > > > >> > people
> > >> > > > > >> > >>> want to use (Docker, Kubernetes, various
> cloud-specific
> > >> > deploy
> > >> > > > > >> tools,
> > >> > > > > >> > >> etc)
> > >> > > > > >> > >>> I really think it is important to get this right.
> > >> > > > > >> > >>>
> > >> > > > > >> > >>> -Jay
> > >> > > > > >> > >>>
> > >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > >> > > > > >> > >>>
> > >> > > > > >> > >>>> Hi all,
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>> I think the question below re does Samza become a
> > >> > sub-project
> > >> > > > of
> > >> > > > > >> Kafka
> > >> > > > > >> > >>>> highlights the broader point around migration. Chris
> > >> > mentions
> > >> > > > > >> Samza's
> > >> > > > > >> > >>>> maturity is heading towards a v1 release but I'm not
> > sure
> > >> > it
> > >> > > > > feels
> > >> > > > > >> > >> right to
> > >> > > > > >> > >>>> launch a v1 then immediately plan to deprecate most
> of
> > >> it.
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>> From a selfish perspective I have some guys who have
> > >> > started
> > >> > > > > >> working
> > >> > > > > >> > >> with
> > >> > > > > >> > >>>> Samza and building some new consumers/producers was
> > next
> > >> > up.
> > >> > > > > Sounds
> > >> > > > > >> > like
> > >> > > > > >> > >>>> that is absolutely not the direction to go. I need
> to
> > >> look
> > >> > > into
> > >> > > > > the
> > >> > > > > >> > KIP
> > >> > > > > >> > >> in
> > >> > > > > >> > >>>> more detail but for me the attractiveness of adding
> > new
> > >> > Samza
> > >> > > > > >> > >>>> consumer/producers -- even if yes all they were
> doing
> > was
> > >> > > > really
> > >> > > > > >> > getting
> > >> > > > > >> > >>>> data into and out of Kafka --  was to avoid  having
> to
> > >> > worry
> > >> > > > > about
> > >> > > > > >> the
> > >> > > > > >> > >>>> lifecycle management of external clients. If there
> is
> > a
> > >> > > generic
> > >> > > > > >> Kafka
> > >> > > > > >> > >>>> ingress/egress layer that I can plug a new connector
> > into
> > >> > and
> > >> > > > > have
> > >> > > > > >> a
> > >> > > > > >> > >> lot of
> > >> > > > > >> > >>>> the heavy lifting re scale and reliability done for
> me
> > >> then
> > >> > > it
> > >> > > > > >> gives
> > >> > > > > >> > me
> > >> > > > > >> > >> all
> > >> > > > > >> > >>>> the pushing new consumers/producers would. If not
> > then it
> > >> > > > > >> complicates
> > >> > > > > >> > my
> > >> > > > > >> > >>>> operational deployments.
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>> Which is similar to my other question with the
> > proposal
> > >> --
> > >> > if
> > >> > > > we
> > >> > > > > >> > build a
> > >> > > > > >> > >>>> fully available/stand-alone Samza plus the requisite
> > >> shims
> > >> > to
> > >> > > > > >> > integrate
> > >> > > > > >> > >>>> with Slider etc I suspect the former may be a lot
> more
> > >> work
> > >> > > > than
> > >> > > > > we
> > >> > > > > >> > >> think.
> > >> > > > > >> > >>>> We may make it much easier for a newcomer to get
> > >> something
> > >> > > > > running
> > >> > > > > >> but
> > >> > > > > >> > >>>> having them step up and get a reliable production
> > >> > deployment
> > >> > > > may
> > >> > > > > >> still
> > >> > > > > >> > >>>> dominate mailing list  traffic, if for different
> > reasons
> > >> > than
> > >> > > > > >> today.
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with making
> the
> > >> Samza
> > >> > > > > >> dependency
> > >> > > > > >> > >> on
> > >> > > > > >> > >>>> Kafka much more explicit and I absolutely see the
> > >> benefits
> > >> > > in
> > >> > > > > the
> > >> > > > > >> > >>>> reduction of duplication and clashing
> > >> > > > terminologies/abstractions
> > >> > > > > >> that
> > >> > > > > >> > >>>> Chris/Jay describe. Samza as a library would likely
> > be a
> > >> > very
> > >> > > > > nice
> > >> > > > > >> > tool
> > >> > > > > >> > >> to
> > >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the concerns
> > >> above
> > >> > re
> > >> > > > the
> > >> > > > > >> > >>>> operational side.
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>> Garry
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>> -----Original Message-----
> > >> > > > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
> > >> > gdfm@apache.org
> > >> > > ]
> > >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > >> > > > > >> > >>>> To: dev@samza.apache.org
> > >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>> Very interesting thoughts.
> > >> > > > > >> > >>>> From outside, I have always perceived Samza as a
> > >> computing
> > >> > > > layer
> > >> > > > > >> over
> > >> > > > > >> > >>>> Kafka.
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>> The question, maybe a bit provocative, is "should
> > Samza
> > >> be
> > >> > a
> > >> > > > > >> > sub-project
> > >> > > > > >> > >>>> of Kafka then?"
> > >> > > > > >> > >>>> Or does it make sense to keep it as a separate
> project
> > >> > with a
> > >> > > > > >> separate
> > >> > > > > >> > >>>> governance?
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>> Cheers,
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>> --
> > >> > > > > >> > >>>> Gianmarco
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> > yanfang724@gmail.com>
> > >> > > > wrote:
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more tightly.
> > >> > Because
> > >> > > > > Samza
> > >> > > > > >> de
> > >> > > > > >> > >>>>> facto is based on Kafka, and it should leverage
> what
> > >> Kafka
> > >> > > > has.
> > >> > > > > At
> > >> > > > > >> > the
> > >> > > > > >> > >>>>> same time, Kafka does not need to reinvent what
> Samza
> > >> > > already
> > >> > > > > >> has. I
> > >> > > > > >> > >>>>> also like the idea of separating the ingestion and
> > >> > > > > transformation.
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>> But it is a little difficult for me to image how
> the
> > >> Samza
> > >> > > > will
> > >> > > > > >> look
> > >> > > > > >> > >>>> like.
> > >> > > > > >> > >>>>> And I feel Chris and Jay have a little difference
> in
> > >> terms
> > >> > > of
> > >> > > > > how
> > >> > > > > >> > >>>>> Samza should look like.
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>> *** Will it look like what Jay's code shows (A
> > client of
> > >> > > > Kakfa)
> > >> > > > > ?
> > >> > > > > >> And
> > >> > > > > >> > >>>>> user's application code calls this client?
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka (like
> what
> > the
> > >> > > code
> > >> > > > > >> shows),
> > >> > > > > >> > >>>>> how do we implement auto-balance and
> fault-tolerance?
> > >> Are
> > >> > > they
> > >> > > > > >> taken
> > >> > > > > >> > >>>>> care by the Kafka broker or other mechanism, such
> as
> > >> > "Samza
> > >> > > > > >> worker"
> > >> > > > > >> > >>>>> (just make up the name) ?
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>> 2. What about other features, such as auto-scaling,
> > >> shared
> > >> > > > > state,
> > >> > > > > >> > >>>>> monitoring?
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>> *** If we have Samza standalone, (is this what
> Chris
> > >> > > > suggests?)
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa and
> > produce
> > >> to
> > >> > > it.
> > >> > > > > >> Then it
> > >> > > > > >> > >>>>> becomes the same as what Samza looks like now,
> > except it
> > >> > > does
> > >> > > > > not
> > >> > > > > >> > rely
> > >> > > > > >> > >>>>> on Yarn anymore.
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's
> > >> > metrics,
> > >> > > > > logs,
> > >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>> Thanks,
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>> Fang, Yan
> > >> > > > > >> > >>>>> yanfang724@gmail.com
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > >> > > > > wangguoz@gmail.com
> > >> > > > > >> >
> > >> > > > > >> > >>>> wrote:
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>>> Read through the code example and it looks good to
> > me.
> > >> A
> > >> > > few
> > >> > > > > >> > >>>>>> thoughts regarding deployment:
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>> Today Samza deploys as executable runnable like:
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> > >> > > > > >> > >>>> --config-path=file://...
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>> And this proposal advocate for deploying Samza
> more
> > as
> > >> > > > embedded
> > >> > > > > >> > >>>>>> libraries in user application code (ignoring the
> > >> > > terminology
> > >> > > > > >> since
> > >> > > > > >> > >>>>>> it is not the
> > >> > > > > >> > >>>>> same
> > >> > > > > >> > >>>>>> as the prototype code):
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>> StreamTask task = new MyStreamTask(configs);
> Thread
> > >> > thread
> > >> > > =
> > >> > > > > new
> > >> > > > > >> > >>>>>> Thread(task); thread.start();
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>> I think both of these deployment modes are
> important
> > >> for
> > >> > > > > >> different
> > >> > > > > >> > >>>>>> types
> > >> > > > > >> > >>>>> of
> > >> > > > > >> > >>>>>> users. That said, I think making Samza purely
> > >> standalone
> > >> > is
> > >> > > > > still
> > >> > > > > >> > >>>>>> sufficient for either runnable or library modes.
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>> Guozhang
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> > >> > > > jay@confluent.io>
> > >> > > > > >> > wrote:
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>>> Looks like gmail mangled the code example, it was
> > >> > supposed
> > >> > > > to
> > >> > > > > >> look
> > >> > > > > >> > >>>>>>> like
> > >> > > > > >> > >>>>>>> this:
> > >> > > > > >> > >>>>>>>
> > >> > > > > >> > >>>>>>> Properties props = new Properties();
> > >> > > > > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > >> > > > > >> StreamingConfig
> > >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> > >> > > > > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> > >> > > > > >> > >>>>>>> config.serialization(new StringSerializer(), new
> > >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming container =
> > new
> > >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
> > >> > > > > >> > >>>>>>>
> > >> > > > > >> > >>>>>>> -Jay
> > >> > > > > >> > >>>>>>>
> > >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> > >> > > > jay@confluent.io
> > >> > > > > >
> > >> > > > > >> > >>>> wrote:
> > >> > > > > >> > >>>>>>>
> > >> > > > > >> > >>>>>>>> Hey guys,
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> This came out of some conversations Chris and I
> > were
> > >> > > having
> > >> > > > > >> > >>>>>>>> around
> > >> > > > > >> > >>>>>>> whether
> > >> > > > > >> > >>>>>>>> it would make sense to use Samza as a kind of
> data
> > >> > > > ingestion
> > >> > > > > >> > >>>>> framework
> > >> > > > > >> > >>>>>>> for
> > >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26
> "copycat").
> > >> This
> > >> > > > kind
> > >> > > > > of
> > >> > > > > >> > >>>>>> combined
> > >> > > > > >> > >>>>>>>> with complaints around config and YARN and the
> > >> > discussion
> > >> > > > > >> around
> > >> > > > > >> > >>>>>>>> how
> > >> > > > > >> > >>>>> to
> > >> > > > > >> > >>>>>>>> best do a standalone mode.
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> So the thought experiment was, given that Samza
> > was
> > >> > > > basically
> > >> > > > > >> > >>>>>>>> already totally Kafka specific, what if you just
> > >> > embraced
> > >> > > > > that
> > >> > > > > >> > >>>>>>>> and turned it
> > >> > > > > >> > >>>>>> into
> > >> > > > > >> > >>>>>>>> something less like a heavyweight framework and
> > more
> > >> > > like a
> > >> > > > > >> > >>>>>>>> third
> > >> > > > > >> > >>>>> Kafka
> > >> > > > > >> > >>>>>>>> client--a kind of "producing consumer" with
> state
> > >> > > > management
> > >> > > > > >> > >>>>>> facilities.
> > >> > > > > >> > >>>>>>>> Basically a library. Instead of a complex stream
> > >> > > processing
> > >> > > > > >> > >>>>>>>> framework
> > >> > > > > >> > >>>>>>> this
> > >> > > > > >> > >>>>>>>> would actually be a very simple thing, not much
> > more
> > >> > > > > >> complicated
> > >> > > > > >> > >>>>>>>> to
> > >> > > > > >> > >>>>> use
> > >> > > > > >> > >>>>>>> or
> > >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we
> > >> thought
> > >> > > > about
> > >> > > > > >> it
> > >> > > > > >> > >>>>>>>> a
> > >> > > > > >> > >>>>> lot
> > >> > > > > >> > >>>>>> of
> > >> > > > > >> > >>>>>>>> what Samza (and the other stream processing
> > systems
> > >> > were
> > >> > > > > doing)
> > >> > > > > >> > >>>>> seemed
> > >> > > > > >> > >>>>>>> like
> > >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> Of course you need to ingest/output data to and
> > from
> > >> > the
> > >> > > > > stream
> > >> > > > > >> > >>>>>>>> processing. But when we actually looked into how
> > that
> > >> > > would
> > >> > > > > >> > >>>>>>>> work,
> > >> > > > > >> > >>>>> Samza
> > >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion framework
> > for a
> > >> > > bunch
> > >> > > > of
> > >> > > > > >> > >>>>> reasons.
> > >> > > > > >> > >>>>>> To
> > >> > > > > >> > >>>>>>>> really do that right you need a pretty different
> > >> > internal
> > >> > > > > data
> > >> > > > > >> > >>>>>>>> model
> > >> > > > > >> > >>>>>> and
> > >> > > > > >> > >>>>>>>> set of apis. So what if you split them and had
> an
> > api
> > >> > for
> > >> > > > > Kafka
> > >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a
> separate
> > >> api
> > >> > > for
> > >> > > > > >> Kafka
> > >> > > > > >> > >>>>>>>> transformation (Samza).
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> This would also allow really embracing the same
> > >> > > terminology
> > >> > > > > and
> > >> > > > > >> > >>>>>>>> conventions. One complaint about the current
> > state is
> > >> > > that
> > >> > > > > the
> > >> > > > > >> > >>>>>>>> two
> > >> > > > > >> > >>>>>>> systems
> > >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like
> "stream"
> > vs
> > >> > > > "topic"
> > >> > > > > >> and
> > >> > > > > >> > >>>>>>> different
> > >> > > > > >> > >>>>>>>> config and monitoring systems means you kind of
> > have
> > >> to
> > >> > > > learn
> > >> > > > > >> > >>>>>>>> Kafka's
> > >> > > > > >> > >>>>>>> way,
> > >> > > > > >> > >>>>>>>> then learn Samza's slightly different way, then
> > kind
> > >> of
> > >> > > > > >> > >>>>>>>> understand
> > >> > > > > >> > >>>>> how
> > >> > > > > >> > >>>>>>> they
> > >> > > > > >> > >>>>>>>> map to each other, which having walked a few
> > people
> > >> > > through
> > >> > > > > >> this
> > >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> Since I have been spending a lot of time on
> > >> airplanes I
> > >> > > > > hacked
> > >> > > > > >> > >>>>>>>> up an ernest but still somewhat incomplete
> > prototype
> > >> of
> > >> > > > what
> > >> > > > > >> > >>>>>>>> this would
> > >> > > > > >> > >>>>> look
> > >> > > > > >> > >>>>>>>> like. This is just unceremoniously dumped into
> > Kafka
> > >> as
> > >> > > it
> > >> > > > > >> > >>>>>>>> required a
> > >> > > > > >> > >>>>>> few
> > >> > > > > >> > >>>>>>>> changes to the new consumer. Here is the code:
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>
> > >> > > > > >> >
> > >> > > > >
> > >> >
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> For the purpose of the prototype I just
> liberally
> > >> > renamed
> > >> > > > > >> > >>>>>>>> everything
> > >> > > > > >> > >>>>> to
> > >> > > > > >> > >>>>>>>> try to align it with Kafka with no regard for
> > >> > > > compatibility.
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> To use this would be something like this:
> > >> > > > > >> > >>>>>>>> Properties props = new Properties();
> > >> > > > > >> > >>>>>>>> props.put("bootstrap.servers",
> "localhost:4242");
> > >> > > > > >> > >>>>>>>> StreamingConfig config = new
> > >> > > > > >> > >>>>> StreamingConfig(props);
> > >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > > > > >> > >>>>>>>> "test-topic-2");
> > >> > > > > >> config.processor(ExampleStreamProcessor.class);
> > >> > > > > >> > >>>>>>> config.serialization(new
> > >> > > > > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
> > >> > > > KafkaStreaming
> > >> > > > > >> > >>>>>> container =
> > >> > > > > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> > >> > > > > StreamProcessor
> > >> > > > > >> > >>>>>>>> is basically StreamTask.
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> So rather than putting all the class names in a
> > file
> > >> > and
> > >> > > > then
> > >> > > > > >> > >>>>>>>> having
> > >> > > > > >> > >>>>>> the
> > >> > > > > >> > >>>>>>>> job assembled by reflection, you just
> instantiate
> > the
> > >> > > > > container
> > >> > > > > >> > >>>>>>>> programmatically. Work is balanced over however
> > many
> > >> > > > > instances
> > >> > > > > >> > >>>>>>>> of
> > >> > > > > >> > >>>>> this
> > >> > > > > >> > >>>>>>> are
> > >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new
> > >> tasks
> > >> > > are
> > >> > > > > >> added
> > >> > > > > >> > >>>>>>>> to
> > >> > > > > >> > >>>>> the
> > >> > > > > >> > >>>>>>>> existing containers without shutting them down).
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> We would provide some glue for running this
> stuff
> > in
> > >> > YARN
> > >> > > > via
> > >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some
> of
> > >> their
> > >> > > > tools
> > >> > > > > >> > >>>>>>>> but from the
> > >> > > > > >> > >>>>>> point
> > >> > > > > >> > >>>>>>> of
> > >> > > > > >> > >>>>>>>> view of these frameworks these stream processing
> > jobs
> > >> > are
> > >> > > > > just
> > >> > > > > >> > >>>>>> stateless
> > >> > > > > >> > >>>>>>>> services that can come and go and expand and
> > contract
> > >> > at
> > >> > > > > will.
> > >> > > > > >> > >>>>>>>> There
> > >> > > > > >> > >>>>> is
> > >> > > > > >> > >>>>>>> no
> > >> > > > > >> > >>>>>>>> more custom scheduler.
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> Here are some relevant details:
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get
> > >> larger
> > >> > > if
> > >> > > > we
> > >> > > > > >> > >>>>>>>>  productionized but not vastly larger. We really
> > do
> > >> > get a
> > >> > > > ton
> > >> > > > > >> > >>>>>>>> of
> > >> > > > > >> > >>>>>>> leverage
> > >> > > > > >> > >>>>>>>>  out of Kafka.
> > >> > > > > >> > >>>>>>>>  2. Partition management is fully delegated to
> the
> > >> new
> > >> > > > > >> consumer.
> > >> > > > > >> > >>>>> This
> > >> > > > > >> > >>>>>>>>  is nice since now any partition management
> > strategy
> > >> > > > > available
> > >> > > > > >> > >>>>>>>> to
> > >> > > > > >> > >>>>>> Kafka
> > >> > > > > >> > >>>>>>>>  consumer is also available to Samza (and vice
> > versa)
> > >> > and
> > >> > > > > with
> > >> > > > > >> > >>>>>>>> the
> > >> > > > > >> > >>>>>>> exact
> > >> > > > > >> > >>>>>>>>  same configs.
> > >> > > > > >> > >>>>>>>>  3. It supports state as well as state reuse
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is thought
> > >> provoking.
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> -Jay
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris
> Riccomini <
> > >> > > > > >> > >>>>>> criccomini@apache.org>
> > >> > > > > >> > >>>>>>>> wrote:
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>>> Hey all,
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> I have had some discussions with Samza
> engineers
> > at
> > >> > > > LinkedIn
> > >> > > > > >> > >>>>>>>>> and
> > >> > > > > >> > >>>>>>> Confluent
> > >> > > > > >> > >>>>>>>>> and we came up with a few observations and
> would
> > >> like
> > >> > to
> > >> > > > > >> > >>>>>>>>> propose
> > >> > > > > >> > >>>>> some
> > >> > > > > >> > >>>>>>>>> changes.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> We've observed some things that I want to call
> > out
> > >> > about
> > >> > > > > >> > >>>>>>>>> Samza's
> > >> > > > > >> > >>>>>> design,
> > >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment
> > >> system.
> > >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > >> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and
> > Kafka's
> > >> > > > consumer
> > >> > > > > >> > >>>>>>>>> APIs
> > >> > > > > >> > >>>>> are
> > >> > > > > >> > >>>>>>>>> trying to solve a lot of the same problems.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> All three of these issues are related, but I'll
> > >> > address
> > >> > > > them
> > >> > > > > >> in
> > >> > > > > >> > >>>>> order.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Deployment
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic
> > >> > > deployment
> > >> > > > > >> > >>>>>>>>> scheduler
> > >> > > > > >> > >>>>>> such
> > >> > > > > >> > >>>>>>>>> as
> > >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built
> Samza,
> > we
> > >> > bet
> > >> > > > that
> > >> > > > > >> > >>>>>>>>> there
> > >> > > > > >> > >>>>>> would
> > >> > > > > >> > >>>>>>>>> be
> > >> > > > > >> > >>>>>>>>> one or two winners in this area, and we could
> > >> support
> > >> > > > them,
> > >> > > > > >> and
> > >> > > > > >> > >>>>>>>>> the
> > >> > > > > >> > >>>>>> rest
> > >> > > > > >> > >>>>>>>>> would go away. In reality, there are many
> > >> variations.
> > >> > > > > >> > >>>>>>>>> Furthermore,
> > >> > > > > >> > >>>>>> many
> > >> > > > > >> > >>>>>>>>> people still prefer to just start their
> > processors
> > >> > like
> > >> > > > > normal
> > >> > > > > >> > >>>>>>>>> Java processes, and use traditional deployment
> > >> scripts
> > >> > > > such
> > >> > > > > as
> > >> > > > > >> > >>>>>>>>> Fabric,
> > >> > > > > >> > >>>>>> Chef,
> > >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on
> > users
> > >> > makes
> > >> > > > the
> > >> > > > > >> > >>>>>>>>> Samza start-up process really painful for first
> > time
> > >> > > > users.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was also a
> > bit
> > >> of
> > >> > a
> > >> > > > > >> > >>>>>>>>> mis-fire
> > >> > > > > >> > >>>>>> because
> > >> > > > > >> > >>>>>>>>> of
> > >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between the
> > nature of
> > >> > > batch
> > >> > > > > >> jobs
> > >> > > > > >> > >>>>>>>>> and
> > >> > > > > >> > >>>>>>> stream
> > >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made conscious
> > effort
> > >> to
> > >> > > > favor
> > >> > > > > >> > >>>>>>>>> the
> > >> > > > > >> > >>>>>> Hadoop
> > >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it
> worked
> > >> and
> > >> > > was
> > >> > > > > well
> > >> > > > > >> > >>>>>>> understood.
> > >> > > > > >> > >>>>>>>>> One thing that we missed was that batch jobs
> > have a
> > >> > > > definite
> > >> > > > > >> > >>>>>> beginning,
> > >> > > > > >> > >>>>>>>>> and
> > >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't
> (usually).
> > >> This
> > >> > > > leads
> > >> > > > > to
> > >> > > > > >> > >>>>>>>>> a
> > >> > > > > >> > >>>>> much
> > >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream
> processors.
> > >> You
> > >> > > > > >> basically
> > >> > > > > >> > >>>>>>>>> just
> > >> > > > > >> > >>>>>>> need
> > >> > > > > >> > >>>>>>>>> to find a place to start the processor, and
> start
> > >> it.
> > >> > > The
> > >> > > > > way
> > >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept
> of
> > a
> > >> > > cluster
> > >> > > > > >> > >>>>>>>>> being "full". We always
> > >> > > > > >> > >>>>>> add
> > >> > > > > >> > >>>>>>>>> more machines. The problem with coupling Samza
> > with
> > >> a
> > >> > > > > >> scheduler
> > >> > > > > >> > >>>>>>>>> is
> > >> > > > > >> > >>>>>> that
> > >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to handle
> > deployment.
> > >> > > This
> > >> > > > > >> pulls
> > >> > > > > >> > >>>>>>>>> in a
> > >> > > > > >> > >>>>>>> bunch
> > >> > > > > >> > >>>>>>>>> of things such as configuration distribution
> > (config
> > >> > > > > stream),
> > >> > > > > >> > >>>>>>>>> shell
> > >> > > > > >> > >>>>>>> scrips
> > >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the
> > .tgz
> > >> > > > stuff),
> > >> > > > > >> etc.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic deployment
> > was
> > >> to
> > >> > > > > support
> > >> > > > > >> > >>>>>>>>> data locality. If you want to have locality,
> you
> > >> need
> > >> > to
> > >> > > > put
> > >> > > > > >> > >>>>>>>>> your
> > >> > > > > >> > >>>>>> processors
> > >> > > > > >> > >>>>>>>>> close to the data they're processing. Upon
> > further
> > >> > > > > >> > >>>>>>>>> investigation,
> > >> > > > > >> > >>>>>>> though,
> > >> > > > > >> > >>>>>>>>> this feature is not that beneficial. There is
> > some
> > >> > good
> > >> > > > > >> > >>>>>>>>> discussion
> > >> > > > > >> > >>>>>> about
> > >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we
> > took
> > >> the
> > >> > > > > >> > >>>>>>>>> Map/Reduce
> > >> > > > > >> > >>>>>> path,
> > >> > > > > >> > >>>>>>>>> but
> > >> > > > > >> > >>>>>>>>> there are some fundamental differences between
> > HDFS
> > >> > and
> > >> > > > > Kafka.
> > >> > > > > >> > >>>>>>>>> HDFS
> > >> > > > > >> > >>>>>> has
> > >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads
> to
> > >> less
> > >> > > > > >> > >>>>>>>>> optimization potential with stream processors
> on
> > top
> > >> > of
> > >> > > > > Kafka.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> This feature is also used as a crutch. Samza
> > doesn't
> > >> > > have
> > >> > > > > any
> > >> > > > > >> > >>>>>>>>> built
> > >> > > > > >> > >>>>> in
> > >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on
> the
> > >> > > dynamic
> > >> > > > > >> > >>>>>>>>> deployment scheduling system to handle restarts
> > >> when a
> > >> > > > > >> > >>>>>>>>> processor dies. This has
> > >> > > > > >> > >>>>>>> made
> > >> > > > > >> > >>>>>>>>> it very difficult to write a standalone Samza
> > >> > container
> > >> > > > > >> > >>>> (SAMZA-516).
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Pluggability
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> In some cases pluggability is good, but I think
> > that
> > >> > > we've
> > >> > > > > >> gone
> > >> > > > > >> > >>>>>>>>> too
> > >> > > > > >> > >>>>>> far
> > >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> * Pluggable config.
> > >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > >> > > > > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> > >> > > > > SystemProducer,
> > >> > > > > >> > >>>> etc).
> > >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about every
> > >> component
> > >> > > > > >> > >>>>> (MessageChooser,
> > >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter,
> > etc).
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> There's probably more that I've forgotten, as
> > well.
> > >> > Some
> > >> > > > of
> > >> > > > > >> > >>>>>>>>> these
> > >> > > > > >> > >>>>> are
> > >> > > > > >> > >>>>>>>>> useful, but some have proven not to be. This
> all
> > >> comes
> > >> > > at
> > >> > > > a
> > >> > > > > >> cost:
> > >> > > > > >> > >>>>>>>>> complexity. This complexity is making it harder
> > for
> > >> > our
> > >> > > > > users
> > >> > > > > >> > >>>>>>>>> to
> > >> > > > > >> > >>>>> pick
> > >> > > > > >> > >>>>>> up
> > >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also makes it
> > >> > difficult
> > >> > > > for
> > >> > > > > >> > >>>>>>>>> Samza developers to reason about what the
> > >> > > characteristics
> > >> > > > of
> > >> > > > > >> > >>>>>>>>> the container (since the characteristics change
> > >> > > depending
> > >> > > > on
> > >> > > > > >> > >>>>>>>>> which plugins are use).
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> The issues with pluggability are most visible
> in
> > the
> > >> > > > System
> > >> > > > > >> APIs.
> > >> > > > > >> > >>>>> What
> > >> > > > > >> > >>>>>>>>> Samza really requires to be functional is Kafka
> > as
> > >> its
> > >> > > > > >> > >>>>>>>>> transport
> > >> > > > > >> > >>>>>> layer.
> > >> > > > > >> > >>>>>>>>> But
> > >> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases into
> one
> > >> API:
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> The current System API supports both of these
> use
> > >> > cases.
> > >> > > > The
> > >> > > > > >> > >>>>>>>>> problem
> > >> > > > > >> > >>>>>> is,
> > >> > > > > >> > >>>>>>>>> we
> > >> > > > > >> > >>>>>>>>> actually want different features for each use
> > case.
> > >> By
> > >> > > > > >> papering
> > >> > > > > >> > >>>>>>>>> over
> > >> > > > > >> > >>>>>>> these
> > >> > > > > >> > >>>>>>>>> two use cases, and providing a single API,
> we've
> > >> > > > introduced
> > >> > > > > a
> > >> > > > > >> > >>>>>>>>> ton of
> > >> > > > > >> > >>>>>>> leaky
> > >> > > > > >> > >>>>>>>>> abstractions.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> For example, what we'd really like in (2) is to
> > have
> > >> > > > > >> > >>>>>>>>> monotonically increasing longs for offsets
> (like
> > >> > Kafka).
> > >> > > > > This
> > >> > > > > >> > >>>>>>>>> would be at odds
> > >> > > > > >> > >>>>> with
> > >> > > > > >> > >>>>>>> (1),
> > >> > > > > >> > >>>>>>>>> though, since different systems have different
> > >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > >> > > > > >> > >>>>>>>>> There was discussion both on the mailing list
> and
> > >> the
> > >> > > SQL
> > >> > > > > >> JIRAs
> > >> > > > > >> > >>>>> about
> > >> > > > > >> > >>>>>>> the
> > >> > > > > >> > >>>>>>>>> need for this.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> The same thing holds true for replayability.
> > Kafka
> > >> > > allows
> > >> > > > us
> > >> > > > > >> to
> > >> > > > > >> > >>>>> rewind
> > >> > > > > >> > >>>>>>>>> when
> > >> > > > > >> > >>>>>>>>> we have a failure. Many other systems don't. In
> > some
> > >> > > > cases,
> > >> > > > > >> > >>>>>>>>> systems
> > >> > > > > >> > >>>>>>> return
> > >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> > >> WikipediaSystemConsumer)
> > >> > > > > because
> > >> > > > > >> > >>>>>>>>> they
> > >> > > > > >> > >>>>>> have
> > >> > > > > >> > >>>>>>> no
> > >> > > > > >> > >>>>>>>>> offsets.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka supports
> > >> > > > > partitioning,
> > >> > > > > >> > >>>>>>>>> but
> > >> > > > > >> > >>>>> many
> > >> > > > > >> > >>>>>>>>> systems don't. We model this by having a single
> > >> > > partition
> > >> > > > > for
> > >> > > > > >> > >>>>>>>>> those systems. Still, other systems model
> > >> partitioning
> > >> > > > > >> > >>>> differently (e.g.
> > >> > > > > >> > >>>>>>>>> Kinesis).
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a mess.
> > Creating
> > >> > > streams
> > >> > > > > in
> > >> > > > > >> a
> > >> > > > > >> > >>>>>>>>> system-agnostic way is almost impossible. As is
> > >> > modeling
> > >> > > > > >> > >>>>>>>>> metadata
> > >> > > > > >> > >>>>> for
> > >> > > > > >> > >>>>>>> the
> > >> > > > > >> > >>>>>>>>> system (replication factor, partitions,
> location,
> > >> > etc).
> > >> > > > The
> > >> > > > > >> > >>>>>>>>> list
> > >> > > > > >> > >>>>> goes
> > >> > > > > >> > >>>>>>> on.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Duplicate work
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> At the time that we began writing Samza,
> Kafka's
> > >> > > consumer
> > >> > > > > and
> > >> > > > > >> > >>>>> producer
> > >> > > > > >> > >>>>>>>>> APIs
> > >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On the
> > >> > consumer-side,
> > >> > > > you
> > >> > > > > >> > >>>>>>>>> had two
> > >> > > > > >> > >>>>>>>>> options: use the high level consumer, or the
> > simple
> > >> > > > > consumer.
> > >> > > > > >> > >>>>>>>>> The
> > >> > > > > >> > >>>>>>> problem
> > >> > > > > >> > >>>>>>>>> with the high-level consumer was that it
> > controlled
> > >> > your
> > >> > > > > >> > >>>>>>>>> offsets, partition assignments, and the order
> in
> > >> which
> > >> > > you
> > >> > > > > >> > >>>>>>>>> received messages. The
> > >> > > > > >> > >>>>> problem
> > >> > > > > >> > >>>>>>>>> with
> > >> > > > > >> > >>>>>>>>> the simple consumer is that it's not simple.
> It's
> > >> > basic.
> > >> > > > You
> > >> > > > > >> > >>>>>>>>> end up
> > >> > > > > >> > >>>>>>> having
> > >> > > > > >> > >>>>>>>>> to handle a lot of really low-level stuff that
> > you
> > >> > > > > shouldn't.
> > >> > > > > >> > >>>>>>>>> We
> > >> > > > > >> > >>>>>> spent a
> > >> > > > > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer
> > very
> > >> > > > robust.
> > >> > > > > >> It
> > >> > > > > >> > >>>>>>>>> also allows us to support some cool features:
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
> > prioritization.
> > >> > > > > >> > >>>>>>>>> * Tight control over partition assignment to
> > support
> > >> > > > joins,
> > >> > > > > >> > >>>>>>>>> global
> > >> > > > > >> > >>>>>> state
> > >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > >> > > > > >> > >>>>>>>>> * Tight control over offset checkpointing.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> What we didn't realize at the time is that
> these
> > >> > > features
> > >> > > > > >> > >>>>>>>>> should
> > >> > > > > >> > >>>>>>> actually
> > >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just
> > >> Samza
> > >> > > > stream
> > >> > > > > >> > >>>>>> processors)
> > >> > > > > >> > >>>>>>>>> end up wanting to do things like joins and
> > partition
> > >> > > > > >> > >>>>>>>>> assignment. The
> > >> > > > > >> > >>>>>>> Kafka
> > >> > > > > >> > >>>>>>>>> community has come to the same conclusion.
> > They're
> > >> > > adding
> > >> > > > a
> > >> > > > > >> ton
> > >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka consumer
> > >> > > implementation.
> > >> > > > > To a
> > >> > > > > >> > >>>>>>>>> large extent,
> > >> > > > > >> > >>>>> it's
> > >> > > > > >> > >>>>>>>>> duplicate work to what we've already done in
> > Samza.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a very
> > similar
> > >> > > > > approach
> > >> > > > > >> > >>>>>>>>> to
> > >> > > > > >> > >>>>>> Samza's
> > >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation for
> > handling
> > >> > > offset
> > >> > > > > >> > >>>>>> checkpointing.
> > >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset management
> feature
> > >> > stores
> > >> > > > > >> offset
> > >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch
> > them
> > >> > > from
> > >> > > > > the
> > >> > > > > >> > >>>>>>>>> broker.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since we
> could
> > >> have
> > >> > > > shared
> > >> > > > > >> > >>>>>>>>> the
> > >> > > > > >> > >>>>> work
> > >> > > > > >> > >>>>>> if
> > >> > > > > >> > >>>>>>>>> it
> > >> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Vision
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> All of this leads me to a rather radical
> > proposal.
> > >> > Samza
> > >> > > > is
> > >> > > > > >> > >>>>> relatively
> > >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to say that
> > we're
> > >> > > near a
> > >> > > > > 1.0
> > >> > > > > >> > >>>>>> release.
> > >> > > > > >> > >>>>>>>>> I'd
> > >> > > > > >> > >>>>>>>>> like to propose that we take what we've
> learned,
> > and
> > >> > > begin
> > >> > > > > >> > >>>>>>>>> thinking
> > >> > > > > >> > >>>>>>> about
> > >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we
> were
> > >> > > starting
> > >> > > > > >> from
> > >> > > > > >> > >>>>>> scratch?
> > >> > > > > >> > >>>>>>>>> My
> > >> > > > > >> > >>>>>>>>> proposal is to:
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run
> > Samza
> > >> > > > > >> > >>>>>>>>> processors, and eliminate all direct
> dependences
> > on
> > >> > > YARN,
> > >> > > > > >> Mesos,
> > >> > > > > >> > >>>> etc.
> > >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka
> > as
> > >> the
> > >> > > > > stream
> > >> > > > > >> > >>>>>> processing
> > >> > > > > >> > >>>>>>>>> layer.
> > >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
> > >> serialization,
> > >> > > and
> > >> > > > > >> > >>>>>>>>> config
> > >> > > > > >> > >>>>>>> systems,
> > >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> This would fix all of the issues that I
> outlined
> > >> > above.
> > >> > > It
> > >> > > > > >> > >>>>>>>>> should
> > >> > > > > >> > >>>>> also
> > >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty dramatically.
> > >> > > Supporting
> > >> > > > > >> only
> > >> > > > > >> > >>>>>>>>> a standalone container will allow Samza to be
> > >> executed
> > >> > > on
> > >> > > > > YARN
> > >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora),
> or
> > >> most
> > >> > > > other
> > >> > > > > >> > >>>>>>>>> in-house
> > >> > > > > >> > >>>>>>> deployment
> > >> > > > > >> > >>>>>>>>> systems. This should make life a lot easier for
> > new
> > >> > > users.
> > >> > > > > >> > >>>>>>>>> Imagine
> > >> > > > > >> > >>>>>>> having
> > >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop
> > in
> > >> > > mailing
> > >> > > > > >> list
> > >> > > > > >> > >>>>>> traffic
> > >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me.
> The
> > >> > > reality
> > >> > > > > is,
> > >> > > > > >> > >>>>> everyone
> > >> > > > > >> > >>>>>>>>> that
> > >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We
> > basically
> > >> > > > require
> > >> > > > > >> it
> > >> > > > > >> > >>>>>> already
> > >> > > > > >> > >>>>>>> in
> > >> > > > > >> > >>>>>>>>> order for most features to work. Those that are
> > >> using
> > >> > > > other
> > >> > > > > >> > >>>>>>>>> systems
> > >> > > > > >> > >>>>>> are
> > >> > > > > >> > >>>>>>>>> generally using it for ingest into Kafka (1),
> and
> > >> then
> > >> > > > they
> > >> > > > > do
> > >> > > > > >> > >>>>>>>>> the processing on top. There is already
> > discussion (
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>
> > >> > > > > >> >
> > >> > > > >
> > >> >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >> > > > > >> > >>>>> 767
> > >> > > > > >> > >>>>>>>>> )
> > >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely
> > >> easy.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Once we make the call to couple with Kafka, we
> > can
> > >> > > > leverage
> > >> > > > > a
> > >> > > > > >> > >>>>>>>>> ton of
> > >> > > > > >> > >>>>>>> their
> > >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to maintain our
> own
> > >> > config,
> > >> > > > > >> > >>>>>>>>> metrics,
> > >> > > > > >> > >>>>> etc.
> > >> > > > > >> > >>>>>>> We
> > >> > > > > >> > >>>>>>>>> can all share the same libraries, and make them
> > >> > better.
> > >> > > > This
> > >> > > > > >> > >>>>>>>>> will
> > >> > > > > >> > >>>>> also
> > >> > > > > >> > >>>>>>>>> allow us to share the consumer/producer APIs,
> and
> > >> will
> > >> > > let
> > >> > > > > us
> > >> > > > > >> > >>>>> leverage
> > >> > > > > >> > >>>>>>>>> their offset management and partition
> management,
> > >> > rather
> > >> > > > > than
> > >> > > > > >> > >>>>>>>>> having
> > >> > > > > >> > >>>>>> our
> > >> > > > > >> > >>>>>>>>> own. All of the coordinator stream code would
> go
> > >> away,
> > >> > > as
> > >> > > > > >> would
> > >> > > > > >> > >>>>>>>>> most
> > >> > > > > >> > >>>>>> of
> > >> > > > > >> > >>>>>>>>> the
> > >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push
> > some
> > >> > > > > partition
> > >> > > > > >> > >>>>>>> management
> > >> > > > > >> > >>>>>>>>> features into the Kafka broker, but they're
> > already
> > >> > > moving
> > >> > > > > in
> > >> > > > > >> > >>>>>>>>> that direction with the new consumer API. The
> > >> features
> > >> > > we
> > >> > > > > have
> > >> > > > > >> > >>>>>>>>> for
> > >> > > > > >> > >>>>>> partition
> > >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and seem
> like
> > >> they
> > >> > > > should
> > >> > > > > >> be
> > >> > > > > >> > >>>>>>>>> in
> > >> > > > > >> > >>>>>> Kafka
> > >> > > > > >> > >>>>>>>>> anyway. There will always be some niche usages
> > which
> > >> > > will
> > >> > > > > >> > >>>>>>>>> require
> > >> > > > > >> > >>>>>> extra
> > >> > > > > >> > >>>>>>>>> care and hence full control over partition
> > >> assignments
> > >> > > > much
> > >> > > > > >> > >>>>>>>>> like the
> > >> > > > > >> > >>>>>>> Kafka
> > >> > > > > >> > >>>>>>>>> low level consumer api. These would continue to
> > be
> > >> > > > > supported.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> These items will be good for the Samza
> community.
> > >> > > They'll
> > >> > > > > make
> > >> > > > > >> > >>>>>>>>> Samza easier to use, and make it easier for
> > >> developers
> > >> > > to
> > >> > > > > add
> > >> > > > > >> > >>>>>>>>> new features.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat
> > >> > backwards
> > >> > > > > >> > >>>>> incompatible
> > >> > > > > >> > >>>>>>>>> change). If we choose to go this route, it's
> > >> important
> > >> > > > that
> > >> > > > > we
> > >> > > > > >> > >>>>> openly
> > >> > > > > >> > >>>>>>>>> communicate how we're going to provide a
> > migration
> > >> > path
> > >> > > > from
> > >> > > > > >> > >>>>>>>>> the
> > >> > > > > >> > >>>>>>> existing
> > >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make incompatible
> > >> > changes).
> > >> > > I
> > >> > > > > >> think
> > >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to provide a
> > >> wrapper
> > >> > to
> > >> > > > > allow
> > >> > > > > >> > >>>>>>>>> existing StreamTask implementations to continue
> > >> > running
> > >> > > on
> > >> > > > > the
> > >> > > > > >> > >>>> new container.
> > >> > > > > >> > >>>>>>> It's
> > >> > > > > >> > >>>>>>>>> also important that we openly communicate about
> > >> > timing,
> > >> > > > and
> > >> > > > > >> > >>>>>>>>> stages
> > >> > > > > >> > >>>>> of
> > >> > > > > >> > >>>>>>> the
> > >> > > > > >> > >>>>>>>>> migration.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you have
> > opinions.
> > >> > :)
> > >> > > > > Please
> > >> > > > > >> > >>>>>>>>> send
> > >> > > > > >> > >>>>>> your
> > >> > > > > >> > >>>>>>>>> thoughts and feedback.
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>> Cheers,
> > >> > > > > >> > >>>>>>>>> Chris
> > >> > > > > >> > >>>>>>>>>
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>>
> > >> > > > > >> > >>>>>>>
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>> --
> > >> > > > > >> > >>>>>> -- Guozhang
> > >> > > > > >> > >>>>>>
> > >> > > > > >> > >>>>>
> > >> > > > > >> > >>>>
> > >> > > > > >> > >>
> > >> > > > > >> > >>
> > >> > > > > >> >
> > >> > > > > >> >
> > >> > > > > >> >
> > >> > > > > >>
> > >> > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@confluent.io>.
Yeah I agree with this summary. I think there are kind of two questions
here:
1. Technically does alignment/reliance on Kafka make sense
2. Branding wise (naming, website, concepts, etc) does alignment with Kafka
make sense

Personally I do think both of these things would be really valuable, and
would dramatically alter the trajectory of the project.

My preference would be to see if people can mostly agree on a direction
rather than splintering things off. From my point of view the ideal outcome
of all the options discussed would be to make Samza a closely aligned
subproject, maintained in a separate repository and retaining the existing
committership but sharing as much else as possible (website, etc). No idea
about how these things work, Jacob, you probably know more.

No discussion amongst the Kafka folks has happened on this, but likely we
should figure out what the Samza community actually wants first.

I admit that this is a fairly radical departure from how things are.

If that doesn't fly, I think, yeah we could leave Samza as it is and do the
more radical reboot inside Kafka. From my point of view that does leave
things in a somewhat confusing state since now there are two stream
processing systems more or less coupled to Kafka in large part made by the
same people. But, arguably that might be a cleaner way to make the cut-over
and perhaps less risky for Samza community since if it works people can
switch and if it doesn't nothing will have changed. Dunno, how do people
feel about this?

-Jay

On Fri, Jul 10, 2015 at 11:49 AM, Jakob Homan <jg...@gmail.com> wrote:

> >  This leads me to thinking that merging projects and communities might
> be a good idea: with the union of experience from both communities, we will
> probably build a better system that is better for users.
> Is this what's being proposed though? Merging the projects seems like
> a consequence of at most one of the three directions under discussion:
> 1) Samza 2.0: The Samza community relies more heavily on Kafka for
> configuration, etc. (to a greater or lesser extent to be determined)
> but the Samza community would not automatically merge withe Kafka
> community (the Phoenix/HBase example is a good one here).
> 2) Samza Reboot: The Samza community continues to exist with a limited
> project scope, but similarly would not need to be part of the Kafka
> community (ie given committership) to progress.  Here, maybe the Samza
> team would become a subproject of Kafka (the Board frowns on
> subprojects at the moment, so I'm not sure if that's even feasible),
> but that would not be required.
> 3) Hey Samza! FYI, Kafka does streaming now: In this option the Kafka
> team builds its own streaming library, possibly off of Jay's
> prototype, which has not direct lineage to the Samza team.  There's no
> reason for the Kafka team to bring in the Samza team.
>
> Is the Kafka community on board with this?
>
> To be clear, all three options under discussion are interesting,
> technically valid and likely healthy directions for the project.
> Also, they are not mutually exclusive.  The Samza community could
> decide to pursue, say, 'Samza 2.0', while the Kafka community went
> forward with 'Hey Samza!'  My points above are directed entirely at
> the community aspect of these choices.
> -Jakob
>
> On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com> wrote:
> > That's great.  Thanks, Jay.
> >
> > On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io> wrote:
> >
> >> Yeah totally agree. I think you have this issue even today, right? I.e.
> if
> >> you need to make a simple config change and you're running in YARN today
> >> you end up bouncing the job which then rebuilds state. I think the fix
> is
> >> exactly what you described which is to have a long timeout on partition
> >> movement for stateful jobs so that if a job is just getting bounced, and
> >> the cluster manager (or admin) is smart enough to restart it on the same
> >> host when possible, it can optimistically reuse any existing state it
> finds
> >> on disk (if it is valid).
> >>
> >> So in this model the charter of the CM is to place processes as
> stickily as
> >> possible and to restart or re-place failed processes. The charter of the
> >> partition management system is to control the assignment of work to
> these
> >> processes. The nice thing about this is that the work assignment,
> timeouts,
> >> behavior, configs, and code will all be the same across all cluster
> >> managers.
> >>
> >> So I think that prototype would actually give you exactly what you want
> >> today for any cluster manager (or manual placement + restart script)
> that
> >> was sticky in terms of host placement since there is already a
> configurable
> >> partition movement timeout and task-by-task state reuse with a check on
> >> state validity.
> >>
> >> -Jay
> >>
> >> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <ro...@gmail.com>
> >> wrote:
> >>
> >> > That would be great to let Kafka do as much heavy lifting as possible
> and
> >> > make it easier for other languages to implement Samza apis.
> >> >
> >> > One thing to watch out for is the interplay between Kafka's group
> >> > management and the external scheduler/process manager's fault
> tolerance.
> >> > If a container dies, the Kafka group membership protocol will try to
> >> assign
> >> > it's tasks to other containers while at the same time the process
> manager
> >> > is trying to relaunch the container.  Without some consideration for
> this
> >> > (like a configurable amount of time to wait before Kafka alters the
> group
> >> > membership), there may be thrashing going on which is especially bad
> for
> >> > containers with large amounts of local state.
> >> >
> >> > Someone else pointed this out already but I thought it might be worth
> >> > calling out again.
> >> >
> >> > Cheers,
> >> >
> >> > Roger
> >> >
> >> >
> >> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <ja...@confluent.io> wrote:
> >> >
> >> > > Hey Roger,
> >> > >
> >> > > I couldn't agree more. We spent a bunch of time talking to people
> and
> >> > that
> >> > > is exactly the stuff we heard time and again. What makes it hard, of
> >> > > course, is that there is some tension between compatibility with
> what's
> >> > > there now and making things better for new users.
> >> > >
> >> > > I also strongly agree with the importance of multi-language
> support. We
> >> > are
> >> > > talking now about Java, but for application development use cases
> >> people
> >> > > want to work in whatever language they are using elsewhere. I think
> >> > moving
> >> > > to a model where Kafka itself does the group membership, lifecycle
> >> > control,
> >> > > and partition assignment has the advantage of putting all that
> complex
> >> > > stuff behind a clean api that the clients are already going to be
> >> > > implementing for their consumer, so the added functionality for
> stream
> >> > > processing beyond a consumer becomes very minor.
> >> > >
> >> > > -Jay
> >> > >
> >> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <
> roger.hoover@gmail.com>
> >> > > wrote:
> >> > >
> >> > > > Metamorphosis...nice. :)
> >> > > >
> >> > > > This has been a great discussion.  As a user of Samza who's
> recently
> >> > > > integrated it into a relatively large organization, I just want to
> >> add
> >> > > > support to a few points already made.
> >> > > >
> >> > > > The biggest hurdles to adoption of Samza as it currently exists
> that
> >> > I've
> >> > > > experienced are:
> >> > > > 1) YARN - YARN is overly complex in many environments where Puppet
> >> > would
> >> > > do
> >> > > > just fine but it was the only mechanism to get fault tolerance.
> >> > > > 2) Configuration - I think I like the idea of configuring most of
> the
> >> > job
> >> > > > in code rather than config files.  In general, I think the goal
> >> should
> >> > be
> >> > > > to make it harder to make mistakes, especially of the kind where
> the
> >> > code
> >> > > > expects something and the config doesn't match.  The current
> config
> >> is
> >> > > > quite intricate and error-prone.  For example, the application
> logic
> >> > may
> >> > > > depend on bootstrapping a topic but rather than asserting that in
> the
> >> > > code,
> >> > > > you have to rely on getting the config right.  Likewise with
> serdes,
> >> > the
> >> > > > Java representations produced by various serdes (JSON, Avro, etc.)
> >> are
> >> > > not
> >> > > > equivalent so you cannot just reconfigure a serde without changing
> >> the
> >> > > > code.   It would be nice for jobs to be able to assert what they
> >> expect
> >> > > > from their input topics in terms of partitioning.  This is
> getting a
> >> > > little
> >> > > > off topic but I was even thinking about creating a "Samza config
> >> > linter"
> >> > > > that would sanity check a set of configs.  Especially in
> >> organizations
> >> > > > where config is managed by a different team than the application
> >> > > developer,
> >> > > > it's very hard to get avoid config mistakes.
> >> > > > 3) Java/Scala centric - for many teams (especially DevOps-type
> >> folks),
> >> > > the
> >> > > > pain of the Java toolchain (maven, slow builds, weak command line
> >> > > support,
> >> > > > configuration over convention) really inhibits productivity.  As
> more
> >> > and
> >> > > > more high-quality clients become available for Kafka, I hope
> they'll
> >> > > follow
> >> > > > Samza's model.  Not sure how much it affects the proposals in this
> >> > thread
> >> > > > but please consider other languages in the ecosystem as well.
> From
> >> > what
> >> > > > I've heard, Spark has more Python users than Java/Scala.
> >> > > > (FYI, we added a Jython wrapper for the Samza API
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> >> > > > and are working on a Yeoman generator
> >> > > > https://github.com/Quantiply/generator-rico for Jython/Samza
> >> projects
> >> > to
> >> > > > alleviate some of the pain)
> >> > > >
> >> > > > I also want to underscore Jay's point about improving the user
> >> > > experience.
> >> > > > That's a very important factor for adoption.  I think the goal
> should
> >> > be
> >> > > to
> >> > > > make Samza as easy to get started with as something like Logstash.
> >> > > > Logstash is vastly inferior in terms of capabilities to Samza but
> >> it's
> >> > > easy
> >> > > > to get started and that makes a big difference.
> >> > > >
> >> > > > Cheers,
> >> > > >
> >> > > > Roger
> >> > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
> >> > > > gdfm@apache.org> wrote:
> >> > > >
> >> > > > > Forgot to add. On the naming issues, Kafka Metamorphosis is a
> clear
> >> > > > winner
> >> > > > > :)
> >> > > > >
> >> > > > > --
> >> > > > > Gianmarco
> >> > > > >
> >> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
> >> > > gdfm@apache.org
> >> > > > >
> >> > > > > wrote:
> >> > > > >
> >> > > > > > Hi,
> >> > > > > >
> >> > > > > > @Martin, thanks for you comments.
> >> > > > > > Maybe I'm missing some important point, but I think coupling
> the
> >> > > > releases
> >> > > > > > is actually a *good* thing.
> >> > > > > > To make an example, would it be better if the MR and HDFS
> >> > components
> >> > > of
> >> > > > > > Hadoop had different release schedules?
> >> > > > > >
> >> > > > > > Actually, keeping the discussion in a single place would make
> >> > > agreeing
> >> > > > on
> >> > > > > > releases (and backwards compatibility) much easier, as
> everybody
> >> > > would
> >> > > > be
> >> > > > > > responsible for the whole codebase.
> >> > > > > >
> >> > > > > > That said, I like the idea of absorbing samza-core as a
> >> > sub-project,
> >> > > > and
> >> > > > > > leave the fancy stuff separate.
> >> > > > > > It probably gives 90% of the benefits we have been discussing
> >> here.
> >> > > > > >
> >> > > > > > Cheers,
> >> > > > > >
> >> > > > > > --
> >> > > > > > Gianmarco
> >> > > > > >
> >> > > > > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com>
> wrote:
> >> > > > > >
> >> > > > > >> Hey Martin,
> >> > > > > >>
> >> > > > > >> I agree coupling release schedules is a downside.
> >> > > > > >>
> >> > > > > >> Definitely we can try to solve some of the integration
> problems
> >> in
> >> > > > > >> Confluent Platform or in other distributions. But I think
> this
> >> > ends
> >> > > up
> >> > > > > >> being really shallow. I guess I feel to really get a good
> user
> >> > > > > experience
> >> > > > > >> the two systems have to kind of feel like part of the same
> thing
> >> > and
> >> > > > you
> >> > > > > >> can't really add that in later--you can put both in the same
> >> > > > > downloadable
> >> > > > > >> tar file but it doesn't really give a very cohesive feeling.
> I
> >> > agree
> >> > > > > that
> >> > > > > >> ultimately any of the project stuff is as much social and
> naming
> >> > as
> >> > > > > >> anything else--theoretically two totally independent projects
> >> > could
> >> > > > work
> >> > > > > >> to
> >> > > > > >> tightly align. In practice this seems to be quite difficult
> >> > though.
> >> > > > > >>
> >> > > > > >> For the frameworks--totally agree it would be good to
> maintain
> >> the
> >> > > > > >> framework support with the project. In some cases there may
> not
> >> be
> >> > > too
> >> > > > > >> much
> >> > > > > >> there since the integration gets lighter but I think whatever
> >> > stubs
> >> > > > you
> >> > > > > >> need should be included. So no I definitely wasn't trying to
> >> imply
> >> > > > > >> dropping
> >> > > > > >> support for these frameworks, just making the integration
> >> lighter
> >> > by
> >> > > > > >> separating process management from partition management.
> >> > > > > >>
> >> > > > > >> You raise two good points we would have to figure out if we
> went
> >> > > down
> >> > > > > the
> >> > > > > >> alignment path:
> >> > > > > >> 1. With respect to the name, yeah I think the first question
> is
> >> > > > whether
> >> > > > > >> some "re-branding" would be worth it. If so then I think we
> can
> >> > > have a
> >> > > > > big
> >> > > > > >> thread on the name. I'm definitely not set on Kafka
> Streaming or
> >> > > Kafka
> >> > > > > >> Streams I was just using them to be kind of illustrative. I
> >> agree
> >> > > with
> >> > > > > >> your
> >> > > > > >> critique of these names, though I think people would get the
> >> idea.
> >> > > > > >> 2. Yeah you also raise a good point about how to "factor" it.
> >> Here
> >> > > are
> >> > > > > the
> >> > > > > >> options I see (I could get enthusiastic about any of them):
> >> > > > > >>    a. One repo for both Kafka and Samza
> >> > > > > >>    b. Two repos, retaining the current seperation
> >> > > > > >>    c. Two repos, the equivalent of samza-api and samza-core
> is
> >> > > > absorbed
> >> > > > > >> almost like a third client
> >> > > > > >>
> >> > > > > >> Cheers,
> >> > > > > >>
> >> > > > > >> -Jay
> >> > > > > >>
> >> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> >> > > > martin@kleppmann.com>
> >> > > > > >> wrote:
> >> > > > > >>
> >> > > > > >> > Ok, thanks for the clarifications. Just a few follow-up
> >> > comments.
> >> > > > > >> >
> >> > > > > >> > - I see the appeal of merging with Kafka or becoming a
> >> > subproject:
> >> > > > the
> >> > > > > >> > reasons you mention are good. The risk I see is that
> release
> >> > > > schedules
> >> > > > > >> > become coupled to each other, which can slow everyone down,
> >> and
> >> > > > large
> >> > > > > >> > projects with many contributors are harder to manage.
> (Jakob,
> >> > can
> >> > > > you
> >> > > > > >> speak
> >> > > > > >> > from experience, having seen a wider range of Hadoop
> ecosystem
> >> > > > > >> projects?)
> >> > > > > >> >
> >> > > > > >> > Some of the goals of a better unified developer experience
> >> could
> >> > > > also
> >> > > > > be
> >> > > > > >> > solved by integrating Samza nicely into a Kafka
> distribution
> >> > (such
> >> > > > as
> >> > > > > >> > Confluent's). I'm not against merging projects if we decide
> >> > that's
> >> > > > the
> >> > > > > >> way
> >> > > > > >> > to go, just pointing out the same goals can perhaps also be
> >> > > achieved
> >> > > > > in
> >> > > > > >> > other ways.
> >> > > > > >> >
> >> > > > > >> > - With regard to dropping the YARN dependency: are you
> >> proposing
> >> > > > that
> >> > > > > >> > Samza doesn't give any help to people wanting to run on
> >> > > > > >> YARN/Mesos/AWS/etc?
> >> > > > > >> > So the docs would basically have a link to Slider and
> nothing
> >> > > else?
> >> > > > Or
> >> > > > > >> > would we maintain integrations with a bunch of popular
> >> > deployment
> >> > > > > >> methods
> >> > > > > >> > (e.g. the necessary glue and shell scripts to make Samza
> work
> >> > with
> >> > > > > >> Slider)?
> >> > > > > >> >
> >> > > > > >> > I absolutely think it's a good idea to have the "as a
> library"
> >> > and
> >> > > > > "as a
> >> > > > > >> > process" (using Yi's taxonomy) options for people who want
> >> them,
> >> > > > but I
> >> > > > > >> > think there should also be a low-friction path for common
> "as
> >> a
> >> > > > > service"
> >> > > > > >> > deployment methods, for which we probably need to maintain
> >> > > > > integrations.
> >> > > > > >> >
> >> > > > > >> > - Project naming: "Kafka Streams" seems odd to me, because
> >> Kafka
> >> > > is
> >> > > > > all
> >> > > > > >> > about streams already. Perhaps "Kafka Transformers" or
> "Kafka
> >> > > > Filters"
> >> > > > > >> > would be more apt?
> >> > > > > >> >
> >> > > > > >> > One suggestion: perhaps the core of Samza (stream
> >> transformation
> >> > > > with
> >> > > > > >> > state management -- i.e. the "Samza as a library" bit)
> could
> >> > > become
> >> > > > > >> part of
> >> > > > > >> > Kafka, while higher-level tools such as streaming SQL and
> >> > > > integrations
> >> > > > > >> with
> >> > > > > >> > deployment frameworks remain in a separate project? In
> other
> >> > > words,
> >> > > > > >> Kafka
> >> > > > > >> > would absorb the proven, stable core of Samza, which would
> >> > become
> >> > > > the
> >> > > > > >> > "third Kafka client" mentioned early in this thread. The
> Samza
> >> > > > project
> >> > > > > >> > would then target that third Kafka client as its base API,
> and
> >> > the
> >> > > > > >> project
> >> > > > > >> > would be freed up to explore more experimental new
> horizons.
> >> > > > > >> >
> >> > > > > >> > Martin
> >> > > > > >> >
> >> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com>
> >> wrote:
> >> > > > > >> >
> >> > > > > >> > > Hey Martin,
> >> > > > > >> > >
> >> > > > > >> > > For the YARN/Mesos/etc decoupling I actually don't think
> it
> >> > ties
> >> > > > our
> >> > > > > >> > hands
> >> > > > > >> > > at all, all it does is refactor things. The division of
> >> > > > > >> responsibility is
> >> > > > > >> > > that Samza core is responsible for task lifecycle, state,
> >> and
> >> > > > > >> partition
> >> > > > > >> > > management (using the Kafka co-ordinator) but it is NOT
> >> > > > responsible
> >> > > > > >> for
> >> > > > > >> > > packaging, configuration deployment or execution of
> >> processes.
> >> > > The
> >> > > > > >> > problem
> >> > > > > >> > > of packaging and starting these processes is
> >> > > > > >> > > framework/environment-specific. This leaves individual
> >> > > frameworks
> >> > > > to
> >> > > > > >> be
> >> > > > > >> > as
> >> > > > > >> > > fancy or vanilla as they like. So you can get simple
> >> stateless
> >> > > > > >> support in
> >> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app framework
> >> > > (Slider,
> >> > > > > >> > Marathon,
> >> > > > > >> > > etc). These are well known by people and have nice UIs
> and a
> >> > lot
> >> > > > of
> >> > > > > >> > > flexibility. I don't think they have node affinity as a
> >> built
> >> > in
> >> > > > > >> option
> >> > > > > >> > > (though I could be wrong). So if we want that we can
> either
> >> > wait
> >> > > > for
> >> > > > > >> them
> >> > > > > >> > > to add it or do a custom framework to add that feature
> (as
> >> > now).
> >> > > > > >> > Obviously
> >> > > > > >> > > if you manage things with old-school ops tools
> >> > (puppet/chef/etc)
> >> > > > you
> >> > > > > >> get
> >> > > > > >> > > locality easily. The nice thing, though, is that all the
> >> samza
> >> > > > > >> "business
> >> > > > > >> > > logic" around partition management and fault tolerance
> is in
> >> > > Samza
> >> > > > > >> core
> >> > > > > >> > so
> >> > > > > >> > > it is shared across frameworks and the framework specific
> >> bit
> >> > is
> >> > > > > just
> >> > > > > >> > > whether it is smart enough to try to get the same host
> when
> >> a
> >> > > job
> >> > > > is
> >> > > > > >> > > restarted.
> >> > > > > >> > >
> >> > > > > >> > > With respect to the Kafka-alignment, yeah I think the
> goal
> >> > would
> >> > > > be
> >> > > > > >> (a)
> >> > > > > >> > > actually get better alignment in user experience, and (b)
> >> > > express
> >> > > > > >> this in
> >> > > > > >> > > the naming and project branding. Specifically:
> >> > > > > >> > > 1. Website/docs, it would be nice for the
> "transformation"
> >> api
> >> > > to
> >> > > > be
> >> > > > > >> > > discoverable in the main Kafka docs--i.e. be able to
> explain
> >> > > when
> >> > > > to
> >> > > > > >> use
> >> > > > > >> > > the consumer and when to use the stream processing
> >> > functionality
> >> > > > and
> >> > > > > >> lead
> >> > > > > >> > > people into that experience.
> >> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or
> whatever)
> >> > that
> >> > > > has
> >> > > > > >> both
> >> > > > > >> > > Kafka and the stream processing part and they actually
> work
> >> > > > > together.
> >> > > > > >> > > 3. Unify the programming experience so the client and
> Samza
> >> > api
> >> > > > > share
> >> > > > > >> > > config/monitoring/naming/packaging/etc.
> >> > > > > >> > >
> >> > > > > >> > > I think sub-projects keep separate committers and can
> have a
> >> > > > > separate
> >> > > > > >> > repo,
> >> > > > > >> > > but I'm actually not really sure (I can't find a
> definition
> >> > of a
> >> > > > > >> > subproject
> >> > > > > >> > > in Apache).
> >> > > > > >> > >
> >> > > > > >> > > Basically at a high-level you want the experience to
> "feel"
> >> > > like a
> >> > > > > >> single
> >> > > > > >> > > system, not to relatively independent things that are
> kind
> >> of
> >> > > > > >> awkwardly
> >> > > > > >> > > glued together.
> >> > > > > >> > >
> >> > > > > >> > > I think if we did that they having naming or branding
> like
> >> > > "kafka
> >> > > > > >> > > streaming" or "kafka streams" or something like that
> would
> >> > > > actually
> >> > > > > >> do a
> >> > > > > >> > > good job of conveying what it is. I do that this would
> help
> >> > > > adoption
> >> > > > > >> > quite
> >> > > > > >> > > a lot as it would correctly convey that using Kafka
> >> Streaming
> >> > > with
> >> > > > > >> Kafka
> >> > > > > >> > is
> >> > > > > >> > > a fairly seamless experience and Kafka is pretty heavily
> >> > adopted
> >> > > > at
> >> > > > > >> this
> >> > > > > >> > > point.
> >> > > > > >> > >
> >> > > > > >> > > Fwiw we actually considered this model originally when
> open
> >> > > > sourcing
> >> > > > > >> > Samza,
> >> > > > > >> > > however at that time Kafka was relatively unknown and we
> >> > decided
> >> > > > not
> >> > > > > >> to
> >> > > > > >> > do
> >> > > > > >> > > it since we felt it would be limiting. From my point of
> view
> >> > the
> >> > > > > three
> >> > > > > >> > > things have changed (1) Kafka is now really heavily used
> for
> >> > > > stream
> >> > > > > >> > > processing, (2) we learned that abstracting out the
> stream
> >> > well
> >> > > is
> >> > > > > >> > > basically impossible, (3) we learned it is really hard to
> >> keep
> >> > > the
> >> > > > > two
> >> > > > > >> > > things feeling like a single product.
> >> > > > > >> > >
> >> > > > > >> > > -Jay
> >> > > > > >> > >
> >> > > > > >> > >
> >> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> >> > > > > >> martin@kleppmann.com>
> >> > > > > >> > > wrote:
> >> > > > > >> > >
> >> > > > > >> > >> Hi all,
> >> > > > > >> > >>
> >> > > > > >> > >> Lots of good thoughts here.
> >> > > > > >> > >>
> >> > > > > >> > >> I agree with the general philosophy of tying Samza more
> >> > firmly
> >> > > to
> >> > > > > >> Kafka.
> >> > > > > >> > >> After I spent a while looking at integrating other
> message
> >> > > > brokers
> >> > > > > >> (e.g.
> >> > > > > >> > >> Kinesis) with SystemConsumer, I came to the conclusion
> that
> >> > > > > >> > SystemConsumer
> >> > > > > >> > >> tacitly assumes a model so much like Kafka's that pretty
> >> much
> >> > > > > nobody
> >> > > > > >> but
> >> > > > > >> > >> Kafka actually implements it. (Databus is perhaps an
> >> > exception,
> >> > > > but
> >> > > > > >> it
> >> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus, making
> Samza
> >> > > fully
> >> > > > > >> > dependent
> >> > > > > >> > >> on Kafka acknowledges that the system-independence was
> >> never
> >> > as
> >> > > > > real
> >> > > > > >> as
> >> > > > > >> > we
> >> > > > > >> > >> perhaps made it out to be. The gains of code reuse are
> >> real.
> >> > > > > >> > >>
> >> > > > > >> > >> The idea of decoupling Samza from YARN has also always
> been
> >> > > > > >> appealing to
> >> > > > > >> > >> me, for various reasons already mentioned in this
> thread.
> >> > > > Although
> >> > > > > >> > making
> >> > > > > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc)
> >> seems
> >> > > > > >> laudable,
> >> > > > > >> > I am
> >> > > > > >> > >> a little concerned that it will restrict us to a lowest
> >> > common
> >> > > > > >> > denominator.
> >> > > > > >> > >> For example, would host affinity (SAMZA-617) still be
> >> > possible?
> >> > > > For
> >> > > > > >> jobs
> >> > > > > >> > >> with large amounts of state, I think SAMZA-617 would be
> a
> >> big
> >> > > > boon,
> >> > > > > >> > since
> >> > > > > >> > >> restoring state off the changelog on every single
> restart
> >> is
> >> > > > > painful,
> >> > > > > >> > due
> >> > > > > >> > >> to long recovery times. It would be a shame if the
> >> decoupling
> >> > > > from
> >> > > > > >> YARN
> >> > > > > >> > >> made host affinity impossible.
> >> > > > > >> > >>
> >> > > > > >> > >> Jay, a question about the proposed API for
> instantiating a
> >> > job
> >> > > in
> >> > > > > >> code
> >> > > > > >> > >> (rather than a properties file): when submitting a job
> to a
> >> > > > > cluster,
> >> > > > > >> is
> >> > > > > >> > the
> >> > > > > >> > >> idea that the instantiation code runs on a client
> >> somewhere,
> >> > > > which
> >> > > > > >> then
> >> > > > > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or
> >> does
> >> > > that
> >> > > > > >> code
> >> > > > > >> > run
> >> > > > > >> > >> on each container that is part of the job (in which
> case,
> >> how
> >> > > > does
> >> > > > > >> the
> >> > > > > >> > job
> >> > > > > >> > >> submission to the cluster work)?
> >> > > > > >> > >>
> >> > > > > >> > >> I agree with Garry that it doesn't feel right to make a
> 1.0
> >> > > > release
> >> > > > > >> > with a
> >> > > > > >> > >> plan for it to be immediately obsolete. So if this is
> going
> >> > to
> >> > > > > >> happen, I
> >> > > > > >> > >> think it would be more honest to stick with 0.* version
> >> > numbers
> >> > > > > until
> >> > > > > >> > the
> >> > > > > >> > >> library-ified Samza has been implemented, is stable and
> >> > widely
> >> > > > > used.
> >> > > > > >> > >>
> >> > > > > >> > >> Should the new Samza be a subproject of Kafka? There is
> >> > > precedent
> >> > > > > for
> >> > > > > >> > >> tight coupling between different Apache projects (e.g.
> >> > Curator
> >> > > > and
> >> > > > > >> > >> Zookeeper, or Slider and YARN), so I think remaining
> >> separate
> >> > > > would
> >> > > > > >> be
> >> > > > > >> > ok.
> >> > > > > >> > >> Even if Samza is fully dependent on Kafka, there is
> enough
> >> > > > > substance
> >> > > > > >> in
> >> > > > > >> > >> Samza that it warrants being a separate project. An
> >> argument
> >> > in
> >> > > > > >> favour
> >> > > > > >> > of
> >> > > > > >> > >> merging would be if we think Kafka has a much stronger
> >> "brand
> >> > > > > >> presence"
> >> > > > > >> > >> than Samza; I'm ambivalent on that one. If the Kafka
> >> project
> >> > is
> >> > > > > >> willing
> >> > > > > >> > to
> >> > > > > >> > >> endorse Samza as the "official" way of doing stateful
> >> stream
> >> > > > > >> > >> transformations, that would probably have much the same
> >> > effect
> >> > > as
> >> > > > > >> > >> re-branding Samza as "Kafka Stream Processors" or
> suchlike.
> >> > > Close
> >> > > > > >> > >> collaboration between the two projects will be needed in
> >> any
> >> > > > case.
> >> > > > > >> > >>
> >> > > > > >> > >> From a project management perspective, I guess the "new
> >> > Samza"
> >> > > > > would
> >> > > > > >> > have
> >> > > > > >> > >> to be developed on a branch alongside ongoing
> maintenance
> >> of
> >> > > the
> >> > > > > >> current
> >> > > > > >> > >> line of development? I think it would be important to
> >> > continue
> >> > > > > >> > supporting
> >> > > > > >> > >> existing users, and provide a graceful migration path to
> >> the
> >> > > new
> >> > > > > >> > version.
> >> > > > > >> > >> Leaving the current versions unsupported and forcing
> people
> >> > to
> >> > > > > >> rewrite
> >> > > > > >> > >> their jobs would send a bad signal.
> >> > > > > >> > >>
> >> > > > > >> > >> Best,
> >> > > > > >> > >> Martin
> >> > > > > >> > >>
> >> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io>
> >> wrote:
> >> > > > > >> > >>
> >> > > > > >> > >>> Hey Garry,
> >> > > > > >> > >>>
> >> > > > > >> > >>> Yeah that's super frustrating. I'd be happy to chat
> more
> >> > about
> >> > > > > this
> >> > > > > >> if
> >> > > > > >> > >>> you'd be interested. I think Chris and I started with
> the
> >> > idea
> >> > > > of
> >> > > > > >> "what
> >> > > > > >> > >>> would it take to make Samza a kick-ass ingestion tool"
> but
> >> > > > > >> ultimately
> >> > > > > >> > we
> >> > > > > >> > >>> kind of came around to the idea that ingestion and
> >> > > > transformation
> >> > > > > >> had
> >> > > > > >> > >>> pretty different needs and coupling the two made things
> >> > hard.
> >> > > > > >> > >>>
> >> > > > > >> > >>> For what it's worth I think copycat (KIP-26) actually
> will
> >> > do
> >> > > > what
> >> > > > > >> you
> >> > > > > >> > >> are
> >> > > > > >> > >>> looking for.
> >> > > > > >> > >>>
> >> > > > > >> > >>> With regard to your point about slider, I don't
> >> necessarily
> >> > > > > >> disagree.
> >> > > > > >> > >> But I
> >> > > > > >> > >>> think getting good YARN support is quite doable and I
> >> think
> >> > we
> >> > > > can
> >> > > > > >> make
> >> > > > > >> > >>> that work well. I think the issue this proposal solves
> is
> >> > that
> >> > > > > >> > >> technically
> >> > > > > >> > >>> it is pretty hard to support multiple cluster
> management
> >> > > systems
> >> > > > > the
> >> > > > > >> > way
> >> > > > > >> > >>> things are now, you need to write an "app master" or
> >> > > "framework"
> >> > > > > for
> >> > > > > >> > each
> >> > > > > >> > >>> and they are all a little different so testing is
> really
> >> > hard.
> >> > > > In
> >> > > > > >> the
> >> > > > > >> > >>> absence of this we have been stuck with just YARN which
> >> has
> >> > > > > >> fantastic
> >> > > > > >> > >>> penetration in the Hadoopy part of the org, but zero
> >> > > penetration
> >> > > > > >> > >> elsewhere.
> >> > > > > >> > >>> Given the huge amount of work being put in to slider,
> >> > > marathon,
> >> > > > > aws
> >> > > > > >> > >>> tooling, not to mention the umpteen related packaging
> >> > > > technologies
> >> > > > > >> > people
> >> > > > > >> > >>> want to use (Docker, Kubernetes, various cloud-specific
> >> > deploy
> >> > > > > >> tools,
> >> > > > > >> > >> etc)
> >> > > > > >> > >>> I really think it is important to get this right.
> >> > > > > >> > >>>
> >> > > > > >> > >>> -Jay
> >> > > > > >> > >>>
> >> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> >> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> >> > > > > >> > >>>
> >> > > > > >> > >>>> Hi all,
> >> > > > > >> > >>>>
> >> > > > > >> > >>>> I think the question below re does Samza become a
> >> > sub-project
> >> > > > of
> >> > > > > >> Kafka
> >> > > > > >> > >>>> highlights the broader point around migration. Chris
> >> > mentions
> >> > > > > >> Samza's
> >> > > > > >> > >>>> maturity is heading towards a v1 release but I'm not
> sure
> >> > it
> >> > > > > feels
> >> > > > > >> > >> right to
> >> > > > > >> > >>>> launch a v1 then immediately plan to deprecate most of
> >> it.
> >> > > > > >> > >>>>
> >> > > > > >> > >>>> From a selfish perspective I have some guys who have
> >> > started
> >> > > > > >> working
> >> > > > > >> > >> with
> >> > > > > >> > >>>> Samza and building some new consumers/producers was
> next
> >> > up.
> >> > > > > Sounds
> >> > > > > >> > like
> >> > > > > >> > >>>> that is absolutely not the direction to go. I need to
> >> look
> >> > > into
> >> > > > > the
> >> > > > > >> > KIP
> >> > > > > >> > >> in
> >> > > > > >> > >>>> more detail but for me the attractiveness of adding
> new
> >> > Samza
> >> > > > > >> > >>>> consumer/producers -- even if yes all they were doing
> was
> >> > > > really
> >> > > > > >> > getting
> >> > > > > >> > >>>> data into and out of Kafka --  was to avoid  having to
> >> > worry
> >> > > > > about
> >> > > > > >> the
> >> > > > > >> > >>>> lifecycle management of external clients. If there is
> a
> >> > > generic
> >> > > > > >> Kafka
> >> > > > > >> > >>>> ingress/egress layer that I can plug a new connector
> into
> >> > and
> >> > > > > have
> >> > > > > >> a
> >> > > > > >> > >> lot of
> >> > > > > >> > >>>> the heavy lifting re scale and reliability done for me
> >> then
> >> > > it
> >> > > > > >> gives
> >> > > > > >> > me
> >> > > > > >> > >> all
> >> > > > > >> > >>>> the pushing new consumers/producers would. If not
> then it
> >> > > > > >> complicates
> >> > > > > >> > my
> >> > > > > >> > >>>> operational deployments.
> >> > > > > >> > >>>>
> >> > > > > >> > >>>> Which is similar to my other question with the
> proposal
> >> --
> >> > if
> >> > > > we
> >> > > > > >> > build a
> >> > > > > >> > >>>> fully available/stand-alone Samza plus the requisite
> >> shims
> >> > to
> >> > > > > >> > integrate
> >> > > > > >> > >>>> with Slider etc I suspect the former may be a lot more
> >> work
> >> > > > than
> >> > > > > we
> >> > > > > >> > >> think.
> >> > > > > >> > >>>> We may make it much easier for a newcomer to get
> >> something
> >> > > > > running
> >> > > > > >> but
> >> > > > > >> > >>>> having them step up and get a reliable production
> >> > deployment
> >> > > > may
> >> > > > > >> still
> >> > > > > >> > >>>> dominate mailing list  traffic, if for different
> reasons
> >> > than
> >> > > > > >> today.
> >> > > > > >> > >>>>
> >> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with making the
> >> Samza
> >> > > > > >> dependency
> >> > > > > >> > >> on
> >> > > > > >> > >>>> Kafka much more explicit and I absolutely see the
> >> benefits
> >> > > in
> >> > > > > the
> >> > > > > >> > >>>> reduction of duplication and clashing
> >> > > > terminologies/abstractions
> >> > > > > >> that
> >> > > > > >> > >>>> Chris/Jay describe. Samza as a library would likely
> be a
> >> > very
> >> > > > > nice
> >> > > > > >> > tool
> >> > > > > >> > >> to
> >> > > > > >> > >>>> add to the Kafka ecosystem. I just have the concerns
> >> above
> >> > re
> >> > > > the
> >> > > > > >> > >>>> operational side.
> >> > > > > >> > >>>>
> >> > > > > >> > >>>> Garry
> >> > > > > >> > >>>>
> >> > > > > >> > >>>> -----Original Message-----
> >> > > > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
> >> > gdfm@apache.org
> >> > > ]
> >> > > > > >> > >>>> Sent: 02 July 2015 12:56
> >> > > > > >> > >>>> To: dev@samza.apache.org
> >> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> >> > > > > >> > >>>>
> >> > > > > >> > >>>> Very interesting thoughts.
> >> > > > > >> > >>>> From outside, I have always perceived Samza as a
> >> computing
> >> > > > layer
> >> > > > > >> over
> >> > > > > >> > >>>> Kafka.
> >> > > > > >> > >>>>
> >> > > > > >> > >>>> The question, maybe a bit provocative, is "should
> Samza
> >> be
> >> > a
> >> > > > > >> > sub-project
> >> > > > > >> > >>>> of Kafka then?"
> >> > > > > >> > >>>> Or does it make sense to keep it as a separate project
> >> > with a
> >> > > > > >> separate
> >> > > > > >> > >>>> governance?
> >> > > > > >> > >>>>
> >> > > > > >> > >>>> Cheers,
> >> > > > > >> > >>>>
> >> > > > > >> > >>>> --
> >> > > > > >> > >>>> Gianmarco
> >> > > > > >> > >>>>
> >> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <
> yanfang724@gmail.com>
> >> > > > wrote:
> >> > > > > >> > >>>>
> >> > > > > >> > >>>>> Overall, I agree to couple with Kafka more tightly.
> >> > Because
> >> > > > > Samza
> >> > > > > >> de
> >> > > > > >> > >>>>> facto is based on Kafka, and it should leverage what
> >> Kafka
> >> > > > has.
> >> > > > > At
> >> > > > > >> > the
> >> > > > > >> > >>>>> same time, Kafka does not need to reinvent what Samza
> >> > > already
> >> > > > > >> has. I
> >> > > > > >> > >>>>> also like the idea of separating the ingestion and
> >> > > > > transformation.
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>> But it is a little difficult for me to image how the
> >> Samza
> >> > > > will
> >> > > > > >> look
> >> > > > > >> > >>>> like.
> >> > > > > >> > >>>>> And I feel Chris and Jay have a little difference in
> >> terms
> >> > > of
> >> > > > > how
> >> > > > > >> > >>>>> Samza should look like.
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>> *** Will it look like what Jay's code shows (A
> client of
> >> > > > Kakfa)
> >> > > > > ?
> >> > > > > >> And
> >> > > > > >> > >>>>> user's application code calls this client?
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka (like what
> the
> >> > > code
> >> > > > > >> shows),
> >> > > > > >> > >>>>> how do we implement auto-balance and fault-tolerance?
> >> Are
> >> > > they
> >> > > > > >> taken
> >> > > > > >> > >>>>> care by the Kafka broker or other mechanism, such as
> >> > "Samza
> >> > > > > >> worker"
> >> > > > > >> > >>>>> (just make up the name) ?
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>> 2. What about other features, such as auto-scaling,
> >> shared
> >> > > > > state,
> >> > > > > >> > >>>>> monitoring?
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>> *** If we have Samza standalone, (is this what Chris
> >> > > > suggests?)
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa and
> produce
> >> to
> >> > > it.
> >> > > > > >> Then it
> >> > > > > >> > >>>>> becomes the same as what Samza looks like now,
> except it
> >> > > does
> >> > > > > not
> >> > > > > >> > rely
> >> > > > > >> > >>>>> on Yarn anymore.
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's
> >> > metrics,
> >> > > > > logs,
> >> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>> Thanks,
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>> Fang, Yan
> >> > > > > >> > >>>>> yanfang724@gmail.com
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> >> > > > > wangguoz@gmail.com
> >> > > > > >> >
> >> > > > > >> > >>>> wrote:
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>>> Read through the code example and it looks good to
> me.
> >> A
> >> > > few
> >> > > > > >> > >>>>>> thoughts regarding deployment:
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>> Today Samza deploys as executable runnable like:
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> >> > > > > >> > >>>> --config-path=file://...
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>> And this proposal advocate for deploying Samza more
> as
> >> > > > embedded
> >> > > > > >> > >>>>>> libraries in user application code (ignoring the
> >> > > terminology
> >> > > > > >> since
> >> > > > > >> > >>>>>> it is not the
> >> > > > > >> > >>>>> same
> >> > > > > >> > >>>>>> as the prototype code):
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread
> >> > thread
> >> > > =
> >> > > > > new
> >> > > > > >> > >>>>>> Thread(task); thread.start();
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>> I think both of these deployment modes are important
> >> for
> >> > > > > >> different
> >> > > > > >> > >>>>>> types
> >> > > > > >> > >>>>> of
> >> > > > > >> > >>>>>> users. That said, I think making Samza purely
> >> standalone
> >> > is
> >> > > > > still
> >> > > > > >> > >>>>>> sufficient for either runnable or library modes.
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>> Guozhang
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> >> > > > jay@confluent.io>
> >> > > > > >> > wrote:
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>>> Looks like gmail mangled the code example, it was
> >> > supposed
> >> > > > to
> >> > > > > >> look
> >> > > > > >> > >>>>>>> like
> >> > > > > >> > >>>>>>> this:
> >> > > > > >> > >>>>>>>
> >> > > > > >> > >>>>>>> Properties props = new Properties();
> >> > > > > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
> >> > > > > >> StreamingConfig
> >> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> >> > > > > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> >> > > > > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> >> > > > > >> > >>>>>>> config.serialization(new StringSerializer(), new
> >> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming container =
> new
> >> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
> >> > > > > >> > >>>>>>>
> >> > > > > >> > >>>>>>> -Jay
> >> > > > > >> > >>>>>>>
> >> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> >> > > > jay@confluent.io
> >> > > > > >
> >> > > > > >> > >>>> wrote:
> >> > > > > >> > >>>>>>>
> >> > > > > >> > >>>>>>>> Hey guys,
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> This came out of some conversations Chris and I
> were
> >> > > having
> >> > > > > >> > >>>>>>>> around
> >> > > > > >> > >>>>>>> whether
> >> > > > > >> > >>>>>>>> it would make sense to use Samza as a kind of data
> >> > > > ingestion
> >> > > > > >> > >>>>> framework
> >> > > > > >> > >>>>>>> for
> >> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat").
> >> This
> >> > > > kind
> >> > > > > of
> >> > > > > >> > >>>>>> combined
> >> > > > > >> > >>>>>>>> with complaints around config and YARN and the
> >> > discussion
> >> > > > > >> around
> >> > > > > >> > >>>>>>>> how
> >> > > > > >> > >>>>> to
> >> > > > > >> > >>>>>>>> best do a standalone mode.
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> So the thought experiment was, given that Samza
> was
> >> > > > basically
> >> > > > > >> > >>>>>>>> already totally Kafka specific, what if you just
> >> > embraced
> >> > > > > that
> >> > > > > >> > >>>>>>>> and turned it
> >> > > > > >> > >>>>>> into
> >> > > > > >> > >>>>>>>> something less like a heavyweight framework and
> more
> >> > > like a
> >> > > > > >> > >>>>>>>> third
> >> > > > > >> > >>>>> Kafka
> >> > > > > >> > >>>>>>>> client--a kind of "producing consumer" with state
> >> > > > management
> >> > > > > >> > >>>>>> facilities.
> >> > > > > >> > >>>>>>>> Basically a library. Instead of a complex stream
> >> > > processing
> >> > > > > >> > >>>>>>>> framework
> >> > > > > >> > >>>>>>> this
> >> > > > > >> > >>>>>>>> would actually be a very simple thing, not much
> more
> >> > > > > >> complicated
> >> > > > > >> > >>>>>>>> to
> >> > > > > >> > >>>>> use
> >> > > > > >> > >>>>>>> or
> >> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we
> >> thought
> >> > > > about
> >> > > > > >> it
> >> > > > > >> > >>>>>>>> a
> >> > > > > >> > >>>>> lot
> >> > > > > >> > >>>>>> of
> >> > > > > >> > >>>>>>>> what Samza (and the other stream processing
> systems
> >> > were
> >> > > > > doing)
> >> > > > > >> > >>>>> seemed
> >> > > > > >> > >>>>>>> like
> >> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> Of course you need to ingest/output data to and
> from
> >> > the
> >> > > > > stream
> >> > > > > >> > >>>>>>>> processing. But when we actually looked into how
> that
> >> > > would
> >> > > > > >> > >>>>>>>> work,
> >> > > > > >> > >>>>> Samza
> >> > > > > >> > >>>>>>>> isn't really an ideal data ingestion framework
> for a
> >> > > bunch
> >> > > > of
> >> > > > > >> > >>>>> reasons.
> >> > > > > >> > >>>>>> To
> >> > > > > >> > >>>>>>>> really do that right you need a pretty different
> >> > internal
> >> > > > > data
> >> > > > > >> > >>>>>>>> model
> >> > > > > >> > >>>>>> and
> >> > > > > >> > >>>>>>>> set of apis. So what if you split them and had an
> api
> >> > for
> >> > > > > Kafka
> >> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate
> >> api
> >> > > for
> >> > > > > >> Kafka
> >> > > > > >> > >>>>>>>> transformation (Samza).
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> This would also allow really embracing the same
> >> > > terminology
> >> > > > > and
> >> > > > > >> > >>>>>>>> conventions. One complaint about the current
> state is
> >> > > that
> >> > > > > the
> >> > > > > >> > >>>>>>>> two
> >> > > > > >> > >>>>>>> systems
> >> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream"
> vs
> >> > > > "topic"
> >> > > > > >> and
> >> > > > > >> > >>>>>>> different
> >> > > > > >> > >>>>>>>> config and monitoring systems means you kind of
> have
> >> to
> >> > > > learn
> >> > > > > >> > >>>>>>>> Kafka's
> >> > > > > >> > >>>>>>> way,
> >> > > > > >> > >>>>>>>> then learn Samza's slightly different way, then
> kind
> >> of
> >> > > > > >> > >>>>>>>> understand
> >> > > > > >> > >>>>> how
> >> > > > > >> > >>>>>>> they
> >> > > > > >> > >>>>>>>> map to each other, which having walked a few
> people
> >> > > through
> >> > > > > >> this
> >> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> Since I have been spending a lot of time on
> >> airplanes I
> >> > > > > hacked
> >> > > > > >> > >>>>>>>> up an ernest but still somewhat incomplete
> prototype
> >> of
> >> > > > what
> >> > > > > >> > >>>>>>>> this would
> >> > > > > >> > >>>>> look
> >> > > > > >> > >>>>>>>> like. This is just unceremoniously dumped into
> Kafka
> >> as
> >> > > it
> >> > > > > >> > >>>>>>>> required a
> >> > > > > >> > >>>>>> few
> >> > > > > >> > >>>>>>>> changes to the new consumer. Here is the code:
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>
> >> > > > > >> >
> >> > > > >
> >> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >> > > > > >> > >>>>> /apache/kafka/clients/streaming
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> For the purpose of the prototype I just liberally
> >> > renamed
> >> > > > > >> > >>>>>>>> everything
> >> > > > > >> > >>>>> to
> >> > > > > >> > >>>>>>>> try to align it with Kafka with no regard for
> >> > > > compatibility.
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> To use this would be something like this:
> >> > > > > >> > >>>>>>>> Properties props = new Properties();
> >> > > > > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> >> > > > > >> > >>>>>>>> StreamingConfig config = new
> >> > > > > >> > >>>>> StreamingConfig(props);
> >> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> >> > > > > >> > >>>>>>>> "test-topic-2");
> >> > > > > >> config.processor(ExampleStreamProcessor.class);
> >> > > > > >> > >>>>>>> config.serialization(new
> >> > > > > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
> >> > > > KafkaStreaming
> >> > > > > >> > >>>>>> container =
> >> > > > > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> >> > > > > StreamProcessor
> >> > > > > >> > >>>>>>>> is basically StreamTask.
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> So rather than putting all the class names in a
> file
> >> > and
> >> > > > then
> >> > > > > >> > >>>>>>>> having
> >> > > > > >> > >>>>>> the
> >> > > > > >> > >>>>>>>> job assembled by reflection, you just instantiate
> the
> >> > > > > container
> >> > > > > >> > >>>>>>>> programmatically. Work is balanced over however
> many
> >> > > > > instances
> >> > > > > >> > >>>>>>>> of
> >> > > > > >> > >>>>> this
> >> > > > > >> > >>>>>>> are
> >> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new
> >> tasks
> >> > > are
> >> > > > > >> added
> >> > > > > >> > >>>>>>>> to
> >> > > > > >> > >>>>> the
> >> > > > > >> > >>>>>>>> existing containers without shutting them down).
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> We would provide some glue for running this stuff
> in
> >> > YARN
> >> > > > via
> >> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of
> >> their
> >> > > > tools
> >> > > > > >> > >>>>>>>> but from the
> >> > > > > >> > >>>>>> point
> >> > > > > >> > >>>>>>> of
> >> > > > > >> > >>>>>>>> view of these frameworks these stream processing
> jobs
> >> > are
> >> > > > > just
> >> > > > > >> > >>>>>> stateless
> >> > > > > >> > >>>>>>>> services that can come and go and expand and
> contract
> >> > at
> >> > > > > will.
> >> > > > > >> > >>>>>>>> There
> >> > > > > >> > >>>>> is
> >> > > > > >> > >>>>>>> no
> >> > > > > >> > >>>>>>>> more custom scheduler.
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> Here are some relevant details:
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get
> >> larger
> >> > > if
> >> > > > we
> >> > > > > >> > >>>>>>>>  productionized but not vastly larger. We really
> do
> >> > get a
> >> > > > ton
> >> > > > > >> > >>>>>>>> of
> >> > > > > >> > >>>>>>> leverage
> >> > > > > >> > >>>>>>>>  out of Kafka.
> >> > > > > >> > >>>>>>>>  2. Partition management is fully delegated to the
> >> new
> >> > > > > >> consumer.
> >> > > > > >> > >>>>> This
> >> > > > > >> > >>>>>>>>  is nice since now any partition management
> strategy
> >> > > > > available
> >> > > > > >> > >>>>>>>> to
> >> > > > > >> > >>>>>> Kafka
> >> > > > > >> > >>>>>>>>  consumer is also available to Samza (and vice
> versa)
> >> > and
> >> > > > > with
> >> > > > > >> > >>>>>>>> the
> >> > > > > >> > >>>>>>> exact
> >> > > > > >> > >>>>>>>>  same configs.
> >> > > > > >> > >>>>>>>>  3. It supports state as well as state reuse
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is thought
> >> provoking.
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> -Jay
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> >> > > > > >> > >>>>>> criccomini@apache.org>
> >> > > > > >> > >>>>>>>> wrote:
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>>> Hey all,
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> I have had some discussions with Samza engineers
> at
> >> > > > LinkedIn
> >> > > > > >> > >>>>>>>>> and
> >> > > > > >> > >>>>>>> Confluent
> >> > > > > >> > >>>>>>>>> and we came up with a few observations and would
> >> like
> >> > to
> >> > > > > >> > >>>>>>>>> propose
> >> > > > > >> > >>>>> some
> >> > > > > >> > >>>>>>>>> changes.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> We've observed some things that I want to call
> out
> >> > about
> >> > > > > >> > >>>>>>>>> Samza's
> >> > > > > >> > >>>>>> design,
> >> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment
> >> system.
> >> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> >> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and
> Kafka's
> >> > > > consumer
> >> > > > > >> > >>>>>>>>> APIs
> >> > > > > >> > >>>>> are
> >> > > > > >> > >>>>>>>>> trying to solve a lot of the same problems.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> All three of these issues are related, but I'll
> >> > address
> >> > > > them
> >> > > > > >> in
> >> > > > > >> > >>>>> order.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Deployment
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic
> >> > > deployment
> >> > > > > >> > >>>>>>>>> scheduler
> >> > > > > >> > >>>>>> such
> >> > > > > >> > >>>>>>>>> as
> >> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza,
> we
> >> > bet
> >> > > > that
> >> > > > > >> > >>>>>>>>> there
> >> > > > > >> > >>>>>> would
> >> > > > > >> > >>>>>>>>> be
> >> > > > > >> > >>>>>>>>> one or two winners in this area, and we could
> >> support
> >> > > > them,
> >> > > > > >> and
> >> > > > > >> > >>>>>>>>> the
> >> > > > > >> > >>>>>> rest
> >> > > > > >> > >>>>>>>>> would go away. In reality, there are many
> >> variations.
> >> > > > > >> > >>>>>>>>> Furthermore,
> >> > > > > >> > >>>>>> many
> >> > > > > >> > >>>>>>>>> people still prefer to just start their
> processors
> >> > like
> >> > > > > normal
> >> > > > > >> > >>>>>>>>> Java processes, and use traditional deployment
> >> scripts
> >> > > > such
> >> > > > > as
> >> > > > > >> > >>>>>>>>> Fabric,
> >> > > > > >> > >>>>>> Chef,
> >> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on
> users
> >> > makes
> >> > > > the
> >> > > > > >> > >>>>>>>>> Samza start-up process really painful for first
> time
> >> > > > users.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was also a
> bit
> >> of
> >> > a
> >> > > > > >> > >>>>>>>>> mis-fire
> >> > > > > >> > >>>>>> because
> >> > > > > >> > >>>>>>>>> of
> >> > > > > >> > >>>>>>>>> a fundamental misunderstanding between the
> nature of
> >> > > batch
> >> > > > > >> jobs
> >> > > > > >> > >>>>>>>>> and
> >> > > > > >> > >>>>>>> stream
> >> > > > > >> > >>>>>>>>> processing jobs. Early on, we made conscious
> effort
> >> to
> >> > > > favor
> >> > > > > >> > >>>>>>>>> the
> >> > > > > >> > >>>>>> Hadoop
> >> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked
> >> and
> >> > > was
> >> > > > > well
> >> > > > > >> > >>>>>>> understood.
> >> > > > > >> > >>>>>>>>> One thing that we missed was that batch jobs
> have a
> >> > > > definite
> >> > > > > >> > >>>>>> beginning,
> >> > > > > >> > >>>>>>>>> and
> >> > > > > >> > >>>>>>>>> end, and stream processing jobs don't (usually).
> >> This
> >> > > > leads
> >> > > > > to
> >> > > > > >> > >>>>>>>>> a
> >> > > > > >> > >>>>> much
> >> > > > > >> > >>>>>>>>> simpler scheduling problem for stream processors.
> >> You
> >> > > > > >> basically
> >> > > > > >> > >>>>>>>>> just
> >> > > > > >> > >>>>>>> need
> >> > > > > >> > >>>>>>>>> to find a place to start the processor, and start
> >> it.
> >> > > The
> >> > > > > way
> >> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of
> a
> >> > > cluster
> >> > > > > >> > >>>>>>>>> being "full". We always
> >> > > > > >> > >>>>>> add
> >> > > > > >> > >>>>>>>>> more machines. The problem with coupling Samza
> with
> >> a
> >> > > > > >> scheduler
> >> > > > > >> > >>>>>>>>> is
> >> > > > > >> > >>>>>> that
> >> > > > > >> > >>>>>>>>> Samza (as a framework) now has to handle
> deployment.
> >> > > This
> >> > > > > >> pulls
> >> > > > > >> > >>>>>>>>> in a
> >> > > > > >> > >>>>>>> bunch
> >> > > > > >> > >>>>>>>>> of things such as configuration distribution
> (config
> >> > > > > stream),
> >> > > > > >> > >>>>>>>>> shell
> >> > > > > >> > >>>>>>> scrips
> >> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the
> .tgz
> >> > > > stuff),
> >> > > > > >> etc.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Another reason for requiring dynamic deployment
> was
> >> to
> >> > > > > support
> >> > > > > >> > >>>>>>>>> data locality. If you want to have locality, you
> >> need
> >> > to
> >> > > > put
> >> > > > > >> > >>>>>>>>> your
> >> > > > > >> > >>>>>> processors
> >> > > > > >> > >>>>>>>>> close to the data they're processing. Upon
> further
> >> > > > > >> > >>>>>>>>> investigation,
> >> > > > > >> > >>>>>>> though,
> >> > > > > >> > >>>>>>>>> this feature is not that beneficial. There is
> some
> >> > good
> >> > > > > >> > >>>>>>>>> discussion
> >> > > > > >> > >>>>>> about
> >> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we
> took
> >> the
> >> > > > > >> > >>>>>>>>> Map/Reduce
> >> > > > > >> > >>>>>> path,
> >> > > > > >> > >>>>>>>>> but
> >> > > > > >> > >>>>>>>>> there are some fundamental differences between
> HDFS
> >> > and
> >> > > > > Kafka.
> >> > > > > >> > >>>>>>>>> HDFS
> >> > > > > >> > >>>>>> has
> >> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to
> >> less
> >> > > > > >> > >>>>>>>>> optimization potential with stream processors on
> top
> >> > of
> >> > > > > Kafka.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> This feature is also used as a crutch. Samza
> doesn't
> >> > > have
> >> > > > > any
> >> > > > > >> > >>>>>>>>> built
> >> > > > > >> > >>>>> in
> >> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the
> >> > > dynamic
> >> > > > > >> > >>>>>>>>> deployment scheduling system to handle restarts
> >> when a
> >> > > > > >> > >>>>>>>>> processor dies. This has
> >> > > > > >> > >>>>>>> made
> >> > > > > >> > >>>>>>>>> it very difficult to write a standalone Samza
> >> > container
> >> > > > > >> > >>>> (SAMZA-516).
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Pluggability
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> In some cases pluggability is good, but I think
> that
> >> > > we've
> >> > > > > >> gone
> >> > > > > >> > >>>>>>>>> too
> >> > > > > >> > >>>>>> far
> >> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> * Pluggable config.
> >> > > > > >> > >>>>>>>>> * Pluggable metrics.
> >> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> >> > > > > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> >> > > > > SystemProducer,
> >> > > > > >> > >>>> etc).
> >> > > > > >> > >>>>>>>>> * Pluggable serdes.
> >> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> >> > > > > >> > >>>>>>>>> * Pluggable strategies for just about every
> >> component
> >> > > > > >> > >>>>> (MessageChooser,
> >> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter,
> etc).
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> There's probably more that I've forgotten, as
> well.
> >> > Some
> >> > > > of
> >> > > > > >> > >>>>>>>>> these
> >> > > > > >> > >>>>> are
> >> > > > > >> > >>>>>>>>> useful, but some have proven not to be. This all
> >> comes
> >> > > at
> >> > > > a
> >> > > > > >> cost:
> >> > > > > >> > >>>>>>>>> complexity. This complexity is making it harder
> for
> >> > our
> >> > > > > users
> >> > > > > >> > >>>>>>>>> to
> >> > > > > >> > >>>>> pick
> >> > > > > >> > >>>>>> up
> >> > > > > >> > >>>>>>>>> and use Samza out of the box. It also makes it
> >> > difficult
> >> > > > for
> >> > > > > >> > >>>>>>>>> Samza developers to reason about what the
> >> > > characteristics
> >> > > > of
> >> > > > > >> > >>>>>>>>> the container (since the characteristics change
> >> > > depending
> >> > > > on
> >> > > > > >> > >>>>>>>>> which plugins are use).
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> The issues with pluggability are most visible in
> the
> >> > > > System
> >> > > > > >> APIs.
> >> > > > > >> > >>>>> What
> >> > > > > >> > >>>>>>>>> Samza really requires to be functional is Kafka
> as
> >> its
> >> > > > > >> > >>>>>>>>> transport
> >> > > > > >> > >>>>>> layer.
> >> > > > > >> > >>>>>>>>> But
> >> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases into one
> >> API:
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> >> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> The current System API supports both of these use
> >> > cases.
> >> > > > The
> >> > > > > >> > >>>>>>>>> problem
> >> > > > > >> > >>>>>> is,
> >> > > > > >> > >>>>>>>>> we
> >> > > > > >> > >>>>>>>>> actually want different features for each use
> case.
> >> By
> >> > > > > >> papering
> >> > > > > >> > >>>>>>>>> over
> >> > > > > >> > >>>>>>> these
> >> > > > > >> > >>>>>>>>> two use cases, and providing a single API, we've
> >> > > > introduced
> >> > > > > a
> >> > > > > >> > >>>>>>>>> ton of
> >> > > > > >> > >>>>>>> leaky
> >> > > > > >> > >>>>>>>>> abstractions.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> For example, what we'd really like in (2) is to
> have
> >> > > > > >> > >>>>>>>>> monotonically increasing longs for offsets (like
> >> > Kafka).
> >> > > > > This
> >> > > > > >> > >>>>>>>>> would be at odds
> >> > > > > >> > >>>>> with
> >> > > > > >> > >>>>>>> (1),
> >> > > > > >> > >>>>>>>>> though, since different systems have different
> >> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> >> > > > > >> > >>>>>>>>> There was discussion both on the mailing list and
> >> the
> >> > > SQL
> >> > > > > >> JIRAs
> >> > > > > >> > >>>>> about
> >> > > > > >> > >>>>>>> the
> >> > > > > >> > >>>>>>>>> need for this.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> The same thing holds true for replayability.
> Kafka
> >> > > allows
> >> > > > us
> >> > > > > >> to
> >> > > > > >> > >>>>> rewind
> >> > > > > >> > >>>>>>>>> when
> >> > > > > >> > >>>>>>>>> we have a failure. Many other systems don't. In
> some
> >> > > > cases,
> >> > > > > >> > >>>>>>>>> systems
> >> > > > > >> > >>>>>>> return
> >> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> >> WikipediaSystemConsumer)
> >> > > > > because
> >> > > > > >> > >>>>>>>>> they
> >> > > > > >> > >>>>>> have
> >> > > > > >> > >>>>>>> no
> >> > > > > >> > >>>>>>>>> offsets.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka supports
> >> > > > > partitioning,
> >> > > > > >> > >>>>>>>>> but
> >> > > > > >> > >>>>> many
> >> > > > > >> > >>>>>>>>> systems don't. We model this by having a single
> >> > > partition
> >> > > > > for
> >> > > > > >> > >>>>>>>>> those systems. Still, other systems model
> >> partitioning
> >> > > > > >> > >>>> differently (e.g.
> >> > > > > >> > >>>>>>>>> Kinesis).
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a mess.
> Creating
> >> > > streams
> >> > > > > in
> >> > > > > >> a
> >> > > > > >> > >>>>>>>>> system-agnostic way is almost impossible. As is
> >> > modeling
> >> > > > > >> > >>>>>>>>> metadata
> >> > > > > >> > >>>>> for
> >> > > > > >> > >>>>>>> the
> >> > > > > >> > >>>>>>>>> system (replication factor, partitions, location,
> >> > etc).
> >> > > > The
> >> > > > > >> > >>>>>>>>> list
> >> > > > > >> > >>>>> goes
> >> > > > > >> > >>>>>>> on.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Duplicate work
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's
> >> > > consumer
> >> > > > > and
> >> > > > > >> > >>>>> producer
> >> > > > > >> > >>>>>>>>> APIs
> >> > > > > >> > >>>>>>>>> had a relatively weak feature set. On the
> >> > consumer-side,
> >> > > > you
> >> > > > > >> > >>>>>>>>> had two
> >> > > > > >> > >>>>>>>>> options: use the high level consumer, or the
> simple
> >> > > > > consumer.
> >> > > > > >> > >>>>>>>>> The
> >> > > > > >> > >>>>>>> problem
> >> > > > > >> > >>>>>>>>> with the high-level consumer was that it
> controlled
> >> > your
> >> > > > > >> > >>>>>>>>> offsets, partition assignments, and the order in
> >> which
> >> > > you
> >> > > > > >> > >>>>>>>>> received messages. The
> >> > > > > >> > >>>>> problem
> >> > > > > >> > >>>>>>>>> with
> >> > > > > >> > >>>>>>>>> the simple consumer is that it's not simple. It's
> >> > basic.
> >> > > > You
> >> > > > > >> > >>>>>>>>> end up
> >> > > > > >> > >>>>>>> having
> >> > > > > >> > >>>>>>>>> to handle a lot of really low-level stuff that
> you
> >> > > > > shouldn't.
> >> > > > > >> > >>>>>>>>> We
> >> > > > > >> > >>>>>> spent a
> >> > > > > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer
> very
> >> > > > robust.
> >> > > > > >> It
> >> > > > > >> > >>>>>>>>> also allows us to support some cool features:
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> * Per-partition message ordering and
> prioritization.
> >> > > > > >> > >>>>>>>>> * Tight control over partition assignment to
> support
> >> > > > joins,
> >> > > > > >> > >>>>>>>>> global
> >> > > > > >> > >>>>>> state
> >> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> >> > > > > >> > >>>>>>>>> * Tight control over offset checkpointing.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> What we didn't realize at the time is that these
> >> > > features
> >> > > > > >> > >>>>>>>>> should
> >> > > > > >> > >>>>>>> actually
> >> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just
> >> Samza
> >> > > > stream
> >> > > > > >> > >>>>>> processors)
> >> > > > > >> > >>>>>>>>> end up wanting to do things like joins and
> partition
> >> > > > > >> > >>>>>>>>> assignment. The
> >> > > > > >> > >>>>>>> Kafka
> >> > > > > >> > >>>>>>>>> community has come to the same conclusion.
> They're
> >> > > adding
> >> > > > a
> >> > > > > >> ton
> >> > > > > >> > >>>>>>>>> of upgrades into their new Kafka consumer
> >> > > implementation.
> >> > > > > To a
> >> > > > > >> > >>>>>>>>> large extent,
> >> > > > > >> > >>>>> it's
> >> > > > > >> > >>>>>>>>> duplicate work to what we've already done in
> Samza.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a very
> similar
> >> > > > > approach
> >> > > > > >> > >>>>>>>>> to
> >> > > > > >> > >>>>>> Samza's
> >> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation for
> handling
> >> > > offset
> >> > > > > >> > >>>>>> checkpointing.
> >> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature
> >> > stores
> >> > > > > >> offset
> >> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch
> them
> >> > > from
> >> > > > > the
> >> > > > > >> > >>>>>>>>> broker.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since we could
> >> have
> >> > > > shared
> >> > > > > >> > >>>>>>>>> the
> >> > > > > >> > >>>>> work
> >> > > > > >> > >>>>>> if
> >> > > > > >> > >>>>>>>>> it
> >> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Vision
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> All of this leads me to a rather radical
> proposal.
> >> > Samza
> >> > > > is
> >> > > > > >> > >>>>> relatively
> >> > > > > >> > >>>>>>>>> stable at this point. I'd venture to say that
> we're
> >> > > near a
> >> > > > > 1.0
> >> > > > > >> > >>>>>> release.
> >> > > > > >> > >>>>>>>>> I'd
> >> > > > > >> > >>>>>>>>> like to propose that we take what we've learned,
> and
> >> > > begin
> >> > > > > >> > >>>>>>>>> thinking
> >> > > > > >> > >>>>>>> about
> >> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were
> >> > > starting
> >> > > > > >> from
> >> > > > > >> > >>>>>> scratch?
> >> > > > > >> > >>>>>>>>> My
> >> > > > > >> > >>>>>>>>> proposal is to:
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run
> Samza
> >> > > > > >> > >>>>>>>>> processors, and eliminate all direct dependences
> on
> >> > > YARN,
> >> > > > > >> Mesos,
> >> > > > > >> > >>>> etc.
> >> > > > > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka
> as
> >> the
> >> > > > > stream
> >> > > > > >> > >>>>>> processing
> >> > > > > >> > >>>>>>>>> layer.
> >> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
> >> serialization,
> >> > > and
> >> > > > > >> > >>>>>>>>> config
> >> > > > > >> > >>>>>>> systems,
> >> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> This would fix all of the issues that I outlined
> >> > above.
> >> > > It
> >> > > > > >> > >>>>>>>>> should
> >> > > > > >> > >>>>> also
> >> > > > > >> > >>>>>>>>> shrink the Samza code base pretty dramatically.
> >> > > Supporting
> >> > > > > >> only
> >> > > > > >> > >>>>>>>>> a standalone container will allow Samza to be
> >> executed
> >> > > on
> >> > > > > YARN
> >> > > > > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or
> >> most
> >> > > > other
> >> > > > > >> > >>>>>>>>> in-house
> >> > > > > >> > >>>>>>> deployment
> >> > > > > >> > >>>>>>>>> systems. This should make life a lot easier for
> new
> >> > > users.
> >> > > > > >> > >>>>>>>>> Imagine
> >> > > > > >> > >>>>>>> having
> >> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop
> in
> >> > > mailing
> >> > > > > >> list
> >> > > > > >> > >>>>>> traffic
> >> > > > > >> > >>>>>>>>> will be pretty dramatic.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The
> >> > > reality
> >> > > > > is,
> >> > > > > >> > >>>>> everyone
> >> > > > > >> > >>>>>>>>> that
> >> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We
> basically
> >> > > > require
> >> > > > > >> it
> >> > > > > >> > >>>>>> already
> >> > > > > >> > >>>>>>> in
> >> > > > > >> > >>>>>>>>> order for most features to work. Those that are
> >> using
> >> > > > other
> >> > > > > >> > >>>>>>>>> systems
> >> > > > > >> > >>>>>> are
> >> > > > > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and
> >> then
> >> > > > they
> >> > > > > do
> >> > > > > >> > >>>>>>>>> the processing on top. There is already
> discussion (
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>
> >> > > > > >> >
> >> > > > >
> >> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >> > > > > >> > >>>>> 767
> >> > > > > >> > >>>>>>>>> )
> >> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely
> >> easy.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Once we make the call to couple with Kafka, we
> can
> >> > > > leverage
> >> > > > > a
> >> > > > > >> > >>>>>>>>> ton of
> >> > > > > >> > >>>>>>> their
> >> > > > > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own
> >> > config,
> >> > > > > >> > >>>>>>>>> metrics,
> >> > > > > >> > >>>>> etc.
> >> > > > > >> > >>>>>>> We
> >> > > > > >> > >>>>>>>>> can all share the same libraries, and make them
> >> > better.
> >> > > > This
> >> > > > > >> > >>>>>>>>> will
> >> > > > > >> > >>>>> also
> >> > > > > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and
> >> will
> >> > > let
> >> > > > > us
> >> > > > > >> > >>>>> leverage
> >> > > > > >> > >>>>>>>>> their offset management and partition management,
> >> > rather
> >> > > > > than
> >> > > > > >> > >>>>>>>>> having
> >> > > > > >> > >>>>>> our
> >> > > > > >> > >>>>>>>>> own. All of the coordinator stream code would go
> >> away,
> >> > > as
> >> > > > > >> would
> >> > > > > >> > >>>>>>>>> most
> >> > > > > >> > >>>>>> of
> >> > > > > >> > >>>>>>>>> the
> >> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push
> some
> >> > > > > partition
> >> > > > > >> > >>>>>>> management
> >> > > > > >> > >>>>>>>>> features into the Kafka broker, but they're
> already
> >> > > moving
> >> > > > > in
> >> > > > > >> > >>>>>>>>> that direction with the new consumer API. The
> >> features
> >> > > we
> >> > > > > have
> >> > > > > >> > >>>>>>>>> for
> >> > > > > >> > >>>>>> partition
> >> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like
> >> they
> >> > > > should
> >> > > > > >> be
> >> > > > > >> > >>>>>>>>> in
> >> > > > > >> > >>>>>> Kafka
> >> > > > > >> > >>>>>>>>> anyway. There will always be some niche usages
> which
> >> > > will
> >> > > > > >> > >>>>>>>>> require
> >> > > > > >> > >>>>>> extra
> >> > > > > >> > >>>>>>>>> care and hence full control over partition
> >> assignments
> >> > > > much
> >> > > > > >> > >>>>>>>>> like the
> >> > > > > >> > >>>>>>> Kafka
> >> > > > > >> > >>>>>>>>> low level consumer api. These would continue to
> be
> >> > > > > supported.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> These items will be good for the Samza community.
> >> > > They'll
> >> > > > > make
> >> > > > > >> > >>>>>>>>> Samza easier to use, and make it easier for
> >> developers
> >> > > to
> >> > > > > add
> >> > > > > >> > >>>>>>>>> new features.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat
> >> > backwards
> >> > > > > >> > >>>>> incompatible
> >> > > > > >> > >>>>>>>>> change). If we choose to go this route, it's
> >> important
> >> > > > that
> >> > > > > we
> >> > > > > >> > >>>>> openly
> >> > > > > >> > >>>>>>>>> communicate how we're going to provide a
> migration
> >> > path
> >> > > > from
> >> > > > > >> > >>>>>>>>> the
> >> > > > > >> > >>>>>>> existing
> >> > > > > >> > >>>>>>>>> APIs to the new ones (if we make incompatible
> >> > changes).
> >> > > I
> >> > > > > >> think
> >> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to provide a
> >> wrapper
> >> > to
> >> > > > > allow
> >> > > > > >> > >>>>>>>>> existing StreamTask implementations to continue
> >> > running
> >> > > on
> >> > > > > the
> >> > > > > >> > >>>> new container.
> >> > > > > >> > >>>>>>> It's
> >> > > > > >> > >>>>>>>>> also important that we openly communicate about
> >> > timing,
> >> > > > and
> >> > > > > >> > >>>>>>>>> stages
> >> > > > > >> > >>>>> of
> >> > > > > >> > >>>>>>> the
> >> > > > > >> > >>>>>>>>> migration.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you have
> opinions.
> >> > :)
> >> > > > > Please
> >> > > > > >> > >>>>>>>>> send
> >> > > > > >> > >>>>>> your
> >> > > > > >> > >>>>>>>>> thoughts and feedback.
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>> Cheers,
> >> > > > > >> > >>>>>>>>> Chris
> >> > > > > >> > >>>>>>>>>
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>>
> >> > > > > >> > >>>>>>>
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>> --
> >> > > > > >> > >>>>>> -- Guozhang
> >> > > > > >> > >>>>>>
> >> > > > > >> > >>>>>
> >> > > > > >> > >>>>
> >> > > > > >> > >>
> >> > > > > >> > >>
> >> > > > > >> >
> >> > > > > >> >
> >> > > > > >> >
> >> > > > > >>
> >> > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
>

Re: Thoughts and obesrvations on Samza

Posted by Jakob Homan <jg...@gmail.com>.
>  This leads me to thinking that merging projects and communities might be a good idea: with the union of experience from both communities, we will probably build a better system that is better for users.
Is this what's being proposed though? Merging the projects seems like
a consequence of at most one of the three directions under discussion:
1) Samza 2.0: The Samza community relies more heavily on Kafka for
configuration, etc. (to a greater or lesser extent to be determined)
but the Samza community would not automatically merge withe Kafka
community (the Phoenix/HBase example is a good one here).
2) Samza Reboot: The Samza community continues to exist with a limited
project scope, but similarly would not need to be part of the Kafka
community (ie given committership) to progress.  Here, maybe the Samza
team would become a subproject of Kafka (the Board frowns on
subprojects at the moment, so I'm not sure if that's even feasible),
but that would not be required.
3) Hey Samza! FYI, Kafka does streaming now: In this option the Kafka
team builds its own streaming library, possibly off of Jay's
prototype, which has not direct lineage to the Samza team.  There's no
reason for the Kafka team to bring in the Samza team.

Is the Kafka community on board with this?

To be clear, all three options under discussion are interesting,
technically valid and likely healthy directions for the project.
Also, they are not mutually exclusive.  The Samza community could
decide to pursue, say, 'Samza 2.0', while the Kafka community went
forward with 'Hey Samza!'  My points above are directed entirely at
the community aspect of these choices.
-Jakob

On 10 July 2015 at 09:10, Roger Hoover <ro...@gmail.com> wrote:
> That's great.  Thanks, Jay.
>
> On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io> wrote:
>
>> Yeah totally agree. I think you have this issue even today, right? I.e. if
>> you need to make a simple config change and you're running in YARN today
>> you end up bouncing the job which then rebuilds state. I think the fix is
>> exactly what you described which is to have a long timeout on partition
>> movement for stateful jobs so that if a job is just getting bounced, and
>> the cluster manager (or admin) is smart enough to restart it on the same
>> host when possible, it can optimistically reuse any existing state it finds
>> on disk (if it is valid).
>>
>> So in this model the charter of the CM is to place processes as stickily as
>> possible and to restart or re-place failed processes. The charter of the
>> partition management system is to control the assignment of work to these
>> processes. The nice thing about this is that the work assignment, timeouts,
>> behavior, configs, and code will all be the same across all cluster
>> managers.
>>
>> So I think that prototype would actually give you exactly what you want
>> today for any cluster manager (or manual placement + restart script) that
>> was sticky in terms of host placement since there is already a configurable
>> partition movement timeout and task-by-task state reuse with a check on
>> state validity.
>>
>> -Jay
>>
>> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <ro...@gmail.com>
>> wrote:
>>
>> > That would be great to let Kafka do as much heavy lifting as possible and
>> > make it easier for other languages to implement Samza apis.
>> >
>> > One thing to watch out for is the interplay between Kafka's group
>> > management and the external scheduler/process manager's fault tolerance.
>> > If a container dies, the Kafka group membership protocol will try to
>> assign
>> > it's tasks to other containers while at the same time the process manager
>> > is trying to relaunch the container.  Without some consideration for this
>> > (like a configurable amount of time to wait before Kafka alters the group
>> > membership), there may be thrashing going on which is especially bad for
>> > containers with large amounts of local state.
>> >
>> > Someone else pointed this out already but I thought it might be worth
>> > calling out again.
>> >
>> > Cheers,
>> >
>> > Roger
>> >
>> >
>> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <ja...@confluent.io> wrote:
>> >
>> > > Hey Roger,
>> > >
>> > > I couldn't agree more. We spent a bunch of time talking to people and
>> > that
>> > > is exactly the stuff we heard time and again. What makes it hard, of
>> > > course, is that there is some tension between compatibility with what's
>> > > there now and making things better for new users.
>> > >
>> > > I also strongly agree with the importance of multi-language support. We
>> > are
>> > > talking now about Java, but for application development use cases
>> people
>> > > want to work in whatever language they are using elsewhere. I think
>> > moving
>> > > to a model where Kafka itself does the group membership, lifecycle
>> > control,
>> > > and partition assignment has the advantage of putting all that complex
>> > > stuff behind a clean api that the clients are already going to be
>> > > implementing for their consumer, so the added functionality for stream
>> > > processing beyond a consumer becomes very minor.
>> > >
>> > > -Jay
>> > >
>> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <ro...@gmail.com>
>> > > wrote:
>> > >
>> > > > Metamorphosis...nice. :)
>> > > >
>> > > > This has been a great discussion.  As a user of Samza who's recently
>> > > > integrated it into a relatively large organization, I just want to
>> add
>> > > > support to a few points already made.
>> > > >
>> > > > The biggest hurdles to adoption of Samza as it currently exists that
>> > I've
>> > > > experienced are:
>> > > > 1) YARN - YARN is overly complex in many environments where Puppet
>> > would
>> > > do
>> > > > just fine but it was the only mechanism to get fault tolerance.
>> > > > 2) Configuration - I think I like the idea of configuring most of the
>> > job
>> > > > in code rather than config files.  In general, I think the goal
>> should
>> > be
>> > > > to make it harder to make mistakes, especially of the kind where the
>> > code
>> > > > expects something and the config doesn't match.  The current config
>> is
>> > > > quite intricate and error-prone.  For example, the application logic
>> > may
>> > > > depend on bootstrapping a topic but rather than asserting that in the
>> > > code,
>> > > > you have to rely on getting the config right.  Likewise with serdes,
>> > the
>> > > > Java representations produced by various serdes (JSON, Avro, etc.)
>> are
>> > > not
>> > > > equivalent so you cannot just reconfigure a serde without changing
>> the
>> > > > code.   It would be nice for jobs to be able to assert what they
>> expect
>> > > > from their input topics in terms of partitioning.  This is getting a
>> > > little
>> > > > off topic but I was even thinking about creating a "Samza config
>> > linter"
>> > > > that would sanity check a set of configs.  Especially in
>> organizations
>> > > > where config is managed by a different team than the application
>> > > developer,
>> > > > it's very hard to get avoid config mistakes.
>> > > > 3) Java/Scala centric - for many teams (especially DevOps-type
>> folks),
>> > > the
>> > > > pain of the Java toolchain (maven, slow builds, weak command line
>> > > support,
>> > > > configuration over convention) really inhibits productivity.  As more
>> > and
>> > > > more high-quality clients become available for Kafka, I hope they'll
>> > > follow
>> > > > Samza's model.  Not sure how much it affects the proposals in this
>> > thread
>> > > > but please consider other languages in the ecosystem as well.  From
>> > what
>> > > > I've heard, Spark has more Python users than Java/Scala.
>> > > > (FYI, we added a Jython wrapper for the Samza API
>> > > >
>> > > >
>> > >
>> >
>> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
>> > > > and are working on a Yeoman generator
>> > > > https://github.com/Quantiply/generator-rico for Jython/Samza
>> projects
>> > to
>> > > > alleviate some of the pain)
>> > > >
>> > > > I also want to underscore Jay's point about improving the user
>> > > experience.
>> > > > That's a very important factor for adoption.  I think the goal should
>> > be
>> > > to
>> > > > make Samza as easy to get started with as something like Logstash.
>> > > > Logstash is vastly inferior in terms of capabilities to Samza but
>> it's
>> > > easy
>> > > > to get started and that makes a big difference.
>> > > >
>> > > > Cheers,
>> > > >
>> > > > Roger
>> > > >
>> > > >
>> > > >
>> > > >
>> > > >
>> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
>> > > > gdfm@apache.org> wrote:
>> > > >
>> > > > > Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
>> > > > winner
>> > > > > :)
>> > > > >
>> > > > > --
>> > > > > Gianmarco
>> > > > >
>> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
>> > > gdfm@apache.org
>> > > > >
>> > > > > wrote:
>> > > > >
>> > > > > > Hi,
>> > > > > >
>> > > > > > @Martin, thanks for you comments.
>> > > > > > Maybe I'm missing some important point, but I think coupling the
>> > > > releases
>> > > > > > is actually a *good* thing.
>> > > > > > To make an example, would it be better if the MR and HDFS
>> > components
>> > > of
>> > > > > > Hadoop had different release schedules?
>> > > > > >
>> > > > > > Actually, keeping the discussion in a single place would make
>> > > agreeing
>> > > > on
>> > > > > > releases (and backwards compatibility) much easier, as everybody
>> > > would
>> > > > be
>> > > > > > responsible for the whole codebase.
>> > > > > >
>> > > > > > That said, I like the idea of absorbing samza-core as a
>> > sub-project,
>> > > > and
>> > > > > > leave the fancy stuff separate.
>> > > > > > It probably gives 90% of the benefits we have been discussing
>> here.
>> > > > > >
>> > > > > > Cheers,
>> > > > > >
>> > > > > > --
>> > > > > > Gianmarco
>> > > > > >
>> > > > > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
>> > > > > >
>> > > > > >> Hey Martin,
>> > > > > >>
>> > > > > >> I agree coupling release schedules is a downside.
>> > > > > >>
>> > > > > >> Definitely we can try to solve some of the integration problems
>> in
>> > > > > >> Confluent Platform or in other distributions. But I think this
>> > ends
>> > > up
>> > > > > >> being really shallow. I guess I feel to really get a good user
>> > > > > experience
>> > > > > >> the two systems have to kind of feel like part of the same thing
>> > and
>> > > > you
>> > > > > >> can't really add that in later--you can put both in the same
>> > > > > downloadable
>> > > > > >> tar file but it doesn't really give a very cohesive feeling. I
>> > agree
>> > > > > that
>> > > > > >> ultimately any of the project stuff is as much social and naming
>> > as
>> > > > > >> anything else--theoretically two totally independent projects
>> > could
>> > > > work
>> > > > > >> to
>> > > > > >> tightly align. In practice this seems to be quite difficult
>> > though.
>> > > > > >>
>> > > > > >> For the frameworks--totally agree it would be good to maintain
>> the
>> > > > > >> framework support with the project. In some cases there may not
>> be
>> > > too
>> > > > > >> much
>> > > > > >> there since the integration gets lighter but I think whatever
>> > stubs
>> > > > you
>> > > > > >> need should be included. So no I definitely wasn't trying to
>> imply
>> > > > > >> dropping
>> > > > > >> support for these frameworks, just making the integration
>> lighter
>> > by
>> > > > > >> separating process management from partition management.
>> > > > > >>
>> > > > > >> You raise two good points we would have to figure out if we went
>> > > down
>> > > > > the
>> > > > > >> alignment path:
>> > > > > >> 1. With respect to the name, yeah I think the first question is
>> > > > whether
>> > > > > >> some "re-branding" would be worth it. If so then I think we can
>> > > have a
>> > > > > big
>> > > > > >> thread on the name. I'm definitely not set on Kafka Streaming or
>> > > Kafka
>> > > > > >> Streams I was just using them to be kind of illustrative. I
>> agree
>> > > with
>> > > > > >> your
>> > > > > >> critique of these names, though I think people would get the
>> idea.
>> > > > > >> 2. Yeah you also raise a good point about how to "factor" it.
>> Here
>> > > are
>> > > > > the
>> > > > > >> options I see (I could get enthusiastic about any of them):
>> > > > > >>    a. One repo for both Kafka and Samza
>> > > > > >>    b. Two repos, retaining the current seperation
>> > > > > >>    c. Two repos, the equivalent of samza-api and samza-core is
>> > > > absorbed
>> > > > > >> almost like a third client
>> > > > > >>
>> > > > > >> Cheers,
>> > > > > >>
>> > > > > >> -Jay
>> > > > > >>
>> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
>> > > > martin@kleppmann.com>
>> > > > > >> wrote:
>> > > > > >>
>> > > > > >> > Ok, thanks for the clarifications. Just a few follow-up
>> > comments.
>> > > > > >> >
>> > > > > >> > - I see the appeal of merging with Kafka or becoming a
>> > subproject:
>> > > > the
>> > > > > >> > reasons you mention are good. The risk I see is that release
>> > > > schedules
>> > > > > >> > become coupled to each other, which can slow everyone down,
>> and
>> > > > large
>> > > > > >> > projects with many contributors are harder to manage. (Jakob,
>> > can
>> > > > you
>> > > > > >> speak
>> > > > > >> > from experience, having seen a wider range of Hadoop ecosystem
>> > > > > >> projects?)
>> > > > > >> >
>> > > > > >> > Some of the goals of a better unified developer experience
>> could
>> > > > also
>> > > > > be
>> > > > > >> > solved by integrating Samza nicely into a Kafka distribution
>> > (such
>> > > > as
>> > > > > >> > Confluent's). I'm not against merging projects if we decide
>> > that's
>> > > > the
>> > > > > >> way
>> > > > > >> > to go, just pointing out the same goals can perhaps also be
>> > > achieved
>> > > > > in
>> > > > > >> > other ways.
>> > > > > >> >
>> > > > > >> > - With regard to dropping the YARN dependency: are you
>> proposing
>> > > > that
>> > > > > >> > Samza doesn't give any help to people wanting to run on
>> > > > > >> YARN/Mesos/AWS/etc?
>> > > > > >> > So the docs would basically have a link to Slider and nothing
>> > > else?
>> > > > Or
>> > > > > >> > would we maintain integrations with a bunch of popular
>> > deployment
>> > > > > >> methods
>> > > > > >> > (e.g. the necessary glue and shell scripts to make Samza work
>> > with
>> > > > > >> Slider)?
>> > > > > >> >
>> > > > > >> > I absolutely think it's a good idea to have the "as a library"
>> > and
>> > > > > "as a
>> > > > > >> > process" (using Yi's taxonomy) options for people who want
>> them,
>> > > > but I
>> > > > > >> > think there should also be a low-friction path for common "as
>> a
>> > > > > service"
>> > > > > >> > deployment methods, for which we probably need to maintain
>> > > > > integrations.
>> > > > > >> >
>> > > > > >> > - Project naming: "Kafka Streams" seems odd to me, because
>> Kafka
>> > > is
>> > > > > all
>> > > > > >> > about streams already. Perhaps "Kafka Transformers" or "Kafka
>> > > > Filters"
>> > > > > >> > would be more apt?
>> > > > > >> >
>> > > > > >> > One suggestion: perhaps the core of Samza (stream
>> transformation
>> > > > with
>> > > > > >> > state management -- i.e. the "Samza as a library" bit) could
>> > > become
>> > > > > >> part of
>> > > > > >> > Kafka, while higher-level tools such as streaming SQL and
>> > > > integrations
>> > > > > >> with
>> > > > > >> > deployment frameworks remain in a separate project? In other
>> > > words,
>> > > > > >> Kafka
>> > > > > >> > would absorb the proven, stable core of Samza, which would
>> > become
>> > > > the
>> > > > > >> > "third Kafka client" mentioned early in this thread. The Samza
>> > > > project
>> > > > > >> > would then target that third Kafka client as its base API, and
>> > the
>> > > > > >> project
>> > > > > >> > would be freed up to explore more experimental new horizons.
>> > > > > >> >
>> > > > > >> > Martin
>> > > > > >> >
>> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com>
>> wrote:
>> > > > > >> >
>> > > > > >> > > Hey Martin,
>> > > > > >> > >
>> > > > > >> > > For the YARN/Mesos/etc decoupling I actually don't think it
>> > ties
>> > > > our
>> > > > > >> > hands
>> > > > > >> > > at all, all it does is refactor things. The division of
>> > > > > >> responsibility is
>> > > > > >> > > that Samza core is responsible for task lifecycle, state,
>> and
>> > > > > >> partition
>> > > > > >> > > management (using the Kafka co-ordinator) but it is NOT
>> > > > responsible
>> > > > > >> for
>> > > > > >> > > packaging, configuration deployment or execution of
>> processes.
>> > > The
>> > > > > >> > problem
>> > > > > >> > > of packaging and starting these processes is
>> > > > > >> > > framework/environment-specific. This leaves individual
>> > > frameworks
>> > > > to
>> > > > > >> be
>> > > > > >> > as
>> > > > > >> > > fancy or vanilla as they like. So you can get simple
>> stateless
>> > > > > >> support in
>> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app framework
>> > > (Slider,
>> > > > > >> > Marathon,
>> > > > > >> > > etc). These are well known by people and have nice UIs and a
>> > lot
>> > > > of
>> > > > > >> > > flexibility. I don't think they have node affinity as a
>> built
>> > in
>> > > > > >> option
>> > > > > >> > > (though I could be wrong). So if we want that we can either
>> > wait
>> > > > for
>> > > > > >> them
>> > > > > >> > > to add it or do a custom framework to add that feature (as
>> > now).
>> > > > > >> > Obviously
>> > > > > >> > > if you manage things with old-school ops tools
>> > (puppet/chef/etc)
>> > > > you
>> > > > > >> get
>> > > > > >> > > locality easily. The nice thing, though, is that all the
>> samza
>> > > > > >> "business
>> > > > > >> > > logic" around partition management and fault tolerance is in
>> > > Samza
>> > > > > >> core
>> > > > > >> > so
>> > > > > >> > > it is shared across frameworks and the framework specific
>> bit
>> > is
>> > > > > just
>> > > > > >> > > whether it is smart enough to try to get the same host when
>> a
>> > > job
>> > > > is
>> > > > > >> > > restarted.
>> > > > > >> > >
>> > > > > >> > > With respect to the Kafka-alignment, yeah I think the goal
>> > would
>> > > > be
>> > > > > >> (a)
>> > > > > >> > > actually get better alignment in user experience, and (b)
>> > > express
>> > > > > >> this in
>> > > > > >> > > the naming and project branding. Specifically:
>> > > > > >> > > 1. Website/docs, it would be nice for the "transformation"
>> api
>> > > to
>> > > > be
>> > > > > >> > > discoverable in the main Kafka docs--i.e. be able to explain
>> > > when
>> > > > to
>> > > > > >> use
>> > > > > >> > > the consumer and when to use the stream processing
>> > functionality
>> > > > and
>> > > > > >> lead
>> > > > > >> > > people into that experience.
>> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever)
>> > that
>> > > > has
>> > > > > >> both
>> > > > > >> > > Kafka and the stream processing part and they actually work
>> > > > > together.
>> > > > > >> > > 3. Unify the programming experience so the client and Samza
>> > api
>> > > > > share
>> > > > > >> > > config/monitoring/naming/packaging/etc.
>> > > > > >> > >
>> > > > > >> > > I think sub-projects keep separate committers and can have a
>> > > > > separate
>> > > > > >> > repo,
>> > > > > >> > > but I'm actually not really sure (I can't find a definition
>> > of a
>> > > > > >> > subproject
>> > > > > >> > > in Apache).
>> > > > > >> > >
>> > > > > >> > > Basically at a high-level you want the experience to "feel"
>> > > like a
>> > > > > >> single
>> > > > > >> > > system, not to relatively independent things that are kind
>> of
>> > > > > >> awkwardly
>> > > > > >> > > glued together.
>> > > > > >> > >
>> > > > > >> > > I think if we did that they having naming or branding like
>> > > "kafka
>> > > > > >> > > streaming" or "kafka streams" or something like that would
>> > > > actually
>> > > > > >> do a
>> > > > > >> > > good job of conveying what it is. I do that this would help
>> > > > adoption
>> > > > > >> > quite
>> > > > > >> > > a lot as it would correctly convey that using Kafka
>> Streaming
>> > > with
>> > > > > >> Kafka
>> > > > > >> > is
>> > > > > >> > > a fairly seamless experience and Kafka is pretty heavily
>> > adopted
>> > > > at
>> > > > > >> this
>> > > > > >> > > point.
>> > > > > >> > >
>> > > > > >> > > Fwiw we actually considered this model originally when open
>> > > > sourcing
>> > > > > >> > Samza,
>> > > > > >> > > however at that time Kafka was relatively unknown and we
>> > decided
>> > > > not
>> > > > > >> to
>> > > > > >> > do
>> > > > > >> > > it since we felt it would be limiting. From my point of view
>> > the
>> > > > > three
>> > > > > >> > > things have changed (1) Kafka is now really heavily used for
>> > > > stream
>> > > > > >> > > processing, (2) we learned that abstracting out the stream
>> > well
>> > > is
>> > > > > >> > > basically impossible, (3) we learned it is really hard to
>> keep
>> > > the
>> > > > > two
>> > > > > >> > > things feeling like a single product.
>> > > > > >> > >
>> > > > > >> > > -Jay
>> > > > > >> > >
>> > > > > >> > >
>> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
>> > > > > >> martin@kleppmann.com>
>> > > > > >> > > wrote:
>> > > > > >> > >
>> > > > > >> > >> Hi all,
>> > > > > >> > >>
>> > > > > >> > >> Lots of good thoughts here.
>> > > > > >> > >>
>> > > > > >> > >> I agree with the general philosophy of tying Samza more
>> > firmly
>> > > to
>> > > > > >> Kafka.
>> > > > > >> > >> After I spent a while looking at integrating other message
>> > > > brokers
>> > > > > >> (e.g.
>> > > > > >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
>> > > > > >> > SystemConsumer
>> > > > > >> > >> tacitly assumes a model so much like Kafka's that pretty
>> much
>> > > > > nobody
>> > > > > >> but
>> > > > > >> > >> Kafka actually implements it. (Databus is perhaps an
>> > exception,
>> > > > but
>> > > > > >> it
>> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza
>> > > fully
>> > > > > >> > dependent
>> > > > > >> > >> on Kafka acknowledges that the system-independence was
>> never
>> > as
>> > > > > real
>> > > > > >> as
>> > > > > >> > we
>> > > > > >> > >> perhaps made it out to be. The gains of code reuse are
>> real.
>> > > > > >> > >>
>> > > > > >> > >> The idea of decoupling Samza from YARN has also always been
>> > > > > >> appealing to
>> > > > > >> > >> me, for various reasons already mentioned in this thread.
>> > > > Although
>> > > > > >> > making
>> > > > > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc)
>> seems
>> > > > > >> laudable,
>> > > > > >> > I am
>> > > > > >> > >> a little concerned that it will restrict us to a lowest
>> > common
>> > > > > >> > denominator.
>> > > > > >> > >> For example, would host affinity (SAMZA-617) still be
>> > possible?
>> > > > For
>> > > > > >> jobs
>> > > > > >> > >> with large amounts of state, I think SAMZA-617 would be a
>> big
>> > > > boon,
>> > > > > >> > since
>> > > > > >> > >> restoring state off the changelog on every single restart
>> is
>> > > > > painful,
>> > > > > >> > due
>> > > > > >> > >> to long recovery times. It would be a shame if the
>> decoupling
>> > > > from
>> > > > > >> YARN
>> > > > > >> > >> made host affinity impossible.
>> > > > > >> > >>
>> > > > > >> > >> Jay, a question about the proposed API for instantiating a
>> > job
>> > > in
>> > > > > >> code
>> > > > > >> > >> (rather than a properties file): when submitting a job to a
>> > > > > cluster,
>> > > > > >> is
>> > > > > >> > the
>> > > > > >> > >> idea that the instantiation code runs on a client
>> somewhere,
>> > > > which
>> > > > > >> then
>> > > > > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or
>> does
>> > > that
>> > > > > >> code
>> > > > > >> > run
>> > > > > >> > >> on each container that is part of the job (in which case,
>> how
>> > > > does
>> > > > > >> the
>> > > > > >> > job
>> > > > > >> > >> submission to the cluster work)?
>> > > > > >> > >>
>> > > > > >> > >> I agree with Garry that it doesn't feel right to make a 1.0
>> > > > release
>> > > > > >> > with a
>> > > > > >> > >> plan for it to be immediately obsolete. So if this is going
>> > to
>> > > > > >> happen, I
>> > > > > >> > >> think it would be more honest to stick with 0.* version
>> > numbers
>> > > > > until
>> > > > > >> > the
>> > > > > >> > >> library-ified Samza has been implemented, is stable and
>> > widely
>> > > > > used.
>> > > > > >> > >>
>> > > > > >> > >> Should the new Samza be a subproject of Kafka? There is
>> > > precedent
>> > > > > for
>> > > > > >> > >> tight coupling between different Apache projects (e.g.
>> > Curator
>> > > > and
>> > > > > >> > >> Zookeeper, or Slider and YARN), so I think remaining
>> separate
>> > > > would
>> > > > > >> be
>> > > > > >> > ok.
>> > > > > >> > >> Even if Samza is fully dependent on Kafka, there is enough
>> > > > > substance
>> > > > > >> in
>> > > > > >> > >> Samza that it warrants being a separate project. An
>> argument
>> > in
>> > > > > >> favour
>> > > > > >> > of
>> > > > > >> > >> merging would be if we think Kafka has a much stronger
>> "brand
>> > > > > >> presence"
>> > > > > >> > >> than Samza; I'm ambivalent on that one. If the Kafka
>> project
>> > is
>> > > > > >> willing
>> > > > > >> > to
>> > > > > >> > >> endorse Samza as the "official" way of doing stateful
>> stream
>> > > > > >> > >> transformations, that would probably have much the same
>> > effect
>> > > as
>> > > > > >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike.
>> > > Close
>> > > > > >> > >> collaboration between the two projects will be needed in
>> any
>> > > > case.
>> > > > > >> > >>
>> > > > > >> > >> From a project management perspective, I guess the "new
>> > Samza"
>> > > > > would
>> > > > > >> > have
>> > > > > >> > >> to be developed on a branch alongside ongoing maintenance
>> of
>> > > the
>> > > > > >> current
>> > > > > >> > >> line of development? I think it would be important to
>> > continue
>> > > > > >> > supporting
>> > > > > >> > >> existing users, and provide a graceful migration path to
>> the
>> > > new
>> > > > > >> > version.
>> > > > > >> > >> Leaving the current versions unsupported and forcing people
>> > to
>> > > > > >> rewrite
>> > > > > >> > >> their jobs would send a bad signal.
>> > > > > >> > >>
>> > > > > >> > >> Best,
>> > > > > >> > >> Martin
>> > > > > >> > >>
>> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io>
>> wrote:
>> > > > > >> > >>
>> > > > > >> > >>> Hey Garry,
>> > > > > >> > >>>
>> > > > > >> > >>> Yeah that's super frustrating. I'd be happy to chat more
>> > about
>> > > > > this
>> > > > > >> if
>> > > > > >> > >>> you'd be interested. I think Chris and I started with the
>> > idea
>> > > > of
>> > > > > >> "what
>> > > > > >> > >>> would it take to make Samza a kick-ass ingestion tool" but
>> > > > > >> ultimately
>> > > > > >> > we
>> > > > > >> > >>> kind of came around to the idea that ingestion and
>> > > > transformation
>> > > > > >> had
>> > > > > >> > >>> pretty different needs and coupling the two made things
>> > hard.
>> > > > > >> > >>>
>> > > > > >> > >>> For what it's worth I think copycat (KIP-26) actually will
>> > do
>> > > > what
>> > > > > >> you
>> > > > > >> > >> are
>> > > > > >> > >>> looking for.
>> > > > > >> > >>>
>> > > > > >> > >>> With regard to your point about slider, I don't
>> necessarily
>> > > > > >> disagree.
>> > > > > >> > >> But I
>> > > > > >> > >>> think getting good YARN support is quite doable and I
>> think
>> > we
>> > > > can
>> > > > > >> make
>> > > > > >> > >>> that work well. I think the issue this proposal solves is
>> > that
>> > > > > >> > >> technically
>> > > > > >> > >>> it is pretty hard to support multiple cluster management
>> > > systems
>> > > > > the
>> > > > > >> > way
>> > > > > >> > >>> things are now, you need to write an "app master" or
>> > > "framework"
>> > > > > for
>> > > > > >> > each
>> > > > > >> > >>> and they are all a little different so testing is really
>> > hard.
>> > > > In
>> > > > > >> the
>> > > > > >> > >>> absence of this we have been stuck with just YARN which
>> has
>> > > > > >> fantastic
>> > > > > >> > >>> penetration in the Hadoopy part of the org, but zero
>> > > penetration
>> > > > > >> > >> elsewhere.
>> > > > > >> > >>> Given the huge amount of work being put in to slider,
>> > > marathon,
>> > > > > aws
>> > > > > >> > >>> tooling, not to mention the umpteen related packaging
>> > > > technologies
>> > > > > >> > people
>> > > > > >> > >>> want to use (Docker, Kubernetes, various cloud-specific
>> > deploy
>> > > > > >> tools,
>> > > > > >> > >> etc)
>> > > > > >> > >>> I really think it is important to get this right.
>> > > > > >> > >>>
>> > > > > >> > >>> -Jay
>> > > > > >> > >>>
>> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
>> > > > > >> > >>>
>> > > > > >> > >>>> Hi all,
>> > > > > >> > >>>>
>> > > > > >> > >>>> I think the question below re does Samza become a
>> > sub-project
>> > > > of
>> > > > > >> Kafka
>> > > > > >> > >>>> highlights the broader point around migration. Chris
>> > mentions
>> > > > > >> Samza's
>> > > > > >> > >>>> maturity is heading towards a v1 release but I'm not sure
>> > it
>> > > > > feels
>> > > > > >> > >> right to
>> > > > > >> > >>>> launch a v1 then immediately plan to deprecate most of
>> it.
>> > > > > >> > >>>>
>> > > > > >> > >>>> From a selfish perspective I have some guys who have
>> > started
>> > > > > >> working
>> > > > > >> > >> with
>> > > > > >> > >>>> Samza and building some new consumers/producers was next
>> > up.
>> > > > > Sounds
>> > > > > >> > like
>> > > > > >> > >>>> that is absolutely not the direction to go. I need to
>> look
>> > > into
>> > > > > the
>> > > > > >> > KIP
>> > > > > >> > >> in
>> > > > > >> > >>>> more detail but for me the attractiveness of adding new
>> > Samza
>> > > > > >> > >>>> consumer/producers -- even if yes all they were doing was
>> > > > really
>> > > > > >> > getting
>> > > > > >> > >>>> data into and out of Kafka --  was to avoid  having to
>> > worry
>> > > > > about
>> > > > > >> the
>> > > > > >> > >>>> lifecycle management of external clients. If there is a
>> > > generic
>> > > > > >> Kafka
>> > > > > >> > >>>> ingress/egress layer that I can plug a new connector into
>> > and
>> > > > > have
>> > > > > >> a
>> > > > > >> > >> lot of
>> > > > > >> > >>>> the heavy lifting re scale and reliability done for me
>> then
>> > > it
>> > > > > >> gives
>> > > > > >> > me
>> > > > > >> > >> all
>> > > > > >> > >>>> the pushing new consumers/producers would. If not then it
>> > > > > >> complicates
>> > > > > >> > my
>> > > > > >> > >>>> operational deployments.
>> > > > > >> > >>>>
>> > > > > >> > >>>> Which is similar to my other question with the proposal
>> --
>> > if
>> > > > we
>> > > > > >> > build a
>> > > > > >> > >>>> fully available/stand-alone Samza plus the requisite
>> shims
>> > to
>> > > > > >> > integrate
>> > > > > >> > >>>> with Slider etc I suspect the former may be a lot more
>> work
>> > > > than
>> > > > > we
>> > > > > >> > >> think.
>> > > > > >> > >>>> We may make it much easier for a newcomer to get
>> something
>> > > > > running
>> > > > > >> but
>> > > > > >> > >>>> having them step up and get a reliable production
>> > deployment
>> > > > may
>> > > > > >> still
>> > > > > >> > >>>> dominate mailing list  traffic, if for different reasons
>> > than
>> > > > > >> today.
>> > > > > >> > >>>>
>> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with making the
>> Samza
>> > > > > >> dependency
>> > > > > >> > >> on
>> > > > > >> > >>>> Kafka much more explicit and I absolutely see the
>> benefits
>> > > in
>> > > > > the
>> > > > > >> > >>>> reduction of duplication and clashing
>> > > > terminologies/abstractions
>> > > > > >> that
>> > > > > >> > >>>> Chris/Jay describe. Samza as a library would likely be a
>> > very
>> > > > > nice
>> > > > > >> > tool
>> > > > > >> > >> to
>> > > > > >> > >>>> add to the Kafka ecosystem. I just have the concerns
>> above
>> > re
>> > > > the
>> > > > > >> > >>>> operational side.
>> > > > > >> > >>>>
>> > > > > >> > >>>> Garry
>> > > > > >> > >>>>
>> > > > > >> > >>>> -----Original Message-----
>> > > > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
>> > gdfm@apache.org
>> > > ]
>> > > > > >> > >>>> Sent: 02 July 2015 12:56
>> > > > > >> > >>>> To: dev@samza.apache.org
>> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
>> > > > > >> > >>>>
>> > > > > >> > >>>> Very interesting thoughts.
>> > > > > >> > >>>> From outside, I have always perceived Samza as a
>> computing
>> > > > layer
>> > > > > >> over
>> > > > > >> > >>>> Kafka.
>> > > > > >> > >>>>
>> > > > > >> > >>>> The question, maybe a bit provocative, is "should Samza
>> be
>> > a
>> > > > > >> > sub-project
>> > > > > >> > >>>> of Kafka then?"
>> > > > > >> > >>>> Or does it make sense to keep it as a separate project
>> > with a
>> > > > > >> separate
>> > > > > >> > >>>> governance?
>> > > > > >> > >>>>
>> > > > > >> > >>>> Cheers,
>> > > > > >> > >>>>
>> > > > > >> > >>>> --
>> > > > > >> > >>>> Gianmarco
>> > > > > >> > >>>>
>> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
>> > > > wrote:
>> > > > > >> > >>>>
>> > > > > >> > >>>>> Overall, I agree to couple with Kafka more tightly.
>> > Because
>> > > > > Samza
>> > > > > >> de
>> > > > > >> > >>>>> facto is based on Kafka, and it should leverage what
>> Kafka
>> > > > has.
>> > > > > At
>> > > > > >> > the
>> > > > > >> > >>>>> same time, Kafka does not need to reinvent what Samza
>> > > already
>> > > > > >> has. I
>> > > > > >> > >>>>> also like the idea of separating the ingestion and
>> > > > > transformation.
>> > > > > >> > >>>>>
>> > > > > >> > >>>>> But it is a little difficult for me to image how the
>> Samza
>> > > > will
>> > > > > >> look
>> > > > > >> > >>>> like.
>> > > > > >> > >>>>> And I feel Chris and Jay have a little difference in
>> terms
>> > > of
>> > > > > how
>> > > > > >> > >>>>> Samza should look like.
>> > > > > >> > >>>>>
>> > > > > >> > >>>>> *** Will it look like what Jay's code shows (A client of
>> > > > Kakfa)
>> > > > > ?
>> > > > > >> And
>> > > > > >> > >>>>> user's application code calls this client?
>> > > > > >> > >>>>>
>> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka (like what the
>> > > code
>> > > > > >> shows),
>> > > > > >> > >>>>> how do we implement auto-balance and fault-tolerance?
>> Are
>> > > they
>> > > > > >> taken
>> > > > > >> > >>>>> care by the Kafka broker or other mechanism, such as
>> > "Samza
>> > > > > >> worker"
>> > > > > >> > >>>>> (just make up the name) ?
>> > > > > >> > >>>>>
>> > > > > >> > >>>>> 2. What about other features, such as auto-scaling,
>> shared
>> > > > > state,
>> > > > > >> > >>>>> monitoring?
>> > > > > >> > >>>>>
>> > > > > >> > >>>>>
>> > > > > >> > >>>>> *** If we have Samza standalone, (is this what Chris
>> > > > suggests?)
>> > > > > >> > >>>>>
>> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa and produce
>> to
>> > > it.
>> > > > > >> Then it
>> > > > > >> > >>>>> becomes the same as what Samza looks like now, except it
>> > > does
>> > > > > not
>> > > > > >> > rely
>> > > > > >> > >>>>> on Yarn anymore.
>> > > > > >> > >>>>>
>> > > > > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's
>> > metrics,
>> > > > > logs,
>> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
>> > > > > >> > >>>>>
>> > > > > >> > >>>>>
>> > > > > >> > >>>>> Thanks,
>> > > > > >> > >>>>>
>> > > > > >> > >>>>> Fang, Yan
>> > > > > >> > >>>>> yanfang724@gmail.com
>> > > > > >> > >>>>>
>> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
>> > > > > wangguoz@gmail.com
>> > > > > >> >
>> > > > > >> > >>>> wrote:
>> > > > > >> > >>>>>
>> > > > > >> > >>>>>> Read through the code example and it looks good to me.
>> A
>> > > few
>> > > > > >> > >>>>>> thoughts regarding deployment:
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>> Today Samza deploys as executable runnable like:
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
>> > > > > >> > >>>> --config-path=file://...
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>> And this proposal advocate for deploying Samza more as
>> > > > embedded
>> > > > > >> > >>>>>> libraries in user application code (ignoring the
>> > > terminology
>> > > > > >> since
>> > > > > >> > >>>>>> it is not the
>> > > > > >> > >>>>> same
>> > > > > >> > >>>>>> as the prototype code):
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread
>> > thread
>> > > =
>> > > > > new
>> > > > > >> > >>>>>> Thread(task); thread.start();
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>> I think both of these deployment modes are important
>> for
>> > > > > >> different
>> > > > > >> > >>>>>> types
>> > > > > >> > >>>>> of
>> > > > > >> > >>>>>> users. That said, I think making Samza purely
>> standalone
>> > is
>> > > > > still
>> > > > > >> > >>>>>> sufficient for either runnable or library modes.
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>> Guozhang
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
>> > > > jay@confluent.io>
>> > > > > >> > wrote:
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>>> Looks like gmail mangled the code example, it was
>> > supposed
>> > > > to
>> > > > > >> look
>> > > > > >> > >>>>>>> like
>> > > > > >> > >>>>>>> this:
>> > > > > >> > >>>>>>>
>> > > > > >> > >>>>>>> Properties props = new Properties();
>> > > > > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
>> > > > > >> StreamingConfig
>> > > > > >> > >>>>>>> config = new StreamingConfig(props);
>> > > > > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
>> > > > > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
>> > > > > >> > >>>>>>> config.serialization(new StringSerializer(), new
>> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
>> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
>> > > > > >> > >>>>>>>
>> > > > > >> > >>>>>>> -Jay
>> > > > > >> > >>>>>>>
>> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
>> > > > jay@confluent.io
>> > > > > >
>> > > > > >> > >>>> wrote:
>> > > > > >> > >>>>>>>
>> > > > > >> > >>>>>>>> Hey guys,
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> This came out of some conversations Chris and I were
>> > > having
>> > > > > >> > >>>>>>>> around
>> > > > > >> > >>>>>>> whether
>> > > > > >> > >>>>>>>> it would make sense to use Samza as a kind of data
>> > > > ingestion
>> > > > > >> > >>>>> framework
>> > > > > >> > >>>>>>> for
>> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat").
>> This
>> > > > kind
>> > > > > of
>> > > > > >> > >>>>>> combined
>> > > > > >> > >>>>>>>> with complaints around config and YARN and the
>> > discussion
>> > > > > >> around
>> > > > > >> > >>>>>>>> how
>> > > > > >> > >>>>> to
>> > > > > >> > >>>>>>>> best do a standalone mode.
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> So the thought experiment was, given that Samza was
>> > > > basically
>> > > > > >> > >>>>>>>> already totally Kafka specific, what if you just
>> > embraced
>> > > > > that
>> > > > > >> > >>>>>>>> and turned it
>> > > > > >> > >>>>>> into
>> > > > > >> > >>>>>>>> something less like a heavyweight framework and more
>> > > like a
>> > > > > >> > >>>>>>>> third
>> > > > > >> > >>>>> Kafka
>> > > > > >> > >>>>>>>> client--a kind of "producing consumer" with state
>> > > > management
>> > > > > >> > >>>>>> facilities.
>> > > > > >> > >>>>>>>> Basically a library. Instead of a complex stream
>> > > processing
>> > > > > >> > >>>>>>>> framework
>> > > > > >> > >>>>>>> this
>> > > > > >> > >>>>>>>> would actually be a very simple thing, not much more
>> > > > > >> complicated
>> > > > > >> > >>>>>>>> to
>> > > > > >> > >>>>> use
>> > > > > >> > >>>>>>> or
>> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we
>> thought
>> > > > about
>> > > > > >> it
>> > > > > >> > >>>>>>>> a
>> > > > > >> > >>>>> lot
>> > > > > >> > >>>>>> of
>> > > > > >> > >>>>>>>> what Samza (and the other stream processing systems
>> > were
>> > > > > doing)
>> > > > > >> > >>>>> seemed
>> > > > > >> > >>>>>>> like
>> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> Of course you need to ingest/output data to and from
>> > the
>> > > > > stream
>> > > > > >> > >>>>>>>> processing. But when we actually looked into how that
>> > > would
>> > > > > >> > >>>>>>>> work,
>> > > > > >> > >>>>> Samza
>> > > > > >> > >>>>>>>> isn't really an ideal data ingestion framework for a
>> > > bunch
>> > > > of
>> > > > > >> > >>>>> reasons.
>> > > > > >> > >>>>>> To
>> > > > > >> > >>>>>>>> really do that right you need a pretty different
>> > internal
>> > > > > data
>> > > > > >> > >>>>>>>> model
>> > > > > >> > >>>>>> and
>> > > > > >> > >>>>>>>> set of apis. So what if you split them and had an api
>> > for
>> > > > > Kafka
>> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate
>> api
>> > > for
>> > > > > >> Kafka
>> > > > > >> > >>>>>>>> transformation (Samza).
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> This would also allow really embracing the same
>> > > terminology
>> > > > > and
>> > > > > >> > >>>>>>>> conventions. One complaint about the current state is
>> > > that
>> > > > > the
>> > > > > >> > >>>>>>>> two
>> > > > > >> > >>>>>>> systems
>> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs
>> > > > "topic"
>> > > > > >> and
>> > > > > >> > >>>>>>> different
>> > > > > >> > >>>>>>>> config and monitoring systems means you kind of have
>> to
>> > > > learn
>> > > > > >> > >>>>>>>> Kafka's
>> > > > > >> > >>>>>>> way,
>> > > > > >> > >>>>>>>> then learn Samza's slightly different way, then kind
>> of
>> > > > > >> > >>>>>>>> understand
>> > > > > >> > >>>>> how
>> > > > > >> > >>>>>>> they
>> > > > > >> > >>>>>>>> map to each other, which having walked a few people
>> > > through
>> > > > > >> this
>> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> Since I have been spending a lot of time on
>> airplanes I
>> > > > > hacked
>> > > > > >> > >>>>>>>> up an ernest but still somewhat incomplete prototype
>> of
>> > > > what
>> > > > > >> > >>>>>>>> this would
>> > > > > >> > >>>>> look
>> > > > > >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka
>> as
>> > > it
>> > > > > >> > >>>>>>>> required a
>> > > > > >> > >>>>>> few
>> > > > > >> > >>>>>>>> changes to the new consumer. Here is the code:
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>
>> > > > > >> >
>> > > > >
>> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> > > > > >> > >>>>> /apache/kafka/clients/streaming
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> For the purpose of the prototype I just liberally
>> > renamed
>> > > > > >> > >>>>>>>> everything
>> > > > > >> > >>>>> to
>> > > > > >> > >>>>>>>> try to align it with Kafka with no regard for
>> > > > compatibility.
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> To use this would be something like this:
>> > > > > >> > >>>>>>>> Properties props = new Properties();
>> > > > > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
>> > > > > >> > >>>>>>>> StreamingConfig config = new
>> > > > > >> > >>>>> StreamingConfig(props);
>> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
>> > > > > >> > >>>>>>>> "test-topic-2");
>> > > > > >> config.processor(ExampleStreamProcessor.class);
>> > > > > >> > >>>>>>> config.serialization(new
>> > > > > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
>> > > > KafkaStreaming
>> > > > > >> > >>>>>> container =
>> > > > > >> > >>>>>>>> new KafkaStreaming(config); container.run();
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
>> > > > > StreamProcessor
>> > > > > >> > >>>>>>>> is basically StreamTask.
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> So rather than putting all the class names in a file
>> > and
>> > > > then
>> > > > > >> > >>>>>>>> having
>> > > > > >> > >>>>>> the
>> > > > > >> > >>>>>>>> job assembled by reflection, you just instantiate the
>> > > > > container
>> > > > > >> > >>>>>>>> programmatically. Work is balanced over however many
>> > > > > instances
>> > > > > >> > >>>>>>>> of
>> > > > > >> > >>>>> this
>> > > > > >> > >>>>>>> are
>> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new
>> tasks
>> > > are
>> > > > > >> added
>> > > > > >> > >>>>>>>> to
>> > > > > >> > >>>>> the
>> > > > > >> > >>>>>>>> existing containers without shutting them down).
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> We would provide some glue for running this stuff in
>> > YARN
>> > > > via
>> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of
>> their
>> > > > tools
>> > > > > >> > >>>>>>>> but from the
>> > > > > >> > >>>>>> point
>> > > > > >> > >>>>>>> of
>> > > > > >> > >>>>>>>> view of these frameworks these stream processing jobs
>> > are
>> > > > > just
>> > > > > >> > >>>>>> stateless
>> > > > > >> > >>>>>>>> services that can come and go and expand and contract
>> > at
>> > > > > will.
>> > > > > >> > >>>>>>>> There
>> > > > > >> > >>>>> is
>> > > > > >> > >>>>>>> no
>> > > > > >> > >>>>>>>> more custom scheduler.
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> Here are some relevant details:
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get
>> larger
>> > > if
>> > > > we
>> > > > > >> > >>>>>>>>  productionized but not vastly larger. We really do
>> > get a
>> > > > ton
>> > > > > >> > >>>>>>>> of
>> > > > > >> > >>>>>>> leverage
>> > > > > >> > >>>>>>>>  out of Kafka.
>> > > > > >> > >>>>>>>>  2. Partition management is fully delegated to the
>> new
>> > > > > >> consumer.
>> > > > > >> > >>>>> This
>> > > > > >> > >>>>>>>>  is nice since now any partition management strategy
>> > > > > available
>> > > > > >> > >>>>>>>> to
>> > > > > >> > >>>>>> Kafka
>> > > > > >> > >>>>>>>>  consumer is also available to Samza (and vice versa)
>> > and
>> > > > > with
>> > > > > >> > >>>>>>>> the
>> > > > > >> > >>>>>>> exact
>> > > > > >> > >>>>>>>>  same configs.
>> > > > > >> > >>>>>>>>  3. It supports state as well as state reuse
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is thought
>> provoking.
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> -Jay
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>> > > > > >> > >>>>>> criccomini@apache.org>
>> > > > > >> > >>>>>>>> wrote:
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>>> Hey all,
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> I have had some discussions with Samza engineers at
>> > > > LinkedIn
>> > > > > >> > >>>>>>>>> and
>> > > > > >> > >>>>>>> Confluent
>> > > > > >> > >>>>>>>>> and we came up with a few observations and would
>> like
>> > to
>> > > > > >> > >>>>>>>>> propose
>> > > > > >> > >>>>> some
>> > > > > >> > >>>>>>>>> changes.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> We've observed some things that I want to call out
>> > about
>> > > > > >> > >>>>>>>>> Samza's
>> > > > > >> > >>>>>> design,
>> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment
>> system.
>> > > > > >> > >>>>>>>>> * Samza is too pluggable.
>> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
>> > > > consumer
>> > > > > >> > >>>>>>>>> APIs
>> > > > > >> > >>>>> are
>> > > > > >> > >>>>>>>>> trying to solve a lot of the same problems.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> All three of these issues are related, but I'll
>> > address
>> > > > them
>> > > > > >> in
>> > > > > >> > >>>>> order.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Deployment
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic
>> > > deployment
>> > > > > >> > >>>>>>>>> scheduler
>> > > > > >> > >>>>>> such
>> > > > > >> > >>>>>>>>> as
>> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we
>> > bet
>> > > > that
>> > > > > >> > >>>>>>>>> there
>> > > > > >> > >>>>>> would
>> > > > > >> > >>>>>>>>> be
>> > > > > >> > >>>>>>>>> one or two winners in this area, and we could
>> support
>> > > > them,
>> > > > > >> and
>> > > > > >> > >>>>>>>>> the
>> > > > > >> > >>>>>> rest
>> > > > > >> > >>>>>>>>> would go away. In reality, there are many
>> variations.
>> > > > > >> > >>>>>>>>> Furthermore,
>> > > > > >> > >>>>>> many
>> > > > > >> > >>>>>>>>> people still prefer to just start their processors
>> > like
>> > > > > normal
>> > > > > >> > >>>>>>>>> Java processes, and use traditional deployment
>> scripts
>> > > > such
>> > > > > as
>> > > > > >> > >>>>>>>>> Fabric,
>> > > > > >> > >>>>>> Chef,
>> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users
>> > makes
>> > > > the
>> > > > > >> > >>>>>>>>> Samza start-up process really painful for first time
>> > > > users.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit
>> of
>> > a
>> > > > > >> > >>>>>>>>> mis-fire
>> > > > > >> > >>>>>> because
>> > > > > >> > >>>>>>>>> of
>> > > > > >> > >>>>>>>>> a fundamental misunderstanding between the nature of
>> > > batch
>> > > > > >> jobs
>> > > > > >> > >>>>>>>>> and
>> > > > > >> > >>>>>>> stream
>> > > > > >> > >>>>>>>>> processing jobs. Early on, we made conscious effort
>> to
>> > > > favor
>> > > > > >> > >>>>>>>>> the
>> > > > > >> > >>>>>> Hadoop
>> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked
>> and
>> > > was
>> > > > > well
>> > > > > >> > >>>>>>> understood.
>> > > > > >> > >>>>>>>>> One thing that we missed was that batch jobs have a
>> > > > definite
>> > > > > >> > >>>>>> beginning,
>> > > > > >> > >>>>>>>>> and
>> > > > > >> > >>>>>>>>> end, and stream processing jobs don't (usually).
>> This
>> > > > leads
>> > > > > to
>> > > > > >> > >>>>>>>>> a
>> > > > > >> > >>>>> much
>> > > > > >> > >>>>>>>>> simpler scheduling problem for stream processors.
>> You
>> > > > > >> basically
>> > > > > >> > >>>>>>>>> just
>> > > > > >> > >>>>>>> need
>> > > > > >> > >>>>>>>>> to find a place to start the processor, and start
>> it.
>> > > The
>> > > > > way
>> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a
>> > > cluster
>> > > > > >> > >>>>>>>>> being "full". We always
>> > > > > >> > >>>>>> add
>> > > > > >> > >>>>>>>>> more machines. The problem with coupling Samza with
>> a
>> > > > > >> scheduler
>> > > > > >> > >>>>>>>>> is
>> > > > > >> > >>>>>> that
>> > > > > >> > >>>>>>>>> Samza (as a framework) now has to handle deployment.
>> > > This
>> > > > > >> pulls
>> > > > > >> > >>>>>>>>> in a
>> > > > > >> > >>>>>>> bunch
>> > > > > >> > >>>>>>>>> of things such as configuration distribution (config
>> > > > > stream),
>> > > > > >> > >>>>>>>>> shell
>> > > > > >> > >>>>>>> scrips
>> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
>> > > > stuff),
>> > > > > >> etc.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Another reason for requiring dynamic deployment was
>> to
>> > > > > support
>> > > > > >> > >>>>>>>>> data locality. If you want to have locality, you
>> need
>> > to
>> > > > put
>> > > > > >> > >>>>>>>>> your
>> > > > > >> > >>>>>> processors
>> > > > > >> > >>>>>>>>> close to the data they're processing. Upon further
>> > > > > >> > >>>>>>>>> investigation,
>> > > > > >> > >>>>>>> though,
>> > > > > >> > >>>>>>>>> this feature is not that beneficial. There is some
>> > good
>> > > > > >> > >>>>>>>>> discussion
>> > > > > >> > >>>>>> about
>> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took
>> the
>> > > > > >> > >>>>>>>>> Map/Reduce
>> > > > > >> > >>>>>> path,
>> > > > > >> > >>>>>>>>> but
>> > > > > >> > >>>>>>>>> there are some fundamental differences between HDFS
>> > and
>> > > > > Kafka.
>> > > > > >> > >>>>>>>>> HDFS
>> > > > > >> > >>>>>> has
>> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to
>> less
>> > > > > >> > >>>>>>>>> optimization potential with stream processors on top
>> > of
>> > > > > Kafka.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't
>> > > have
>> > > > > any
>> > > > > >> > >>>>>>>>> built
>> > > > > >> > >>>>> in
>> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the
>> > > dynamic
>> > > > > >> > >>>>>>>>> deployment scheduling system to handle restarts
>> when a
>> > > > > >> > >>>>>>>>> processor dies. This has
>> > > > > >> > >>>>>>> made
>> > > > > >> > >>>>>>>>> it very difficult to write a standalone Samza
>> > container
>> > > > > >> > >>>> (SAMZA-516).
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Pluggability
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> In some cases pluggability is good, but I think that
>> > > we've
>> > > > > >> gone
>> > > > > >> > >>>>>>>>> too
>> > > > > >> > >>>>>> far
>> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> * Pluggable config.
>> > > > > >> > >>>>>>>>> * Pluggable metrics.
>> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
>> > > > > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
>> > > > > SystemProducer,
>> > > > > >> > >>>> etc).
>> > > > > >> > >>>>>>>>> * Pluggable serdes.
>> > > > > >> > >>>>>>>>> * Pluggable storage engines.
>> > > > > >> > >>>>>>>>> * Pluggable strategies for just about every
>> component
>> > > > > >> > >>>>> (MessageChooser,
>> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> There's probably more that I've forgotten, as well.
>> > Some
>> > > > of
>> > > > > >> > >>>>>>>>> these
>> > > > > >> > >>>>> are
>> > > > > >> > >>>>>>>>> useful, but some have proven not to be. This all
>> comes
>> > > at
>> > > > a
>> > > > > >> cost:
>> > > > > >> > >>>>>>>>> complexity. This complexity is making it harder for
>> > our
>> > > > > users
>> > > > > >> > >>>>>>>>> to
>> > > > > >> > >>>>> pick
>> > > > > >> > >>>>>> up
>> > > > > >> > >>>>>>>>> and use Samza out of the box. It also makes it
>> > difficult
>> > > > for
>> > > > > >> > >>>>>>>>> Samza developers to reason about what the
>> > > characteristics
>> > > > of
>> > > > > >> > >>>>>>>>> the container (since the characteristics change
>> > > depending
>> > > > on
>> > > > > >> > >>>>>>>>> which plugins are use).
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> The issues with pluggability are most visible in the
>> > > > System
>> > > > > >> APIs.
>> > > > > >> > >>>>> What
>> > > > > >> > >>>>>>>>> Samza really requires to be functional is Kafka as
>> its
>> > > > > >> > >>>>>>>>> transport
>> > > > > >> > >>>>>> layer.
>> > > > > >> > >>>>>>>>> But
>> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases into one
>> API:
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
>> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> The current System API supports both of these use
>> > cases.
>> > > > The
>> > > > > >> > >>>>>>>>> problem
>> > > > > >> > >>>>>> is,
>> > > > > >> > >>>>>>>>> we
>> > > > > >> > >>>>>>>>> actually want different features for each use case.
>> By
>> > > > > >> papering
>> > > > > >> > >>>>>>>>> over
>> > > > > >> > >>>>>>> these
>> > > > > >> > >>>>>>>>> two use cases, and providing a single API, we've
>> > > > introduced
>> > > > > a
>> > > > > >> > >>>>>>>>> ton of
>> > > > > >> > >>>>>>> leaky
>> > > > > >> > >>>>>>>>> abstractions.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> For example, what we'd really like in (2) is to have
>> > > > > >> > >>>>>>>>> monotonically increasing longs for offsets (like
>> > Kafka).
>> > > > > This
>> > > > > >> > >>>>>>>>> would be at odds
>> > > > > >> > >>>>> with
>> > > > > >> > >>>>>>> (1),
>> > > > > >> > >>>>>>>>> though, since different systems have different
>> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
>> > > > > >> > >>>>>>>>> There was discussion both on the mailing list and
>> the
>> > > SQL
>> > > > > >> JIRAs
>> > > > > >> > >>>>> about
>> > > > > >> > >>>>>>> the
>> > > > > >> > >>>>>>>>> need for this.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> The same thing holds true for replayability. Kafka
>> > > allows
>> > > > us
>> > > > > >> to
>> > > > > >> > >>>>> rewind
>> > > > > >> > >>>>>>>>> when
>> > > > > >> > >>>>>>>>> we have a failure. Many other systems don't. In some
>> > > > cases,
>> > > > > >> > >>>>>>>>> systems
>> > > > > >> > >>>>>>> return
>> > > > > >> > >>>>>>>>> null for their offsets (e.g.
>> WikipediaSystemConsumer)
>> > > > > because
>> > > > > >> > >>>>>>>>> they
>> > > > > >> > >>>>>> have
>> > > > > >> > >>>>>>> no
>> > > > > >> > >>>>>>>>> offsets.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka supports
>> > > > > partitioning,
>> > > > > >> > >>>>>>>>> but
>> > > > > >> > >>>>> many
>> > > > > >> > >>>>>>>>> systems don't. We model this by having a single
>> > > partition
>> > > > > for
>> > > > > >> > >>>>>>>>> those systems. Still, other systems model
>> partitioning
>> > > > > >> > >>>> differently (e.g.
>> > > > > >> > >>>>>>>>> Kinesis).
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating
>> > > streams
>> > > > > in
>> > > > > >> a
>> > > > > >> > >>>>>>>>> system-agnostic way is almost impossible. As is
>> > modeling
>> > > > > >> > >>>>>>>>> metadata
>> > > > > >> > >>>>> for
>> > > > > >> > >>>>>>> the
>> > > > > >> > >>>>>>>>> system (replication factor, partitions, location,
>> > etc).
>> > > > The
>> > > > > >> > >>>>>>>>> list
>> > > > > >> > >>>>> goes
>> > > > > >> > >>>>>>> on.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Duplicate work
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's
>> > > consumer
>> > > > > and
>> > > > > >> > >>>>> producer
>> > > > > >> > >>>>>>>>> APIs
>> > > > > >> > >>>>>>>>> had a relatively weak feature set. On the
>> > consumer-side,
>> > > > you
>> > > > > >> > >>>>>>>>> had two
>> > > > > >> > >>>>>>>>> options: use the high level consumer, or the simple
>> > > > > consumer.
>> > > > > >> > >>>>>>>>> The
>> > > > > >> > >>>>>>> problem
>> > > > > >> > >>>>>>>>> with the high-level consumer was that it controlled
>> > your
>> > > > > >> > >>>>>>>>> offsets, partition assignments, and the order in
>> which
>> > > you
>> > > > > >> > >>>>>>>>> received messages. The
>> > > > > >> > >>>>> problem
>> > > > > >> > >>>>>>>>> with
>> > > > > >> > >>>>>>>>> the simple consumer is that it's not simple. It's
>> > basic.
>> > > > You
>> > > > > >> > >>>>>>>>> end up
>> > > > > >> > >>>>>>> having
>> > > > > >> > >>>>>>>>> to handle a lot of really low-level stuff that you
>> > > > > shouldn't.
>> > > > > >> > >>>>>>>>> We
>> > > > > >> > >>>>>> spent a
>> > > > > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
>> > > > robust.
>> > > > > >> It
>> > > > > >> > >>>>>>>>> also allows us to support some cool features:
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> * Per-partition message ordering and prioritization.
>> > > > > >> > >>>>>>>>> * Tight control over partition assignment to support
>> > > > joins,
>> > > > > >> > >>>>>>>>> global
>> > > > > >> > >>>>>> state
>> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
>> > > > > >> > >>>>>>>>> * Tight control over offset checkpointing.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> What we didn't realize at the time is that these
>> > > features
>> > > > > >> > >>>>>>>>> should
>> > > > > >> > >>>>>>> actually
>> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just
>> Samza
>> > > > stream
>> > > > > >> > >>>>>> processors)
>> > > > > >> > >>>>>>>>> end up wanting to do things like joins and partition
>> > > > > >> > >>>>>>>>> assignment. The
>> > > > > >> > >>>>>>> Kafka
>> > > > > >> > >>>>>>>>> community has come to the same conclusion. They're
>> > > adding
>> > > > a
>> > > > > >> ton
>> > > > > >> > >>>>>>>>> of upgrades into their new Kafka consumer
>> > > implementation.
>> > > > > To a
>> > > > > >> > >>>>>>>>> large extent,
>> > > > > >> > >>>>> it's
>> > > > > >> > >>>>>>>>> duplicate work to what we've already done in Samza.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
>> > > > > approach
>> > > > > >> > >>>>>>>>> to
>> > > > > >> > >>>>>> Samza's
>> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation for handling
>> > > offset
>> > > > > >> > >>>>>> checkpointing.
>> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature
>> > stores
>> > > > > >> offset
>> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them
>> > > from
>> > > > > the
>> > > > > >> > >>>>>>>>> broker.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since we could
>> have
>> > > > shared
>> > > > > >> > >>>>>>>>> the
>> > > > > >> > >>>>> work
>> > > > > >> > >>>>>> if
>> > > > > >> > >>>>>>>>> it
>> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Vision
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> All of this leads me to a rather radical proposal.
>> > Samza
>> > > > is
>> > > > > >> > >>>>> relatively
>> > > > > >> > >>>>>>>>> stable at this point. I'd venture to say that we're
>> > > near a
>> > > > > 1.0
>> > > > > >> > >>>>>> release.
>> > > > > >> > >>>>>>>>> I'd
>> > > > > >> > >>>>>>>>> like to propose that we take what we've learned, and
>> > > begin
>> > > > > >> > >>>>>>>>> thinking
>> > > > > >> > >>>>>>> about
>> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were
>> > > starting
>> > > > > >> from
>> > > > > >> > >>>>>> scratch?
>> > > > > >> > >>>>>>>>> My
>> > > > > >> > >>>>>>>>> proposal is to:
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
>> > > > > >> > >>>>>>>>> processors, and eliminate all direct dependences on
>> > > YARN,
>> > > > > >> Mesos,
>> > > > > >> > >>>> etc.
>> > > > > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as
>> the
>> > > > > stream
>> > > > > >> > >>>>>> processing
>> > > > > >> > >>>>>>>>> layer.
>> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
>> serialization,
>> > > and
>> > > > > >> > >>>>>>>>> config
>> > > > > >> > >>>>>>> systems,
>> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> This would fix all of the issues that I outlined
>> > above.
>> > > It
>> > > > > >> > >>>>>>>>> should
>> > > > > >> > >>>>> also
>> > > > > >> > >>>>>>>>> shrink the Samza code base pretty dramatically.
>> > > Supporting
>> > > > > >> only
>> > > > > >> > >>>>>>>>> a standalone container will allow Samza to be
>> executed
>> > > on
>> > > > > YARN
>> > > > > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or
>> most
>> > > > other
>> > > > > >> > >>>>>>>>> in-house
>> > > > > >> > >>>>>>> deployment
>> > > > > >> > >>>>>>>>> systems. This should make life a lot easier for new
>> > > users.
>> > > > > >> > >>>>>>>>> Imagine
>> > > > > >> > >>>>>>> having
>> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in
>> > > mailing
>> > > > > >> list
>> > > > > >> > >>>>>> traffic
>> > > > > >> > >>>>>>>>> will be pretty dramatic.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The
>> > > reality
>> > > > > is,
>> > > > > >> > >>>>> everyone
>> > > > > >> > >>>>>>>>> that
>> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically
>> > > > require
>> > > > > >> it
>> > > > > >> > >>>>>> already
>> > > > > >> > >>>>>>> in
>> > > > > >> > >>>>>>>>> order for most features to work. Those that are
>> using
>> > > > other
>> > > > > >> > >>>>>>>>> systems
>> > > > > >> > >>>>>> are
>> > > > > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and
>> then
>> > > > they
>> > > > > do
>> > > > > >> > >>>>>>>>> the processing on top. There is already discussion (
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>
>> > > > > >> >
>> > > > >
>> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> > > > > >> > >>>>> 767
>> > > > > >> > >>>>>>>>> )
>> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely
>> easy.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Once we make the call to couple with Kafka, we can
>> > > > leverage
>> > > > > a
>> > > > > >> > >>>>>>>>> ton of
>> > > > > >> > >>>>>>> their
>> > > > > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own
>> > config,
>> > > > > >> > >>>>>>>>> metrics,
>> > > > > >> > >>>>> etc.
>> > > > > >> > >>>>>>> We
>> > > > > >> > >>>>>>>>> can all share the same libraries, and make them
>> > better.
>> > > > This
>> > > > > >> > >>>>>>>>> will
>> > > > > >> > >>>>> also
>> > > > > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and
>> will
>> > > let
>> > > > > us
>> > > > > >> > >>>>> leverage
>> > > > > >> > >>>>>>>>> their offset management and partition management,
>> > rather
>> > > > > than
>> > > > > >> > >>>>>>>>> having
>> > > > > >> > >>>>>> our
>> > > > > >> > >>>>>>>>> own. All of the coordinator stream code would go
>> away,
>> > > as
>> > > > > >> would
>> > > > > >> > >>>>>>>>> most
>> > > > > >> > >>>>>> of
>> > > > > >> > >>>>>>>>> the
>> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
>> > > > > partition
>> > > > > >> > >>>>>>> management
>> > > > > >> > >>>>>>>>> features into the Kafka broker, but they're already
>> > > moving
>> > > > > in
>> > > > > >> > >>>>>>>>> that direction with the new consumer API. The
>> features
>> > > we
>> > > > > have
>> > > > > >> > >>>>>>>>> for
>> > > > > >> > >>>>>> partition
>> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like
>> they
>> > > > should
>> > > > > >> be
>> > > > > >> > >>>>>>>>> in
>> > > > > >> > >>>>>> Kafka
>> > > > > >> > >>>>>>>>> anyway. There will always be some niche usages which
>> > > will
>> > > > > >> > >>>>>>>>> require
>> > > > > >> > >>>>>> extra
>> > > > > >> > >>>>>>>>> care and hence full control over partition
>> assignments
>> > > > much
>> > > > > >> > >>>>>>>>> like the
>> > > > > >> > >>>>>>> Kafka
>> > > > > >> > >>>>>>>>> low level consumer api. These would continue to be
>> > > > > supported.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> These items will be good for the Samza community.
>> > > They'll
>> > > > > make
>> > > > > >> > >>>>>>>>> Samza easier to use, and make it easier for
>> developers
>> > > to
>> > > > > add
>> > > > > >> > >>>>>>>>> new features.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat
>> > backwards
>> > > > > >> > >>>>> incompatible
>> > > > > >> > >>>>>>>>> change). If we choose to go this route, it's
>> important
>> > > > that
>> > > > > we
>> > > > > >> > >>>>> openly
>> > > > > >> > >>>>>>>>> communicate how we're going to provide a migration
>> > path
>> > > > from
>> > > > > >> > >>>>>>>>> the
>> > > > > >> > >>>>>>> existing
>> > > > > >> > >>>>>>>>> APIs to the new ones (if we make incompatible
>> > changes).
>> > > I
>> > > > > >> think
>> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to provide a
>> wrapper
>> > to
>> > > > > allow
>> > > > > >> > >>>>>>>>> existing StreamTask implementations to continue
>> > running
>> > > on
>> > > > > the
>> > > > > >> > >>>> new container.
>> > > > > >> > >>>>>>> It's
>> > > > > >> > >>>>>>>>> also important that we openly communicate about
>> > timing,
>> > > > and
>> > > > > >> > >>>>>>>>> stages
>> > > > > >> > >>>>> of
>> > > > > >> > >>>>>>> the
>> > > > > >> > >>>>>>>>> migration.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you have opinions.
>> > :)
>> > > > > Please
>> > > > > >> > >>>>>>>>> send
>> > > > > >> > >>>>>> your
>> > > > > >> > >>>>>>>>> thoughts and feedback.
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>> Cheers,
>> > > > > >> > >>>>>>>>> Chris
>> > > > > >> > >>>>>>>>>
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>>
>> > > > > >> > >>>>>>>
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>> --
>> > > > > >> > >>>>>> -- Guozhang
>> > > > > >> > >>>>>>
>> > > > > >> > >>>>>
>> > > > > >> > >>>>
>> > > > > >> > >>
>> > > > > >> > >>
>> > > > > >> >
>> > > > > >> >
>> > > > > >> >
>> > > > > >>
>> > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>

Re: Thoughts and obesrvations on Samza

Posted by Roger Hoover <ro...@gmail.com>.
That's great.  Thanks, Jay.

On Fri, Jul 10, 2015 at 8:46 AM, Jay Kreps <ja...@confluent.io> wrote:

> Yeah totally agree. I think you have this issue even today, right? I.e. if
> you need to make a simple config change and you're running in YARN today
> you end up bouncing the job which then rebuilds state. I think the fix is
> exactly what you described which is to have a long timeout on partition
> movement for stateful jobs so that if a job is just getting bounced, and
> the cluster manager (or admin) is smart enough to restart it on the same
> host when possible, it can optimistically reuse any existing state it finds
> on disk (if it is valid).
>
> So in this model the charter of the CM is to place processes as stickily as
> possible and to restart or re-place failed processes. The charter of the
> partition management system is to control the assignment of work to these
> processes. The nice thing about this is that the work assignment, timeouts,
> behavior, configs, and code will all be the same across all cluster
> managers.
>
> So I think that prototype would actually give you exactly what you want
> today for any cluster manager (or manual placement + restart script) that
> was sticky in terms of host placement since there is already a configurable
> partition movement timeout and task-by-task state reuse with a check on
> state validity.
>
> -Jay
>
> On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <ro...@gmail.com>
> wrote:
>
> > That would be great to let Kafka do as much heavy lifting as possible and
> > make it easier for other languages to implement Samza apis.
> >
> > One thing to watch out for is the interplay between Kafka's group
> > management and the external scheduler/process manager's fault tolerance.
> > If a container dies, the Kafka group membership protocol will try to
> assign
> > it's tasks to other containers while at the same time the process manager
> > is trying to relaunch the container.  Without some consideration for this
> > (like a configurable amount of time to wait before Kafka alters the group
> > membership), there may be thrashing going on which is especially bad for
> > containers with large amounts of local state.
> >
> > Someone else pointed this out already but I thought it might be worth
> > calling out again.
> >
> > Cheers,
> >
> > Roger
> >
> >
> > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Hey Roger,
> > >
> > > I couldn't agree more. We spent a bunch of time talking to people and
> > that
> > > is exactly the stuff we heard time and again. What makes it hard, of
> > > course, is that there is some tension between compatibility with what's
> > > there now and making things better for new users.
> > >
> > > I also strongly agree with the importance of multi-language support. We
> > are
> > > talking now about Java, but for application development use cases
> people
> > > want to work in whatever language they are using elsewhere. I think
> > moving
> > > to a model where Kafka itself does the group membership, lifecycle
> > control,
> > > and partition assignment has the advantage of putting all that complex
> > > stuff behind a clean api that the clients are already going to be
> > > implementing for their consumer, so the added functionality for stream
> > > processing beyond a consumer becomes very minor.
> > >
> > > -Jay
> > >
> > > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <ro...@gmail.com>
> > > wrote:
> > >
> > > > Metamorphosis...nice. :)
> > > >
> > > > This has been a great discussion.  As a user of Samza who's recently
> > > > integrated it into a relatively large organization, I just want to
> add
> > > > support to a few points already made.
> > > >
> > > > The biggest hurdles to adoption of Samza as it currently exists that
> > I've
> > > > experienced are:
> > > > 1) YARN - YARN is overly complex in many environments where Puppet
> > would
> > > do
> > > > just fine but it was the only mechanism to get fault tolerance.
> > > > 2) Configuration - I think I like the idea of configuring most of the
> > job
> > > > in code rather than config files.  In general, I think the goal
> should
> > be
> > > > to make it harder to make mistakes, especially of the kind where the
> > code
> > > > expects something and the config doesn't match.  The current config
> is
> > > > quite intricate and error-prone.  For example, the application logic
> > may
> > > > depend on bootstrapping a topic but rather than asserting that in the
> > > code,
> > > > you have to rely on getting the config right.  Likewise with serdes,
> > the
> > > > Java representations produced by various serdes (JSON, Avro, etc.)
> are
> > > not
> > > > equivalent so you cannot just reconfigure a serde without changing
> the
> > > > code.   It would be nice for jobs to be able to assert what they
> expect
> > > > from their input topics in terms of partitioning.  This is getting a
> > > little
> > > > off topic but I was even thinking about creating a "Samza config
> > linter"
> > > > that would sanity check a set of configs.  Especially in
> organizations
> > > > where config is managed by a different team than the application
> > > developer,
> > > > it's very hard to get avoid config mistakes.
> > > > 3) Java/Scala centric - for many teams (especially DevOps-type
> folks),
> > > the
> > > > pain of the Java toolchain (maven, slow builds, weak command line
> > > support,
> > > > configuration over convention) really inhibits productivity.  As more
> > and
> > > > more high-quality clients become available for Kafka, I hope they'll
> > > follow
> > > > Samza's model.  Not sure how much it affects the proposals in this
> > thread
> > > > but please consider other languages in the ecosystem as well.  From
> > what
> > > > I've heard, Spark has more Python users than Java/Scala.
> > > > (FYI, we added a Jython wrapper for the Samza API
> > > >
> > > >
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > > and are working on a Yeoman generator
> > > > https://github.com/Quantiply/generator-rico for Jython/Samza
> projects
> > to
> > > > alleviate some of the pain)
> > > >
> > > > I also want to underscore Jay's point about improving the user
> > > experience.
> > > > That's a very important factor for adoption.  I think the goal should
> > be
> > > to
> > > > make Samza as easy to get started with as something like Logstash.
> > > > Logstash is vastly inferior in terms of capabilities to Samza but
> it's
> > > easy
> > > > to get started and that makes a big difference.
> > > >
> > > > Cheers,
> > > >
> > > > Roger
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
> > > > gdfm@apache.org> wrote:
> > > >
> > > > > Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
> > > > winner
> > > > > :)
> > > > >
> > > > > --
> > > > > Gianmarco
> > > > >
> > > > > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
> > > gdfm@apache.org
> > > > >
> > > > > wrote:
> > > > >
> > > > > > Hi,
> > > > > >
> > > > > > @Martin, thanks for you comments.
> > > > > > Maybe I'm missing some important point, but I think coupling the
> > > > releases
> > > > > > is actually a *good* thing.
> > > > > > To make an example, would it be better if the MR and HDFS
> > components
> > > of
> > > > > > Hadoop had different release schedules?
> > > > > >
> > > > > > Actually, keeping the discussion in a single place would make
> > > agreeing
> > > > on
> > > > > > releases (and backwards compatibility) much easier, as everybody
> > > would
> > > > be
> > > > > > responsible for the whole codebase.
> > > > > >
> > > > > > That said, I like the idea of absorbing samza-core as a
> > sub-project,
> > > > and
> > > > > > leave the fancy stuff separate.
> > > > > > It probably gives 90% of the benefits we have been discussing
> here.
> > > > > >
> > > > > > Cheers,
> > > > > >
> > > > > > --
> > > > > > Gianmarco
> > > > > >
> > > > > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
> > > > > >
> > > > > >> Hey Martin,
> > > > > >>
> > > > > >> I agree coupling release schedules is a downside.
> > > > > >>
> > > > > >> Definitely we can try to solve some of the integration problems
> in
> > > > > >> Confluent Platform or in other distributions. But I think this
> > ends
> > > up
> > > > > >> being really shallow. I guess I feel to really get a good user
> > > > > experience
> > > > > >> the two systems have to kind of feel like part of the same thing
> > and
> > > > you
> > > > > >> can't really add that in later--you can put both in the same
> > > > > downloadable
> > > > > >> tar file but it doesn't really give a very cohesive feeling. I
> > agree
> > > > > that
> > > > > >> ultimately any of the project stuff is as much social and naming
> > as
> > > > > >> anything else--theoretically two totally independent projects
> > could
> > > > work
> > > > > >> to
> > > > > >> tightly align. In practice this seems to be quite difficult
> > though.
> > > > > >>
> > > > > >> For the frameworks--totally agree it would be good to maintain
> the
> > > > > >> framework support with the project. In some cases there may not
> be
> > > too
> > > > > >> much
> > > > > >> there since the integration gets lighter but I think whatever
> > stubs
> > > > you
> > > > > >> need should be included. So no I definitely wasn't trying to
> imply
> > > > > >> dropping
> > > > > >> support for these frameworks, just making the integration
> lighter
> > by
> > > > > >> separating process management from partition management.
> > > > > >>
> > > > > >> You raise two good points we would have to figure out if we went
> > > down
> > > > > the
> > > > > >> alignment path:
> > > > > >> 1. With respect to the name, yeah I think the first question is
> > > > whether
> > > > > >> some "re-branding" would be worth it. If so then I think we can
> > > have a
> > > > > big
> > > > > >> thread on the name. I'm definitely not set on Kafka Streaming or
> > > Kafka
> > > > > >> Streams I was just using them to be kind of illustrative. I
> agree
> > > with
> > > > > >> your
> > > > > >> critique of these names, though I think people would get the
> idea.
> > > > > >> 2. Yeah you also raise a good point about how to "factor" it.
> Here
> > > are
> > > > > the
> > > > > >> options I see (I could get enthusiastic about any of them):
> > > > > >>    a. One repo for both Kafka and Samza
> > > > > >>    b. Two repos, retaining the current seperation
> > > > > >>    c. Two repos, the equivalent of samza-api and samza-core is
> > > > absorbed
> > > > > >> almost like a third client
> > > > > >>
> > > > > >> Cheers,
> > > > > >>
> > > > > >> -Jay
> > > > > >>
> > > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> > > > martin@kleppmann.com>
> > > > > >> wrote:
> > > > > >>
> > > > > >> > Ok, thanks for the clarifications. Just a few follow-up
> > comments.
> > > > > >> >
> > > > > >> > - I see the appeal of merging with Kafka or becoming a
> > subproject:
> > > > the
> > > > > >> > reasons you mention are good. The risk I see is that release
> > > > schedules
> > > > > >> > become coupled to each other, which can slow everyone down,
> and
> > > > large
> > > > > >> > projects with many contributors are harder to manage. (Jakob,
> > can
> > > > you
> > > > > >> speak
> > > > > >> > from experience, having seen a wider range of Hadoop ecosystem
> > > > > >> projects?)
> > > > > >> >
> > > > > >> > Some of the goals of a better unified developer experience
> could
> > > > also
> > > > > be
> > > > > >> > solved by integrating Samza nicely into a Kafka distribution
> > (such
> > > > as
> > > > > >> > Confluent's). I'm not against merging projects if we decide
> > that's
> > > > the
> > > > > >> way
> > > > > >> > to go, just pointing out the same goals can perhaps also be
> > > achieved
> > > > > in
> > > > > >> > other ways.
> > > > > >> >
> > > > > >> > - With regard to dropping the YARN dependency: are you
> proposing
> > > > that
> > > > > >> > Samza doesn't give any help to people wanting to run on
> > > > > >> YARN/Mesos/AWS/etc?
> > > > > >> > So the docs would basically have a link to Slider and nothing
> > > else?
> > > > Or
> > > > > >> > would we maintain integrations with a bunch of popular
> > deployment
> > > > > >> methods
> > > > > >> > (e.g. the necessary glue and shell scripts to make Samza work
> > with
> > > > > >> Slider)?
> > > > > >> >
> > > > > >> > I absolutely think it's a good idea to have the "as a library"
> > and
> > > > > "as a
> > > > > >> > process" (using Yi's taxonomy) options for people who want
> them,
> > > > but I
> > > > > >> > think there should also be a low-friction path for common "as
> a
> > > > > service"
> > > > > >> > deployment methods, for which we probably need to maintain
> > > > > integrations.
> > > > > >> >
> > > > > >> > - Project naming: "Kafka Streams" seems odd to me, because
> Kafka
> > > is
> > > > > all
> > > > > >> > about streams already. Perhaps "Kafka Transformers" or "Kafka
> > > > Filters"
> > > > > >> > would be more apt?
> > > > > >> >
> > > > > >> > One suggestion: perhaps the core of Samza (stream
> transformation
> > > > with
> > > > > >> > state management -- i.e. the "Samza as a library" bit) could
> > > become
> > > > > >> part of
> > > > > >> > Kafka, while higher-level tools such as streaming SQL and
> > > > integrations
> > > > > >> with
> > > > > >> > deployment frameworks remain in a separate project? In other
> > > words,
> > > > > >> Kafka
> > > > > >> > would absorb the proven, stable core of Samza, which would
> > become
> > > > the
> > > > > >> > "third Kafka client" mentioned early in this thread. The Samza
> > > > project
> > > > > >> > would then target that third Kafka client as its base API, and
> > the
> > > > > >> project
> > > > > >> > would be freed up to explore more experimental new horizons.
> > > > > >> >
> > > > > >> > Martin
> > > > > >> >
> > > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com>
> wrote:
> > > > > >> >
> > > > > >> > > Hey Martin,
> > > > > >> > >
> > > > > >> > > For the YARN/Mesos/etc decoupling I actually don't think it
> > ties
> > > > our
> > > > > >> > hands
> > > > > >> > > at all, all it does is refactor things. The division of
> > > > > >> responsibility is
> > > > > >> > > that Samza core is responsible for task lifecycle, state,
> and
> > > > > >> partition
> > > > > >> > > management (using the Kafka co-ordinator) but it is NOT
> > > > responsible
> > > > > >> for
> > > > > >> > > packaging, configuration deployment or execution of
> processes.
> > > The
> > > > > >> > problem
> > > > > >> > > of packaging and starting these processes is
> > > > > >> > > framework/environment-specific. This leaves individual
> > > frameworks
> > > > to
> > > > > >> be
> > > > > >> > as
> > > > > >> > > fancy or vanilla as they like. So you can get simple
> stateless
> > > > > >> support in
> > > > > >> > > YARN, Mesos, etc using their off-the-shelf app framework
> > > (Slider,
> > > > > >> > Marathon,
> > > > > >> > > etc). These are well known by people and have nice UIs and a
> > lot
> > > > of
> > > > > >> > > flexibility. I don't think they have node affinity as a
> built
> > in
> > > > > >> option
> > > > > >> > > (though I could be wrong). So if we want that we can either
> > wait
> > > > for
> > > > > >> them
> > > > > >> > > to add it or do a custom framework to add that feature (as
> > now).
> > > > > >> > Obviously
> > > > > >> > > if you manage things with old-school ops tools
> > (puppet/chef/etc)
> > > > you
> > > > > >> get
> > > > > >> > > locality easily. The nice thing, though, is that all the
> samza
> > > > > >> "business
> > > > > >> > > logic" around partition management and fault tolerance is in
> > > Samza
> > > > > >> core
> > > > > >> > so
> > > > > >> > > it is shared across frameworks and the framework specific
> bit
> > is
> > > > > just
> > > > > >> > > whether it is smart enough to try to get the same host when
> a
> > > job
> > > > is
> > > > > >> > > restarted.
> > > > > >> > >
> > > > > >> > > With respect to the Kafka-alignment, yeah I think the goal
> > would
> > > > be
> > > > > >> (a)
> > > > > >> > > actually get better alignment in user experience, and (b)
> > > express
> > > > > >> this in
> > > > > >> > > the naming and project branding. Specifically:
> > > > > >> > > 1. Website/docs, it would be nice for the "transformation"
> api
> > > to
> > > > be
> > > > > >> > > discoverable in the main Kafka docs--i.e. be able to explain
> > > when
> > > > to
> > > > > >> use
> > > > > >> > > the consumer and when to use the stream processing
> > functionality
> > > > and
> > > > > >> lead
> > > > > >> > > people into that experience.
> > > > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever)
> > that
> > > > has
> > > > > >> both
> > > > > >> > > Kafka and the stream processing part and they actually work
> > > > > together.
> > > > > >> > > 3. Unify the programming experience so the client and Samza
> > api
> > > > > share
> > > > > >> > > config/monitoring/naming/packaging/etc.
> > > > > >> > >
> > > > > >> > > I think sub-projects keep separate committers and can have a
> > > > > separate
> > > > > >> > repo,
> > > > > >> > > but I'm actually not really sure (I can't find a definition
> > of a
> > > > > >> > subproject
> > > > > >> > > in Apache).
> > > > > >> > >
> > > > > >> > > Basically at a high-level you want the experience to "feel"
> > > like a
> > > > > >> single
> > > > > >> > > system, not to relatively independent things that are kind
> of
> > > > > >> awkwardly
> > > > > >> > > glued together.
> > > > > >> > >
> > > > > >> > > I think if we did that they having naming or branding like
> > > "kafka
> > > > > >> > > streaming" or "kafka streams" or something like that would
> > > > actually
> > > > > >> do a
> > > > > >> > > good job of conveying what it is. I do that this would help
> > > > adoption
> > > > > >> > quite
> > > > > >> > > a lot as it would correctly convey that using Kafka
> Streaming
> > > with
> > > > > >> Kafka
> > > > > >> > is
> > > > > >> > > a fairly seamless experience and Kafka is pretty heavily
> > adopted
> > > > at
> > > > > >> this
> > > > > >> > > point.
> > > > > >> > >
> > > > > >> > > Fwiw we actually considered this model originally when open
> > > > sourcing
> > > > > >> > Samza,
> > > > > >> > > however at that time Kafka was relatively unknown and we
> > decided
> > > > not
> > > > > >> to
> > > > > >> > do
> > > > > >> > > it since we felt it would be limiting. From my point of view
> > the
> > > > > three
> > > > > >> > > things have changed (1) Kafka is now really heavily used for
> > > > stream
> > > > > >> > > processing, (2) we learned that abstracting out the stream
> > well
> > > is
> > > > > >> > > basically impossible, (3) we learned it is really hard to
> keep
> > > the
> > > > > two
> > > > > >> > > things feeling like a single product.
> > > > > >> > >
> > > > > >> > > -Jay
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> > > > > >> martin@kleppmann.com>
> > > > > >> > > wrote:
> > > > > >> > >
> > > > > >> > >> Hi all,
> > > > > >> > >>
> > > > > >> > >> Lots of good thoughts here.
> > > > > >> > >>
> > > > > >> > >> I agree with the general philosophy of tying Samza more
> > firmly
> > > to
> > > > > >> Kafka.
> > > > > >> > >> After I spent a while looking at integrating other message
> > > > brokers
> > > > > >> (e.g.
> > > > > >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
> > > > > >> > SystemConsumer
> > > > > >> > >> tacitly assumes a model so much like Kafka's that pretty
> much
> > > > > nobody
> > > > > >> but
> > > > > >> > >> Kafka actually implements it. (Databus is perhaps an
> > exception,
> > > > but
> > > > > >> it
> > > > > >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza
> > > fully
> > > > > >> > dependent
> > > > > >> > >> on Kafka acknowledges that the system-independence was
> never
> > as
> > > > > real
> > > > > >> as
> > > > > >> > we
> > > > > >> > >> perhaps made it out to be. The gains of code reuse are
> real.
> > > > > >> > >>
> > > > > >> > >> The idea of decoupling Samza from YARN has also always been
> > > > > >> appealing to
> > > > > >> > >> me, for various reasons already mentioned in this thread.
> > > > Although
> > > > > >> > making
> > > > > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc)
> seems
> > > > > >> laudable,
> > > > > >> > I am
> > > > > >> > >> a little concerned that it will restrict us to a lowest
> > common
> > > > > >> > denominator.
> > > > > >> > >> For example, would host affinity (SAMZA-617) still be
> > possible?
> > > > For
> > > > > >> jobs
> > > > > >> > >> with large amounts of state, I think SAMZA-617 would be a
> big
> > > > boon,
> > > > > >> > since
> > > > > >> > >> restoring state off the changelog on every single restart
> is
> > > > > painful,
> > > > > >> > due
> > > > > >> > >> to long recovery times. It would be a shame if the
> decoupling
> > > > from
> > > > > >> YARN
> > > > > >> > >> made host affinity impossible.
> > > > > >> > >>
> > > > > >> > >> Jay, a question about the proposed API for instantiating a
> > job
> > > in
> > > > > >> code
> > > > > >> > >> (rather than a properties file): when submitting a job to a
> > > > > cluster,
> > > > > >> is
> > > > > >> > the
> > > > > >> > >> idea that the instantiation code runs on a client
> somewhere,
> > > > which
> > > > > >> then
> > > > > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or
> does
> > > that
> > > > > >> code
> > > > > >> > run
> > > > > >> > >> on each container that is part of the job (in which case,
> how
> > > > does
> > > > > >> the
> > > > > >> > job
> > > > > >> > >> submission to the cluster work)?
> > > > > >> > >>
> > > > > >> > >> I agree with Garry that it doesn't feel right to make a 1.0
> > > > release
> > > > > >> > with a
> > > > > >> > >> plan for it to be immediately obsolete. So if this is going
> > to
> > > > > >> happen, I
> > > > > >> > >> think it would be more honest to stick with 0.* version
> > numbers
> > > > > until
> > > > > >> > the
> > > > > >> > >> library-ified Samza has been implemented, is stable and
> > widely
> > > > > used.
> > > > > >> > >>
> > > > > >> > >> Should the new Samza be a subproject of Kafka? There is
> > > precedent
> > > > > for
> > > > > >> > >> tight coupling between different Apache projects (e.g.
> > Curator
> > > > and
> > > > > >> > >> Zookeeper, or Slider and YARN), so I think remaining
> separate
> > > > would
> > > > > >> be
> > > > > >> > ok.
> > > > > >> > >> Even if Samza is fully dependent on Kafka, there is enough
> > > > > substance
> > > > > >> in
> > > > > >> > >> Samza that it warrants being a separate project. An
> argument
> > in
> > > > > >> favour
> > > > > >> > of
> > > > > >> > >> merging would be if we think Kafka has a much stronger
> "brand
> > > > > >> presence"
> > > > > >> > >> than Samza; I'm ambivalent on that one. If the Kafka
> project
> > is
> > > > > >> willing
> > > > > >> > to
> > > > > >> > >> endorse Samza as the "official" way of doing stateful
> stream
> > > > > >> > >> transformations, that would probably have much the same
> > effect
> > > as
> > > > > >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike.
> > > Close
> > > > > >> > >> collaboration between the two projects will be needed in
> any
> > > > case.
> > > > > >> > >>
> > > > > >> > >> From a project management perspective, I guess the "new
> > Samza"
> > > > > would
> > > > > >> > have
> > > > > >> > >> to be developed on a branch alongside ongoing maintenance
> of
> > > the
> > > > > >> current
> > > > > >> > >> line of development? I think it would be important to
> > continue
> > > > > >> > supporting
> > > > > >> > >> existing users, and provide a graceful migration path to
> the
> > > new
> > > > > >> > version.
> > > > > >> > >> Leaving the current versions unsupported and forcing people
> > to
> > > > > >> rewrite
> > > > > >> > >> their jobs would send a bad signal.
> > > > > >> > >>
> > > > > >> > >> Best,
> > > > > >> > >> Martin
> > > > > >> > >>
> > > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io>
> wrote:
> > > > > >> > >>
> > > > > >> > >>> Hey Garry,
> > > > > >> > >>>
> > > > > >> > >>> Yeah that's super frustrating. I'd be happy to chat more
> > about
> > > > > this
> > > > > >> if
> > > > > >> > >>> you'd be interested. I think Chris and I started with the
> > idea
> > > > of
> > > > > >> "what
> > > > > >> > >>> would it take to make Samza a kick-ass ingestion tool" but
> > > > > >> ultimately
> > > > > >> > we
> > > > > >> > >>> kind of came around to the idea that ingestion and
> > > > transformation
> > > > > >> had
> > > > > >> > >>> pretty different needs and coupling the two made things
> > hard.
> > > > > >> > >>>
> > > > > >> > >>> For what it's worth I think copycat (KIP-26) actually will
> > do
> > > > what
> > > > > >> you
> > > > > >> > >> are
> > > > > >> > >>> looking for.
> > > > > >> > >>>
> > > > > >> > >>> With regard to your point about slider, I don't
> necessarily
> > > > > >> disagree.
> > > > > >> > >> But I
> > > > > >> > >>> think getting good YARN support is quite doable and I
> think
> > we
> > > > can
> > > > > >> make
> > > > > >> > >>> that work well. I think the issue this proposal solves is
> > that
> > > > > >> > >> technically
> > > > > >> > >>> it is pretty hard to support multiple cluster management
> > > systems
> > > > > the
> > > > > >> > way
> > > > > >> > >>> things are now, you need to write an "app master" or
> > > "framework"
> > > > > for
> > > > > >> > each
> > > > > >> > >>> and they are all a little different so testing is really
> > hard.
> > > > In
> > > > > >> the
> > > > > >> > >>> absence of this we have been stuck with just YARN which
> has
> > > > > >> fantastic
> > > > > >> > >>> penetration in the Hadoopy part of the org, but zero
> > > penetration
> > > > > >> > >> elsewhere.
> > > > > >> > >>> Given the huge amount of work being put in to slider,
> > > marathon,
> > > > > aws
> > > > > >> > >>> tooling, not to mention the umpteen related packaging
> > > > technologies
> > > > > >> > people
> > > > > >> > >>> want to use (Docker, Kubernetes, various cloud-specific
> > deploy
> > > > > >> tools,
> > > > > >> > >> etc)
> > > > > >> > >>> I really think it is important to get this right.
> > > > > >> > >>>
> > > > > >> > >>> -Jay
> > > > > >> > >>>
> > > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > > > > >> > >>>
> > > > > >> > >>>> Hi all,
> > > > > >> > >>>>
> > > > > >> > >>>> I think the question below re does Samza become a
> > sub-project
> > > > of
> > > > > >> Kafka
> > > > > >> > >>>> highlights the broader point around migration. Chris
> > mentions
> > > > > >> Samza's
> > > > > >> > >>>> maturity is heading towards a v1 release but I'm not sure
> > it
> > > > > feels
> > > > > >> > >> right to
> > > > > >> > >>>> launch a v1 then immediately plan to deprecate most of
> it.
> > > > > >> > >>>>
> > > > > >> > >>>> From a selfish perspective I have some guys who have
> > started
> > > > > >> working
> > > > > >> > >> with
> > > > > >> > >>>> Samza and building some new consumers/producers was next
> > up.
> > > > > Sounds
> > > > > >> > like
> > > > > >> > >>>> that is absolutely not the direction to go. I need to
> look
> > > into
> > > > > the
> > > > > >> > KIP
> > > > > >> > >> in
> > > > > >> > >>>> more detail but for me the attractiveness of adding new
> > Samza
> > > > > >> > >>>> consumer/producers -- even if yes all they were doing was
> > > > really
> > > > > >> > getting
> > > > > >> > >>>> data into and out of Kafka --  was to avoid  having to
> > worry
> > > > > about
> > > > > >> the
> > > > > >> > >>>> lifecycle management of external clients. If there is a
> > > generic
> > > > > >> Kafka
> > > > > >> > >>>> ingress/egress layer that I can plug a new connector into
> > and
> > > > > have
> > > > > >> a
> > > > > >> > >> lot of
> > > > > >> > >>>> the heavy lifting re scale and reliability done for me
> then
> > > it
> > > > > >> gives
> > > > > >> > me
> > > > > >> > >> all
> > > > > >> > >>>> the pushing new consumers/producers would. If not then it
> > > > > >> complicates
> > > > > >> > my
> > > > > >> > >>>> operational deployments.
> > > > > >> > >>>>
> > > > > >> > >>>> Which is similar to my other question with the proposal
> --
> > if
> > > > we
> > > > > >> > build a
> > > > > >> > >>>> fully available/stand-alone Samza plus the requisite
> shims
> > to
> > > > > >> > integrate
> > > > > >> > >>>> with Slider etc I suspect the former may be a lot more
> work
> > > > than
> > > > > we
> > > > > >> > >> think.
> > > > > >> > >>>> We may make it much easier for a newcomer to get
> something
> > > > > running
> > > > > >> but
> > > > > >> > >>>> having them step up and get a reliable production
> > deployment
> > > > may
> > > > > >> still
> > > > > >> > >>>> dominate mailing list  traffic, if for different reasons
> > than
> > > > > >> today.
> > > > > >> > >>>>
> > > > > >> > >>>> Don't get me wrong -- I'm comfortable with making the
> Samza
> > > > > >> dependency
> > > > > >> > >> on
> > > > > >> > >>>> Kafka much more explicit and I absolutely see the
> benefits
> > > in
> > > > > the
> > > > > >> > >>>> reduction of duplication and clashing
> > > > terminologies/abstractions
> > > > > >> that
> > > > > >> > >>>> Chris/Jay describe. Samza as a library would likely be a
> > very
> > > > > nice
> > > > > >> > tool
> > > > > >> > >> to
> > > > > >> > >>>> add to the Kafka ecosystem. I just have the concerns
> above
> > re
> > > > the
> > > > > >> > >>>> operational side.
> > > > > >> > >>>>
> > > > > >> > >>>> Garry
> > > > > >> > >>>>
> > > > > >> > >>>> -----Original Message-----
> > > > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
> > gdfm@apache.org
> > > ]
> > > > > >> > >>>> Sent: 02 July 2015 12:56
> > > > > >> > >>>> To: dev@samza.apache.org
> > > > > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> > > > > >> > >>>>
> > > > > >> > >>>> Very interesting thoughts.
> > > > > >> > >>>> From outside, I have always perceived Samza as a
> computing
> > > > layer
> > > > > >> over
> > > > > >> > >>>> Kafka.
> > > > > >> > >>>>
> > > > > >> > >>>> The question, maybe a bit provocative, is "should Samza
> be
> > a
> > > > > >> > sub-project
> > > > > >> > >>>> of Kafka then?"
> > > > > >> > >>>> Or does it make sense to keep it as a separate project
> > with a
> > > > > >> separate
> > > > > >> > >>>> governance?
> > > > > >> > >>>>
> > > > > >> > >>>> Cheers,
> > > > > >> > >>>>
> > > > > >> > >>>> --
> > > > > >> > >>>> Gianmarco
> > > > > >> > >>>>
> > > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> > > > wrote:
> > > > > >> > >>>>
> > > > > >> > >>>>> Overall, I agree to couple with Kafka more tightly.
> > Because
> > > > > Samza
> > > > > >> de
> > > > > >> > >>>>> facto is based on Kafka, and it should leverage what
> Kafka
> > > > has.
> > > > > At
> > > > > >> > the
> > > > > >> > >>>>> same time, Kafka does not need to reinvent what Samza
> > > already
> > > > > >> has. I
> > > > > >> > >>>>> also like the idea of separating the ingestion and
> > > > > transformation.
> > > > > >> > >>>>>
> > > > > >> > >>>>> But it is a little difficult for me to image how the
> Samza
> > > > will
> > > > > >> look
> > > > > >> > >>>> like.
> > > > > >> > >>>>> And I feel Chris and Jay have a little difference in
> terms
> > > of
> > > > > how
> > > > > >> > >>>>> Samza should look like.
> > > > > >> > >>>>>
> > > > > >> > >>>>> *** Will it look like what Jay's code shows (A client of
> > > > Kakfa)
> > > > > ?
> > > > > >> And
> > > > > >> > >>>>> user's application code calls this client?
> > > > > >> > >>>>>
> > > > > >> > >>>>> 1. If we make Samza be a library of Kafka (like what the
> > > code
> > > > > >> shows),
> > > > > >> > >>>>> how do we implement auto-balance and fault-tolerance?
> Are
> > > they
> > > > > >> taken
> > > > > >> > >>>>> care by the Kafka broker or other mechanism, such as
> > "Samza
> > > > > >> worker"
> > > > > >> > >>>>> (just make up the name) ?
> > > > > >> > >>>>>
> > > > > >> > >>>>> 2. What about other features, such as auto-scaling,
> shared
> > > > > state,
> > > > > >> > >>>>> monitoring?
> > > > > >> > >>>>>
> > > > > >> > >>>>>
> > > > > >> > >>>>> *** If we have Samza standalone, (is this what Chris
> > > > suggests?)
> > > > > >> > >>>>>
> > > > > >> > >>>>> 1. we still need to ingest data from Kakfa and produce
> to
> > > it.
> > > > > >> Then it
> > > > > >> > >>>>> becomes the same as what Samza looks like now, except it
> > > does
> > > > > not
> > > > > >> > rely
> > > > > >> > >>>>> on Yarn anymore.
> > > > > >> > >>>>>
> > > > > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's
> > metrics,
> > > > > logs,
> > > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > > > > >> > >>>>>
> > > > > >> > >>>>>
> > > > > >> > >>>>> Thanks,
> > > > > >> > >>>>>
> > > > > >> > >>>>> Fang, Yan
> > > > > >> > >>>>> yanfang724@gmail.com
> > > > > >> > >>>>>
> > > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > > > > wangguoz@gmail.com
> > > > > >> >
> > > > > >> > >>>> wrote:
> > > > > >> > >>>>>
> > > > > >> > >>>>>> Read through the code example and it looks good to me.
> A
> > > few
> > > > > >> > >>>>>> thoughts regarding deployment:
> > > > > >> > >>>>>>
> > > > > >> > >>>>>> Today Samza deploys as executable runnable like:
> > > > > >> > >>>>>>
> > > > > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> > > > > >> > >>>> --config-path=file://...
> > > > > >> > >>>>>>
> > > > > >> > >>>>>> And this proposal advocate for deploying Samza more as
> > > > embedded
> > > > > >> > >>>>>> libraries in user application code (ignoring the
> > > terminology
> > > > > >> since
> > > > > >> > >>>>>> it is not the
> > > > > >> > >>>>> same
> > > > > >> > >>>>>> as the prototype code):
> > > > > >> > >>>>>>
> > > > > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread
> > thread
> > > =
> > > > > new
> > > > > >> > >>>>>> Thread(task); thread.start();
> > > > > >> > >>>>>>
> > > > > >> > >>>>>> I think both of these deployment modes are important
> for
> > > > > >> different
> > > > > >> > >>>>>> types
> > > > > >> > >>>>> of
> > > > > >> > >>>>>> users. That said, I think making Samza purely
> standalone
> > is
> > > > > still
> > > > > >> > >>>>>> sufficient for either runnable or library modes.
> > > > > >> > >>>>>>
> > > > > >> > >>>>>> Guozhang
> > > > > >> > >>>>>>
> > > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> > > > jay@confluent.io>
> > > > > >> > wrote:
> > > > > >> > >>>>>>
> > > > > >> > >>>>>>> Looks like gmail mangled the code example, it was
> > supposed
> > > > to
> > > > > >> look
> > > > > >> > >>>>>>> like
> > > > > >> > >>>>>>> this:
> > > > > >> > >>>>>>>
> > > > > >> > >>>>>>> Properties props = new Properties();
> > > > > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > > > >> StreamingConfig
> > > > > >> > >>>>>>> config = new StreamingConfig(props);
> > > > > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> > > > > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> > > > > >> > >>>>>>> config.serialization(new StringSerializer(), new
> > > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
> > > > > >> > >>>>>>> KafkaStreaming(config); container.run();
> > > > > >> > >>>>>>>
> > > > > >> > >>>>>>> -Jay
> > > > > >> > >>>>>>>
> > > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> > > > jay@confluent.io
> > > > > >
> > > > > >> > >>>> wrote:
> > > > > >> > >>>>>>>
> > > > > >> > >>>>>>>> Hey guys,
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> This came out of some conversations Chris and I were
> > > having
> > > > > >> > >>>>>>>> around
> > > > > >> > >>>>>>> whether
> > > > > >> > >>>>>>>> it would make sense to use Samza as a kind of data
> > > > ingestion
> > > > > >> > >>>>> framework
> > > > > >> > >>>>>>> for
> > > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat").
> This
> > > > kind
> > > > > of
> > > > > >> > >>>>>> combined
> > > > > >> > >>>>>>>> with complaints around config and YARN and the
> > discussion
> > > > > >> around
> > > > > >> > >>>>>>>> how
> > > > > >> > >>>>> to
> > > > > >> > >>>>>>>> best do a standalone mode.
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> So the thought experiment was, given that Samza was
> > > > basically
> > > > > >> > >>>>>>>> already totally Kafka specific, what if you just
> > embraced
> > > > > that
> > > > > >> > >>>>>>>> and turned it
> > > > > >> > >>>>>> into
> > > > > >> > >>>>>>>> something less like a heavyweight framework and more
> > > like a
> > > > > >> > >>>>>>>> third
> > > > > >> > >>>>> Kafka
> > > > > >> > >>>>>>>> client--a kind of "producing consumer" with state
> > > > management
> > > > > >> > >>>>>> facilities.
> > > > > >> > >>>>>>>> Basically a library. Instead of a complex stream
> > > processing
> > > > > >> > >>>>>>>> framework
> > > > > >> > >>>>>>> this
> > > > > >> > >>>>>>>> would actually be a very simple thing, not much more
> > > > > >> complicated
> > > > > >> > >>>>>>>> to
> > > > > >> > >>>>> use
> > > > > >> > >>>>>>> or
> > > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we
> thought
> > > > about
> > > > > >> it
> > > > > >> > >>>>>>>> a
> > > > > >> > >>>>> lot
> > > > > >> > >>>>>> of
> > > > > >> > >>>>>>>> what Samza (and the other stream processing systems
> > were
> > > > > doing)
> > > > > >> > >>>>> seemed
> > > > > >> > >>>>>>> like
> > > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> Of course you need to ingest/output data to and from
> > the
> > > > > stream
> > > > > >> > >>>>>>>> processing. But when we actually looked into how that
> > > would
> > > > > >> > >>>>>>>> work,
> > > > > >> > >>>>> Samza
> > > > > >> > >>>>>>>> isn't really an ideal data ingestion framework for a
> > > bunch
> > > > of
> > > > > >> > >>>>> reasons.
> > > > > >> > >>>>>> To
> > > > > >> > >>>>>>>> really do that right you need a pretty different
> > internal
> > > > > data
> > > > > >> > >>>>>>>> model
> > > > > >> > >>>>>> and
> > > > > >> > >>>>>>>> set of apis. So what if you split them and had an api
> > for
> > > > > Kafka
> > > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate
> api
> > > for
> > > > > >> Kafka
> > > > > >> > >>>>>>>> transformation (Samza).
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> This would also allow really embracing the same
> > > terminology
> > > > > and
> > > > > >> > >>>>>>>> conventions. One complaint about the current state is
> > > that
> > > > > the
> > > > > >> > >>>>>>>> two
> > > > > >> > >>>>>>> systems
> > > > > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs
> > > > "topic"
> > > > > >> and
> > > > > >> > >>>>>>> different
> > > > > >> > >>>>>>>> config and monitoring systems means you kind of have
> to
> > > > learn
> > > > > >> > >>>>>>>> Kafka's
> > > > > >> > >>>>>>> way,
> > > > > >> > >>>>>>>> then learn Samza's slightly different way, then kind
> of
> > > > > >> > >>>>>>>> understand
> > > > > >> > >>>>> how
> > > > > >> > >>>>>>> they
> > > > > >> > >>>>>>>> map to each other, which having walked a few people
> > > through
> > > > > >> this
> > > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> Since I have been spending a lot of time on
> airplanes I
> > > > > hacked
> > > > > >> > >>>>>>>> up an ernest but still somewhat incomplete prototype
> of
> > > > what
> > > > > >> > >>>>>>>> this would
> > > > > >> > >>>>> look
> > > > > >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka
> as
> > > it
> > > > > >> > >>>>>>>> required a
> > > > > >> > >>>>>> few
> > > > > >> > >>>>>>>> changes to the new consumer. Here is the code:
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>
> > > > > >> > >>>>>>
> > > > > >> > >>>>>
> > > > > >> >
> > > > >
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > > >> > >>>>> /apache/kafka/clients/streaming
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> For the purpose of the prototype I just liberally
> > renamed
> > > > > >> > >>>>>>>> everything
> > > > > >> > >>>>> to
> > > > > >> > >>>>>>>> try to align it with Kafka with no regard for
> > > > compatibility.
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> To use this would be something like this:
> > > > > >> > >>>>>>>> Properties props = new Properties();
> > > > > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > > > >> > >>>>>>>> StreamingConfig config = new
> > > > > >> > >>>>> StreamingConfig(props);
> > > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > > > >> > >>>>>>>> "test-topic-2");
> > > > > >> config.processor(ExampleStreamProcessor.class);
> > > > > >> > >>>>>>> config.serialization(new
> > > > > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
> > > > KafkaStreaming
> > > > > >> > >>>>>> container =
> > > > > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> > > > > StreamProcessor
> > > > > >> > >>>>>>>> is basically StreamTask.
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> So rather than putting all the class names in a file
> > and
> > > > then
> > > > > >> > >>>>>>>> having
> > > > > >> > >>>>>> the
> > > > > >> > >>>>>>>> job assembled by reflection, you just instantiate the
> > > > > container
> > > > > >> > >>>>>>>> programmatically. Work is balanced over however many
> > > > > instances
> > > > > >> > >>>>>>>> of
> > > > > >> > >>>>> this
> > > > > >> > >>>>>>> are
> > > > > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new
> tasks
> > > are
> > > > > >> added
> > > > > >> > >>>>>>>> to
> > > > > >> > >>>>> the
> > > > > >> > >>>>>>>> existing containers without shutting them down).
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> We would provide some glue for running this stuff in
> > YARN
> > > > via
> > > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of
> their
> > > > tools
> > > > > >> > >>>>>>>> but from the
> > > > > >> > >>>>>> point
> > > > > >> > >>>>>>> of
> > > > > >> > >>>>>>>> view of these frameworks these stream processing jobs
> > are
> > > > > just
> > > > > >> > >>>>>> stateless
> > > > > >> > >>>>>>>> services that can come and go and expand and contract
> > at
> > > > > will.
> > > > > >> > >>>>>>>> There
> > > > > >> > >>>>> is
> > > > > >> > >>>>>>> no
> > > > > >> > >>>>>>>> more custom scheduler.
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> Here are some relevant details:
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get
> larger
> > > if
> > > > we
> > > > > >> > >>>>>>>>  productionized but not vastly larger. We really do
> > get a
> > > > ton
> > > > > >> > >>>>>>>> of
> > > > > >> > >>>>>>> leverage
> > > > > >> > >>>>>>>>  out of Kafka.
> > > > > >> > >>>>>>>>  2. Partition management is fully delegated to the
> new
> > > > > >> consumer.
> > > > > >> > >>>>> This
> > > > > >> > >>>>>>>>  is nice since now any partition management strategy
> > > > > available
> > > > > >> > >>>>>>>> to
> > > > > >> > >>>>>> Kafka
> > > > > >> > >>>>>>>>  consumer is also available to Samza (and vice versa)
> > and
> > > > > with
> > > > > >> > >>>>>>>> the
> > > > > >> > >>>>>>> exact
> > > > > >> > >>>>>>>>  same configs.
> > > > > >> > >>>>>>>>  3. It supports state as well as state reuse
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> Anyhow take a look, hopefully it is thought
> provoking.
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> -Jay
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > > > >> > >>>>>> criccomini@apache.org>
> > > > > >> > >>>>>>>> wrote:
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>>> Hey all,
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> I have had some discussions with Samza engineers at
> > > > LinkedIn
> > > > > >> > >>>>>>>>> and
> > > > > >> > >>>>>>> Confluent
> > > > > >> > >>>>>>>>> and we came up with a few observations and would
> like
> > to
> > > > > >> > >>>>>>>>> propose
> > > > > >> > >>>>> some
> > > > > >> > >>>>>>>>> changes.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> We've observed some things that I want to call out
> > about
> > > > > >> > >>>>>>>>> Samza's
> > > > > >> > >>>>>> design,
> > > > > >> > >>>>>>>>> and I'd like to propose some changes.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment
> system.
> > > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> > > > consumer
> > > > > >> > >>>>>>>>> APIs
> > > > > >> > >>>>> are
> > > > > >> > >>>>>>>>> trying to solve a lot of the same problems.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> All three of these issues are related, but I'll
> > address
> > > > them
> > > > > >> in
> > > > > >> > >>>>> order.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Deployment
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic
> > > deployment
> > > > > >> > >>>>>>>>> scheduler
> > > > > >> > >>>>>> such
> > > > > >> > >>>>>>>>> as
> > > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we
> > bet
> > > > that
> > > > > >> > >>>>>>>>> there
> > > > > >> > >>>>>> would
> > > > > >> > >>>>>>>>> be
> > > > > >> > >>>>>>>>> one or two winners in this area, and we could
> support
> > > > them,
> > > > > >> and
> > > > > >> > >>>>>>>>> the
> > > > > >> > >>>>>> rest
> > > > > >> > >>>>>>>>> would go away. In reality, there are many
> variations.
> > > > > >> > >>>>>>>>> Furthermore,
> > > > > >> > >>>>>> many
> > > > > >> > >>>>>>>>> people still prefer to just start their processors
> > like
> > > > > normal
> > > > > >> > >>>>>>>>> Java processes, and use traditional deployment
> scripts
> > > > such
> > > > > as
> > > > > >> > >>>>>>>>> Fabric,
> > > > > >> > >>>>>> Chef,
> > > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users
> > makes
> > > > the
> > > > > >> > >>>>>>>>> Samza start-up process really painful for first time
> > > > users.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit
> of
> > a
> > > > > >> > >>>>>>>>> mis-fire
> > > > > >> > >>>>>> because
> > > > > >> > >>>>>>>>> of
> > > > > >> > >>>>>>>>> a fundamental misunderstanding between the nature of
> > > batch
> > > > > >> jobs
> > > > > >> > >>>>>>>>> and
> > > > > >> > >>>>>>> stream
> > > > > >> > >>>>>>>>> processing jobs. Early on, we made conscious effort
> to
> > > > favor
> > > > > >> > >>>>>>>>> the
> > > > > >> > >>>>>> Hadoop
> > > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked
> and
> > > was
> > > > > well
> > > > > >> > >>>>>>> understood.
> > > > > >> > >>>>>>>>> One thing that we missed was that batch jobs have a
> > > > definite
> > > > > >> > >>>>>> beginning,
> > > > > >> > >>>>>>>>> and
> > > > > >> > >>>>>>>>> end, and stream processing jobs don't (usually).
> This
> > > > leads
> > > > > to
> > > > > >> > >>>>>>>>> a
> > > > > >> > >>>>> much
> > > > > >> > >>>>>>>>> simpler scheduling problem for stream processors.
> You
> > > > > >> basically
> > > > > >> > >>>>>>>>> just
> > > > > >> > >>>>>>> need
> > > > > >> > >>>>>>>>> to find a place to start the processor, and start
> it.
> > > The
> > > > > way
> > > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a
> > > cluster
> > > > > >> > >>>>>>>>> being "full". We always
> > > > > >> > >>>>>> add
> > > > > >> > >>>>>>>>> more machines. The problem with coupling Samza with
> a
> > > > > >> scheduler
> > > > > >> > >>>>>>>>> is
> > > > > >> > >>>>>> that
> > > > > >> > >>>>>>>>> Samza (as a framework) now has to handle deployment.
> > > This
> > > > > >> pulls
> > > > > >> > >>>>>>>>> in a
> > > > > >> > >>>>>>> bunch
> > > > > >> > >>>>>>>>> of things such as configuration distribution (config
> > > > > stream),
> > > > > >> > >>>>>>>>> shell
> > > > > >> > >>>>>>> scrips
> > > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> > > > stuff),
> > > > > >> etc.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Another reason for requiring dynamic deployment was
> to
> > > > > support
> > > > > >> > >>>>>>>>> data locality. If you want to have locality, you
> need
> > to
> > > > put
> > > > > >> > >>>>>>>>> your
> > > > > >> > >>>>>> processors
> > > > > >> > >>>>>>>>> close to the data they're processing. Upon further
> > > > > >> > >>>>>>>>> investigation,
> > > > > >> > >>>>>>> though,
> > > > > >> > >>>>>>>>> this feature is not that beneficial. There is some
> > good
> > > > > >> > >>>>>>>>> discussion
> > > > > >> > >>>>>> about
> > > > > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took
> the
> > > > > >> > >>>>>>>>> Map/Reduce
> > > > > >> > >>>>>> path,
> > > > > >> > >>>>>>>>> but
> > > > > >> > >>>>>>>>> there are some fundamental differences between HDFS
> > and
> > > > > Kafka.
> > > > > >> > >>>>>>>>> HDFS
> > > > > >> > >>>>>> has
> > > > > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to
> less
> > > > > >> > >>>>>>>>> optimization potential with stream processors on top
> > of
> > > > > Kafka.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't
> > > have
> > > > > any
> > > > > >> > >>>>>>>>> built
> > > > > >> > >>>>> in
> > > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the
> > > dynamic
> > > > > >> > >>>>>>>>> deployment scheduling system to handle restarts
> when a
> > > > > >> > >>>>>>>>> processor dies. This has
> > > > > >> > >>>>>>> made
> > > > > >> > >>>>>>>>> it very difficult to write a standalone Samza
> > container
> > > > > >> > >>>> (SAMZA-516).
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Pluggability
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> In some cases pluggability is good, but I think that
> > > we've
> > > > > >> gone
> > > > > >> > >>>>>>>>> too
> > > > > >> > >>>>>> far
> > > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> * Pluggable config.
> > > > > >> > >>>>>>>>> * Pluggable metrics.
> > > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > > > > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> > > > > SystemProducer,
> > > > > >> > >>>> etc).
> > > > > >> > >>>>>>>>> * Pluggable serdes.
> > > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > > > >> > >>>>>>>>> * Pluggable strategies for just about every
> component
> > > > > >> > >>>>> (MessageChooser,
> > > > > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> There's probably more that I've forgotten, as well.
> > Some
> > > > of
> > > > > >> > >>>>>>>>> these
> > > > > >> > >>>>> are
> > > > > >> > >>>>>>>>> useful, but some have proven not to be. This all
> comes
> > > at
> > > > a
> > > > > >> cost:
> > > > > >> > >>>>>>>>> complexity. This complexity is making it harder for
> > our
> > > > > users
> > > > > >> > >>>>>>>>> to
> > > > > >> > >>>>> pick
> > > > > >> > >>>>>> up
> > > > > >> > >>>>>>>>> and use Samza out of the box. It also makes it
> > difficult
> > > > for
> > > > > >> > >>>>>>>>> Samza developers to reason about what the
> > > characteristics
> > > > of
> > > > > >> > >>>>>>>>> the container (since the characteristics change
> > > depending
> > > > on
> > > > > >> > >>>>>>>>> which plugins are use).
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> The issues with pluggability are most visible in the
> > > > System
> > > > > >> APIs.
> > > > > >> > >>>>> What
> > > > > >> > >>>>>>>>> Samza really requires to be functional is Kafka as
> its
> > > > > >> > >>>>>>>>> transport
> > > > > >> > >>>>>> layer.
> > > > > >> > >>>>>>>>> But
> > > > > >> > >>>>>>>>> we've conflated two unrelated use cases into one
> API:
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> The current System API supports both of these use
> > cases.
> > > > The
> > > > > >> > >>>>>>>>> problem
> > > > > >> > >>>>>> is,
> > > > > >> > >>>>>>>>> we
> > > > > >> > >>>>>>>>> actually want different features for each use case.
> By
> > > > > >> papering
> > > > > >> > >>>>>>>>> over
> > > > > >> > >>>>>>> these
> > > > > >> > >>>>>>>>> two use cases, and providing a single API, we've
> > > > introduced
> > > > > a
> > > > > >> > >>>>>>>>> ton of
> > > > > >> > >>>>>>> leaky
> > > > > >> > >>>>>>>>> abstractions.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> For example, what we'd really like in (2) is to have
> > > > > >> > >>>>>>>>> monotonically increasing longs for offsets (like
> > Kafka).
> > > > > This
> > > > > >> > >>>>>>>>> would be at odds
> > > > > >> > >>>>> with
> > > > > >> > >>>>>>> (1),
> > > > > >> > >>>>>>>>> though, since different systems have different
> > > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > > > >> > >>>>>>>>> There was discussion both on the mailing list and
> the
> > > SQL
> > > > > >> JIRAs
> > > > > >> > >>>>> about
> > > > > >> > >>>>>>> the
> > > > > >> > >>>>>>>>> need for this.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> The same thing holds true for replayability. Kafka
> > > allows
> > > > us
> > > > > >> to
> > > > > >> > >>>>> rewind
> > > > > >> > >>>>>>>>> when
> > > > > >> > >>>>>>>>> we have a failure. Many other systems don't. In some
> > > > cases,
> > > > > >> > >>>>>>>>> systems
> > > > > >> > >>>>>>> return
> > > > > >> > >>>>>>>>> null for their offsets (e.g.
> WikipediaSystemConsumer)
> > > > > because
> > > > > >> > >>>>>>>>> they
> > > > > >> > >>>>>> have
> > > > > >> > >>>>>>> no
> > > > > >> > >>>>>>>>> offsets.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Partitioning is another example. Kafka supports
> > > > > partitioning,
> > > > > >> > >>>>>>>>> but
> > > > > >> > >>>>> many
> > > > > >> > >>>>>>>>> systems don't. We model this by having a single
> > > partition
> > > > > for
> > > > > >> > >>>>>>>>> those systems. Still, other systems model
> partitioning
> > > > > >> > >>>> differently (e.g.
> > > > > >> > >>>>>>>>> Kinesis).
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating
> > > streams
> > > > > in
> > > > > >> a
> > > > > >> > >>>>>>>>> system-agnostic way is almost impossible. As is
> > modeling
> > > > > >> > >>>>>>>>> metadata
> > > > > >> > >>>>> for
> > > > > >> > >>>>>>> the
> > > > > >> > >>>>>>>>> system (replication factor, partitions, location,
> > etc).
> > > > The
> > > > > >> > >>>>>>>>> list
> > > > > >> > >>>>> goes
> > > > > >> > >>>>>>> on.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Duplicate work
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's
> > > consumer
> > > > > and
> > > > > >> > >>>>> producer
> > > > > >> > >>>>>>>>> APIs
> > > > > >> > >>>>>>>>> had a relatively weak feature set. On the
> > consumer-side,
> > > > you
> > > > > >> > >>>>>>>>> had two
> > > > > >> > >>>>>>>>> options: use the high level consumer, or the simple
> > > > > consumer.
> > > > > >> > >>>>>>>>> The
> > > > > >> > >>>>>>> problem
> > > > > >> > >>>>>>>>> with the high-level consumer was that it controlled
> > your
> > > > > >> > >>>>>>>>> offsets, partition assignments, and the order in
> which
> > > you
> > > > > >> > >>>>>>>>> received messages. The
> > > > > >> > >>>>> problem
> > > > > >> > >>>>>>>>> with
> > > > > >> > >>>>>>>>> the simple consumer is that it's not simple. It's
> > basic.
> > > > You
> > > > > >> > >>>>>>>>> end up
> > > > > >> > >>>>>>> having
> > > > > >> > >>>>>>>>> to handle a lot of really low-level stuff that you
> > > > > shouldn't.
> > > > > >> > >>>>>>>>> We
> > > > > >> > >>>>>> spent a
> > > > > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> > > > robust.
> > > > > >> It
> > > > > >> > >>>>>>>>> also allows us to support some cool features:
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> * Per-partition message ordering and prioritization.
> > > > > >> > >>>>>>>>> * Tight control over partition assignment to support
> > > > joins,
> > > > > >> > >>>>>>>>> global
> > > > > >> > >>>>>> state
> > > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > > > > >> > >>>>>>>>> * Tight control over offset checkpointing.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> What we didn't realize at the time is that these
> > > features
> > > > > >> > >>>>>>>>> should
> > > > > >> > >>>>>>> actually
> > > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just
> Samza
> > > > stream
> > > > > >> > >>>>>> processors)
> > > > > >> > >>>>>>>>> end up wanting to do things like joins and partition
> > > > > >> > >>>>>>>>> assignment. The
> > > > > >> > >>>>>>> Kafka
> > > > > >> > >>>>>>>>> community has come to the same conclusion. They're
> > > adding
> > > > a
> > > > > >> ton
> > > > > >> > >>>>>>>>> of upgrades into their new Kafka consumer
> > > implementation.
> > > > > To a
> > > > > >> > >>>>>>>>> large extent,
> > > > > >> > >>>>> it's
> > > > > >> > >>>>>>>>> duplicate work to what we've already done in Samza.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
> > > > > approach
> > > > > >> > >>>>>>>>> to
> > > > > >> > >>>>>> Samza's
> > > > > >> > >>>>>>>>> KafkaCheckpointManager implementation for handling
> > > offset
> > > > > >> > >>>>>> checkpointing.
> > > > > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature
> > stores
> > > > > >> offset
> > > > > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them
> > > from
> > > > > the
> > > > > >> > >>>>>>>>> broker.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> A lot of this seems like a waste, since we could
> have
> > > > shared
> > > > > >> > >>>>>>>>> the
> > > > > >> > >>>>> work
> > > > > >> > >>>>>> if
> > > > > >> > >>>>>>>>> it
> > > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Vision
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> All of this leads me to a rather radical proposal.
> > Samza
> > > > is
> > > > > >> > >>>>> relatively
> > > > > >> > >>>>>>>>> stable at this point. I'd venture to say that we're
> > > near a
> > > > > 1.0
> > > > > >> > >>>>>> release.
> > > > > >> > >>>>>>>>> I'd
> > > > > >> > >>>>>>>>> like to propose that we take what we've learned, and
> > > begin
> > > > > >> > >>>>>>>>> thinking
> > > > > >> > >>>>>>> about
> > > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were
> > > starting
> > > > > >> from
> > > > > >> > >>>>>> scratch?
> > > > > >> > >>>>>>>>> My
> > > > > >> > >>>>>>>>> proposal is to:
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > > > > >> > >>>>>>>>> processors, and eliminate all direct dependences on
> > > YARN,
> > > > > >> Mesos,
> > > > > >> > >>>> etc.
> > > > > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as
> the
> > > > > stream
> > > > > >> > >>>>>> processing
> > > > > >> > >>>>>>>>> layer.
> > > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging,
> serialization,
> > > and
> > > > > >> > >>>>>>>>> config
> > > > > >> > >>>>>>> systems,
> > > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> This would fix all of the issues that I outlined
> > above.
> > > It
> > > > > >> > >>>>>>>>> should
> > > > > >> > >>>>> also
> > > > > >> > >>>>>>>>> shrink the Samza code base pretty dramatically.
> > > Supporting
> > > > > >> only
> > > > > >> > >>>>>>>>> a standalone container will allow Samza to be
> executed
> > > on
> > > > > YARN
> > > > > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or
> most
> > > > other
> > > > > >> > >>>>>>>>> in-house
> > > > > >> > >>>>>>> deployment
> > > > > >> > >>>>>>>>> systems. This should make life a lot easier for new
> > > users.
> > > > > >> > >>>>>>>>> Imagine
> > > > > >> > >>>>>>> having
> > > > > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in
> > > mailing
> > > > > >> list
> > > > > >> > >>>>>> traffic
> > > > > >> > >>>>>>>>> will be pretty dramatic.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The
> > > reality
> > > > > is,
> > > > > >> > >>>>> everyone
> > > > > >> > >>>>>>>>> that
> > > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically
> > > > require
> > > > > >> it
> > > > > >> > >>>>>> already
> > > > > >> > >>>>>>> in
> > > > > >> > >>>>>>>>> order for most features to work. Those that are
> using
> > > > other
> > > > > >> > >>>>>>>>> systems
> > > > > >> > >>>>>> are
> > > > > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and
> then
> > > > they
> > > > > do
> > > > > >> > >>>>>>>>> the processing on top. There is already discussion (
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>
> > > > > >> > >>>>>>
> > > > > >> > >>>>>
> > > > > >> >
> > > > >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > > >> > >>>>> 767
> > > > > >> > >>>>>>>>> )
> > > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely
> easy.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Once we make the call to couple with Kafka, we can
> > > > leverage
> > > > > a
> > > > > >> > >>>>>>>>> ton of
> > > > > >> > >>>>>>> their
> > > > > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own
> > config,
> > > > > >> > >>>>>>>>> metrics,
> > > > > >> > >>>>> etc.
> > > > > >> > >>>>>>> We
> > > > > >> > >>>>>>>>> can all share the same libraries, and make them
> > better.
> > > > This
> > > > > >> > >>>>>>>>> will
> > > > > >> > >>>>> also
> > > > > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and
> will
> > > let
> > > > > us
> > > > > >> > >>>>> leverage
> > > > > >> > >>>>>>>>> their offset management and partition management,
> > rather
> > > > > than
> > > > > >> > >>>>>>>>> having
> > > > > >> > >>>>>> our
> > > > > >> > >>>>>>>>> own. All of the coordinator stream code would go
> away,
> > > as
> > > > > >> would
> > > > > >> > >>>>>>>>> most
> > > > > >> > >>>>>> of
> > > > > >> > >>>>>>>>> the
> > > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
> > > > > partition
> > > > > >> > >>>>>>> management
> > > > > >> > >>>>>>>>> features into the Kafka broker, but they're already
> > > moving
> > > > > in
> > > > > >> > >>>>>>>>> that direction with the new consumer API. The
> features
> > > we
> > > > > have
> > > > > >> > >>>>>>>>> for
> > > > > >> > >>>>>> partition
> > > > > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like
> they
> > > > should
> > > > > >> be
> > > > > >> > >>>>>>>>> in
> > > > > >> > >>>>>> Kafka
> > > > > >> > >>>>>>>>> anyway. There will always be some niche usages which
> > > will
> > > > > >> > >>>>>>>>> require
> > > > > >> > >>>>>> extra
> > > > > >> > >>>>>>>>> care and hence full control over partition
> assignments
> > > > much
> > > > > >> > >>>>>>>>> like the
> > > > > >> > >>>>>>> Kafka
> > > > > >> > >>>>>>>>> low level consumer api. These would continue to be
> > > > > supported.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> These items will be good for the Samza community.
> > > They'll
> > > > > make
> > > > > >> > >>>>>>>>> Samza easier to use, and make it easier for
> developers
> > > to
> > > > > add
> > > > > >> > >>>>>>>>> new features.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat
> > backwards
> > > > > >> > >>>>> incompatible
> > > > > >> > >>>>>>>>> change). If we choose to go this route, it's
> important
> > > > that
> > > > > we
> > > > > >> > >>>>> openly
> > > > > >> > >>>>>>>>> communicate how we're going to provide a migration
> > path
> > > > from
> > > > > >> > >>>>>>>>> the
> > > > > >> > >>>>>>> existing
> > > > > >> > >>>>>>>>> APIs to the new ones (if we make incompatible
> > changes).
> > > I
> > > > > >> think
> > > > > >> > >>>>>>>>> at a minimum, we'd probably need to provide a
> wrapper
> > to
> > > > > allow
> > > > > >> > >>>>>>>>> existing StreamTask implementations to continue
> > running
> > > on
> > > > > the
> > > > > >> > >>>> new container.
> > > > > >> > >>>>>>> It's
> > > > > >> > >>>>>>>>> also important that we openly communicate about
> > timing,
> > > > and
> > > > > >> > >>>>>>>>> stages
> > > > > >> > >>>>> of
> > > > > >> > >>>>>>> the
> > > > > >> > >>>>>>>>> migration.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> If you made it this far, I'm sure you have opinions.
> > :)
> > > > > Please
> > > > > >> > >>>>>>>>> send
> > > > > >> > >>>>>> your
> > > > > >> > >>>>>>>>> thoughts and feedback.
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>> Cheers,
> > > > > >> > >>>>>>>>> Chris
> > > > > >> > >>>>>>>>>
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>>
> > > > > >> > >>>>>>>
> > > > > >> > >>>>>>
> > > > > >> > >>>>>>
> > > > > >> > >>>>>>
> > > > > >> > >>>>>> --
> > > > > >> > >>>>>> -- Guozhang
> > > > > >> > >>>>>>
> > > > > >> > >>>>>
> > > > > >> > >>>>
> > > > > >> > >>
> > > > > >> > >>
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@confluent.io>.
Yeah totally agree. I think you have this issue even today, right? I.e. if
you need to make a simple config change and you're running in YARN today
you end up bouncing the job which then rebuilds state. I think the fix is
exactly what you described which is to have a long timeout on partition
movement for stateful jobs so that if a job is just getting bounced, and
the cluster manager (or admin) is smart enough to restart it on the same
host when possible, it can optimistically reuse any existing state it finds
on disk (if it is valid).

So in this model the charter of the CM is to place processes as stickily as
possible and to restart or re-place failed processes. The charter of the
partition management system is to control the assignment of work to these
processes. The nice thing about this is that the work assignment, timeouts,
behavior, configs, and code will all be the same across all cluster
managers.

So I think that prototype would actually give you exactly what you want
today for any cluster manager (or manual placement + restart script) that
was sticky in terms of host placement since there is already a configurable
partition movement timeout and task-by-task state reuse with a check on
state validity.

-Jay

On Fri, Jul 10, 2015 at 8:34 AM, Roger Hoover <ro...@gmail.com>
wrote:

> That would be great to let Kafka do as much heavy lifting as possible and
> make it easier for other languages to implement Samza apis.
>
> One thing to watch out for is the interplay between Kafka's group
> management and the external scheduler/process manager's fault tolerance.
> If a container dies, the Kafka group membership protocol will try to assign
> it's tasks to other containers while at the same time the process manager
> is trying to relaunch the container.  Without some consideration for this
> (like a configurable amount of time to wait before Kafka alters the group
> membership), there may be thrashing going on which is especially bad for
> containers with large amounts of local state.
>
> Someone else pointed this out already but I thought it might be worth
> calling out again.
>
> Cheers,
>
> Roger
>
>
> On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey Roger,
> >
> > I couldn't agree more. We spent a bunch of time talking to people and
> that
> > is exactly the stuff we heard time and again. What makes it hard, of
> > course, is that there is some tension between compatibility with what's
> > there now and making things better for new users.
> >
> > I also strongly agree with the importance of multi-language support. We
> are
> > talking now about Java, but for application development use cases people
> > want to work in whatever language they are using elsewhere. I think
> moving
> > to a model where Kafka itself does the group membership, lifecycle
> control,
> > and partition assignment has the advantage of putting all that complex
> > stuff behind a clean api that the clients are already going to be
> > implementing for their consumer, so the added functionality for stream
> > processing beyond a consumer becomes very minor.
> >
> > -Jay
> >
> > On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <ro...@gmail.com>
> > wrote:
> >
> > > Metamorphosis...nice. :)
> > >
> > > This has been a great discussion.  As a user of Samza who's recently
> > > integrated it into a relatively large organization, I just want to add
> > > support to a few points already made.
> > >
> > > The biggest hurdles to adoption of Samza as it currently exists that
> I've
> > > experienced are:
> > > 1) YARN - YARN is overly complex in many environments where Puppet
> would
> > do
> > > just fine but it was the only mechanism to get fault tolerance.
> > > 2) Configuration - I think I like the idea of configuring most of the
> job
> > > in code rather than config files.  In general, I think the goal should
> be
> > > to make it harder to make mistakes, especially of the kind where the
> code
> > > expects something and the config doesn't match.  The current config is
> > > quite intricate and error-prone.  For example, the application logic
> may
> > > depend on bootstrapping a topic but rather than asserting that in the
> > code,
> > > you have to rely on getting the config right.  Likewise with serdes,
> the
> > > Java representations produced by various serdes (JSON, Avro, etc.) are
> > not
> > > equivalent so you cannot just reconfigure a serde without changing the
> > > code.   It would be nice for jobs to be able to assert what they expect
> > > from their input topics in terms of partitioning.  This is getting a
> > little
> > > off topic but I was even thinking about creating a "Samza config
> linter"
> > > that would sanity check a set of configs.  Especially in organizations
> > > where config is managed by a different team than the application
> > developer,
> > > it's very hard to get avoid config mistakes.
> > > 3) Java/Scala centric - for many teams (especially DevOps-type folks),
> > the
> > > pain of the Java toolchain (maven, slow builds, weak command line
> > support,
> > > configuration over convention) really inhibits productivity.  As more
> and
> > > more high-quality clients become available for Kafka, I hope they'll
> > follow
> > > Samza's model.  Not sure how much it affects the proposals in this
> thread
> > > but please consider other languages in the ecosystem as well.  From
> what
> > > I've heard, Spark has more Python users than Java/Scala.
> > > (FYI, we added a Jython wrapper for the Samza API
> > >
> > >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > > and are working on a Yeoman generator
> > > https://github.com/Quantiply/generator-rico for Jython/Samza projects
> to
> > > alleviate some of the pain)
> > >
> > > I also want to underscore Jay's point about improving the user
> > experience.
> > > That's a very important factor for adoption.  I think the goal should
> be
> > to
> > > make Samza as easy to get started with as something like Logstash.
> > > Logstash is vastly inferior in terms of capabilities to Samza but it's
> > easy
> > > to get started and that makes a big difference.
> > >
> > > Cheers,
> > >
> > > Roger
> > >
> > >
> > >
> > >
> > >
> > > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
> > > gdfm@apache.org> wrote:
> > >
> > > > Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
> > > winner
> > > > :)
> > > >
> > > > --
> > > > Gianmarco
> > > >
> > > > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
> > gdfm@apache.org
> > > >
> > > > wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > @Martin, thanks for you comments.
> > > > > Maybe I'm missing some important point, but I think coupling the
> > > releases
> > > > > is actually a *good* thing.
> > > > > To make an example, would it be better if the MR and HDFS
> components
> > of
> > > > > Hadoop had different release schedules?
> > > > >
> > > > > Actually, keeping the discussion in a single place would make
> > agreeing
> > > on
> > > > > releases (and backwards compatibility) much easier, as everybody
> > would
> > > be
> > > > > responsible for the whole codebase.
> > > > >
> > > > > That said, I like the idea of absorbing samza-core as a
> sub-project,
> > > and
> > > > > leave the fancy stuff separate.
> > > > > It probably gives 90% of the benefits we have been discussing here.
> > > > >
> > > > > Cheers,
> > > > >
> > > > > --
> > > > > Gianmarco
> > > > >
> > > > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
> > > > >
> > > > >> Hey Martin,
> > > > >>
> > > > >> I agree coupling release schedules is a downside.
> > > > >>
> > > > >> Definitely we can try to solve some of the integration problems in
> > > > >> Confluent Platform or in other distributions. But I think this
> ends
> > up
> > > > >> being really shallow. I guess I feel to really get a good user
> > > > experience
> > > > >> the two systems have to kind of feel like part of the same thing
> and
> > > you
> > > > >> can't really add that in later--you can put both in the same
> > > > downloadable
> > > > >> tar file but it doesn't really give a very cohesive feeling. I
> agree
> > > > that
> > > > >> ultimately any of the project stuff is as much social and naming
> as
> > > > >> anything else--theoretically two totally independent projects
> could
> > > work
> > > > >> to
> > > > >> tightly align. In practice this seems to be quite difficult
> though.
> > > > >>
> > > > >> For the frameworks--totally agree it would be good to maintain the
> > > > >> framework support with the project. In some cases there may not be
> > too
> > > > >> much
> > > > >> there since the integration gets lighter but I think whatever
> stubs
> > > you
> > > > >> need should be included. So no I definitely wasn't trying to imply
> > > > >> dropping
> > > > >> support for these frameworks, just making the integration lighter
> by
> > > > >> separating process management from partition management.
> > > > >>
> > > > >> You raise two good points we would have to figure out if we went
> > down
> > > > the
> > > > >> alignment path:
> > > > >> 1. With respect to the name, yeah I think the first question is
> > > whether
> > > > >> some "re-branding" would be worth it. If so then I think we can
> > have a
> > > > big
> > > > >> thread on the name. I'm definitely not set on Kafka Streaming or
> > Kafka
> > > > >> Streams I was just using them to be kind of illustrative. I agree
> > with
> > > > >> your
> > > > >> critique of these names, though I think people would get the idea.
> > > > >> 2. Yeah you also raise a good point about how to "factor" it. Here
> > are
> > > > the
> > > > >> options I see (I could get enthusiastic about any of them):
> > > > >>    a. One repo for both Kafka and Samza
> > > > >>    b. Two repos, retaining the current seperation
> > > > >>    c. Two repos, the equivalent of samza-api and samza-core is
> > > absorbed
> > > > >> almost like a third client
> > > > >>
> > > > >> Cheers,
> > > > >>
> > > > >> -Jay
> > > > >>
> > > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> > > martin@kleppmann.com>
> > > > >> wrote:
> > > > >>
> > > > >> > Ok, thanks for the clarifications. Just a few follow-up
> comments.
> > > > >> >
> > > > >> > - I see the appeal of merging with Kafka or becoming a
> subproject:
> > > the
> > > > >> > reasons you mention are good. The risk I see is that release
> > > schedules
> > > > >> > become coupled to each other, which can slow everyone down, and
> > > large
> > > > >> > projects with many contributors are harder to manage. (Jakob,
> can
> > > you
> > > > >> speak
> > > > >> > from experience, having seen a wider range of Hadoop ecosystem
> > > > >> projects?)
> > > > >> >
> > > > >> > Some of the goals of a better unified developer experience could
> > > also
> > > > be
> > > > >> > solved by integrating Samza nicely into a Kafka distribution
> (such
> > > as
> > > > >> > Confluent's). I'm not against merging projects if we decide
> that's
> > > the
> > > > >> way
> > > > >> > to go, just pointing out the same goals can perhaps also be
> > achieved
> > > > in
> > > > >> > other ways.
> > > > >> >
> > > > >> > - With regard to dropping the YARN dependency: are you proposing
> > > that
> > > > >> > Samza doesn't give any help to people wanting to run on
> > > > >> YARN/Mesos/AWS/etc?
> > > > >> > So the docs would basically have a link to Slider and nothing
> > else?
> > > Or
> > > > >> > would we maintain integrations with a bunch of popular
> deployment
> > > > >> methods
> > > > >> > (e.g. the necessary glue and shell scripts to make Samza work
> with
> > > > >> Slider)?
> > > > >> >
> > > > >> > I absolutely think it's a good idea to have the "as a library"
> and
> > > > "as a
> > > > >> > process" (using Yi's taxonomy) options for people who want them,
> > > but I
> > > > >> > think there should also be a low-friction path for common "as a
> > > > service"
> > > > >> > deployment methods, for which we probably need to maintain
> > > > integrations.
> > > > >> >
> > > > >> > - Project naming: "Kafka Streams" seems odd to me, because Kafka
> > is
> > > > all
> > > > >> > about streams already. Perhaps "Kafka Transformers" or "Kafka
> > > Filters"
> > > > >> > would be more apt?
> > > > >> >
> > > > >> > One suggestion: perhaps the core of Samza (stream transformation
> > > with
> > > > >> > state management -- i.e. the "Samza as a library" bit) could
> > become
> > > > >> part of
> > > > >> > Kafka, while higher-level tools such as streaming SQL and
> > > integrations
> > > > >> with
> > > > >> > deployment frameworks remain in a separate project? In other
> > words,
> > > > >> Kafka
> > > > >> > would absorb the proven, stable core of Samza, which would
> become
> > > the
> > > > >> > "third Kafka client" mentioned early in this thread. The Samza
> > > project
> > > > >> > would then target that third Kafka client as its base API, and
> the
> > > > >> project
> > > > >> > would be freed up to explore more experimental new horizons.
> > > > >> >
> > > > >> > Martin
> > > > >> >
> > > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
> > > > >> >
> > > > >> > > Hey Martin,
> > > > >> > >
> > > > >> > > For the YARN/Mesos/etc decoupling I actually don't think it
> ties
> > > our
> > > > >> > hands
> > > > >> > > at all, all it does is refactor things. The division of
> > > > >> responsibility is
> > > > >> > > that Samza core is responsible for task lifecycle, state, and
> > > > >> partition
> > > > >> > > management (using the Kafka co-ordinator) but it is NOT
> > > responsible
> > > > >> for
> > > > >> > > packaging, configuration deployment or execution of processes.
> > The
> > > > >> > problem
> > > > >> > > of packaging and starting these processes is
> > > > >> > > framework/environment-specific. This leaves individual
> > frameworks
> > > to
> > > > >> be
> > > > >> > as
> > > > >> > > fancy or vanilla as they like. So you can get simple stateless
> > > > >> support in
> > > > >> > > YARN, Mesos, etc using their off-the-shelf app framework
> > (Slider,
> > > > >> > Marathon,
> > > > >> > > etc). These are well known by people and have nice UIs and a
> lot
> > > of
> > > > >> > > flexibility. I don't think they have node affinity as a built
> in
> > > > >> option
> > > > >> > > (though I could be wrong). So if we want that we can either
> wait
> > > for
> > > > >> them
> > > > >> > > to add it or do a custom framework to add that feature (as
> now).
> > > > >> > Obviously
> > > > >> > > if you manage things with old-school ops tools
> (puppet/chef/etc)
> > > you
> > > > >> get
> > > > >> > > locality easily. The nice thing, though, is that all the samza
> > > > >> "business
> > > > >> > > logic" around partition management and fault tolerance is in
> > Samza
> > > > >> core
> > > > >> > so
> > > > >> > > it is shared across frameworks and the framework specific bit
> is
> > > > just
> > > > >> > > whether it is smart enough to try to get the same host when a
> > job
> > > is
> > > > >> > > restarted.
> > > > >> > >
> > > > >> > > With respect to the Kafka-alignment, yeah I think the goal
> would
> > > be
> > > > >> (a)
> > > > >> > > actually get better alignment in user experience, and (b)
> > express
> > > > >> this in
> > > > >> > > the naming and project branding. Specifically:
> > > > >> > > 1. Website/docs, it would be nice for the "transformation" api
> > to
> > > be
> > > > >> > > discoverable in the main Kafka docs--i.e. be able to explain
> > when
> > > to
> > > > >> use
> > > > >> > > the consumer and when to use the stream processing
> functionality
> > > and
> > > > >> lead
> > > > >> > > people into that experience.
> > > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever)
> that
> > > has
> > > > >> both
> > > > >> > > Kafka and the stream processing part and they actually work
> > > > together.
> > > > >> > > 3. Unify the programming experience so the client and Samza
> api
> > > > share
> > > > >> > > config/monitoring/naming/packaging/etc.
> > > > >> > >
> > > > >> > > I think sub-projects keep separate committers and can have a
> > > > separate
> > > > >> > repo,
> > > > >> > > but I'm actually not really sure (I can't find a definition
> of a
> > > > >> > subproject
> > > > >> > > in Apache).
> > > > >> > >
> > > > >> > > Basically at a high-level you want the experience to "feel"
> > like a
> > > > >> single
> > > > >> > > system, not to relatively independent things that are kind of
> > > > >> awkwardly
> > > > >> > > glued together.
> > > > >> > >
> > > > >> > > I think if we did that they having naming or branding like
> > "kafka
> > > > >> > > streaming" or "kafka streams" or something like that would
> > > actually
> > > > >> do a
> > > > >> > > good job of conveying what it is. I do that this would help
> > > adoption
> > > > >> > quite
> > > > >> > > a lot as it would correctly convey that using Kafka Streaming
> > with
> > > > >> Kafka
> > > > >> > is
> > > > >> > > a fairly seamless experience and Kafka is pretty heavily
> adopted
> > > at
> > > > >> this
> > > > >> > > point.
> > > > >> > >
> > > > >> > > Fwiw we actually considered this model originally when open
> > > sourcing
> > > > >> > Samza,
> > > > >> > > however at that time Kafka was relatively unknown and we
> decided
> > > not
> > > > >> to
> > > > >> > do
> > > > >> > > it since we felt it would be limiting. From my point of view
> the
> > > > three
> > > > >> > > things have changed (1) Kafka is now really heavily used for
> > > stream
> > > > >> > > processing, (2) we learned that abstracting out the stream
> well
> > is
> > > > >> > > basically impossible, (3) we learned it is really hard to keep
> > the
> > > > two
> > > > >> > > things feeling like a single product.
> > > > >> > >
> > > > >> > > -Jay
> > > > >> > >
> > > > >> > >
> > > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> > > > >> martin@kleppmann.com>
> > > > >> > > wrote:
> > > > >> > >
> > > > >> > >> Hi all,
> > > > >> > >>
> > > > >> > >> Lots of good thoughts here.
> > > > >> > >>
> > > > >> > >> I agree with the general philosophy of tying Samza more
> firmly
> > to
> > > > >> Kafka.
> > > > >> > >> After I spent a while looking at integrating other message
> > > brokers
> > > > >> (e.g.
> > > > >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
> > > > >> > SystemConsumer
> > > > >> > >> tacitly assumes a model so much like Kafka's that pretty much
> > > > nobody
> > > > >> but
> > > > >> > >> Kafka actually implements it. (Databus is perhaps an
> exception,
> > > but
> > > > >> it
> > > > >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza
> > fully
> > > > >> > dependent
> > > > >> > >> on Kafka acknowledges that the system-independence was never
> as
> > > > real
> > > > >> as
> > > > >> > we
> > > > >> > >> perhaps made it out to be. The gains of code reuse are real.
> > > > >> > >>
> > > > >> > >> The idea of decoupling Samza from YARN has also always been
> > > > >> appealing to
> > > > >> > >> me, for various reasons already mentioned in this thread.
> > > Although
> > > > >> > making
> > > > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems
> > > > >> laudable,
> > > > >> > I am
> > > > >> > >> a little concerned that it will restrict us to a lowest
> common
> > > > >> > denominator.
> > > > >> > >> For example, would host affinity (SAMZA-617) still be
> possible?
> > > For
> > > > >> jobs
> > > > >> > >> with large amounts of state, I think SAMZA-617 would be a big
> > > boon,
> > > > >> > since
> > > > >> > >> restoring state off the changelog on every single restart is
> > > > painful,
> > > > >> > due
> > > > >> > >> to long recovery times. It would be a shame if the decoupling
> > > from
> > > > >> YARN
> > > > >> > >> made host affinity impossible.
> > > > >> > >>
> > > > >> > >> Jay, a question about the proposed API for instantiating a
> job
> > in
> > > > >> code
> > > > >> > >> (rather than a properties file): when submitting a job to a
> > > > cluster,
> > > > >> is
> > > > >> > the
> > > > >> > >> idea that the instantiation code runs on a client somewhere,
> > > which
> > > > >> then
> > > > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does
> > that
> > > > >> code
> > > > >> > run
> > > > >> > >> on each container that is part of the job (in which case, how
> > > does
> > > > >> the
> > > > >> > job
> > > > >> > >> submission to the cluster work)?
> > > > >> > >>
> > > > >> > >> I agree with Garry that it doesn't feel right to make a 1.0
> > > release
> > > > >> > with a
> > > > >> > >> plan for it to be immediately obsolete. So if this is going
> to
> > > > >> happen, I
> > > > >> > >> think it would be more honest to stick with 0.* version
> numbers
> > > > until
> > > > >> > the
> > > > >> > >> library-ified Samza has been implemented, is stable and
> widely
> > > > used.
> > > > >> > >>
> > > > >> > >> Should the new Samza be a subproject of Kafka? There is
> > precedent
> > > > for
> > > > >> > >> tight coupling between different Apache projects (e.g.
> Curator
> > > and
> > > > >> > >> Zookeeper, or Slider and YARN), so I think remaining separate
> > > would
> > > > >> be
> > > > >> > ok.
> > > > >> > >> Even if Samza is fully dependent on Kafka, there is enough
> > > > substance
> > > > >> in
> > > > >> > >> Samza that it warrants being a separate project. An argument
> in
> > > > >> favour
> > > > >> > of
> > > > >> > >> merging would be if we think Kafka has a much stronger "brand
> > > > >> presence"
> > > > >> > >> than Samza; I'm ambivalent on that one. If the Kafka project
> is
> > > > >> willing
> > > > >> > to
> > > > >> > >> endorse Samza as the "official" way of doing stateful stream
> > > > >> > >> transformations, that would probably have much the same
> effect
> > as
> > > > >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike.
> > Close
> > > > >> > >> collaboration between the two projects will be needed in any
> > > case.
> > > > >> > >>
> > > > >> > >> From a project management perspective, I guess the "new
> Samza"
> > > > would
> > > > >> > have
> > > > >> > >> to be developed on a branch alongside ongoing maintenance of
> > the
> > > > >> current
> > > > >> > >> line of development? I think it would be important to
> continue
> > > > >> > supporting
> > > > >> > >> existing users, and provide a graceful migration path to the
> > new
> > > > >> > version.
> > > > >> > >> Leaving the current versions unsupported and forcing people
> to
> > > > >> rewrite
> > > > >> > >> their jobs would send a bad signal.
> > > > >> > >>
> > > > >> > >> Best,
> > > > >> > >> Martin
> > > > >> > >>
> > > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
> > > > >> > >>
> > > > >> > >>> Hey Garry,
> > > > >> > >>>
> > > > >> > >>> Yeah that's super frustrating. I'd be happy to chat more
> about
> > > > this
> > > > >> if
> > > > >> > >>> you'd be interested. I think Chris and I started with the
> idea
> > > of
> > > > >> "what
> > > > >> > >>> would it take to make Samza a kick-ass ingestion tool" but
> > > > >> ultimately
> > > > >> > we
> > > > >> > >>> kind of came around to the idea that ingestion and
> > > transformation
> > > > >> had
> > > > >> > >>> pretty different needs and coupling the two made things
> hard.
> > > > >> > >>>
> > > > >> > >>> For what it's worth I think copycat (KIP-26) actually will
> do
> > > what
> > > > >> you
> > > > >> > >> are
> > > > >> > >>> looking for.
> > > > >> > >>>
> > > > >> > >>> With regard to your point about slider, I don't necessarily
> > > > >> disagree.
> > > > >> > >> But I
> > > > >> > >>> think getting good YARN support is quite doable and I think
> we
> > > can
> > > > >> make
> > > > >> > >>> that work well. I think the issue this proposal solves is
> that
> > > > >> > >> technically
> > > > >> > >>> it is pretty hard to support multiple cluster management
> > systems
> > > > the
> > > > >> > way
> > > > >> > >>> things are now, you need to write an "app master" or
> > "framework"
> > > > for
> > > > >> > each
> > > > >> > >>> and they are all a little different so testing is really
> hard.
> > > In
> > > > >> the
> > > > >> > >>> absence of this we have been stuck with just YARN which has
> > > > >> fantastic
> > > > >> > >>> penetration in the Hadoopy part of the org, but zero
> > penetration
> > > > >> > >> elsewhere.
> > > > >> > >>> Given the huge amount of work being put in to slider,
> > marathon,
> > > > aws
> > > > >> > >>> tooling, not to mention the umpteen related packaging
> > > technologies
> > > > >> > people
> > > > >> > >>> want to use (Docker, Kubernetes, various cloud-specific
> deploy
> > > > >> tools,
> > > > >> > >> etc)
> > > > >> > >>> I really think it is important to get this right.
> > > > >> > >>>
> > > > >> > >>> -Jay
> > > > >> > >>>
> > > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > > >> > >>> g.turkington@improvedigital.com> wrote:
> > > > >> > >>>
> > > > >> > >>>> Hi all,
> > > > >> > >>>>
> > > > >> > >>>> I think the question below re does Samza become a
> sub-project
> > > of
> > > > >> Kafka
> > > > >> > >>>> highlights the broader point around migration. Chris
> mentions
> > > > >> Samza's
> > > > >> > >>>> maturity is heading towards a v1 release but I'm not sure
> it
> > > > feels
> > > > >> > >> right to
> > > > >> > >>>> launch a v1 then immediately plan to deprecate most of it.
> > > > >> > >>>>
> > > > >> > >>>> From a selfish perspective I have some guys who have
> started
> > > > >> working
> > > > >> > >> with
> > > > >> > >>>> Samza and building some new consumers/producers was next
> up.
> > > > Sounds
> > > > >> > like
> > > > >> > >>>> that is absolutely not the direction to go. I need to look
> > into
> > > > the
> > > > >> > KIP
> > > > >> > >> in
> > > > >> > >>>> more detail but for me the attractiveness of adding new
> Samza
> > > > >> > >>>> consumer/producers -- even if yes all they were doing was
> > > really
> > > > >> > getting
> > > > >> > >>>> data into and out of Kafka --  was to avoid  having to
> worry
> > > > about
> > > > >> the
> > > > >> > >>>> lifecycle management of external clients. If there is a
> > generic
> > > > >> Kafka
> > > > >> > >>>> ingress/egress layer that I can plug a new connector into
> and
> > > > have
> > > > >> a
> > > > >> > >> lot of
> > > > >> > >>>> the heavy lifting re scale and reliability done for me then
> > it
> > > > >> gives
> > > > >> > me
> > > > >> > >> all
> > > > >> > >>>> the pushing new consumers/producers would. If not then it
> > > > >> complicates
> > > > >> > my
> > > > >> > >>>> operational deployments.
> > > > >> > >>>>
> > > > >> > >>>> Which is similar to my other question with the proposal --
> if
> > > we
> > > > >> > build a
> > > > >> > >>>> fully available/stand-alone Samza plus the requisite shims
> to
> > > > >> > integrate
> > > > >> > >>>> with Slider etc I suspect the former may be a lot more work
> > > than
> > > > we
> > > > >> > >> think.
> > > > >> > >>>> We may make it much easier for a newcomer to get something
> > > > running
> > > > >> but
> > > > >> > >>>> having them step up and get a reliable production
> deployment
> > > may
> > > > >> still
> > > > >> > >>>> dominate mailing list  traffic, if for different reasons
> than
> > > > >> today.
> > > > >> > >>>>
> > > > >> > >>>> Don't get me wrong -- I'm comfortable with making the Samza
> > > > >> dependency
> > > > >> > >> on
> > > > >> > >>>> Kafka much more explicit and I absolutely see the benefits
> > in
> > > > the
> > > > >> > >>>> reduction of duplication and clashing
> > > terminologies/abstractions
> > > > >> that
> > > > >> > >>>> Chris/Jay describe. Samza as a library would likely be a
> very
> > > > nice
> > > > >> > tool
> > > > >> > >> to
> > > > >> > >>>> add to the Kafka ecosystem. I just have the concerns above
> re
> > > the
> > > > >> > >>>> operational side.
> > > > >> > >>>>
> > > > >> > >>>> Garry
> > > > >> > >>>>
> > > > >> > >>>> -----Original Message-----
> > > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:
> gdfm@apache.org
> > ]
> > > > >> > >>>> Sent: 02 July 2015 12:56
> > > > >> > >>>> To: dev@samza.apache.org
> > > > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> > > > >> > >>>>
> > > > >> > >>>> Very interesting thoughts.
> > > > >> > >>>> From outside, I have always perceived Samza as a computing
> > > layer
> > > > >> over
> > > > >> > >>>> Kafka.
> > > > >> > >>>>
> > > > >> > >>>> The question, maybe a bit provocative, is "should Samza be
> a
> > > > >> > sub-project
> > > > >> > >>>> of Kafka then?"
> > > > >> > >>>> Or does it make sense to keep it as a separate project
> with a
> > > > >> separate
> > > > >> > >>>> governance?
> > > > >> > >>>>
> > > > >> > >>>> Cheers,
> > > > >> > >>>>
> > > > >> > >>>> --
> > > > >> > >>>> Gianmarco
> > > > >> > >>>>
> > > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> > > wrote:
> > > > >> > >>>>
> > > > >> > >>>>> Overall, I agree to couple with Kafka more tightly.
> Because
> > > > Samza
> > > > >> de
> > > > >> > >>>>> facto is based on Kafka, and it should leverage what Kafka
> > > has.
> > > > At
> > > > >> > the
> > > > >> > >>>>> same time, Kafka does not need to reinvent what Samza
> > already
> > > > >> has. I
> > > > >> > >>>>> also like the idea of separating the ingestion and
> > > > transformation.
> > > > >> > >>>>>
> > > > >> > >>>>> But it is a little difficult for me to image how the Samza
> > > will
> > > > >> look
> > > > >> > >>>> like.
> > > > >> > >>>>> And I feel Chris and Jay have a little difference in terms
> > of
> > > > how
> > > > >> > >>>>> Samza should look like.
> > > > >> > >>>>>
> > > > >> > >>>>> *** Will it look like what Jay's code shows (A client of
> > > Kakfa)
> > > > ?
> > > > >> And
> > > > >> > >>>>> user's application code calls this client?
> > > > >> > >>>>>
> > > > >> > >>>>> 1. If we make Samza be a library of Kafka (like what the
> > code
> > > > >> shows),
> > > > >> > >>>>> how do we implement auto-balance and fault-tolerance? Are
> > they
> > > > >> taken
> > > > >> > >>>>> care by the Kafka broker or other mechanism, such as
> "Samza
> > > > >> worker"
> > > > >> > >>>>> (just make up the name) ?
> > > > >> > >>>>>
> > > > >> > >>>>> 2. What about other features, such as auto-scaling, shared
> > > > state,
> > > > >> > >>>>> monitoring?
> > > > >> > >>>>>
> > > > >> > >>>>>
> > > > >> > >>>>> *** If we have Samza standalone, (is this what Chris
> > > suggests?)
> > > > >> > >>>>>
> > > > >> > >>>>> 1. we still need to ingest data from Kakfa and produce to
> > it.
> > > > >> Then it
> > > > >> > >>>>> becomes the same as what Samza looks like now, except it
> > does
> > > > not
> > > > >> > rely
> > > > >> > >>>>> on Yarn anymore.
> > > > >> > >>>>>
> > > > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's
> metrics,
> > > > logs,
> > > > >> > >>>>> etc? Use Kafka code as the dependency?
> > > > >> > >>>>>
> > > > >> > >>>>>
> > > > >> > >>>>> Thanks,
> > > > >> > >>>>>
> > > > >> > >>>>> Fang, Yan
> > > > >> > >>>>> yanfang724@gmail.com
> > > > >> > >>>>>
> > > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > > > wangguoz@gmail.com
> > > > >> >
> > > > >> > >>>> wrote:
> > > > >> > >>>>>
> > > > >> > >>>>>> Read through the code example and it looks good to me. A
> > few
> > > > >> > >>>>>> thoughts regarding deployment:
> > > > >> > >>>>>>
> > > > >> > >>>>>> Today Samza deploys as executable runnable like:
> > > > >> > >>>>>>
> > > > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> > > > >> > >>>> --config-path=file://...
> > > > >> > >>>>>>
> > > > >> > >>>>>> And this proposal advocate for deploying Samza more as
> > > embedded
> > > > >> > >>>>>> libraries in user application code (ignoring the
> > terminology
> > > > >> since
> > > > >> > >>>>>> it is not the
> > > > >> > >>>>> same
> > > > >> > >>>>>> as the prototype code):
> > > > >> > >>>>>>
> > > > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread
> thread
> > =
> > > > new
> > > > >> > >>>>>> Thread(task); thread.start();
> > > > >> > >>>>>>
> > > > >> > >>>>>> I think both of these deployment modes are important for
> > > > >> different
> > > > >> > >>>>>> types
> > > > >> > >>>>> of
> > > > >> > >>>>>> users. That said, I think making Samza purely standalone
> is
> > > > still
> > > > >> > >>>>>> sufficient for either runnable or library modes.
> > > > >> > >>>>>>
> > > > >> > >>>>>> Guozhang
> > > > >> > >>>>>>
> > > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> > > jay@confluent.io>
> > > > >> > wrote:
> > > > >> > >>>>>>
> > > > >> > >>>>>>> Looks like gmail mangled the code example, it was
> supposed
> > > to
> > > > >> look
> > > > >> > >>>>>>> like
> > > > >> > >>>>>>> this:
> > > > >> > >>>>>>>
> > > > >> > >>>>>>> Properties props = new Properties();
> > > > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > > >> StreamingConfig
> > > > >> > >>>>>>> config = new StreamingConfig(props);
> > > > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> > > > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> > > > >> > >>>>>>> config.serialization(new StringSerializer(), new
> > > > >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
> > > > >> > >>>>>>> KafkaStreaming(config); container.run();
> > > > >> > >>>>>>>
> > > > >> > >>>>>>> -Jay
> > > > >> > >>>>>>>
> > > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> > > jay@confluent.io
> > > > >
> > > > >> > >>>> wrote:
> > > > >> > >>>>>>>
> > > > >> > >>>>>>>> Hey guys,
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> This came out of some conversations Chris and I were
> > having
> > > > >> > >>>>>>>> around
> > > > >> > >>>>>>> whether
> > > > >> > >>>>>>>> it would make sense to use Samza as a kind of data
> > > ingestion
> > > > >> > >>>>> framework
> > > > >> > >>>>>>> for
> > > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This
> > > kind
> > > > of
> > > > >> > >>>>>> combined
> > > > >> > >>>>>>>> with complaints around config and YARN and the
> discussion
> > > > >> around
> > > > >> > >>>>>>>> how
> > > > >> > >>>>> to
> > > > >> > >>>>>>>> best do a standalone mode.
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> So the thought experiment was, given that Samza was
> > > basically
> > > > >> > >>>>>>>> already totally Kafka specific, what if you just
> embraced
> > > > that
> > > > >> > >>>>>>>> and turned it
> > > > >> > >>>>>> into
> > > > >> > >>>>>>>> something less like a heavyweight framework and more
> > like a
> > > > >> > >>>>>>>> third
> > > > >> > >>>>> Kafka
> > > > >> > >>>>>>>> client--a kind of "producing consumer" with state
> > > management
> > > > >> > >>>>>> facilities.
> > > > >> > >>>>>>>> Basically a library. Instead of a complex stream
> > processing
> > > > >> > >>>>>>>> framework
> > > > >> > >>>>>>> this
> > > > >> > >>>>>>>> would actually be a very simple thing, not much more
> > > > >> complicated
> > > > >> > >>>>>>>> to
> > > > >> > >>>>> use
> > > > >> > >>>>>>> or
> > > > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we thought
> > > about
> > > > >> it
> > > > >> > >>>>>>>> a
> > > > >> > >>>>> lot
> > > > >> > >>>>>> of
> > > > >> > >>>>>>>> what Samza (and the other stream processing systems
> were
> > > > doing)
> > > > >> > >>>>> seemed
> > > > >> > >>>>>>> like
> > > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> Of course you need to ingest/output data to and from
> the
> > > > stream
> > > > >> > >>>>>>>> processing. But when we actually looked into how that
> > would
> > > > >> > >>>>>>>> work,
> > > > >> > >>>>> Samza
> > > > >> > >>>>>>>> isn't really an ideal data ingestion framework for a
> > bunch
> > > of
> > > > >> > >>>>> reasons.
> > > > >> > >>>>>> To
> > > > >> > >>>>>>>> really do that right you need a pretty different
> internal
> > > > data
> > > > >> > >>>>>>>> model
> > > > >> > >>>>>> and
> > > > >> > >>>>>>>> set of apis. So what if you split them and had an api
> for
> > > > Kafka
> > > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api
> > for
> > > > >> Kafka
> > > > >> > >>>>>>>> transformation (Samza).
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> This would also allow really embracing the same
> > terminology
> > > > and
> > > > >> > >>>>>>>> conventions. One complaint about the current state is
> > that
> > > > the
> > > > >> > >>>>>>>> two
> > > > >> > >>>>>>> systems
> > > > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs
> > > "topic"
> > > > >> and
> > > > >> > >>>>>>> different
> > > > >> > >>>>>>>> config and monitoring systems means you kind of have to
> > > learn
> > > > >> > >>>>>>>> Kafka's
> > > > >> > >>>>>>> way,
> > > > >> > >>>>>>>> then learn Samza's slightly different way, then kind of
> > > > >> > >>>>>>>> understand
> > > > >> > >>>>> how
> > > > >> > >>>>>>> they
> > > > >> > >>>>>>>> map to each other, which having walked a few people
> > through
> > > > >> this
> > > > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> Since I have been spending a lot of time on airplanes I
> > > > hacked
> > > > >> > >>>>>>>> up an ernest but still somewhat incomplete prototype of
> > > what
> > > > >> > >>>>>>>> this would
> > > > >> > >>>>> look
> > > > >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka as
> > it
> > > > >> > >>>>>>>> required a
> > > > >> > >>>>>> few
> > > > >> > >>>>>>>> changes to the new consumer. Here is the code:
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>
> > > > >> > >>>>>>
> > > > >> > >>>>>
> > > > >> >
> > > >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > >> > >>>>> /apache/kafka/clients/streaming
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> For the purpose of the prototype I just liberally
> renamed
> > > > >> > >>>>>>>> everything
> > > > >> > >>>>> to
> > > > >> > >>>>>>>> try to align it with Kafka with no regard for
> > > compatibility.
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> To use this would be something like this:
> > > > >> > >>>>>>>> Properties props = new Properties();
> > > > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > > >> > >>>>>>>> StreamingConfig config = new
> > > > >> > >>>>> StreamingConfig(props);
> > > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > > >> > >>>>>>>> "test-topic-2");
> > > > >> config.processor(ExampleStreamProcessor.class);
> > > > >> > >>>>>>> config.serialization(new
> > > > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
> > > KafkaStreaming
> > > > >> > >>>>>> container =
> > > > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> > > > StreamProcessor
> > > > >> > >>>>>>>> is basically StreamTask.
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> So rather than putting all the class names in a file
> and
> > > then
> > > > >> > >>>>>>>> having
> > > > >> > >>>>>> the
> > > > >> > >>>>>>>> job assembled by reflection, you just instantiate the
> > > > container
> > > > >> > >>>>>>>> programmatically. Work is balanced over however many
> > > > instances
> > > > >> > >>>>>>>> of
> > > > >> > >>>>> this
> > > > >> > >>>>>>> are
> > > > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new tasks
> > are
> > > > >> added
> > > > >> > >>>>>>>> to
> > > > >> > >>>>> the
> > > > >> > >>>>>>>> existing containers without shutting them down).
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> We would provide some glue for running this stuff in
> YARN
> > > via
> > > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their
> > > tools
> > > > >> > >>>>>>>> but from the
> > > > >> > >>>>>> point
> > > > >> > >>>>>>> of
> > > > >> > >>>>>>>> view of these frameworks these stream processing jobs
> are
> > > > just
> > > > >> > >>>>>> stateless
> > > > >> > >>>>>>>> services that can come and go and expand and contract
> at
> > > > will.
> > > > >> > >>>>>>>> There
> > > > >> > >>>>> is
> > > > >> > >>>>>>> no
> > > > >> > >>>>>>>> more custom scheduler.
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> Here are some relevant details:
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get larger
> > if
> > > we
> > > > >> > >>>>>>>>  productionized but not vastly larger. We really do
> get a
> > > ton
> > > > >> > >>>>>>>> of
> > > > >> > >>>>>>> leverage
> > > > >> > >>>>>>>>  out of Kafka.
> > > > >> > >>>>>>>>  2. Partition management is fully delegated to the new
> > > > >> consumer.
> > > > >> > >>>>> This
> > > > >> > >>>>>>>>  is nice since now any partition management strategy
> > > > available
> > > > >> > >>>>>>>> to
> > > > >> > >>>>>> Kafka
> > > > >> > >>>>>>>>  consumer is also available to Samza (and vice versa)
> and
> > > > with
> > > > >> > >>>>>>>> the
> > > > >> > >>>>>>> exact
> > > > >> > >>>>>>>>  same configs.
> > > > >> > >>>>>>>>  3. It supports state as well as state reuse
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> -Jay
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > > >> > >>>>>> criccomini@apache.org>
> > > > >> > >>>>>>>> wrote:
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>>> Hey all,
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> I have had some discussions with Samza engineers at
> > > LinkedIn
> > > > >> > >>>>>>>>> and
> > > > >> > >>>>>>> Confluent
> > > > >> > >>>>>>>>> and we came up with a few observations and would like
> to
> > > > >> > >>>>>>>>> propose
> > > > >> > >>>>> some
> > > > >> > >>>>>>>>> changes.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> We've observed some things that I want to call out
> about
> > > > >> > >>>>>>>>> Samza's
> > > > >> > >>>>>> design,
> > > > >> > >>>>>>>>> and I'd like to propose some changes.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
> > > > >> > >>>>>>>>> * Samza is too pluggable.
> > > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> > > consumer
> > > > >> > >>>>>>>>> APIs
> > > > >> > >>>>> are
> > > > >> > >>>>>>>>> trying to solve a lot of the same problems.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> All three of these issues are related, but I'll
> address
> > > them
> > > > >> in
> > > > >> > >>>>> order.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Deployment
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic
> > deployment
> > > > >> > >>>>>>>>> scheduler
> > > > >> > >>>>>> such
> > > > >> > >>>>>>>>> as
> > > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we
> bet
> > > that
> > > > >> > >>>>>>>>> there
> > > > >> > >>>>>> would
> > > > >> > >>>>>>>>> be
> > > > >> > >>>>>>>>> one or two winners in this area, and we could support
> > > them,
> > > > >> and
> > > > >> > >>>>>>>>> the
> > > > >> > >>>>>> rest
> > > > >> > >>>>>>>>> would go away. In reality, there are many variations.
> > > > >> > >>>>>>>>> Furthermore,
> > > > >> > >>>>>> many
> > > > >> > >>>>>>>>> people still prefer to just start their processors
> like
> > > > normal
> > > > >> > >>>>>>>>> Java processes, and use traditional deployment scripts
> > > such
> > > > as
> > > > >> > >>>>>>>>> Fabric,
> > > > >> > >>>>>> Chef,
> > > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users
> makes
> > > the
> > > > >> > >>>>>>>>> Samza start-up process really painful for first time
> > > users.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit of
> a
> > > > >> > >>>>>>>>> mis-fire
> > > > >> > >>>>>> because
> > > > >> > >>>>>>>>> of
> > > > >> > >>>>>>>>> a fundamental misunderstanding between the nature of
> > batch
> > > > >> jobs
> > > > >> > >>>>>>>>> and
> > > > >> > >>>>>>> stream
> > > > >> > >>>>>>>>> processing jobs. Early on, we made conscious effort to
> > > favor
> > > > >> > >>>>>>>>> the
> > > > >> > >>>>>> Hadoop
> > > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked and
> > was
> > > > well
> > > > >> > >>>>>>> understood.
> > > > >> > >>>>>>>>> One thing that we missed was that batch jobs have a
> > > definite
> > > > >> > >>>>>> beginning,
> > > > >> > >>>>>>>>> and
> > > > >> > >>>>>>>>> end, and stream processing jobs don't (usually). This
> > > leads
> > > > to
> > > > >> > >>>>>>>>> a
> > > > >> > >>>>> much
> > > > >> > >>>>>>>>> simpler scheduling problem for stream processors. You
> > > > >> basically
> > > > >> > >>>>>>>>> just
> > > > >> > >>>>>>> need
> > > > >> > >>>>>>>>> to find a place to start the processor, and start it.
> > The
> > > > way
> > > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a
> > cluster
> > > > >> > >>>>>>>>> being "full". We always
> > > > >> > >>>>>> add
> > > > >> > >>>>>>>>> more machines. The problem with coupling Samza with a
> > > > >> scheduler
> > > > >> > >>>>>>>>> is
> > > > >> > >>>>>> that
> > > > >> > >>>>>>>>> Samza (as a framework) now has to handle deployment.
> > This
> > > > >> pulls
> > > > >> > >>>>>>>>> in a
> > > > >> > >>>>>>> bunch
> > > > >> > >>>>>>>>> of things such as configuration distribution (config
> > > > stream),
> > > > >> > >>>>>>>>> shell
> > > > >> > >>>>>>> scrips
> > > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> > > stuff),
> > > > >> etc.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Another reason for requiring dynamic deployment was to
> > > > support
> > > > >> > >>>>>>>>> data locality. If you want to have locality, you need
> to
> > > put
> > > > >> > >>>>>>>>> your
> > > > >> > >>>>>> processors
> > > > >> > >>>>>>>>> close to the data they're processing. Upon further
> > > > >> > >>>>>>>>> investigation,
> > > > >> > >>>>>>> though,
> > > > >> > >>>>>>>>> this feature is not that beneficial. There is some
> good
> > > > >> > >>>>>>>>> discussion
> > > > >> > >>>>>> about
> > > > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
> > > > >> > >>>>>>>>> Map/Reduce
> > > > >> > >>>>>> path,
> > > > >> > >>>>>>>>> but
> > > > >> > >>>>>>>>> there are some fundamental differences between HDFS
> and
> > > > Kafka.
> > > > >> > >>>>>>>>> HDFS
> > > > >> > >>>>>> has
> > > > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to less
> > > > >> > >>>>>>>>> optimization potential with stream processors on top
> of
> > > > Kafka.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't
> > have
> > > > any
> > > > >> > >>>>>>>>> built
> > > > >> > >>>>> in
> > > > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the
> > dynamic
> > > > >> > >>>>>>>>> deployment scheduling system to handle restarts when a
> > > > >> > >>>>>>>>> processor dies. This has
> > > > >> > >>>>>>> made
> > > > >> > >>>>>>>>> it very difficult to write a standalone Samza
> container
> > > > >> > >>>> (SAMZA-516).
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Pluggability
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> In some cases pluggability is good, but I think that
> > we've
> > > > >> gone
> > > > >> > >>>>>>>>> too
> > > > >> > >>>>>> far
> > > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> * Pluggable config.
> > > > >> > >>>>>>>>> * Pluggable metrics.
> > > > >> > >>>>>>>>> * Pluggable deployment systems.
> > > > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> > > > SystemProducer,
> > > > >> > >>>> etc).
> > > > >> > >>>>>>>>> * Pluggable serdes.
> > > > >> > >>>>>>>>> * Pluggable storage engines.
> > > > >> > >>>>>>>>> * Pluggable strategies for just about every component
> > > > >> > >>>>> (MessageChooser,
> > > > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> There's probably more that I've forgotten, as well.
> Some
> > > of
> > > > >> > >>>>>>>>> these
> > > > >> > >>>>> are
> > > > >> > >>>>>>>>> useful, but some have proven not to be. This all comes
> > at
> > > a
> > > > >> cost:
> > > > >> > >>>>>>>>> complexity. This complexity is making it harder for
> our
> > > > users
> > > > >> > >>>>>>>>> to
> > > > >> > >>>>> pick
> > > > >> > >>>>>> up
> > > > >> > >>>>>>>>> and use Samza out of the box. It also makes it
> difficult
> > > for
> > > > >> > >>>>>>>>> Samza developers to reason about what the
> > characteristics
> > > of
> > > > >> > >>>>>>>>> the container (since the characteristics change
> > depending
> > > on
> > > > >> > >>>>>>>>> which plugins are use).
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> The issues with pluggability are most visible in the
> > > System
> > > > >> APIs.
> > > > >> > >>>>> What
> > > > >> > >>>>>>>>> Samza really requires to be functional is Kafka as its
> > > > >> > >>>>>>>>> transport
> > > > >> > >>>>>> layer.
> > > > >> > >>>>>>>>> But
> > > > >> > >>>>>>>>> we've conflated two unrelated use cases into one API:
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> The current System API supports both of these use
> cases.
> > > The
> > > > >> > >>>>>>>>> problem
> > > > >> > >>>>>> is,
> > > > >> > >>>>>>>>> we
> > > > >> > >>>>>>>>> actually want different features for each use case. By
> > > > >> papering
> > > > >> > >>>>>>>>> over
> > > > >> > >>>>>>> these
> > > > >> > >>>>>>>>> two use cases, and providing a single API, we've
> > > introduced
> > > > a
> > > > >> > >>>>>>>>> ton of
> > > > >> > >>>>>>> leaky
> > > > >> > >>>>>>>>> abstractions.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> For example, what we'd really like in (2) is to have
> > > > >> > >>>>>>>>> monotonically increasing longs for offsets (like
> Kafka).
> > > > This
> > > > >> > >>>>>>>>> would be at odds
> > > > >> > >>>>> with
> > > > >> > >>>>>>> (1),
> > > > >> > >>>>>>>>> though, since different systems have different
> > > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > > >> > >>>>>>>>> There was discussion both on the mailing list and the
> > SQL
> > > > >> JIRAs
> > > > >> > >>>>> about
> > > > >> > >>>>>>> the
> > > > >> > >>>>>>>>> need for this.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> The same thing holds true for replayability. Kafka
> > allows
> > > us
> > > > >> to
> > > > >> > >>>>> rewind
> > > > >> > >>>>>>>>> when
> > > > >> > >>>>>>>>> we have a failure. Many other systems don't. In some
> > > cases,
> > > > >> > >>>>>>>>> systems
> > > > >> > >>>>>>> return
> > > > >> > >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> > > > because
> > > > >> > >>>>>>>>> they
> > > > >> > >>>>>> have
> > > > >> > >>>>>>> no
> > > > >> > >>>>>>>>> offsets.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Partitioning is another example. Kafka supports
> > > > partitioning,
> > > > >> > >>>>>>>>> but
> > > > >> > >>>>> many
> > > > >> > >>>>>>>>> systems don't. We model this by having a single
> > partition
> > > > for
> > > > >> > >>>>>>>>> those systems. Still, other systems model partitioning
> > > > >> > >>>> differently (e.g.
> > > > >> > >>>>>>>>> Kinesis).
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating
> > streams
> > > > in
> > > > >> a
> > > > >> > >>>>>>>>> system-agnostic way is almost impossible. As is
> modeling
> > > > >> > >>>>>>>>> metadata
> > > > >> > >>>>> for
> > > > >> > >>>>>>> the
> > > > >> > >>>>>>>>> system (replication factor, partitions, location,
> etc).
> > > The
> > > > >> > >>>>>>>>> list
> > > > >> > >>>>> goes
> > > > >> > >>>>>>> on.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Duplicate work
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's
> > consumer
> > > > and
> > > > >> > >>>>> producer
> > > > >> > >>>>>>>>> APIs
> > > > >> > >>>>>>>>> had a relatively weak feature set. On the
> consumer-side,
> > > you
> > > > >> > >>>>>>>>> had two
> > > > >> > >>>>>>>>> options: use the high level consumer, or the simple
> > > > consumer.
> > > > >> > >>>>>>>>> The
> > > > >> > >>>>>>> problem
> > > > >> > >>>>>>>>> with the high-level consumer was that it controlled
> your
> > > > >> > >>>>>>>>> offsets, partition assignments, and the order in which
> > you
> > > > >> > >>>>>>>>> received messages. The
> > > > >> > >>>>> problem
> > > > >> > >>>>>>>>> with
> > > > >> > >>>>>>>>> the simple consumer is that it's not simple. It's
> basic.
> > > You
> > > > >> > >>>>>>>>> end up
> > > > >> > >>>>>>> having
> > > > >> > >>>>>>>>> to handle a lot of really low-level stuff that you
> > > > shouldn't.
> > > > >> > >>>>>>>>> We
> > > > >> > >>>>>> spent a
> > > > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> > > robust.
> > > > >> It
> > > > >> > >>>>>>>>> also allows us to support some cool features:
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> * Per-partition message ordering and prioritization.
> > > > >> > >>>>>>>>> * Tight control over partition assignment to support
> > > joins,
> > > > >> > >>>>>>>>> global
> > > > >> > >>>>>> state
> > > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > > > >> > >>>>>>>>> * Tight control over offset checkpointing.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> What we didn't realize at the time is that these
> > features
> > > > >> > >>>>>>>>> should
> > > > >> > >>>>>>> actually
> > > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
> > > stream
> > > > >> > >>>>>> processors)
> > > > >> > >>>>>>>>> end up wanting to do things like joins and partition
> > > > >> > >>>>>>>>> assignment. The
> > > > >> > >>>>>>> Kafka
> > > > >> > >>>>>>>>> community has come to the same conclusion. They're
> > adding
> > > a
> > > > >> ton
> > > > >> > >>>>>>>>> of upgrades into their new Kafka consumer
> > implementation.
> > > > To a
> > > > >> > >>>>>>>>> large extent,
> > > > >> > >>>>> it's
> > > > >> > >>>>>>>>> duplicate work to what we've already done in Samza.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
> > > > approach
> > > > >> > >>>>>>>>> to
> > > > >> > >>>>>> Samza's
> > > > >> > >>>>>>>>> KafkaCheckpointManager implementation for handling
> > offset
> > > > >> > >>>>>> checkpointing.
> > > > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature
> stores
> > > > >> offset
> > > > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them
> > from
> > > > the
> > > > >> > >>>>>>>>> broker.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> A lot of this seems like a waste, since we could have
> > > shared
> > > > >> > >>>>>>>>> the
> > > > >> > >>>>> work
> > > > >> > >>>>>> if
> > > > >> > >>>>>>>>> it
> > > > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Vision
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> All of this leads me to a rather radical proposal.
> Samza
> > > is
> > > > >> > >>>>> relatively
> > > > >> > >>>>>>>>> stable at this point. I'd venture to say that we're
> > near a
> > > > 1.0
> > > > >> > >>>>>> release.
> > > > >> > >>>>>>>>> I'd
> > > > >> > >>>>>>>>> like to propose that we take what we've learned, and
> > begin
> > > > >> > >>>>>>>>> thinking
> > > > >> > >>>>>>> about
> > > > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were
> > starting
> > > > >> from
> > > > >> > >>>>>> scratch?
> > > > >> > >>>>>>>>> My
> > > > >> > >>>>>>>>> proposal is to:
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > > > >> > >>>>>>>>> processors, and eliminate all direct dependences on
> > YARN,
> > > > >> Mesos,
> > > > >> > >>>> etc.
> > > > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as the
> > > > stream
> > > > >> > >>>>>> processing
> > > > >> > >>>>>>>>> layer.
> > > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization,
> > and
> > > > >> > >>>>>>>>> config
> > > > >> > >>>>>>> systems,
> > > > >> > >>>>>>>>> and simply use Kafka's instead.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> This would fix all of the issues that I outlined
> above.
> > It
> > > > >> > >>>>>>>>> should
> > > > >> > >>>>> also
> > > > >> > >>>>>>>>> shrink the Samza code base pretty dramatically.
> > Supporting
> > > > >> only
> > > > >> > >>>>>>>>> a standalone container will allow Samza to be executed
> > on
> > > > YARN
> > > > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most
> > > other
> > > > >> > >>>>>>>>> in-house
> > > > >> > >>>>>>> deployment
> > > > >> > >>>>>>>>> systems. This should make life a lot easier for new
> > users.
> > > > >> > >>>>>>>>> Imagine
> > > > >> > >>>>>>> having
> > > > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in
> > mailing
> > > > >> list
> > > > >> > >>>>>> traffic
> > > > >> > >>>>>>>>> will be pretty dramatic.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The
> > reality
> > > > is,
> > > > >> > >>>>> everyone
> > > > >> > >>>>>>>>> that
> > > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically
> > > require
> > > > >> it
> > > > >> > >>>>>> already
> > > > >> > >>>>>>> in
> > > > >> > >>>>>>>>> order for most features to work. Those that are using
> > > other
> > > > >> > >>>>>>>>> systems
> > > > >> > >>>>>> are
> > > > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and then
> > > they
> > > > do
> > > > >> > >>>>>>>>> the processing on top. There is already discussion (
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>
> > > > >> > >>>>>>
> > > > >> > >>>>>
> > > > >> >
> > > >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > >> > >>>>> 767
> > > > >> > >>>>>>>>> )
> > > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Once we make the call to couple with Kafka, we can
> > > leverage
> > > > a
> > > > >> > >>>>>>>>> ton of
> > > > >> > >>>>>>> their
> > > > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own
> config,
> > > > >> > >>>>>>>>> metrics,
> > > > >> > >>>>> etc.
> > > > >> > >>>>>>> We
> > > > >> > >>>>>>>>> can all share the same libraries, and make them
> better.
> > > This
> > > > >> > >>>>>>>>> will
> > > > >> > >>>>> also
> > > > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and will
> > let
> > > > us
> > > > >> > >>>>> leverage
> > > > >> > >>>>>>>>> their offset management and partition management,
> rather
> > > > than
> > > > >> > >>>>>>>>> having
> > > > >> > >>>>>> our
> > > > >> > >>>>>>>>> own. All of the coordinator stream code would go away,
> > as
> > > > >> would
> > > > >> > >>>>>>>>> most
> > > > >> > >>>>>> of
> > > > >> > >>>>>>>>> the
> > > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
> > > > partition
> > > > >> > >>>>>>> management
> > > > >> > >>>>>>>>> features into the Kafka broker, but they're already
> > moving
> > > > in
> > > > >> > >>>>>>>>> that direction with the new consumer API. The features
> > we
> > > > have
> > > > >> > >>>>>>>>> for
> > > > >> > >>>>>> partition
> > > > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like they
> > > should
> > > > >> be
> > > > >> > >>>>>>>>> in
> > > > >> > >>>>>> Kafka
> > > > >> > >>>>>>>>> anyway. There will always be some niche usages which
> > will
> > > > >> > >>>>>>>>> require
> > > > >> > >>>>>> extra
> > > > >> > >>>>>>>>> care and hence full control over partition assignments
> > > much
> > > > >> > >>>>>>>>> like the
> > > > >> > >>>>>>> Kafka
> > > > >> > >>>>>>>>> low level consumer api. These would continue to be
> > > > supported.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> These items will be good for the Samza community.
> > They'll
> > > > make
> > > > >> > >>>>>>>>> Samza easier to use, and make it easier for developers
> > to
> > > > add
> > > > >> > >>>>>>>>> new features.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat
> backwards
> > > > >> > >>>>> incompatible
> > > > >> > >>>>>>>>> change). If we choose to go this route, it's important
> > > that
> > > > we
> > > > >> > >>>>> openly
> > > > >> > >>>>>>>>> communicate how we're going to provide a migration
> path
> > > from
> > > > >> > >>>>>>>>> the
> > > > >> > >>>>>>> existing
> > > > >> > >>>>>>>>> APIs to the new ones (if we make incompatible
> changes).
> > I
> > > > >> think
> > > > >> > >>>>>>>>> at a minimum, we'd probably need to provide a wrapper
> to
> > > > allow
> > > > >> > >>>>>>>>> existing StreamTask implementations to continue
> running
> > on
> > > > the
> > > > >> > >>>> new container.
> > > > >> > >>>>>>> It's
> > > > >> > >>>>>>>>> also important that we openly communicate about
> timing,
> > > and
> > > > >> > >>>>>>>>> stages
> > > > >> > >>>>> of
> > > > >> > >>>>>>> the
> > > > >> > >>>>>>>>> migration.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> If you made it this far, I'm sure you have opinions.
> :)
> > > > Please
> > > > >> > >>>>>>>>> send
> > > > >> > >>>>>> your
> > > > >> > >>>>>>>>> thoughts and feedback.
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>> Cheers,
> > > > >> > >>>>>>>>> Chris
> > > > >> > >>>>>>>>>
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>>
> > > > >> > >>>>>>>
> > > > >> > >>>>>>
> > > > >> > >>>>>>
> > > > >> > >>>>>>
> > > > >> > >>>>>> --
> > > > >> > >>>>>> -- Guozhang
> > > > >> > >>>>>>
> > > > >> > >>>>>
> > > > >> > >>>>
> > > > >> > >>
> > > > >> > >>
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Roger Hoover <ro...@gmail.com>.
That would be great to let Kafka do as much heavy lifting as possible and
make it easier for other languages to implement Samza apis.

One thing to watch out for is the interplay between Kafka's group
management and the external scheduler/process manager's fault tolerance.
If a container dies, the Kafka group membership protocol will try to assign
it's tasks to other containers while at the same time the process manager
is trying to relaunch the container.  Without some consideration for this
(like a configurable amount of time to wait before Kafka alters the group
membership), there may be thrashing going on which is especially bad for
containers with large amounts of local state.

Someone else pointed this out already but I thought it might be worth
calling out again.

Cheers,

Roger


On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps <ja...@confluent.io> wrote:

> Hey Roger,
>
> I couldn't agree more. We spent a bunch of time talking to people and that
> is exactly the stuff we heard time and again. What makes it hard, of
> course, is that there is some tension between compatibility with what's
> there now and making things better for new users.
>
> I also strongly agree with the importance of multi-language support. We are
> talking now about Java, but for application development use cases people
> want to work in whatever language they are using elsewhere. I think moving
> to a model where Kafka itself does the group membership, lifecycle control,
> and partition assignment has the advantage of putting all that complex
> stuff behind a clean api that the clients are already going to be
> implementing for their consumer, so the added functionality for stream
> processing beyond a consumer becomes very minor.
>
> -Jay
>
> On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <ro...@gmail.com>
> wrote:
>
> > Metamorphosis...nice. :)
> >
> > This has been a great discussion.  As a user of Samza who's recently
> > integrated it into a relatively large organization, I just want to add
> > support to a few points already made.
> >
> > The biggest hurdles to adoption of Samza as it currently exists that I've
> > experienced are:
> > 1) YARN - YARN is overly complex in many environments where Puppet would
> do
> > just fine but it was the only mechanism to get fault tolerance.
> > 2) Configuration - I think I like the idea of configuring most of the job
> > in code rather than config files.  In general, I think the goal should be
> > to make it harder to make mistakes, especially of the kind where the code
> > expects something and the config doesn't match.  The current config is
> > quite intricate and error-prone.  For example, the application logic may
> > depend on bootstrapping a topic but rather than asserting that in the
> code,
> > you have to rely on getting the config right.  Likewise with serdes, the
> > Java representations produced by various serdes (JSON, Avro, etc.) are
> not
> > equivalent so you cannot just reconfigure a serde without changing the
> > code.   It would be nice for jobs to be able to assert what they expect
> > from their input topics in terms of partitioning.  This is getting a
> little
> > off topic but I was even thinking about creating a "Samza config linter"
> > that would sanity check a set of configs.  Especially in organizations
> > where config is managed by a different team than the application
> developer,
> > it's very hard to get avoid config mistakes.
> > 3) Java/Scala centric - for many teams (especially DevOps-type folks),
> the
> > pain of the Java toolchain (maven, slow builds, weak command line
> support,
> > configuration over convention) really inhibits productivity.  As more and
> > more high-quality clients become available for Kafka, I hope they'll
> follow
> > Samza's model.  Not sure how much it affects the proposals in this thread
> > but please consider other languages in the ecosystem as well.  From what
> > I've heard, Spark has more Python users than Java/Scala.
> > (FYI, we added a Jython wrapper for the Samza API
> >
> >
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> > and are working on a Yeoman generator
> > https://github.com/Quantiply/generator-rico for Jython/Samza projects to
> > alleviate some of the pain)
> >
> > I also want to underscore Jay's point about improving the user
> experience.
> > That's a very important factor for adoption.  I think the goal should be
> to
> > make Samza as easy to get started with as something like Logstash.
> > Logstash is vastly inferior in terms of capabilities to Samza but it's
> easy
> > to get started and that makes a big difference.
> >
> > Cheers,
> >
> > Roger
> >
> >
> >
> >
> >
> > On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
> > gdfm@apache.org> wrote:
> >
> > > Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
> > winner
> > > :)
> > >
> > > --
> > > Gianmarco
> > >
> > > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <
> gdfm@apache.org
> > >
> > > wrote:
> > >
> > > > Hi,
> > > >
> > > > @Martin, thanks for you comments.
> > > > Maybe I'm missing some important point, but I think coupling the
> > releases
> > > > is actually a *good* thing.
> > > > To make an example, would it be better if the MR and HDFS components
> of
> > > > Hadoop had different release schedules?
> > > >
> > > > Actually, keeping the discussion in a single place would make
> agreeing
> > on
> > > > releases (and backwards compatibility) much easier, as everybody
> would
> > be
> > > > responsible for the whole codebase.
> > > >
> > > > That said, I like the idea of absorbing samza-core as a sub-project,
> > and
> > > > leave the fancy stuff separate.
> > > > It probably gives 90% of the benefits we have been discussing here.
> > > >
> > > > Cheers,
> > > >
> > > > --
> > > > Gianmarco
> > > >
> > > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
> > > >
> > > >> Hey Martin,
> > > >>
> > > >> I agree coupling release schedules is a downside.
> > > >>
> > > >> Definitely we can try to solve some of the integration problems in
> > > >> Confluent Platform or in other distributions. But I think this ends
> up
> > > >> being really shallow. I guess I feel to really get a good user
> > > experience
> > > >> the two systems have to kind of feel like part of the same thing and
> > you
> > > >> can't really add that in later--you can put both in the same
> > > downloadable
> > > >> tar file but it doesn't really give a very cohesive feeling. I agree
> > > that
> > > >> ultimately any of the project stuff is as much social and naming as
> > > >> anything else--theoretically two totally independent projects could
> > work
> > > >> to
> > > >> tightly align. In practice this seems to be quite difficult though.
> > > >>
> > > >> For the frameworks--totally agree it would be good to maintain the
> > > >> framework support with the project. In some cases there may not be
> too
> > > >> much
> > > >> there since the integration gets lighter but I think whatever stubs
> > you
> > > >> need should be included. So no I definitely wasn't trying to imply
> > > >> dropping
> > > >> support for these frameworks, just making the integration lighter by
> > > >> separating process management from partition management.
> > > >>
> > > >> You raise two good points we would have to figure out if we went
> down
> > > the
> > > >> alignment path:
> > > >> 1. With respect to the name, yeah I think the first question is
> > whether
> > > >> some "re-branding" would be worth it. If so then I think we can
> have a
> > > big
> > > >> thread on the name. I'm definitely not set on Kafka Streaming or
> Kafka
> > > >> Streams I was just using them to be kind of illustrative. I agree
> with
> > > >> your
> > > >> critique of these names, though I think people would get the idea.
> > > >> 2. Yeah you also raise a good point about how to "factor" it. Here
> are
> > > the
> > > >> options I see (I could get enthusiastic about any of them):
> > > >>    a. One repo for both Kafka and Samza
> > > >>    b. Two repos, retaining the current seperation
> > > >>    c. Two repos, the equivalent of samza-api and samza-core is
> > absorbed
> > > >> almost like a third client
> > > >>
> > > >> Cheers,
> > > >>
> > > >> -Jay
> > > >>
> > > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> > martin@kleppmann.com>
> > > >> wrote:
> > > >>
> > > >> > Ok, thanks for the clarifications. Just a few follow-up comments.
> > > >> >
> > > >> > - I see the appeal of merging with Kafka or becoming a subproject:
> > the
> > > >> > reasons you mention are good. The risk I see is that release
> > schedules
> > > >> > become coupled to each other, which can slow everyone down, and
> > large
> > > >> > projects with many contributors are harder to manage. (Jakob, can
> > you
> > > >> speak
> > > >> > from experience, having seen a wider range of Hadoop ecosystem
> > > >> projects?)
> > > >> >
> > > >> > Some of the goals of a better unified developer experience could
> > also
> > > be
> > > >> > solved by integrating Samza nicely into a Kafka distribution (such
> > as
> > > >> > Confluent's). I'm not against merging projects if we decide that's
> > the
> > > >> way
> > > >> > to go, just pointing out the same goals can perhaps also be
> achieved
> > > in
> > > >> > other ways.
> > > >> >
> > > >> > - With regard to dropping the YARN dependency: are you proposing
> > that
> > > >> > Samza doesn't give any help to people wanting to run on
> > > >> YARN/Mesos/AWS/etc?
> > > >> > So the docs would basically have a link to Slider and nothing
> else?
> > Or
> > > >> > would we maintain integrations with a bunch of popular deployment
> > > >> methods
> > > >> > (e.g. the necessary glue and shell scripts to make Samza work with
> > > >> Slider)?
> > > >> >
> > > >> > I absolutely think it's a good idea to have the "as a library" and
> > > "as a
> > > >> > process" (using Yi's taxonomy) options for people who want them,
> > but I
> > > >> > think there should also be a low-friction path for common "as a
> > > service"
> > > >> > deployment methods, for which we probably need to maintain
> > > integrations.
> > > >> >
> > > >> > - Project naming: "Kafka Streams" seems odd to me, because Kafka
> is
> > > all
> > > >> > about streams already. Perhaps "Kafka Transformers" or "Kafka
> > Filters"
> > > >> > would be more apt?
> > > >> >
> > > >> > One suggestion: perhaps the core of Samza (stream transformation
> > with
> > > >> > state management -- i.e. the "Samza as a library" bit) could
> become
> > > >> part of
> > > >> > Kafka, while higher-level tools such as streaming SQL and
> > integrations
> > > >> with
> > > >> > deployment frameworks remain in a separate project? In other
> words,
> > > >> Kafka
> > > >> > would absorb the proven, stable core of Samza, which would become
> > the
> > > >> > "third Kafka client" mentioned early in this thread. The Samza
> > project
> > > >> > would then target that third Kafka client as its base API, and the
> > > >> project
> > > >> > would be freed up to explore more experimental new horizons.
> > > >> >
> > > >> > Martin
> > > >> >
> > > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
> > > >> >
> > > >> > > Hey Martin,
> > > >> > >
> > > >> > > For the YARN/Mesos/etc decoupling I actually don't think it ties
> > our
> > > >> > hands
> > > >> > > at all, all it does is refactor things. The division of
> > > >> responsibility is
> > > >> > > that Samza core is responsible for task lifecycle, state, and
> > > >> partition
> > > >> > > management (using the Kafka co-ordinator) but it is NOT
> > responsible
> > > >> for
> > > >> > > packaging, configuration deployment or execution of processes.
> The
> > > >> > problem
> > > >> > > of packaging and starting these processes is
> > > >> > > framework/environment-specific. This leaves individual
> frameworks
> > to
> > > >> be
> > > >> > as
> > > >> > > fancy or vanilla as they like. So you can get simple stateless
> > > >> support in
> > > >> > > YARN, Mesos, etc using their off-the-shelf app framework
> (Slider,
> > > >> > Marathon,
> > > >> > > etc). These are well known by people and have nice UIs and a lot
> > of
> > > >> > > flexibility. I don't think they have node affinity as a built in
> > > >> option
> > > >> > > (though I could be wrong). So if we want that we can either wait
> > for
> > > >> them
> > > >> > > to add it or do a custom framework to add that feature (as now).
> > > >> > Obviously
> > > >> > > if you manage things with old-school ops tools (puppet/chef/etc)
> > you
> > > >> get
> > > >> > > locality easily. The nice thing, though, is that all the samza
> > > >> "business
> > > >> > > logic" around partition management and fault tolerance is in
> Samza
> > > >> core
> > > >> > so
> > > >> > > it is shared across frameworks and the framework specific bit is
> > > just
> > > >> > > whether it is smart enough to try to get the same host when a
> job
> > is
> > > >> > > restarted.
> > > >> > >
> > > >> > > With respect to the Kafka-alignment, yeah I think the goal would
> > be
> > > >> (a)
> > > >> > > actually get better alignment in user experience, and (b)
> express
> > > >> this in
> > > >> > > the naming and project branding. Specifically:
> > > >> > > 1. Website/docs, it would be nice for the "transformation" api
> to
> > be
> > > >> > > discoverable in the main Kafka docs--i.e. be able to explain
> when
> > to
> > > >> use
> > > >> > > the consumer and when to use the stream processing functionality
> > and
> > > >> lead
> > > >> > > people into that experience.
> > > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that
> > has
> > > >> both
> > > >> > > Kafka and the stream processing part and they actually work
> > > together.
> > > >> > > 3. Unify the programming experience so the client and Samza api
> > > share
> > > >> > > config/monitoring/naming/packaging/etc.
> > > >> > >
> > > >> > > I think sub-projects keep separate committers and can have a
> > > separate
> > > >> > repo,
> > > >> > > but I'm actually not really sure (I can't find a definition of a
> > > >> > subproject
> > > >> > > in Apache).
> > > >> > >
> > > >> > > Basically at a high-level you want the experience to "feel"
> like a
> > > >> single
> > > >> > > system, not to relatively independent things that are kind of
> > > >> awkwardly
> > > >> > > glued together.
> > > >> > >
> > > >> > > I think if we did that they having naming or branding like
> "kafka
> > > >> > > streaming" or "kafka streams" or something like that would
> > actually
> > > >> do a
> > > >> > > good job of conveying what it is. I do that this would help
> > adoption
> > > >> > quite
> > > >> > > a lot as it would correctly convey that using Kafka Streaming
> with
> > > >> Kafka
> > > >> > is
> > > >> > > a fairly seamless experience and Kafka is pretty heavily adopted
> > at
> > > >> this
> > > >> > > point.
> > > >> > >
> > > >> > > Fwiw we actually considered this model originally when open
> > sourcing
> > > >> > Samza,
> > > >> > > however at that time Kafka was relatively unknown and we decided
> > not
> > > >> to
> > > >> > do
> > > >> > > it since we felt it would be limiting. From my point of view the
> > > three
> > > >> > > things have changed (1) Kafka is now really heavily used for
> > stream
> > > >> > > processing, (2) we learned that abstracting out the stream well
> is
> > > >> > > basically impossible, (3) we learned it is really hard to keep
> the
> > > two
> > > >> > > things feeling like a single product.
> > > >> > >
> > > >> > > -Jay
> > > >> > >
> > > >> > >
> > > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> > > >> martin@kleppmann.com>
> > > >> > > wrote:
> > > >> > >
> > > >> > >> Hi all,
> > > >> > >>
> > > >> > >> Lots of good thoughts here.
> > > >> > >>
> > > >> > >> I agree with the general philosophy of tying Samza more firmly
> to
> > > >> Kafka.
> > > >> > >> After I spent a while looking at integrating other message
> > brokers
> > > >> (e.g.
> > > >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
> > > >> > SystemConsumer
> > > >> > >> tacitly assumes a model so much like Kafka's that pretty much
> > > nobody
> > > >> but
> > > >> > >> Kafka actually implements it. (Databus is perhaps an exception,
> > but
> > > >> it
> > > >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza
> fully
> > > >> > dependent
> > > >> > >> on Kafka acknowledges that the system-independence was never as
> > > real
> > > >> as
> > > >> > we
> > > >> > >> perhaps made it out to be. The gains of code reuse are real.
> > > >> > >>
> > > >> > >> The idea of decoupling Samza from YARN has also always been
> > > >> appealing to
> > > >> > >> me, for various reasons already mentioned in this thread.
> > Although
> > > >> > making
> > > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems
> > > >> laudable,
> > > >> > I am
> > > >> > >> a little concerned that it will restrict us to a lowest common
> > > >> > denominator.
> > > >> > >> For example, would host affinity (SAMZA-617) still be possible?
> > For
> > > >> jobs
> > > >> > >> with large amounts of state, I think SAMZA-617 would be a big
> > boon,
> > > >> > since
> > > >> > >> restoring state off the changelog on every single restart is
> > > painful,
> > > >> > due
> > > >> > >> to long recovery times. It would be a shame if the decoupling
> > from
> > > >> YARN
> > > >> > >> made host affinity impossible.
> > > >> > >>
> > > >> > >> Jay, a question about the proposed API for instantiating a job
> in
> > > >> code
> > > >> > >> (rather than a properties file): when submitting a job to a
> > > cluster,
> > > >> is
> > > >> > the
> > > >> > >> idea that the instantiation code runs on a client somewhere,
> > which
> > > >> then
> > > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does
> that
> > > >> code
> > > >> > run
> > > >> > >> on each container that is part of the job (in which case, how
> > does
> > > >> the
> > > >> > job
> > > >> > >> submission to the cluster work)?
> > > >> > >>
> > > >> > >> I agree with Garry that it doesn't feel right to make a 1.0
> > release
> > > >> > with a
> > > >> > >> plan for it to be immediately obsolete. So if this is going to
> > > >> happen, I
> > > >> > >> think it would be more honest to stick with 0.* version numbers
> > > until
> > > >> > the
> > > >> > >> library-ified Samza has been implemented, is stable and widely
> > > used.
> > > >> > >>
> > > >> > >> Should the new Samza be a subproject of Kafka? There is
> precedent
> > > for
> > > >> > >> tight coupling between different Apache projects (e.g. Curator
> > and
> > > >> > >> Zookeeper, or Slider and YARN), so I think remaining separate
> > would
> > > >> be
> > > >> > ok.
> > > >> > >> Even if Samza is fully dependent on Kafka, there is enough
> > > substance
> > > >> in
> > > >> > >> Samza that it warrants being a separate project. An argument in
> > > >> favour
> > > >> > of
> > > >> > >> merging would be if we think Kafka has a much stronger "brand
> > > >> presence"
> > > >> > >> than Samza; I'm ambivalent on that one. If the Kafka project is
> > > >> willing
> > > >> > to
> > > >> > >> endorse Samza as the "official" way of doing stateful stream
> > > >> > >> transformations, that would probably have much the same effect
> as
> > > >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike.
> Close
> > > >> > >> collaboration between the two projects will be needed in any
> > case.
> > > >> > >>
> > > >> > >> From a project management perspective, I guess the "new Samza"
> > > would
> > > >> > have
> > > >> > >> to be developed on a branch alongside ongoing maintenance of
> the
> > > >> current
> > > >> > >> line of development? I think it would be important to continue
> > > >> > supporting
> > > >> > >> existing users, and provide a graceful migration path to the
> new
> > > >> > version.
> > > >> > >> Leaving the current versions unsupported and forcing people to
> > > >> rewrite
> > > >> > >> their jobs would send a bad signal.
> > > >> > >>
> > > >> > >> Best,
> > > >> > >> Martin
> > > >> > >>
> > > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
> > > >> > >>
> > > >> > >>> Hey Garry,
> > > >> > >>>
> > > >> > >>> Yeah that's super frustrating. I'd be happy to chat more about
> > > this
> > > >> if
> > > >> > >>> you'd be interested. I think Chris and I started with the idea
> > of
> > > >> "what
> > > >> > >>> would it take to make Samza a kick-ass ingestion tool" but
> > > >> ultimately
> > > >> > we
> > > >> > >>> kind of came around to the idea that ingestion and
> > transformation
> > > >> had
> > > >> > >>> pretty different needs and coupling the two made things hard.
> > > >> > >>>
> > > >> > >>> For what it's worth I think copycat (KIP-26) actually will do
> > what
> > > >> you
> > > >> > >> are
> > > >> > >>> looking for.
> > > >> > >>>
> > > >> > >>> With regard to your point about slider, I don't necessarily
> > > >> disagree.
> > > >> > >> But I
> > > >> > >>> think getting good YARN support is quite doable and I think we
> > can
> > > >> make
> > > >> > >>> that work well. I think the issue this proposal solves is that
> > > >> > >> technically
> > > >> > >>> it is pretty hard to support multiple cluster management
> systems
> > > the
> > > >> > way
> > > >> > >>> things are now, you need to write an "app master" or
> "framework"
> > > for
> > > >> > each
> > > >> > >>> and they are all a little different so testing is really hard.
> > In
> > > >> the
> > > >> > >>> absence of this we have been stuck with just YARN which has
> > > >> fantastic
> > > >> > >>> penetration in the Hadoopy part of the org, but zero
> penetration
> > > >> > >> elsewhere.
> > > >> > >>> Given the huge amount of work being put in to slider,
> marathon,
> > > aws
> > > >> > >>> tooling, not to mention the umpteen related packaging
> > technologies
> > > >> > people
> > > >> > >>> want to use (Docker, Kubernetes, various cloud-specific deploy
> > > >> tools,
> > > >> > >> etc)
> > > >> > >>> I really think it is important to get this right.
> > > >> > >>>
> > > >> > >>> -Jay
> > > >> > >>>
> > > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > >> > >>> g.turkington@improvedigital.com> wrote:
> > > >> > >>>
> > > >> > >>>> Hi all,
> > > >> > >>>>
> > > >> > >>>> I think the question below re does Samza become a sub-project
> > of
> > > >> Kafka
> > > >> > >>>> highlights the broader point around migration. Chris mentions
> > > >> Samza's
> > > >> > >>>> maturity is heading towards a v1 release but I'm not sure it
> > > feels
> > > >> > >> right to
> > > >> > >>>> launch a v1 then immediately plan to deprecate most of it.
> > > >> > >>>>
> > > >> > >>>> From a selfish perspective I have some guys who have started
> > > >> working
> > > >> > >> with
> > > >> > >>>> Samza and building some new consumers/producers was next up.
> > > Sounds
> > > >> > like
> > > >> > >>>> that is absolutely not the direction to go. I need to look
> into
> > > the
> > > >> > KIP
> > > >> > >> in
> > > >> > >>>> more detail but for me the attractiveness of adding new Samza
> > > >> > >>>> consumer/producers -- even if yes all they were doing was
> > really
> > > >> > getting
> > > >> > >>>> data into and out of Kafka --  was to avoid  having to worry
> > > about
> > > >> the
> > > >> > >>>> lifecycle management of external clients. If there is a
> generic
> > > >> Kafka
> > > >> > >>>> ingress/egress layer that I can plug a new connector into and
> > > have
> > > >> a
> > > >> > >> lot of
> > > >> > >>>> the heavy lifting re scale and reliability done for me then
> it
> > > >> gives
> > > >> > me
> > > >> > >> all
> > > >> > >>>> the pushing new consumers/producers would. If not then it
> > > >> complicates
> > > >> > my
> > > >> > >>>> operational deployments.
> > > >> > >>>>
> > > >> > >>>> Which is similar to my other question with the proposal -- if
> > we
> > > >> > build a
> > > >> > >>>> fully available/stand-alone Samza plus the requisite shims to
> > > >> > integrate
> > > >> > >>>> with Slider etc I suspect the former may be a lot more work
> > than
> > > we
> > > >> > >> think.
> > > >> > >>>> We may make it much easier for a newcomer to get something
> > > running
> > > >> but
> > > >> > >>>> having them step up and get a reliable production deployment
> > may
> > > >> still
> > > >> > >>>> dominate mailing list  traffic, if for different reasons than
> > > >> today.
> > > >> > >>>>
> > > >> > >>>> Don't get me wrong -- I'm comfortable with making the Samza
> > > >> dependency
> > > >> > >> on
> > > >> > >>>> Kafka much more explicit and I absolutely see the benefits
> in
> > > the
> > > >> > >>>> reduction of duplication and clashing
> > terminologies/abstractions
> > > >> that
> > > >> > >>>> Chris/Jay describe. Samza as a library would likely be a very
> > > nice
> > > >> > tool
> > > >> > >> to
> > > >> > >>>> add to the Kafka ecosystem. I just have the concerns above re
> > the
> > > >> > >>>> operational side.
> > > >> > >>>>
> > > >> > >>>> Garry
> > > >> > >>>>
> > > >> > >>>> -----Original Message-----
> > > >> > >>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org
> ]
> > > >> > >>>> Sent: 02 July 2015 12:56
> > > >> > >>>> To: dev@samza.apache.org
> > > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> > > >> > >>>>
> > > >> > >>>> Very interesting thoughts.
> > > >> > >>>> From outside, I have always perceived Samza as a computing
> > layer
> > > >> over
> > > >> > >>>> Kafka.
> > > >> > >>>>
> > > >> > >>>> The question, maybe a bit provocative, is "should Samza be a
> > > >> > sub-project
> > > >> > >>>> of Kafka then?"
> > > >> > >>>> Or does it make sense to keep it as a separate project with a
> > > >> separate
> > > >> > >>>> governance?
> > > >> > >>>>
> > > >> > >>>> Cheers,
> > > >> > >>>>
> > > >> > >>>> --
> > > >> > >>>> Gianmarco
> > > >> > >>>>
> > > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> > wrote:
> > > >> > >>>>
> > > >> > >>>>> Overall, I agree to couple with Kafka more tightly. Because
> > > Samza
> > > >> de
> > > >> > >>>>> facto is based on Kafka, and it should leverage what Kafka
> > has.
> > > At
> > > >> > the
> > > >> > >>>>> same time, Kafka does not need to reinvent what Samza
> already
> > > >> has. I
> > > >> > >>>>> also like the idea of separating the ingestion and
> > > transformation.
> > > >> > >>>>>
> > > >> > >>>>> But it is a little difficult for me to image how the Samza
> > will
> > > >> look
> > > >> > >>>> like.
> > > >> > >>>>> And I feel Chris and Jay have a little difference in terms
> of
> > > how
> > > >> > >>>>> Samza should look like.
> > > >> > >>>>>
> > > >> > >>>>> *** Will it look like what Jay's code shows (A client of
> > Kakfa)
> > > ?
> > > >> And
> > > >> > >>>>> user's application code calls this client?
> > > >> > >>>>>
> > > >> > >>>>> 1. If we make Samza be a library of Kafka (like what the
> code
> > > >> shows),
> > > >> > >>>>> how do we implement auto-balance and fault-tolerance? Are
> they
> > > >> taken
> > > >> > >>>>> care by the Kafka broker or other mechanism, such as "Samza
> > > >> worker"
> > > >> > >>>>> (just make up the name) ?
> > > >> > >>>>>
> > > >> > >>>>> 2. What about other features, such as auto-scaling, shared
> > > state,
> > > >> > >>>>> monitoring?
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> *** If we have Samza standalone, (is this what Chris
> > suggests?)
> > > >> > >>>>>
> > > >> > >>>>> 1. we still need to ingest data from Kakfa and produce to
> it.
> > > >> Then it
> > > >> > >>>>> becomes the same as what Samza looks like now, except it
> does
> > > not
> > > >> > rely
> > > >> > >>>>> on Yarn anymore.
> > > >> > >>>>>
> > > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's metrics,
> > > logs,
> > > >> > >>>>> etc? Use Kafka code as the dependency?
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> Thanks,
> > > >> > >>>>>
> > > >> > >>>>> Fang, Yan
> > > >> > >>>>> yanfang724@gmail.com
> > > >> > >>>>>
> > > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > > wangguoz@gmail.com
> > > >> >
> > > >> > >>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>>> Read through the code example and it looks good to me. A
> few
> > > >> > >>>>>> thoughts regarding deployment:
> > > >> > >>>>>>
> > > >> > >>>>>> Today Samza deploys as executable runnable like:
> > > >> > >>>>>>
> > > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> > > >> > >>>> --config-path=file://...
> > > >> > >>>>>>
> > > >> > >>>>>> And this proposal advocate for deploying Samza more as
> > embedded
> > > >> > >>>>>> libraries in user application code (ignoring the
> terminology
> > > >> since
> > > >> > >>>>>> it is not the
> > > >> > >>>>> same
> > > >> > >>>>>> as the prototype code):
> > > >> > >>>>>>
> > > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread thread
> =
> > > new
> > > >> > >>>>>> Thread(task); thread.start();
> > > >> > >>>>>>
> > > >> > >>>>>> I think both of these deployment modes are important for
> > > >> different
> > > >> > >>>>>> types
> > > >> > >>>>> of
> > > >> > >>>>>> users. That said, I think making Samza purely standalone is
> > > still
> > > >> > >>>>>> sufficient for either runnable or library modes.
> > > >> > >>>>>>
> > > >> > >>>>>> Guozhang
> > > >> > >>>>>>
> > > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> > jay@confluent.io>
> > > >> > wrote:
> > > >> > >>>>>>
> > > >> > >>>>>>> Looks like gmail mangled the code example, it was supposed
> > to
> > > >> look
> > > >> > >>>>>>> like
> > > >> > >>>>>>> this:
> > > >> > >>>>>>>
> > > >> > >>>>>>> Properties props = new Properties();
> > > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > >> StreamingConfig
> > > >> > >>>>>>> config = new StreamingConfig(props);
> > > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> > > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> > > >> > >>>>>>> config.serialization(new StringSerializer(), new
> > > >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
> > > >> > >>>>>>> KafkaStreaming(config); container.run();
> > > >> > >>>>>>>
> > > >> > >>>>>>> -Jay
> > > >> > >>>>>>>
> > > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> > jay@confluent.io
> > > >
> > > >> > >>>> wrote:
> > > >> > >>>>>>>
> > > >> > >>>>>>>> Hey guys,
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> This came out of some conversations Chris and I were
> having
> > > >> > >>>>>>>> around
> > > >> > >>>>>>> whether
> > > >> > >>>>>>>> it would make sense to use Samza as a kind of data
> > ingestion
> > > >> > >>>>> framework
> > > >> > >>>>>>> for
> > > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This
> > kind
> > > of
> > > >> > >>>>>> combined
> > > >> > >>>>>>>> with complaints around config and YARN and the discussion
> > > >> around
> > > >> > >>>>>>>> how
> > > >> > >>>>> to
> > > >> > >>>>>>>> best do a standalone mode.
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> So the thought experiment was, given that Samza was
> > basically
> > > >> > >>>>>>>> already totally Kafka specific, what if you just embraced
> > > that
> > > >> > >>>>>>>> and turned it
> > > >> > >>>>>> into
> > > >> > >>>>>>>> something less like a heavyweight framework and more
> like a
> > > >> > >>>>>>>> third
> > > >> > >>>>> Kafka
> > > >> > >>>>>>>> client--a kind of "producing consumer" with state
> > management
> > > >> > >>>>>> facilities.
> > > >> > >>>>>>>> Basically a library. Instead of a complex stream
> processing
> > > >> > >>>>>>>> framework
> > > >> > >>>>>>> this
> > > >> > >>>>>>>> would actually be a very simple thing, not much more
> > > >> complicated
> > > >> > >>>>>>>> to
> > > >> > >>>>> use
> > > >> > >>>>>>> or
> > > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we thought
> > about
> > > >> it
> > > >> > >>>>>>>> a
> > > >> > >>>>> lot
> > > >> > >>>>>> of
> > > >> > >>>>>>>> what Samza (and the other stream processing systems were
> > > doing)
> > > >> > >>>>> seemed
> > > >> > >>>>>>> like
> > > >> > >>>>>>>> kind of a hangover from MapReduce.
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> Of course you need to ingest/output data to and from the
> > > stream
> > > >> > >>>>>>>> processing. But when we actually looked into how that
> would
> > > >> > >>>>>>>> work,
> > > >> > >>>>> Samza
> > > >> > >>>>>>>> isn't really an ideal data ingestion framework for a
> bunch
> > of
> > > >> > >>>>> reasons.
> > > >> > >>>>>> To
> > > >> > >>>>>>>> really do that right you need a pretty different internal
> > > data
> > > >> > >>>>>>>> model
> > > >> > >>>>>> and
> > > >> > >>>>>>>> set of apis. So what if you split them and had an api for
> > > Kafka
> > > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api
> for
> > > >> Kafka
> > > >> > >>>>>>>> transformation (Samza).
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> This would also allow really embracing the same
> terminology
> > > and
> > > >> > >>>>>>>> conventions. One complaint about the current state is
> that
> > > the
> > > >> > >>>>>>>> two
> > > >> > >>>>>>> systems
> > > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs
> > "topic"
> > > >> and
> > > >> > >>>>>>> different
> > > >> > >>>>>>>> config and monitoring systems means you kind of have to
> > learn
> > > >> > >>>>>>>> Kafka's
> > > >> > >>>>>>> way,
> > > >> > >>>>>>>> then learn Samza's slightly different way, then kind of
> > > >> > >>>>>>>> understand
> > > >> > >>>>> how
> > > >> > >>>>>>> they
> > > >> > >>>>>>>> map to each other, which having walked a few people
> through
> > > >> this
> > > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> Since I have been spending a lot of time on airplanes I
> > > hacked
> > > >> > >>>>>>>> up an ernest but still somewhat incomplete prototype of
> > what
> > > >> > >>>>>>>> this would
> > > >> > >>>>> look
> > > >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka as
> it
> > > >> > >>>>>>>> required a
> > > >> > >>>>>> few
> > > >> > >>>>>>>> changes to the new consumer. Here is the code:
> > > >> > >>>>>>>>
> > > >> > >>>>>>>>
> > > >> > >>>>>>>
> > > >> > >>>>>>
> > > >> > >>>>>
> > > >> >
> > > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > >> > >>>>> /apache/kafka/clients/streaming
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> For the purpose of the prototype I just liberally renamed
> > > >> > >>>>>>>> everything
> > > >> > >>>>> to
> > > >> > >>>>>>>> try to align it with Kafka with no regard for
> > compatibility.
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> To use this would be something like this:
> > > >> > >>>>>>>> Properties props = new Properties();
> > > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > >> > >>>>>>>> StreamingConfig config = new
> > > >> > >>>>> StreamingConfig(props);
> > > >> > >>>>>>> config.subscribe("test-topic-1",
> > > >> > >>>>>>>> "test-topic-2");
> > > >> config.processor(ExampleStreamProcessor.class);
> > > >> > >>>>>>> config.serialization(new
> > > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
> > KafkaStreaming
> > > >> > >>>>>> container =
> > > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> > > StreamProcessor
> > > >> > >>>>>>>> is basically StreamTask.
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> So rather than putting all the class names in a file and
> > then
> > > >> > >>>>>>>> having
> > > >> > >>>>>> the
> > > >> > >>>>>>>> job assembled by reflection, you just instantiate the
> > > container
> > > >> > >>>>>>>> programmatically. Work is balanced over however many
> > > instances
> > > >> > >>>>>>>> of
> > > >> > >>>>> this
> > > >> > >>>>>>> are
> > > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new tasks
> are
> > > >> added
> > > >> > >>>>>>>> to
> > > >> > >>>>> the
> > > >> > >>>>>>>> existing containers without shutting them down).
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> We would provide some glue for running this stuff in YARN
> > via
> > > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their
> > tools
> > > >> > >>>>>>>> but from the
> > > >> > >>>>>> point
> > > >> > >>>>>>> of
> > > >> > >>>>>>>> view of these frameworks these stream processing jobs are
> > > just
> > > >> > >>>>>> stateless
> > > >> > >>>>>>>> services that can come and go and expand and contract at
> > > will.
> > > >> > >>>>>>>> There
> > > >> > >>>>> is
> > > >> > >>>>>>> no
> > > >> > >>>>>>>> more custom scheduler.
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> Here are some relevant details:
> > > >> > >>>>>>>>
> > > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get larger
> if
> > we
> > > >> > >>>>>>>>  productionized but not vastly larger. We really do get a
> > ton
> > > >> > >>>>>>>> of
> > > >> > >>>>>>> leverage
> > > >> > >>>>>>>>  out of Kafka.
> > > >> > >>>>>>>>  2. Partition management is fully delegated to the new
> > > >> consumer.
> > > >> > >>>>> This
> > > >> > >>>>>>>>  is nice since now any partition management strategy
> > > available
> > > >> > >>>>>>>> to
> > > >> > >>>>>> Kafka
> > > >> > >>>>>>>>  consumer is also available to Samza (and vice versa) and
> > > with
> > > >> > >>>>>>>> the
> > > >> > >>>>>>> exact
> > > >> > >>>>>>>>  same configs.
> > > >> > >>>>>>>>  3. It supports state as well as state reuse
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> -Jay
> > > >> > >>>>>>>>
> > > >> > >>>>>>>>
> > > >> > >>>>>>>>
> > > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > >> > >>>>>> criccomini@apache.org>
> > > >> > >>>>>>>> wrote:
> > > >> > >>>>>>>>
> > > >> > >>>>>>>>> Hey all,
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> I have had some discussions with Samza engineers at
> > LinkedIn
> > > >> > >>>>>>>>> and
> > > >> > >>>>>>> Confluent
> > > >> > >>>>>>>>> and we came up with a few observations and would like to
> > > >> > >>>>>>>>> propose
> > > >> > >>>>> some
> > > >> > >>>>>>>>> changes.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> We've observed some things that I want to call out about
> > > >> > >>>>>>>>> Samza's
> > > >> > >>>>>> design,
> > > >> > >>>>>>>>> and I'd like to propose some changes.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
> > > >> > >>>>>>>>> * Samza is too pluggable.
> > > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> > consumer
> > > >> > >>>>>>>>> APIs
> > > >> > >>>>> are
> > > >> > >>>>>>>>> trying to solve a lot of the same problems.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> All three of these issues are related, but I'll address
> > them
> > > >> in
> > > >> > >>>>> order.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Deployment
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic
> deployment
> > > >> > >>>>>>>>> scheduler
> > > >> > >>>>>> such
> > > >> > >>>>>>>>> as
> > > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet
> > that
> > > >> > >>>>>>>>> there
> > > >> > >>>>>> would
> > > >> > >>>>>>>>> be
> > > >> > >>>>>>>>> one or two winners in this area, and we could support
> > them,
> > > >> and
> > > >> > >>>>>>>>> the
> > > >> > >>>>>> rest
> > > >> > >>>>>>>>> would go away. In reality, there are many variations.
> > > >> > >>>>>>>>> Furthermore,
> > > >> > >>>>>> many
> > > >> > >>>>>>>>> people still prefer to just start their processors like
> > > normal
> > > >> > >>>>>>>>> Java processes, and use traditional deployment scripts
> > such
> > > as
> > > >> > >>>>>>>>> Fabric,
> > > >> > >>>>>> Chef,
> > > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users makes
> > the
> > > >> > >>>>>>>>> Samza start-up process really painful for first time
> > users.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit of a
> > > >> > >>>>>>>>> mis-fire
> > > >> > >>>>>> because
> > > >> > >>>>>>>>> of
> > > >> > >>>>>>>>> a fundamental misunderstanding between the nature of
> batch
> > > >> jobs
> > > >> > >>>>>>>>> and
> > > >> > >>>>>>> stream
> > > >> > >>>>>>>>> processing jobs. Early on, we made conscious effort to
> > favor
> > > >> > >>>>>>>>> the
> > > >> > >>>>>> Hadoop
> > > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked and
> was
> > > well
> > > >> > >>>>>>> understood.
> > > >> > >>>>>>>>> One thing that we missed was that batch jobs have a
> > definite
> > > >> > >>>>>> beginning,
> > > >> > >>>>>>>>> and
> > > >> > >>>>>>>>> end, and stream processing jobs don't (usually). This
> > leads
> > > to
> > > >> > >>>>>>>>> a
> > > >> > >>>>> much
> > > >> > >>>>>>>>> simpler scheduling problem for stream processors. You
> > > >> basically
> > > >> > >>>>>>>>> just
> > > >> > >>>>>>> need
> > > >> > >>>>>>>>> to find a place to start the processor, and start it.
> The
> > > way
> > > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a
> cluster
> > > >> > >>>>>>>>> being "full". We always
> > > >> > >>>>>> add
> > > >> > >>>>>>>>> more machines. The problem with coupling Samza with a
> > > >> scheduler
> > > >> > >>>>>>>>> is
> > > >> > >>>>>> that
> > > >> > >>>>>>>>> Samza (as a framework) now has to handle deployment.
> This
> > > >> pulls
> > > >> > >>>>>>>>> in a
> > > >> > >>>>>>> bunch
> > > >> > >>>>>>>>> of things such as configuration distribution (config
> > > stream),
> > > >> > >>>>>>>>> shell
> > > >> > >>>>>>> scrips
> > > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> > stuff),
> > > >> etc.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Another reason for requiring dynamic deployment was to
> > > support
> > > >> > >>>>>>>>> data locality. If you want to have locality, you need to
> > put
> > > >> > >>>>>>>>> your
> > > >> > >>>>>> processors
> > > >> > >>>>>>>>> close to the data they're processing. Upon further
> > > >> > >>>>>>>>> investigation,
> > > >> > >>>>>>> though,
> > > >> > >>>>>>>>> this feature is not that beneficial. There is some good
> > > >> > >>>>>>>>> discussion
> > > >> > >>>>>> about
> > > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
> > > >> > >>>>>>>>> Map/Reduce
> > > >> > >>>>>> path,
> > > >> > >>>>>>>>> but
> > > >> > >>>>>>>>> there are some fundamental differences between HDFS and
> > > Kafka.
> > > >> > >>>>>>>>> HDFS
> > > >> > >>>>>> has
> > > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to less
> > > >> > >>>>>>>>> optimization potential with stream processors on top of
> > > Kafka.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't
> have
> > > any
> > > >> > >>>>>>>>> built
> > > >> > >>>>> in
> > > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the
> dynamic
> > > >> > >>>>>>>>> deployment scheduling system to handle restarts when a
> > > >> > >>>>>>>>> processor dies. This has
> > > >> > >>>>>>> made
> > > >> > >>>>>>>>> it very difficult to write a standalone Samza container
> > > >> > >>>> (SAMZA-516).
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Pluggability
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> In some cases pluggability is good, but I think that
> we've
> > > >> gone
> > > >> > >>>>>>>>> too
> > > >> > >>>>>> far
> > > >> > >>>>>>>>> with it. Currently, Samza has:
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> * Pluggable config.
> > > >> > >>>>>>>>> * Pluggable metrics.
> > > >> > >>>>>>>>> * Pluggable deployment systems.
> > > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> > > SystemProducer,
> > > >> > >>>> etc).
> > > >> > >>>>>>>>> * Pluggable serdes.
> > > >> > >>>>>>>>> * Pluggable storage engines.
> > > >> > >>>>>>>>> * Pluggable strategies for just about every component
> > > >> > >>>>> (MessageChooser,
> > > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> There's probably more that I've forgotten, as well. Some
> > of
> > > >> > >>>>>>>>> these
> > > >> > >>>>> are
> > > >> > >>>>>>>>> useful, but some have proven not to be. This all comes
> at
> > a
> > > >> cost:
> > > >> > >>>>>>>>> complexity. This complexity is making it harder for our
> > > users
> > > >> > >>>>>>>>> to
> > > >> > >>>>> pick
> > > >> > >>>>>> up
> > > >> > >>>>>>>>> and use Samza out of the box. It also makes it difficult
> > for
> > > >> > >>>>>>>>> Samza developers to reason about what the
> characteristics
> > of
> > > >> > >>>>>>>>> the container (since the characteristics change
> depending
> > on
> > > >> > >>>>>>>>> which plugins are use).
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> The issues with pluggability are most visible in the
> > System
> > > >> APIs.
> > > >> > >>>>> What
> > > >> > >>>>>>>>> Samza really requires to be functional is Kafka as its
> > > >> > >>>>>>>>> transport
> > > >> > >>>>>> layer.
> > > >> > >>>>>>>>> But
> > > >> > >>>>>>>>> we've conflated two unrelated use cases into one API:
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > > >> > >>>>>>>>> 2. Process the data in Kafka.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> The current System API supports both of these use cases.
> > The
> > > >> > >>>>>>>>> problem
> > > >> > >>>>>> is,
> > > >> > >>>>>>>>> we
> > > >> > >>>>>>>>> actually want different features for each use case. By
> > > >> papering
> > > >> > >>>>>>>>> over
> > > >> > >>>>>>> these
> > > >> > >>>>>>>>> two use cases, and providing a single API, we've
> > introduced
> > > a
> > > >> > >>>>>>>>> ton of
> > > >> > >>>>>>> leaky
> > > >> > >>>>>>>>> abstractions.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> For example, what we'd really like in (2) is to have
> > > >> > >>>>>>>>> monotonically increasing longs for offsets (like Kafka).
> > > This
> > > >> > >>>>>>>>> would be at odds
> > > >> > >>>>> with
> > > >> > >>>>>>> (1),
> > > >> > >>>>>>>>> though, since different systems have different
> > > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > > >> > >>>>>>>>> There was discussion both on the mailing list and the
> SQL
> > > >> JIRAs
> > > >> > >>>>> about
> > > >> > >>>>>>> the
> > > >> > >>>>>>>>> need for this.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> The same thing holds true for replayability. Kafka
> allows
> > us
> > > >> to
> > > >> > >>>>> rewind
> > > >> > >>>>>>>>> when
> > > >> > >>>>>>>>> we have a failure. Many other systems don't. In some
> > cases,
> > > >> > >>>>>>>>> systems
> > > >> > >>>>>>> return
> > > >> > >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> > > because
> > > >> > >>>>>>>>> they
> > > >> > >>>>>> have
> > > >> > >>>>>>> no
> > > >> > >>>>>>>>> offsets.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Partitioning is another example. Kafka supports
> > > partitioning,
> > > >> > >>>>>>>>> but
> > > >> > >>>>> many
> > > >> > >>>>>>>>> systems don't. We model this by having a single
> partition
> > > for
> > > >> > >>>>>>>>> those systems. Still, other systems model partitioning
> > > >> > >>>> differently (e.g.
> > > >> > >>>>>>>>> Kinesis).
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating
> streams
> > > in
> > > >> a
> > > >> > >>>>>>>>> system-agnostic way is almost impossible. As is modeling
> > > >> > >>>>>>>>> metadata
> > > >> > >>>>> for
> > > >> > >>>>>>> the
> > > >> > >>>>>>>>> system (replication factor, partitions, location, etc).
> > The
> > > >> > >>>>>>>>> list
> > > >> > >>>>> goes
> > > >> > >>>>>>> on.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Duplicate work
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's
> consumer
> > > and
> > > >> > >>>>> producer
> > > >> > >>>>>>>>> APIs
> > > >> > >>>>>>>>> had a relatively weak feature set. On the consumer-side,
> > you
> > > >> > >>>>>>>>> had two
> > > >> > >>>>>>>>> options: use the high level consumer, or the simple
> > > consumer.
> > > >> > >>>>>>>>> The
> > > >> > >>>>>>> problem
> > > >> > >>>>>>>>> with the high-level consumer was that it controlled your
> > > >> > >>>>>>>>> offsets, partition assignments, and the order in which
> you
> > > >> > >>>>>>>>> received messages. The
> > > >> > >>>>> problem
> > > >> > >>>>>>>>> with
> > > >> > >>>>>>>>> the simple consumer is that it's not simple. It's basic.
> > You
> > > >> > >>>>>>>>> end up
> > > >> > >>>>>>> having
> > > >> > >>>>>>>>> to handle a lot of really low-level stuff that you
> > > shouldn't.
> > > >> > >>>>>>>>> We
> > > >> > >>>>>> spent a
> > > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> > robust.
> > > >> It
> > > >> > >>>>>>>>> also allows us to support some cool features:
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> * Per-partition message ordering and prioritization.
> > > >> > >>>>>>>>> * Tight control over partition assignment to support
> > joins,
> > > >> > >>>>>>>>> global
> > > >> > >>>>>> state
> > > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > > >> > >>>>>>>>> * Tight control over offset checkpointing.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> What we didn't realize at the time is that these
> features
> > > >> > >>>>>>>>> should
> > > >> > >>>>>>> actually
> > > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
> > stream
> > > >> > >>>>>> processors)
> > > >> > >>>>>>>>> end up wanting to do things like joins and partition
> > > >> > >>>>>>>>> assignment. The
> > > >> > >>>>>>> Kafka
> > > >> > >>>>>>>>> community has come to the same conclusion. They're
> adding
> > a
> > > >> ton
> > > >> > >>>>>>>>> of upgrades into their new Kafka consumer
> implementation.
> > > To a
> > > >> > >>>>>>>>> large extent,
> > > >> > >>>>> it's
> > > >> > >>>>>>>>> duplicate work to what we've already done in Samza.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
> > > approach
> > > >> > >>>>>>>>> to
> > > >> > >>>>>> Samza's
> > > >> > >>>>>>>>> KafkaCheckpointManager implementation for handling
> offset
> > > >> > >>>>>> checkpointing.
> > > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature stores
> > > >> offset
> > > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them
> from
> > > the
> > > >> > >>>>>>>>> broker.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> A lot of this seems like a waste, since we could have
> > shared
> > > >> > >>>>>>>>> the
> > > >> > >>>>> work
> > > >> > >>>>>> if
> > > >> > >>>>>>>>> it
> > > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Vision
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> All of this leads me to a rather radical proposal. Samza
> > is
> > > >> > >>>>> relatively
> > > >> > >>>>>>>>> stable at this point. I'd venture to say that we're
> near a
> > > 1.0
> > > >> > >>>>>> release.
> > > >> > >>>>>>>>> I'd
> > > >> > >>>>>>>>> like to propose that we take what we've learned, and
> begin
> > > >> > >>>>>>>>> thinking
> > > >> > >>>>>>> about
> > > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were
> starting
> > > >> from
> > > >> > >>>>>> scratch?
> > > >> > >>>>>>>>> My
> > > >> > >>>>>>>>> proposal is to:
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > > >> > >>>>>>>>> processors, and eliminate all direct dependences on
> YARN,
> > > >> Mesos,
> > > >> > >>>> etc.
> > > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as the
> > > stream
> > > >> > >>>>>> processing
> > > >> > >>>>>>>>> layer.
> > > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization,
> and
> > > >> > >>>>>>>>> config
> > > >> > >>>>>>> systems,
> > > >> > >>>>>>>>> and simply use Kafka's instead.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> This would fix all of the issues that I outlined above.
> It
> > > >> > >>>>>>>>> should
> > > >> > >>>>> also
> > > >> > >>>>>>>>> shrink the Samza code base pretty dramatically.
> Supporting
> > > >> only
> > > >> > >>>>>>>>> a standalone container will allow Samza to be executed
> on
> > > YARN
> > > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most
> > other
> > > >> > >>>>>>>>> in-house
> > > >> > >>>>>>> deployment
> > > >> > >>>>>>>>> systems. This should make life a lot easier for new
> users.
> > > >> > >>>>>>>>> Imagine
> > > >> > >>>>>>> having
> > > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in
> mailing
> > > >> list
> > > >> > >>>>>> traffic
> > > >> > >>>>>>>>> will be pretty dramatic.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The
> reality
> > > is,
> > > >> > >>>>> everyone
> > > >> > >>>>>>>>> that
> > > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically
> > require
> > > >> it
> > > >> > >>>>>> already
> > > >> > >>>>>>> in
> > > >> > >>>>>>>>> order for most features to work. Those that are using
> > other
> > > >> > >>>>>>>>> systems
> > > >> > >>>>>> are
> > > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and then
> > they
> > > do
> > > >> > >>>>>>>>> the processing on top. There is already discussion (
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>
> > > >> > >>>>>>
> > > >> > >>>>>
> > > >> >
> > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > >> > >>>>> 767
> > > >> > >>>>>>>>> )
> > > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Once we make the call to couple with Kafka, we can
> > leverage
> > > a
> > > >> > >>>>>>>>> ton of
> > > >> > >>>>>>> their
> > > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own config,
> > > >> > >>>>>>>>> metrics,
> > > >> > >>>>> etc.
> > > >> > >>>>>>> We
> > > >> > >>>>>>>>> can all share the same libraries, and make them better.
> > This
> > > >> > >>>>>>>>> will
> > > >> > >>>>> also
> > > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and will
> let
> > > us
> > > >> > >>>>> leverage
> > > >> > >>>>>>>>> their offset management and partition management, rather
> > > than
> > > >> > >>>>>>>>> having
> > > >> > >>>>>> our
> > > >> > >>>>>>>>> own. All of the coordinator stream code would go away,
> as
> > > >> would
> > > >> > >>>>>>>>> most
> > > >> > >>>>>> of
> > > >> > >>>>>>>>> the
> > > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
> > > partition
> > > >> > >>>>>>> management
> > > >> > >>>>>>>>> features into the Kafka broker, but they're already
> moving
> > > in
> > > >> > >>>>>>>>> that direction with the new consumer API. The features
> we
> > > have
> > > >> > >>>>>>>>> for
> > > >> > >>>>>> partition
> > > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like they
> > should
> > > >> be
> > > >> > >>>>>>>>> in
> > > >> > >>>>>> Kafka
> > > >> > >>>>>>>>> anyway. There will always be some niche usages which
> will
> > > >> > >>>>>>>>> require
> > > >> > >>>>>> extra
> > > >> > >>>>>>>>> care and hence full control over partition assignments
> > much
> > > >> > >>>>>>>>> like the
> > > >> > >>>>>>> Kafka
> > > >> > >>>>>>>>> low level consumer api. These would continue to be
> > > supported.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> These items will be good for the Samza community.
> They'll
> > > make
> > > >> > >>>>>>>>> Samza easier to use, and make it easier for developers
> to
> > > add
> > > >> > >>>>>>>>> new features.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat backwards
> > > >> > >>>>> incompatible
> > > >> > >>>>>>>>> change). If we choose to go this route, it's important
> > that
> > > we
> > > >> > >>>>> openly
> > > >> > >>>>>>>>> communicate how we're going to provide a migration path
> > from
> > > >> > >>>>>>>>> the
> > > >> > >>>>>>> existing
> > > >> > >>>>>>>>> APIs to the new ones (if we make incompatible changes).
> I
> > > >> think
> > > >> > >>>>>>>>> at a minimum, we'd probably need to provide a wrapper to
> > > allow
> > > >> > >>>>>>>>> existing StreamTask implementations to continue running
> on
> > > the
> > > >> > >>>> new container.
> > > >> > >>>>>>> It's
> > > >> > >>>>>>>>> also important that we openly communicate about timing,
> > and
> > > >> > >>>>>>>>> stages
> > > >> > >>>>> of
> > > >> > >>>>>>> the
> > > >> > >>>>>>>>> migration.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> If you made it this far, I'm sure you have opinions. :)
> > > Please
> > > >> > >>>>>>>>> send
> > > >> > >>>>>> your
> > > >> > >>>>>>>>> thoughts and feedback.
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>> Cheers,
> > > >> > >>>>>>>>> Chris
> > > >> > >>>>>>>>>
> > > >> > >>>>>>>>
> > > >> > >>>>>>>>
> > > >> > >>>>>>>
> > > >> > >>>>>>
> > > >> > >>>>>>
> > > >> > >>>>>>
> > > >> > >>>>>> --
> > > >> > >>>>>> -- Guozhang
> > > >> > >>>>>>
> > > >> > >>>>>
> > > >> > >>>>
> > > >> > >>
> > > >> > >>
> > > >> >
> > > >> >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

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

I couldn't agree more. We spent a bunch of time talking to people and that
is exactly the stuff we heard time and again. What makes it hard, of
course, is that there is some tension between compatibility with what's
there now and making things better for new users.

I also strongly agree with the importance of multi-language support. We are
talking now about Java, but for application development use cases people
want to work in whatever language they are using elsewhere. I think moving
to a model where Kafka itself does the group membership, lifecycle control,
and partition assignment has the advantage of putting all that complex
stuff behind a clean api that the clients are already going to be
implementing for their consumer, so the added functionality for stream
processing beyond a consumer becomes very minor.

-Jay

On Tue, Jul 7, 2015 at 10:49 AM, Roger Hoover <ro...@gmail.com>
wrote:

> Metamorphosis...nice. :)
>
> This has been a great discussion.  As a user of Samza who's recently
> integrated it into a relatively large organization, I just want to add
> support to a few points already made.
>
> The biggest hurdles to adoption of Samza as it currently exists that I've
> experienced are:
> 1) YARN - YARN is overly complex in many environments where Puppet would do
> just fine but it was the only mechanism to get fault tolerance.
> 2) Configuration - I think I like the idea of configuring most of the job
> in code rather than config files.  In general, I think the goal should be
> to make it harder to make mistakes, especially of the kind where the code
> expects something and the config doesn't match.  The current config is
> quite intricate and error-prone.  For example, the application logic may
> depend on bootstrapping a topic but rather than asserting that in the code,
> you have to rely on getting the config right.  Likewise with serdes, the
> Java representations produced by various serdes (JSON, Avro, etc.) are not
> equivalent so you cannot just reconfigure a serde without changing the
> code.   It would be nice for jobs to be able to assert what they expect
> from their input topics in terms of partitioning.  This is getting a little
> off topic but I was even thinking about creating a "Samza config linter"
> that would sanity check a set of configs.  Especially in organizations
> where config is managed by a different team than the application developer,
> it's very hard to get avoid config mistakes.
> 3) Java/Scala centric - for many teams (especially DevOps-type folks), the
> pain of the Java toolchain (maven, slow builds, weak command line support,
> configuration over convention) really inhibits productivity.  As more and
> more high-quality clients become available for Kafka, I hope they'll follow
> Samza's model.  Not sure how much it affects the proposals in this thread
> but please consider other languages in the ecosystem as well.  From what
> I've heard, Spark has more Python users than Java/Scala.
> (FYI, we added a Jython wrapper for the Samza API
>
> https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
> and are working on a Yeoman generator
> https://github.com/Quantiply/generator-rico for Jython/Samza projects to
> alleviate some of the pain)
>
> I also want to underscore Jay's point about improving the user experience.
> That's a very important factor for adoption.  I think the goal should be to
> make Samza as easy to get started with as something like Logstash.
> Logstash is vastly inferior in terms of capabilities to Samza but it's easy
> to get started and that makes a big difference.
>
> Cheers,
>
> Roger
>
>
>
>
>
> On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
> gdfm@apache.org> wrote:
>
> > Forgot to add. On the naming issues, Kafka Metamorphosis is a clear
> winner
> > :)
> >
> > --
> > Gianmarco
> >
> > On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <gdfm@apache.org
> >
> > wrote:
> >
> > > Hi,
> > >
> > > @Martin, thanks for you comments.
> > > Maybe I'm missing some important point, but I think coupling the
> releases
> > > is actually a *good* thing.
> > > To make an example, would it be better if the MR and HDFS components of
> > > Hadoop had different release schedules?
> > >
> > > Actually, keeping the discussion in a single place would make agreeing
> on
> > > releases (and backwards compatibility) much easier, as everybody would
> be
> > > responsible for the whole codebase.
> > >
> > > That said, I like the idea of absorbing samza-core as a sub-project,
> and
> > > leave the fancy stuff separate.
> > > It probably gives 90% of the benefits we have been discussing here.
> > >
> > > Cheers,
> > >
> > > --
> > > Gianmarco
> > >
> > > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
> > >
> > >> Hey Martin,
> > >>
> > >> I agree coupling release schedules is a downside.
> > >>
> > >> Definitely we can try to solve some of the integration problems in
> > >> Confluent Platform or in other distributions. But I think this ends up
> > >> being really shallow. I guess I feel to really get a good user
> > experience
> > >> the two systems have to kind of feel like part of the same thing and
> you
> > >> can't really add that in later--you can put both in the same
> > downloadable
> > >> tar file but it doesn't really give a very cohesive feeling. I agree
> > that
> > >> ultimately any of the project stuff is as much social and naming as
> > >> anything else--theoretically two totally independent projects could
> work
> > >> to
> > >> tightly align. In practice this seems to be quite difficult though.
> > >>
> > >> For the frameworks--totally agree it would be good to maintain the
> > >> framework support with the project. In some cases there may not be too
> > >> much
> > >> there since the integration gets lighter but I think whatever stubs
> you
> > >> need should be included. So no I definitely wasn't trying to imply
> > >> dropping
> > >> support for these frameworks, just making the integration lighter by
> > >> separating process management from partition management.
> > >>
> > >> You raise two good points we would have to figure out if we went down
> > the
> > >> alignment path:
> > >> 1. With respect to the name, yeah I think the first question is
> whether
> > >> some "re-branding" would be worth it. If so then I think we can have a
> > big
> > >> thread on the name. I'm definitely not set on Kafka Streaming or Kafka
> > >> Streams I was just using them to be kind of illustrative. I agree with
> > >> your
> > >> critique of these names, though I think people would get the idea.
> > >> 2. Yeah you also raise a good point about how to "factor" it. Here are
> > the
> > >> options I see (I could get enthusiastic about any of them):
> > >>    a. One repo for both Kafka and Samza
> > >>    b. Two repos, retaining the current seperation
> > >>    c. Two repos, the equivalent of samza-api and samza-core is
> absorbed
> > >> almost like a third client
> > >>
> > >> Cheers,
> > >>
> > >> -Jay
> > >>
> > >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <
> martin@kleppmann.com>
> > >> wrote:
> > >>
> > >> > Ok, thanks for the clarifications. Just a few follow-up comments.
> > >> >
> > >> > - I see the appeal of merging with Kafka or becoming a subproject:
> the
> > >> > reasons you mention are good. The risk I see is that release
> schedules
> > >> > become coupled to each other, which can slow everyone down, and
> large
> > >> > projects with many contributors are harder to manage. (Jakob, can
> you
> > >> speak
> > >> > from experience, having seen a wider range of Hadoop ecosystem
> > >> projects?)
> > >> >
> > >> > Some of the goals of a better unified developer experience could
> also
> > be
> > >> > solved by integrating Samza nicely into a Kafka distribution (such
> as
> > >> > Confluent's). I'm not against merging projects if we decide that's
> the
> > >> way
> > >> > to go, just pointing out the same goals can perhaps also be achieved
> > in
> > >> > other ways.
> > >> >
> > >> > - With regard to dropping the YARN dependency: are you proposing
> that
> > >> > Samza doesn't give any help to people wanting to run on
> > >> YARN/Mesos/AWS/etc?
> > >> > So the docs would basically have a link to Slider and nothing else?
> Or
> > >> > would we maintain integrations with a bunch of popular deployment
> > >> methods
> > >> > (e.g. the necessary glue and shell scripts to make Samza work with
> > >> Slider)?
> > >> >
> > >> > I absolutely think it's a good idea to have the "as a library" and
> > "as a
> > >> > process" (using Yi's taxonomy) options for people who want them,
> but I
> > >> > think there should also be a low-friction path for common "as a
> > service"
> > >> > deployment methods, for which we probably need to maintain
> > integrations.
> > >> >
> > >> > - Project naming: "Kafka Streams" seems odd to me, because Kafka is
> > all
> > >> > about streams already. Perhaps "Kafka Transformers" or "Kafka
> Filters"
> > >> > would be more apt?
> > >> >
> > >> > One suggestion: perhaps the core of Samza (stream transformation
> with
> > >> > state management -- i.e. the "Samza as a library" bit) could become
> > >> part of
> > >> > Kafka, while higher-level tools such as streaming SQL and
> integrations
> > >> with
> > >> > deployment frameworks remain in a separate project? In other words,
> > >> Kafka
> > >> > would absorb the proven, stable core of Samza, which would become
> the
> > >> > "third Kafka client" mentioned early in this thread. The Samza
> project
> > >> > would then target that third Kafka client as its base API, and the
> > >> project
> > >> > would be freed up to explore more experimental new horizons.
> > >> >
> > >> > Martin
> > >> >
> > >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
> > >> >
> > >> > > Hey Martin,
> > >> > >
> > >> > > For the YARN/Mesos/etc decoupling I actually don't think it ties
> our
> > >> > hands
> > >> > > at all, all it does is refactor things. The division of
> > >> responsibility is
> > >> > > that Samza core is responsible for task lifecycle, state, and
> > >> partition
> > >> > > management (using the Kafka co-ordinator) but it is NOT
> responsible
> > >> for
> > >> > > packaging, configuration deployment or execution of processes. The
> > >> > problem
> > >> > > of packaging and starting these processes is
> > >> > > framework/environment-specific. This leaves individual frameworks
> to
> > >> be
> > >> > as
> > >> > > fancy or vanilla as they like. So you can get simple stateless
> > >> support in
> > >> > > YARN, Mesos, etc using their off-the-shelf app framework (Slider,
> > >> > Marathon,
> > >> > > etc). These are well known by people and have nice UIs and a lot
> of
> > >> > > flexibility. I don't think they have node affinity as a built in
> > >> option
> > >> > > (though I could be wrong). So if we want that we can either wait
> for
> > >> them
> > >> > > to add it or do a custom framework to add that feature (as now).
> > >> > Obviously
> > >> > > if you manage things with old-school ops tools (puppet/chef/etc)
> you
> > >> get
> > >> > > locality easily. The nice thing, though, is that all the samza
> > >> "business
> > >> > > logic" around partition management and fault tolerance is in Samza
> > >> core
> > >> > so
> > >> > > it is shared across frameworks and the framework specific bit is
> > just
> > >> > > whether it is smart enough to try to get the same host when a job
> is
> > >> > > restarted.
> > >> > >
> > >> > > With respect to the Kafka-alignment, yeah I think the goal would
> be
> > >> (a)
> > >> > > actually get better alignment in user experience, and (b) express
> > >> this in
> > >> > > the naming and project branding. Specifically:
> > >> > > 1. Website/docs, it would be nice for the "transformation" api to
> be
> > >> > > discoverable in the main Kafka docs--i.e. be able to explain when
> to
> > >> use
> > >> > > the consumer and when to use the stream processing functionality
> and
> > >> lead
> > >> > > people into that experience.
> > >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that
> has
> > >> both
> > >> > > Kafka and the stream processing part and they actually work
> > together.
> > >> > > 3. Unify the programming experience so the client and Samza api
> > share
> > >> > > config/monitoring/naming/packaging/etc.
> > >> > >
> > >> > > I think sub-projects keep separate committers and can have a
> > separate
> > >> > repo,
> > >> > > but I'm actually not really sure (I can't find a definition of a
> > >> > subproject
> > >> > > in Apache).
> > >> > >
> > >> > > Basically at a high-level you want the experience to "feel" like a
> > >> single
> > >> > > system, not to relatively independent things that are kind of
> > >> awkwardly
> > >> > > glued together.
> > >> > >
> > >> > > I think if we did that they having naming or branding like "kafka
> > >> > > streaming" or "kafka streams" or something like that would
> actually
> > >> do a
> > >> > > good job of conveying what it is. I do that this would help
> adoption
> > >> > quite
> > >> > > a lot as it would correctly convey that using Kafka Streaming with
> > >> Kafka
> > >> > is
> > >> > > a fairly seamless experience and Kafka is pretty heavily adopted
> at
> > >> this
> > >> > > point.
> > >> > >
> > >> > > Fwiw we actually considered this model originally when open
> sourcing
> > >> > Samza,
> > >> > > however at that time Kafka was relatively unknown and we decided
> not
> > >> to
> > >> > do
> > >> > > it since we felt it would be limiting. From my point of view the
> > three
> > >> > > things have changed (1) Kafka is now really heavily used for
> stream
> > >> > > processing, (2) we learned that abstracting out the stream well is
> > >> > > basically impossible, (3) we learned it is really hard to keep the
> > two
> > >> > > things feeling like a single product.
> > >> > >
> > >> > > -Jay
> > >> > >
> > >> > >
> > >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> > >> martin@kleppmann.com>
> > >> > > wrote:
> > >> > >
> > >> > >> Hi all,
> > >> > >>
> > >> > >> Lots of good thoughts here.
> > >> > >>
> > >> > >> I agree with the general philosophy of tying Samza more firmly to
> > >> Kafka.
> > >> > >> After I spent a while looking at integrating other message
> brokers
> > >> (e.g.
> > >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
> > >> > SystemConsumer
> > >> > >> tacitly assumes a model so much like Kafka's that pretty much
> > nobody
> > >> but
> > >> > >> Kafka actually implements it. (Databus is perhaps an exception,
> but
> > >> it
> > >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza fully
> > >> > dependent
> > >> > >> on Kafka acknowledges that the system-independence was never as
> > real
> > >> as
> > >> > we
> > >> > >> perhaps made it out to be. The gains of code reuse are real.
> > >> > >>
> > >> > >> The idea of decoupling Samza from YARN has also always been
> > >> appealing to
> > >> > >> me, for various reasons already mentioned in this thread.
> Although
> > >> > making
> > >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems
> > >> laudable,
> > >> > I am
> > >> > >> a little concerned that it will restrict us to a lowest common
> > >> > denominator.
> > >> > >> For example, would host affinity (SAMZA-617) still be possible?
> For
> > >> jobs
> > >> > >> with large amounts of state, I think SAMZA-617 would be a big
> boon,
> > >> > since
> > >> > >> restoring state off the changelog on every single restart is
> > painful,
> > >> > due
> > >> > >> to long recovery times. It would be a shame if the decoupling
> from
> > >> YARN
> > >> > >> made host affinity impossible.
> > >> > >>
> > >> > >> Jay, a question about the proposed API for instantiating a job in
> > >> code
> > >> > >> (rather than a properties file): when submitting a job to a
> > cluster,
> > >> is
> > >> > the
> > >> > >> idea that the instantiation code runs on a client somewhere,
> which
> > >> then
> > >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that
> > >> code
> > >> > run
> > >> > >> on each container that is part of the job (in which case, how
> does
> > >> the
> > >> > job
> > >> > >> submission to the cluster work)?
> > >> > >>
> > >> > >> I agree with Garry that it doesn't feel right to make a 1.0
> release
> > >> > with a
> > >> > >> plan for it to be immediately obsolete. So if this is going to
> > >> happen, I
> > >> > >> think it would be more honest to stick with 0.* version numbers
> > until
> > >> > the
> > >> > >> library-ified Samza has been implemented, is stable and widely
> > used.
> > >> > >>
> > >> > >> Should the new Samza be a subproject of Kafka? There is precedent
> > for
> > >> > >> tight coupling between different Apache projects (e.g. Curator
> and
> > >> > >> Zookeeper, or Slider and YARN), so I think remaining separate
> would
> > >> be
> > >> > ok.
> > >> > >> Even if Samza is fully dependent on Kafka, there is enough
> > substance
> > >> in
> > >> > >> Samza that it warrants being a separate project. An argument in
> > >> favour
> > >> > of
> > >> > >> merging would be if we think Kafka has a much stronger "brand
> > >> presence"
> > >> > >> than Samza; I'm ambivalent on that one. If the Kafka project is
> > >> willing
> > >> > to
> > >> > >> endorse Samza as the "official" way of doing stateful stream
> > >> > >> transformations, that would probably have much the same effect as
> > >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
> > >> > >> collaboration between the two projects will be needed in any
> case.
> > >> > >>
> > >> > >> From a project management perspective, I guess the "new Samza"
> > would
> > >> > have
> > >> > >> to be developed on a branch alongside ongoing maintenance of the
> > >> current
> > >> > >> line of development? I think it would be important to continue
> > >> > supporting
> > >> > >> existing users, and provide a graceful migration path to the new
> > >> > version.
> > >> > >> Leaving the current versions unsupported and forcing people to
> > >> rewrite
> > >> > >> their jobs would send a bad signal.
> > >> > >>
> > >> > >> Best,
> > >> > >> Martin
> > >> > >>
> > >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
> > >> > >>
> > >> > >>> Hey Garry,
> > >> > >>>
> > >> > >>> Yeah that's super frustrating. I'd be happy to chat more about
> > this
> > >> if
> > >> > >>> you'd be interested. I think Chris and I started with the idea
> of
> > >> "what
> > >> > >>> would it take to make Samza a kick-ass ingestion tool" but
> > >> ultimately
> > >> > we
> > >> > >>> kind of came around to the idea that ingestion and
> transformation
> > >> had
> > >> > >>> pretty different needs and coupling the two made things hard.
> > >> > >>>
> > >> > >>> For what it's worth I think copycat (KIP-26) actually will do
> what
> > >> you
> > >> > >> are
> > >> > >>> looking for.
> > >> > >>>
> > >> > >>> With regard to your point about slider, I don't necessarily
> > >> disagree.
> > >> > >> But I
> > >> > >>> think getting good YARN support is quite doable and I think we
> can
> > >> make
> > >> > >>> that work well. I think the issue this proposal solves is that
> > >> > >> technically
> > >> > >>> it is pretty hard to support multiple cluster management systems
> > the
> > >> > way
> > >> > >>> things are now, you need to write an "app master" or "framework"
> > for
> > >> > each
> > >> > >>> and they are all a little different so testing is really hard.
> In
> > >> the
> > >> > >>> absence of this we have been stuck with just YARN which has
> > >> fantastic
> > >> > >>> penetration in the Hadoopy part of the org, but zero penetration
> > >> > >> elsewhere.
> > >> > >>> Given the huge amount of work being put in to slider, marathon,
> > aws
> > >> > >>> tooling, not to mention the umpteen related packaging
> technologies
> > >> > people
> > >> > >>> want to use (Docker, Kubernetes, various cloud-specific deploy
> > >> tools,
> > >> > >> etc)
> > >> > >>> I really think it is important to get this right.
> > >> > >>>
> > >> > >>> -Jay
> > >> > >>>
> > >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > >> > >>> g.turkington@improvedigital.com> wrote:
> > >> > >>>
> > >> > >>>> Hi all,
> > >> > >>>>
> > >> > >>>> I think the question below re does Samza become a sub-project
> of
> > >> Kafka
> > >> > >>>> highlights the broader point around migration. Chris mentions
> > >> Samza's
> > >> > >>>> maturity is heading towards a v1 release but I'm not sure it
> > feels
> > >> > >> right to
> > >> > >>>> launch a v1 then immediately plan to deprecate most of it.
> > >> > >>>>
> > >> > >>>> From a selfish perspective I have some guys who have started
> > >> working
> > >> > >> with
> > >> > >>>> Samza and building some new consumers/producers was next up.
> > Sounds
> > >> > like
> > >> > >>>> that is absolutely not the direction to go. I need to look into
> > the
> > >> > KIP
> > >> > >> in
> > >> > >>>> more detail but for me the attractiveness of adding new Samza
> > >> > >>>> consumer/producers -- even if yes all they were doing was
> really
> > >> > getting
> > >> > >>>> data into and out of Kafka --  was to avoid  having to worry
> > about
> > >> the
> > >> > >>>> lifecycle management of external clients. If there is a generic
> > >> Kafka
> > >> > >>>> ingress/egress layer that I can plug a new connector into and
> > have
> > >> a
> > >> > >> lot of
> > >> > >>>> the heavy lifting re scale and reliability done for me then it
> > >> gives
> > >> > me
> > >> > >> all
> > >> > >>>> the pushing new consumers/producers would. If not then it
> > >> complicates
> > >> > my
> > >> > >>>> operational deployments.
> > >> > >>>>
> > >> > >>>> Which is similar to my other question with the proposal -- if
> we
> > >> > build a
> > >> > >>>> fully available/stand-alone Samza plus the requisite shims to
> > >> > integrate
> > >> > >>>> with Slider etc I suspect the former may be a lot more work
> than
> > we
> > >> > >> think.
> > >> > >>>> We may make it much easier for a newcomer to get something
> > running
> > >> but
> > >> > >>>> having them step up and get a reliable production deployment
> may
> > >> still
> > >> > >>>> dominate mailing list  traffic, if for different reasons than
> > >> today.
> > >> > >>>>
> > >> > >>>> Don't get me wrong -- I'm comfortable with making the Samza
> > >> dependency
> > >> > >> on
> > >> > >>>> Kafka much more explicit and I absolutely see the benefits  in
> > the
> > >> > >>>> reduction of duplication and clashing
> terminologies/abstractions
> > >> that
> > >> > >>>> Chris/Jay describe. Samza as a library would likely be a very
> > nice
> > >> > tool
> > >> > >> to
> > >> > >>>> add to the Kafka ecosystem. I just have the concerns above re
> the
> > >> > >>>> operational side.
> > >> > >>>>
> > >> > >>>> Garry
> > >> > >>>>
> > >> > >>>> -----Original Message-----
> > >> > >>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> > >> > >>>> Sent: 02 July 2015 12:56
> > >> > >>>> To: dev@samza.apache.org
> > >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> > >> > >>>>
> > >> > >>>> Very interesting thoughts.
> > >> > >>>> From outside, I have always perceived Samza as a computing
> layer
> > >> over
> > >> > >>>> Kafka.
> > >> > >>>>
> > >> > >>>> The question, maybe a bit provocative, is "should Samza be a
> > >> > sub-project
> > >> > >>>> of Kafka then?"
> > >> > >>>> Or does it make sense to keep it as a separate project with a
> > >> separate
> > >> > >>>> governance?
> > >> > >>>>
> > >> > >>>> Cheers,
> > >> > >>>>
> > >> > >>>> --
> > >> > >>>> Gianmarco
> > >> > >>>>
> > >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> wrote:
> > >> > >>>>
> > >> > >>>>> Overall, I agree to couple with Kafka more tightly. Because
> > Samza
> > >> de
> > >> > >>>>> facto is based on Kafka, and it should leverage what Kafka
> has.
> > At
> > >> > the
> > >> > >>>>> same time, Kafka does not need to reinvent what Samza already
> > >> has. I
> > >> > >>>>> also like the idea of separating the ingestion and
> > transformation.
> > >> > >>>>>
> > >> > >>>>> But it is a little difficult for me to image how the Samza
> will
> > >> look
> > >> > >>>> like.
> > >> > >>>>> And I feel Chris and Jay have a little difference in terms of
> > how
> > >> > >>>>> Samza should look like.
> > >> > >>>>>
> > >> > >>>>> *** Will it look like what Jay's code shows (A client of
> Kakfa)
> > ?
> > >> And
> > >> > >>>>> user's application code calls this client?
> > >> > >>>>>
> > >> > >>>>> 1. If we make Samza be a library of Kafka (like what the code
> > >> shows),
> > >> > >>>>> how do we implement auto-balance and fault-tolerance? Are they
> > >> taken
> > >> > >>>>> care by the Kafka broker or other mechanism, such as "Samza
> > >> worker"
> > >> > >>>>> (just make up the name) ?
> > >> > >>>>>
> > >> > >>>>> 2. What about other features, such as auto-scaling, shared
> > state,
> > >> > >>>>> monitoring?
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> *** If we have Samza standalone, (is this what Chris
> suggests?)
> > >> > >>>>>
> > >> > >>>>> 1. we still need to ingest data from Kakfa and produce to it.
> > >> Then it
> > >> > >>>>> becomes the same as what Samza looks like now, except it does
> > not
> > >> > rely
> > >> > >>>>> on Yarn anymore.
> > >> > >>>>>
> > >> > >>>>> 2. if it is standalone, how can it leverage Kafka's metrics,
> > logs,
> > >> > >>>>> etc? Use Kafka code as the dependency?
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> Thanks,
> > >> > >>>>>
> > >> > >>>>> Fang, Yan
> > >> > >>>>> yanfang724@gmail.com
> > >> > >>>>>
> > >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > wangguoz@gmail.com
> > >> >
> > >> > >>>> wrote:
> > >> > >>>>>
> > >> > >>>>>> Read through the code example and it looks good to me. A few
> > >> > >>>>>> thoughts regarding deployment:
> > >> > >>>>>>
> > >> > >>>>>> Today Samza deploys as executable runnable like:
> > >> > >>>>>>
> > >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> > >> > >>>> --config-path=file://...
> > >> > >>>>>>
> > >> > >>>>>> And this proposal advocate for deploying Samza more as
> embedded
> > >> > >>>>>> libraries in user application code (ignoring the terminology
> > >> since
> > >> > >>>>>> it is not the
> > >> > >>>>> same
> > >> > >>>>>> as the prototype code):
> > >> > >>>>>>
> > >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread thread =
> > new
> > >> > >>>>>> Thread(task); thread.start();
> > >> > >>>>>>
> > >> > >>>>>> I think both of these deployment modes are important for
> > >> different
> > >> > >>>>>> types
> > >> > >>>>> of
> > >> > >>>>>> users. That said, I think making Samza purely standalone is
> > still
> > >> > >>>>>> sufficient for either runnable or library modes.
> > >> > >>>>>>
> > >> > >>>>>> Guozhang
> > >> > >>>>>>
> > >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> jay@confluent.io>
> > >> > wrote:
> > >> > >>>>>>
> > >> > >>>>>>> Looks like gmail mangled the code example, it was supposed
> to
> > >> look
> > >> > >>>>>>> like
> > >> > >>>>>>> this:
> > >> > >>>>>>>
> > >> > >>>>>>> Properties props = new Properties();
> > >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > >> StreamingConfig
> > >> > >>>>>>> config = new StreamingConfig(props);
> > >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> > >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> > >> > >>>>>>> config.serialization(new StringSerializer(), new
> > >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
> > >> > >>>>>>> KafkaStreaming(config); container.run();
> > >> > >>>>>>>
> > >> > >>>>>>> -Jay
> > >> > >>>>>>>
> > >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> jay@confluent.io
> > >
> > >> > >>>> wrote:
> > >> > >>>>>>>
> > >> > >>>>>>>> Hey guys,
> > >> > >>>>>>>>
> > >> > >>>>>>>> This came out of some conversations Chris and I were having
> > >> > >>>>>>>> around
> > >> > >>>>>>> whether
> > >> > >>>>>>>> it would make sense to use Samza as a kind of data
> ingestion
> > >> > >>>>> framework
> > >> > >>>>>>> for
> > >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This
> kind
> > of
> > >> > >>>>>> combined
> > >> > >>>>>>>> with complaints around config and YARN and the discussion
> > >> around
> > >> > >>>>>>>> how
> > >> > >>>>> to
> > >> > >>>>>>>> best do a standalone mode.
> > >> > >>>>>>>>
> > >> > >>>>>>>> So the thought experiment was, given that Samza was
> basically
> > >> > >>>>>>>> already totally Kafka specific, what if you just embraced
> > that
> > >> > >>>>>>>> and turned it
> > >> > >>>>>> into
> > >> > >>>>>>>> something less like a heavyweight framework and more like a
> > >> > >>>>>>>> third
> > >> > >>>>> Kafka
> > >> > >>>>>>>> client--a kind of "producing consumer" with state
> management
> > >> > >>>>>> facilities.
> > >> > >>>>>>>> Basically a library. Instead of a complex stream processing
> > >> > >>>>>>>> framework
> > >> > >>>>>>> this
> > >> > >>>>>>>> would actually be a very simple thing, not much more
> > >> complicated
> > >> > >>>>>>>> to
> > >> > >>>>> use
> > >> > >>>>>>> or
> > >> > >>>>>>>> operate than a Kafka consumer. As Chris said we thought
> about
> > >> it
> > >> > >>>>>>>> a
> > >> > >>>>> lot
> > >> > >>>>>> of
> > >> > >>>>>>>> what Samza (and the other stream processing systems were
> > doing)
> > >> > >>>>> seemed
> > >> > >>>>>>> like
> > >> > >>>>>>>> kind of a hangover from MapReduce.
> > >> > >>>>>>>>
> > >> > >>>>>>>> Of course you need to ingest/output data to and from the
> > stream
> > >> > >>>>>>>> processing. But when we actually looked into how that would
> > >> > >>>>>>>> work,
> > >> > >>>>> Samza
> > >> > >>>>>>>> isn't really an ideal data ingestion framework for a bunch
> of
> > >> > >>>>> reasons.
> > >> > >>>>>> To
> > >> > >>>>>>>> really do that right you need a pretty different internal
> > data
> > >> > >>>>>>>> model
> > >> > >>>>>> and
> > >> > >>>>>>>> set of apis. So what if you split them and had an api for
> > Kafka
> > >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
> > >> Kafka
> > >> > >>>>>>>> transformation (Samza).
> > >> > >>>>>>>>
> > >> > >>>>>>>> This would also allow really embracing the same terminology
> > and
> > >> > >>>>>>>> conventions. One complaint about the current state is that
> > the
> > >> > >>>>>>>> two
> > >> > >>>>>>> systems
> > >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs
> "topic"
> > >> and
> > >> > >>>>>>> different
> > >> > >>>>>>>> config and monitoring systems means you kind of have to
> learn
> > >> > >>>>>>>> Kafka's
> > >> > >>>>>>> way,
> > >> > >>>>>>>> then learn Samza's slightly different way, then kind of
> > >> > >>>>>>>> understand
> > >> > >>>>> how
> > >> > >>>>>>> they
> > >> > >>>>>>>> map to each other, which having walked a few people through
> > >> this
> > >> > >>>>>>>> is surprisingly tricky for folks to get.
> > >> > >>>>>>>>
> > >> > >>>>>>>> Since I have been spending a lot of time on airplanes I
> > hacked
> > >> > >>>>>>>> up an ernest but still somewhat incomplete prototype of
> what
> > >> > >>>>>>>> this would
> > >> > >>>>> look
> > >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka as it
> > >> > >>>>>>>> required a
> > >> > >>>>>> few
> > >> > >>>>>>>> changes to the new consumer. Here is the code:
> > >> > >>>>>>>>
> > >> > >>>>>>>>
> > >> > >>>>>>>
> > >> > >>>>>>
> > >> > >>>>>
> > >> >
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >> > >>>>> /apache/kafka/clients/streaming
> > >> > >>>>>>>>
> > >> > >>>>>>>> For the purpose of the prototype I just liberally renamed
> > >> > >>>>>>>> everything
> > >> > >>>>> to
> > >> > >>>>>>>> try to align it with Kafka with no regard for
> compatibility.
> > >> > >>>>>>>>
> > >> > >>>>>>>> To use this would be something like this:
> > >> > >>>>>>>> Properties props = new Properties();
> > >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > >> > >>>>>>>> StreamingConfig config = new
> > >> > >>>>> StreamingConfig(props);
> > >> > >>>>>>> config.subscribe("test-topic-1",
> > >> > >>>>>>>> "test-topic-2");
> > >> config.processor(ExampleStreamProcessor.class);
> > >> > >>>>>>> config.serialization(new
> > >> > >>>>>>>> StringSerializer(), new StringDeserializer());
> KafkaStreaming
> > >> > >>>>>> container =
> > >> > >>>>>>>> new KafkaStreaming(config); container.run();
> > >> > >>>>>>>>
> > >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> > StreamProcessor
> > >> > >>>>>>>> is basically StreamTask.
> > >> > >>>>>>>>
> > >> > >>>>>>>> So rather than putting all the class names in a file and
> then
> > >> > >>>>>>>> having
> > >> > >>>>>> the
> > >> > >>>>>>>> job assembled by reflection, you just instantiate the
> > container
> > >> > >>>>>>>> programmatically. Work is balanced over however many
> > instances
> > >> > >>>>>>>> of
> > >> > >>>>> this
> > >> > >>>>>>> are
> > >> > >>>>>>>> alive at any time (i.e. if an instance dies, new tasks are
> > >> added
> > >> > >>>>>>>> to
> > >> > >>>>> the
> > >> > >>>>>>>> existing containers without shutting them down).
> > >> > >>>>>>>>
> > >> > >>>>>>>> We would provide some glue for running this stuff in YARN
> via
> > >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their
> tools
> > >> > >>>>>>>> but from the
> > >> > >>>>>> point
> > >> > >>>>>>> of
> > >> > >>>>>>>> view of these frameworks these stream processing jobs are
> > just
> > >> > >>>>>> stateless
> > >> > >>>>>>>> services that can come and go and expand and contract at
> > will.
> > >> > >>>>>>>> There
> > >> > >>>>> is
> > >> > >>>>>>> no
> > >> > >>>>>>>> more custom scheduler.
> > >> > >>>>>>>>
> > >> > >>>>>>>> Here are some relevant details:
> > >> > >>>>>>>>
> > >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get larger if
> we
> > >> > >>>>>>>>  productionized but not vastly larger. We really do get a
> ton
> > >> > >>>>>>>> of
> > >> > >>>>>>> leverage
> > >> > >>>>>>>>  out of Kafka.
> > >> > >>>>>>>>  2. Partition management is fully delegated to the new
> > >> consumer.
> > >> > >>>>> This
> > >> > >>>>>>>>  is nice since now any partition management strategy
> > available
> > >> > >>>>>>>> to
> > >> > >>>>>> Kafka
> > >> > >>>>>>>>  consumer is also available to Samza (and vice versa) and
> > with
> > >> > >>>>>>>> the
> > >> > >>>>>>> exact
> > >> > >>>>>>>>  same configs.
> > >> > >>>>>>>>  3. It supports state as well as state reuse
> > >> > >>>>>>>>
> > >> > >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
> > >> > >>>>>>>>
> > >> > >>>>>>>> -Jay
> > >> > >>>>>>>>
> > >> > >>>>>>>>
> > >> > >>>>>>>>
> > >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > >> > >>>>>> criccomini@apache.org>
> > >> > >>>>>>>> wrote:
> > >> > >>>>>>>>
> > >> > >>>>>>>>> Hey all,
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> I have had some discussions with Samza engineers at
> LinkedIn
> > >> > >>>>>>>>> and
> > >> > >>>>>>> Confluent
> > >> > >>>>>>>>> and we came up with a few observations and would like to
> > >> > >>>>>>>>> propose
> > >> > >>>>> some
> > >> > >>>>>>>>> changes.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> We've observed some things that I want to call out about
> > >> > >>>>>>>>> Samza's
> > >> > >>>>>> design,
> > >> > >>>>>>>>> and I'd like to propose some changes.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
> > >> > >>>>>>>>> * Samza is too pluggable.
> > >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> consumer
> > >> > >>>>>>>>> APIs
> > >> > >>>>> are
> > >> > >>>>>>>>> trying to solve a lot of the same problems.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> All three of these issues are related, but I'll address
> them
> > >> in
> > >> > >>>>> order.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Deployment
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Samza strongly depends on the use of a dynamic deployment
> > >> > >>>>>>>>> scheduler
> > >> > >>>>>> such
> > >> > >>>>>>>>> as
> > >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet
> that
> > >> > >>>>>>>>> there
> > >> > >>>>>> would
> > >> > >>>>>>>>> be
> > >> > >>>>>>>>> one or two winners in this area, and we could support
> them,
> > >> and
> > >> > >>>>>>>>> the
> > >> > >>>>>> rest
> > >> > >>>>>>>>> would go away. In reality, there are many variations.
> > >> > >>>>>>>>> Furthermore,
> > >> > >>>>>> many
> > >> > >>>>>>>>> people still prefer to just start their processors like
> > normal
> > >> > >>>>>>>>> Java processes, and use traditional deployment scripts
> such
> > as
> > >> > >>>>>>>>> Fabric,
> > >> > >>>>>> Chef,
> > >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users makes
> the
> > >> > >>>>>>>>> Samza start-up process really painful for first time
> users.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit of a
> > >> > >>>>>>>>> mis-fire
> > >> > >>>>>> because
> > >> > >>>>>>>>> of
> > >> > >>>>>>>>> a fundamental misunderstanding between the nature of batch
> > >> jobs
> > >> > >>>>>>>>> and
> > >> > >>>>>>> stream
> > >> > >>>>>>>>> processing jobs. Early on, we made conscious effort to
> favor
> > >> > >>>>>>>>> the
> > >> > >>>>>> Hadoop
> > >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked and was
> > well
> > >> > >>>>>>> understood.
> > >> > >>>>>>>>> One thing that we missed was that batch jobs have a
> definite
> > >> > >>>>>> beginning,
> > >> > >>>>>>>>> and
> > >> > >>>>>>>>> end, and stream processing jobs don't (usually). This
> leads
> > to
> > >> > >>>>>>>>> a
> > >> > >>>>> much
> > >> > >>>>>>>>> simpler scheduling problem for stream processors. You
> > >> basically
> > >> > >>>>>>>>> just
> > >> > >>>>>>> need
> > >> > >>>>>>>>> to find a place to start the processor, and start it. The
> > way
> > >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> > >> > >>>>>>>>> being "full". We always
> > >> > >>>>>> add
> > >> > >>>>>>>>> more machines. The problem with coupling Samza with a
> > >> scheduler
> > >> > >>>>>>>>> is
> > >> > >>>>>> that
> > >> > >>>>>>>>> Samza (as a framework) now has to handle deployment. This
> > >> pulls
> > >> > >>>>>>>>> in a
> > >> > >>>>>>> bunch
> > >> > >>>>>>>>> of things such as configuration distribution (config
> > stream),
> > >> > >>>>>>>>> shell
> > >> > >>>>>>> scrips
> > >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> stuff),
> > >> etc.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Another reason for requiring dynamic deployment was to
> > support
> > >> > >>>>>>>>> data locality. If you want to have locality, you need to
> put
> > >> > >>>>>>>>> your
> > >> > >>>>>> processors
> > >> > >>>>>>>>> close to the data they're processing. Upon further
> > >> > >>>>>>>>> investigation,
> > >> > >>>>>>> though,
> > >> > >>>>>>>>> this feature is not that beneficial. There is some good
> > >> > >>>>>>>>> discussion
> > >> > >>>>>> about
> > >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
> > >> > >>>>>>>>> Map/Reduce
> > >> > >>>>>> path,
> > >> > >>>>>>>>> but
> > >> > >>>>>>>>> there are some fundamental differences between HDFS and
> > Kafka.
> > >> > >>>>>>>>> HDFS
> > >> > >>>>>> has
> > >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to less
> > >> > >>>>>>>>> optimization potential with stream processors on top of
> > Kafka.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't have
> > any
> > >> > >>>>>>>>> built
> > >> > >>>>> in
> > >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> > >> > >>>>>>>>> deployment scheduling system to handle restarts when a
> > >> > >>>>>>>>> processor dies. This has
> > >> > >>>>>>> made
> > >> > >>>>>>>>> it very difficult to write a standalone Samza container
> > >> > >>>> (SAMZA-516).
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Pluggability
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> In some cases pluggability is good, but I think that we've
> > >> gone
> > >> > >>>>>>>>> too
> > >> > >>>>>> far
> > >> > >>>>>>>>> with it. Currently, Samza has:
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> * Pluggable config.
> > >> > >>>>>>>>> * Pluggable metrics.
> > >> > >>>>>>>>> * Pluggable deployment systems.
> > >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> > SystemProducer,
> > >> > >>>> etc).
> > >> > >>>>>>>>> * Pluggable serdes.
> > >> > >>>>>>>>> * Pluggable storage engines.
> > >> > >>>>>>>>> * Pluggable strategies for just about every component
> > >> > >>>>> (MessageChooser,
> > >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> There's probably more that I've forgotten, as well. Some
> of
> > >> > >>>>>>>>> these
> > >> > >>>>> are
> > >> > >>>>>>>>> useful, but some have proven not to be. This all comes at
> a
> > >> cost:
> > >> > >>>>>>>>> complexity. This complexity is making it harder for our
> > users
> > >> > >>>>>>>>> to
> > >> > >>>>> pick
> > >> > >>>>>> up
> > >> > >>>>>>>>> and use Samza out of the box. It also makes it difficult
> for
> > >> > >>>>>>>>> Samza developers to reason about what the characteristics
> of
> > >> > >>>>>>>>> the container (since the characteristics change depending
> on
> > >> > >>>>>>>>> which plugins are use).
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> The issues with pluggability are most visible in the
> System
> > >> APIs.
> > >> > >>>>> What
> > >> > >>>>>>>>> Samza really requires to be functional is Kafka as its
> > >> > >>>>>>>>> transport
> > >> > >>>>>> layer.
> > >> > >>>>>>>>> But
> > >> > >>>>>>>>> we've conflated two unrelated use cases into one API:
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> 1. Get data into/out of Kafka.
> > >> > >>>>>>>>> 2. Process the data in Kafka.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> The current System API supports both of these use cases.
> The
> > >> > >>>>>>>>> problem
> > >> > >>>>>> is,
> > >> > >>>>>>>>> we
> > >> > >>>>>>>>> actually want different features for each use case. By
> > >> papering
> > >> > >>>>>>>>> over
> > >> > >>>>>>> these
> > >> > >>>>>>>>> two use cases, and providing a single API, we've
> introduced
> > a
> > >> > >>>>>>>>> ton of
> > >> > >>>>>>> leaky
> > >> > >>>>>>>>> abstractions.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> For example, what we'd really like in (2) is to have
> > >> > >>>>>>>>> monotonically increasing longs for offsets (like Kafka).
> > This
> > >> > >>>>>>>>> would be at odds
> > >> > >>>>> with
> > >> > >>>>>>> (1),
> > >> > >>>>>>>>> though, since different systems have different
> > >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > >> > >>>>>>>>> There was discussion both on the mailing list and the SQL
> > >> JIRAs
> > >> > >>>>> about
> > >> > >>>>>>> the
> > >> > >>>>>>>>> need for this.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> The same thing holds true for replayability. Kafka allows
> us
> > >> to
> > >> > >>>>> rewind
> > >> > >>>>>>>>> when
> > >> > >>>>>>>>> we have a failure. Many other systems don't. In some
> cases,
> > >> > >>>>>>>>> systems
> > >> > >>>>>>> return
> > >> > >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> > because
> > >> > >>>>>>>>> they
> > >> > >>>>>> have
> > >> > >>>>>>> no
> > >> > >>>>>>>>> offsets.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Partitioning is another example. Kafka supports
> > partitioning,
> > >> > >>>>>>>>> but
> > >> > >>>>> many
> > >> > >>>>>>>>> systems don't. We model this by having a single partition
> > for
> > >> > >>>>>>>>> those systems. Still, other systems model partitioning
> > >> > >>>> differently (e.g.
> > >> > >>>>>>>>> Kinesis).
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating streams
> > in
> > >> a
> > >> > >>>>>>>>> system-agnostic way is almost impossible. As is modeling
> > >> > >>>>>>>>> metadata
> > >> > >>>>> for
> > >> > >>>>>>> the
> > >> > >>>>>>>>> system (replication factor, partitions, location, etc).
> The
> > >> > >>>>>>>>> list
> > >> > >>>>> goes
> > >> > >>>>>>> on.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Duplicate work
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> At the time that we began writing Samza, Kafka's consumer
> > and
> > >> > >>>>> producer
> > >> > >>>>>>>>> APIs
> > >> > >>>>>>>>> had a relatively weak feature set. On the consumer-side,
> you
> > >> > >>>>>>>>> had two
> > >> > >>>>>>>>> options: use the high level consumer, or the simple
> > consumer.
> > >> > >>>>>>>>> The
> > >> > >>>>>>> problem
> > >> > >>>>>>>>> with the high-level consumer was that it controlled your
> > >> > >>>>>>>>> offsets, partition assignments, and the order in which you
> > >> > >>>>>>>>> received messages. The
> > >> > >>>>> problem
> > >> > >>>>>>>>> with
> > >> > >>>>>>>>> the simple consumer is that it's not simple. It's basic.
> You
> > >> > >>>>>>>>> end up
> > >> > >>>>>>> having
> > >> > >>>>>>>>> to handle a lot of really low-level stuff that you
> > shouldn't.
> > >> > >>>>>>>>> We
> > >> > >>>>>> spent a
> > >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> robust.
> > >> It
> > >> > >>>>>>>>> also allows us to support some cool features:
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> * Per-partition message ordering and prioritization.
> > >> > >>>>>>>>> * Tight control over partition assignment to support
> joins,
> > >> > >>>>>>>>> global
> > >> > >>>>>> state
> > >> > >>>>>>>>> (if we want to implement it :)), etc.
> > >> > >>>>>>>>> * Tight control over offset checkpointing.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> What we didn't realize at the time is that these features
> > >> > >>>>>>>>> should
> > >> > >>>>>>> actually
> > >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
> stream
> > >> > >>>>>> processors)
> > >> > >>>>>>>>> end up wanting to do things like joins and partition
> > >> > >>>>>>>>> assignment. The
> > >> > >>>>>>> Kafka
> > >> > >>>>>>>>> community has come to the same conclusion. They're adding
> a
> > >> ton
> > >> > >>>>>>>>> of upgrades into their new Kafka consumer implementation.
> > To a
> > >> > >>>>>>>>> large extent,
> > >> > >>>>> it's
> > >> > >>>>>>>>> duplicate work to what we've already done in Samza.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
> > approach
> > >> > >>>>>>>>> to
> > >> > >>>>>> Samza's
> > >> > >>>>>>>>> KafkaCheckpointManager implementation for handling offset
> > >> > >>>>>> checkpointing.
> > >> > >>>>>>>>> Like Samza, Kafka's new offset management feature stores
> > >> offset
> > >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them from
> > the
> > >> > >>>>>>>>> broker.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> A lot of this seems like a waste, since we could have
> shared
> > >> > >>>>>>>>> the
> > >> > >>>>> work
> > >> > >>>>>> if
> > >> > >>>>>>>>> it
> > >> > >>>>>>>>> had been done in Kafka from the get-go.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Vision
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> All of this leads me to a rather radical proposal. Samza
> is
> > >> > >>>>> relatively
> > >> > >>>>>>>>> stable at this point. I'd venture to say that we're near a
> > 1.0
> > >> > >>>>>> release.
> > >> > >>>>>>>>> I'd
> > >> > >>>>>>>>> like to propose that we take what we've learned, and begin
> > >> > >>>>>>>>> thinking
> > >> > >>>>>>> about
> > >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were starting
> > >> from
> > >> > >>>>>> scratch?
> > >> > >>>>>>>>> My
> > >> > >>>>>>>>> proposal is to:
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > >> > >>>>>>>>> processors, and eliminate all direct dependences on YARN,
> > >> Mesos,
> > >> > >>>> etc.
> > >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as the
> > stream
> > >> > >>>>>> processing
> > >> > >>>>>>>>> layer.
> > >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> > >> > >>>>>>>>> config
> > >> > >>>>>>> systems,
> > >> > >>>>>>>>> and simply use Kafka's instead.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> This would fix all of the issues that I outlined above. It
> > >> > >>>>>>>>> should
> > >> > >>>>> also
> > >> > >>>>>>>>> shrink the Samza code base pretty dramatically. Supporting
> > >> only
> > >> > >>>>>>>>> a standalone container will allow Samza to be executed on
> > YARN
> > >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most
> other
> > >> > >>>>>>>>> in-house
> > >> > >>>>>>> deployment
> > >> > >>>>>>>>> systems. This should make life a lot easier for new users.
> > >> > >>>>>>>>> Imagine
> > >> > >>>>>>> having
> > >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in mailing
> > >> list
> > >> > >>>>>> traffic
> > >> > >>>>>>>>> will be pretty dramatic.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The reality
> > is,
> > >> > >>>>> everyone
> > >> > >>>>>>>>> that
> > >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically
> require
> > >> it
> > >> > >>>>>> already
> > >> > >>>>>>> in
> > >> > >>>>>>>>> order for most features to work. Those that are using
> other
> > >> > >>>>>>>>> systems
> > >> > >>>>>> are
> > >> > >>>>>>>>> generally using it for ingest into Kafka (1), and then
> they
> > do
> > >> > >>>>>>>>> the processing on top. There is already discussion (
> > >> > >>>>>>>>>
> > >> > >>>>>>>
> > >> > >>>>>>
> > >> > >>>>>
> > >> >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >> > >>>>> 767
> > >> > >>>>>>>>> )
> > >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Once we make the call to couple with Kafka, we can
> leverage
> > a
> > >> > >>>>>>>>> ton of
> > >> > >>>>>>> their
> > >> > >>>>>>>>> ecosystem. We no longer have to maintain our own config,
> > >> > >>>>>>>>> metrics,
> > >> > >>>>> etc.
> > >> > >>>>>>> We
> > >> > >>>>>>>>> can all share the same libraries, and make them better.
> This
> > >> > >>>>>>>>> will
> > >> > >>>>> also
> > >> > >>>>>>>>> allow us to share the consumer/producer APIs, and will let
> > us
> > >> > >>>>> leverage
> > >> > >>>>>>>>> their offset management and partition management, rather
> > than
> > >> > >>>>>>>>> having
> > >> > >>>>>> our
> > >> > >>>>>>>>> own. All of the coordinator stream code would go away, as
> > >> would
> > >> > >>>>>>>>> most
> > >> > >>>>>> of
> > >> > >>>>>>>>> the
> > >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
> > partition
> > >> > >>>>>>> management
> > >> > >>>>>>>>> features into the Kafka broker, but they're already moving
> > in
> > >> > >>>>>>>>> that direction with the new consumer API. The features we
> > have
> > >> > >>>>>>>>> for
> > >> > >>>>>> partition
> > >> > >>>>>>>>> assignment aren't unique to Samza, and seem like they
> should
> > >> be
> > >> > >>>>>>>>> in
> > >> > >>>>>> Kafka
> > >> > >>>>>>>>> anyway. There will always be some niche usages which will
> > >> > >>>>>>>>> require
> > >> > >>>>>> extra
> > >> > >>>>>>>>> care and hence full control over partition assignments
> much
> > >> > >>>>>>>>> like the
> > >> > >>>>>>> Kafka
> > >> > >>>>>>>>> low level consumer api. These would continue to be
> > supported.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> These items will be good for the Samza community. They'll
> > make
> > >> > >>>>>>>>> Samza easier to use, and make it easier for developers to
> > add
> > >> > >>>>>>>>> new features.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Obviously this is a fairly large (and somewhat backwards
> > >> > >>>>> incompatible
> > >> > >>>>>>>>> change). If we choose to go this route, it's important
> that
> > we
> > >> > >>>>> openly
> > >> > >>>>>>>>> communicate how we're going to provide a migration path
> from
> > >> > >>>>>>>>> the
> > >> > >>>>>>> existing
> > >> > >>>>>>>>> APIs to the new ones (if we make incompatible changes). I
> > >> think
> > >> > >>>>>>>>> at a minimum, we'd probably need to provide a wrapper to
> > allow
> > >> > >>>>>>>>> existing StreamTask implementations to continue running on
> > the
> > >> > >>>> new container.
> > >> > >>>>>>> It's
> > >> > >>>>>>>>> also important that we openly communicate about timing,
> and
> > >> > >>>>>>>>> stages
> > >> > >>>>> of
> > >> > >>>>>>> the
> > >> > >>>>>>>>> migration.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> If you made it this far, I'm sure you have opinions. :)
> > Please
> > >> > >>>>>>>>> send
> > >> > >>>>>> your
> > >> > >>>>>>>>> thoughts and feedback.
> > >> > >>>>>>>>>
> > >> > >>>>>>>>> Cheers,
> > >> > >>>>>>>>> Chris
> > >> > >>>>>>>>>
> > >> > >>>>>>>>
> > >> > >>>>>>>>
> > >> > >>>>>>>
> > >> > >>>>>>
> > >> > >>>>>>
> > >> > >>>>>>
> > >> > >>>>>> --
> > >> > >>>>>> -- Guozhang
> > >> > >>>>>>
> > >> > >>>>>
> > >> > >>>>
> > >> > >>
> > >> > >>
> > >> >
> > >> >
> > >> >
> > >>
> > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Roger Hoover <ro...@gmail.com>.
Metamorphosis...nice. :)

This has been a great discussion.  As a user of Samza who's recently
integrated it into a relatively large organization, I just want to add
support to a few points already made.

The biggest hurdles to adoption of Samza as it currently exists that I've
experienced are:
1) YARN - YARN is overly complex in many environments where Puppet would do
just fine but it was the only mechanism to get fault tolerance.
2) Configuration - I think I like the idea of configuring most of the job
in code rather than config files.  In general, I think the goal should be
to make it harder to make mistakes, especially of the kind where the code
expects something and the config doesn't match.  The current config is
quite intricate and error-prone.  For example, the application logic may
depend on bootstrapping a topic but rather than asserting that in the code,
you have to rely on getting the config right.  Likewise with serdes, the
Java representations produced by various serdes (JSON, Avro, etc.) are not
equivalent so you cannot just reconfigure a serde without changing the
code.   It would be nice for jobs to be able to assert what they expect
from their input topics in terms of partitioning.  This is getting a little
off topic but I was even thinking about creating a "Samza config linter"
that would sanity check a set of configs.  Especially in organizations
where config is managed by a different team than the application developer,
it's very hard to get avoid config mistakes.
3) Java/Scala centric - for many teams (especially DevOps-type folks), the
pain of the Java toolchain (maven, slow builds, weak command line support,
configuration over convention) really inhibits productivity.  As more and
more high-quality clients become available for Kafka, I hope they'll follow
Samza's model.  Not sure how much it affects the proposals in this thread
but please consider other languages in the ecosystem as well.  From what
I've heard, Spark has more Python users than Java/Scala.
(FYI, we added a Jython wrapper for the Samza API
https://github.com/Quantiply/rico/tree/master/jython/src/main/java/com/quantiply/samza
and are working on a Yeoman generator
https://github.com/Quantiply/generator-rico for Jython/Samza projects to
alleviate some of the pain)

I also want to underscore Jay's point about improving the user experience.
That's a very important factor for adoption.  I think the goal should be to
make Samza as easy to get started with as something like Logstash.
Logstash is vastly inferior in terms of capabilities to Samza but it's easy
to get started and that makes a big difference.

Cheers,

Roger





On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
gdfm@apache.org> wrote:

> Forgot to add. On the naming issues, Kafka Metamorphosis is a clear winner
> :)
>
> --
> Gianmarco
>
> On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <gd...@apache.org>
> wrote:
>
> > Hi,
> >
> > @Martin, thanks for you comments.
> > Maybe I'm missing some important point, but I think coupling the releases
> > is actually a *good* thing.
> > To make an example, would it be better if the MR and HDFS components of
> > Hadoop had different release schedules?
> >
> > Actually, keeping the discussion in a single place would make agreeing on
> > releases (and backwards compatibility) much easier, as everybody would be
> > responsible for the whole codebase.
> >
> > That said, I like the idea of absorbing samza-core as a sub-project, and
> > leave the fancy stuff separate.
> > It probably gives 90% of the benefits we have been discussing here.
> >
> > Cheers,
> >
> > --
> > Gianmarco
> >
> > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
> >
> >> Hey Martin,
> >>
> >> I agree coupling release schedules is a downside.
> >>
> >> Definitely we can try to solve some of the integration problems in
> >> Confluent Platform or in other distributions. But I think this ends up
> >> being really shallow. I guess I feel to really get a good user
> experience
> >> the two systems have to kind of feel like part of the same thing and you
> >> can't really add that in later--you can put both in the same
> downloadable
> >> tar file but it doesn't really give a very cohesive feeling. I agree
> that
> >> ultimately any of the project stuff is as much social and naming as
> >> anything else--theoretically two totally independent projects could work
> >> to
> >> tightly align. In practice this seems to be quite difficult though.
> >>
> >> For the frameworks--totally agree it would be good to maintain the
> >> framework support with the project. In some cases there may not be too
> >> much
> >> there since the integration gets lighter but I think whatever stubs you
> >> need should be included. So no I definitely wasn't trying to imply
> >> dropping
> >> support for these frameworks, just making the integration lighter by
> >> separating process management from partition management.
> >>
> >> You raise two good points we would have to figure out if we went down
> the
> >> alignment path:
> >> 1. With respect to the name, yeah I think the first question is whether
> >> some "re-branding" would be worth it. If so then I think we can have a
> big
> >> thread on the name. I'm definitely not set on Kafka Streaming or Kafka
> >> Streams I was just using them to be kind of illustrative. I agree with
> >> your
> >> critique of these names, though I think people would get the idea.
> >> 2. Yeah you also raise a good point about how to "factor" it. Here are
> the
> >> options I see (I could get enthusiastic about any of them):
> >>    a. One repo for both Kafka and Samza
> >>    b. Two repos, retaining the current seperation
> >>    c. Two repos, the equivalent of samza-api and samza-core is absorbed
> >> almost like a third client
> >>
> >> Cheers,
> >>
> >> -Jay
> >>
> >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <ma...@kleppmann.com>
> >> wrote:
> >>
> >> > Ok, thanks for the clarifications. Just a few follow-up comments.
> >> >
> >> > - I see the appeal of merging with Kafka or becoming a subproject: the
> >> > reasons you mention are good. The risk I see is that release schedules
> >> > become coupled to each other, which can slow everyone down, and large
> >> > projects with many contributors are harder to manage. (Jakob, can you
> >> speak
> >> > from experience, having seen a wider range of Hadoop ecosystem
> >> projects?)
> >> >
> >> > Some of the goals of a better unified developer experience could also
> be
> >> > solved by integrating Samza nicely into a Kafka distribution (such as
> >> > Confluent's). I'm not against merging projects if we decide that's the
> >> way
> >> > to go, just pointing out the same goals can perhaps also be achieved
> in
> >> > other ways.
> >> >
> >> > - With regard to dropping the YARN dependency: are you proposing that
> >> > Samza doesn't give any help to people wanting to run on
> >> YARN/Mesos/AWS/etc?
> >> > So the docs would basically have a link to Slider and nothing else? Or
> >> > would we maintain integrations with a bunch of popular deployment
> >> methods
> >> > (e.g. the necessary glue and shell scripts to make Samza work with
> >> Slider)?
> >> >
> >> > I absolutely think it's a good idea to have the "as a library" and
> "as a
> >> > process" (using Yi's taxonomy) options for people who want them, but I
> >> > think there should also be a low-friction path for common "as a
> service"
> >> > deployment methods, for which we probably need to maintain
> integrations.
> >> >
> >> > - Project naming: "Kafka Streams" seems odd to me, because Kafka is
> all
> >> > about streams already. Perhaps "Kafka Transformers" or "Kafka Filters"
> >> > would be more apt?
> >> >
> >> > One suggestion: perhaps the core of Samza (stream transformation with
> >> > state management -- i.e. the "Samza as a library" bit) could become
> >> part of
> >> > Kafka, while higher-level tools such as streaming SQL and integrations
> >> with
> >> > deployment frameworks remain in a separate project? In other words,
> >> Kafka
> >> > would absorb the proven, stable core of Samza, which would become the
> >> > "third Kafka client" mentioned early in this thread. The Samza project
> >> > would then target that third Kafka client as its base API, and the
> >> project
> >> > would be freed up to explore more experimental new horizons.
> >> >
> >> > Martin
> >> >
> >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
> >> >
> >> > > Hey Martin,
> >> > >
> >> > > For the YARN/Mesos/etc decoupling I actually don't think it ties our
> >> > hands
> >> > > at all, all it does is refactor things. The division of
> >> responsibility is
> >> > > that Samza core is responsible for task lifecycle, state, and
> >> partition
> >> > > management (using the Kafka co-ordinator) but it is NOT responsible
> >> for
> >> > > packaging, configuration deployment or execution of processes. The
> >> > problem
> >> > > of packaging and starting these processes is
> >> > > framework/environment-specific. This leaves individual frameworks to
> >> be
> >> > as
> >> > > fancy or vanilla as they like. So you can get simple stateless
> >> support in
> >> > > YARN, Mesos, etc using their off-the-shelf app framework (Slider,
> >> > Marathon,
> >> > > etc). These are well known by people and have nice UIs and a lot of
> >> > > flexibility. I don't think they have node affinity as a built in
> >> option
> >> > > (though I could be wrong). So if we want that we can either wait for
> >> them
> >> > > to add it or do a custom framework to add that feature (as now).
> >> > Obviously
> >> > > if you manage things with old-school ops tools (puppet/chef/etc) you
> >> get
> >> > > locality easily. The nice thing, though, is that all the samza
> >> "business
> >> > > logic" around partition management and fault tolerance is in Samza
> >> core
> >> > so
> >> > > it is shared across frameworks and the framework specific bit is
> just
> >> > > whether it is smart enough to try to get the same host when a job is
> >> > > restarted.
> >> > >
> >> > > With respect to the Kafka-alignment, yeah I think the goal would be
> >> (a)
> >> > > actually get better alignment in user experience, and (b) express
> >> this in
> >> > > the naming and project branding. Specifically:
> >> > > 1. Website/docs, it would be nice for the "transformation" api to be
> >> > > discoverable in the main Kafka docs--i.e. be able to explain when to
> >> use
> >> > > the consumer and when to use the stream processing functionality and
> >> lead
> >> > > people into that experience.
> >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that has
> >> both
> >> > > Kafka and the stream processing part and they actually work
> together.
> >> > > 3. Unify the programming experience so the client and Samza api
> share
> >> > > config/monitoring/naming/packaging/etc.
> >> > >
> >> > > I think sub-projects keep separate committers and can have a
> separate
> >> > repo,
> >> > > but I'm actually not really sure (I can't find a definition of a
> >> > subproject
> >> > > in Apache).
> >> > >
> >> > > Basically at a high-level you want the experience to "feel" like a
> >> single
> >> > > system, not to relatively independent things that are kind of
> >> awkwardly
> >> > > glued together.
> >> > >
> >> > > I think if we did that they having naming or branding like "kafka
> >> > > streaming" or "kafka streams" or something like that would actually
> >> do a
> >> > > good job of conveying what it is. I do that this would help adoption
> >> > quite
> >> > > a lot as it would correctly convey that using Kafka Streaming with
> >> Kafka
> >> > is
> >> > > a fairly seamless experience and Kafka is pretty heavily adopted at
> >> this
> >> > > point.
> >> > >
> >> > > Fwiw we actually considered this model originally when open sourcing
> >> > Samza,
> >> > > however at that time Kafka was relatively unknown and we decided not
> >> to
> >> > do
> >> > > it since we felt it would be limiting. From my point of view the
> three
> >> > > things have changed (1) Kafka is now really heavily used for stream
> >> > > processing, (2) we learned that abstracting out the stream well is
> >> > > basically impossible, (3) we learned it is really hard to keep the
> two
> >> > > things feeling like a single product.
> >> > >
> >> > > -Jay
> >> > >
> >> > >
> >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> >> martin@kleppmann.com>
> >> > > wrote:
> >> > >
> >> > >> Hi all,
> >> > >>
> >> > >> Lots of good thoughts here.
> >> > >>
> >> > >> I agree with the general philosophy of tying Samza more firmly to
> >> Kafka.
> >> > >> After I spent a while looking at integrating other message brokers
> >> (e.g.
> >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
> >> > SystemConsumer
> >> > >> tacitly assumes a model so much like Kafka's that pretty much
> nobody
> >> but
> >> > >> Kafka actually implements it. (Databus is perhaps an exception, but
> >> it
> >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza fully
> >> > dependent
> >> > >> on Kafka acknowledges that the system-independence was never as
> real
> >> as
> >> > we
> >> > >> perhaps made it out to be. The gains of code reuse are real.
> >> > >>
> >> > >> The idea of decoupling Samza from YARN has also always been
> >> appealing to
> >> > >> me, for various reasons already mentioned in this thread. Although
> >> > making
> >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems
> >> laudable,
> >> > I am
> >> > >> a little concerned that it will restrict us to a lowest common
> >> > denominator.
> >> > >> For example, would host affinity (SAMZA-617) still be possible? For
> >> jobs
> >> > >> with large amounts of state, I think SAMZA-617 would be a big boon,
> >> > since
> >> > >> restoring state off the changelog on every single restart is
> painful,
> >> > due
> >> > >> to long recovery times. It would be a shame if the decoupling from
> >> YARN
> >> > >> made host affinity impossible.
> >> > >>
> >> > >> Jay, a question about the proposed API for instantiating a job in
> >> code
> >> > >> (rather than a properties file): when submitting a job to a
> cluster,
> >> is
> >> > the
> >> > >> idea that the instantiation code runs on a client somewhere, which
> >> then
> >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that
> >> code
> >> > run
> >> > >> on each container that is part of the job (in which case, how does
> >> the
> >> > job
> >> > >> submission to the cluster work)?
> >> > >>
> >> > >> I agree with Garry that it doesn't feel right to make a 1.0 release
> >> > with a
> >> > >> plan for it to be immediately obsolete. So if this is going to
> >> happen, I
> >> > >> think it would be more honest to stick with 0.* version numbers
> until
> >> > the
> >> > >> library-ified Samza has been implemented, is stable and widely
> used.
> >> > >>
> >> > >> Should the new Samza be a subproject of Kafka? There is precedent
> for
> >> > >> tight coupling between different Apache projects (e.g. Curator and
> >> > >> Zookeeper, or Slider and YARN), so I think remaining separate would
> >> be
> >> > ok.
> >> > >> Even if Samza is fully dependent on Kafka, there is enough
> substance
> >> in
> >> > >> Samza that it warrants being a separate project. An argument in
> >> favour
> >> > of
> >> > >> merging would be if we think Kafka has a much stronger "brand
> >> presence"
> >> > >> than Samza; I'm ambivalent on that one. If the Kafka project is
> >> willing
> >> > to
> >> > >> endorse Samza as the "official" way of doing stateful stream
> >> > >> transformations, that would probably have much the same effect as
> >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
> >> > >> collaboration between the two projects will be needed in any case.
> >> > >>
> >> > >> From a project management perspective, I guess the "new Samza"
> would
> >> > have
> >> > >> to be developed on a branch alongside ongoing maintenance of the
> >> current
> >> > >> line of development? I think it would be important to continue
> >> > supporting
> >> > >> existing users, and provide a graceful migration path to the new
> >> > version.
> >> > >> Leaving the current versions unsupported and forcing people to
> >> rewrite
> >> > >> their jobs would send a bad signal.
> >> > >>
> >> > >> Best,
> >> > >> Martin
> >> > >>
> >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
> >> > >>
> >> > >>> Hey Garry,
> >> > >>>
> >> > >>> Yeah that's super frustrating. I'd be happy to chat more about
> this
> >> if
> >> > >>> you'd be interested. I think Chris and I started with the idea of
> >> "what
> >> > >>> would it take to make Samza a kick-ass ingestion tool" but
> >> ultimately
> >> > we
> >> > >>> kind of came around to the idea that ingestion and transformation
> >> had
> >> > >>> pretty different needs and coupling the two made things hard.
> >> > >>>
> >> > >>> For what it's worth I think copycat (KIP-26) actually will do what
> >> you
> >> > >> are
> >> > >>> looking for.
> >> > >>>
> >> > >>> With regard to your point about slider, I don't necessarily
> >> disagree.
> >> > >> But I
> >> > >>> think getting good YARN support is quite doable and I think we can
> >> make
> >> > >>> that work well. I think the issue this proposal solves is that
> >> > >> technically
> >> > >>> it is pretty hard to support multiple cluster management systems
> the
> >> > way
> >> > >>> things are now, you need to write an "app master" or "framework"
> for
> >> > each
> >> > >>> and they are all a little different so testing is really hard. In
> >> the
> >> > >>> absence of this we have been stuck with just YARN which has
> >> fantastic
> >> > >>> penetration in the Hadoopy part of the org, but zero penetration
> >> > >> elsewhere.
> >> > >>> Given the huge amount of work being put in to slider, marathon,
> aws
> >> > >>> tooling, not to mention the umpteen related packaging technologies
> >> > people
> >> > >>> want to use (Docker, Kubernetes, various cloud-specific deploy
> >> tools,
> >> > >> etc)
> >> > >>> I really think it is important to get this right.
> >> > >>>
> >> > >>> -Jay
> >> > >>>
> >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> >> > >>> g.turkington@improvedigital.com> wrote:
> >> > >>>
> >> > >>>> Hi all,
> >> > >>>>
> >> > >>>> I think the question below re does Samza become a sub-project of
> >> Kafka
> >> > >>>> highlights the broader point around migration. Chris mentions
> >> Samza's
> >> > >>>> maturity is heading towards a v1 release but I'm not sure it
> feels
> >> > >> right to
> >> > >>>> launch a v1 then immediately plan to deprecate most of it.
> >> > >>>>
> >> > >>>> From a selfish perspective I have some guys who have started
> >> working
> >> > >> with
> >> > >>>> Samza and building some new consumers/producers was next up.
> Sounds
> >> > like
> >> > >>>> that is absolutely not the direction to go. I need to look into
> the
> >> > KIP
> >> > >> in
> >> > >>>> more detail but for me the attractiveness of adding new Samza
> >> > >>>> consumer/producers -- even if yes all they were doing was really
> >> > getting
> >> > >>>> data into and out of Kafka --  was to avoid  having to worry
> about
> >> the
> >> > >>>> lifecycle management of external clients. If there is a generic
> >> Kafka
> >> > >>>> ingress/egress layer that I can plug a new connector into and
> have
> >> a
> >> > >> lot of
> >> > >>>> the heavy lifting re scale and reliability done for me then it
> >> gives
> >> > me
> >> > >> all
> >> > >>>> the pushing new consumers/producers would. If not then it
> >> complicates
> >> > my
> >> > >>>> operational deployments.
> >> > >>>>
> >> > >>>> Which is similar to my other question with the proposal -- if we
> >> > build a
> >> > >>>> fully available/stand-alone Samza plus the requisite shims to
> >> > integrate
> >> > >>>> with Slider etc I suspect the former may be a lot more work than
> we
> >> > >> think.
> >> > >>>> We may make it much easier for a newcomer to get something
> running
> >> but
> >> > >>>> having them step up and get a reliable production deployment may
> >> still
> >> > >>>> dominate mailing list  traffic, if for different reasons than
> >> today.
> >> > >>>>
> >> > >>>> Don't get me wrong -- I'm comfortable with making the Samza
> >> dependency
> >> > >> on
> >> > >>>> Kafka much more explicit and I absolutely see the benefits  in
> the
> >> > >>>> reduction of duplication and clashing terminologies/abstractions
> >> that
> >> > >>>> Chris/Jay describe. Samza as a library would likely be a very
> nice
> >> > tool
> >> > >> to
> >> > >>>> add to the Kafka ecosystem. I just have the concerns above re the
> >> > >>>> operational side.
> >> > >>>>
> >> > >>>> Garry
> >> > >>>>
> >> > >>>> -----Original Message-----
> >> > >>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> >> > >>>> Sent: 02 July 2015 12:56
> >> > >>>> To: dev@samza.apache.org
> >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> >> > >>>>
> >> > >>>> Very interesting thoughts.
> >> > >>>> From outside, I have always perceived Samza as a computing layer
> >> over
> >> > >>>> Kafka.
> >> > >>>>
> >> > >>>> The question, maybe a bit provocative, is "should Samza be a
> >> > sub-project
> >> > >>>> of Kafka then?"
> >> > >>>> Or does it make sense to keep it as a separate project with a
> >> separate
> >> > >>>> governance?
> >> > >>>>
> >> > >>>> Cheers,
> >> > >>>>
> >> > >>>> --
> >> > >>>> Gianmarco
> >> > >>>>
> >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> >> > >>>>
> >> > >>>>> Overall, I agree to couple with Kafka more tightly. Because
> Samza
> >> de
> >> > >>>>> facto is based on Kafka, and it should leverage what Kafka has.
> At
> >> > the
> >> > >>>>> same time, Kafka does not need to reinvent what Samza already
> >> has. I
> >> > >>>>> also like the idea of separating the ingestion and
> transformation.
> >> > >>>>>
> >> > >>>>> But it is a little difficult for me to image how the Samza will
> >> look
> >> > >>>> like.
> >> > >>>>> And I feel Chris and Jay have a little difference in terms of
> how
> >> > >>>>> Samza should look like.
> >> > >>>>>
> >> > >>>>> *** Will it look like what Jay's code shows (A client of Kakfa)
> ?
> >> And
> >> > >>>>> user's application code calls this client?
> >> > >>>>>
> >> > >>>>> 1. If we make Samza be a library of Kafka (like what the code
> >> shows),
> >> > >>>>> how do we implement auto-balance and fault-tolerance? Are they
> >> taken
> >> > >>>>> care by the Kafka broker or other mechanism, such as "Samza
> >> worker"
> >> > >>>>> (just make up the name) ?
> >> > >>>>>
> >> > >>>>> 2. What about other features, such as auto-scaling, shared
> state,
> >> > >>>>> monitoring?
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> *** If we have Samza standalone, (is this what Chris suggests?)
> >> > >>>>>
> >> > >>>>> 1. we still need to ingest data from Kakfa and produce to it.
> >> Then it
> >> > >>>>> becomes the same as what Samza looks like now, except it does
> not
> >> > rely
> >> > >>>>> on Yarn anymore.
> >> > >>>>>
> >> > >>>>> 2. if it is standalone, how can it leverage Kafka's metrics,
> logs,
> >> > >>>>> etc? Use Kafka code as the dependency?
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Thanks,
> >> > >>>>>
> >> > >>>>> Fang, Yan
> >> > >>>>> yanfang724@gmail.com
> >> > >>>>>
> >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> wangguoz@gmail.com
> >> >
> >> > >>>> wrote:
> >> > >>>>>
> >> > >>>>>> Read through the code example and it looks good to me. A few
> >> > >>>>>> thoughts regarding deployment:
> >> > >>>>>>
> >> > >>>>>> Today Samza deploys as executable runnable like:
> >> > >>>>>>
> >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> >> > >>>> --config-path=file://...
> >> > >>>>>>
> >> > >>>>>> And this proposal advocate for deploying Samza more as embedded
> >> > >>>>>> libraries in user application code (ignoring the terminology
> >> since
> >> > >>>>>> it is not the
> >> > >>>>> same
> >> > >>>>>> as the prototype code):
> >> > >>>>>>
> >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread thread =
> new
> >> > >>>>>> Thread(task); thread.start();
> >> > >>>>>>
> >> > >>>>>> I think both of these deployment modes are important for
> >> different
> >> > >>>>>> types
> >> > >>>>> of
> >> > >>>>>> users. That said, I think making Samza purely standalone is
> still
> >> > >>>>>> sufficient for either runnable or library modes.
> >> > >>>>>>
> >> > >>>>>> Guozhang
> >> > >>>>>>
> >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
> >> > wrote:
> >> > >>>>>>
> >> > >>>>>>> Looks like gmail mangled the code example, it was supposed to
> >> look
> >> > >>>>>>> like
> >> > >>>>>>> this:
> >> > >>>>>>>
> >> > >>>>>>> Properties props = new Properties();
> >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
> >> StreamingConfig
> >> > >>>>>>> config = new StreamingConfig(props);
> >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> >> > >>>>>>> config.serialization(new StringSerializer(), new
> >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
> >> > >>>>>>> KafkaStreaming(config); container.run();
> >> > >>>>>>>
> >> > >>>>>>> -Jay
> >> > >>>>>>>
> >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <jay@confluent.io
> >
> >> > >>>> wrote:
> >> > >>>>>>>
> >> > >>>>>>>> Hey guys,
> >> > >>>>>>>>
> >> > >>>>>>>> This came out of some conversations Chris and I were having
> >> > >>>>>>>> around
> >> > >>>>>>> whether
> >> > >>>>>>>> it would make sense to use Samza as a kind of data ingestion
> >> > >>>>> framework
> >> > >>>>>>> for
> >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind
> of
> >> > >>>>>> combined
> >> > >>>>>>>> with complaints around config and YARN and the discussion
> >> around
> >> > >>>>>>>> how
> >> > >>>>> to
> >> > >>>>>>>> best do a standalone mode.
> >> > >>>>>>>>
> >> > >>>>>>>> So the thought experiment was, given that Samza was basically
> >> > >>>>>>>> already totally Kafka specific, what if you just embraced
> that
> >> > >>>>>>>> and turned it
> >> > >>>>>> into
> >> > >>>>>>>> something less like a heavyweight framework and more like a
> >> > >>>>>>>> third
> >> > >>>>> Kafka
> >> > >>>>>>>> client--a kind of "producing consumer" with state management
> >> > >>>>>> facilities.
> >> > >>>>>>>> Basically a library. Instead of a complex stream processing
> >> > >>>>>>>> framework
> >> > >>>>>>> this
> >> > >>>>>>>> would actually be a very simple thing, not much more
> >> complicated
> >> > >>>>>>>> to
> >> > >>>>> use
> >> > >>>>>>> or
> >> > >>>>>>>> operate than a Kafka consumer. As Chris said we thought about
> >> it
> >> > >>>>>>>> a
> >> > >>>>> lot
> >> > >>>>>> of
> >> > >>>>>>>> what Samza (and the other stream processing systems were
> doing)
> >> > >>>>> seemed
> >> > >>>>>>> like
> >> > >>>>>>>> kind of a hangover from MapReduce.
> >> > >>>>>>>>
> >> > >>>>>>>> Of course you need to ingest/output data to and from the
> stream
> >> > >>>>>>>> processing. But when we actually looked into how that would
> >> > >>>>>>>> work,
> >> > >>>>> Samza
> >> > >>>>>>>> isn't really an ideal data ingestion framework for a bunch of
> >> > >>>>> reasons.
> >> > >>>>>> To
> >> > >>>>>>>> really do that right you need a pretty different internal
> data
> >> > >>>>>>>> model
> >> > >>>>>> and
> >> > >>>>>>>> set of apis. So what if you split them and had an api for
> Kafka
> >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
> >> Kafka
> >> > >>>>>>>> transformation (Samza).
> >> > >>>>>>>>
> >> > >>>>>>>> This would also allow really embracing the same terminology
> and
> >> > >>>>>>>> conventions. One complaint about the current state is that
> the
> >> > >>>>>>>> two
> >> > >>>>>>> systems
> >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs "topic"
> >> and
> >> > >>>>>>> different
> >> > >>>>>>>> config and monitoring systems means you kind of have to learn
> >> > >>>>>>>> Kafka's
> >> > >>>>>>> way,
> >> > >>>>>>>> then learn Samza's slightly different way, then kind of
> >> > >>>>>>>> understand
> >> > >>>>> how
> >> > >>>>>>> they
> >> > >>>>>>>> map to each other, which having walked a few people through
> >> this
> >> > >>>>>>>> is surprisingly tricky for folks to get.
> >> > >>>>>>>>
> >> > >>>>>>>> Since I have been spending a lot of time on airplanes I
> hacked
> >> > >>>>>>>> up an ernest but still somewhat incomplete prototype of what
> >> > >>>>>>>> this would
> >> > >>>>> look
> >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka as it
> >> > >>>>>>>> required a
> >> > >>>>>> few
> >> > >>>>>>>> changes to the new consumer. Here is the code:
> >> > >>>>>>>>
> >> > >>>>>>>>
> >> > >>>>>>>
> >> > >>>>>>
> >> > >>>>>
> >> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >> > >>>>> /apache/kafka/clients/streaming
> >> > >>>>>>>>
> >> > >>>>>>>> For the purpose of the prototype I just liberally renamed
> >> > >>>>>>>> everything
> >> > >>>>> to
> >> > >>>>>>>> try to align it with Kafka with no regard for compatibility.
> >> > >>>>>>>>
> >> > >>>>>>>> To use this would be something like this:
> >> > >>>>>>>> Properties props = new Properties();
> >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> >> > >>>>>>>> StreamingConfig config = new
> >> > >>>>> StreamingConfig(props);
> >> > >>>>>>> config.subscribe("test-topic-1",
> >> > >>>>>>>> "test-topic-2");
> >> config.processor(ExampleStreamProcessor.class);
> >> > >>>>>>> config.serialization(new
> >> > >>>>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> >> > >>>>>> container =
> >> > >>>>>>>> new KafkaStreaming(config); container.run();
> >> > >>>>>>>>
> >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> StreamProcessor
> >> > >>>>>>>> is basically StreamTask.
> >> > >>>>>>>>
> >> > >>>>>>>> So rather than putting all the class names in a file and then
> >> > >>>>>>>> having
> >> > >>>>>> the
> >> > >>>>>>>> job assembled by reflection, you just instantiate the
> container
> >> > >>>>>>>> programmatically. Work is balanced over however many
> instances
> >> > >>>>>>>> of
> >> > >>>>> this
> >> > >>>>>>> are
> >> > >>>>>>>> alive at any time (i.e. if an instance dies, new tasks are
> >> added
> >> > >>>>>>>> to
> >> > >>>>> the
> >> > >>>>>>>> existing containers without shutting them down).
> >> > >>>>>>>>
> >> > >>>>>>>> We would provide some glue for running this stuff in YARN via
> >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> >> > >>>>>>>> but from the
> >> > >>>>>> point
> >> > >>>>>>> of
> >> > >>>>>>>> view of these frameworks these stream processing jobs are
> just
> >> > >>>>>> stateless
> >> > >>>>>>>> services that can come and go and expand and contract at
> will.
> >> > >>>>>>>> There
> >> > >>>>> is
> >> > >>>>>>> no
> >> > >>>>>>>> more custom scheduler.
> >> > >>>>>>>>
> >> > >>>>>>>> Here are some relevant details:
> >> > >>>>>>>>
> >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get larger if we
> >> > >>>>>>>>  productionized but not vastly larger. We really do get a ton
> >> > >>>>>>>> of
> >> > >>>>>>> leverage
> >> > >>>>>>>>  out of Kafka.
> >> > >>>>>>>>  2. Partition management is fully delegated to the new
> >> consumer.
> >> > >>>>> This
> >> > >>>>>>>>  is nice since now any partition management strategy
> available
> >> > >>>>>>>> to
> >> > >>>>>> Kafka
> >> > >>>>>>>>  consumer is also available to Samza (and vice versa) and
> with
> >> > >>>>>>>> the
> >> > >>>>>>> exact
> >> > >>>>>>>>  same configs.
> >> > >>>>>>>>  3. It supports state as well as state reuse
> >> > >>>>>>>>
> >> > >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
> >> > >>>>>>>>
> >> > >>>>>>>> -Jay
> >> > >>>>>>>>
> >> > >>>>>>>>
> >> > >>>>>>>>
> >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> >> > >>>>>> criccomini@apache.org>
> >> > >>>>>>>> wrote:
> >> > >>>>>>>>
> >> > >>>>>>>>> Hey all,
> >> > >>>>>>>>>
> >> > >>>>>>>>> I have had some discussions with Samza engineers at LinkedIn
> >> > >>>>>>>>> and
> >> > >>>>>>> Confluent
> >> > >>>>>>>>> and we came up with a few observations and would like to
> >> > >>>>>>>>> propose
> >> > >>>>> some
> >> > >>>>>>>>> changes.
> >> > >>>>>>>>>
> >> > >>>>>>>>> We've observed some things that I want to call out about
> >> > >>>>>>>>> Samza's
> >> > >>>>>> design,
> >> > >>>>>>>>> and I'd like to propose some changes.
> >> > >>>>>>>>>
> >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
> >> > >>>>>>>>> * Samza is too pluggable.
> >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> >> > >>>>>>>>> APIs
> >> > >>>>> are
> >> > >>>>>>>>> trying to solve a lot of the same problems.
> >> > >>>>>>>>>
> >> > >>>>>>>>> All three of these issues are related, but I'll address them
> >> in
> >> > >>>>> order.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Deployment
> >> > >>>>>>>>>
> >> > >>>>>>>>> Samza strongly depends on the use of a dynamic deployment
> >> > >>>>>>>>> scheduler
> >> > >>>>>> such
> >> > >>>>>>>>> as
> >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> >> > >>>>>>>>> there
> >> > >>>>>> would
> >> > >>>>>>>>> be
> >> > >>>>>>>>> one or two winners in this area, and we could support them,
> >> and
> >> > >>>>>>>>> the
> >> > >>>>>> rest
> >> > >>>>>>>>> would go away. In reality, there are many variations.
> >> > >>>>>>>>> Furthermore,
> >> > >>>>>> many
> >> > >>>>>>>>> people still prefer to just start their processors like
> normal
> >> > >>>>>>>>> Java processes, and use traditional deployment scripts such
> as
> >> > >>>>>>>>> Fabric,
> >> > >>>>>> Chef,
> >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> >> > >>>>>>>>> Samza start-up process really painful for first time users.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit of a
> >> > >>>>>>>>> mis-fire
> >> > >>>>>> because
> >> > >>>>>>>>> of
> >> > >>>>>>>>> a fundamental misunderstanding between the nature of batch
> >> jobs
> >> > >>>>>>>>> and
> >> > >>>>>>> stream
> >> > >>>>>>>>> processing jobs. Early on, we made conscious effort to favor
> >> > >>>>>>>>> the
> >> > >>>>>> Hadoop
> >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked and was
> well
> >> > >>>>>>> understood.
> >> > >>>>>>>>> One thing that we missed was that batch jobs have a definite
> >> > >>>>>> beginning,
> >> > >>>>>>>>> and
> >> > >>>>>>>>> end, and stream processing jobs don't (usually). This leads
> to
> >> > >>>>>>>>> a
> >> > >>>>> much
> >> > >>>>>>>>> simpler scheduling problem for stream processors. You
> >> basically
> >> > >>>>>>>>> just
> >> > >>>>>>> need
> >> > >>>>>>>>> to find a place to start the processor, and start it. The
> way
> >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> >> > >>>>>>>>> being "full". We always
> >> > >>>>>> add
> >> > >>>>>>>>> more machines. The problem with coupling Samza with a
> >> scheduler
> >> > >>>>>>>>> is
> >> > >>>>>> that
> >> > >>>>>>>>> Samza (as a framework) now has to handle deployment. This
> >> pulls
> >> > >>>>>>>>> in a
> >> > >>>>>>> bunch
> >> > >>>>>>>>> of things such as configuration distribution (config
> stream),
> >> > >>>>>>>>> shell
> >> > >>>>>>> scrips
> >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff),
> >> etc.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Another reason for requiring dynamic deployment was to
> support
> >> > >>>>>>>>> data locality. If you want to have locality, you need to put
> >> > >>>>>>>>> your
> >> > >>>>>> processors
> >> > >>>>>>>>> close to the data they're processing. Upon further
> >> > >>>>>>>>> investigation,
> >> > >>>>>>> though,
> >> > >>>>>>>>> this feature is not that beneficial. There is some good
> >> > >>>>>>>>> discussion
> >> > >>>>>> about
> >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
> >> > >>>>>>>>> Map/Reduce
> >> > >>>>>> path,
> >> > >>>>>>>>> but
> >> > >>>>>>>>> there are some fundamental differences between HDFS and
> Kafka.
> >> > >>>>>>>>> HDFS
> >> > >>>>>> has
> >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to less
> >> > >>>>>>>>> optimization potential with stream processors on top of
> Kafka.
> >> > >>>>>>>>>
> >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't have
> any
> >> > >>>>>>>>> built
> >> > >>>>> in
> >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> >> > >>>>>>>>> deployment scheduling system to handle restarts when a
> >> > >>>>>>>>> processor dies. This has
> >> > >>>>>>> made
> >> > >>>>>>>>> it very difficult to write a standalone Samza container
> >> > >>>> (SAMZA-516).
> >> > >>>>>>>>>
> >> > >>>>>>>>> Pluggability
> >> > >>>>>>>>>
> >> > >>>>>>>>> In some cases pluggability is good, but I think that we've
> >> gone
> >> > >>>>>>>>> too
> >> > >>>>>> far
> >> > >>>>>>>>> with it. Currently, Samza has:
> >> > >>>>>>>>>
> >> > >>>>>>>>> * Pluggable config.
> >> > >>>>>>>>> * Pluggable metrics.
> >> > >>>>>>>>> * Pluggable deployment systems.
> >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> SystemProducer,
> >> > >>>> etc).
> >> > >>>>>>>>> * Pluggable serdes.
> >> > >>>>>>>>> * Pluggable storage engines.
> >> > >>>>>>>>> * Pluggable strategies for just about every component
> >> > >>>>> (MessageChooser,
> >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> >> > >>>>>>>>>
> >> > >>>>>>>>> There's probably more that I've forgotten, as well. Some of
> >> > >>>>>>>>> these
> >> > >>>>> are
> >> > >>>>>>>>> useful, but some have proven not to be. This all comes at a
> >> cost:
> >> > >>>>>>>>> complexity. This complexity is making it harder for our
> users
> >> > >>>>>>>>> to
> >> > >>>>> pick
> >> > >>>>>> up
> >> > >>>>>>>>> and use Samza out of the box. It also makes it difficult for
> >> > >>>>>>>>> Samza developers to reason about what the characteristics of
> >> > >>>>>>>>> the container (since the characteristics change depending on
> >> > >>>>>>>>> which plugins are use).
> >> > >>>>>>>>>
> >> > >>>>>>>>> The issues with pluggability are most visible in the System
> >> APIs.
> >> > >>>>> What
> >> > >>>>>>>>> Samza really requires to be functional is Kafka as its
> >> > >>>>>>>>> transport
> >> > >>>>>> layer.
> >> > >>>>>>>>> But
> >> > >>>>>>>>> we've conflated two unrelated use cases into one API:
> >> > >>>>>>>>>
> >> > >>>>>>>>> 1. Get data into/out of Kafka.
> >> > >>>>>>>>> 2. Process the data in Kafka.
> >> > >>>>>>>>>
> >> > >>>>>>>>> The current System API supports both of these use cases. The
> >> > >>>>>>>>> problem
> >> > >>>>>> is,
> >> > >>>>>>>>> we
> >> > >>>>>>>>> actually want different features for each use case. By
> >> papering
> >> > >>>>>>>>> over
> >> > >>>>>>> these
> >> > >>>>>>>>> two use cases, and providing a single API, we've introduced
> a
> >> > >>>>>>>>> ton of
> >> > >>>>>>> leaky
> >> > >>>>>>>>> abstractions.
> >> > >>>>>>>>>
> >> > >>>>>>>>> For example, what we'd really like in (2) is to have
> >> > >>>>>>>>> monotonically increasing longs for offsets (like Kafka).
> This
> >> > >>>>>>>>> would be at odds
> >> > >>>>> with
> >> > >>>>>>> (1),
> >> > >>>>>>>>> though, since different systems have different
> >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> >> > >>>>>>>>> There was discussion both on the mailing list and the SQL
> >> JIRAs
> >> > >>>>> about
> >> > >>>>>>> the
> >> > >>>>>>>>> need for this.
> >> > >>>>>>>>>
> >> > >>>>>>>>> The same thing holds true for replayability. Kafka allows us
> >> to
> >> > >>>>> rewind
> >> > >>>>>>>>> when
> >> > >>>>>>>>> we have a failure. Many other systems don't. In some cases,
> >> > >>>>>>>>> systems
> >> > >>>>>>> return
> >> > >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> because
> >> > >>>>>>>>> they
> >> > >>>>>> have
> >> > >>>>>>> no
> >> > >>>>>>>>> offsets.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Partitioning is another example. Kafka supports
> partitioning,
> >> > >>>>>>>>> but
> >> > >>>>> many
> >> > >>>>>>>>> systems don't. We model this by having a single partition
> for
> >> > >>>>>>>>> those systems. Still, other systems model partitioning
> >> > >>>> differently (e.g.
> >> > >>>>>>>>> Kinesis).
> >> > >>>>>>>>>
> >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating streams
> in
> >> a
> >> > >>>>>>>>> system-agnostic way is almost impossible. As is modeling
> >> > >>>>>>>>> metadata
> >> > >>>>> for
> >> > >>>>>>> the
> >> > >>>>>>>>> system (replication factor, partitions, location, etc). The
> >> > >>>>>>>>> list
> >> > >>>>> goes
> >> > >>>>>>> on.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Duplicate work
> >> > >>>>>>>>>
> >> > >>>>>>>>> At the time that we began writing Samza, Kafka's consumer
> and
> >> > >>>>> producer
> >> > >>>>>>>>> APIs
> >> > >>>>>>>>> had a relatively weak feature set. On the consumer-side, you
> >> > >>>>>>>>> had two
> >> > >>>>>>>>> options: use the high level consumer, or the simple
> consumer.
> >> > >>>>>>>>> The
> >> > >>>>>>> problem
> >> > >>>>>>>>> with the high-level consumer was that it controlled your
> >> > >>>>>>>>> offsets, partition assignments, and the order in which you
> >> > >>>>>>>>> received messages. The
> >> > >>>>> problem
> >> > >>>>>>>>> with
> >> > >>>>>>>>> the simple consumer is that it's not simple. It's basic. You
> >> > >>>>>>>>> end up
> >> > >>>>>>> having
> >> > >>>>>>>>> to handle a lot of really low-level stuff that you
> shouldn't.
> >> > >>>>>>>>> We
> >> > >>>>>> spent a
> >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust.
> >> It
> >> > >>>>>>>>> also allows us to support some cool features:
> >> > >>>>>>>>>
> >> > >>>>>>>>> * Per-partition message ordering and prioritization.
> >> > >>>>>>>>> * Tight control over partition assignment to support joins,
> >> > >>>>>>>>> global
> >> > >>>>>> state
> >> > >>>>>>>>> (if we want to implement it :)), etc.
> >> > >>>>>>>>> * Tight control over offset checkpointing.
> >> > >>>>>>>>>
> >> > >>>>>>>>> What we didn't realize at the time is that these features
> >> > >>>>>>>>> should
> >> > >>>>>>> actually
> >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> >> > >>>>>> processors)
> >> > >>>>>>>>> end up wanting to do things like joins and partition
> >> > >>>>>>>>> assignment. The
> >> > >>>>>>> Kafka
> >> > >>>>>>>>> community has come to the same conclusion. They're adding a
> >> ton
> >> > >>>>>>>>> of upgrades into their new Kafka consumer implementation.
> To a
> >> > >>>>>>>>> large extent,
> >> > >>>>> it's
> >> > >>>>>>>>> duplicate work to what we've already done in Samza.
> >> > >>>>>>>>>
> >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
> approach
> >> > >>>>>>>>> to
> >> > >>>>>> Samza's
> >> > >>>>>>>>> KafkaCheckpointManager implementation for handling offset
> >> > >>>>>> checkpointing.
> >> > >>>>>>>>> Like Samza, Kafka's new offset management feature stores
> >> offset
> >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them from
> the
> >> > >>>>>>>>> broker.
> >> > >>>>>>>>>
> >> > >>>>>>>>> A lot of this seems like a waste, since we could have shared
> >> > >>>>>>>>> the
> >> > >>>>> work
> >> > >>>>>> if
> >> > >>>>>>>>> it
> >> > >>>>>>>>> had been done in Kafka from the get-go.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Vision
> >> > >>>>>>>>>
> >> > >>>>>>>>> All of this leads me to a rather radical proposal. Samza is
> >> > >>>>> relatively
> >> > >>>>>>>>> stable at this point. I'd venture to say that we're near a
> 1.0
> >> > >>>>>> release.
> >> > >>>>>>>>> I'd
> >> > >>>>>>>>> like to propose that we take what we've learned, and begin
> >> > >>>>>>>>> thinking
> >> > >>>>>>> about
> >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were starting
> >> from
> >> > >>>>>> scratch?
> >> > >>>>>>>>> My
> >> > >>>>>>>>> proposal is to:
> >> > >>>>>>>>>
> >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> >> > >>>>>>>>> processors, and eliminate all direct dependences on YARN,
> >> Mesos,
> >> > >>>> etc.
> >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as the
> stream
> >> > >>>>>> processing
> >> > >>>>>>>>> layer.
> >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> >> > >>>>>>>>> config
> >> > >>>>>>> systems,
> >> > >>>>>>>>> and simply use Kafka's instead.
> >> > >>>>>>>>>
> >> > >>>>>>>>> This would fix all of the issues that I outlined above. It
> >> > >>>>>>>>> should
> >> > >>>>> also
> >> > >>>>>>>>> shrink the Samza code base pretty dramatically. Supporting
> >> only
> >> > >>>>>>>>> a standalone container will allow Samza to be executed on
> YARN
> >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> >> > >>>>>>>>> in-house
> >> > >>>>>>> deployment
> >> > >>>>>>>>> systems. This should make life a lot easier for new users.
> >> > >>>>>>>>> Imagine
> >> > >>>>>>> having
> >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in mailing
> >> list
> >> > >>>>>> traffic
> >> > >>>>>>>>> will be pretty dramatic.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The reality
> is,
> >> > >>>>> everyone
> >> > >>>>>>>>> that
> >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically require
> >> it
> >> > >>>>>> already
> >> > >>>>>>> in
> >> > >>>>>>>>> order for most features to work. Those that are using other
> >> > >>>>>>>>> systems
> >> > >>>>>> are
> >> > >>>>>>>>> generally using it for ingest into Kafka (1), and then they
> do
> >> > >>>>>>>>> the processing on top. There is already discussion (
> >> > >>>>>>>>>
> >> > >>>>>>>
> >> > >>>>>>
> >> > >>>>>
> >> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >> > >>>>> 767
> >> > >>>>>>>>> )
> >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Once we make the call to couple with Kafka, we can leverage
> a
> >> > >>>>>>>>> ton of
> >> > >>>>>>> their
> >> > >>>>>>>>> ecosystem. We no longer have to maintain our own config,
> >> > >>>>>>>>> metrics,
> >> > >>>>> etc.
> >> > >>>>>>> We
> >> > >>>>>>>>> can all share the same libraries, and make them better. This
> >> > >>>>>>>>> will
> >> > >>>>> also
> >> > >>>>>>>>> allow us to share the consumer/producer APIs, and will let
> us
> >> > >>>>> leverage
> >> > >>>>>>>>> their offset management and partition management, rather
> than
> >> > >>>>>>>>> having
> >> > >>>>>> our
> >> > >>>>>>>>> own. All of the coordinator stream code would go away, as
> >> would
> >> > >>>>>>>>> most
> >> > >>>>>> of
> >> > >>>>>>>>> the
> >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
> partition
> >> > >>>>>>> management
> >> > >>>>>>>>> features into the Kafka broker, but they're already moving
> in
> >> > >>>>>>>>> that direction with the new consumer API. The features we
> have
> >> > >>>>>>>>> for
> >> > >>>>>> partition
> >> > >>>>>>>>> assignment aren't unique to Samza, and seem like they should
> >> be
> >> > >>>>>>>>> in
> >> > >>>>>> Kafka
> >> > >>>>>>>>> anyway. There will always be some niche usages which will
> >> > >>>>>>>>> require
> >> > >>>>>> extra
> >> > >>>>>>>>> care and hence full control over partition assignments much
> >> > >>>>>>>>> like the
> >> > >>>>>>> Kafka
> >> > >>>>>>>>> low level consumer api. These would continue to be
> supported.
> >> > >>>>>>>>>
> >> > >>>>>>>>> These items will be good for the Samza community. They'll
> make
> >> > >>>>>>>>> Samza easier to use, and make it easier for developers to
> add
> >> > >>>>>>>>> new features.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Obviously this is a fairly large (and somewhat backwards
> >> > >>>>> incompatible
> >> > >>>>>>>>> change). If we choose to go this route, it's important that
> we
> >> > >>>>> openly
> >> > >>>>>>>>> communicate how we're going to provide a migration path from
> >> > >>>>>>>>> the
> >> > >>>>>>> existing
> >> > >>>>>>>>> APIs to the new ones (if we make incompatible changes). I
> >> think
> >> > >>>>>>>>> at a minimum, we'd probably need to provide a wrapper to
> allow
> >> > >>>>>>>>> existing StreamTask implementations to continue running on
> the
> >> > >>>> new container.
> >> > >>>>>>> It's
> >> > >>>>>>>>> also important that we openly communicate about timing, and
> >> > >>>>>>>>> stages
> >> > >>>>> of
> >> > >>>>>>> the
> >> > >>>>>>>>> migration.
> >> > >>>>>>>>>
> >> > >>>>>>>>> If you made it this far, I'm sure you have opinions. :)
> Please
> >> > >>>>>>>>> send
> >> > >>>>>> your
> >> > >>>>>>>>> thoughts and feedback.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Cheers,
> >> > >>>>>>>>> Chris
> >> > >>>>>>>>>
> >> > >>>>>>>>
> >> > >>>>>>>>
> >> > >>>>>>>
> >> > >>>>>>
> >> > >>>>>>
> >> > >>>>>>
> >> > >>>>>> --
> >> > >>>>>> -- Guozhang
> >> > >>>>>>
> >> > >>>>>
> >> > >>>>
> >> > >>
> >> > >>
> >> >
> >> >
> >> >
> >>
> >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@confluent.io>.
Kafka Metamorphasis: Data streams in, cockroaches stream out :-)

On Tue, Jul 7, 2015 at 3:29 AM, Gianmarco De Francisci Morales <
gdfm@apache.org> wrote:

> Forgot to add. On the naming issues, Kafka Metamorphosis is a clear winner
> :)
>
> --
> Gianmarco
>
> On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <gd...@apache.org>
> wrote:
>
> > Hi,
> >
> > @Martin, thanks for you comments.
> > Maybe I'm missing some important point, but I think coupling the releases
> > is actually a *good* thing.
> > To make an example, would it be better if the MR and HDFS components of
> > Hadoop had different release schedules?
> >
> > Actually, keeping the discussion in a single place would make agreeing on
> > releases (and backwards compatibility) much easier, as everybody would be
> > responsible for the whole codebase.
> >
> > That said, I like the idea of absorbing samza-core as a sub-project, and
> > leave the fancy stuff separate.
> > It probably gives 90% of the benefits we have been discussing here.
> >
> > Cheers,
> >
> > --
> > Gianmarco
> >
> > On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
> >
> >> Hey Martin,
> >>
> >> I agree coupling release schedules is a downside.
> >>
> >> Definitely we can try to solve some of the integration problems in
> >> Confluent Platform or in other distributions. But I think this ends up
> >> being really shallow. I guess I feel to really get a good user
> experience
> >> the two systems have to kind of feel like part of the same thing and you
> >> can't really add that in later--you can put both in the same
> downloadable
> >> tar file but it doesn't really give a very cohesive feeling. I agree
> that
> >> ultimately any of the project stuff is as much social and naming as
> >> anything else--theoretically two totally independent projects could work
> >> to
> >> tightly align. In practice this seems to be quite difficult though.
> >>
> >> For the frameworks--totally agree it would be good to maintain the
> >> framework support with the project. In some cases there may not be too
> >> much
> >> there since the integration gets lighter but I think whatever stubs you
> >> need should be included. So no I definitely wasn't trying to imply
> >> dropping
> >> support for these frameworks, just making the integration lighter by
> >> separating process management from partition management.
> >>
> >> You raise two good points we would have to figure out if we went down
> the
> >> alignment path:
> >> 1. With respect to the name, yeah I think the first question is whether
> >> some "re-branding" would be worth it. If so then I think we can have a
> big
> >> thread on the name. I'm definitely not set on Kafka Streaming or Kafka
> >> Streams I was just using them to be kind of illustrative. I agree with
> >> your
> >> critique of these names, though I think people would get the idea.
> >> 2. Yeah you also raise a good point about how to "factor" it. Here are
> the
> >> options I see (I could get enthusiastic about any of them):
> >>    a. One repo for both Kafka and Samza
> >>    b. Two repos, retaining the current seperation
> >>    c. Two repos, the equivalent of samza-api and samza-core is absorbed
> >> almost like a third client
> >>
> >> Cheers,
> >>
> >> -Jay
> >>
> >> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <ma...@kleppmann.com>
> >> wrote:
> >>
> >> > Ok, thanks for the clarifications. Just a few follow-up comments.
> >> >
> >> > - I see the appeal of merging with Kafka or becoming a subproject: the
> >> > reasons you mention are good. The risk I see is that release schedules
> >> > become coupled to each other, which can slow everyone down, and large
> >> > projects with many contributors are harder to manage. (Jakob, can you
> >> speak
> >> > from experience, having seen a wider range of Hadoop ecosystem
> >> projects?)
> >> >
> >> > Some of the goals of a better unified developer experience could also
> be
> >> > solved by integrating Samza nicely into a Kafka distribution (such as
> >> > Confluent's). I'm not against merging projects if we decide that's the
> >> way
> >> > to go, just pointing out the same goals can perhaps also be achieved
> in
> >> > other ways.
> >> >
> >> > - With regard to dropping the YARN dependency: are you proposing that
> >> > Samza doesn't give any help to people wanting to run on
> >> YARN/Mesos/AWS/etc?
> >> > So the docs would basically have a link to Slider and nothing else? Or
> >> > would we maintain integrations with a bunch of popular deployment
> >> methods
> >> > (e.g. the necessary glue and shell scripts to make Samza work with
> >> Slider)?
> >> >
> >> > I absolutely think it's a good idea to have the "as a library" and
> "as a
> >> > process" (using Yi's taxonomy) options for people who want them, but I
> >> > think there should also be a low-friction path for common "as a
> service"
> >> > deployment methods, for which we probably need to maintain
> integrations.
> >> >
> >> > - Project naming: "Kafka Streams" seems odd to me, because Kafka is
> all
> >> > about streams already. Perhaps "Kafka Transformers" or "Kafka Filters"
> >> > would be more apt?
> >> >
> >> > One suggestion: perhaps the core of Samza (stream transformation with
> >> > state management -- i.e. the "Samza as a library" bit) could become
> >> part of
> >> > Kafka, while higher-level tools such as streaming SQL and integrations
> >> with
> >> > deployment frameworks remain in a separate project? In other words,
> >> Kafka
> >> > would absorb the proven, stable core of Samza, which would become the
> >> > "third Kafka client" mentioned early in this thread. The Samza project
> >> > would then target that third Kafka client as its base API, and the
> >> project
> >> > would be freed up to explore more experimental new horizons.
> >> >
> >> > Martin
> >> >
> >> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
> >> >
> >> > > Hey Martin,
> >> > >
> >> > > For the YARN/Mesos/etc decoupling I actually don't think it ties our
> >> > hands
> >> > > at all, all it does is refactor things. The division of
> >> responsibility is
> >> > > that Samza core is responsible for task lifecycle, state, and
> >> partition
> >> > > management (using the Kafka co-ordinator) but it is NOT responsible
> >> for
> >> > > packaging, configuration deployment or execution of processes. The
> >> > problem
> >> > > of packaging and starting these processes is
> >> > > framework/environment-specific. This leaves individual frameworks to
> >> be
> >> > as
> >> > > fancy or vanilla as they like. So you can get simple stateless
> >> support in
> >> > > YARN, Mesos, etc using their off-the-shelf app framework (Slider,
> >> > Marathon,
> >> > > etc). These are well known by people and have nice UIs and a lot of
> >> > > flexibility. I don't think they have node affinity as a built in
> >> option
> >> > > (though I could be wrong). So if we want that we can either wait for
> >> them
> >> > > to add it or do a custom framework to add that feature (as now).
> >> > Obviously
> >> > > if you manage things with old-school ops tools (puppet/chef/etc) you
> >> get
> >> > > locality easily. The nice thing, though, is that all the samza
> >> "business
> >> > > logic" around partition management and fault tolerance is in Samza
> >> core
> >> > so
> >> > > it is shared across frameworks and the framework specific bit is
> just
> >> > > whether it is smart enough to try to get the same host when a job is
> >> > > restarted.
> >> > >
> >> > > With respect to the Kafka-alignment, yeah I think the goal would be
> >> (a)
> >> > > actually get better alignment in user experience, and (b) express
> >> this in
> >> > > the naming and project branding. Specifically:
> >> > > 1. Website/docs, it would be nice for the "transformation" api to be
> >> > > discoverable in the main Kafka docs--i.e. be able to explain when to
> >> use
> >> > > the consumer and when to use the stream processing functionality and
> >> lead
> >> > > people into that experience.
> >> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that has
> >> both
> >> > > Kafka and the stream processing part and they actually work
> together.
> >> > > 3. Unify the programming experience so the client and Samza api
> share
> >> > > config/monitoring/naming/packaging/etc.
> >> > >
> >> > > I think sub-projects keep separate committers and can have a
> separate
> >> > repo,
> >> > > but I'm actually not really sure (I can't find a definition of a
> >> > subproject
> >> > > in Apache).
> >> > >
> >> > > Basically at a high-level you want the experience to "feel" like a
> >> single
> >> > > system, not to relatively independent things that are kind of
> >> awkwardly
> >> > > glued together.
> >> > >
> >> > > I think if we did that they having naming or branding like "kafka
> >> > > streaming" or "kafka streams" or something like that would actually
> >> do a
> >> > > good job of conveying what it is. I do that this would help adoption
> >> > quite
> >> > > a lot as it would correctly convey that using Kafka Streaming with
> >> Kafka
> >> > is
> >> > > a fairly seamless experience and Kafka is pretty heavily adopted at
> >> this
> >> > > point.
> >> > >
> >> > > Fwiw we actually considered this model originally when open sourcing
> >> > Samza,
> >> > > however at that time Kafka was relatively unknown and we decided not
> >> to
> >> > do
> >> > > it since we felt it would be limiting. From my point of view the
> three
> >> > > things have changed (1) Kafka is now really heavily used for stream
> >> > > processing, (2) we learned that abstracting out the stream well is
> >> > > basically impossible, (3) we learned it is really hard to keep the
> two
> >> > > things feeling like a single product.
> >> > >
> >> > > -Jay
> >> > >
> >> > >
> >> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
> >> martin@kleppmann.com>
> >> > > wrote:
> >> > >
> >> > >> Hi all,
> >> > >>
> >> > >> Lots of good thoughts here.
> >> > >>
> >> > >> I agree with the general philosophy of tying Samza more firmly to
> >> Kafka.
> >> > >> After I spent a while looking at integrating other message brokers
> >> (e.g.
> >> > >> Kinesis) with SystemConsumer, I came to the conclusion that
> >> > SystemConsumer
> >> > >> tacitly assumes a model so much like Kafka's that pretty much
> nobody
> >> but
> >> > >> Kafka actually implements it. (Databus is perhaps an exception, but
> >> it
> >> > >> isn't widely used outside of LinkedIn.) Thus, making Samza fully
> >> > dependent
> >> > >> on Kafka acknowledges that the system-independence was never as
> real
> >> as
> >> > we
> >> > >> perhaps made it out to be. The gains of code reuse are real.
> >> > >>
> >> > >> The idea of decoupling Samza from YARN has also always been
> >> appealing to
> >> > >> me, for various reasons already mentioned in this thread. Although
> >> > making
> >> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems
> >> laudable,
> >> > I am
> >> > >> a little concerned that it will restrict us to a lowest common
> >> > denominator.
> >> > >> For example, would host affinity (SAMZA-617) still be possible? For
> >> jobs
> >> > >> with large amounts of state, I think SAMZA-617 would be a big boon,
> >> > since
> >> > >> restoring state off the changelog on every single restart is
> painful,
> >> > due
> >> > >> to long recovery times. It would be a shame if the decoupling from
> >> YARN
> >> > >> made host affinity impossible.
> >> > >>
> >> > >> Jay, a question about the proposed API for instantiating a job in
> >> code
> >> > >> (rather than a properties file): when submitting a job to a
> cluster,
> >> is
> >> > the
> >> > >> idea that the instantiation code runs on a client somewhere, which
> >> then
> >> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that
> >> code
> >> > run
> >> > >> on each container that is part of the job (in which case, how does
> >> the
> >> > job
> >> > >> submission to the cluster work)?
> >> > >>
> >> > >> I agree with Garry that it doesn't feel right to make a 1.0 release
> >> > with a
> >> > >> plan for it to be immediately obsolete. So if this is going to
> >> happen, I
> >> > >> think it would be more honest to stick with 0.* version numbers
> until
> >> > the
> >> > >> library-ified Samza has been implemented, is stable and widely
> used.
> >> > >>
> >> > >> Should the new Samza be a subproject of Kafka? There is precedent
> for
> >> > >> tight coupling between different Apache projects (e.g. Curator and
> >> > >> Zookeeper, or Slider and YARN), so I think remaining separate would
> >> be
> >> > ok.
> >> > >> Even if Samza is fully dependent on Kafka, there is enough
> substance
> >> in
> >> > >> Samza that it warrants being a separate project. An argument in
> >> favour
> >> > of
> >> > >> merging would be if we think Kafka has a much stronger "brand
> >> presence"
> >> > >> than Samza; I'm ambivalent on that one. If the Kafka project is
> >> willing
> >> > to
> >> > >> endorse Samza as the "official" way of doing stateful stream
> >> > >> transformations, that would probably have much the same effect as
> >> > >> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
> >> > >> collaboration between the two projects will be needed in any case.
> >> > >>
> >> > >> From a project management perspective, I guess the "new Samza"
> would
> >> > have
> >> > >> to be developed on a branch alongside ongoing maintenance of the
> >> current
> >> > >> line of development? I think it would be important to continue
> >> > supporting
> >> > >> existing users, and provide a graceful migration path to the new
> >> > version.
> >> > >> Leaving the current versions unsupported and forcing people to
> >> rewrite
> >> > >> their jobs would send a bad signal.
> >> > >>
> >> > >> Best,
> >> > >> Martin
> >> > >>
> >> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
> >> > >>
> >> > >>> Hey Garry,
> >> > >>>
> >> > >>> Yeah that's super frustrating. I'd be happy to chat more about
> this
> >> if
> >> > >>> you'd be interested. I think Chris and I started with the idea of
> >> "what
> >> > >>> would it take to make Samza a kick-ass ingestion tool" but
> >> ultimately
> >> > we
> >> > >>> kind of came around to the idea that ingestion and transformation
> >> had
> >> > >>> pretty different needs and coupling the two made things hard.
> >> > >>>
> >> > >>> For what it's worth I think copycat (KIP-26) actually will do what
> >> you
> >> > >> are
> >> > >>> looking for.
> >> > >>>
> >> > >>> With regard to your point about slider, I don't necessarily
> >> disagree.
> >> > >> But I
> >> > >>> think getting good YARN support is quite doable and I think we can
> >> make
> >> > >>> that work well. I think the issue this proposal solves is that
> >> > >> technically
> >> > >>> it is pretty hard to support multiple cluster management systems
> the
> >> > way
> >> > >>> things are now, you need to write an "app master" or "framework"
> for
> >> > each
> >> > >>> and they are all a little different so testing is really hard. In
> >> the
> >> > >>> absence of this we have been stuck with just YARN which has
> >> fantastic
> >> > >>> penetration in the Hadoopy part of the org, but zero penetration
> >> > >> elsewhere.
> >> > >>> Given the huge amount of work being put in to slider, marathon,
> aws
> >> > >>> tooling, not to mention the umpteen related packaging technologies
> >> > people
> >> > >>> want to use (Docker, Kubernetes, various cloud-specific deploy
> >> tools,
> >> > >> etc)
> >> > >>> I really think it is important to get this right.
> >> > >>>
> >> > >>> -Jay
> >> > >>>
> >> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> >> > >>> g.turkington@improvedigital.com> wrote:
> >> > >>>
> >> > >>>> Hi all,
> >> > >>>>
> >> > >>>> I think the question below re does Samza become a sub-project of
> >> Kafka
> >> > >>>> highlights the broader point around migration. Chris mentions
> >> Samza's
> >> > >>>> maturity is heading towards a v1 release but I'm not sure it
> feels
> >> > >> right to
> >> > >>>> launch a v1 then immediately plan to deprecate most of it.
> >> > >>>>
> >> > >>>> From a selfish perspective I have some guys who have started
> >> working
> >> > >> with
> >> > >>>> Samza and building some new consumers/producers was next up.
> Sounds
> >> > like
> >> > >>>> that is absolutely not the direction to go. I need to look into
> the
> >> > KIP
> >> > >> in
> >> > >>>> more detail but for me the attractiveness of adding new Samza
> >> > >>>> consumer/producers -- even if yes all they were doing was really
> >> > getting
> >> > >>>> data into and out of Kafka --  was to avoid  having to worry
> about
> >> the
> >> > >>>> lifecycle management of external clients. If there is a generic
> >> Kafka
> >> > >>>> ingress/egress layer that I can plug a new connector into and
> have
> >> a
> >> > >> lot of
> >> > >>>> the heavy lifting re scale and reliability done for me then it
> >> gives
> >> > me
> >> > >> all
> >> > >>>> the pushing new consumers/producers would. If not then it
> >> complicates
> >> > my
> >> > >>>> operational deployments.
> >> > >>>>
> >> > >>>> Which is similar to my other question with the proposal -- if we
> >> > build a
> >> > >>>> fully available/stand-alone Samza plus the requisite shims to
> >> > integrate
> >> > >>>> with Slider etc I suspect the former may be a lot more work than
> we
> >> > >> think.
> >> > >>>> We may make it much easier for a newcomer to get something
> running
> >> but
> >> > >>>> having them step up and get a reliable production deployment may
> >> still
> >> > >>>> dominate mailing list  traffic, if for different reasons than
> >> today.
> >> > >>>>
> >> > >>>> Don't get me wrong -- I'm comfortable with making the Samza
> >> dependency
> >> > >> on
> >> > >>>> Kafka much more explicit and I absolutely see the benefits  in
> the
> >> > >>>> reduction of duplication and clashing terminologies/abstractions
> >> that
> >> > >>>> Chris/Jay describe. Samza as a library would likely be a very
> nice
> >> > tool
> >> > >> to
> >> > >>>> add to the Kafka ecosystem. I just have the concerns above re the
> >> > >>>> operational side.
> >> > >>>>
> >> > >>>> Garry
> >> > >>>>
> >> > >>>> -----Original Message-----
> >> > >>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> >> > >>>> Sent: 02 July 2015 12:56
> >> > >>>> To: dev@samza.apache.org
> >> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> >> > >>>>
> >> > >>>> Very interesting thoughts.
> >> > >>>> From outside, I have always perceived Samza as a computing layer
> >> over
> >> > >>>> Kafka.
> >> > >>>>
> >> > >>>> The question, maybe a bit provocative, is "should Samza be a
> >> > sub-project
> >> > >>>> of Kafka then?"
> >> > >>>> Or does it make sense to keep it as a separate project with a
> >> separate
> >> > >>>> governance?
> >> > >>>>
> >> > >>>> Cheers,
> >> > >>>>
> >> > >>>> --
> >> > >>>> Gianmarco
> >> > >>>>
> >> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> >> > >>>>
> >> > >>>>> Overall, I agree to couple with Kafka more tightly. Because
> Samza
> >> de
> >> > >>>>> facto is based on Kafka, and it should leverage what Kafka has.
> At
> >> > the
> >> > >>>>> same time, Kafka does not need to reinvent what Samza already
> >> has. I
> >> > >>>>> also like the idea of separating the ingestion and
> transformation.
> >> > >>>>>
> >> > >>>>> But it is a little difficult for me to image how the Samza will
> >> look
> >> > >>>> like.
> >> > >>>>> And I feel Chris and Jay have a little difference in terms of
> how
> >> > >>>>> Samza should look like.
> >> > >>>>>
> >> > >>>>> *** Will it look like what Jay's code shows (A client of Kakfa)
> ?
> >> And
> >> > >>>>> user's application code calls this client?
> >> > >>>>>
> >> > >>>>> 1. If we make Samza be a library of Kafka (like what the code
> >> shows),
> >> > >>>>> how do we implement auto-balance and fault-tolerance? Are they
> >> taken
> >> > >>>>> care by the Kafka broker or other mechanism, such as "Samza
> >> worker"
> >> > >>>>> (just make up the name) ?
> >> > >>>>>
> >> > >>>>> 2. What about other features, such as auto-scaling, shared
> state,
> >> > >>>>> monitoring?
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> *** If we have Samza standalone, (is this what Chris suggests?)
> >> > >>>>>
> >> > >>>>> 1. we still need to ingest data from Kakfa and produce to it.
> >> Then it
> >> > >>>>> becomes the same as what Samza looks like now, except it does
> not
> >> > rely
> >> > >>>>> on Yarn anymore.
> >> > >>>>>
> >> > >>>>> 2. if it is standalone, how can it leverage Kafka's metrics,
> logs,
> >> > >>>>> etc? Use Kafka code as the dependency?
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Thanks,
> >> > >>>>>
> >> > >>>>> Fang, Yan
> >> > >>>>> yanfang724@gmail.com
> >> > >>>>>
> >> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> wangguoz@gmail.com
> >> >
> >> > >>>> wrote:
> >> > >>>>>
> >> > >>>>>> Read through the code example and it looks good to me. A few
> >> > >>>>>> thoughts regarding deployment:
> >> > >>>>>>
> >> > >>>>>> Today Samza deploys as executable runnable like:
> >> > >>>>>>
> >> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> >> > >>>> --config-path=file://...
> >> > >>>>>>
> >> > >>>>>> And this proposal advocate for deploying Samza more as embedded
> >> > >>>>>> libraries in user application code (ignoring the terminology
> >> since
> >> > >>>>>> it is not the
> >> > >>>>> same
> >> > >>>>>> as the prototype code):
> >> > >>>>>>
> >> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread thread =
> new
> >> > >>>>>> Thread(task); thread.start();
> >> > >>>>>>
> >> > >>>>>> I think both of these deployment modes are important for
> >> different
> >> > >>>>>> types
> >> > >>>>> of
> >> > >>>>>> users. That said, I think making Samza purely standalone is
> still
> >> > >>>>>> sufficient for either runnable or library modes.
> >> > >>>>>>
> >> > >>>>>> Guozhang
> >> > >>>>>>
> >> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
> >> > wrote:
> >> > >>>>>>
> >> > >>>>>>> Looks like gmail mangled the code example, it was supposed to
> >> look
> >> > >>>>>>> like
> >> > >>>>>>> this:
> >> > >>>>>>>
> >> > >>>>>>> Properties props = new Properties();
> >> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
> >> StreamingConfig
> >> > >>>>>>> config = new StreamingConfig(props);
> >> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> >> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> >> > >>>>>>> config.serialization(new StringSerializer(), new
> >> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
> >> > >>>>>>> KafkaStreaming(config); container.run();
> >> > >>>>>>>
> >> > >>>>>>> -Jay
> >> > >>>>>>>
> >> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <jay@confluent.io
> >
> >> > >>>> wrote:
> >> > >>>>>>>
> >> > >>>>>>>> Hey guys,
> >> > >>>>>>>>
> >> > >>>>>>>> This came out of some conversations Chris and I were having
> >> > >>>>>>>> around
> >> > >>>>>>> whether
> >> > >>>>>>>> it would make sense to use Samza as a kind of data ingestion
> >> > >>>>> framework
> >> > >>>>>>> for
> >> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind
> of
> >> > >>>>>> combined
> >> > >>>>>>>> with complaints around config and YARN and the discussion
> >> around
> >> > >>>>>>>> how
> >> > >>>>> to
> >> > >>>>>>>> best do a standalone mode.
> >> > >>>>>>>>
> >> > >>>>>>>> So the thought experiment was, given that Samza was basically
> >> > >>>>>>>> already totally Kafka specific, what if you just embraced
> that
> >> > >>>>>>>> and turned it
> >> > >>>>>> into
> >> > >>>>>>>> something less like a heavyweight framework and more like a
> >> > >>>>>>>> third
> >> > >>>>> Kafka
> >> > >>>>>>>> client--a kind of "producing consumer" with state management
> >> > >>>>>> facilities.
> >> > >>>>>>>> Basically a library. Instead of a complex stream processing
> >> > >>>>>>>> framework
> >> > >>>>>>> this
> >> > >>>>>>>> would actually be a very simple thing, not much more
> >> complicated
> >> > >>>>>>>> to
> >> > >>>>> use
> >> > >>>>>>> or
> >> > >>>>>>>> operate than a Kafka consumer. As Chris said we thought about
> >> it
> >> > >>>>>>>> a
> >> > >>>>> lot
> >> > >>>>>> of
> >> > >>>>>>>> what Samza (and the other stream processing systems were
> doing)
> >> > >>>>> seemed
> >> > >>>>>>> like
> >> > >>>>>>>> kind of a hangover from MapReduce.
> >> > >>>>>>>>
> >> > >>>>>>>> Of course you need to ingest/output data to and from the
> stream
> >> > >>>>>>>> processing. But when we actually looked into how that would
> >> > >>>>>>>> work,
> >> > >>>>> Samza
> >> > >>>>>>>> isn't really an ideal data ingestion framework for a bunch of
> >> > >>>>> reasons.
> >> > >>>>>> To
> >> > >>>>>>>> really do that right you need a pretty different internal
> data
> >> > >>>>>>>> model
> >> > >>>>>> and
> >> > >>>>>>>> set of apis. So what if you split them and had an api for
> Kafka
> >> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
> >> Kafka
> >> > >>>>>>>> transformation (Samza).
> >> > >>>>>>>>
> >> > >>>>>>>> This would also allow really embracing the same terminology
> and
> >> > >>>>>>>> conventions. One complaint about the current state is that
> the
> >> > >>>>>>>> two
> >> > >>>>>>> systems
> >> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs "topic"
> >> and
> >> > >>>>>>> different
> >> > >>>>>>>> config and monitoring systems means you kind of have to learn
> >> > >>>>>>>> Kafka's
> >> > >>>>>>> way,
> >> > >>>>>>>> then learn Samza's slightly different way, then kind of
> >> > >>>>>>>> understand
> >> > >>>>> how
> >> > >>>>>>> they
> >> > >>>>>>>> map to each other, which having walked a few people through
> >> this
> >> > >>>>>>>> is surprisingly tricky for folks to get.
> >> > >>>>>>>>
> >> > >>>>>>>> Since I have been spending a lot of time on airplanes I
> hacked
> >> > >>>>>>>> up an ernest but still somewhat incomplete prototype of what
> >> > >>>>>>>> this would
> >> > >>>>> look
> >> > >>>>>>>> like. This is just unceremoniously dumped into Kafka as it
> >> > >>>>>>>> required a
> >> > >>>>>> few
> >> > >>>>>>>> changes to the new consumer. Here is the code:
> >> > >>>>>>>>
> >> > >>>>>>>>
> >> > >>>>>>>
> >> > >>>>>>
> >> > >>>>>
> >> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >> > >>>>> /apache/kafka/clients/streaming
> >> > >>>>>>>>
> >> > >>>>>>>> For the purpose of the prototype I just liberally renamed
> >> > >>>>>>>> everything
> >> > >>>>> to
> >> > >>>>>>>> try to align it with Kafka with no regard for compatibility.
> >> > >>>>>>>>
> >> > >>>>>>>> To use this would be something like this:
> >> > >>>>>>>> Properties props = new Properties();
> >> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> >> > >>>>>>>> StreamingConfig config = new
> >> > >>>>> StreamingConfig(props);
> >> > >>>>>>> config.subscribe("test-topic-1",
> >> > >>>>>>>> "test-topic-2");
> >> config.processor(ExampleStreamProcessor.class);
> >> > >>>>>>> config.serialization(new
> >> > >>>>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> >> > >>>>>> container =
> >> > >>>>>>>> new KafkaStreaming(config); container.run();
> >> > >>>>>>>>
> >> > >>>>>>>> KafkaStreaming is basically the SamzaContainer;
> StreamProcessor
> >> > >>>>>>>> is basically StreamTask.
> >> > >>>>>>>>
> >> > >>>>>>>> So rather than putting all the class names in a file and then
> >> > >>>>>>>> having
> >> > >>>>>> the
> >> > >>>>>>>> job assembled by reflection, you just instantiate the
> container
> >> > >>>>>>>> programmatically. Work is balanced over however many
> instances
> >> > >>>>>>>> of
> >> > >>>>> this
> >> > >>>>>>> are
> >> > >>>>>>>> alive at any time (i.e. if an instance dies, new tasks are
> >> added
> >> > >>>>>>>> to
> >> > >>>>> the
> >> > >>>>>>>> existing containers without shutting them down).
> >> > >>>>>>>>
> >> > >>>>>>>> We would provide some glue for running this stuff in YARN via
> >> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> >> > >>>>>>>> but from the
> >> > >>>>>> point
> >> > >>>>>>> of
> >> > >>>>>>>> view of these frameworks these stream processing jobs are
> just
> >> > >>>>>> stateless
> >> > >>>>>>>> services that can come and go and expand and contract at
> will.
> >> > >>>>>>>> There
> >> > >>>>> is
> >> > >>>>>>> no
> >> > >>>>>>>> more custom scheduler.
> >> > >>>>>>>>
> >> > >>>>>>>> Here are some relevant details:
> >> > >>>>>>>>
> >> > >>>>>>>>  1. It is only ~1300 lines of code, it would get larger if we
> >> > >>>>>>>>  productionized but not vastly larger. We really do get a ton
> >> > >>>>>>>> of
> >> > >>>>>>> leverage
> >> > >>>>>>>>  out of Kafka.
> >> > >>>>>>>>  2. Partition management is fully delegated to the new
> >> consumer.
> >> > >>>>> This
> >> > >>>>>>>>  is nice since now any partition management strategy
> available
> >> > >>>>>>>> to
> >> > >>>>>> Kafka
> >> > >>>>>>>>  consumer is also available to Samza (and vice versa) and
> with
> >> > >>>>>>>> the
> >> > >>>>>>> exact
> >> > >>>>>>>>  same configs.
> >> > >>>>>>>>  3. It supports state as well as state reuse
> >> > >>>>>>>>
> >> > >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
> >> > >>>>>>>>
> >> > >>>>>>>> -Jay
> >> > >>>>>>>>
> >> > >>>>>>>>
> >> > >>>>>>>>
> >> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> >> > >>>>>> criccomini@apache.org>
> >> > >>>>>>>> wrote:
> >> > >>>>>>>>
> >> > >>>>>>>>> Hey all,
> >> > >>>>>>>>>
> >> > >>>>>>>>> I have had some discussions with Samza engineers at LinkedIn
> >> > >>>>>>>>> and
> >> > >>>>>>> Confluent
> >> > >>>>>>>>> and we came up with a few observations and would like to
> >> > >>>>>>>>> propose
> >> > >>>>> some
> >> > >>>>>>>>> changes.
> >> > >>>>>>>>>
> >> > >>>>>>>>> We've observed some things that I want to call out about
> >> > >>>>>>>>> Samza's
> >> > >>>>>> design,
> >> > >>>>>>>>> and I'd like to propose some changes.
> >> > >>>>>>>>>
> >> > >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
> >> > >>>>>>>>> * Samza is too pluggable.
> >> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> >> > >>>>>>>>> APIs
> >> > >>>>> are
> >> > >>>>>>>>> trying to solve a lot of the same problems.
> >> > >>>>>>>>>
> >> > >>>>>>>>> All three of these issues are related, but I'll address them
> >> in
> >> > >>>>> order.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Deployment
> >> > >>>>>>>>>
> >> > >>>>>>>>> Samza strongly depends on the use of a dynamic deployment
> >> > >>>>>>>>> scheduler
> >> > >>>>>> such
> >> > >>>>>>>>> as
> >> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> >> > >>>>>>>>> there
> >> > >>>>>> would
> >> > >>>>>>>>> be
> >> > >>>>>>>>> one or two winners in this area, and we could support them,
> >> and
> >> > >>>>>>>>> the
> >> > >>>>>> rest
> >> > >>>>>>>>> would go away. In reality, there are many variations.
> >> > >>>>>>>>> Furthermore,
> >> > >>>>>> many
> >> > >>>>>>>>> people still prefer to just start their processors like
> normal
> >> > >>>>>>>>> Java processes, and use traditional deployment scripts such
> as
> >> > >>>>>>>>> Fabric,
> >> > >>>>>> Chef,
> >> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> >> > >>>>>>>>> Samza start-up process really painful for first time users.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Dynamic deployment as a requirement was also a bit of a
> >> > >>>>>>>>> mis-fire
> >> > >>>>>> because
> >> > >>>>>>>>> of
> >> > >>>>>>>>> a fundamental misunderstanding between the nature of batch
> >> jobs
> >> > >>>>>>>>> and
> >> > >>>>>>> stream
> >> > >>>>>>>>> processing jobs. Early on, we made conscious effort to favor
> >> > >>>>>>>>> the
> >> > >>>>>> Hadoop
> >> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked and was
> well
> >> > >>>>>>> understood.
> >> > >>>>>>>>> One thing that we missed was that batch jobs have a definite
> >> > >>>>>> beginning,
> >> > >>>>>>>>> and
> >> > >>>>>>>>> end, and stream processing jobs don't (usually). This leads
> to
> >> > >>>>>>>>> a
> >> > >>>>> much
> >> > >>>>>>>>> simpler scheduling problem for stream processors. You
> >> basically
> >> > >>>>>>>>> just
> >> > >>>>>>> need
> >> > >>>>>>>>> to find a place to start the processor, and start it. The
> way
> >> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> >> > >>>>>>>>> being "full". We always
> >> > >>>>>> add
> >> > >>>>>>>>> more machines. The problem with coupling Samza with a
> >> scheduler
> >> > >>>>>>>>> is
> >> > >>>>>> that
> >> > >>>>>>>>> Samza (as a framework) now has to handle deployment. This
> >> pulls
> >> > >>>>>>>>> in a
> >> > >>>>>>> bunch
> >> > >>>>>>>>> of things such as configuration distribution (config
> stream),
> >> > >>>>>>>>> shell
> >> > >>>>>>> scrips
> >> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff),
> >> etc.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Another reason for requiring dynamic deployment was to
> support
> >> > >>>>>>>>> data locality. If you want to have locality, you need to put
> >> > >>>>>>>>> your
> >> > >>>>>> processors
> >> > >>>>>>>>> close to the data they're processing. Upon further
> >> > >>>>>>>>> investigation,
> >> > >>>>>>> though,
> >> > >>>>>>>>> this feature is not that beneficial. There is some good
> >> > >>>>>>>>> discussion
> >> > >>>>>> about
> >> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
> >> > >>>>>>>>> Map/Reduce
> >> > >>>>>> path,
> >> > >>>>>>>>> but
> >> > >>>>>>>>> there are some fundamental differences between HDFS and
> Kafka.
> >> > >>>>>>>>> HDFS
> >> > >>>>>> has
> >> > >>>>>>>>> blocks, while Kafka has partitions. This leads to less
> >> > >>>>>>>>> optimization potential with stream processors on top of
> Kafka.
> >> > >>>>>>>>>
> >> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't have
> any
> >> > >>>>>>>>> built
> >> > >>>>> in
> >> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> >> > >>>>>>>>> deployment scheduling system to handle restarts when a
> >> > >>>>>>>>> processor dies. This has
> >> > >>>>>>> made
> >> > >>>>>>>>> it very difficult to write a standalone Samza container
> >> > >>>> (SAMZA-516).
> >> > >>>>>>>>>
> >> > >>>>>>>>> Pluggability
> >> > >>>>>>>>>
> >> > >>>>>>>>> In some cases pluggability is good, but I think that we've
> >> gone
> >> > >>>>>>>>> too
> >> > >>>>>> far
> >> > >>>>>>>>> with it. Currently, Samza has:
> >> > >>>>>>>>>
> >> > >>>>>>>>> * Pluggable config.
> >> > >>>>>>>>> * Pluggable metrics.
> >> > >>>>>>>>> * Pluggable deployment systems.
> >> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer,
> SystemProducer,
> >> > >>>> etc).
> >> > >>>>>>>>> * Pluggable serdes.
> >> > >>>>>>>>> * Pluggable storage engines.
> >> > >>>>>>>>> * Pluggable strategies for just about every component
> >> > >>>>> (MessageChooser,
> >> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> >> > >>>>>>>>>
> >> > >>>>>>>>> There's probably more that I've forgotten, as well. Some of
> >> > >>>>>>>>> these
> >> > >>>>> are
> >> > >>>>>>>>> useful, but some have proven not to be. This all comes at a
> >> cost:
> >> > >>>>>>>>> complexity. This complexity is making it harder for our
> users
> >> > >>>>>>>>> to
> >> > >>>>> pick
> >> > >>>>>> up
> >> > >>>>>>>>> and use Samza out of the box. It also makes it difficult for
> >> > >>>>>>>>> Samza developers to reason about what the characteristics of
> >> > >>>>>>>>> the container (since the characteristics change depending on
> >> > >>>>>>>>> which plugins are use).
> >> > >>>>>>>>>
> >> > >>>>>>>>> The issues with pluggability are most visible in the System
> >> APIs.
> >> > >>>>> What
> >> > >>>>>>>>> Samza really requires to be functional is Kafka as its
> >> > >>>>>>>>> transport
> >> > >>>>>> layer.
> >> > >>>>>>>>> But
> >> > >>>>>>>>> we've conflated two unrelated use cases into one API:
> >> > >>>>>>>>>
> >> > >>>>>>>>> 1. Get data into/out of Kafka.
> >> > >>>>>>>>> 2. Process the data in Kafka.
> >> > >>>>>>>>>
> >> > >>>>>>>>> The current System API supports both of these use cases. The
> >> > >>>>>>>>> problem
> >> > >>>>>> is,
> >> > >>>>>>>>> we
> >> > >>>>>>>>> actually want different features for each use case. By
> >> papering
> >> > >>>>>>>>> over
> >> > >>>>>>> these
> >> > >>>>>>>>> two use cases, and providing a single API, we've introduced
> a
> >> > >>>>>>>>> ton of
> >> > >>>>>>> leaky
> >> > >>>>>>>>> abstractions.
> >> > >>>>>>>>>
> >> > >>>>>>>>> For example, what we'd really like in (2) is to have
> >> > >>>>>>>>> monotonically increasing longs for offsets (like Kafka).
> This
> >> > >>>>>>>>> would be at odds
> >> > >>>>> with
> >> > >>>>>>> (1),
> >> > >>>>>>>>> though, since different systems have different
> >> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> >> > >>>>>>>>> There was discussion both on the mailing list and the SQL
> >> JIRAs
> >> > >>>>> about
> >> > >>>>>>> the
> >> > >>>>>>>>> need for this.
> >> > >>>>>>>>>
> >> > >>>>>>>>> The same thing holds true for replayability. Kafka allows us
> >> to
> >> > >>>>> rewind
> >> > >>>>>>>>> when
> >> > >>>>>>>>> we have a failure. Many other systems don't. In some cases,
> >> > >>>>>>>>> systems
> >> > >>>>>>> return
> >> > >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> because
> >> > >>>>>>>>> they
> >> > >>>>>> have
> >> > >>>>>>> no
> >> > >>>>>>>>> offsets.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Partitioning is another example. Kafka supports
> partitioning,
> >> > >>>>>>>>> but
> >> > >>>>> many
> >> > >>>>>>>>> systems don't. We model this by having a single partition
> for
> >> > >>>>>>>>> those systems. Still, other systems model partitioning
> >> > >>>> differently (e.g.
> >> > >>>>>>>>> Kinesis).
> >> > >>>>>>>>>
> >> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating streams
> in
> >> a
> >> > >>>>>>>>> system-agnostic way is almost impossible. As is modeling
> >> > >>>>>>>>> metadata
> >> > >>>>> for
> >> > >>>>>>> the
> >> > >>>>>>>>> system (replication factor, partitions, location, etc). The
> >> > >>>>>>>>> list
> >> > >>>>> goes
> >> > >>>>>>> on.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Duplicate work
> >> > >>>>>>>>>
> >> > >>>>>>>>> At the time that we began writing Samza, Kafka's consumer
> and
> >> > >>>>> producer
> >> > >>>>>>>>> APIs
> >> > >>>>>>>>> had a relatively weak feature set. On the consumer-side, you
> >> > >>>>>>>>> had two
> >> > >>>>>>>>> options: use the high level consumer, or the simple
> consumer.
> >> > >>>>>>>>> The
> >> > >>>>>>> problem
> >> > >>>>>>>>> with the high-level consumer was that it controlled your
> >> > >>>>>>>>> offsets, partition assignments, and the order in which you
> >> > >>>>>>>>> received messages. The
> >> > >>>>> problem
> >> > >>>>>>>>> with
> >> > >>>>>>>>> the simple consumer is that it's not simple. It's basic. You
> >> > >>>>>>>>> end up
> >> > >>>>>>> having
> >> > >>>>>>>>> to handle a lot of really low-level stuff that you
> shouldn't.
> >> > >>>>>>>>> We
> >> > >>>>>> spent a
> >> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust.
> >> It
> >> > >>>>>>>>> also allows us to support some cool features:
> >> > >>>>>>>>>
> >> > >>>>>>>>> * Per-partition message ordering and prioritization.
> >> > >>>>>>>>> * Tight control over partition assignment to support joins,
> >> > >>>>>>>>> global
> >> > >>>>>> state
> >> > >>>>>>>>> (if we want to implement it :)), etc.
> >> > >>>>>>>>> * Tight control over offset checkpointing.
> >> > >>>>>>>>>
> >> > >>>>>>>>> What we didn't realize at the time is that these features
> >> > >>>>>>>>> should
> >> > >>>>>>> actually
> >> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> >> > >>>>>> processors)
> >> > >>>>>>>>> end up wanting to do things like joins and partition
> >> > >>>>>>>>> assignment. The
> >> > >>>>>>> Kafka
> >> > >>>>>>>>> community has come to the same conclusion. They're adding a
> >> ton
> >> > >>>>>>>>> of upgrades into their new Kafka consumer implementation.
> To a
> >> > >>>>>>>>> large extent,
> >> > >>>>> it's
> >> > >>>>>>>>> duplicate work to what we've already done in Samza.
> >> > >>>>>>>>>
> >> > >>>>>>>>> On top of this, Kafka ended up taking a very similar
> approach
> >> > >>>>>>>>> to
> >> > >>>>>> Samza's
> >> > >>>>>>>>> KafkaCheckpointManager implementation for handling offset
> >> > >>>>>> checkpointing.
> >> > >>>>>>>>> Like Samza, Kafka's new offset management feature stores
> >> offset
> >> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them from
> the
> >> > >>>>>>>>> broker.
> >> > >>>>>>>>>
> >> > >>>>>>>>> A lot of this seems like a waste, since we could have shared
> >> > >>>>>>>>> the
> >> > >>>>> work
> >> > >>>>>> if
> >> > >>>>>>>>> it
> >> > >>>>>>>>> had been done in Kafka from the get-go.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Vision
> >> > >>>>>>>>>
> >> > >>>>>>>>> All of this leads me to a rather radical proposal. Samza is
> >> > >>>>> relatively
> >> > >>>>>>>>> stable at this point. I'd venture to say that we're near a
> 1.0
> >> > >>>>>> release.
> >> > >>>>>>>>> I'd
> >> > >>>>>>>>> like to propose that we take what we've learned, and begin
> >> > >>>>>>>>> thinking
> >> > >>>>>>> about
> >> > >>>>>>>>> Samza beyond 1.0. What would we change if we were starting
> >> from
> >> > >>>>>> scratch?
> >> > >>>>>>>>> My
> >> > >>>>>>>>> proposal is to:
> >> > >>>>>>>>>
> >> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> >> > >>>>>>>>> processors, and eliminate all direct dependences on YARN,
> >> Mesos,
> >> > >>>> etc.
> >> > >>>>>>>>> 2. Make a definitive call to support only Kafka as the
> stream
> >> > >>>>>> processing
> >> > >>>>>>>>> layer.
> >> > >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> >> > >>>>>>>>> config
> >> > >>>>>>> systems,
> >> > >>>>>>>>> and simply use Kafka's instead.
> >> > >>>>>>>>>
> >> > >>>>>>>>> This would fix all of the issues that I outlined above. It
> >> > >>>>>>>>> should
> >> > >>>>> also
> >> > >>>>>>>>> shrink the Samza code base pretty dramatically. Supporting
> >> only
> >> > >>>>>>>>> a standalone container will allow Samza to be executed on
> YARN
> >> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> >> > >>>>>>>>> in-house
> >> > >>>>>>> deployment
> >> > >>>>>>>>> systems. This should make life a lot easier for new users.
> >> > >>>>>>>>> Imagine
> >> > >>>>>>> having
> >> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in mailing
> >> list
> >> > >>>>>> traffic
> >> > >>>>>>>>> will be pretty dramatic.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The reality
> is,
> >> > >>>>> everyone
> >> > >>>>>>>>> that
> >> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically require
> >> it
> >> > >>>>>> already
> >> > >>>>>>> in
> >> > >>>>>>>>> order for most features to work. Those that are using other
> >> > >>>>>>>>> systems
> >> > >>>>>> are
> >> > >>>>>>>>> generally using it for ingest into Kafka (1), and then they
> do
> >> > >>>>>>>>> the processing on top. There is already discussion (
> >> > >>>>>>>>>
> >> > >>>>>>>
> >> > >>>>>>
> >> > >>>>>
> >> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >> > >>>>> 767
> >> > >>>>>>>>> )
> >> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Once we make the call to couple with Kafka, we can leverage
> a
> >> > >>>>>>>>> ton of
> >> > >>>>>>> their
> >> > >>>>>>>>> ecosystem. We no longer have to maintain our own config,
> >> > >>>>>>>>> metrics,
> >> > >>>>> etc.
> >> > >>>>>>> We
> >> > >>>>>>>>> can all share the same libraries, and make them better. This
> >> > >>>>>>>>> will
> >> > >>>>> also
> >> > >>>>>>>>> allow us to share the consumer/producer APIs, and will let
> us
> >> > >>>>> leverage
> >> > >>>>>>>>> their offset management and partition management, rather
> than
> >> > >>>>>>>>> having
> >> > >>>>>> our
> >> > >>>>>>>>> own. All of the coordinator stream code would go away, as
> >> would
> >> > >>>>>>>>> most
> >> > >>>>>> of
> >> > >>>>>>>>> the
> >> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some
> partition
> >> > >>>>>>> management
> >> > >>>>>>>>> features into the Kafka broker, but they're already moving
> in
> >> > >>>>>>>>> that direction with the new consumer API. The features we
> have
> >> > >>>>>>>>> for
> >> > >>>>>> partition
> >> > >>>>>>>>> assignment aren't unique to Samza, and seem like they should
> >> be
> >> > >>>>>>>>> in
> >> > >>>>>> Kafka
> >> > >>>>>>>>> anyway. There will always be some niche usages which will
> >> > >>>>>>>>> require
> >> > >>>>>> extra
> >> > >>>>>>>>> care and hence full control over partition assignments much
> >> > >>>>>>>>> like the
> >> > >>>>>>> Kafka
> >> > >>>>>>>>> low level consumer api. These would continue to be
> supported.
> >> > >>>>>>>>>
> >> > >>>>>>>>> These items will be good for the Samza community. They'll
> make
> >> > >>>>>>>>> Samza easier to use, and make it easier for developers to
> add
> >> > >>>>>>>>> new features.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Obviously this is a fairly large (and somewhat backwards
> >> > >>>>> incompatible
> >> > >>>>>>>>> change). If we choose to go this route, it's important that
> we
> >> > >>>>> openly
> >> > >>>>>>>>> communicate how we're going to provide a migration path from
> >> > >>>>>>>>> the
> >> > >>>>>>> existing
> >> > >>>>>>>>> APIs to the new ones (if we make incompatible changes). I
> >> think
> >> > >>>>>>>>> at a minimum, we'd probably need to provide a wrapper to
> allow
> >> > >>>>>>>>> existing StreamTask implementations to continue running on
> the
> >> > >>>> new container.
> >> > >>>>>>> It's
> >> > >>>>>>>>> also important that we openly communicate about timing, and
> >> > >>>>>>>>> stages
> >> > >>>>> of
> >> > >>>>>>> the
> >> > >>>>>>>>> migration.
> >> > >>>>>>>>>
> >> > >>>>>>>>> If you made it this far, I'm sure you have opinions. :)
> Please
> >> > >>>>>>>>> send
> >> > >>>>>> your
> >> > >>>>>>>>> thoughts and feedback.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Cheers,
> >> > >>>>>>>>> Chris
> >> > >>>>>>>>>
> >> > >>>>>>>>
> >> > >>>>>>>>
> >> > >>>>>>>
> >> > >>>>>>
> >> > >>>>>>
> >> > >>>>>>
> >> > >>>>>> --
> >> > >>>>>> -- Guozhang
> >> > >>>>>>
> >> > >>>>>
> >> > >>>>
> >> > >>
> >> > >>
> >> >
> >> >
> >> >
> >>
> >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Gianmarco De Francisci Morales <gd...@apache.org>.
Forgot to add. On the naming issues, Kafka Metamorphosis is a clear winner
:)

--
Gianmarco

On 7 July 2015 at 13:26, Gianmarco De Francisci Morales <gd...@apache.org>
wrote:

> Hi,
>
> @Martin, thanks for you comments.
> Maybe I'm missing some important point, but I think coupling the releases
> is actually a *good* thing.
> To make an example, would it be better if the MR and HDFS components of
> Hadoop had different release schedules?
>
> Actually, keeping the discussion in a single place would make agreeing on
> releases (and backwards compatibility) much easier, as everybody would be
> responsible for the whole codebase.
>
> That said, I like the idea of absorbing samza-core as a sub-project, and
> leave the fancy stuff separate.
> It probably gives 90% of the benefits we have been discussing here.
>
> Cheers,
>
> --
> Gianmarco
>
> On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:
>
>> Hey Martin,
>>
>> I agree coupling release schedules is a downside.
>>
>> Definitely we can try to solve some of the integration problems in
>> Confluent Platform or in other distributions. But I think this ends up
>> being really shallow. I guess I feel to really get a good user experience
>> the two systems have to kind of feel like part of the same thing and you
>> can't really add that in later--you can put both in the same downloadable
>> tar file but it doesn't really give a very cohesive feeling. I agree that
>> ultimately any of the project stuff is as much social and naming as
>> anything else--theoretically two totally independent projects could work
>> to
>> tightly align. In practice this seems to be quite difficult though.
>>
>> For the frameworks--totally agree it would be good to maintain the
>> framework support with the project. In some cases there may not be too
>> much
>> there since the integration gets lighter but I think whatever stubs you
>> need should be included. So no I definitely wasn't trying to imply
>> dropping
>> support for these frameworks, just making the integration lighter by
>> separating process management from partition management.
>>
>> You raise two good points we would have to figure out if we went down the
>> alignment path:
>> 1. With respect to the name, yeah I think the first question is whether
>> some "re-branding" would be worth it. If so then I think we can have a big
>> thread on the name. I'm definitely not set on Kafka Streaming or Kafka
>> Streams I was just using them to be kind of illustrative. I agree with
>> your
>> critique of these names, though I think people would get the idea.
>> 2. Yeah you also raise a good point about how to "factor" it. Here are the
>> options I see (I could get enthusiastic about any of them):
>>    a. One repo for both Kafka and Samza
>>    b. Two repos, retaining the current seperation
>>    c. Two repos, the equivalent of samza-api and samza-core is absorbed
>> almost like a third client
>>
>> Cheers,
>>
>> -Jay
>>
>> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <ma...@kleppmann.com>
>> wrote:
>>
>> > Ok, thanks for the clarifications. Just a few follow-up comments.
>> >
>> > - I see the appeal of merging with Kafka or becoming a subproject: the
>> > reasons you mention are good. The risk I see is that release schedules
>> > become coupled to each other, which can slow everyone down, and large
>> > projects with many contributors are harder to manage. (Jakob, can you
>> speak
>> > from experience, having seen a wider range of Hadoop ecosystem
>> projects?)
>> >
>> > Some of the goals of a better unified developer experience could also be
>> > solved by integrating Samza nicely into a Kafka distribution (such as
>> > Confluent's). I'm not against merging projects if we decide that's the
>> way
>> > to go, just pointing out the same goals can perhaps also be achieved in
>> > other ways.
>> >
>> > - With regard to dropping the YARN dependency: are you proposing that
>> > Samza doesn't give any help to people wanting to run on
>> YARN/Mesos/AWS/etc?
>> > So the docs would basically have a link to Slider and nothing else? Or
>> > would we maintain integrations with a bunch of popular deployment
>> methods
>> > (e.g. the necessary glue and shell scripts to make Samza work with
>> Slider)?
>> >
>> > I absolutely think it's a good idea to have the "as a library" and "as a
>> > process" (using Yi's taxonomy) options for people who want them, but I
>> > think there should also be a low-friction path for common "as a service"
>> > deployment methods, for which we probably need to maintain integrations.
>> >
>> > - Project naming: "Kafka Streams" seems odd to me, because Kafka is all
>> > about streams already. Perhaps "Kafka Transformers" or "Kafka Filters"
>> > would be more apt?
>> >
>> > One suggestion: perhaps the core of Samza (stream transformation with
>> > state management -- i.e. the "Samza as a library" bit) could become
>> part of
>> > Kafka, while higher-level tools such as streaming SQL and integrations
>> with
>> > deployment frameworks remain in a separate project? In other words,
>> Kafka
>> > would absorb the proven, stable core of Samza, which would become the
>> > "third Kafka client" mentioned early in this thread. The Samza project
>> > would then target that third Kafka client as its base API, and the
>> project
>> > would be freed up to explore more experimental new horizons.
>> >
>> > Martin
>> >
>> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
>> >
>> > > Hey Martin,
>> > >
>> > > For the YARN/Mesos/etc decoupling I actually don't think it ties our
>> > hands
>> > > at all, all it does is refactor things. The division of
>> responsibility is
>> > > that Samza core is responsible for task lifecycle, state, and
>> partition
>> > > management (using the Kafka co-ordinator) but it is NOT responsible
>> for
>> > > packaging, configuration deployment or execution of processes. The
>> > problem
>> > > of packaging and starting these processes is
>> > > framework/environment-specific. This leaves individual frameworks to
>> be
>> > as
>> > > fancy or vanilla as they like. So you can get simple stateless
>> support in
>> > > YARN, Mesos, etc using their off-the-shelf app framework (Slider,
>> > Marathon,
>> > > etc). These are well known by people and have nice UIs and a lot of
>> > > flexibility. I don't think they have node affinity as a built in
>> option
>> > > (though I could be wrong). So if we want that we can either wait for
>> them
>> > > to add it or do a custom framework to add that feature (as now).
>> > Obviously
>> > > if you manage things with old-school ops tools (puppet/chef/etc) you
>> get
>> > > locality easily. The nice thing, though, is that all the samza
>> "business
>> > > logic" around partition management and fault tolerance is in Samza
>> core
>> > so
>> > > it is shared across frameworks and the framework specific bit is just
>> > > whether it is smart enough to try to get the same host when a job is
>> > > restarted.
>> > >
>> > > With respect to the Kafka-alignment, yeah I think the goal would be
>> (a)
>> > > actually get better alignment in user experience, and (b) express
>> this in
>> > > the naming and project branding. Specifically:
>> > > 1. Website/docs, it would be nice for the "transformation" api to be
>> > > discoverable in the main Kafka docs--i.e. be able to explain when to
>> use
>> > > the consumer and when to use the stream processing functionality and
>> lead
>> > > people into that experience.
>> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that has
>> both
>> > > Kafka and the stream processing part and they actually work together.
>> > > 3. Unify the programming experience so the client and Samza api share
>> > > config/monitoring/naming/packaging/etc.
>> > >
>> > > I think sub-projects keep separate committers and can have a separate
>> > repo,
>> > > but I'm actually not really sure (I can't find a definition of a
>> > subproject
>> > > in Apache).
>> > >
>> > > Basically at a high-level you want the experience to "feel" like a
>> single
>> > > system, not to relatively independent things that are kind of
>> awkwardly
>> > > glued together.
>> > >
>> > > I think if we did that they having naming or branding like "kafka
>> > > streaming" or "kafka streams" or something like that would actually
>> do a
>> > > good job of conveying what it is. I do that this would help adoption
>> > quite
>> > > a lot as it would correctly convey that using Kafka Streaming with
>> Kafka
>> > is
>> > > a fairly seamless experience and Kafka is pretty heavily adopted at
>> this
>> > > point.
>> > >
>> > > Fwiw we actually considered this model originally when open sourcing
>> > Samza,
>> > > however at that time Kafka was relatively unknown and we decided not
>> to
>> > do
>> > > it since we felt it would be limiting. From my point of view the three
>> > > things have changed (1) Kafka is now really heavily used for stream
>> > > processing, (2) we learned that abstracting out the stream well is
>> > > basically impossible, (3) we learned it is really hard to keep the two
>> > > things feeling like a single product.
>> > >
>> > > -Jay
>> > >
>> > >
>> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <
>> martin@kleppmann.com>
>> > > wrote:
>> > >
>> > >> Hi all,
>> > >>
>> > >> Lots of good thoughts here.
>> > >>
>> > >> I agree with the general philosophy of tying Samza more firmly to
>> Kafka.
>> > >> After I spent a while looking at integrating other message brokers
>> (e.g.
>> > >> Kinesis) with SystemConsumer, I came to the conclusion that
>> > SystemConsumer
>> > >> tacitly assumes a model so much like Kafka's that pretty much nobody
>> but
>> > >> Kafka actually implements it. (Databus is perhaps an exception, but
>> it
>> > >> isn't widely used outside of LinkedIn.) Thus, making Samza fully
>> > dependent
>> > >> on Kafka acknowledges that the system-independence was never as real
>> as
>> > we
>> > >> perhaps made it out to be. The gains of code reuse are real.
>> > >>
>> > >> The idea of decoupling Samza from YARN has also always been
>> appealing to
>> > >> me, for various reasons already mentioned in this thread. Although
>> > making
>> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems
>> laudable,
>> > I am
>> > >> a little concerned that it will restrict us to a lowest common
>> > denominator.
>> > >> For example, would host affinity (SAMZA-617) still be possible? For
>> jobs
>> > >> with large amounts of state, I think SAMZA-617 would be a big boon,
>> > since
>> > >> restoring state off the changelog on every single restart is painful,
>> > due
>> > >> to long recovery times. It would be a shame if the decoupling from
>> YARN
>> > >> made host affinity impossible.
>> > >>
>> > >> Jay, a question about the proposed API for instantiating a job in
>> code
>> > >> (rather than a properties file): when submitting a job to a cluster,
>> is
>> > the
>> > >> idea that the instantiation code runs on a client somewhere, which
>> then
>> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that
>> code
>> > run
>> > >> on each container that is part of the job (in which case, how does
>> the
>> > job
>> > >> submission to the cluster work)?
>> > >>
>> > >> I agree with Garry that it doesn't feel right to make a 1.0 release
>> > with a
>> > >> plan for it to be immediately obsolete. So if this is going to
>> happen, I
>> > >> think it would be more honest to stick with 0.* version numbers until
>> > the
>> > >> library-ified Samza has been implemented, is stable and widely used.
>> > >>
>> > >> Should the new Samza be a subproject of Kafka? There is precedent for
>> > >> tight coupling between different Apache projects (e.g. Curator and
>> > >> Zookeeper, or Slider and YARN), so I think remaining separate would
>> be
>> > ok.
>> > >> Even if Samza is fully dependent on Kafka, there is enough substance
>> in
>> > >> Samza that it warrants being a separate project. An argument in
>> favour
>> > of
>> > >> merging would be if we think Kafka has a much stronger "brand
>> presence"
>> > >> than Samza; I'm ambivalent on that one. If the Kafka project is
>> willing
>> > to
>> > >> endorse Samza as the "official" way of doing stateful stream
>> > >> transformations, that would probably have much the same effect as
>> > >> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
>> > >> collaboration between the two projects will be needed in any case.
>> > >>
>> > >> From a project management perspective, I guess the "new Samza" would
>> > have
>> > >> to be developed on a branch alongside ongoing maintenance of the
>> current
>> > >> line of development? I think it would be important to continue
>> > supporting
>> > >> existing users, and provide a graceful migration path to the new
>> > version.
>> > >> Leaving the current versions unsupported and forcing people to
>> rewrite
>> > >> their jobs would send a bad signal.
>> > >>
>> > >> Best,
>> > >> Martin
>> > >>
>> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
>> > >>
>> > >>> Hey Garry,
>> > >>>
>> > >>> Yeah that's super frustrating. I'd be happy to chat more about this
>> if
>> > >>> you'd be interested. I think Chris and I started with the idea of
>> "what
>> > >>> would it take to make Samza a kick-ass ingestion tool" but
>> ultimately
>> > we
>> > >>> kind of came around to the idea that ingestion and transformation
>> had
>> > >>> pretty different needs and coupling the two made things hard.
>> > >>>
>> > >>> For what it's worth I think copycat (KIP-26) actually will do what
>> you
>> > >> are
>> > >>> looking for.
>> > >>>
>> > >>> With regard to your point about slider, I don't necessarily
>> disagree.
>> > >> But I
>> > >>> think getting good YARN support is quite doable and I think we can
>> make
>> > >>> that work well. I think the issue this proposal solves is that
>> > >> technically
>> > >>> it is pretty hard to support multiple cluster management systems the
>> > way
>> > >>> things are now, you need to write an "app master" or "framework" for
>> > each
>> > >>> and they are all a little different so testing is really hard. In
>> the
>> > >>> absence of this we have been stuck with just YARN which has
>> fantastic
>> > >>> penetration in the Hadoopy part of the org, but zero penetration
>> > >> elsewhere.
>> > >>> Given the huge amount of work being put in to slider, marathon, aws
>> > >>> tooling, not to mention the umpteen related packaging technologies
>> > people
>> > >>> want to use (Docker, Kubernetes, various cloud-specific deploy
>> tools,
>> > >> etc)
>> > >>> I really think it is important to get this right.
>> > >>>
>> > >>> -Jay
>> > >>>
>> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>> > >>> g.turkington@improvedigital.com> wrote:
>> > >>>
>> > >>>> Hi all,
>> > >>>>
>> > >>>> I think the question below re does Samza become a sub-project of
>> Kafka
>> > >>>> highlights the broader point around migration. Chris mentions
>> Samza's
>> > >>>> maturity is heading towards a v1 release but I'm not sure it feels
>> > >> right to
>> > >>>> launch a v1 then immediately plan to deprecate most of it.
>> > >>>>
>> > >>>> From a selfish perspective I have some guys who have started
>> working
>> > >> with
>> > >>>> Samza and building some new consumers/producers was next up. Sounds
>> > like
>> > >>>> that is absolutely not the direction to go. I need to look into the
>> > KIP
>> > >> in
>> > >>>> more detail but for me the attractiveness of adding new Samza
>> > >>>> consumer/producers -- even if yes all they were doing was really
>> > getting
>> > >>>> data into and out of Kafka --  was to avoid  having to worry about
>> the
>> > >>>> lifecycle management of external clients. If there is a generic
>> Kafka
>> > >>>> ingress/egress layer that I can plug a new connector into and have
>> a
>> > >> lot of
>> > >>>> the heavy lifting re scale and reliability done for me then it
>> gives
>> > me
>> > >> all
>> > >>>> the pushing new consumers/producers would. If not then it
>> complicates
>> > my
>> > >>>> operational deployments.
>> > >>>>
>> > >>>> Which is similar to my other question with the proposal -- if we
>> > build a
>> > >>>> fully available/stand-alone Samza plus the requisite shims to
>> > integrate
>> > >>>> with Slider etc I suspect the former may be a lot more work than we
>> > >> think.
>> > >>>> We may make it much easier for a newcomer to get something running
>> but
>> > >>>> having them step up and get a reliable production deployment may
>> still
>> > >>>> dominate mailing list  traffic, if for different reasons than
>> today.
>> > >>>>
>> > >>>> Don't get me wrong -- I'm comfortable with making the Samza
>> dependency
>> > >> on
>> > >>>> Kafka much more explicit and I absolutely see the benefits  in the
>> > >>>> reduction of duplication and clashing terminologies/abstractions
>> that
>> > >>>> Chris/Jay describe. Samza as a library would likely be a very nice
>> > tool
>> > >> to
>> > >>>> add to the Kafka ecosystem. I just have the concerns above re the
>> > >>>> operational side.
>> > >>>>
>> > >>>> Garry
>> > >>>>
>> > >>>> -----Original Message-----
>> > >>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
>> > >>>> Sent: 02 July 2015 12:56
>> > >>>> To: dev@samza.apache.org
>> > >>>> Subject: Re: Thoughts and obesrvations on Samza
>> > >>>>
>> > >>>> Very interesting thoughts.
>> > >>>> From outside, I have always perceived Samza as a computing layer
>> over
>> > >>>> Kafka.
>> > >>>>
>> > >>>> The question, maybe a bit provocative, is "should Samza be a
>> > sub-project
>> > >>>> of Kafka then?"
>> > >>>> Or does it make sense to keep it as a separate project with a
>> separate
>> > >>>> governance?
>> > >>>>
>> > >>>> Cheers,
>> > >>>>
>> > >>>> --
>> > >>>> Gianmarco
>> > >>>>
>> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
>> > >>>>
>> > >>>>> Overall, I agree to couple with Kafka more tightly. Because Samza
>> de
>> > >>>>> facto is based on Kafka, and it should leverage what Kafka has. At
>> > the
>> > >>>>> same time, Kafka does not need to reinvent what Samza already
>> has. I
>> > >>>>> also like the idea of separating the ingestion and transformation.
>> > >>>>>
>> > >>>>> But it is a little difficult for me to image how the Samza will
>> look
>> > >>>> like.
>> > >>>>> And I feel Chris and Jay have a little difference in terms of how
>> > >>>>> Samza should look like.
>> > >>>>>
>> > >>>>> *** Will it look like what Jay's code shows (A client of Kakfa) ?
>> And
>> > >>>>> user's application code calls this client?
>> > >>>>>
>> > >>>>> 1. If we make Samza be a library of Kafka (like what the code
>> shows),
>> > >>>>> how do we implement auto-balance and fault-tolerance? Are they
>> taken
>> > >>>>> care by the Kafka broker or other mechanism, such as "Samza
>> worker"
>> > >>>>> (just make up the name) ?
>> > >>>>>
>> > >>>>> 2. What about other features, such as auto-scaling, shared state,
>> > >>>>> monitoring?
>> > >>>>>
>> > >>>>>
>> > >>>>> *** If we have Samza standalone, (is this what Chris suggests?)
>> > >>>>>
>> > >>>>> 1. we still need to ingest data from Kakfa and produce to it.
>> Then it
>> > >>>>> becomes the same as what Samza looks like now, except it does not
>> > rely
>> > >>>>> on Yarn anymore.
>> > >>>>>
>> > >>>>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
>> > >>>>> etc? Use Kafka code as the dependency?
>> > >>>>>
>> > >>>>>
>> > >>>>> Thanks,
>> > >>>>>
>> > >>>>> Fang, Yan
>> > >>>>> yanfang724@gmail.com
>> > >>>>>
>> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wangguoz@gmail.com
>> >
>> > >>>> wrote:
>> > >>>>>
>> > >>>>>> Read through the code example and it looks good to me. A few
>> > >>>>>> thoughts regarding deployment:
>> > >>>>>>
>> > >>>>>> Today Samza deploys as executable runnable like:
>> > >>>>>>
>> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
>> > >>>> --config-path=file://...
>> > >>>>>>
>> > >>>>>> And this proposal advocate for deploying Samza more as embedded
>> > >>>>>> libraries in user application code (ignoring the terminology
>> since
>> > >>>>>> it is not the
>> > >>>>> same
>> > >>>>>> as the prototype code):
>> > >>>>>>
>> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
>> > >>>>>> Thread(task); thread.start();
>> > >>>>>>
>> > >>>>>> I think both of these deployment modes are important for
>> different
>> > >>>>>> types
>> > >>>>> of
>> > >>>>>> users. That said, I think making Samza purely standalone is still
>> > >>>>>> sufficient for either runnable or library modes.
>> > >>>>>>
>> > >>>>>> Guozhang
>> > >>>>>>
>> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
>> > wrote:
>> > >>>>>>
>> > >>>>>>> Looks like gmail mangled the code example, it was supposed to
>> look
>> > >>>>>>> like
>> > >>>>>>> this:
>> > >>>>>>>
>> > >>>>>>> Properties props = new Properties();
>> > >>>>>>> props.put("bootstrap.servers", "localhost:4242");
>> StreamingConfig
>> > >>>>>>> config = new StreamingConfig(props);
>> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
>> > >>>>>>> config.processor(ExampleStreamProcessor.class);
>> > >>>>>>> config.serialization(new StringSerializer(), new
>> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
>> > >>>>>>> KafkaStreaming(config); container.run();
>> > >>>>>>>
>> > >>>>>>> -Jay
>> > >>>>>>>
>> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
>> > >>>> wrote:
>> > >>>>>>>
>> > >>>>>>>> Hey guys,
>> > >>>>>>>>
>> > >>>>>>>> This came out of some conversations Chris and I were having
>> > >>>>>>>> around
>> > >>>>>>> whether
>> > >>>>>>>> it would make sense to use Samza as a kind of data ingestion
>> > >>>>> framework
>> > >>>>>>> for
>> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
>> > >>>>>> combined
>> > >>>>>>>> with complaints around config and YARN and the discussion
>> around
>> > >>>>>>>> how
>> > >>>>> to
>> > >>>>>>>> best do a standalone mode.
>> > >>>>>>>>
>> > >>>>>>>> So the thought experiment was, given that Samza was basically
>> > >>>>>>>> already totally Kafka specific, what if you just embraced that
>> > >>>>>>>> and turned it
>> > >>>>>> into
>> > >>>>>>>> something less like a heavyweight framework and more like a
>> > >>>>>>>> third
>> > >>>>> Kafka
>> > >>>>>>>> client--a kind of "producing consumer" with state management
>> > >>>>>> facilities.
>> > >>>>>>>> Basically a library. Instead of a complex stream processing
>> > >>>>>>>> framework
>> > >>>>>>> this
>> > >>>>>>>> would actually be a very simple thing, not much more
>> complicated
>> > >>>>>>>> to
>> > >>>>> use
>> > >>>>>>> or
>> > >>>>>>>> operate than a Kafka consumer. As Chris said we thought about
>> it
>> > >>>>>>>> a
>> > >>>>> lot
>> > >>>>>> of
>> > >>>>>>>> what Samza (and the other stream processing systems were doing)
>> > >>>>> seemed
>> > >>>>>>> like
>> > >>>>>>>> kind of a hangover from MapReduce.
>> > >>>>>>>>
>> > >>>>>>>> Of course you need to ingest/output data to and from the stream
>> > >>>>>>>> processing. But when we actually looked into how that would
>> > >>>>>>>> work,
>> > >>>>> Samza
>> > >>>>>>>> isn't really an ideal data ingestion framework for a bunch of
>> > >>>>> reasons.
>> > >>>>>> To
>> > >>>>>>>> really do that right you need a pretty different internal data
>> > >>>>>>>> model
>> > >>>>>> and
>> > >>>>>>>> set of apis. So what if you split them and had an api for Kafka
>> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
>> Kafka
>> > >>>>>>>> transformation (Samza).
>> > >>>>>>>>
>> > >>>>>>>> This would also allow really embracing the same terminology and
>> > >>>>>>>> conventions. One complaint about the current state is that the
>> > >>>>>>>> two
>> > >>>>>>> systems
>> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs "topic"
>> and
>> > >>>>>>> different
>> > >>>>>>>> config and monitoring systems means you kind of have to learn
>> > >>>>>>>> Kafka's
>> > >>>>>>> way,
>> > >>>>>>>> then learn Samza's slightly different way, then kind of
>> > >>>>>>>> understand
>> > >>>>> how
>> > >>>>>>> they
>> > >>>>>>>> map to each other, which having walked a few people through
>> this
>> > >>>>>>>> is surprisingly tricky for folks to get.
>> > >>>>>>>>
>> > >>>>>>>> Since I have been spending a lot of time on airplanes I hacked
>> > >>>>>>>> up an ernest but still somewhat incomplete prototype of what
>> > >>>>>>>> this would
>> > >>>>> look
>> > >>>>>>>> like. This is just unceremoniously dumped into Kafka as it
>> > >>>>>>>> required a
>> > >>>>>> few
>> > >>>>>>>> changes to the new consumer. Here is the code:
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>>
>> > >>>>>>
>> > >>>>>
>> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> > >>>>> /apache/kafka/clients/streaming
>> > >>>>>>>>
>> > >>>>>>>> For the purpose of the prototype I just liberally renamed
>> > >>>>>>>> everything
>> > >>>>> to
>> > >>>>>>>> try to align it with Kafka with no regard for compatibility.
>> > >>>>>>>>
>> > >>>>>>>> To use this would be something like this:
>> > >>>>>>>> Properties props = new Properties();
>> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
>> > >>>>>>>> StreamingConfig config = new
>> > >>>>> StreamingConfig(props);
>> > >>>>>>> config.subscribe("test-topic-1",
>> > >>>>>>>> "test-topic-2");
>> config.processor(ExampleStreamProcessor.class);
>> > >>>>>>> config.serialization(new
>> > >>>>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
>> > >>>>>> container =
>> > >>>>>>>> new KafkaStreaming(config); container.run();
>> > >>>>>>>>
>> > >>>>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
>> > >>>>>>>> is basically StreamTask.
>> > >>>>>>>>
>> > >>>>>>>> So rather than putting all the class names in a file and then
>> > >>>>>>>> having
>> > >>>>>> the
>> > >>>>>>>> job assembled by reflection, you just instantiate the container
>> > >>>>>>>> programmatically. Work is balanced over however many instances
>> > >>>>>>>> of
>> > >>>>> this
>> > >>>>>>> are
>> > >>>>>>>> alive at any time (i.e. if an instance dies, new tasks are
>> added
>> > >>>>>>>> to
>> > >>>>> the
>> > >>>>>>>> existing containers without shutting them down).
>> > >>>>>>>>
>> > >>>>>>>> We would provide some glue for running this stuff in YARN via
>> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
>> > >>>>>>>> but from the
>> > >>>>>> point
>> > >>>>>>> of
>> > >>>>>>>> view of these frameworks these stream processing jobs are just
>> > >>>>>> stateless
>> > >>>>>>>> services that can come and go and expand and contract at will.
>> > >>>>>>>> There
>> > >>>>> is
>> > >>>>>>> no
>> > >>>>>>>> more custom scheduler.
>> > >>>>>>>>
>> > >>>>>>>> Here are some relevant details:
>> > >>>>>>>>
>> > >>>>>>>>  1. It is only ~1300 lines of code, it would get larger if we
>> > >>>>>>>>  productionized but not vastly larger. We really do get a ton
>> > >>>>>>>> of
>> > >>>>>>> leverage
>> > >>>>>>>>  out of Kafka.
>> > >>>>>>>>  2. Partition management is fully delegated to the new
>> consumer.
>> > >>>>> This
>> > >>>>>>>>  is nice since now any partition management strategy available
>> > >>>>>>>> to
>> > >>>>>> Kafka
>> > >>>>>>>>  consumer is also available to Samza (and vice versa) and with
>> > >>>>>>>> the
>> > >>>>>>> exact
>> > >>>>>>>>  same configs.
>> > >>>>>>>>  3. It supports state as well as state reuse
>> > >>>>>>>>
>> > >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
>> > >>>>>>>>
>> > >>>>>>>> -Jay
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>> > >>>>>> criccomini@apache.org>
>> > >>>>>>>> wrote:
>> > >>>>>>>>
>> > >>>>>>>>> Hey all,
>> > >>>>>>>>>
>> > >>>>>>>>> I have had some discussions with Samza engineers at LinkedIn
>> > >>>>>>>>> and
>> > >>>>>>> Confluent
>> > >>>>>>>>> and we came up with a few observations and would like to
>> > >>>>>>>>> propose
>> > >>>>> some
>> > >>>>>>>>> changes.
>> > >>>>>>>>>
>> > >>>>>>>>> We've observed some things that I want to call out about
>> > >>>>>>>>> Samza's
>> > >>>>>> design,
>> > >>>>>>>>> and I'd like to propose some changes.
>> > >>>>>>>>>
>> > >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
>> > >>>>>>>>> * Samza is too pluggable.
>> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
>> > >>>>>>>>> APIs
>> > >>>>> are
>> > >>>>>>>>> trying to solve a lot of the same problems.
>> > >>>>>>>>>
>> > >>>>>>>>> All three of these issues are related, but I'll address them
>> in
>> > >>>>> order.
>> > >>>>>>>>>
>> > >>>>>>>>> Deployment
>> > >>>>>>>>>
>> > >>>>>>>>> Samza strongly depends on the use of a dynamic deployment
>> > >>>>>>>>> scheduler
>> > >>>>>> such
>> > >>>>>>>>> as
>> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
>> > >>>>>>>>> there
>> > >>>>>> would
>> > >>>>>>>>> be
>> > >>>>>>>>> one or two winners in this area, and we could support them,
>> and
>> > >>>>>>>>> the
>> > >>>>>> rest
>> > >>>>>>>>> would go away. In reality, there are many variations.
>> > >>>>>>>>> Furthermore,
>> > >>>>>> many
>> > >>>>>>>>> people still prefer to just start their processors like normal
>> > >>>>>>>>> Java processes, and use traditional deployment scripts such as
>> > >>>>>>>>> Fabric,
>> > >>>>>> Chef,
>> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users makes the
>> > >>>>>>>>> Samza start-up process really painful for first time users.
>> > >>>>>>>>>
>> > >>>>>>>>> Dynamic deployment as a requirement was also a bit of a
>> > >>>>>>>>> mis-fire
>> > >>>>>> because
>> > >>>>>>>>> of
>> > >>>>>>>>> a fundamental misunderstanding between the nature of batch
>> jobs
>> > >>>>>>>>> and
>> > >>>>>>> stream
>> > >>>>>>>>> processing jobs. Early on, we made conscious effort to favor
>> > >>>>>>>>> the
>> > >>>>>> Hadoop
>> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked and was well
>> > >>>>>>> understood.
>> > >>>>>>>>> One thing that we missed was that batch jobs have a definite
>> > >>>>>> beginning,
>> > >>>>>>>>> and
>> > >>>>>>>>> end, and stream processing jobs don't (usually). This leads to
>> > >>>>>>>>> a
>> > >>>>> much
>> > >>>>>>>>> simpler scheduling problem for stream processors. You
>> basically
>> > >>>>>>>>> just
>> > >>>>>>> need
>> > >>>>>>>>> to find a place to start the processor, and start it. The way
>> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
>> > >>>>>>>>> being "full". We always
>> > >>>>>> add
>> > >>>>>>>>> more machines. The problem with coupling Samza with a
>> scheduler
>> > >>>>>>>>> is
>> > >>>>>> that
>> > >>>>>>>>> Samza (as a framework) now has to handle deployment. This
>> pulls
>> > >>>>>>>>> in a
>> > >>>>>>> bunch
>> > >>>>>>>>> of things such as configuration distribution (config stream),
>> > >>>>>>>>> shell
>> > >>>>>>> scrips
>> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff),
>> etc.
>> > >>>>>>>>>
>> > >>>>>>>>> Another reason for requiring dynamic deployment was to support
>> > >>>>>>>>> data locality. If you want to have locality, you need to put
>> > >>>>>>>>> your
>> > >>>>>> processors
>> > >>>>>>>>> close to the data they're processing. Upon further
>> > >>>>>>>>> investigation,
>> > >>>>>>> though,
>> > >>>>>>>>> this feature is not that beneficial. There is some good
>> > >>>>>>>>> discussion
>> > >>>>>> about
>> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
>> > >>>>>>>>> Map/Reduce
>> > >>>>>> path,
>> > >>>>>>>>> but
>> > >>>>>>>>> there are some fundamental differences between HDFS and Kafka.
>> > >>>>>>>>> HDFS
>> > >>>>>> has
>> > >>>>>>>>> blocks, while Kafka has partitions. This leads to less
>> > >>>>>>>>> optimization potential with stream processors on top of Kafka.
>> > >>>>>>>>>
>> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't have any
>> > >>>>>>>>> built
>> > >>>>> in
>> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
>> > >>>>>>>>> deployment scheduling system to handle restarts when a
>> > >>>>>>>>> processor dies. This has
>> > >>>>>>> made
>> > >>>>>>>>> it very difficult to write a standalone Samza container
>> > >>>> (SAMZA-516).
>> > >>>>>>>>>
>> > >>>>>>>>> Pluggability
>> > >>>>>>>>>
>> > >>>>>>>>> In some cases pluggability is good, but I think that we've
>> gone
>> > >>>>>>>>> too
>> > >>>>>> far
>> > >>>>>>>>> with it. Currently, Samza has:
>> > >>>>>>>>>
>> > >>>>>>>>> * Pluggable config.
>> > >>>>>>>>> * Pluggable metrics.
>> > >>>>>>>>> * Pluggable deployment systems.
>> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
>> > >>>> etc).
>> > >>>>>>>>> * Pluggable serdes.
>> > >>>>>>>>> * Pluggable storage engines.
>> > >>>>>>>>> * Pluggable strategies for just about every component
>> > >>>>> (MessageChooser,
>> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>> > >>>>>>>>>
>> > >>>>>>>>> There's probably more that I've forgotten, as well. Some of
>> > >>>>>>>>> these
>> > >>>>> are
>> > >>>>>>>>> useful, but some have proven not to be. This all comes at a
>> cost:
>> > >>>>>>>>> complexity. This complexity is making it harder for our users
>> > >>>>>>>>> to
>> > >>>>> pick
>> > >>>>>> up
>> > >>>>>>>>> and use Samza out of the box. It also makes it difficult for
>> > >>>>>>>>> Samza developers to reason about what the characteristics of
>> > >>>>>>>>> the container (since the characteristics change depending on
>> > >>>>>>>>> which plugins are use).
>> > >>>>>>>>>
>> > >>>>>>>>> The issues with pluggability are most visible in the System
>> APIs.
>> > >>>>> What
>> > >>>>>>>>> Samza really requires to be functional is Kafka as its
>> > >>>>>>>>> transport
>> > >>>>>> layer.
>> > >>>>>>>>> But
>> > >>>>>>>>> we've conflated two unrelated use cases into one API:
>> > >>>>>>>>>
>> > >>>>>>>>> 1. Get data into/out of Kafka.
>> > >>>>>>>>> 2. Process the data in Kafka.
>> > >>>>>>>>>
>> > >>>>>>>>> The current System API supports both of these use cases. The
>> > >>>>>>>>> problem
>> > >>>>>> is,
>> > >>>>>>>>> we
>> > >>>>>>>>> actually want different features for each use case. By
>> papering
>> > >>>>>>>>> over
>> > >>>>>>> these
>> > >>>>>>>>> two use cases, and providing a single API, we've introduced a
>> > >>>>>>>>> ton of
>> > >>>>>>> leaky
>> > >>>>>>>>> abstractions.
>> > >>>>>>>>>
>> > >>>>>>>>> For example, what we'd really like in (2) is to have
>> > >>>>>>>>> monotonically increasing longs for offsets (like Kafka). This
>> > >>>>>>>>> would be at odds
>> > >>>>> with
>> > >>>>>>> (1),
>> > >>>>>>>>> though, since different systems have different
>> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
>> > >>>>>>>>> There was discussion both on the mailing list and the SQL
>> JIRAs
>> > >>>>> about
>> > >>>>>>> the
>> > >>>>>>>>> need for this.
>> > >>>>>>>>>
>> > >>>>>>>>> The same thing holds true for replayability. Kafka allows us
>> to
>> > >>>>> rewind
>> > >>>>>>>>> when
>> > >>>>>>>>> we have a failure. Many other systems don't. In some cases,
>> > >>>>>>>>> systems
>> > >>>>>>> return
>> > >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
>> > >>>>>>>>> they
>> > >>>>>> have
>> > >>>>>>> no
>> > >>>>>>>>> offsets.
>> > >>>>>>>>>
>> > >>>>>>>>> Partitioning is another example. Kafka supports partitioning,
>> > >>>>>>>>> but
>> > >>>>> many
>> > >>>>>>>>> systems don't. We model this by having a single partition for
>> > >>>>>>>>> those systems. Still, other systems model partitioning
>> > >>>> differently (e.g.
>> > >>>>>>>>> Kinesis).
>> > >>>>>>>>>
>> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating streams in
>> a
>> > >>>>>>>>> system-agnostic way is almost impossible. As is modeling
>> > >>>>>>>>> metadata
>> > >>>>> for
>> > >>>>>>> the
>> > >>>>>>>>> system (replication factor, partitions, location, etc). The
>> > >>>>>>>>> list
>> > >>>>> goes
>> > >>>>>>> on.
>> > >>>>>>>>>
>> > >>>>>>>>> Duplicate work
>> > >>>>>>>>>
>> > >>>>>>>>> At the time that we began writing Samza, Kafka's consumer and
>> > >>>>> producer
>> > >>>>>>>>> APIs
>> > >>>>>>>>> had a relatively weak feature set. On the consumer-side, you
>> > >>>>>>>>> had two
>> > >>>>>>>>> options: use the high level consumer, or the simple consumer.
>> > >>>>>>>>> The
>> > >>>>>>> problem
>> > >>>>>>>>> with the high-level consumer was that it controlled your
>> > >>>>>>>>> offsets, partition assignments, and the order in which you
>> > >>>>>>>>> received messages. The
>> > >>>>> problem
>> > >>>>>>>>> with
>> > >>>>>>>>> the simple consumer is that it's not simple. It's basic. You
>> > >>>>>>>>> end up
>> > >>>>>>> having
>> > >>>>>>>>> to handle a lot of really low-level stuff that you shouldn't.
>> > >>>>>>>>> We
>> > >>>>>> spent a
>> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust.
>> It
>> > >>>>>>>>> also allows us to support some cool features:
>> > >>>>>>>>>
>> > >>>>>>>>> * Per-partition message ordering and prioritization.
>> > >>>>>>>>> * Tight control over partition assignment to support joins,
>> > >>>>>>>>> global
>> > >>>>>> state
>> > >>>>>>>>> (if we want to implement it :)), etc.
>> > >>>>>>>>> * Tight control over offset checkpointing.
>> > >>>>>>>>>
>> > >>>>>>>>> What we didn't realize at the time is that these features
>> > >>>>>>>>> should
>> > >>>>>>> actually
>> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
>> > >>>>>> processors)
>> > >>>>>>>>> end up wanting to do things like joins and partition
>> > >>>>>>>>> assignment. The
>> > >>>>>>> Kafka
>> > >>>>>>>>> community has come to the same conclusion. They're adding a
>> ton
>> > >>>>>>>>> of upgrades into their new Kafka consumer implementation. To a
>> > >>>>>>>>> large extent,
>> > >>>>> it's
>> > >>>>>>>>> duplicate work to what we've already done in Samza.
>> > >>>>>>>>>
>> > >>>>>>>>> On top of this, Kafka ended up taking a very similar approach
>> > >>>>>>>>> to
>> > >>>>>> Samza's
>> > >>>>>>>>> KafkaCheckpointManager implementation for handling offset
>> > >>>>>> checkpointing.
>> > >>>>>>>>> Like Samza, Kafka's new offset management feature stores
>> offset
>> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them from the
>> > >>>>>>>>> broker.
>> > >>>>>>>>>
>> > >>>>>>>>> A lot of this seems like a waste, since we could have shared
>> > >>>>>>>>> the
>> > >>>>> work
>> > >>>>>> if
>> > >>>>>>>>> it
>> > >>>>>>>>> had been done in Kafka from the get-go.
>> > >>>>>>>>>
>> > >>>>>>>>> Vision
>> > >>>>>>>>>
>> > >>>>>>>>> All of this leads me to a rather radical proposal. Samza is
>> > >>>>> relatively
>> > >>>>>>>>> stable at this point. I'd venture to say that we're near a 1.0
>> > >>>>>> release.
>> > >>>>>>>>> I'd
>> > >>>>>>>>> like to propose that we take what we've learned, and begin
>> > >>>>>>>>> thinking
>> > >>>>>>> about
>> > >>>>>>>>> Samza beyond 1.0. What would we change if we were starting
>> from
>> > >>>>>> scratch?
>> > >>>>>>>>> My
>> > >>>>>>>>> proposal is to:
>> > >>>>>>>>>
>> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
>> > >>>>>>>>> processors, and eliminate all direct dependences on YARN,
>> Mesos,
>> > >>>> etc.
>> > >>>>>>>>> 2. Make a definitive call to support only Kafka as the stream
>> > >>>>>> processing
>> > >>>>>>>>> layer.
>> > >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
>> > >>>>>>>>> config
>> > >>>>>>> systems,
>> > >>>>>>>>> and simply use Kafka's instead.
>> > >>>>>>>>>
>> > >>>>>>>>> This would fix all of the issues that I outlined above. It
>> > >>>>>>>>> should
>> > >>>>> also
>> > >>>>>>>>> shrink the Samza code base pretty dramatically. Supporting
>> only
>> > >>>>>>>>> a standalone container will allow Samza to be executed on YARN
>> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
>> > >>>>>>>>> in-house
>> > >>>>>>> deployment
>> > >>>>>>>>> systems. This should make life a lot easier for new users.
>> > >>>>>>>>> Imagine
>> > >>>>>>> having
>> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in mailing
>> list
>> > >>>>>> traffic
>> > >>>>>>>>> will be pretty dramatic.
>> > >>>>>>>>>
>> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
>> > >>>>> everyone
>> > >>>>>>>>> that
>> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically require
>> it
>> > >>>>>> already
>> > >>>>>>> in
>> > >>>>>>>>> order for most features to work. Those that are using other
>> > >>>>>>>>> systems
>> > >>>>>> are
>> > >>>>>>>>> generally using it for ingest into Kafka (1), and then they do
>> > >>>>>>>>> the processing on top. There is already discussion (
>> > >>>>>>>>>
>> > >>>>>>>
>> > >>>>>>
>> > >>>>>
>> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> > >>>>> 767
>> > >>>>>>>>> )
>> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
>> > >>>>>>>>>
>> > >>>>>>>>> Once we make the call to couple with Kafka, we can leverage a
>> > >>>>>>>>> ton of
>> > >>>>>>> their
>> > >>>>>>>>> ecosystem. We no longer have to maintain our own config,
>> > >>>>>>>>> metrics,
>> > >>>>> etc.
>> > >>>>>>> We
>> > >>>>>>>>> can all share the same libraries, and make them better. This
>> > >>>>>>>>> will
>> > >>>>> also
>> > >>>>>>>>> allow us to share the consumer/producer APIs, and will let us
>> > >>>>> leverage
>> > >>>>>>>>> their offset management and partition management, rather than
>> > >>>>>>>>> having
>> > >>>>>> our
>> > >>>>>>>>> own. All of the coordinator stream code would go away, as
>> would
>> > >>>>>>>>> most
>> > >>>>>> of
>> > >>>>>>>>> the
>> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some partition
>> > >>>>>>> management
>> > >>>>>>>>> features into the Kafka broker, but they're already moving in
>> > >>>>>>>>> that direction with the new consumer API. The features we have
>> > >>>>>>>>> for
>> > >>>>>> partition
>> > >>>>>>>>> assignment aren't unique to Samza, and seem like they should
>> be
>> > >>>>>>>>> in
>> > >>>>>> Kafka
>> > >>>>>>>>> anyway. There will always be some niche usages which will
>> > >>>>>>>>> require
>> > >>>>>> extra
>> > >>>>>>>>> care and hence full control over partition assignments much
>> > >>>>>>>>> like the
>> > >>>>>>> Kafka
>> > >>>>>>>>> low level consumer api. These would continue to be supported.
>> > >>>>>>>>>
>> > >>>>>>>>> These items will be good for the Samza community. They'll make
>> > >>>>>>>>> Samza easier to use, and make it easier for developers to add
>> > >>>>>>>>> new features.
>> > >>>>>>>>>
>> > >>>>>>>>> Obviously this is a fairly large (and somewhat backwards
>> > >>>>> incompatible
>> > >>>>>>>>> change). If we choose to go this route, it's important that we
>> > >>>>> openly
>> > >>>>>>>>> communicate how we're going to provide a migration path from
>> > >>>>>>>>> the
>> > >>>>>>> existing
>> > >>>>>>>>> APIs to the new ones (if we make incompatible changes). I
>> think
>> > >>>>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
>> > >>>>>>>>> existing StreamTask implementations to continue running on the
>> > >>>> new container.
>> > >>>>>>> It's
>> > >>>>>>>>> also important that we openly communicate about timing, and
>> > >>>>>>>>> stages
>> > >>>>> of
>> > >>>>>>> the
>> > >>>>>>>>> migration.
>> > >>>>>>>>>
>> > >>>>>>>>> If you made it this far, I'm sure you have opinions. :) Please
>> > >>>>>>>>> send
>> > >>>>>> your
>> > >>>>>>>>> thoughts and feedback.
>> > >>>>>>>>>
>> > >>>>>>>>> Cheers,
>> > >>>>>>>>> Chris
>> > >>>>>>>>>
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>>
>> > >>>>>>
>> > >>>>>>
>> > >>>>>>
>> > >>>>>> --
>> > >>>>>> -- Guozhang
>> > >>>>>>
>> > >>>>>
>> > >>>>
>> > >>
>> > >>
>> >
>> >
>> >
>>
>
>

Re: Thoughts and obesrvations on Samza

Posted by Gianmarco De Francisci Morales <gd...@apache.org>.
Hi,

@Martin, thanks for you comments.
Maybe I'm missing some important point, but I think coupling the releases
is actually a *good* thing.
To make an example, would it be better if the MR and HDFS components of
Hadoop had different release schedules?

Actually, keeping the discussion in a single place would make agreeing on
releases (and backwards compatibility) much easier, as everybody would be
responsible for the whole codebase.

That said, I like the idea of absorbing samza-core as a sub-project, and
leave the fancy stuff separate.
It probably gives 90% of the benefits we have been discussing here.

Cheers,

--
Gianmarco

On 7 July 2015 at 02:30, Jay Kreps <ja...@gmail.com> wrote:

> Hey Martin,
>
> I agree coupling release schedules is a downside.
>
> Definitely we can try to solve some of the integration problems in
> Confluent Platform or in other distributions. But I think this ends up
> being really shallow. I guess I feel to really get a good user experience
> the two systems have to kind of feel like part of the same thing and you
> can't really add that in later--you can put both in the same downloadable
> tar file but it doesn't really give a very cohesive feeling. I agree that
> ultimately any of the project stuff is as much social and naming as
> anything else--theoretically two totally independent projects could work to
> tightly align. In practice this seems to be quite difficult though.
>
> For the frameworks--totally agree it would be good to maintain the
> framework support with the project. In some cases there may not be too much
> there since the integration gets lighter but I think whatever stubs you
> need should be included. So no I definitely wasn't trying to imply dropping
> support for these frameworks, just making the integration lighter by
> separating process management from partition management.
>
> You raise two good points we would have to figure out if we went down the
> alignment path:
> 1. With respect to the name, yeah I think the first question is whether
> some "re-branding" would be worth it. If so then I think we can have a big
> thread on the name. I'm definitely not set on Kafka Streaming or Kafka
> Streams I was just using them to be kind of illustrative. I agree with your
> critique of these names, though I think people would get the idea.
> 2. Yeah you also raise a good point about how to "factor" it. Here are the
> options I see (I could get enthusiastic about any of them):
>    a. One repo for both Kafka and Samza
>    b. Two repos, retaining the current seperation
>    c. Two repos, the equivalent of samza-api and samza-core is absorbed
> almost like a third client
>
> Cheers,
>
> -Jay
>
> On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <ma...@kleppmann.com>
> wrote:
>
> > Ok, thanks for the clarifications. Just a few follow-up comments.
> >
> > - I see the appeal of merging with Kafka or becoming a subproject: the
> > reasons you mention are good. The risk I see is that release schedules
> > become coupled to each other, which can slow everyone down, and large
> > projects with many contributors are harder to manage. (Jakob, can you
> speak
> > from experience, having seen a wider range of Hadoop ecosystem projects?)
> >
> > Some of the goals of a better unified developer experience could also be
> > solved by integrating Samza nicely into a Kafka distribution (such as
> > Confluent's). I'm not against merging projects if we decide that's the
> way
> > to go, just pointing out the same goals can perhaps also be achieved in
> > other ways.
> >
> > - With regard to dropping the YARN dependency: are you proposing that
> > Samza doesn't give any help to people wanting to run on
> YARN/Mesos/AWS/etc?
> > So the docs would basically have a link to Slider and nothing else? Or
> > would we maintain integrations with a bunch of popular deployment methods
> > (e.g. the necessary glue and shell scripts to make Samza work with
> Slider)?
> >
> > I absolutely think it's a good idea to have the "as a library" and "as a
> > process" (using Yi's taxonomy) options for people who want them, but I
> > think there should also be a low-friction path for common "as a service"
> > deployment methods, for which we probably need to maintain integrations.
> >
> > - Project naming: "Kafka Streams" seems odd to me, because Kafka is all
> > about streams already. Perhaps "Kafka Transformers" or "Kafka Filters"
> > would be more apt?
> >
> > One suggestion: perhaps the core of Samza (stream transformation with
> > state management -- i.e. the "Samza as a library" bit) could become part
> of
> > Kafka, while higher-level tools such as streaming SQL and integrations
> with
> > deployment frameworks remain in a separate project? In other words, Kafka
> > would absorb the proven, stable core of Samza, which would become the
> > "third Kafka client" mentioned early in this thread. The Samza project
> > would then target that third Kafka client as its base API, and the
> project
> > would be freed up to explore more experimental new horizons.
> >
> > Martin
> >
> > On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Martin,
> > >
> > > For the YARN/Mesos/etc decoupling I actually don't think it ties our
> > hands
> > > at all, all it does is refactor things. The division of responsibility
> is
> > > that Samza core is responsible for task lifecycle, state, and partition
> > > management (using the Kafka co-ordinator) but it is NOT responsible for
> > > packaging, configuration deployment or execution of processes. The
> > problem
> > > of packaging and starting these processes is
> > > framework/environment-specific. This leaves individual frameworks to be
> > as
> > > fancy or vanilla as they like. So you can get simple stateless support
> in
> > > YARN, Mesos, etc using their off-the-shelf app framework (Slider,
> > Marathon,
> > > etc). These are well known by people and have nice UIs and a lot of
> > > flexibility. I don't think they have node affinity as a built in option
> > > (though I could be wrong). So if we want that we can either wait for
> them
> > > to add it or do a custom framework to add that feature (as now).
> > Obviously
> > > if you manage things with old-school ops tools (puppet/chef/etc) you
> get
> > > locality easily. The nice thing, though, is that all the samza
> "business
> > > logic" around partition management and fault tolerance is in Samza core
> > so
> > > it is shared across frameworks and the framework specific bit is just
> > > whether it is smart enough to try to get the same host when a job is
> > > restarted.
> > >
> > > With respect to the Kafka-alignment, yeah I think the goal would be (a)
> > > actually get better alignment in user experience, and (b) express this
> in
> > > the naming and project branding. Specifically:
> > > 1. Website/docs, it would be nice for the "transformation" api to be
> > > discoverable in the main Kafka docs--i.e. be able to explain when to
> use
> > > the consumer and when to use the stream processing functionality and
> lead
> > > people into that experience.
> > > 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that has
> both
> > > Kafka and the stream processing part and they actually work together.
> > > 3. Unify the programming experience so the client and Samza api share
> > > config/monitoring/naming/packaging/etc.
> > >
> > > I think sub-projects keep separate committers and can have a separate
> > repo,
> > > but I'm actually not really sure (I can't find a definition of a
> > subproject
> > > in Apache).
> > >
> > > Basically at a high-level you want the experience to "feel" like a
> single
> > > system, not to relatively independent things that are kind of awkwardly
> > > glued together.
> > >
> > > I think if we did that they having naming or branding like "kafka
> > > streaming" or "kafka streams" or something like that would actually do
> a
> > > good job of conveying what it is. I do that this would help adoption
> > quite
> > > a lot as it would correctly convey that using Kafka Streaming with
> Kafka
> > is
> > > a fairly seamless experience and Kafka is pretty heavily adopted at
> this
> > > point.
> > >
> > > Fwiw we actually considered this model originally when open sourcing
> > Samza,
> > > however at that time Kafka was relatively unknown and we decided not to
> > do
> > > it since we felt it would be limiting. From my point of view the three
> > > things have changed (1) Kafka is now really heavily used for stream
> > > processing, (2) we learned that abstracting out the stream well is
> > > basically impossible, (3) we learned it is really hard to keep the two
> > > things feeling like a single product.
> > >
> > > -Jay
> > >
> > >
> > > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <martin@kleppmann.com
> >
> > > wrote:
> > >
> > >> Hi all,
> > >>
> > >> Lots of good thoughts here.
> > >>
> > >> I agree with the general philosophy of tying Samza more firmly to
> Kafka.
> > >> After I spent a while looking at integrating other message brokers
> (e.g.
> > >> Kinesis) with SystemConsumer, I came to the conclusion that
> > SystemConsumer
> > >> tacitly assumes a model so much like Kafka's that pretty much nobody
> but
> > >> Kafka actually implements it. (Databus is perhaps an exception, but it
> > >> isn't widely used outside of LinkedIn.) Thus, making Samza fully
> > dependent
> > >> on Kafka acknowledges that the system-independence was never as real
> as
> > we
> > >> perhaps made it out to be. The gains of code reuse are real.
> > >>
> > >> The idea of decoupling Samza from YARN has also always been appealing
> to
> > >> me, for various reasons already mentioned in this thread. Although
> > making
> > >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems laudable,
> > I am
> > >> a little concerned that it will restrict us to a lowest common
> > denominator.
> > >> For example, would host affinity (SAMZA-617) still be possible? For
> jobs
> > >> with large amounts of state, I think SAMZA-617 would be a big boon,
> > since
> > >> restoring state off the changelog on every single restart is painful,
> > due
> > >> to long recovery times. It would be a shame if the decoupling from
> YARN
> > >> made host affinity impossible.
> > >>
> > >> Jay, a question about the proposed API for instantiating a job in code
> > >> (rather than a properties file): when submitting a job to a cluster,
> is
> > the
> > >> idea that the instantiation code runs on a client somewhere, which
> then
> > >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code
> > run
> > >> on each container that is part of the job (in which case, how does the
> > job
> > >> submission to the cluster work)?
> > >>
> > >> I agree with Garry that it doesn't feel right to make a 1.0 release
> > with a
> > >> plan for it to be immediately obsolete. So if this is going to
> happen, I
> > >> think it would be more honest to stick with 0.* version numbers until
> > the
> > >> library-ified Samza has been implemented, is stable and widely used.
> > >>
> > >> Should the new Samza be a subproject of Kafka? There is precedent for
> > >> tight coupling between different Apache projects (e.g. Curator and
> > >> Zookeeper, or Slider and YARN), so I think remaining separate would be
> > ok.
> > >> Even if Samza is fully dependent on Kafka, there is enough substance
> in
> > >> Samza that it warrants being a separate project. An argument in favour
> > of
> > >> merging would be if we think Kafka has a much stronger "brand
> presence"
> > >> than Samza; I'm ambivalent on that one. If the Kafka project is
> willing
> > to
> > >> endorse Samza as the "official" way of doing stateful stream
> > >> transformations, that would probably have much the same effect as
> > >> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
> > >> collaboration between the two projects will be needed in any case.
> > >>
> > >> From a project management perspective, I guess the "new Samza" would
> > have
> > >> to be developed on a branch alongside ongoing maintenance of the
> current
> > >> line of development? I think it would be important to continue
> > supporting
> > >> existing users, and provide a graceful migration path to the new
> > version.
> > >> Leaving the current versions unsupported and forcing people to rewrite
> > >> their jobs would send a bad signal.
> > >>
> > >> Best,
> > >> Martin
> > >>
> > >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
> > >>
> > >>> Hey Garry,
> > >>>
> > >>> Yeah that's super frustrating. I'd be happy to chat more about this
> if
> > >>> you'd be interested. I think Chris and I started with the idea of
> "what
> > >>> would it take to make Samza a kick-ass ingestion tool" but ultimately
> > we
> > >>> kind of came around to the idea that ingestion and transformation had
> > >>> pretty different needs and coupling the two made things hard.
> > >>>
> > >>> For what it's worth I think copycat (KIP-26) actually will do what
> you
> > >> are
> > >>> looking for.
> > >>>
> > >>> With regard to your point about slider, I don't necessarily disagree.
> > >> But I
> > >>> think getting good YARN support is quite doable and I think we can
> make
> > >>> that work well. I think the issue this proposal solves is that
> > >> technically
> > >>> it is pretty hard to support multiple cluster management systems the
> > way
> > >>> things are now, you need to write an "app master" or "framework" for
> > each
> > >>> and they are all a little different so testing is really hard. In the
> > >>> absence of this we have been stuck with just YARN which has fantastic
> > >>> penetration in the Hadoopy part of the org, but zero penetration
> > >> elsewhere.
> > >>> Given the huge amount of work being put in to slider, marathon, aws
> > >>> tooling, not to mention the umpteen related packaging technologies
> > people
> > >>> want to use (Docker, Kubernetes, various cloud-specific deploy tools,
> > >> etc)
> > >>> I really think it is important to get this right.
> > >>>
> > >>> -Jay
> > >>>
> > >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > >>> g.turkington@improvedigital.com> wrote:
> > >>>
> > >>>> Hi all,
> > >>>>
> > >>>> I think the question below re does Samza become a sub-project of
> Kafka
> > >>>> highlights the broader point around migration. Chris mentions
> Samza's
> > >>>> maturity is heading towards a v1 release but I'm not sure it feels
> > >> right to
> > >>>> launch a v1 then immediately plan to deprecate most of it.
> > >>>>
> > >>>> From a selfish perspective I have some guys who have started working
> > >> with
> > >>>> Samza and building some new consumers/producers was next up. Sounds
> > like
> > >>>> that is absolutely not the direction to go. I need to look into the
> > KIP
> > >> in
> > >>>> more detail but for me the attractiveness of adding new Samza
> > >>>> consumer/producers -- even if yes all they were doing was really
> > getting
> > >>>> data into and out of Kafka --  was to avoid  having to worry about
> the
> > >>>> lifecycle management of external clients. If there is a generic
> Kafka
> > >>>> ingress/egress layer that I can plug a new connector into and have a
> > >> lot of
> > >>>> the heavy lifting re scale and reliability done for me then it gives
> > me
> > >> all
> > >>>> the pushing new consumers/producers would. If not then it
> complicates
> > my
> > >>>> operational deployments.
> > >>>>
> > >>>> Which is similar to my other question with the proposal -- if we
> > build a
> > >>>> fully available/stand-alone Samza plus the requisite shims to
> > integrate
> > >>>> with Slider etc I suspect the former may be a lot more work than we
> > >> think.
> > >>>> We may make it much easier for a newcomer to get something running
> but
> > >>>> having them step up and get a reliable production deployment may
> still
> > >>>> dominate mailing list  traffic, if for different reasons than today.
> > >>>>
> > >>>> Don't get me wrong -- I'm comfortable with making the Samza
> dependency
> > >> on
> > >>>> Kafka much more explicit and I absolutely see the benefits  in the
> > >>>> reduction of duplication and clashing terminologies/abstractions
> that
> > >>>> Chris/Jay describe. Samza as a library would likely be a very nice
> > tool
> > >> to
> > >>>> add to the Kafka ecosystem. I just have the concerns above re the
> > >>>> operational side.
> > >>>>
> > >>>> Garry
> > >>>>
> > >>>> -----Original Message-----
> > >>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> > >>>> Sent: 02 July 2015 12:56
> > >>>> To: dev@samza.apache.org
> > >>>> Subject: Re: Thoughts and obesrvations on Samza
> > >>>>
> > >>>> Very interesting thoughts.
> > >>>> From outside, I have always perceived Samza as a computing layer
> over
> > >>>> Kafka.
> > >>>>
> > >>>> The question, maybe a bit provocative, is "should Samza be a
> > sub-project
> > >>>> of Kafka then?"
> > >>>> Or does it make sense to keep it as a separate project with a
> separate
> > >>>> governance?
> > >>>>
> > >>>> Cheers,
> > >>>>
> > >>>> --
> > >>>> Gianmarco
> > >>>>
> > >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> > >>>>
> > >>>>> Overall, I agree to couple with Kafka more tightly. Because Samza
> de
> > >>>>> facto is based on Kafka, and it should leverage what Kafka has. At
> > the
> > >>>>> same time, Kafka does not need to reinvent what Samza already has.
> I
> > >>>>> also like the idea of separating the ingestion and transformation.
> > >>>>>
> > >>>>> But it is a little difficult for me to image how the Samza will
> look
> > >>>> like.
> > >>>>> And I feel Chris and Jay have a little difference in terms of how
> > >>>>> Samza should look like.
> > >>>>>
> > >>>>> *** Will it look like what Jay's code shows (A client of Kakfa) ?
> And
> > >>>>> user's application code calls this client?
> > >>>>>
> > >>>>> 1. If we make Samza be a library of Kafka (like what the code
> shows),
> > >>>>> how do we implement auto-balance and fault-tolerance? Are they
> taken
> > >>>>> care by the Kafka broker or other mechanism, such as "Samza worker"
> > >>>>> (just make up the name) ?
> > >>>>>
> > >>>>> 2. What about other features, such as auto-scaling, shared state,
> > >>>>> monitoring?
> > >>>>>
> > >>>>>
> > >>>>> *** If we have Samza standalone, (is this what Chris suggests?)
> > >>>>>
> > >>>>> 1. we still need to ingest data from Kakfa and produce to it. Then
> it
> > >>>>> becomes the same as what Samza looks like now, except it does not
> > rely
> > >>>>> on Yarn anymore.
> > >>>>>
> > >>>>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
> > >>>>> etc? Use Kafka code as the dependency?
> > >>>>>
> > >>>>>
> > >>>>> Thanks,
> > >>>>>
> > >>>>> Fang, Yan
> > >>>>> yanfang724@gmail.com
> > >>>>>
> > >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
> > >>>> wrote:
> > >>>>>
> > >>>>>> Read through the code example and it looks good to me. A few
> > >>>>>> thoughts regarding deployment:
> > >>>>>>
> > >>>>>> Today Samza deploys as executable runnable like:
> > >>>>>>
> > >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> > >>>> --config-path=file://...
> > >>>>>>
> > >>>>>> And this proposal advocate for deploying Samza more as embedded
> > >>>>>> libraries in user application code (ignoring the terminology since
> > >>>>>> it is not the
> > >>>>> same
> > >>>>>> as the prototype code):
> > >>>>>>
> > >>>>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
> > >>>>>> Thread(task); thread.start();
> > >>>>>>
> > >>>>>> I think both of these deployment modes are important for different
> > >>>>>> types
> > >>>>> of
> > >>>>>> users. That said, I think making Samza purely standalone is still
> > >>>>>> sufficient for either runnable or library modes.
> > >>>>>>
> > >>>>>> Guozhang
> > >>>>>>
> > >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
> > wrote:
> > >>>>>>
> > >>>>>>> Looks like gmail mangled the code example, it was supposed to
> look
> > >>>>>>> like
> > >>>>>>> this:
> > >>>>>>>
> > >>>>>>> Properties props = new Properties();
> > >>>>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
> > >>>>>>> config = new StreamingConfig(props);
> > >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> > >>>>>>> config.processor(ExampleStreamProcessor.class);
> > >>>>>>> config.serialization(new StringSerializer(), new
> > >>>>>>> StringDeserializer()); KafkaStreaming container = new
> > >>>>>>> KafkaStreaming(config); container.run();
> > >>>>>>>
> > >>>>>>> -Jay
> > >>>>>>>
> > >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> > >>>> wrote:
> > >>>>>>>
> > >>>>>>>> Hey guys,
> > >>>>>>>>
> > >>>>>>>> This came out of some conversations Chris and I were having
> > >>>>>>>> around
> > >>>>>>> whether
> > >>>>>>>> it would make sense to use Samza as a kind of data ingestion
> > >>>>> framework
> > >>>>>>> for
> > >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> > >>>>>> combined
> > >>>>>>>> with complaints around config and YARN and the discussion around
> > >>>>>>>> how
> > >>>>> to
> > >>>>>>>> best do a standalone mode.
> > >>>>>>>>
> > >>>>>>>> So the thought experiment was, given that Samza was basically
> > >>>>>>>> already totally Kafka specific, what if you just embraced that
> > >>>>>>>> and turned it
> > >>>>>> into
> > >>>>>>>> something less like a heavyweight framework and more like a
> > >>>>>>>> third
> > >>>>> Kafka
> > >>>>>>>> client--a kind of "producing consumer" with state management
> > >>>>>> facilities.
> > >>>>>>>> Basically a library. Instead of a complex stream processing
> > >>>>>>>> framework
> > >>>>>>> this
> > >>>>>>>> would actually be a very simple thing, not much more complicated
> > >>>>>>>> to
> > >>>>> use
> > >>>>>>> or
> > >>>>>>>> operate than a Kafka consumer. As Chris said we thought about it
> > >>>>>>>> a
> > >>>>> lot
> > >>>>>> of
> > >>>>>>>> what Samza (and the other stream processing systems were doing)
> > >>>>> seemed
> > >>>>>>> like
> > >>>>>>>> kind of a hangover from MapReduce.
> > >>>>>>>>
> > >>>>>>>> Of course you need to ingest/output data to and from the stream
> > >>>>>>>> processing. But when we actually looked into how that would
> > >>>>>>>> work,
> > >>>>> Samza
> > >>>>>>>> isn't really an ideal data ingestion framework for a bunch of
> > >>>>> reasons.
> > >>>>>> To
> > >>>>>>>> really do that right you need a pretty different internal data
> > >>>>>>>> model
> > >>>>>> and
> > >>>>>>>> set of apis. So what if you split them and had an api for Kafka
> > >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > >>>>>>>> transformation (Samza).
> > >>>>>>>>
> > >>>>>>>> This would also allow really embracing the same terminology and
> > >>>>>>>> conventions. One complaint about the current state is that the
> > >>>>>>>> two
> > >>>>>>> systems
> > >>>>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
> > >>>>>>> different
> > >>>>>>>> config and monitoring systems means you kind of have to learn
> > >>>>>>>> Kafka's
> > >>>>>>> way,
> > >>>>>>>> then learn Samza's slightly different way, then kind of
> > >>>>>>>> understand
> > >>>>> how
> > >>>>>>> they
> > >>>>>>>> map to each other, which having walked a few people through this
> > >>>>>>>> is surprisingly tricky for folks to get.
> > >>>>>>>>
> > >>>>>>>> Since I have been spending a lot of time on airplanes I hacked
> > >>>>>>>> up an ernest but still somewhat incomplete prototype of what
> > >>>>>>>> this would
> > >>>>> look
> > >>>>>>>> like. This is just unceremoniously dumped into Kafka as it
> > >>>>>>>> required a
> > >>>>>> few
> > >>>>>>>> changes to the new consumer. Here is the code:
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >>>>> /apache/kafka/clients/streaming
> > >>>>>>>>
> > >>>>>>>> For the purpose of the prototype I just liberally renamed
> > >>>>>>>> everything
> > >>>>> to
> > >>>>>>>> try to align it with Kafka with no regard for compatibility.
> > >>>>>>>>
> > >>>>>>>> To use this would be something like this:
> > >>>>>>>> Properties props = new Properties();
> > >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> > >>>>>>>> StreamingConfig config = new
> > >>>>> StreamingConfig(props);
> > >>>>>>> config.subscribe("test-topic-1",
> > >>>>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > >>>>>>> config.serialization(new
> > >>>>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> > >>>>>> container =
> > >>>>>>>> new KafkaStreaming(config); container.run();
> > >>>>>>>>
> > >>>>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
> > >>>>>>>> is basically StreamTask.
> > >>>>>>>>
> > >>>>>>>> So rather than putting all the class names in a file and then
> > >>>>>>>> having
> > >>>>>> the
> > >>>>>>>> job assembled by reflection, you just instantiate the container
> > >>>>>>>> programmatically. Work is balanced over however many instances
> > >>>>>>>> of
> > >>>>> this
> > >>>>>>> are
> > >>>>>>>> alive at any time (i.e. if an instance dies, new tasks are added
> > >>>>>>>> to
> > >>>>> the
> > >>>>>>>> existing containers without shutting them down).
> > >>>>>>>>
> > >>>>>>>> We would provide some glue for running this stuff in YARN via
> > >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> > >>>>>>>> but from the
> > >>>>>> point
> > >>>>>>> of
> > >>>>>>>> view of these frameworks these stream processing jobs are just
> > >>>>>> stateless
> > >>>>>>>> services that can come and go and expand and contract at will.
> > >>>>>>>> There
> > >>>>> is
> > >>>>>>> no
> > >>>>>>>> more custom scheduler.
> > >>>>>>>>
> > >>>>>>>> Here are some relevant details:
> > >>>>>>>>
> > >>>>>>>>  1. It is only ~1300 lines of code, it would get larger if we
> > >>>>>>>>  productionized but not vastly larger. We really do get a ton
> > >>>>>>>> of
> > >>>>>>> leverage
> > >>>>>>>>  out of Kafka.
> > >>>>>>>>  2. Partition management is fully delegated to the new consumer.
> > >>>>> This
> > >>>>>>>>  is nice since now any partition management strategy available
> > >>>>>>>> to
> > >>>>>> Kafka
> > >>>>>>>>  consumer is also available to Samza (and vice versa) and with
> > >>>>>>>> the
> > >>>>>>> exact
> > >>>>>>>>  same configs.
> > >>>>>>>>  3. It supports state as well as state reuse
> > >>>>>>>>
> > >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
> > >>>>>>>>
> > >>>>>>>> -Jay
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > >>>>>> criccomini@apache.org>
> > >>>>>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> Hey all,
> > >>>>>>>>>
> > >>>>>>>>> I have had some discussions with Samza engineers at LinkedIn
> > >>>>>>>>> and
> > >>>>>>> Confluent
> > >>>>>>>>> and we came up with a few observations and would like to
> > >>>>>>>>> propose
> > >>>>> some
> > >>>>>>>>> changes.
> > >>>>>>>>>
> > >>>>>>>>> We've observed some things that I want to call out about
> > >>>>>>>>> Samza's
> > >>>>>> design,
> > >>>>>>>>> and I'd like to propose some changes.
> > >>>>>>>>>
> > >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
> > >>>>>>>>> * Samza is too pluggable.
> > >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> > >>>>>>>>> APIs
> > >>>>> are
> > >>>>>>>>> trying to solve a lot of the same problems.
> > >>>>>>>>>
> > >>>>>>>>> All three of these issues are related, but I'll address them in
> > >>>>> order.
> > >>>>>>>>>
> > >>>>>>>>> Deployment
> > >>>>>>>>>
> > >>>>>>>>> Samza strongly depends on the use of a dynamic deployment
> > >>>>>>>>> scheduler
> > >>>>>> such
> > >>>>>>>>> as
> > >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> > >>>>>>>>> there
> > >>>>>> would
> > >>>>>>>>> be
> > >>>>>>>>> one or two winners in this area, and we could support them, and
> > >>>>>>>>> the
> > >>>>>> rest
> > >>>>>>>>> would go away. In reality, there are many variations.
> > >>>>>>>>> Furthermore,
> > >>>>>> many
> > >>>>>>>>> people still prefer to just start their processors like normal
> > >>>>>>>>> Java processes, and use traditional deployment scripts such as
> > >>>>>>>>> Fabric,
> > >>>>>> Chef,
> > >>>>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> > >>>>>>>>> Samza start-up process really painful for first time users.
> > >>>>>>>>>
> > >>>>>>>>> Dynamic deployment as a requirement was also a bit of a
> > >>>>>>>>> mis-fire
> > >>>>>> because
> > >>>>>>>>> of
> > >>>>>>>>> a fundamental misunderstanding between the nature of batch jobs
> > >>>>>>>>> and
> > >>>>>>> stream
> > >>>>>>>>> processing jobs. Early on, we made conscious effort to favor
> > >>>>>>>>> the
> > >>>>>> Hadoop
> > >>>>>>>>> (Map/Reduce) way of doing things, since it worked and was well
> > >>>>>>> understood.
> > >>>>>>>>> One thing that we missed was that batch jobs have a definite
> > >>>>>> beginning,
> > >>>>>>>>> and
> > >>>>>>>>> end, and stream processing jobs don't (usually). This leads to
> > >>>>>>>>> a
> > >>>>> much
> > >>>>>>>>> simpler scheduling problem for stream processors. You basically
> > >>>>>>>>> just
> > >>>>>>> need
> > >>>>>>>>> to find a place to start the processor, and start it. The way
> > >>>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> > >>>>>>>>> being "full". We always
> > >>>>>> add
> > >>>>>>>>> more machines. The problem with coupling Samza with a scheduler
> > >>>>>>>>> is
> > >>>>>> that
> > >>>>>>>>> Samza (as a framework) now has to handle deployment. This pulls
> > >>>>>>>>> in a
> > >>>>>>> bunch
> > >>>>>>>>> of things such as configuration distribution (config stream),
> > >>>>>>>>> shell
> > >>>>>>> scrips
> > >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff),
> etc.
> > >>>>>>>>>
> > >>>>>>>>> Another reason for requiring dynamic deployment was to support
> > >>>>>>>>> data locality. If you want to have locality, you need to put
> > >>>>>>>>> your
> > >>>>>> processors
> > >>>>>>>>> close to the data they're processing. Upon further
> > >>>>>>>>> investigation,
> > >>>>>>> though,
> > >>>>>>>>> this feature is not that beneficial. There is some good
> > >>>>>>>>> discussion
> > >>>>>> about
> > >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
> > >>>>>>>>> Map/Reduce
> > >>>>>> path,
> > >>>>>>>>> but
> > >>>>>>>>> there are some fundamental differences between HDFS and Kafka.
> > >>>>>>>>> HDFS
> > >>>>>> has
> > >>>>>>>>> blocks, while Kafka has partitions. This leads to less
> > >>>>>>>>> optimization potential with stream processors on top of Kafka.
> > >>>>>>>>>
> > >>>>>>>>> This feature is also used as a crutch. Samza doesn't have any
> > >>>>>>>>> built
> > >>>>> in
> > >>>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> > >>>>>>>>> deployment scheduling system to handle restarts when a
> > >>>>>>>>> processor dies. This has
> > >>>>>>> made
> > >>>>>>>>> it very difficult to write a standalone Samza container
> > >>>> (SAMZA-516).
> > >>>>>>>>>
> > >>>>>>>>> Pluggability
> > >>>>>>>>>
> > >>>>>>>>> In some cases pluggability is good, but I think that we've gone
> > >>>>>>>>> too
> > >>>>>> far
> > >>>>>>>>> with it. Currently, Samza has:
> > >>>>>>>>>
> > >>>>>>>>> * Pluggable config.
> > >>>>>>>>> * Pluggable metrics.
> > >>>>>>>>> * Pluggable deployment systems.
> > >>>>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
> > >>>> etc).
> > >>>>>>>>> * Pluggable serdes.
> > >>>>>>>>> * Pluggable storage engines.
> > >>>>>>>>> * Pluggable strategies for just about every component
> > >>>>> (MessageChooser,
> > >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > >>>>>>>>>
> > >>>>>>>>> There's probably more that I've forgotten, as well. Some of
> > >>>>>>>>> these
> > >>>>> are
> > >>>>>>>>> useful, but some have proven not to be. This all comes at a
> cost:
> > >>>>>>>>> complexity. This complexity is making it harder for our users
> > >>>>>>>>> to
> > >>>>> pick
> > >>>>>> up
> > >>>>>>>>> and use Samza out of the box. It also makes it difficult for
> > >>>>>>>>> Samza developers to reason about what the characteristics of
> > >>>>>>>>> the container (since the characteristics change depending on
> > >>>>>>>>> which plugins are use).
> > >>>>>>>>>
> > >>>>>>>>> The issues with pluggability are most visible in the System
> APIs.
> > >>>>> What
> > >>>>>>>>> Samza really requires to be functional is Kafka as its
> > >>>>>>>>> transport
> > >>>>>> layer.
> > >>>>>>>>> But
> > >>>>>>>>> we've conflated two unrelated use cases into one API:
> > >>>>>>>>>
> > >>>>>>>>> 1. Get data into/out of Kafka.
> > >>>>>>>>> 2. Process the data in Kafka.
> > >>>>>>>>>
> > >>>>>>>>> The current System API supports both of these use cases. The
> > >>>>>>>>> problem
> > >>>>>> is,
> > >>>>>>>>> we
> > >>>>>>>>> actually want different features for each use case. By papering
> > >>>>>>>>> over
> > >>>>>>> these
> > >>>>>>>>> two use cases, and providing a single API, we've introduced a
> > >>>>>>>>> ton of
> > >>>>>>> leaky
> > >>>>>>>>> abstractions.
> > >>>>>>>>>
> > >>>>>>>>> For example, what we'd really like in (2) is to have
> > >>>>>>>>> monotonically increasing longs for offsets (like Kafka). This
> > >>>>>>>>> would be at odds
> > >>>>> with
> > >>>>>>> (1),
> > >>>>>>>>> though, since different systems have different
> > >>>>>>> SCNs/Offsets/UUIDs/vectors.
> > >>>>>>>>> There was discussion both on the mailing list and the SQL JIRAs
> > >>>>> about
> > >>>>>>> the
> > >>>>>>>>> need for this.
> > >>>>>>>>>
> > >>>>>>>>> The same thing holds true for replayability. Kafka allows us to
> > >>>>> rewind
> > >>>>>>>>> when
> > >>>>>>>>> we have a failure. Many other systems don't. In some cases,
> > >>>>>>>>> systems
> > >>>>>>> return
> > >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
> > >>>>>>>>> they
> > >>>>>> have
> > >>>>>>> no
> > >>>>>>>>> offsets.
> > >>>>>>>>>
> > >>>>>>>>> Partitioning is another example. Kafka supports partitioning,
> > >>>>>>>>> but
> > >>>>> many
> > >>>>>>>>> systems don't. We model this by having a single partition for
> > >>>>>>>>> those systems. Still, other systems model partitioning
> > >>>> differently (e.g.
> > >>>>>>>>> Kinesis).
> > >>>>>>>>>
> > >>>>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
> > >>>>>>>>> system-agnostic way is almost impossible. As is modeling
> > >>>>>>>>> metadata
> > >>>>> for
> > >>>>>>> the
> > >>>>>>>>> system (replication factor, partitions, location, etc). The
> > >>>>>>>>> list
> > >>>>> goes
> > >>>>>>> on.
> > >>>>>>>>>
> > >>>>>>>>> Duplicate work
> > >>>>>>>>>
> > >>>>>>>>> At the time that we began writing Samza, Kafka's consumer and
> > >>>>> producer
> > >>>>>>>>> APIs
> > >>>>>>>>> had a relatively weak feature set. On the consumer-side, you
> > >>>>>>>>> had two
> > >>>>>>>>> options: use the high level consumer, or the simple consumer.
> > >>>>>>>>> The
> > >>>>>>> problem
> > >>>>>>>>> with the high-level consumer was that it controlled your
> > >>>>>>>>> offsets, partition assignments, and the order in which you
> > >>>>>>>>> received messages. The
> > >>>>> problem
> > >>>>>>>>> with
> > >>>>>>>>> the simple consumer is that it's not simple. It's basic. You
> > >>>>>>>>> end up
> > >>>>>>> having
> > >>>>>>>>> to handle a lot of really low-level stuff that you shouldn't.
> > >>>>>>>>> We
> > >>>>>> spent a
> > >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
> > >>>>>>>>> also allows us to support some cool features:
> > >>>>>>>>>
> > >>>>>>>>> * Per-partition message ordering and prioritization.
> > >>>>>>>>> * Tight control over partition assignment to support joins,
> > >>>>>>>>> global
> > >>>>>> state
> > >>>>>>>>> (if we want to implement it :)), etc.
> > >>>>>>>>> * Tight control over offset checkpointing.
> > >>>>>>>>>
> > >>>>>>>>> What we didn't realize at the time is that these features
> > >>>>>>>>> should
> > >>>>>>> actually
> > >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> > >>>>>> processors)
> > >>>>>>>>> end up wanting to do things like joins and partition
> > >>>>>>>>> assignment. The
> > >>>>>>> Kafka
> > >>>>>>>>> community has come to the same conclusion. They're adding a ton
> > >>>>>>>>> of upgrades into their new Kafka consumer implementation. To a
> > >>>>>>>>> large extent,
> > >>>>> it's
> > >>>>>>>>> duplicate work to what we've already done in Samza.
> > >>>>>>>>>
> > >>>>>>>>> On top of this, Kafka ended up taking a very similar approach
> > >>>>>>>>> to
> > >>>>>> Samza's
> > >>>>>>>>> KafkaCheckpointManager implementation for handling offset
> > >>>>>> checkpointing.
> > >>>>>>>>> Like Samza, Kafka's new offset management feature stores offset
> > >>>>>>>>> checkpoints in a topic, and allows you to fetch them from the
> > >>>>>>>>> broker.
> > >>>>>>>>>
> > >>>>>>>>> A lot of this seems like a waste, since we could have shared
> > >>>>>>>>> the
> > >>>>> work
> > >>>>>> if
> > >>>>>>>>> it
> > >>>>>>>>> had been done in Kafka from the get-go.
> > >>>>>>>>>
> > >>>>>>>>> Vision
> > >>>>>>>>>
> > >>>>>>>>> All of this leads me to a rather radical proposal. Samza is
> > >>>>> relatively
> > >>>>>>>>> stable at this point. I'd venture to say that we're near a 1.0
> > >>>>>> release.
> > >>>>>>>>> I'd
> > >>>>>>>>> like to propose that we take what we've learned, and begin
> > >>>>>>>>> thinking
> > >>>>>>> about
> > >>>>>>>>> Samza beyond 1.0. What would we change if we were starting from
> > >>>>>> scratch?
> > >>>>>>>>> My
> > >>>>>>>>> proposal is to:
> > >>>>>>>>>
> > >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > >>>>>>>>> processors, and eliminate all direct dependences on YARN,
> Mesos,
> > >>>> etc.
> > >>>>>>>>> 2. Make a definitive call to support only Kafka as the stream
> > >>>>>> processing
> > >>>>>>>>> layer.
> > >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> > >>>>>>>>> config
> > >>>>>>> systems,
> > >>>>>>>>> and simply use Kafka's instead.
> > >>>>>>>>>
> > >>>>>>>>> This would fix all of the issues that I outlined above. It
> > >>>>>>>>> should
> > >>>>> also
> > >>>>>>>>> shrink the Samza code base pretty dramatically. Supporting only
> > >>>>>>>>> a standalone container will allow Samza to be executed on YARN
> > >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> > >>>>>>>>> in-house
> > >>>>>>> deployment
> > >>>>>>>>> systems. This should make life a lot easier for new users.
> > >>>>>>>>> Imagine
> > >>>>>>> having
> > >>>>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
> > >>>>>> traffic
> > >>>>>>>>> will be pretty dramatic.
> > >>>>>>>>>
> > >>>>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
> > >>>>> everyone
> > >>>>>>>>> that
> > >>>>>>>>> I'm aware of is using Samza with Kafka. We basically require it
> > >>>>>> already
> > >>>>>>> in
> > >>>>>>>>> order for most features to work. Those that are using other
> > >>>>>>>>> systems
> > >>>>>> are
> > >>>>>>>>> generally using it for ingest into Kafka (1), and then they do
> > >>>>>>>>> the processing on top. There is already discussion (
> > >>>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >>>>> 767
> > >>>>>>>>> )
> > >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > >>>>>>>>>
> > >>>>>>>>> Once we make the call to couple with Kafka, we can leverage a
> > >>>>>>>>> ton of
> > >>>>>>> their
> > >>>>>>>>> ecosystem. We no longer have to maintain our own config,
> > >>>>>>>>> metrics,
> > >>>>> etc.
> > >>>>>>> We
> > >>>>>>>>> can all share the same libraries, and make them better. This
> > >>>>>>>>> will
> > >>>>> also
> > >>>>>>>>> allow us to share the consumer/producer APIs, and will let us
> > >>>>> leverage
> > >>>>>>>>> their offset management and partition management, rather than
> > >>>>>>>>> having
> > >>>>>> our
> > >>>>>>>>> own. All of the coordinator stream code would go away, as would
> > >>>>>>>>> most
> > >>>>>> of
> > >>>>>>>>> the
> > >>>>>>>>> YARN AppMaster code. We'd probably have to push some partition
> > >>>>>>> management
> > >>>>>>>>> features into the Kafka broker, but they're already moving in
> > >>>>>>>>> that direction with the new consumer API. The features we have
> > >>>>>>>>> for
> > >>>>>> partition
> > >>>>>>>>> assignment aren't unique to Samza, and seem like they should be
> > >>>>>>>>> in
> > >>>>>> Kafka
> > >>>>>>>>> anyway. There will always be some niche usages which will
> > >>>>>>>>> require
> > >>>>>> extra
> > >>>>>>>>> care and hence full control over partition assignments much
> > >>>>>>>>> like the
> > >>>>>>> Kafka
> > >>>>>>>>> low level consumer api. These would continue to be supported.
> > >>>>>>>>>
> > >>>>>>>>> These items will be good for the Samza community. They'll make
> > >>>>>>>>> Samza easier to use, and make it easier for developers to add
> > >>>>>>>>> new features.
> > >>>>>>>>>
> > >>>>>>>>> Obviously this is a fairly large (and somewhat backwards
> > >>>>> incompatible
> > >>>>>>>>> change). If we choose to go this route, it's important that we
> > >>>>> openly
> > >>>>>>>>> communicate how we're going to provide a migration path from
> > >>>>>>>>> the
> > >>>>>>> existing
> > >>>>>>>>> APIs to the new ones (if we make incompatible changes). I think
> > >>>>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
> > >>>>>>>>> existing StreamTask implementations to continue running on the
> > >>>> new container.
> > >>>>>>> It's
> > >>>>>>>>> also important that we openly communicate about timing, and
> > >>>>>>>>> stages
> > >>>>> of
> > >>>>>>> the
> > >>>>>>>>> migration.
> > >>>>>>>>>
> > >>>>>>>>> If you made it this far, I'm sure you have opinions. :) Please
> > >>>>>>>>> send
> > >>>>>> your
> > >>>>>>>>> thoughts and feedback.
> > >>>>>>>>>
> > >>>>>>>>> Cheers,
> > >>>>>>>>> Chris
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> --
> > >>>>>> -- Guozhang
> > >>>>>>
> > >>>>>
> > >>>>
> > >>
> > >>
> >
> >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@gmail.com>.
Hey Martin,

I agree coupling release schedules is a downside.

Definitely we can try to solve some of the integration problems in
Confluent Platform or in other distributions. But I think this ends up
being really shallow. I guess I feel to really get a good user experience
the two systems have to kind of feel like part of the same thing and you
can't really add that in later--you can put both in the same downloadable
tar file but it doesn't really give a very cohesive feeling. I agree that
ultimately any of the project stuff is as much social and naming as
anything else--theoretically two totally independent projects could work to
tightly align. In practice this seems to be quite difficult though.

For the frameworks--totally agree it would be good to maintain the
framework support with the project. In some cases there may not be too much
there since the integration gets lighter but I think whatever stubs you
need should be included. So no I definitely wasn't trying to imply dropping
support for these frameworks, just making the integration lighter by
separating process management from partition management.

You raise two good points we would have to figure out if we went down the
alignment path:
1. With respect to the name, yeah I think the first question is whether
some "re-branding" would be worth it. If so then I think we can have a big
thread on the name. I'm definitely not set on Kafka Streaming or Kafka
Streams I was just using them to be kind of illustrative. I agree with your
critique of these names, though I think people would get the idea.
2. Yeah you also raise a good point about how to "factor" it. Here are the
options I see (I could get enthusiastic about any of them):
   a. One repo for both Kafka and Samza
   b. Two repos, retaining the current seperation
   c. Two repos, the equivalent of samza-api and samza-core is absorbed
almost like a third client

Cheers,

-Jay

On Mon, Jul 6, 2015 at 1:18 PM, Martin Kleppmann <ma...@kleppmann.com>
wrote:

> Ok, thanks for the clarifications. Just a few follow-up comments.
>
> - I see the appeal of merging with Kafka or becoming a subproject: the
> reasons you mention are good. The risk I see is that release schedules
> become coupled to each other, which can slow everyone down, and large
> projects with many contributors are harder to manage. (Jakob, can you speak
> from experience, having seen a wider range of Hadoop ecosystem projects?)
>
> Some of the goals of a better unified developer experience could also be
> solved by integrating Samza nicely into a Kafka distribution (such as
> Confluent's). I'm not against merging projects if we decide that's the way
> to go, just pointing out the same goals can perhaps also be achieved in
> other ways.
>
> - With regard to dropping the YARN dependency: are you proposing that
> Samza doesn't give any help to people wanting to run on YARN/Mesos/AWS/etc?
> So the docs would basically have a link to Slider and nothing else? Or
> would we maintain integrations with a bunch of popular deployment methods
> (e.g. the necessary glue and shell scripts to make Samza work with Slider)?
>
> I absolutely think it's a good idea to have the "as a library" and "as a
> process" (using Yi's taxonomy) options for people who want them, but I
> think there should also be a low-friction path for common "as a service"
> deployment methods, for which we probably need to maintain integrations.
>
> - Project naming: "Kafka Streams" seems odd to me, because Kafka is all
> about streams already. Perhaps "Kafka Transformers" or "Kafka Filters"
> would be more apt?
>
> One suggestion: perhaps the core of Samza (stream transformation with
> state management -- i.e. the "Samza as a library" bit) could become part of
> Kafka, while higher-level tools such as streaming SQL and integrations with
> deployment frameworks remain in a separate project? In other words, Kafka
> would absorb the proven, stable core of Samza, which would become the
> "third Kafka client" mentioned early in this thread. The Samza project
> would then target that third Kafka client as its base API, and the project
> would be freed up to explore more experimental new horizons.
>
> Martin
>
> On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Martin,
> >
> > For the YARN/Mesos/etc decoupling I actually don't think it ties our
> hands
> > at all, all it does is refactor things. The division of responsibility is
> > that Samza core is responsible for task lifecycle, state, and partition
> > management (using the Kafka co-ordinator) but it is NOT responsible for
> > packaging, configuration deployment or execution of processes. The
> problem
> > of packaging and starting these processes is
> > framework/environment-specific. This leaves individual frameworks to be
> as
> > fancy or vanilla as they like. So you can get simple stateless support in
> > YARN, Mesos, etc using their off-the-shelf app framework (Slider,
> Marathon,
> > etc). These are well known by people and have nice UIs and a lot of
> > flexibility. I don't think they have node affinity as a built in option
> > (though I could be wrong). So if we want that we can either wait for them
> > to add it or do a custom framework to add that feature (as now).
> Obviously
> > if you manage things with old-school ops tools (puppet/chef/etc) you get
> > locality easily. The nice thing, though, is that all the samza "business
> > logic" around partition management and fault tolerance is in Samza core
> so
> > it is shared across frameworks and the framework specific bit is just
> > whether it is smart enough to try to get the same host when a job is
> > restarted.
> >
> > With respect to the Kafka-alignment, yeah I think the goal would be (a)
> > actually get better alignment in user experience, and (b) express this in
> > the naming and project branding. Specifically:
> > 1. Website/docs, it would be nice for the "transformation" api to be
> > discoverable in the main Kafka docs--i.e. be able to explain when to use
> > the consumer and when to use the stream processing functionality and lead
> > people into that experience.
> > 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that has both
> > Kafka and the stream processing part and they actually work together.
> > 3. Unify the programming experience so the client and Samza api share
> > config/monitoring/naming/packaging/etc.
> >
> > I think sub-projects keep separate committers and can have a separate
> repo,
> > but I'm actually not really sure (I can't find a definition of a
> subproject
> > in Apache).
> >
> > Basically at a high-level you want the experience to "feel" like a single
> > system, not to relatively independent things that are kind of awkwardly
> > glued together.
> >
> > I think if we did that they having naming or branding like "kafka
> > streaming" or "kafka streams" or something like that would actually do a
> > good job of conveying what it is. I do that this would help adoption
> quite
> > a lot as it would correctly convey that using Kafka Streaming with Kafka
> is
> > a fairly seamless experience and Kafka is pretty heavily adopted at this
> > point.
> >
> > Fwiw we actually considered this model originally when open sourcing
> Samza,
> > however at that time Kafka was relatively unknown and we decided not to
> do
> > it since we felt it would be limiting. From my point of view the three
> > things have changed (1) Kafka is now really heavily used for stream
> > processing, (2) we learned that abstracting out the stream well is
> > basically impossible, (3) we learned it is really hard to keep the two
> > things feeling like a single product.
> >
> > -Jay
> >
> >
> > On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <ma...@kleppmann.com>
> > wrote:
> >
> >> Hi all,
> >>
> >> Lots of good thoughts here.
> >>
> >> I agree with the general philosophy of tying Samza more firmly to Kafka.
> >> After I spent a while looking at integrating other message brokers (e.g.
> >> Kinesis) with SystemConsumer, I came to the conclusion that
> SystemConsumer
> >> tacitly assumes a model so much like Kafka's that pretty much nobody but
> >> Kafka actually implements it. (Databus is perhaps an exception, but it
> >> isn't widely used outside of LinkedIn.) Thus, making Samza fully
> dependent
> >> on Kafka acknowledges that the system-independence was never as real as
> we
> >> perhaps made it out to be. The gains of code reuse are real.
> >>
> >> The idea of decoupling Samza from YARN has also always been appealing to
> >> me, for various reasons already mentioned in this thread. Although
> making
> >> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems laudable,
> I am
> >> a little concerned that it will restrict us to a lowest common
> denominator.
> >> For example, would host affinity (SAMZA-617) still be possible? For jobs
> >> with large amounts of state, I think SAMZA-617 would be a big boon,
> since
> >> restoring state off the changelog on every single restart is painful,
> due
> >> to long recovery times. It would be a shame if the decoupling from YARN
> >> made host affinity impossible.
> >>
> >> Jay, a question about the proposed API for instantiating a job in code
> >> (rather than a properties file): when submitting a job to a cluster, is
> the
> >> idea that the instantiation code runs on a client somewhere, which then
> >> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code
> run
> >> on each container that is part of the job (in which case, how does the
> job
> >> submission to the cluster work)?
> >>
> >> I agree with Garry that it doesn't feel right to make a 1.0 release
> with a
> >> plan for it to be immediately obsolete. So if this is going to happen, I
> >> think it would be more honest to stick with 0.* version numbers until
> the
> >> library-ified Samza has been implemented, is stable and widely used.
> >>
> >> Should the new Samza be a subproject of Kafka? There is precedent for
> >> tight coupling between different Apache projects (e.g. Curator and
> >> Zookeeper, or Slider and YARN), so I think remaining separate would be
> ok.
> >> Even if Samza is fully dependent on Kafka, there is enough substance in
> >> Samza that it warrants being a separate project. An argument in favour
> of
> >> merging would be if we think Kafka has a much stronger "brand presence"
> >> than Samza; I'm ambivalent on that one. If the Kafka project is willing
> to
> >> endorse Samza as the "official" way of doing stateful stream
> >> transformations, that would probably have much the same effect as
> >> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
> >> collaboration between the two projects will be needed in any case.
> >>
> >> From a project management perspective, I guess the "new Samza" would
> have
> >> to be developed on a branch alongside ongoing maintenance of the current
> >> line of development? I think it would be important to continue
> supporting
> >> existing users, and provide a graceful migration path to the new
> version.
> >> Leaving the current versions unsupported and forcing people to rewrite
> >> their jobs would send a bad signal.
> >>
> >> Best,
> >> Martin
> >>
> >> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
> >>
> >>> Hey Garry,
> >>>
> >>> Yeah that's super frustrating. I'd be happy to chat more about this if
> >>> you'd be interested. I think Chris and I started with the idea of "what
> >>> would it take to make Samza a kick-ass ingestion tool" but ultimately
> we
> >>> kind of came around to the idea that ingestion and transformation had
> >>> pretty different needs and coupling the two made things hard.
> >>>
> >>> For what it's worth I think copycat (KIP-26) actually will do what you
> >> are
> >>> looking for.
> >>>
> >>> With regard to your point about slider, I don't necessarily disagree.
> >> But I
> >>> think getting good YARN support is quite doable and I think we can make
> >>> that work well. I think the issue this proposal solves is that
> >> technically
> >>> it is pretty hard to support multiple cluster management systems the
> way
> >>> things are now, you need to write an "app master" or "framework" for
> each
> >>> and they are all a little different so testing is really hard. In the
> >>> absence of this we have been stuck with just YARN which has fantastic
> >>> penetration in the Hadoopy part of the org, but zero penetration
> >> elsewhere.
> >>> Given the huge amount of work being put in to slider, marathon, aws
> >>> tooling, not to mention the umpteen related packaging technologies
> people
> >>> want to use (Docker, Kubernetes, various cloud-specific deploy tools,
> >> etc)
> >>> I really think it is important to get this right.
> >>>
> >>> -Jay
> >>>
> >>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> >>> g.turkington@improvedigital.com> wrote:
> >>>
> >>>> Hi all,
> >>>>
> >>>> I think the question below re does Samza become a sub-project of Kafka
> >>>> highlights the broader point around migration. Chris mentions Samza's
> >>>> maturity is heading towards a v1 release but I'm not sure it feels
> >> right to
> >>>> launch a v1 then immediately plan to deprecate most of it.
> >>>>
> >>>> From a selfish perspective I have some guys who have started working
> >> with
> >>>> Samza and building some new consumers/producers was next up. Sounds
> like
> >>>> that is absolutely not the direction to go. I need to look into the
> KIP
> >> in
> >>>> more detail but for me the attractiveness of adding new Samza
> >>>> consumer/producers -- even if yes all they were doing was really
> getting
> >>>> data into and out of Kafka --  was to avoid  having to worry about the
> >>>> lifecycle management of external clients. If there is a generic Kafka
> >>>> ingress/egress layer that I can plug a new connector into and have a
> >> lot of
> >>>> the heavy lifting re scale and reliability done for me then it gives
> me
> >> all
> >>>> the pushing new consumers/producers would. If not then it complicates
> my
> >>>> operational deployments.
> >>>>
> >>>> Which is similar to my other question with the proposal -- if we
> build a
> >>>> fully available/stand-alone Samza plus the requisite shims to
> integrate
> >>>> with Slider etc I suspect the former may be a lot more work than we
> >> think.
> >>>> We may make it much easier for a newcomer to get something running but
> >>>> having them step up and get a reliable production deployment may still
> >>>> dominate mailing list  traffic, if for different reasons than today.
> >>>>
> >>>> Don't get me wrong -- I'm comfortable with making the Samza dependency
> >> on
> >>>> Kafka much more explicit and I absolutely see the benefits  in the
> >>>> reduction of duplication and clashing terminologies/abstractions that
> >>>> Chris/Jay describe. Samza as a library would likely be a very nice
> tool
> >> to
> >>>> add to the Kafka ecosystem. I just have the concerns above re the
> >>>> operational side.
> >>>>
> >>>> Garry
> >>>>
> >>>> -----Original Message-----
> >>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> >>>> Sent: 02 July 2015 12:56
> >>>> To: dev@samza.apache.org
> >>>> Subject: Re: Thoughts and obesrvations on Samza
> >>>>
> >>>> Very interesting thoughts.
> >>>> From outside, I have always perceived Samza as a computing layer over
> >>>> Kafka.
> >>>>
> >>>> The question, maybe a bit provocative, is "should Samza be a
> sub-project
> >>>> of Kafka then?"
> >>>> Or does it make sense to keep it as a separate project with a separate
> >>>> governance?
> >>>>
> >>>> Cheers,
> >>>>
> >>>> --
> >>>> Gianmarco
> >>>>
> >>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> >>>>
> >>>>> Overall, I agree to couple with Kafka more tightly. Because Samza de
> >>>>> facto is based on Kafka, and it should leverage what Kafka has. At
> the
> >>>>> same time, Kafka does not need to reinvent what Samza already has. I
> >>>>> also like the idea of separating the ingestion and transformation.
> >>>>>
> >>>>> But it is a little difficult for me to image how the Samza will look
> >>>> like.
> >>>>> And I feel Chris and Jay have a little difference in terms of how
> >>>>> Samza should look like.
> >>>>>
> >>>>> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
> >>>>> user's application code calls this client?
> >>>>>
> >>>>> 1. If we make Samza be a library of Kafka (like what the code shows),
> >>>>> how do we implement auto-balance and fault-tolerance? Are they taken
> >>>>> care by the Kafka broker or other mechanism, such as "Samza worker"
> >>>>> (just make up the name) ?
> >>>>>
> >>>>> 2. What about other features, such as auto-scaling, shared state,
> >>>>> monitoring?
> >>>>>
> >>>>>
> >>>>> *** If we have Samza standalone, (is this what Chris suggests?)
> >>>>>
> >>>>> 1. we still need to ingest data from Kakfa and produce to it. Then it
> >>>>> becomes the same as what Samza looks like now, except it does not
> rely
> >>>>> on Yarn anymore.
> >>>>>
> >>>>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
> >>>>> etc? Use Kafka code as the dependency?
> >>>>>
> >>>>>
> >>>>> Thanks,
> >>>>>
> >>>>> Fang, Yan
> >>>>> yanfang724@gmail.com
> >>>>>
> >>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
> >>>> wrote:
> >>>>>
> >>>>>> Read through the code example and it looks good to me. A few
> >>>>>> thoughts regarding deployment:
> >>>>>>
> >>>>>> Today Samza deploys as executable runnable like:
> >>>>>>
> >>>>>> deploy/samza/bin/run-job.sh --config-factory=...
> >>>> --config-path=file://...
> >>>>>>
> >>>>>> And this proposal advocate for deploying Samza more as embedded
> >>>>>> libraries in user application code (ignoring the terminology since
> >>>>>> it is not the
> >>>>> same
> >>>>>> as the prototype code):
> >>>>>>
> >>>>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
> >>>>>> Thread(task); thread.start();
> >>>>>>
> >>>>>> I think both of these deployment modes are important for different
> >>>>>> types
> >>>>> of
> >>>>>> users. That said, I think making Samza purely standalone is still
> >>>>>> sufficient for either runnable or library modes.
> >>>>>>
> >>>>>> Guozhang
> >>>>>>
> >>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> >>>>>>
> >>>>>>> Looks like gmail mangled the code example, it was supposed to look
> >>>>>>> like
> >>>>>>> this:
> >>>>>>>
> >>>>>>> Properties props = new Properties();
> >>>>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
> >>>>>>> config = new StreamingConfig(props);
> >>>>>>> config.subscribe("test-topic-1", "test-topic-2");
> >>>>>>> config.processor(ExampleStreamProcessor.class);
> >>>>>>> config.serialization(new StringSerializer(), new
> >>>>>>> StringDeserializer()); KafkaStreaming container = new
> >>>>>>> KafkaStreaming(config); container.run();
> >>>>>>>
> >>>>>>> -Jay
> >>>>>>>
> >>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> >>>> wrote:
> >>>>>>>
> >>>>>>>> Hey guys,
> >>>>>>>>
> >>>>>>>> This came out of some conversations Chris and I were having
> >>>>>>>> around
> >>>>>>> whether
> >>>>>>>> it would make sense to use Samza as a kind of data ingestion
> >>>>> framework
> >>>>>>> for
> >>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> >>>>>> combined
> >>>>>>>> with complaints around config and YARN and the discussion around
> >>>>>>>> how
> >>>>> to
> >>>>>>>> best do a standalone mode.
> >>>>>>>>
> >>>>>>>> So the thought experiment was, given that Samza was basically
> >>>>>>>> already totally Kafka specific, what if you just embraced that
> >>>>>>>> and turned it
> >>>>>> into
> >>>>>>>> something less like a heavyweight framework and more like a
> >>>>>>>> third
> >>>>> Kafka
> >>>>>>>> client--a kind of "producing consumer" with state management
> >>>>>> facilities.
> >>>>>>>> Basically a library. Instead of a complex stream processing
> >>>>>>>> framework
> >>>>>>> this
> >>>>>>>> would actually be a very simple thing, not much more complicated
> >>>>>>>> to
> >>>>> use
> >>>>>>> or
> >>>>>>>> operate than a Kafka consumer. As Chris said we thought about it
> >>>>>>>> a
> >>>>> lot
> >>>>>> of
> >>>>>>>> what Samza (and the other stream processing systems were doing)
> >>>>> seemed
> >>>>>>> like
> >>>>>>>> kind of a hangover from MapReduce.
> >>>>>>>>
> >>>>>>>> Of course you need to ingest/output data to and from the stream
> >>>>>>>> processing. But when we actually looked into how that would
> >>>>>>>> work,
> >>>>> Samza
> >>>>>>>> isn't really an ideal data ingestion framework for a bunch of
> >>>>> reasons.
> >>>>>> To
> >>>>>>>> really do that right you need a pretty different internal data
> >>>>>>>> model
> >>>>>> and
> >>>>>>>> set of apis. So what if you split them and had an api for Kafka
> >>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> >>>>>>>> transformation (Samza).
> >>>>>>>>
> >>>>>>>> This would also allow really embracing the same terminology and
> >>>>>>>> conventions. One complaint about the current state is that the
> >>>>>>>> two
> >>>>>>> systems
> >>>>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
> >>>>>>> different
> >>>>>>>> config and monitoring systems means you kind of have to learn
> >>>>>>>> Kafka's
> >>>>>>> way,
> >>>>>>>> then learn Samza's slightly different way, then kind of
> >>>>>>>> understand
> >>>>> how
> >>>>>>> they
> >>>>>>>> map to each other, which having walked a few people through this
> >>>>>>>> is surprisingly tricky for folks to get.
> >>>>>>>>
> >>>>>>>> Since I have been spending a lot of time on airplanes I hacked
> >>>>>>>> up an ernest but still somewhat incomplete prototype of what
> >>>>>>>> this would
> >>>>> look
> >>>>>>>> like. This is just unceremoniously dumped into Kafka as it
> >>>>>>>> required a
> >>>>>> few
> >>>>>>>> changes to the new consumer. Here is the code:
> >>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >>>>> /apache/kafka/clients/streaming
> >>>>>>>>
> >>>>>>>> For the purpose of the prototype I just liberally renamed
> >>>>>>>> everything
> >>>>> to
> >>>>>>>> try to align it with Kafka with no regard for compatibility.
> >>>>>>>>
> >>>>>>>> To use this would be something like this:
> >>>>>>>> Properties props = new Properties();
> >>>>>>>> props.put("bootstrap.servers", "localhost:4242");
> >>>>>>>> StreamingConfig config = new
> >>>>> StreamingConfig(props);
> >>>>>>> config.subscribe("test-topic-1",
> >>>>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> >>>>>>> config.serialization(new
> >>>>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> >>>>>> container =
> >>>>>>>> new KafkaStreaming(config); container.run();
> >>>>>>>>
> >>>>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
> >>>>>>>> is basically StreamTask.
> >>>>>>>>
> >>>>>>>> So rather than putting all the class names in a file and then
> >>>>>>>> having
> >>>>>> the
> >>>>>>>> job assembled by reflection, you just instantiate the container
> >>>>>>>> programmatically. Work is balanced over however many instances
> >>>>>>>> of
> >>>>> this
> >>>>>>> are
> >>>>>>>> alive at any time (i.e. if an instance dies, new tasks are added
> >>>>>>>> to
> >>>>> the
> >>>>>>>> existing containers without shutting them down).
> >>>>>>>>
> >>>>>>>> We would provide some glue for running this stuff in YARN via
> >>>>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> >>>>>>>> but from the
> >>>>>> point
> >>>>>>> of
> >>>>>>>> view of these frameworks these stream processing jobs are just
> >>>>>> stateless
> >>>>>>>> services that can come and go and expand and contract at will.
> >>>>>>>> There
> >>>>> is
> >>>>>>> no
> >>>>>>>> more custom scheduler.
> >>>>>>>>
> >>>>>>>> Here are some relevant details:
> >>>>>>>>
> >>>>>>>>  1. It is only ~1300 lines of code, it would get larger if we
> >>>>>>>>  productionized but not vastly larger. We really do get a ton
> >>>>>>>> of
> >>>>>>> leverage
> >>>>>>>>  out of Kafka.
> >>>>>>>>  2. Partition management is fully delegated to the new consumer.
> >>>>> This
> >>>>>>>>  is nice since now any partition management strategy available
> >>>>>>>> to
> >>>>>> Kafka
> >>>>>>>>  consumer is also available to Samza (and vice versa) and with
> >>>>>>>> the
> >>>>>>> exact
> >>>>>>>>  same configs.
> >>>>>>>>  3. It supports state as well as state reuse
> >>>>>>>>
> >>>>>>>> Anyhow take a look, hopefully it is thought provoking.
> >>>>>>>>
> >>>>>>>> -Jay
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> >>>>>> criccomini@apache.org>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hey all,
> >>>>>>>>>
> >>>>>>>>> I have had some discussions with Samza engineers at LinkedIn
> >>>>>>>>> and
> >>>>>>> Confluent
> >>>>>>>>> and we came up with a few observations and would like to
> >>>>>>>>> propose
> >>>>> some
> >>>>>>>>> changes.
> >>>>>>>>>
> >>>>>>>>> We've observed some things that I want to call out about
> >>>>>>>>> Samza's
> >>>>>> design,
> >>>>>>>>> and I'd like to propose some changes.
> >>>>>>>>>
> >>>>>>>>> * Samza is dependent upon a dynamic deployment system.
> >>>>>>>>> * Samza is too pluggable.
> >>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> >>>>>>>>> APIs
> >>>>> are
> >>>>>>>>> trying to solve a lot of the same problems.
> >>>>>>>>>
> >>>>>>>>> All three of these issues are related, but I'll address them in
> >>>>> order.
> >>>>>>>>>
> >>>>>>>>> Deployment
> >>>>>>>>>
> >>>>>>>>> Samza strongly depends on the use of a dynamic deployment
> >>>>>>>>> scheduler
> >>>>>> such
> >>>>>>>>> as
> >>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> >>>>>>>>> there
> >>>>>> would
> >>>>>>>>> be
> >>>>>>>>> one or two winners in this area, and we could support them, and
> >>>>>>>>> the
> >>>>>> rest
> >>>>>>>>> would go away. In reality, there are many variations.
> >>>>>>>>> Furthermore,
> >>>>>> many
> >>>>>>>>> people still prefer to just start their processors like normal
> >>>>>>>>> Java processes, and use traditional deployment scripts such as
> >>>>>>>>> Fabric,
> >>>>>> Chef,
> >>>>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> >>>>>>>>> Samza start-up process really painful for first time users.
> >>>>>>>>>
> >>>>>>>>> Dynamic deployment as a requirement was also a bit of a
> >>>>>>>>> mis-fire
> >>>>>> because
> >>>>>>>>> of
> >>>>>>>>> a fundamental misunderstanding between the nature of batch jobs
> >>>>>>>>> and
> >>>>>>> stream
> >>>>>>>>> processing jobs. Early on, we made conscious effort to favor
> >>>>>>>>> the
> >>>>>> Hadoop
> >>>>>>>>> (Map/Reduce) way of doing things, since it worked and was well
> >>>>>>> understood.
> >>>>>>>>> One thing that we missed was that batch jobs have a definite
> >>>>>> beginning,
> >>>>>>>>> and
> >>>>>>>>> end, and stream processing jobs don't (usually). This leads to
> >>>>>>>>> a
> >>>>> much
> >>>>>>>>> simpler scheduling problem for stream processors. You basically
> >>>>>>>>> just
> >>>>>>> need
> >>>>>>>>> to find a place to start the processor, and start it. The way
> >>>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> >>>>>>>>> being "full". We always
> >>>>>> add
> >>>>>>>>> more machines. The problem with coupling Samza with a scheduler
> >>>>>>>>> is
> >>>>>> that
> >>>>>>>>> Samza (as a framework) now has to handle deployment. This pulls
> >>>>>>>>> in a
> >>>>>>> bunch
> >>>>>>>>> of things such as configuration distribution (config stream),
> >>>>>>>>> shell
> >>>>>>> scrips
> >>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> >>>>>>>>>
> >>>>>>>>> Another reason for requiring dynamic deployment was to support
> >>>>>>>>> data locality. If you want to have locality, you need to put
> >>>>>>>>> your
> >>>>>> processors
> >>>>>>>>> close to the data they're processing. Upon further
> >>>>>>>>> investigation,
> >>>>>>> though,
> >>>>>>>>> this feature is not that beneficial. There is some good
> >>>>>>>>> discussion
> >>>>>> about
> >>>>>>>>> some problems with it on SAMZA-335. Again, we took the
> >>>>>>>>> Map/Reduce
> >>>>>> path,
> >>>>>>>>> but
> >>>>>>>>> there are some fundamental differences between HDFS and Kafka.
> >>>>>>>>> HDFS
> >>>>>> has
> >>>>>>>>> blocks, while Kafka has partitions. This leads to less
> >>>>>>>>> optimization potential with stream processors on top of Kafka.
> >>>>>>>>>
> >>>>>>>>> This feature is also used as a crutch. Samza doesn't have any
> >>>>>>>>> built
> >>>>> in
> >>>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> >>>>>>>>> deployment scheduling system to handle restarts when a
> >>>>>>>>> processor dies. This has
> >>>>>>> made
> >>>>>>>>> it very difficult to write a standalone Samza container
> >>>> (SAMZA-516).
> >>>>>>>>>
> >>>>>>>>> Pluggability
> >>>>>>>>>
> >>>>>>>>> In some cases pluggability is good, but I think that we've gone
> >>>>>>>>> too
> >>>>>> far
> >>>>>>>>> with it. Currently, Samza has:
> >>>>>>>>>
> >>>>>>>>> * Pluggable config.
> >>>>>>>>> * Pluggable metrics.
> >>>>>>>>> * Pluggable deployment systems.
> >>>>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
> >>>> etc).
> >>>>>>>>> * Pluggable serdes.
> >>>>>>>>> * Pluggable storage engines.
> >>>>>>>>> * Pluggable strategies for just about every component
> >>>>> (MessageChooser,
> >>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> >>>>>>>>>
> >>>>>>>>> There's probably more that I've forgotten, as well. Some of
> >>>>>>>>> these
> >>>>> are
> >>>>>>>>> useful, but some have proven not to be. This all comes at a cost:
> >>>>>>>>> complexity. This complexity is making it harder for our users
> >>>>>>>>> to
> >>>>> pick
> >>>>>> up
> >>>>>>>>> and use Samza out of the box. It also makes it difficult for
> >>>>>>>>> Samza developers to reason about what the characteristics of
> >>>>>>>>> the container (since the characteristics change depending on
> >>>>>>>>> which plugins are use).
> >>>>>>>>>
> >>>>>>>>> The issues with pluggability are most visible in the System APIs.
> >>>>> What
> >>>>>>>>> Samza really requires to be functional is Kafka as its
> >>>>>>>>> transport
> >>>>>> layer.
> >>>>>>>>> But
> >>>>>>>>> we've conflated two unrelated use cases into one API:
> >>>>>>>>>
> >>>>>>>>> 1. Get data into/out of Kafka.
> >>>>>>>>> 2. Process the data in Kafka.
> >>>>>>>>>
> >>>>>>>>> The current System API supports both of these use cases. The
> >>>>>>>>> problem
> >>>>>> is,
> >>>>>>>>> we
> >>>>>>>>> actually want different features for each use case. By papering
> >>>>>>>>> over
> >>>>>>> these
> >>>>>>>>> two use cases, and providing a single API, we've introduced a
> >>>>>>>>> ton of
> >>>>>>> leaky
> >>>>>>>>> abstractions.
> >>>>>>>>>
> >>>>>>>>> For example, what we'd really like in (2) is to have
> >>>>>>>>> monotonically increasing longs for offsets (like Kafka). This
> >>>>>>>>> would be at odds
> >>>>> with
> >>>>>>> (1),
> >>>>>>>>> though, since different systems have different
> >>>>>>> SCNs/Offsets/UUIDs/vectors.
> >>>>>>>>> There was discussion both on the mailing list and the SQL JIRAs
> >>>>> about
> >>>>>>> the
> >>>>>>>>> need for this.
> >>>>>>>>>
> >>>>>>>>> The same thing holds true for replayability. Kafka allows us to
> >>>>> rewind
> >>>>>>>>> when
> >>>>>>>>> we have a failure. Many other systems don't. In some cases,
> >>>>>>>>> systems
> >>>>>>> return
> >>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
> >>>>>>>>> they
> >>>>>> have
> >>>>>>> no
> >>>>>>>>> offsets.
> >>>>>>>>>
> >>>>>>>>> Partitioning is another example. Kafka supports partitioning,
> >>>>>>>>> but
> >>>>> many
> >>>>>>>>> systems don't. We model this by having a single partition for
> >>>>>>>>> those systems. Still, other systems model partitioning
> >>>> differently (e.g.
> >>>>>>>>> Kinesis).
> >>>>>>>>>
> >>>>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
> >>>>>>>>> system-agnostic way is almost impossible. As is modeling
> >>>>>>>>> metadata
> >>>>> for
> >>>>>>> the
> >>>>>>>>> system (replication factor, partitions, location, etc). The
> >>>>>>>>> list
> >>>>> goes
> >>>>>>> on.
> >>>>>>>>>
> >>>>>>>>> Duplicate work
> >>>>>>>>>
> >>>>>>>>> At the time that we began writing Samza, Kafka's consumer and
> >>>>> producer
> >>>>>>>>> APIs
> >>>>>>>>> had a relatively weak feature set. On the consumer-side, you
> >>>>>>>>> had two
> >>>>>>>>> options: use the high level consumer, or the simple consumer.
> >>>>>>>>> The
> >>>>>>> problem
> >>>>>>>>> with the high-level consumer was that it controlled your
> >>>>>>>>> offsets, partition assignments, and the order in which you
> >>>>>>>>> received messages. The
> >>>>> problem
> >>>>>>>>> with
> >>>>>>>>> the simple consumer is that it's not simple. It's basic. You
> >>>>>>>>> end up
> >>>>>>> having
> >>>>>>>>> to handle a lot of really low-level stuff that you shouldn't.
> >>>>>>>>> We
> >>>>>> spent a
> >>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
> >>>>>>>>> also allows us to support some cool features:
> >>>>>>>>>
> >>>>>>>>> * Per-partition message ordering and prioritization.
> >>>>>>>>> * Tight control over partition assignment to support joins,
> >>>>>>>>> global
> >>>>>> state
> >>>>>>>>> (if we want to implement it :)), etc.
> >>>>>>>>> * Tight control over offset checkpointing.
> >>>>>>>>>
> >>>>>>>>> What we didn't realize at the time is that these features
> >>>>>>>>> should
> >>>>>>> actually
> >>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> >>>>>> processors)
> >>>>>>>>> end up wanting to do things like joins and partition
> >>>>>>>>> assignment. The
> >>>>>>> Kafka
> >>>>>>>>> community has come to the same conclusion. They're adding a ton
> >>>>>>>>> of upgrades into their new Kafka consumer implementation. To a
> >>>>>>>>> large extent,
> >>>>> it's
> >>>>>>>>> duplicate work to what we've already done in Samza.
> >>>>>>>>>
> >>>>>>>>> On top of this, Kafka ended up taking a very similar approach
> >>>>>>>>> to
> >>>>>> Samza's
> >>>>>>>>> KafkaCheckpointManager implementation for handling offset
> >>>>>> checkpointing.
> >>>>>>>>> Like Samza, Kafka's new offset management feature stores offset
> >>>>>>>>> checkpoints in a topic, and allows you to fetch them from the
> >>>>>>>>> broker.
> >>>>>>>>>
> >>>>>>>>> A lot of this seems like a waste, since we could have shared
> >>>>>>>>> the
> >>>>> work
> >>>>>> if
> >>>>>>>>> it
> >>>>>>>>> had been done in Kafka from the get-go.
> >>>>>>>>>
> >>>>>>>>> Vision
> >>>>>>>>>
> >>>>>>>>> All of this leads me to a rather radical proposal. Samza is
> >>>>> relatively
> >>>>>>>>> stable at this point. I'd venture to say that we're near a 1.0
> >>>>>> release.
> >>>>>>>>> I'd
> >>>>>>>>> like to propose that we take what we've learned, and begin
> >>>>>>>>> thinking
> >>>>>>> about
> >>>>>>>>> Samza beyond 1.0. What would we change if we were starting from
> >>>>>> scratch?
> >>>>>>>>> My
> >>>>>>>>> proposal is to:
> >>>>>>>>>
> >>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
> >>>>>>>>> processors, and eliminate all direct dependences on YARN, Mesos,
> >>>> etc.
> >>>>>>>>> 2. Make a definitive call to support only Kafka as the stream
> >>>>>> processing
> >>>>>>>>> layer.
> >>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> >>>>>>>>> config
> >>>>>>> systems,
> >>>>>>>>> and simply use Kafka's instead.
> >>>>>>>>>
> >>>>>>>>> This would fix all of the issues that I outlined above. It
> >>>>>>>>> should
> >>>>> also
> >>>>>>>>> shrink the Samza code base pretty dramatically. Supporting only
> >>>>>>>>> a standalone container will allow Samza to be executed on YARN
> >>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> >>>>>>>>> in-house
> >>>>>>> deployment
> >>>>>>>>> systems. This should make life a lot easier for new users.
> >>>>>>>>> Imagine
> >>>>>>> having
> >>>>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
> >>>>>> traffic
> >>>>>>>>> will be pretty dramatic.
> >>>>>>>>>
> >>>>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
> >>>>> everyone
> >>>>>>>>> that
> >>>>>>>>> I'm aware of is using Samza with Kafka. We basically require it
> >>>>>> already
> >>>>>>> in
> >>>>>>>>> order for most features to work. Those that are using other
> >>>>>>>>> systems
> >>>>>> are
> >>>>>>>>> generally using it for ingest into Kafka (1), and then they do
> >>>>>>>>> the processing on top. There is already discussion (
> >>>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >>>>> 767
> >>>>>>>>> )
> >>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> >>>>>>>>>
> >>>>>>>>> Once we make the call to couple with Kafka, we can leverage a
> >>>>>>>>> ton of
> >>>>>>> their
> >>>>>>>>> ecosystem. We no longer have to maintain our own config,
> >>>>>>>>> metrics,
> >>>>> etc.
> >>>>>>> We
> >>>>>>>>> can all share the same libraries, and make them better. This
> >>>>>>>>> will
> >>>>> also
> >>>>>>>>> allow us to share the consumer/producer APIs, and will let us
> >>>>> leverage
> >>>>>>>>> their offset management and partition management, rather than
> >>>>>>>>> having
> >>>>>> our
> >>>>>>>>> own. All of the coordinator stream code would go away, as would
> >>>>>>>>> most
> >>>>>> of
> >>>>>>>>> the
> >>>>>>>>> YARN AppMaster code. We'd probably have to push some partition
> >>>>>>> management
> >>>>>>>>> features into the Kafka broker, but they're already moving in
> >>>>>>>>> that direction with the new consumer API. The features we have
> >>>>>>>>> for
> >>>>>> partition
> >>>>>>>>> assignment aren't unique to Samza, and seem like they should be
> >>>>>>>>> in
> >>>>>> Kafka
> >>>>>>>>> anyway. There will always be some niche usages which will
> >>>>>>>>> require
> >>>>>> extra
> >>>>>>>>> care and hence full control over partition assignments much
> >>>>>>>>> like the
> >>>>>>> Kafka
> >>>>>>>>> low level consumer api. These would continue to be supported.
> >>>>>>>>>
> >>>>>>>>> These items will be good for the Samza community. They'll make
> >>>>>>>>> Samza easier to use, and make it easier for developers to add
> >>>>>>>>> new features.
> >>>>>>>>>
> >>>>>>>>> Obviously this is a fairly large (and somewhat backwards
> >>>>> incompatible
> >>>>>>>>> change). If we choose to go this route, it's important that we
> >>>>> openly
> >>>>>>>>> communicate how we're going to provide a migration path from
> >>>>>>>>> the
> >>>>>>> existing
> >>>>>>>>> APIs to the new ones (if we make incompatible changes). I think
> >>>>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
> >>>>>>>>> existing StreamTask implementations to continue running on the
> >>>> new container.
> >>>>>>> It's
> >>>>>>>>> also important that we openly communicate about timing, and
> >>>>>>>>> stages
> >>>>> of
> >>>>>>> the
> >>>>>>>>> migration.
> >>>>>>>>>
> >>>>>>>>> If you made it this far, I'm sure you have opinions. :) Please
> >>>>>>>>> send
> >>>>>> your
> >>>>>>>>> thoughts and feedback.
> >>>>>>>>>
> >>>>>>>>> Cheers,
> >>>>>>>>> Chris
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> --
> >>>>>> -- Guozhang
> >>>>>>
> >>>>>
> >>>>
> >>
> >>
>
>
>

Re: Thoughts and obesrvations on Samza

Posted by Martin Kleppmann <ma...@kleppmann.com>.
Ok, thanks for the clarifications. Just a few follow-up comments.

- I see the appeal of merging with Kafka or becoming a subproject: the reasons you mention are good. The risk I see is that release schedules become coupled to each other, which can slow everyone down, and large projects with many contributors are harder to manage. (Jakob, can you speak from experience, having seen a wider range of Hadoop ecosystem projects?)

Some of the goals of a better unified developer experience could also be solved by integrating Samza nicely into a Kafka distribution (such as Confluent's). I'm not against merging projects if we decide that's the way to go, just pointing out the same goals can perhaps also be achieved in other ways.

- With regard to dropping the YARN dependency: are you proposing that Samza doesn't give any help to people wanting to run on YARN/Mesos/AWS/etc? So the docs would basically have a link to Slider and nothing else? Or would we maintain integrations with a bunch of popular deployment methods (e.g. the necessary glue and shell scripts to make Samza work with Slider)?

I absolutely think it's a good idea to have the "as a library" and "as a process" (using Yi's taxonomy) options for people who want them, but I think there should also be a low-friction path for common "as a service" deployment methods, for which we probably need to maintain integrations.

- Project naming: "Kafka Streams" seems odd to me, because Kafka is all about streams already. Perhaps "Kafka Transformers" or "Kafka Filters" would be more apt?

One suggestion: perhaps the core of Samza (stream transformation with state management -- i.e. the "Samza as a library" bit) could become part of Kafka, while higher-level tools such as streaming SQL and integrations with deployment frameworks remain in a separate project? In other words, Kafka would absorb the proven, stable core of Samza, which would become the "third Kafka client" mentioned early in this thread. The Samza project would then target that third Kafka client as its base API, and the project would be freed up to explore more experimental new horizons.

Martin

On 6 Jul 2015, at 18:51, Jay Kreps <ja...@gmail.com> wrote:

> Hey Martin,
> 
> For the YARN/Mesos/etc decoupling I actually don't think it ties our hands
> at all, all it does is refactor things. The division of responsibility is
> that Samza core is responsible for task lifecycle, state, and partition
> management (using the Kafka co-ordinator) but it is NOT responsible for
> packaging, configuration deployment or execution of processes. The problem
> of packaging and starting these processes is
> framework/environment-specific. This leaves individual frameworks to be as
> fancy or vanilla as they like. So you can get simple stateless support in
> YARN, Mesos, etc using their off-the-shelf app framework (Slider, Marathon,
> etc). These are well known by people and have nice UIs and a lot of
> flexibility. I don't think they have node affinity as a built in option
> (though I could be wrong). So if we want that we can either wait for them
> to add it or do a custom framework to add that feature (as now). Obviously
> if you manage things with old-school ops tools (puppet/chef/etc) you get
> locality easily. The nice thing, though, is that all the samza "business
> logic" around partition management and fault tolerance is in Samza core so
> it is shared across frameworks and the framework specific bit is just
> whether it is smart enough to try to get the same host when a job is
> restarted.
> 
> With respect to the Kafka-alignment, yeah I think the goal would be (a)
> actually get better alignment in user experience, and (b) express this in
> the naming and project branding. Specifically:
> 1. Website/docs, it would be nice for the "transformation" api to be
> discoverable in the main Kafka docs--i.e. be able to explain when to use
> the consumer and when to use the stream processing functionality and lead
> people into that experience.
> 2. Align releases so if you get Kafkza 1.4.2 (or whatever) that has both
> Kafka and the stream processing part and they actually work together.
> 3. Unify the programming experience so the client and Samza api share
> config/monitoring/naming/packaging/etc.
> 
> I think sub-projects keep separate committers and can have a separate repo,
> but I'm actually not really sure (I can't find a definition of a subproject
> in Apache).
> 
> Basically at a high-level you want the experience to "feel" like a single
> system, not to relatively independent things that are kind of awkwardly
> glued together.
> 
> I think if we did that they having naming or branding like "kafka
> streaming" or "kafka streams" or something like that would actually do a
> good job of conveying what it is. I do that this would help adoption quite
> a lot as it would correctly convey that using Kafka Streaming with Kafka is
> a fairly seamless experience and Kafka is pretty heavily adopted at this
> point.
> 
> Fwiw we actually considered this model originally when open sourcing Samza,
> however at that time Kafka was relatively unknown and we decided not to do
> it since we felt it would be limiting. From my point of view the three
> things have changed (1) Kafka is now really heavily used for stream
> processing, (2) we learned that abstracting out the stream well is
> basically impossible, (3) we learned it is really hard to keep the two
> things feeling like a single product.
> 
> -Jay
> 
> 
> On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <ma...@kleppmann.com>
> wrote:
> 
>> Hi all,
>> 
>> Lots of good thoughts here.
>> 
>> I agree with the general philosophy of tying Samza more firmly to Kafka.
>> After I spent a while looking at integrating other message brokers (e.g.
>> Kinesis) with SystemConsumer, I came to the conclusion that SystemConsumer
>> tacitly assumes a model so much like Kafka's that pretty much nobody but
>> Kafka actually implements it. (Databus is perhaps an exception, but it
>> isn't widely used outside of LinkedIn.) Thus, making Samza fully dependent
>> on Kafka acknowledges that the system-independence was never as real as we
>> perhaps made it out to be. The gains of code reuse are real.
>> 
>> The idea of decoupling Samza from YARN has also always been appealing to
>> me, for various reasons already mentioned in this thread. Although making
>> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems laudable, I am
>> a little concerned that it will restrict us to a lowest common denominator.
>> For example, would host affinity (SAMZA-617) still be possible? For jobs
>> with large amounts of state, I think SAMZA-617 would be a big boon, since
>> restoring state off the changelog on every single restart is painful, due
>> to long recovery times. It would be a shame if the decoupling from YARN
>> made host affinity impossible.
>> 
>> Jay, a question about the proposed API for instantiating a job in code
>> (rather than a properties file): when submitting a job to a cluster, is the
>> idea that the instantiation code runs on a client somewhere, which then
>> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code run
>> on each container that is part of the job (in which case, how does the job
>> submission to the cluster work)?
>> 
>> I agree with Garry that it doesn't feel right to make a 1.0 release with a
>> plan for it to be immediately obsolete. So if this is going to happen, I
>> think it would be more honest to stick with 0.* version numbers until the
>> library-ified Samza has been implemented, is stable and widely used.
>> 
>> Should the new Samza be a subproject of Kafka? There is precedent for
>> tight coupling between different Apache projects (e.g. Curator and
>> Zookeeper, or Slider and YARN), so I think remaining separate would be ok.
>> Even if Samza is fully dependent on Kafka, there is enough substance in
>> Samza that it warrants being a separate project. An argument in favour of
>> merging would be if we think Kafka has a much stronger "brand presence"
>> than Samza; I'm ambivalent on that one. If the Kafka project is willing to
>> endorse Samza as the "official" way of doing stateful stream
>> transformations, that would probably have much the same effect as
>> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
>> collaboration between the two projects will be needed in any case.
>> 
>> From a project management perspective, I guess the "new Samza" would have
>> to be developed on a branch alongside ongoing maintenance of the current
>> line of development? I think it would be important to continue supporting
>> existing users, and provide a graceful migration path to the new version.
>> Leaving the current versions unsupported and forcing people to rewrite
>> their jobs would send a bad signal.
>> 
>> Best,
>> Martin
>> 
>> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
>> 
>>> Hey Garry,
>>> 
>>> Yeah that's super frustrating. I'd be happy to chat more about this if
>>> you'd be interested. I think Chris and I started with the idea of "what
>>> would it take to make Samza a kick-ass ingestion tool" but ultimately we
>>> kind of came around to the idea that ingestion and transformation had
>>> pretty different needs and coupling the two made things hard.
>>> 
>>> For what it's worth I think copycat (KIP-26) actually will do what you
>> are
>>> looking for.
>>> 
>>> With regard to your point about slider, I don't necessarily disagree.
>> But I
>>> think getting good YARN support is quite doable and I think we can make
>>> that work well. I think the issue this proposal solves is that
>> technically
>>> it is pretty hard to support multiple cluster management systems the way
>>> things are now, you need to write an "app master" or "framework" for each
>>> and they are all a little different so testing is really hard. In the
>>> absence of this we have been stuck with just YARN which has fantastic
>>> penetration in the Hadoopy part of the org, but zero penetration
>> elsewhere.
>>> Given the huge amount of work being put in to slider, marathon, aws
>>> tooling, not to mention the umpteen related packaging technologies people
>>> want to use (Docker, Kubernetes, various cloud-specific deploy tools,
>> etc)
>>> I really think it is important to get this right.
>>> 
>>> -Jay
>>> 
>>> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>>> g.turkington@improvedigital.com> wrote:
>>> 
>>>> Hi all,
>>>> 
>>>> I think the question below re does Samza become a sub-project of Kafka
>>>> highlights the broader point around migration. Chris mentions Samza's
>>>> maturity is heading towards a v1 release but I'm not sure it feels
>> right to
>>>> launch a v1 then immediately plan to deprecate most of it.
>>>> 
>>>> From a selfish perspective I have some guys who have started working
>> with
>>>> Samza and building some new consumers/producers was next up. Sounds like
>>>> that is absolutely not the direction to go. I need to look into the KIP
>> in
>>>> more detail but for me the attractiveness of adding new Samza
>>>> consumer/producers -- even if yes all they were doing was really getting
>>>> data into and out of Kafka --  was to avoid  having to worry about the
>>>> lifecycle management of external clients. If there is a generic Kafka
>>>> ingress/egress layer that I can plug a new connector into and have a
>> lot of
>>>> the heavy lifting re scale and reliability done for me then it gives me
>> all
>>>> the pushing new consumers/producers would. If not then it complicates my
>>>> operational deployments.
>>>> 
>>>> Which is similar to my other question with the proposal -- if we build a
>>>> fully available/stand-alone Samza plus the requisite shims to integrate
>>>> with Slider etc I suspect the former may be a lot more work than we
>> think.
>>>> We may make it much easier for a newcomer to get something running but
>>>> having them step up and get a reliable production deployment may still
>>>> dominate mailing list  traffic, if for different reasons than today.
>>>> 
>>>> Don't get me wrong -- I'm comfortable with making the Samza dependency
>> on
>>>> Kafka much more explicit and I absolutely see the benefits  in the
>>>> reduction of duplication and clashing terminologies/abstractions that
>>>> Chris/Jay describe. Samza as a library would likely be a very nice tool
>> to
>>>> add to the Kafka ecosystem. I just have the concerns above re the
>>>> operational side.
>>>> 
>>>> Garry
>>>> 
>>>> -----Original Message-----
>>>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
>>>> Sent: 02 July 2015 12:56
>>>> To: dev@samza.apache.org
>>>> Subject: Re: Thoughts and obesrvations on Samza
>>>> 
>>>> Very interesting thoughts.
>>>> From outside, I have always perceived Samza as a computing layer over
>>>> Kafka.
>>>> 
>>>> The question, maybe a bit provocative, is "should Samza be a sub-project
>>>> of Kafka then?"
>>>> Or does it make sense to keep it as a separate project with a separate
>>>> governance?
>>>> 
>>>> Cheers,
>>>> 
>>>> --
>>>> Gianmarco
>>>> 
>>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
>>>> 
>>>>> Overall, I agree to couple with Kafka more tightly. Because Samza de
>>>>> facto is based on Kafka, and it should leverage what Kafka has. At the
>>>>> same time, Kafka does not need to reinvent what Samza already has. I
>>>>> also like the idea of separating the ingestion and transformation.
>>>>> 
>>>>> But it is a little difficult for me to image how the Samza will look
>>>> like.
>>>>> And I feel Chris and Jay have a little difference in terms of how
>>>>> Samza should look like.
>>>>> 
>>>>> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
>>>>> user's application code calls this client?
>>>>> 
>>>>> 1. If we make Samza be a library of Kafka (like what the code shows),
>>>>> how do we implement auto-balance and fault-tolerance? Are they taken
>>>>> care by the Kafka broker or other mechanism, such as "Samza worker"
>>>>> (just make up the name) ?
>>>>> 
>>>>> 2. What about other features, such as auto-scaling, shared state,
>>>>> monitoring?
>>>>> 
>>>>> 
>>>>> *** If we have Samza standalone, (is this what Chris suggests?)
>>>>> 
>>>>> 1. we still need to ingest data from Kakfa and produce to it. Then it
>>>>> becomes the same as what Samza looks like now, except it does not rely
>>>>> on Yarn anymore.
>>>>> 
>>>>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
>>>>> etc? Use Kafka code as the dependency?
>>>>> 
>>>>> 
>>>>> Thanks,
>>>>> 
>>>>> Fang, Yan
>>>>> yanfang724@gmail.com
>>>>> 
>>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
>>>> wrote:
>>>>> 
>>>>>> Read through the code example and it looks good to me. A few
>>>>>> thoughts regarding deployment:
>>>>>> 
>>>>>> Today Samza deploys as executable runnable like:
>>>>>> 
>>>>>> deploy/samza/bin/run-job.sh --config-factory=...
>>>> --config-path=file://...
>>>>>> 
>>>>>> And this proposal advocate for deploying Samza more as embedded
>>>>>> libraries in user application code (ignoring the terminology since
>>>>>> it is not the
>>>>> same
>>>>>> as the prototype code):
>>>>>> 
>>>>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
>>>>>> Thread(task); thread.start();
>>>>>> 
>>>>>> I think both of these deployment modes are important for different
>>>>>> types
>>>>> of
>>>>>> users. That said, I think making Samza purely standalone is still
>>>>>> sufficient for either runnable or library modes.
>>>>>> 
>>>>>> Guozhang
>>>>>> 
>>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
>>>>>> 
>>>>>>> Looks like gmail mangled the code example, it was supposed to look
>>>>>>> like
>>>>>>> this:
>>>>>>> 
>>>>>>> Properties props = new Properties();
>>>>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
>>>>>>> config = new StreamingConfig(props);
>>>>>>> config.subscribe("test-topic-1", "test-topic-2");
>>>>>>> config.processor(ExampleStreamProcessor.class);
>>>>>>> config.serialization(new StringSerializer(), new
>>>>>>> StringDeserializer()); KafkaStreaming container = new
>>>>>>> KafkaStreaming(config); container.run();
>>>>>>> 
>>>>>>> -Jay
>>>>>>> 
>>>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
>>>> wrote:
>>>>>>> 
>>>>>>>> Hey guys,
>>>>>>>> 
>>>>>>>> This came out of some conversations Chris and I were having
>>>>>>>> around
>>>>>>> whether
>>>>>>>> it would make sense to use Samza as a kind of data ingestion
>>>>> framework
>>>>>>> for
>>>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
>>>>>> combined
>>>>>>>> with complaints around config and YARN and the discussion around
>>>>>>>> how
>>>>> to
>>>>>>>> best do a standalone mode.
>>>>>>>> 
>>>>>>>> So the thought experiment was, given that Samza was basically
>>>>>>>> already totally Kafka specific, what if you just embraced that
>>>>>>>> and turned it
>>>>>> into
>>>>>>>> something less like a heavyweight framework and more like a
>>>>>>>> third
>>>>> Kafka
>>>>>>>> client--a kind of "producing consumer" with state management
>>>>>> facilities.
>>>>>>>> Basically a library. Instead of a complex stream processing
>>>>>>>> framework
>>>>>>> this
>>>>>>>> would actually be a very simple thing, not much more complicated
>>>>>>>> to
>>>>> use
>>>>>>> or
>>>>>>>> operate than a Kafka consumer. As Chris said we thought about it
>>>>>>>> a
>>>>> lot
>>>>>> of
>>>>>>>> what Samza (and the other stream processing systems were doing)
>>>>> seemed
>>>>>>> like
>>>>>>>> kind of a hangover from MapReduce.
>>>>>>>> 
>>>>>>>> Of course you need to ingest/output data to and from the stream
>>>>>>>> processing. But when we actually looked into how that would
>>>>>>>> work,
>>>>> Samza
>>>>>>>> isn't really an ideal data ingestion framework for a bunch of
>>>>> reasons.
>>>>>> To
>>>>>>>> really do that right you need a pretty different internal data
>>>>>>>> model
>>>>>> and
>>>>>>>> set of apis. So what if you split them and had an api for Kafka
>>>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
>>>>>>>> transformation (Samza).
>>>>>>>> 
>>>>>>>> This would also allow really embracing the same terminology and
>>>>>>>> conventions. One complaint about the current state is that the
>>>>>>>> two
>>>>>>> systems
>>>>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
>>>>>>> different
>>>>>>>> config and monitoring systems means you kind of have to learn
>>>>>>>> Kafka's
>>>>>>> way,
>>>>>>>> then learn Samza's slightly different way, then kind of
>>>>>>>> understand
>>>>> how
>>>>>>> they
>>>>>>>> map to each other, which having walked a few people through this
>>>>>>>> is surprisingly tricky for folks to get.
>>>>>>>> 
>>>>>>>> Since I have been spending a lot of time on airplanes I hacked
>>>>>>>> up an ernest but still somewhat incomplete prototype of what
>>>>>>>> this would
>>>>> look
>>>>>>>> like. This is just unceremoniously dumped into Kafka as it
>>>>>>>> required a
>>>>>> few
>>>>>>>> changes to the new consumer. Here is the code:
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>>>>> /apache/kafka/clients/streaming
>>>>>>>> 
>>>>>>>> For the purpose of the prototype I just liberally renamed
>>>>>>>> everything
>>>>> to
>>>>>>>> try to align it with Kafka with no regard for compatibility.
>>>>>>>> 
>>>>>>>> To use this would be something like this:
>>>>>>>> Properties props = new Properties();
>>>>>>>> props.put("bootstrap.servers", "localhost:4242");
>>>>>>>> StreamingConfig config = new
>>>>> StreamingConfig(props);
>>>>>>> config.subscribe("test-topic-1",
>>>>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
>>>>>>> config.serialization(new
>>>>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
>>>>>> container =
>>>>>>>> new KafkaStreaming(config); container.run();
>>>>>>>> 
>>>>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
>>>>>>>> is basically StreamTask.
>>>>>>>> 
>>>>>>>> So rather than putting all the class names in a file and then
>>>>>>>> having
>>>>>> the
>>>>>>>> job assembled by reflection, you just instantiate the container
>>>>>>>> programmatically. Work is balanced over however many instances
>>>>>>>> of
>>>>> this
>>>>>>> are
>>>>>>>> alive at any time (i.e. if an instance dies, new tasks are added
>>>>>>>> to
>>>>> the
>>>>>>>> existing containers without shutting them down).
>>>>>>>> 
>>>>>>>> We would provide some glue for running this stuff in YARN via
>>>>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
>>>>>>>> but from the
>>>>>> point
>>>>>>> of
>>>>>>>> view of these frameworks these stream processing jobs are just
>>>>>> stateless
>>>>>>>> services that can come and go and expand and contract at will.
>>>>>>>> There
>>>>> is
>>>>>>> no
>>>>>>>> more custom scheduler.
>>>>>>>> 
>>>>>>>> Here are some relevant details:
>>>>>>>> 
>>>>>>>>  1. It is only ~1300 lines of code, it would get larger if we
>>>>>>>>  productionized but not vastly larger. We really do get a ton
>>>>>>>> of
>>>>>>> leverage
>>>>>>>>  out of Kafka.
>>>>>>>>  2. Partition management is fully delegated to the new consumer.
>>>>> This
>>>>>>>>  is nice since now any partition management strategy available
>>>>>>>> to
>>>>>> Kafka
>>>>>>>>  consumer is also available to Samza (and vice versa) and with
>>>>>>>> the
>>>>>>> exact
>>>>>>>>  same configs.
>>>>>>>>  3. It supports state as well as state reuse
>>>>>>>> 
>>>>>>>> Anyhow take a look, hopefully it is thought provoking.
>>>>>>>> 
>>>>>>>> -Jay
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>>>>>> criccomini@apache.org>
>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>>> Hey all,
>>>>>>>>> 
>>>>>>>>> I have had some discussions with Samza engineers at LinkedIn
>>>>>>>>> and
>>>>>>> Confluent
>>>>>>>>> and we came up with a few observations and would like to
>>>>>>>>> propose
>>>>> some
>>>>>>>>> changes.
>>>>>>>>> 
>>>>>>>>> We've observed some things that I want to call out about
>>>>>>>>> Samza's
>>>>>> design,
>>>>>>>>> and I'd like to propose some changes.
>>>>>>>>> 
>>>>>>>>> * Samza is dependent upon a dynamic deployment system.
>>>>>>>>> * Samza is too pluggable.
>>>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
>>>>>>>>> APIs
>>>>> are
>>>>>>>>> trying to solve a lot of the same problems.
>>>>>>>>> 
>>>>>>>>> All three of these issues are related, but I'll address them in
>>>>> order.
>>>>>>>>> 
>>>>>>>>> Deployment
>>>>>>>>> 
>>>>>>>>> Samza strongly depends on the use of a dynamic deployment
>>>>>>>>> scheduler
>>>>>> such
>>>>>>>>> as
>>>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
>>>>>>>>> there
>>>>>> would
>>>>>>>>> be
>>>>>>>>> one or two winners in this area, and we could support them, and
>>>>>>>>> the
>>>>>> rest
>>>>>>>>> would go away. In reality, there are many variations.
>>>>>>>>> Furthermore,
>>>>>> many
>>>>>>>>> people still prefer to just start their processors like normal
>>>>>>>>> Java processes, and use traditional deployment scripts such as
>>>>>>>>> Fabric,
>>>>>> Chef,
>>>>>>>>> Ansible, etc. Forcing a deployment system on users makes the
>>>>>>>>> Samza start-up process really painful for first time users.
>>>>>>>>> 
>>>>>>>>> Dynamic deployment as a requirement was also a bit of a
>>>>>>>>> mis-fire
>>>>>> because
>>>>>>>>> of
>>>>>>>>> a fundamental misunderstanding between the nature of batch jobs
>>>>>>>>> and
>>>>>>> stream
>>>>>>>>> processing jobs. Early on, we made conscious effort to favor
>>>>>>>>> the
>>>>>> Hadoop
>>>>>>>>> (Map/Reduce) way of doing things, since it worked and was well
>>>>>>> understood.
>>>>>>>>> One thing that we missed was that batch jobs have a definite
>>>>>> beginning,
>>>>>>>>> and
>>>>>>>>> end, and stream processing jobs don't (usually). This leads to
>>>>>>>>> a
>>>>> much
>>>>>>>>> simpler scheduling problem for stream processors. You basically
>>>>>>>>> just
>>>>>>> need
>>>>>>>>> to find a place to start the processor, and start it. The way
>>>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
>>>>>>>>> being "full". We always
>>>>>> add
>>>>>>>>> more machines. The problem with coupling Samza with a scheduler
>>>>>>>>> is
>>>>>> that
>>>>>>>>> Samza (as a framework) now has to handle deployment. This pulls
>>>>>>>>> in a
>>>>>>> bunch
>>>>>>>>> of things such as configuration distribution (config stream),
>>>>>>>>> shell
>>>>>>> scrips
>>>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
>>>>>>>>> 
>>>>>>>>> Another reason for requiring dynamic deployment was to support
>>>>>>>>> data locality. If you want to have locality, you need to put
>>>>>>>>> your
>>>>>> processors
>>>>>>>>> close to the data they're processing. Upon further
>>>>>>>>> investigation,
>>>>>>> though,
>>>>>>>>> this feature is not that beneficial. There is some good
>>>>>>>>> discussion
>>>>>> about
>>>>>>>>> some problems with it on SAMZA-335. Again, we took the
>>>>>>>>> Map/Reduce
>>>>>> path,
>>>>>>>>> but
>>>>>>>>> there are some fundamental differences between HDFS and Kafka.
>>>>>>>>> HDFS
>>>>>> has
>>>>>>>>> blocks, while Kafka has partitions. This leads to less
>>>>>>>>> optimization potential with stream processors on top of Kafka.
>>>>>>>>> 
>>>>>>>>> This feature is also used as a crutch. Samza doesn't have any
>>>>>>>>> built
>>>>> in
>>>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
>>>>>>>>> deployment scheduling system to handle restarts when a
>>>>>>>>> processor dies. This has
>>>>>>> made
>>>>>>>>> it very difficult to write a standalone Samza container
>>>> (SAMZA-516).
>>>>>>>>> 
>>>>>>>>> Pluggability
>>>>>>>>> 
>>>>>>>>> In some cases pluggability is good, but I think that we've gone
>>>>>>>>> too
>>>>>> far
>>>>>>>>> with it. Currently, Samza has:
>>>>>>>>> 
>>>>>>>>> * Pluggable config.
>>>>>>>>> * Pluggable metrics.
>>>>>>>>> * Pluggable deployment systems.
>>>>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
>>>> etc).
>>>>>>>>> * Pluggable serdes.
>>>>>>>>> * Pluggable storage engines.
>>>>>>>>> * Pluggable strategies for just about every component
>>>>> (MessageChooser,
>>>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>>>>>>>>> 
>>>>>>>>> There's probably more that I've forgotten, as well. Some of
>>>>>>>>> these
>>>>> are
>>>>>>>>> useful, but some have proven not to be. This all comes at a cost:
>>>>>>>>> complexity. This complexity is making it harder for our users
>>>>>>>>> to
>>>>> pick
>>>>>> up
>>>>>>>>> and use Samza out of the box. It also makes it difficult for
>>>>>>>>> Samza developers to reason about what the characteristics of
>>>>>>>>> the container (since the characteristics change depending on
>>>>>>>>> which plugins are use).
>>>>>>>>> 
>>>>>>>>> The issues with pluggability are most visible in the System APIs.
>>>>> What
>>>>>>>>> Samza really requires to be functional is Kafka as its
>>>>>>>>> transport
>>>>>> layer.
>>>>>>>>> But
>>>>>>>>> we've conflated two unrelated use cases into one API:
>>>>>>>>> 
>>>>>>>>> 1. Get data into/out of Kafka.
>>>>>>>>> 2. Process the data in Kafka.
>>>>>>>>> 
>>>>>>>>> The current System API supports both of these use cases. The
>>>>>>>>> problem
>>>>>> is,
>>>>>>>>> we
>>>>>>>>> actually want different features for each use case. By papering
>>>>>>>>> over
>>>>>>> these
>>>>>>>>> two use cases, and providing a single API, we've introduced a
>>>>>>>>> ton of
>>>>>>> leaky
>>>>>>>>> abstractions.
>>>>>>>>> 
>>>>>>>>> For example, what we'd really like in (2) is to have
>>>>>>>>> monotonically increasing longs for offsets (like Kafka). This
>>>>>>>>> would be at odds
>>>>> with
>>>>>>> (1),
>>>>>>>>> though, since different systems have different
>>>>>>> SCNs/Offsets/UUIDs/vectors.
>>>>>>>>> There was discussion both on the mailing list and the SQL JIRAs
>>>>> about
>>>>>>> the
>>>>>>>>> need for this.
>>>>>>>>> 
>>>>>>>>> The same thing holds true for replayability. Kafka allows us to
>>>>> rewind
>>>>>>>>> when
>>>>>>>>> we have a failure. Many other systems don't. In some cases,
>>>>>>>>> systems
>>>>>>> return
>>>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
>>>>>>>>> they
>>>>>> have
>>>>>>> no
>>>>>>>>> offsets.
>>>>>>>>> 
>>>>>>>>> Partitioning is another example. Kafka supports partitioning,
>>>>>>>>> but
>>>>> many
>>>>>>>>> systems don't. We model this by having a single partition for
>>>>>>>>> those systems. Still, other systems model partitioning
>>>> differently (e.g.
>>>>>>>>> Kinesis).
>>>>>>>>> 
>>>>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
>>>>>>>>> system-agnostic way is almost impossible. As is modeling
>>>>>>>>> metadata
>>>>> for
>>>>>>> the
>>>>>>>>> system (replication factor, partitions, location, etc). The
>>>>>>>>> list
>>>>> goes
>>>>>>> on.
>>>>>>>>> 
>>>>>>>>> Duplicate work
>>>>>>>>> 
>>>>>>>>> At the time that we began writing Samza, Kafka's consumer and
>>>>> producer
>>>>>>>>> APIs
>>>>>>>>> had a relatively weak feature set. On the consumer-side, you
>>>>>>>>> had two
>>>>>>>>> options: use the high level consumer, or the simple consumer.
>>>>>>>>> The
>>>>>>> problem
>>>>>>>>> with the high-level consumer was that it controlled your
>>>>>>>>> offsets, partition assignments, and the order in which you
>>>>>>>>> received messages. The
>>>>> problem
>>>>>>>>> with
>>>>>>>>> the simple consumer is that it's not simple. It's basic. You
>>>>>>>>> end up
>>>>>>> having
>>>>>>>>> to handle a lot of really low-level stuff that you shouldn't.
>>>>>>>>> We
>>>>>> spent a
>>>>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
>>>>>>>>> also allows us to support some cool features:
>>>>>>>>> 
>>>>>>>>> * Per-partition message ordering and prioritization.
>>>>>>>>> * Tight control over partition assignment to support joins,
>>>>>>>>> global
>>>>>> state
>>>>>>>>> (if we want to implement it :)), etc.
>>>>>>>>> * Tight control over offset checkpointing.
>>>>>>>>> 
>>>>>>>>> What we didn't realize at the time is that these features
>>>>>>>>> should
>>>>>>> actually
>>>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
>>>>>> processors)
>>>>>>>>> end up wanting to do things like joins and partition
>>>>>>>>> assignment. The
>>>>>>> Kafka
>>>>>>>>> community has come to the same conclusion. They're adding a ton
>>>>>>>>> of upgrades into their new Kafka consumer implementation. To a
>>>>>>>>> large extent,
>>>>> it's
>>>>>>>>> duplicate work to what we've already done in Samza.
>>>>>>>>> 
>>>>>>>>> On top of this, Kafka ended up taking a very similar approach
>>>>>>>>> to
>>>>>> Samza's
>>>>>>>>> KafkaCheckpointManager implementation for handling offset
>>>>>> checkpointing.
>>>>>>>>> Like Samza, Kafka's new offset management feature stores offset
>>>>>>>>> checkpoints in a topic, and allows you to fetch them from the
>>>>>>>>> broker.
>>>>>>>>> 
>>>>>>>>> A lot of this seems like a waste, since we could have shared
>>>>>>>>> the
>>>>> work
>>>>>> if
>>>>>>>>> it
>>>>>>>>> had been done in Kafka from the get-go.
>>>>>>>>> 
>>>>>>>>> Vision
>>>>>>>>> 
>>>>>>>>> All of this leads me to a rather radical proposal. Samza is
>>>>> relatively
>>>>>>>>> stable at this point. I'd venture to say that we're near a 1.0
>>>>>> release.
>>>>>>>>> I'd
>>>>>>>>> like to propose that we take what we've learned, and begin
>>>>>>>>> thinking
>>>>>>> about
>>>>>>>>> Samza beyond 1.0. What would we change if we were starting from
>>>>>> scratch?
>>>>>>>>> My
>>>>>>>>> proposal is to:
>>>>>>>>> 
>>>>>>>>> 1. Make Samza standalone the *only* way to run Samza
>>>>>>>>> processors, and eliminate all direct dependences on YARN, Mesos,
>>>> etc.
>>>>>>>>> 2. Make a definitive call to support only Kafka as the stream
>>>>>> processing
>>>>>>>>> layer.
>>>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
>>>>>>>>> config
>>>>>>> systems,
>>>>>>>>> and simply use Kafka's instead.
>>>>>>>>> 
>>>>>>>>> This would fix all of the issues that I outlined above. It
>>>>>>>>> should
>>>>> also
>>>>>>>>> shrink the Samza code base pretty dramatically. Supporting only
>>>>>>>>> a standalone container will allow Samza to be executed on YARN
>>>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
>>>>>>>>> in-house
>>>>>>> deployment
>>>>>>>>> systems. This should make life a lot easier for new users.
>>>>>>>>> Imagine
>>>>>>> having
>>>>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
>>>>>> traffic
>>>>>>>>> will be pretty dramatic.
>>>>>>>>> 
>>>>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
>>>>> everyone
>>>>>>>>> that
>>>>>>>>> I'm aware of is using Samza with Kafka. We basically require it
>>>>>> already
>>>>>>> in
>>>>>>>>> order for most features to work. Those that are using other
>>>>>>>>> systems
>>>>>> are
>>>>>>>>> generally using it for ingest into Kafka (1), and then they do
>>>>>>>>> the processing on top. There is already discussion (
>>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>>>>> 767
>>>>>>>>> )
>>>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
>>>>>>>>> 
>>>>>>>>> Once we make the call to couple with Kafka, we can leverage a
>>>>>>>>> ton of
>>>>>>> their
>>>>>>>>> ecosystem. We no longer have to maintain our own config,
>>>>>>>>> metrics,
>>>>> etc.
>>>>>>> We
>>>>>>>>> can all share the same libraries, and make them better. This
>>>>>>>>> will
>>>>> also
>>>>>>>>> allow us to share the consumer/producer APIs, and will let us
>>>>> leverage
>>>>>>>>> their offset management and partition management, rather than
>>>>>>>>> having
>>>>>> our
>>>>>>>>> own. All of the coordinator stream code would go away, as would
>>>>>>>>> most
>>>>>> of
>>>>>>>>> the
>>>>>>>>> YARN AppMaster code. We'd probably have to push some partition
>>>>>>> management
>>>>>>>>> features into the Kafka broker, but they're already moving in
>>>>>>>>> that direction with the new consumer API. The features we have
>>>>>>>>> for
>>>>>> partition
>>>>>>>>> assignment aren't unique to Samza, and seem like they should be
>>>>>>>>> in
>>>>>> Kafka
>>>>>>>>> anyway. There will always be some niche usages which will
>>>>>>>>> require
>>>>>> extra
>>>>>>>>> care and hence full control over partition assignments much
>>>>>>>>> like the
>>>>>>> Kafka
>>>>>>>>> low level consumer api. These would continue to be supported.
>>>>>>>>> 
>>>>>>>>> These items will be good for the Samza community. They'll make
>>>>>>>>> Samza easier to use, and make it easier for developers to add
>>>>>>>>> new features.
>>>>>>>>> 
>>>>>>>>> Obviously this is a fairly large (and somewhat backwards
>>>>> incompatible
>>>>>>>>> change). If we choose to go this route, it's important that we
>>>>> openly
>>>>>>>>> communicate how we're going to provide a migration path from
>>>>>>>>> the
>>>>>>> existing
>>>>>>>>> APIs to the new ones (if we make incompatible changes). I think
>>>>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
>>>>>>>>> existing StreamTask implementations to continue running on the
>>>> new container.
>>>>>>> It's
>>>>>>>>> also important that we openly communicate about timing, and
>>>>>>>>> stages
>>>>> of
>>>>>>> the
>>>>>>>>> migration.
>>>>>>>>> 
>>>>>>>>> If you made it this far, I'm sure you have opinions. :) Please
>>>>>>>>> send
>>>>>> your
>>>>>>>>> thoughts and feedback.
>>>>>>>>> 
>>>>>>>>> Cheers,
>>>>>>>>> Chris
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> --
>>>>>> -- Guozhang
>>>>>> 
>>>>> 
>>>> 
>> 
>> 



Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@gmail.com>.
Hey Martin,

For the YARN/Mesos/etc decoupling I actually don't think it ties our hands
at all, all it does is refactor things. The division of responsibility is
that Samza core is responsible for task lifecycle, state, and partition
management (using the Kafka co-ordinator) but it is NOT responsible for
packaging, configuration deployment or execution of processes. The problem
of packaging and starting these processes is
framework/environment-specific. This leaves individual frameworks to be as
fancy or vanilla as they like. So you can get simple stateless support in
YARN, Mesos, etc using their off-the-shelf app framework (Slider, Marathon,
etc). These are well known by people and have nice UIs and a lot of
flexibility. I don't think they have node affinity as a built in option
(though I could be wrong). So if we want that we can either wait for them
to add it or do a custom framework to add that feature (as now). Obviously
if you manage things with old-school ops tools (puppet/chef/etc) you get
locality easily. The nice thing, though, is that all the samza "business
logic" around partition management and fault tolerance is in Samza core so
it is shared across frameworks and the framework specific bit is just
whether it is smart enough to try to get the same host when a job is
restarted.

 With respect to the Kafka-alignment, yeah I think the goal would be (a)
actually get better alignment in user experience, and (b) express this in
the naming and project branding. Specifically:
1. Website/docs, it would be nice for the "transformation" api to be
discoverable in the main Kafka docs--i.e. be able to explain when to use
the consumer and when to use the stream processing functionality and lead
people into that experience.
2. Align releases so if you get Kafkza 1.4.2 (or whatever) that has both
Kafka and the stream processing part and they actually work together.
3. Unify the programming experience so the client and Samza api share
config/monitoring/naming/packaging/etc.

I think sub-projects keep separate committers and can have a separate repo,
but I'm actually not really sure (I can't find a definition of a subproject
in Apache).

Basically at a high-level you want the experience to "feel" like a single
system, not to relatively independent things that are kind of awkwardly
glued together.

I think if we did that they having naming or branding like "kafka
streaming" or "kafka streams" or something like that would actually do a
good job of conveying what it is. I do that this would help adoption quite
a lot as it would correctly convey that using Kafka Streaming with Kafka is
a fairly seamless experience and Kafka is pretty heavily adopted at this
point.

Fwiw we actually considered this model originally when open sourcing Samza,
however at that time Kafka was relatively unknown and we decided not to do
it since we felt it would be limiting. From my point of view the three
things have changed (1) Kafka is now really heavily used for stream
processing, (2) we learned that abstracting out the stream well is
basically impossible, (3) we learned it is really hard to keep the two
things feeling like a single product.

-Jay


On Mon, Jul 6, 2015 at 3:37 AM, Martin Kleppmann <ma...@kleppmann.com>
wrote:

> Hi all,
>
> Lots of good thoughts here.
>
> I agree with the general philosophy of tying Samza more firmly to Kafka.
> After I spent a while looking at integrating other message brokers (e.g.
> Kinesis) with SystemConsumer, I came to the conclusion that SystemConsumer
> tacitly assumes a model so much like Kafka's that pretty much nobody but
> Kafka actually implements it. (Databus is perhaps an exception, but it
> isn't widely used outside of LinkedIn.) Thus, making Samza fully dependent
> on Kafka acknowledges that the system-independence was never as real as we
> perhaps made it out to be. The gains of code reuse are real.
>
> The idea of decoupling Samza from YARN has also always been appealing to
> me, for various reasons already mentioned in this thread. Although making
> Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems laudable, I am
> a little concerned that it will restrict us to a lowest common denominator.
> For example, would host affinity (SAMZA-617) still be possible? For jobs
> with large amounts of state, I think SAMZA-617 would be a big boon, since
> restoring state off the changelog on every single restart is painful, due
> to long recovery times. It would be a shame if the decoupling from YARN
> made host affinity impossible.
>
> Jay, a question about the proposed API for instantiating a job in code
> (rather than a properties file): when submitting a job to a cluster, is the
> idea that the instantiation code runs on a client somewhere, which then
> pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code run
> on each container that is part of the job (in which case, how does the job
> submission to the cluster work)?
>
> I agree with Garry that it doesn't feel right to make a 1.0 release with a
> plan for it to be immediately obsolete. So if this is going to happen, I
> think it would be more honest to stick with 0.* version numbers until the
> library-ified Samza has been implemented, is stable and widely used.
>
> Should the new Samza be a subproject of Kafka? There is precedent for
> tight coupling between different Apache projects (e.g. Curator and
> Zookeeper, or Slider and YARN), so I think remaining separate would be ok.
> Even if Samza is fully dependent on Kafka, there is enough substance in
> Samza that it warrants being a separate project. An argument in favour of
> merging would be if we think Kafka has a much stronger "brand presence"
> than Samza; I'm ambivalent on that one. If the Kafka project is willing to
> endorse Samza as the "official" way of doing stateful stream
> transformations, that would probably have much the same effect as
> re-branding Samza as "Kafka Stream Processors" or suchlike. Close
> collaboration between the two projects will be needed in any case.
>
> From a project management perspective, I guess the "new Samza" would have
> to be developed on a branch alongside ongoing maintenance of the current
> line of development? I think it would be important to continue supporting
> existing users, and provide a graceful migration path to the new version.
> Leaving the current versions unsupported and forcing people to rewrite
> their jobs would send a bad signal.
>
> Best,
> Martin
>
> On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey Garry,
> >
> > Yeah that's super frustrating. I'd be happy to chat more about this if
> > you'd be interested. I think Chris and I started with the idea of "what
> > would it take to make Samza a kick-ass ingestion tool" but ultimately we
> > kind of came around to the idea that ingestion and transformation had
> > pretty different needs and coupling the two made things hard.
> >
> > For what it's worth I think copycat (KIP-26) actually will do what you
> are
> > looking for.
> >
> > With regard to your point about slider, I don't necessarily disagree.
> But I
> > think getting good YARN support is quite doable and I think we can make
> > that work well. I think the issue this proposal solves is that
> technically
> > it is pretty hard to support multiple cluster management systems the way
> > things are now, you need to write an "app master" or "framework" for each
> > and they are all a little different so testing is really hard. In the
> > absence of this we have been stuck with just YARN which has fantastic
> > penetration in the Hadoopy part of the org, but zero penetration
> elsewhere.
> > Given the huge amount of work being put in to slider, marathon, aws
> > tooling, not to mention the umpteen related packaging technologies people
> > want to use (Docker, Kubernetes, various cloud-specific deploy tools,
> etc)
> > I really think it is important to get this right.
> >
> > -Jay
> >
> > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > g.turkington@improvedigital.com> wrote:
> >
> >> Hi all,
> >>
> >> I think the question below re does Samza become a sub-project of Kafka
> >> highlights the broader point around migration. Chris mentions Samza's
> >> maturity is heading towards a v1 release but I'm not sure it feels
> right to
> >> launch a v1 then immediately plan to deprecate most of it.
> >>
> >> From a selfish perspective I have some guys who have started working
> with
> >> Samza and building some new consumers/producers was next up. Sounds like
> >> that is absolutely not the direction to go. I need to look into the KIP
> in
> >> more detail but for me the attractiveness of adding new Samza
> >> consumer/producers -- even if yes all they were doing was really getting
> >> data into and out of Kafka --  was to avoid  having to worry about the
> >> lifecycle management of external clients. If there is a generic Kafka
> >> ingress/egress layer that I can plug a new connector into and have a
> lot of
> >> the heavy lifting re scale and reliability done for me then it gives me
> all
> >> the pushing new consumers/producers would. If not then it complicates my
> >> operational deployments.
> >>
> >> Which is similar to my other question with the proposal -- if we build a
> >> fully available/stand-alone Samza plus the requisite shims to integrate
> >> with Slider etc I suspect the former may be a lot more work than we
> think.
> >> We may make it much easier for a newcomer to get something running but
> >> having them step up and get a reliable production deployment may still
> >> dominate mailing list  traffic, if for different reasons than today.
> >>
> >> Don't get me wrong -- I'm comfortable with making the Samza dependency
> on
> >> Kafka much more explicit and I absolutely see the benefits  in the
> >> reduction of duplication and clashing terminologies/abstractions that
> >> Chris/Jay describe. Samza as a library would likely be a very nice tool
> to
> >> add to the Kafka ecosystem. I just have the concerns above re the
> >> operational side.
> >>
> >> Garry
> >>
> >> -----Original Message-----
> >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> >> Sent: 02 July 2015 12:56
> >> To: dev@samza.apache.org
> >> Subject: Re: Thoughts and obesrvations on Samza
> >>
> >> Very interesting thoughts.
> >> From outside, I have always perceived Samza as a computing layer over
> >> Kafka.
> >>
> >> The question, maybe a bit provocative, is "should Samza be a sub-project
> >> of Kafka then?"
> >> Or does it make sense to keep it as a separate project with a separate
> >> governance?
> >>
> >> Cheers,
> >>
> >> --
> >> Gianmarco
> >>
> >> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> >>
> >>> Overall, I agree to couple with Kafka more tightly. Because Samza de
> >>> facto is based on Kafka, and it should leverage what Kafka has. At the
> >>> same time, Kafka does not need to reinvent what Samza already has. I
> >>> also like the idea of separating the ingestion and transformation.
> >>>
> >>> But it is a little difficult for me to image how the Samza will look
> >> like.
> >>> And I feel Chris and Jay have a little difference in terms of how
> >>> Samza should look like.
> >>>
> >>> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
> >>> user's application code calls this client?
> >>>
> >>> 1. If we make Samza be a library of Kafka (like what the code shows),
> >>> how do we implement auto-balance and fault-tolerance? Are they taken
> >>> care by the Kafka broker or other mechanism, such as "Samza worker"
> >>> (just make up the name) ?
> >>>
> >>> 2. What about other features, such as auto-scaling, shared state,
> >>> monitoring?
> >>>
> >>>
> >>> *** If we have Samza standalone, (is this what Chris suggests?)
> >>>
> >>> 1. we still need to ingest data from Kakfa and produce to it. Then it
> >>> becomes the same as what Samza looks like now, except it does not rely
> >>> on Yarn anymore.
> >>>
> >>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
> >>> etc? Use Kafka code as the dependency?
> >>>
> >>>
> >>> Thanks,
> >>>
> >>> Fang, Yan
> >>> yanfang724@gmail.com
> >>>
> >>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >>>
> >>>> Read through the code example and it looks good to me. A few
> >>>> thoughts regarding deployment:
> >>>>
> >>>> Today Samza deploys as executable runnable like:
> >>>>
> >>>> deploy/samza/bin/run-job.sh --config-factory=...
> >> --config-path=file://...
> >>>>
> >>>> And this proposal advocate for deploying Samza more as embedded
> >>>> libraries in user application code (ignoring the terminology since
> >>>> it is not the
> >>> same
> >>>> as the prototype code):
> >>>>
> >>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
> >>>> Thread(task); thread.start();
> >>>>
> >>>> I think both of these deployment modes are important for different
> >>>> types
> >>> of
> >>>> users. That said, I think making Samza purely standalone is still
> >>>> sufficient for either runnable or library modes.
> >>>>
> >>>> Guozhang
> >>>>
> >>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> >>>>
> >>>>> Looks like gmail mangled the code example, it was supposed to look
> >>>>> like
> >>>>> this:
> >>>>>
> >>>>> Properties props = new Properties();
> >>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
> >>>>> config = new StreamingConfig(props);
> >>>>> config.subscribe("test-topic-1", "test-topic-2");
> >>>>> config.processor(ExampleStreamProcessor.class);
> >>>>> config.serialization(new StringSerializer(), new
> >>>>> StringDeserializer()); KafkaStreaming container = new
> >>>>> KafkaStreaming(config); container.run();
> >>>>>
> >>>>> -Jay
> >>>>>
> >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> >> wrote:
> >>>>>
> >>>>>> Hey guys,
> >>>>>>
> >>>>>> This came out of some conversations Chris and I were having
> >>>>>> around
> >>>>> whether
> >>>>>> it would make sense to use Samza as a kind of data ingestion
> >>> framework
> >>>>> for
> >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> >>>> combined
> >>>>>> with complaints around config and YARN and the discussion around
> >>>>>> how
> >>> to
> >>>>>> best do a standalone mode.
> >>>>>>
> >>>>>> So the thought experiment was, given that Samza was basically
> >>>>>> already totally Kafka specific, what if you just embraced that
> >>>>>> and turned it
> >>>> into
> >>>>>> something less like a heavyweight framework and more like a
> >>>>>> third
> >>> Kafka
> >>>>>> client--a kind of "producing consumer" with state management
> >>>> facilities.
> >>>>>> Basically a library. Instead of a complex stream processing
> >>>>>> framework
> >>>>> this
> >>>>>> would actually be a very simple thing, not much more complicated
> >>>>>> to
> >>> use
> >>>>> or
> >>>>>> operate than a Kafka consumer. As Chris said we thought about it
> >>>>>> a
> >>> lot
> >>>> of
> >>>>>> what Samza (and the other stream processing systems were doing)
> >>> seemed
> >>>>> like
> >>>>>> kind of a hangover from MapReduce.
> >>>>>>
> >>>>>> Of course you need to ingest/output data to and from the stream
> >>>>>> processing. But when we actually looked into how that would
> >>>>>> work,
> >>> Samza
> >>>>>> isn't really an ideal data ingestion framework for a bunch of
> >>> reasons.
> >>>> To
> >>>>>> really do that right you need a pretty different internal data
> >>>>>> model
> >>>> and
> >>>>>> set of apis. So what if you split them and had an api for Kafka
> >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> >>>>>> transformation (Samza).
> >>>>>>
> >>>>>> This would also allow really embracing the same terminology and
> >>>>>> conventions. One complaint about the current state is that the
> >>>>>> two
> >>>>> systems
> >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
> >>>>> different
> >>>>>> config and monitoring systems means you kind of have to learn
> >>>>>> Kafka's
> >>>>> way,
> >>>>>> then learn Samza's slightly different way, then kind of
> >>>>>> understand
> >>> how
> >>>>> they
> >>>>>> map to each other, which having walked a few people through this
> >>>>>> is surprisingly tricky for folks to get.
> >>>>>>
> >>>>>> Since I have been spending a lot of time on airplanes I hacked
> >>>>>> up an ernest but still somewhat incomplete prototype of what
> >>>>>> this would
> >>> look
> >>>>>> like. This is just unceremoniously dumped into Kafka as it
> >>>>>> required a
> >>>> few
> >>>>>> changes to the new consumer. Here is the code:
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >>> /apache/kafka/clients/streaming
> >>>>>>
> >>>>>> For the purpose of the prototype I just liberally renamed
> >>>>>> everything
> >>> to
> >>>>>> try to align it with Kafka with no regard for compatibility.
> >>>>>>
> >>>>>> To use this would be something like this:
> >>>>>> Properties props = new Properties();
> >>>>>> props.put("bootstrap.servers", "localhost:4242");
> >>>>>> StreamingConfig config = new
> >>> StreamingConfig(props);
> >>>>> config.subscribe("test-topic-1",
> >>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> >>>>> config.serialization(new
> >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> >>>> container =
> >>>>>> new KafkaStreaming(config); container.run();
> >>>>>>
> >>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
> >>>>>> is basically StreamTask.
> >>>>>>
> >>>>>> So rather than putting all the class names in a file and then
> >>>>>> having
> >>>> the
> >>>>>> job assembled by reflection, you just instantiate the container
> >>>>>> programmatically. Work is balanced over however many instances
> >>>>>> of
> >>> this
> >>>>> are
> >>>>>> alive at any time (i.e. if an instance dies, new tasks are added
> >>>>>> to
> >>> the
> >>>>>> existing containers without shutting them down).
> >>>>>>
> >>>>>> We would provide some glue for running this stuff in YARN via
> >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> >>>>>> but from the
> >>>> point
> >>>>> of
> >>>>>> view of these frameworks these stream processing jobs are just
> >>>> stateless
> >>>>>> services that can come and go and expand and contract at will.
> >>>>>> There
> >>> is
> >>>>> no
> >>>>>> more custom scheduler.
> >>>>>>
> >>>>>> Here are some relevant details:
> >>>>>>
> >>>>>>   1. It is only ~1300 lines of code, it would get larger if we
> >>>>>>   productionized but not vastly larger. We really do get a ton
> >>>>>> of
> >>>>> leverage
> >>>>>>   out of Kafka.
> >>>>>>   2. Partition management is fully delegated to the new consumer.
> >>> This
> >>>>>>   is nice since now any partition management strategy available
> >>>>>> to
> >>>> Kafka
> >>>>>>   consumer is also available to Samza (and vice versa) and with
> >>>>>> the
> >>>>> exact
> >>>>>>   same configs.
> >>>>>>   3. It supports state as well as state reuse
> >>>>>>
> >>>>>> Anyhow take a look, hopefully it is thought provoking.
> >>>>>>
> >>>>>> -Jay
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> >>>> criccomini@apache.org>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Hey all,
> >>>>>>>
> >>>>>>> I have had some discussions with Samza engineers at LinkedIn
> >>>>>>> and
> >>>>> Confluent
> >>>>>>> and we came up with a few observations and would like to
> >>>>>>> propose
> >>> some
> >>>>>>> changes.
> >>>>>>>
> >>>>>>> We've observed some things that I want to call out about
> >>>>>>> Samza's
> >>>> design,
> >>>>>>> and I'd like to propose some changes.
> >>>>>>>
> >>>>>>> * Samza is dependent upon a dynamic deployment system.
> >>>>>>> * Samza is too pluggable.
> >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> >>>>>>> APIs
> >>> are
> >>>>>>> trying to solve a lot of the same problems.
> >>>>>>>
> >>>>>>> All three of these issues are related, but I'll address them in
> >>> order.
> >>>>>>>
> >>>>>>> Deployment
> >>>>>>>
> >>>>>>> Samza strongly depends on the use of a dynamic deployment
> >>>>>>> scheduler
> >>>> such
> >>>>>>> as
> >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> >>>>>>> there
> >>>> would
> >>>>>>> be
> >>>>>>> one or two winners in this area, and we could support them, and
> >>>>>>> the
> >>>> rest
> >>>>>>> would go away. In reality, there are many variations.
> >>>>>>> Furthermore,
> >>>> many
> >>>>>>> people still prefer to just start their processors like normal
> >>>>>>> Java processes, and use traditional deployment scripts such as
> >>>>>>> Fabric,
> >>>> Chef,
> >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> >>>>>>> Samza start-up process really painful for first time users.
> >>>>>>>
> >>>>>>> Dynamic deployment as a requirement was also a bit of a
> >>>>>>> mis-fire
> >>>> because
> >>>>>>> of
> >>>>>>> a fundamental misunderstanding between the nature of batch jobs
> >>>>>>> and
> >>>>> stream
> >>>>>>> processing jobs. Early on, we made conscious effort to favor
> >>>>>>> the
> >>>> Hadoop
> >>>>>>> (Map/Reduce) way of doing things, since it worked and was well
> >>>>> understood.
> >>>>>>> One thing that we missed was that batch jobs have a definite
> >>>> beginning,
> >>>>>>> and
> >>>>>>> end, and stream processing jobs don't (usually). This leads to
> >>>>>>> a
> >>> much
> >>>>>>> simpler scheduling problem for stream processors. You basically
> >>>>>>> just
> >>>>> need
> >>>>>>> to find a place to start the processor, and start it. The way
> >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> >>>>>>> being "full". We always
> >>>> add
> >>>>>>> more machines. The problem with coupling Samza with a scheduler
> >>>>>>> is
> >>>> that
> >>>>>>> Samza (as a framework) now has to handle deployment. This pulls
> >>>>>>> in a
> >>>>> bunch
> >>>>>>> of things such as configuration distribution (config stream),
> >>>>>>> shell
> >>>>> scrips
> >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> >>>>>>>
> >>>>>>> Another reason for requiring dynamic deployment was to support
> >>>>>>> data locality. If you want to have locality, you need to put
> >>>>>>> your
> >>>> processors
> >>>>>>> close to the data they're processing. Upon further
> >>>>>>> investigation,
> >>>>> though,
> >>>>>>> this feature is not that beneficial. There is some good
> >>>>>>> discussion
> >>>> about
> >>>>>>> some problems with it on SAMZA-335. Again, we took the
> >>>>>>> Map/Reduce
> >>>> path,
> >>>>>>> but
> >>>>>>> there are some fundamental differences between HDFS and Kafka.
> >>>>>>> HDFS
> >>>> has
> >>>>>>> blocks, while Kafka has partitions. This leads to less
> >>>>>>> optimization potential with stream processors on top of Kafka.
> >>>>>>>
> >>>>>>> This feature is also used as a crutch. Samza doesn't have any
> >>>>>>> built
> >>> in
> >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> >>>>>>> deployment scheduling system to handle restarts when a
> >>>>>>> processor dies. This has
> >>>>> made
> >>>>>>> it very difficult to write a standalone Samza container
> >> (SAMZA-516).
> >>>>>>>
> >>>>>>> Pluggability
> >>>>>>>
> >>>>>>> In some cases pluggability is good, but I think that we've gone
> >>>>>>> too
> >>>> far
> >>>>>>> with it. Currently, Samza has:
> >>>>>>>
> >>>>>>> * Pluggable config.
> >>>>>>> * Pluggable metrics.
> >>>>>>> * Pluggable deployment systems.
> >>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
> >> etc).
> >>>>>>> * Pluggable serdes.
> >>>>>>> * Pluggable storage engines.
> >>>>>>> * Pluggable strategies for just about every component
> >>> (MessageChooser,
> >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> >>>>>>>
> >>>>>>> There's probably more that I've forgotten, as well. Some of
> >>>>>>> these
> >>> are
> >>>>>>> useful, but some have proven not to be. This all comes at a cost:
> >>>>>>> complexity. This complexity is making it harder for our users
> >>>>>>> to
> >>> pick
> >>>> up
> >>>>>>> and use Samza out of the box. It also makes it difficult for
> >>>>>>> Samza developers to reason about what the characteristics of
> >>>>>>> the container (since the characteristics change depending on
> >>>>>>> which plugins are use).
> >>>>>>>
> >>>>>>> The issues with pluggability are most visible in the System APIs.
> >>> What
> >>>>>>> Samza really requires to be functional is Kafka as its
> >>>>>>> transport
> >>>> layer.
> >>>>>>> But
> >>>>>>> we've conflated two unrelated use cases into one API:
> >>>>>>>
> >>>>>>> 1. Get data into/out of Kafka.
> >>>>>>> 2. Process the data in Kafka.
> >>>>>>>
> >>>>>>> The current System API supports both of these use cases. The
> >>>>>>> problem
> >>>> is,
> >>>>>>> we
> >>>>>>> actually want different features for each use case. By papering
> >>>>>>> over
> >>>>> these
> >>>>>>> two use cases, and providing a single API, we've introduced a
> >>>>>>> ton of
> >>>>> leaky
> >>>>>>> abstractions.
> >>>>>>>
> >>>>>>> For example, what we'd really like in (2) is to have
> >>>>>>> monotonically increasing longs for offsets (like Kafka). This
> >>>>>>> would be at odds
> >>> with
> >>>>> (1),
> >>>>>>> though, since different systems have different
> >>>>> SCNs/Offsets/UUIDs/vectors.
> >>>>>>> There was discussion both on the mailing list and the SQL JIRAs
> >>> about
> >>>>> the
> >>>>>>> need for this.
> >>>>>>>
> >>>>>>> The same thing holds true for replayability. Kafka allows us to
> >>> rewind
> >>>>>>> when
> >>>>>>> we have a failure. Many other systems don't. In some cases,
> >>>>>>> systems
> >>>>> return
> >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
> >>>>>>> they
> >>>> have
> >>>>> no
> >>>>>>> offsets.
> >>>>>>>
> >>>>>>> Partitioning is another example. Kafka supports partitioning,
> >>>>>>> but
> >>> many
> >>>>>>> systems don't. We model this by having a single partition for
> >>>>>>> those systems. Still, other systems model partitioning
> >> differently (e.g.
> >>>>>>> Kinesis).
> >>>>>>>
> >>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
> >>>>>>> system-agnostic way is almost impossible. As is modeling
> >>>>>>> metadata
> >>> for
> >>>>> the
> >>>>>>> system (replication factor, partitions, location, etc). The
> >>>>>>> list
> >>> goes
> >>>>> on.
> >>>>>>>
> >>>>>>> Duplicate work
> >>>>>>>
> >>>>>>> At the time that we began writing Samza, Kafka's consumer and
> >>> producer
> >>>>>>> APIs
> >>>>>>> had a relatively weak feature set. On the consumer-side, you
> >>>>>>> had two
> >>>>>>> options: use the high level consumer, or the simple consumer.
> >>>>>>> The
> >>>>> problem
> >>>>>>> with the high-level consumer was that it controlled your
> >>>>>>> offsets, partition assignments, and the order in which you
> >>>>>>> received messages. The
> >>> problem
> >>>>>>> with
> >>>>>>> the simple consumer is that it's not simple. It's basic. You
> >>>>>>> end up
> >>>>> having
> >>>>>>> to handle a lot of really low-level stuff that you shouldn't.
> >>>>>>> We
> >>>> spent a
> >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
> >>>>>>> also allows us to support some cool features:
> >>>>>>>
> >>>>>>> * Per-partition message ordering and prioritization.
> >>>>>>> * Tight control over partition assignment to support joins,
> >>>>>>> global
> >>>> state
> >>>>>>> (if we want to implement it :)), etc.
> >>>>>>> * Tight control over offset checkpointing.
> >>>>>>>
> >>>>>>> What we didn't realize at the time is that these features
> >>>>>>> should
> >>>>> actually
> >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> >>>> processors)
> >>>>>>> end up wanting to do things like joins and partition
> >>>>>>> assignment. The
> >>>>> Kafka
> >>>>>>> community has come to the same conclusion. They're adding a ton
> >>>>>>> of upgrades into their new Kafka consumer implementation. To a
> >>>>>>> large extent,
> >>> it's
> >>>>>>> duplicate work to what we've already done in Samza.
> >>>>>>>
> >>>>>>> On top of this, Kafka ended up taking a very similar approach
> >>>>>>> to
> >>>> Samza's
> >>>>>>> KafkaCheckpointManager implementation for handling offset
> >>>> checkpointing.
> >>>>>>> Like Samza, Kafka's new offset management feature stores offset
> >>>>>>> checkpoints in a topic, and allows you to fetch them from the
> >>>>>>> broker.
> >>>>>>>
> >>>>>>> A lot of this seems like a waste, since we could have shared
> >>>>>>> the
> >>> work
> >>>> if
> >>>>>>> it
> >>>>>>> had been done in Kafka from the get-go.
> >>>>>>>
> >>>>>>> Vision
> >>>>>>>
> >>>>>>> All of this leads me to a rather radical proposal. Samza is
> >>> relatively
> >>>>>>> stable at this point. I'd venture to say that we're near a 1.0
> >>>> release.
> >>>>>>> I'd
> >>>>>>> like to propose that we take what we've learned, and begin
> >>>>>>> thinking
> >>>>> about
> >>>>>>> Samza beyond 1.0. What would we change if we were starting from
> >>>> scratch?
> >>>>>>> My
> >>>>>>> proposal is to:
> >>>>>>>
> >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> >>>>>>> processors, and eliminate all direct dependences on YARN, Mesos,
> >> etc.
> >>>>>>> 2. Make a definitive call to support only Kafka as the stream
> >>>> processing
> >>>>>>> layer.
> >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> >>>>>>> config
> >>>>> systems,
> >>>>>>> and simply use Kafka's instead.
> >>>>>>>
> >>>>>>> This would fix all of the issues that I outlined above. It
> >>>>>>> should
> >>> also
> >>>>>>> shrink the Samza code base pretty dramatically. Supporting only
> >>>>>>> a standalone container will allow Samza to be executed on YARN
> >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> >>>>>>> in-house
> >>>>> deployment
> >>>>>>> systems. This should make life a lot easier for new users.
> >>>>>>> Imagine
> >>>>> having
> >>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
> >>>> traffic
> >>>>>>> will be pretty dramatic.
> >>>>>>>
> >>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
> >>> everyone
> >>>>>>> that
> >>>>>>> I'm aware of is using Samza with Kafka. We basically require it
> >>>> already
> >>>>> in
> >>>>>>> order for most features to work. Those that are using other
> >>>>>>> systems
> >>>> are
> >>>>>>> generally using it for ingest into Kafka (1), and then they do
> >>>>>>> the processing on top. There is already discussion (
> >>>>>>>
> >>>>>
> >>>>
> >>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >>> 767
> >>>>>>> )
> >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> >>>>>>>
> >>>>>>> Once we make the call to couple with Kafka, we can leverage a
> >>>>>>> ton of
> >>>>> their
> >>>>>>> ecosystem. We no longer have to maintain our own config,
> >>>>>>> metrics,
> >>> etc.
> >>>>> We
> >>>>>>> can all share the same libraries, and make them better. This
> >>>>>>> will
> >>> also
> >>>>>>> allow us to share the consumer/producer APIs, and will let us
> >>> leverage
> >>>>>>> their offset management and partition management, rather than
> >>>>>>> having
> >>>> our
> >>>>>>> own. All of the coordinator stream code would go away, as would
> >>>>>>> most
> >>>> of
> >>>>>>> the
> >>>>>>> YARN AppMaster code. We'd probably have to push some partition
> >>>>> management
> >>>>>>> features into the Kafka broker, but they're already moving in
> >>>>>>> that direction with the new consumer API. The features we have
> >>>>>>> for
> >>>> partition
> >>>>>>> assignment aren't unique to Samza, and seem like they should be
> >>>>>>> in
> >>>> Kafka
> >>>>>>> anyway. There will always be some niche usages which will
> >>>>>>> require
> >>>> extra
> >>>>>>> care and hence full control over partition assignments much
> >>>>>>> like the
> >>>>> Kafka
> >>>>>>> low level consumer api. These would continue to be supported.
> >>>>>>>
> >>>>>>> These items will be good for the Samza community. They'll make
> >>>>>>> Samza easier to use, and make it easier for developers to add
> >>>>>>> new features.
> >>>>>>>
> >>>>>>> Obviously this is a fairly large (and somewhat backwards
> >>> incompatible
> >>>>>>> change). If we choose to go this route, it's important that we
> >>> openly
> >>>>>>> communicate how we're going to provide a migration path from
> >>>>>>> the
> >>>>> existing
> >>>>>>> APIs to the new ones (if we make incompatible changes). I think
> >>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
> >>>>>>> existing StreamTask implementations to continue running on the
> >> new container.
> >>>>> It's
> >>>>>>> also important that we openly communicate about timing, and
> >>>>>>> stages
> >>> of
> >>>>> the
> >>>>>>> migration.
> >>>>>>>
> >>>>>>> If you made it this far, I'm sure you have opinions. :) Please
> >>>>>>> send
> >>>> your
> >>>>>>> thoughts and feedback.
> >>>>>>>
> >>>>>>> Cheers,
> >>>>>>> Chris
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>>
> >>>> --
> >>>> -- Guozhang
> >>>>
> >>>
> >>
>
>

Re: Thoughts and obesrvations on Samza

Posted by Martin Kleppmann <ma...@kleppmann.com>.
Hi all,

Lots of good thoughts here.

I agree with the general philosophy of tying Samza more firmly to Kafka. After I spent a while looking at integrating other message brokers (e.g. Kinesis) with SystemConsumer, I came to the conclusion that SystemConsumer tacitly assumes a model so much like Kafka's that pretty much nobody but Kafka actually implements it. (Databus is perhaps an exception, but it isn't widely used outside of LinkedIn.) Thus, making Samza fully dependent on Kafka acknowledges that the system-independence was never as real as we perhaps made it out to be. The gains of code reuse are real.

The idea of decoupling Samza from YARN has also always been appealing to me, for various reasons already mentioned in this thread. Although making Samza jobs deployable on anything (YARN/Mesos/AWS/etc) seems laudable, I am a little concerned that it will restrict us to a lowest common denominator. For example, would host affinity (SAMZA-617) still be possible? For jobs with large amounts of state, I think SAMZA-617 would be a big boon, since restoring state off the changelog on every single restart is painful, due to long recovery times. It would be a shame if the decoupling from YARN made host affinity impossible.

Jay, a question about the proposed API for instantiating a job in code (rather than a properties file): when submitting a job to a cluster, is the idea that the instantiation code runs on a client somewhere, which then pokes the necessary endpoints on YARN/Mesos/AWS/etc? Or does that code run on each container that is part of the job (in which case, how does the job submission to the cluster work)?

I agree with Garry that it doesn't feel right to make a 1.0 release with a plan for it to be immediately obsolete. So if this is going to happen, I think it would be more honest to stick with 0.* version numbers until the library-ified Samza has been implemented, is stable and widely used.

Should the new Samza be a subproject of Kafka? There is precedent for tight coupling between different Apache projects (e.g. Curator and Zookeeper, or Slider and YARN), so I think remaining separate would be ok. Even if Samza is fully dependent on Kafka, there is enough substance in Samza that it warrants being a separate project. An argument in favour of merging would be if we think Kafka has a much stronger "brand presence" than Samza; I'm ambivalent on that one. If the Kafka project is willing to endorse Samza as the "official" way of doing stateful stream transformations, that would probably have much the same effect as re-branding Samza as "Kafka Stream Processors" or suchlike. Close collaboration between the two projects will be needed in any case.

From a project management perspective, I guess the "new Samza" would have to be developed on a branch alongside ongoing maintenance of the current line of development? I think it would be important to continue supporting existing users, and provide a graceful migration path to the new version. Leaving the current versions unsupported and forcing people to rewrite their jobs would send a bad signal.

Best,
Martin

On 2 Jul 2015, at 16:59, Jay Kreps <ja...@confluent.io> wrote:

> Hey Garry,
> 
> Yeah that's super frustrating. I'd be happy to chat more about this if
> you'd be interested. I think Chris and I started with the idea of "what
> would it take to make Samza a kick-ass ingestion tool" but ultimately we
> kind of came around to the idea that ingestion and transformation had
> pretty different needs and coupling the two made things hard.
> 
> For what it's worth I think copycat (KIP-26) actually will do what you are
> looking for.
> 
> With regard to your point about slider, I don't necessarily disagree. But I
> think getting good YARN support is quite doable and I think we can make
> that work well. I think the issue this proposal solves is that technically
> it is pretty hard to support multiple cluster management systems the way
> things are now, you need to write an "app master" or "framework" for each
> and they are all a little different so testing is really hard. In the
> absence of this we have been stuck with just YARN which has fantastic
> penetration in the Hadoopy part of the org, but zero penetration elsewhere.
> Given the huge amount of work being put in to slider, marathon, aws
> tooling, not to mention the umpteen related packaging technologies people
> want to use (Docker, Kubernetes, various cloud-specific deploy tools, etc)
> I really think it is important to get this right.
> 
> -Jay
> 
> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> g.turkington@improvedigital.com> wrote:
> 
>> Hi all,
>> 
>> I think the question below re does Samza become a sub-project of Kafka
>> highlights the broader point around migration. Chris mentions Samza's
>> maturity is heading towards a v1 release but I'm not sure it feels right to
>> launch a v1 then immediately plan to deprecate most of it.
>> 
>> From a selfish perspective I have some guys who have started working with
>> Samza and building some new consumers/producers was next up. Sounds like
>> that is absolutely not the direction to go. I need to look into the KIP in
>> more detail but for me the attractiveness of adding new Samza
>> consumer/producers -- even if yes all they were doing was really getting
>> data into and out of Kafka --  was to avoid  having to worry about the
>> lifecycle management of external clients. If there is a generic Kafka
>> ingress/egress layer that I can plug a new connector into and have a lot of
>> the heavy lifting re scale and reliability done for me then it gives me all
>> the pushing new consumers/producers would. If not then it complicates my
>> operational deployments.
>> 
>> Which is similar to my other question with the proposal -- if we build a
>> fully available/stand-alone Samza plus the requisite shims to integrate
>> with Slider etc I suspect the former may be a lot more work than we think.
>> We may make it much easier for a newcomer to get something running but
>> having them step up and get a reliable production deployment may still
>> dominate mailing list  traffic, if for different reasons than today.
>> 
>> Don't get me wrong -- I'm comfortable with making the Samza dependency on
>> Kafka much more explicit and I absolutely see the benefits  in the
>> reduction of duplication and clashing terminologies/abstractions that
>> Chris/Jay describe. Samza as a library would likely be a very nice tool to
>> add to the Kafka ecosystem. I just have the concerns above re the
>> operational side.
>> 
>> Garry
>> 
>> -----Original Message-----
>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
>> Sent: 02 July 2015 12:56
>> To: dev@samza.apache.org
>> Subject: Re: Thoughts and obesrvations on Samza
>> 
>> Very interesting thoughts.
>> From outside, I have always perceived Samza as a computing layer over
>> Kafka.
>> 
>> The question, maybe a bit provocative, is "should Samza be a sub-project
>> of Kafka then?"
>> Or does it make sense to keep it as a separate project with a separate
>> governance?
>> 
>> Cheers,
>> 
>> --
>> Gianmarco
>> 
>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
>> 
>>> Overall, I agree to couple with Kafka more tightly. Because Samza de
>>> facto is based on Kafka, and it should leverage what Kafka has. At the
>>> same time, Kafka does not need to reinvent what Samza already has. I
>>> also like the idea of separating the ingestion and transformation.
>>> 
>>> But it is a little difficult for me to image how the Samza will look
>> like.
>>> And I feel Chris and Jay have a little difference in terms of how
>>> Samza should look like.
>>> 
>>> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
>>> user's application code calls this client?
>>> 
>>> 1. If we make Samza be a library of Kafka (like what the code shows),
>>> how do we implement auto-balance and fault-tolerance? Are they taken
>>> care by the Kafka broker or other mechanism, such as "Samza worker"
>>> (just make up the name) ?
>>> 
>>> 2. What about other features, such as auto-scaling, shared state,
>>> monitoring?
>>> 
>>> 
>>> *** If we have Samza standalone, (is this what Chris suggests?)
>>> 
>>> 1. we still need to ingest data from Kakfa and produce to it. Then it
>>> becomes the same as what Samza looks like now, except it does not rely
>>> on Yarn anymore.
>>> 
>>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
>>> etc? Use Kafka code as the dependency?
>>> 
>>> 
>>> Thanks,
>>> 
>>> Fang, Yan
>>> yanfang724@gmail.com
>>> 
>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
>> wrote:
>>> 
>>>> Read through the code example and it looks good to me. A few
>>>> thoughts regarding deployment:
>>>> 
>>>> Today Samza deploys as executable runnable like:
>>>> 
>>>> deploy/samza/bin/run-job.sh --config-factory=...
>> --config-path=file://...
>>>> 
>>>> And this proposal advocate for deploying Samza more as embedded
>>>> libraries in user application code (ignoring the terminology since
>>>> it is not the
>>> same
>>>> as the prototype code):
>>>> 
>>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
>>>> Thread(task); thread.start();
>>>> 
>>>> I think both of these deployment modes are important for different
>>>> types
>>> of
>>>> users. That said, I think making Samza purely standalone is still
>>>> sufficient for either runnable or library modes.
>>>> 
>>>> Guozhang
>>>> 
>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
>>>> 
>>>>> Looks like gmail mangled the code example, it was supposed to look
>>>>> like
>>>>> this:
>>>>> 
>>>>> Properties props = new Properties();
>>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
>>>>> config = new StreamingConfig(props);
>>>>> config.subscribe("test-topic-1", "test-topic-2");
>>>>> config.processor(ExampleStreamProcessor.class);
>>>>> config.serialization(new StringSerializer(), new
>>>>> StringDeserializer()); KafkaStreaming container = new
>>>>> KafkaStreaming(config); container.run();
>>>>> 
>>>>> -Jay
>>>>> 
>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
>> wrote:
>>>>> 
>>>>>> Hey guys,
>>>>>> 
>>>>>> This came out of some conversations Chris and I were having
>>>>>> around
>>>>> whether
>>>>>> it would make sense to use Samza as a kind of data ingestion
>>> framework
>>>>> for
>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
>>>> combined
>>>>>> with complaints around config and YARN and the discussion around
>>>>>> how
>>> to
>>>>>> best do a standalone mode.
>>>>>> 
>>>>>> So the thought experiment was, given that Samza was basically
>>>>>> already totally Kafka specific, what if you just embraced that
>>>>>> and turned it
>>>> into
>>>>>> something less like a heavyweight framework and more like a
>>>>>> third
>>> Kafka
>>>>>> client--a kind of "producing consumer" with state management
>>>> facilities.
>>>>>> Basically a library. Instead of a complex stream processing
>>>>>> framework
>>>>> this
>>>>>> would actually be a very simple thing, not much more complicated
>>>>>> to
>>> use
>>>>> or
>>>>>> operate than a Kafka consumer. As Chris said we thought about it
>>>>>> a
>>> lot
>>>> of
>>>>>> what Samza (and the other stream processing systems were doing)
>>> seemed
>>>>> like
>>>>>> kind of a hangover from MapReduce.
>>>>>> 
>>>>>> Of course you need to ingest/output data to and from the stream
>>>>>> processing. But when we actually looked into how that would
>>>>>> work,
>>> Samza
>>>>>> isn't really an ideal data ingestion framework for a bunch of
>>> reasons.
>>>> To
>>>>>> really do that right you need a pretty different internal data
>>>>>> model
>>>> and
>>>>>> set of apis. So what if you split them and had an api for Kafka
>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
>>>>>> transformation (Samza).
>>>>>> 
>>>>>> This would also allow really embracing the same terminology and
>>>>>> conventions. One complaint about the current state is that the
>>>>>> two
>>>>> systems
>>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
>>>>> different
>>>>>> config and monitoring systems means you kind of have to learn
>>>>>> Kafka's
>>>>> way,
>>>>>> then learn Samza's slightly different way, then kind of
>>>>>> understand
>>> how
>>>>> they
>>>>>> map to each other, which having walked a few people through this
>>>>>> is surprisingly tricky for folks to get.
>>>>>> 
>>>>>> Since I have been spending a lot of time on airplanes I hacked
>>>>>> up an ernest but still somewhat incomplete prototype of what
>>>>>> this would
>>> look
>>>>>> like. This is just unceremoniously dumped into Kafka as it
>>>>>> required a
>>>> few
>>>>>> changes to the new consumer. Here is the code:
>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>>> /apache/kafka/clients/streaming
>>>>>> 
>>>>>> For the purpose of the prototype I just liberally renamed
>>>>>> everything
>>> to
>>>>>> try to align it with Kafka with no regard for compatibility.
>>>>>> 
>>>>>> To use this would be something like this:
>>>>>> Properties props = new Properties();
>>>>>> props.put("bootstrap.servers", "localhost:4242");
>>>>>> StreamingConfig config = new
>>> StreamingConfig(props);
>>>>> config.subscribe("test-topic-1",
>>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
>>>>> config.serialization(new
>>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
>>>> container =
>>>>>> new KafkaStreaming(config); container.run();
>>>>>> 
>>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
>>>>>> is basically StreamTask.
>>>>>> 
>>>>>> So rather than putting all the class names in a file and then
>>>>>> having
>>>> the
>>>>>> job assembled by reflection, you just instantiate the container
>>>>>> programmatically. Work is balanced over however many instances
>>>>>> of
>>> this
>>>>> are
>>>>>> alive at any time (i.e. if an instance dies, new tasks are added
>>>>>> to
>>> the
>>>>>> existing containers without shutting them down).
>>>>>> 
>>>>>> We would provide some glue for running this stuff in YARN via
>>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
>>>>>> but from the
>>>> point
>>>>> of
>>>>>> view of these frameworks these stream processing jobs are just
>>>> stateless
>>>>>> services that can come and go and expand and contract at will.
>>>>>> There
>>> is
>>>>> no
>>>>>> more custom scheduler.
>>>>>> 
>>>>>> Here are some relevant details:
>>>>>> 
>>>>>>   1. It is only ~1300 lines of code, it would get larger if we
>>>>>>   productionized but not vastly larger. We really do get a ton
>>>>>> of
>>>>> leverage
>>>>>>   out of Kafka.
>>>>>>   2. Partition management is fully delegated to the new consumer.
>>> This
>>>>>>   is nice since now any partition management strategy available
>>>>>> to
>>>> Kafka
>>>>>>   consumer is also available to Samza (and vice versa) and with
>>>>>> the
>>>>> exact
>>>>>>   same configs.
>>>>>>   3. It supports state as well as state reuse
>>>>>> 
>>>>>> Anyhow take a look, hopefully it is thought provoking.
>>>>>> 
>>>>>> -Jay
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>>>> criccomini@apache.org>
>>>>>> wrote:
>>>>>> 
>>>>>>> Hey all,
>>>>>>> 
>>>>>>> I have had some discussions with Samza engineers at LinkedIn
>>>>>>> and
>>>>> Confluent
>>>>>>> and we came up with a few observations and would like to
>>>>>>> propose
>>> some
>>>>>>> changes.
>>>>>>> 
>>>>>>> We've observed some things that I want to call out about
>>>>>>> Samza's
>>>> design,
>>>>>>> and I'd like to propose some changes.
>>>>>>> 
>>>>>>> * Samza is dependent upon a dynamic deployment system.
>>>>>>> * Samza is too pluggable.
>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
>>>>>>> APIs
>>> are
>>>>>>> trying to solve a lot of the same problems.
>>>>>>> 
>>>>>>> All three of these issues are related, but I'll address them in
>>> order.
>>>>>>> 
>>>>>>> Deployment
>>>>>>> 
>>>>>>> Samza strongly depends on the use of a dynamic deployment
>>>>>>> scheduler
>>>> such
>>>>>>> as
>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
>>>>>>> there
>>>> would
>>>>>>> be
>>>>>>> one or two winners in this area, and we could support them, and
>>>>>>> the
>>>> rest
>>>>>>> would go away. In reality, there are many variations.
>>>>>>> Furthermore,
>>>> many
>>>>>>> people still prefer to just start their processors like normal
>>>>>>> Java processes, and use traditional deployment scripts such as
>>>>>>> Fabric,
>>>> Chef,
>>>>>>> Ansible, etc. Forcing a deployment system on users makes the
>>>>>>> Samza start-up process really painful for first time users.
>>>>>>> 
>>>>>>> Dynamic deployment as a requirement was also a bit of a
>>>>>>> mis-fire
>>>> because
>>>>>>> of
>>>>>>> a fundamental misunderstanding between the nature of batch jobs
>>>>>>> and
>>>>> stream
>>>>>>> processing jobs. Early on, we made conscious effort to favor
>>>>>>> the
>>>> Hadoop
>>>>>>> (Map/Reduce) way of doing things, since it worked and was well
>>>>> understood.
>>>>>>> One thing that we missed was that batch jobs have a definite
>>>> beginning,
>>>>>>> and
>>>>>>> end, and stream processing jobs don't (usually). This leads to
>>>>>>> a
>>> much
>>>>>>> simpler scheduling problem for stream processors. You basically
>>>>>>> just
>>>>> need
>>>>>>> to find a place to start the processor, and start it. The way
>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
>>>>>>> being "full". We always
>>>> add
>>>>>>> more machines. The problem with coupling Samza with a scheduler
>>>>>>> is
>>>> that
>>>>>>> Samza (as a framework) now has to handle deployment. This pulls
>>>>>>> in a
>>>>> bunch
>>>>>>> of things such as configuration distribution (config stream),
>>>>>>> shell
>>>>> scrips
>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
>>>>>>> 
>>>>>>> Another reason for requiring dynamic deployment was to support
>>>>>>> data locality. If you want to have locality, you need to put
>>>>>>> your
>>>> processors
>>>>>>> close to the data they're processing. Upon further
>>>>>>> investigation,
>>>>> though,
>>>>>>> this feature is not that beneficial. There is some good
>>>>>>> discussion
>>>> about
>>>>>>> some problems with it on SAMZA-335. Again, we took the
>>>>>>> Map/Reduce
>>>> path,
>>>>>>> but
>>>>>>> there are some fundamental differences between HDFS and Kafka.
>>>>>>> HDFS
>>>> has
>>>>>>> blocks, while Kafka has partitions. This leads to less
>>>>>>> optimization potential with stream processors on top of Kafka.
>>>>>>> 
>>>>>>> This feature is also used as a crutch. Samza doesn't have any
>>>>>>> built
>>> in
>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
>>>>>>> deployment scheduling system to handle restarts when a
>>>>>>> processor dies. This has
>>>>> made
>>>>>>> it very difficult to write a standalone Samza container
>> (SAMZA-516).
>>>>>>> 
>>>>>>> Pluggability
>>>>>>> 
>>>>>>> In some cases pluggability is good, but I think that we've gone
>>>>>>> too
>>>> far
>>>>>>> with it. Currently, Samza has:
>>>>>>> 
>>>>>>> * Pluggable config.
>>>>>>> * Pluggable metrics.
>>>>>>> * Pluggable deployment systems.
>>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
>> etc).
>>>>>>> * Pluggable serdes.
>>>>>>> * Pluggable storage engines.
>>>>>>> * Pluggable strategies for just about every component
>>> (MessageChooser,
>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>>>>>>> 
>>>>>>> There's probably more that I've forgotten, as well. Some of
>>>>>>> these
>>> are
>>>>>>> useful, but some have proven not to be. This all comes at a cost:
>>>>>>> complexity. This complexity is making it harder for our users
>>>>>>> to
>>> pick
>>>> up
>>>>>>> and use Samza out of the box. It also makes it difficult for
>>>>>>> Samza developers to reason about what the characteristics of
>>>>>>> the container (since the characteristics change depending on
>>>>>>> which plugins are use).
>>>>>>> 
>>>>>>> The issues with pluggability are most visible in the System APIs.
>>> What
>>>>>>> Samza really requires to be functional is Kafka as its
>>>>>>> transport
>>>> layer.
>>>>>>> But
>>>>>>> we've conflated two unrelated use cases into one API:
>>>>>>> 
>>>>>>> 1. Get data into/out of Kafka.
>>>>>>> 2. Process the data in Kafka.
>>>>>>> 
>>>>>>> The current System API supports both of these use cases. The
>>>>>>> problem
>>>> is,
>>>>>>> we
>>>>>>> actually want different features for each use case. By papering
>>>>>>> over
>>>>> these
>>>>>>> two use cases, and providing a single API, we've introduced a
>>>>>>> ton of
>>>>> leaky
>>>>>>> abstractions.
>>>>>>> 
>>>>>>> For example, what we'd really like in (2) is to have
>>>>>>> monotonically increasing longs for offsets (like Kafka). This
>>>>>>> would be at odds
>>> with
>>>>> (1),
>>>>>>> though, since different systems have different
>>>>> SCNs/Offsets/UUIDs/vectors.
>>>>>>> There was discussion both on the mailing list and the SQL JIRAs
>>> about
>>>>> the
>>>>>>> need for this.
>>>>>>> 
>>>>>>> The same thing holds true for replayability. Kafka allows us to
>>> rewind
>>>>>>> when
>>>>>>> we have a failure. Many other systems don't. In some cases,
>>>>>>> systems
>>>>> return
>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
>>>>>>> they
>>>> have
>>>>> no
>>>>>>> offsets.
>>>>>>> 
>>>>>>> Partitioning is another example. Kafka supports partitioning,
>>>>>>> but
>>> many
>>>>>>> systems don't. We model this by having a single partition for
>>>>>>> those systems. Still, other systems model partitioning
>> differently (e.g.
>>>>>>> Kinesis).
>>>>>>> 
>>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
>>>>>>> system-agnostic way is almost impossible. As is modeling
>>>>>>> metadata
>>> for
>>>>> the
>>>>>>> system (replication factor, partitions, location, etc). The
>>>>>>> list
>>> goes
>>>>> on.
>>>>>>> 
>>>>>>> Duplicate work
>>>>>>> 
>>>>>>> At the time that we began writing Samza, Kafka's consumer and
>>> producer
>>>>>>> APIs
>>>>>>> had a relatively weak feature set. On the consumer-side, you
>>>>>>> had two
>>>>>>> options: use the high level consumer, or the simple consumer.
>>>>>>> The
>>>>> problem
>>>>>>> with the high-level consumer was that it controlled your
>>>>>>> offsets, partition assignments, and the order in which you
>>>>>>> received messages. The
>>> problem
>>>>>>> with
>>>>>>> the simple consumer is that it's not simple. It's basic. You
>>>>>>> end up
>>>>> having
>>>>>>> to handle a lot of really low-level stuff that you shouldn't.
>>>>>>> We
>>>> spent a
>>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
>>>>>>> also allows us to support some cool features:
>>>>>>> 
>>>>>>> * Per-partition message ordering and prioritization.
>>>>>>> * Tight control over partition assignment to support joins,
>>>>>>> global
>>>> state
>>>>>>> (if we want to implement it :)), etc.
>>>>>>> * Tight control over offset checkpointing.
>>>>>>> 
>>>>>>> What we didn't realize at the time is that these features
>>>>>>> should
>>>>> actually
>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
>>>> processors)
>>>>>>> end up wanting to do things like joins and partition
>>>>>>> assignment. The
>>>>> Kafka
>>>>>>> community has come to the same conclusion. They're adding a ton
>>>>>>> of upgrades into their new Kafka consumer implementation. To a
>>>>>>> large extent,
>>> it's
>>>>>>> duplicate work to what we've already done in Samza.
>>>>>>> 
>>>>>>> On top of this, Kafka ended up taking a very similar approach
>>>>>>> to
>>>> Samza's
>>>>>>> KafkaCheckpointManager implementation for handling offset
>>>> checkpointing.
>>>>>>> Like Samza, Kafka's new offset management feature stores offset
>>>>>>> checkpoints in a topic, and allows you to fetch them from the
>>>>>>> broker.
>>>>>>> 
>>>>>>> A lot of this seems like a waste, since we could have shared
>>>>>>> the
>>> work
>>>> if
>>>>>>> it
>>>>>>> had been done in Kafka from the get-go.
>>>>>>> 
>>>>>>> Vision
>>>>>>> 
>>>>>>> All of this leads me to a rather radical proposal. Samza is
>>> relatively
>>>>>>> stable at this point. I'd venture to say that we're near a 1.0
>>>> release.
>>>>>>> I'd
>>>>>>> like to propose that we take what we've learned, and begin
>>>>>>> thinking
>>>>> about
>>>>>>> Samza beyond 1.0. What would we change if we were starting from
>>>> scratch?
>>>>>>> My
>>>>>>> proposal is to:
>>>>>>> 
>>>>>>> 1. Make Samza standalone the *only* way to run Samza
>>>>>>> processors, and eliminate all direct dependences on YARN, Mesos,
>> etc.
>>>>>>> 2. Make a definitive call to support only Kafka as the stream
>>>> processing
>>>>>>> layer.
>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
>>>>>>> config
>>>>> systems,
>>>>>>> and simply use Kafka's instead.
>>>>>>> 
>>>>>>> This would fix all of the issues that I outlined above. It
>>>>>>> should
>>> also
>>>>>>> shrink the Samza code base pretty dramatically. Supporting only
>>>>>>> a standalone container will allow Samza to be executed on YARN
>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
>>>>>>> in-house
>>>>> deployment
>>>>>>> systems. This should make life a lot easier for new users.
>>>>>>> Imagine
>>>>> having
>>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
>>>> traffic
>>>>>>> will be pretty dramatic.
>>>>>>> 
>>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
>>> everyone
>>>>>>> that
>>>>>>> I'm aware of is using Samza with Kafka. We basically require it
>>>> already
>>>>> in
>>>>>>> order for most features to work. Those that are using other
>>>>>>> systems
>>>> are
>>>>>>> generally using it for ingest into Kafka (1), and then they do
>>>>>>> the processing on top. There is already discussion (
>>>>>>> 
>>>>> 
>>>> 
>>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>>> 767
>>>>>>> )
>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
>>>>>>> 
>>>>>>> Once we make the call to couple with Kafka, we can leverage a
>>>>>>> ton of
>>>>> their
>>>>>>> ecosystem. We no longer have to maintain our own config,
>>>>>>> metrics,
>>> etc.
>>>>> We
>>>>>>> can all share the same libraries, and make them better. This
>>>>>>> will
>>> also
>>>>>>> allow us to share the consumer/producer APIs, and will let us
>>> leverage
>>>>>>> their offset management and partition management, rather than
>>>>>>> having
>>>> our
>>>>>>> own. All of the coordinator stream code would go away, as would
>>>>>>> most
>>>> of
>>>>>>> the
>>>>>>> YARN AppMaster code. We'd probably have to push some partition
>>>>> management
>>>>>>> features into the Kafka broker, but they're already moving in
>>>>>>> that direction with the new consumer API. The features we have
>>>>>>> for
>>>> partition
>>>>>>> assignment aren't unique to Samza, and seem like they should be
>>>>>>> in
>>>> Kafka
>>>>>>> anyway. There will always be some niche usages which will
>>>>>>> require
>>>> extra
>>>>>>> care and hence full control over partition assignments much
>>>>>>> like the
>>>>> Kafka
>>>>>>> low level consumer api. These would continue to be supported.
>>>>>>> 
>>>>>>> These items will be good for the Samza community. They'll make
>>>>>>> Samza easier to use, and make it easier for developers to add
>>>>>>> new features.
>>>>>>> 
>>>>>>> Obviously this is a fairly large (and somewhat backwards
>>> incompatible
>>>>>>> change). If we choose to go this route, it's important that we
>>> openly
>>>>>>> communicate how we're going to provide a migration path from
>>>>>>> the
>>>>> existing
>>>>>>> APIs to the new ones (if we make incompatible changes). I think
>>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
>>>>>>> existing StreamTask implementations to continue running on the
>> new container.
>>>>> It's
>>>>>>> also important that we openly communicate about timing, and
>>>>>>> stages
>>> of
>>>>> the
>>>>>>> migration.
>>>>>>> 
>>>>>>> If you made it this far, I'm sure you have opinions. :) Please
>>>>>>> send
>>>> your
>>>>>>> thoughts and feedback.
>>>>>>> 
>>>>>>> Cheers,
>>>>>>> Chris
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>> 
>>>> 
>>>> 
>>>> 
>>>> --
>>>> -- Guozhang
>>>> 
>>> 
>> 


Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@gmail.com>.
Hey Yi,

Yeah let me try to be more concrete. Any partition assignment management
requires two things
  G - a group of consumer instances
  f  - a function f(G) that yields the mapping of partitions to consumer
instances
The hard thing is to determine G, this is what requires the heartbeats,
coordinator, etc. So I think we agree that G should be maintained by the
kafka co-ordinator, and I think we all agree that in any design f has to be
pluggable.

Note that if f is deterministic that you can actually do this two ways:
1. Original Proposal: the coordinator determines G and gives back f(G) to
the client
2. Current Proposal: the coordinator determines G and gives back G to the
clients each of whom compute f(G) to determine their own assignment.

As you say the current proposal you link to is (2), but given the needs
arguably we should consider (1). Maintaining G is in any case the really
hard thing, and that will be done in Kafka in either proposal it is just a
question of where the partition assignment function f is plugged in (client
vs server side).

-Jay

On Thu, Jul 2, 2015 at 4:44 PM, Yi Pan <ni...@gmail.com> wrote:

> @Jay,
>
> {quote}
> I think it may be possible to rework the assignment feature in the consumer
> to make this always be a client-side concern so that Samza, the Kafka
> consumer, and Copycat can all use the same facility.
> {quote}
> Thanks! I like that idea.
>
> {quote}
> So it may make sense to revist this, I don't think it is necessarily a
> massive
> change and would give more flexibility for the variety of cases.
> {quote}
> I totally agree.
>
> P.S. just for my education,
> {quote}
> The original design for the kafka coordinator was that the coordinator
> would
> just coordinate *group* membership and the actual assignment of partitions
> to members of the group would be done client side.
> {quote}
> Please correct me if I am wrong. Is the link here still valid:
>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Detailed+Consumer+Coordinator+Design
> ?
> If yes, I thought that the assignment is done by the broker as in
> KAFKA-167? Maybe we can discuss and clarify this in person.
>
> Thanks a lot!
>
> -Yi
>
> On Thu, Jul 2, 2015 at 3:52 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Yeah, hey Yi, I get what you are saying now. At the risk of getting into
> > the weeds a bit you are exactly right, a similar thing is needed for
> > copycat/kip-26. At the risk of getting a bit into the weeds, I think it
> may
> > be possible to rework the assignment feature in the consumer to make this
> > always be a client-side concern so that Samza, the Kafka consumer, and
> > Copycat can all use the same facility.
> >
> > The original design for the kafka coordinator was that the coordinator
> > would just coordinate *group* membership and the actual assignment of
> > partitions to members of the group would be done client side. The
> advantage
> > of this was that it was more general the disadvantage was that the server
> > couldn't really check or monitor the partition assignment. Since we
> didn't
> > have any other use case for generic group management we went with the
> more
> > specific partition assignment.
> >
> > However a few things have changed since that original design:
> > 1. We now have the additional use cases of copycat and Samza
> > 2. We now realize that the assignment strategies don't actually
> necessarily
> > ensure each partition is assigned to only one consumer--there are really
> > valid use cases for broadcast or multiple replica assignment schemes--so
> we
> > can't actually make the a hard assertion on the server.
> >
> > So it may make sense to revist this, I don't think it is necessarily a
> > massive change and would give more flexibility for the variety of cases.
> >
> > -Jay
> >
> > On Thu, Jul 2, 2015 at 3:38 PM, Yi Pan <ni...@gmail.com> wrote:
> >
> > > @Guozhang, yes, that's what I meant. From Kafka consumers' point of
> view,
> > > it pretty much boils down to answer the following question:
> > > 1. For Kafka consumer in each container (i.e. a Samza worker), which
> > topic
> > > partitions it should consume.
> > > Samza's current StreamTask model still makes sense to me and the
> > > partition-to-task mapping is the debatable point that whether that
> should
> > > be in Kafka or stays in a separate module. As we discussed earlier,
> some
> > > simple partition-to-task mapping maybe expressed as co-partition
> > > distribution among different topics in Kafka (forgive me if I had make
> > > mistakes here since I am not 100% sure about how partition distribution
> > > policies work in Kafka). However, more complex and application-specific
> > > partition-to-task mapping would need to stay outside of Kafka. One
> > example
> > > is the discussion on SQL tasks:
> > >
> > >
> >
> https://issues.apache.org/jira/browse/SAMZA-516?focusedCommentId=14288685&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14288685
> > >
> > > On Thu, Jul 2, 2015 at 2:47 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Since the resource scheduling systems like YARN / Mesos only gives
> > Samza
> > > a
> > > > couple of resource units (or "containers") to run processes, while
> > Samza
> > > > still needs to handle task assignment / scheduling like which tasks
> > > should
> > > > be allocated to which containers that consume from which partitions,
> > > etc. I
> > > > think this is want Yi meant for "partition management"?
> > > >
> > > > On Thu, Jul 2, 2015 at 2:35 PM, Yi Pan <ni...@gmail.com> wrote:
> > > >
> > > > > @Jay, yes, the current function in the JobCoordinator is just
> > partition
> > > > > management. Maybe we should just call it PartitionManager to make
> it
> > > > > explicit.
> > > > >
> > > > > -Yi
> > > > >
> > > > > On Thu, Jul 2, 2015 at 2:24 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > > > >
> > > > > > Hey Yi,
> > > > > >
> > > > > > What does the JobCoordinator do? YARN/Mesos/etc would be doing
> the
> > > > actual
> > > > > > resource assignment, process restart, etc, right? Is the
> additional
> > > > value
> > > > > > add of the JobCoordinator just partition management?
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > > On Thu, Jul 2, 2015 at 11:32 AM, Yi Pan <ni...@gmail.com>
> > wrote:
> > > > > >
> > > > > > > Hi, all,
> > > > > > >
> > > > > > >
> > > > > > > Thanks Chris for sending out this proposal and Jay for sharing
> > the
> > > > > > > extremely illustrative prototype code.
> > > > > > >
> > > > > > >
> > > > > > > I have been thinking it over many times and want to list out my
> > > > > personal
> > > > > > > opinions below:
> > > > > > >
> > > > > > > 1. Generally, I agree with most of the people here on the
> mailing
> > > > list
> > > > > on
> > > > > > > two points:
> > > > > > >
> > > > > > >    a. Deeper integration w/ Kafka is great. No more confusing
> > > mapping
> > > > > > from
> > > > > > > SystemStreamPartition to TopicPartition etc.
> > > > > > >
> > > > > > >    b. Separation the ingestion vs transformation greatly
> simplify
> > > the
> > > > > > > systems APIs
> > > > > > >
> > > > > > > Having the above two changes would allow us to remove many
> > > > unnecessary
> > > > > > > complexities introduced by those pluggable interfaces Chris’
> > > pointed
> > > > > out,
> > > > > > > e.g. pluggable streaming systems and serde.
> > > > > > >
> > > > > > >
> > > > > > > To recall one of Chris’s statement on difficulties in dynamic
> > > > > > deployment, I
> > > > > > > believe that the difficulties are mainly the result of
> > > tight-coupling
> > > > > of
> > > > > > > partition assignment vs the container deployment in the current
> > > > system.
> > > > > > The
> > > > > > > current container deployment requires a pre-defined partition
> > > > > assignment
> > > > > > > strategy coupled together w/ the deployment configuration
> before
> > we
> > > > can
> > > > > > > submit to YARN and start the Samza container, which makes the
> > > > launching
> > > > > > > process super long. Also, fault-tolerance and the embedded
> > > > > JobCoordinator
> > > > > > > code in YARN AppMaster is another way of  making dynamic
> > deployment
> > > > > more
> > > > > > > complex and difficult.
> > > > > > >
> > > > > > >
> > > > > > > First, borrowing Yan’s term, let’s call the Samza standalone
> > > process
> > > > a
> > > > > > > Samza worker. Here is what I have been thinking:
> > > > > > >
> > > > > > > 1. Separate the execution framework from partition
> > assignment/load
> > > > > > > balancing:
> > > > > > >
> > > > > > >     a. a Samza worker should be launched by execution framework
> > > that
> > > > > only
> > > > > > > deals w/ process placement to available nodes. The execution
> > > > framework
> > > > > > now
> > > > > > > should only deal w/ how many such processes are needed, where
> to
> > > put
> > > > > > them,
> > > > > > > and how to keep them alive.
> > > > > > >
> > > > > > >     b. Partition assignment/load balancing can be a pluggable
> > > > interface
> > > > > > in
> > > > > > > Samza that allows the Samza workers to ask for partition
> > > assignments.
> > > > > > Let’s
> > > > > > > borrow the name JobCoordinator for now. To allow
> fault-tolerance
> > in
> > > > > case
> > > > > > of
> > > > > > > failure, the partition assignments to workers need to be
> dynamic.
> > > > > Hence,
> > > > > > > the abstract interface would be much like what Jay’s code
> > > illustrate:
> > > > > > > get()/onAssigned()/onRevoke(). The implementation of the
> > partition
> > > > > > > assignment can be either:
> > > > > > >
> > > > > > >         a) completely rely on Kafka.
> > > > > > >
> > > > > > >         b) explicit partition assignment via JobCoordinator.
> > > Chris’s
> > > > > work
> > > > > > > in SAMZA-516 can be easily incorporated here. The use case in
> > > > SAMZA-41
> > > > > > that
> > > > > > > runs Samza ProcessJob w/ static partition assignment can be
> > > > implemented
> > > > > > of
> > > > > > > JobCoordinator via any home-grown implementation of distributed
> > > > > > > coordinator. All the work we did in LinkedIn to support dynamic
> > > > > partition
> > > > > > > assignment and host-affinity SAMZA-617 can be nicely reused as
> an
> > > > > > > implementation of JobCoordinator.
> > > > > > >
> > > > > > >
> > > > > > > When we did the above work, I can see three usage patterns in
> > > Samza:
> > > > > > >
> > > > > > >    a. Samza as a library: Samza has a set of libraries to
> provide
> > > > > stream
> > > > > > > processing, just like a third Kafka client (as illustrated in
> > Jay’s
> > > > > > > example). The execution/deployment is totally controlled by the
> > > > > > application
> > > > > > > and the partition coordination is done via Kafka
> > > > > > >
> > > > > > >    b. Samza as a process: Samza runs as a standalone process.
> > There
> > > > may
> > > > > > not
> > > > > > > be a execution framework to launch and deploy Samza processes.
> > The
> > > > > > > partition assignment is pluggable JobCoordinator.
> > > > > > >
> > > > > > >    c. Samza as a service: Samza runs as a collection of
> > processes.
> > > > > There
> > > > > > > will be an execution framework to allocate resource, launch and
> > > > deploy
> > > > > > > Samza workers and keep them alive. The same pluggable
> > > JobCoordinator
> > > > is
> > > > > > > desirable here as well.
> > > > > > >
> > > > > > >
> > > > > > > Lastly, I would argue that CopyCat in KIP-26 should probably
> > follow
> > > > the
> > > > > > > same model. Hence, in Samza as a service model as in LinkedIn,
> we
> > > can
> > > > > use
> > > > > > > the same fault tolerance execution framework to run CopyCat and
> > > Samza
> > > > > w/o
> > > > > > > the need to operate two service platforms, which should address
> > > > > Sriram’s
> > > > > > > comment in the email thread.
> > > > > > >
> > > > > > >
> > > > > > > Hope the above makes sense. Thanks all!
> > > > > > >
> > > > > > >
> > > > > > > -Yi
> > > > > > >
> > > > > > > On Thu, Jul 2, 2015 at 9:53 AM, Sriram <sr...@gmail.com>
> > > wrote:
> > > > > > >
> > > > > > > > One thing that is worth exploring is to have a transformation
> > and
> > > > > > > > ingestion library in Kafka but use the same framework for
> fault
> > > > > > > tolerance,
> > > > > > > > resource isolation and management. The biggest difference I
> see
> > > in
> > > > > > these
> > > > > > > > two use cases is the API and data model.
> > > > > > > >
> > > > > > > >
> > > > > > > > > On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io>
> > > wrote:
> > > > > > > > >
> > > > > > > > > Hey Garry,
> > > > > > > > >
> > > > > > > > > Yeah that's super frustrating. I'd be happy to chat more
> > about
> > > > this
> > > > > > if
> > > > > > > > > you'd be interested. I think Chris and I started with the
> > idea
> > > of
> > > > > > "what
> > > > > > > > > would it take to make Samza a kick-ass ingestion tool" but
> > > > > ultimately
> > > > > > > we
> > > > > > > > > kind of came around to the idea that ingestion and
> > > transformation
> > > > > had
> > > > > > > > > pretty different needs and coupling the two made things
> hard.
> > > > > > > > >
> > > > > > > > > For what it's worth I think copycat (KIP-26) actually will
> do
> > > > what
> > > > > > you
> > > > > > > > are
> > > > > > > > > looking for.
> > > > > > > > >
> > > > > > > > > With regard to your point about slider, I don't necessarily
> > > > > disagree.
> > > > > > > > But I
> > > > > > > > > think getting good YARN support is quite doable and I think
> > we
> > > > can
> > > > > > make
> > > > > > > > > that work well. I think the issue this proposal solves is
> > that
> > > > > > > > technically
> > > > > > > > > it is pretty hard to support multiple cluster management
> > > systems
> > > > > the
> > > > > > > way
> > > > > > > > > things are now, you need to write an "app master" or
> > > "framework"
> > > > > for
> > > > > > > each
> > > > > > > > > and they are all a little different so testing is really
> > hard.
> > > In
> > > > > the
> > > > > > > > > absence of this we have been stuck with just YARN which has
> > > > > fantastic
> > > > > > > > > penetration in the Hadoopy part of the org, but zero
> > > penetration
> > > > > > > > elsewhere.
> > > > > > > > > Given the huge amount of work being put in to slider,
> > marathon,
> > > > aws
> > > > > > > > > tooling, not to mention the umpteen related packaging
> > > > technologies
> > > > > > > people
> > > > > > > > > want to use (Docker, Kubernetes, various cloud-specific
> > deploy
> > > > > tools,
> > > > > > > > etc)
> > > > > > > > > I really think it is important to get this right.
> > > > > > > > >
> > > > > > > > > -Jay
> > > > > > > > >
> > > > > > > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > > > > > > > g.turkington@improvedigital.com> wrote:
> > > > > > > > >
> > > > > > > > >> Hi all,
> > > > > > > > >>
> > > > > > > > >> I think the question below re does Samza become a
> > sub-project
> > > of
> > > > > > Kafka
> > > > > > > > >> highlights the broader point around migration. Chris
> > mentions
> > > > > > Samza's
> > > > > > > > >> maturity is heading towards a v1 release but I'm not sure
> it
> > > > feels
> > > > > > > > right to
> > > > > > > > >> launch a v1 then immediately plan to deprecate most of it.
> > > > > > > > >>
> > > > > > > > >> From a selfish perspective I have some guys who have
> started
> > > > > working
> > > > > > > > with
> > > > > > > > >> Samza and building some new consumers/producers was next
> up.
> > > > > Sounds
> > > > > > > like
> > > > > > > > >> that is absolutely not the direction to go. I need to look
> > > into
> > > > > the
> > > > > > > KIP
> > > > > > > > in
> > > > > > > > >> more detail but for me the attractiveness of adding new
> > Samza
> > > > > > > > >> consumer/producers -- even if yes all they were doing was
> > > really
> > > > > > > getting
> > > > > > > > >> data into and out of Kafka --  was to avoid  having to
> worry
> > > > about
> > > > > > the
> > > > > > > > >> lifecycle management of external clients. If there is a
> > > generic
> > > > > > Kafka
> > > > > > > > >> ingress/egress layer that I can plug a new connector into
> > and
> > > > > have a
> > > > > > > > lot of
> > > > > > > > >> the heavy lifting re scale and reliability done for me
> then
> > it
> > > > > gives
> > > > > > > me
> > > > > > > > all
> > > > > > > > >> the pushing new consumers/producers would. If not then it
> > > > > > complicates
> > > > > > > my
> > > > > > > > >> operational deployments.
> > > > > > > > >>
> > > > > > > > >> Which is similar to my other question with the proposal --
> > if
> > > we
> > > > > > > build a
> > > > > > > > >> fully available/stand-alone Samza plus the requisite shims
> > to
> > > > > > > integrate
> > > > > > > > >> with Slider etc I suspect the former may be a lot more
> work
> > > than
> > > > > we
> > > > > > > > think.
> > > > > > > > >> We may make it much easier for a newcomer to get something
> > > > running
> > > > > > but
> > > > > > > > >> having them step up and get a reliable production
> deployment
> > > may
> > > > > > still
> > > > > > > > >> dominate mailing list  traffic, if for different reasons
> > than
> > > > > today.
> > > > > > > > >>
> > > > > > > > >> Don't get me wrong -- I'm comfortable with making the
> Samza
> > > > > > dependency
> > > > > > > > on
> > > > > > > > >> Kafka much more explicit and I absolutely see the benefits
> > in
> > > > the
> > > > > > > > >> reduction of duplication and clashing
> > > terminologies/abstractions
> > > > > > that
> > > > > > > > >> Chris/Jay describe. Samza as a library would likely be a
> > very
> > > > nice
> > > > > > > tool
> > > > > > > > to
> > > > > > > > >> add to the Kafka ecosystem. I just have the concerns above
> > re
> > > > the
> > > > > > > > >> operational side.
> > > > > > > > >>
> > > > > > > > >> Garry
> > > > > > > > >>
> > > > > > > > >> -----Original Message-----
> > > > > > > > >> From: Gianmarco De Francisci Morales [mailto:
> > gdfm@apache.org]
> > > > > > > > >> Sent: 02 July 2015 12:56
> > > > > > > > >> To: dev@samza.apache.org
> > > > > > > > >> Subject: Re: Thoughts and obesrvations on Samza
> > > > > > > > >>
> > > > > > > > >> Very interesting thoughts.
> > > > > > > > >> From outside, I have always perceived Samza as a computing
> > > layer
> > > > > > over
> > > > > > > > >> Kafka.
> > > > > > > > >>
> > > > > > > > >> The question, maybe a bit provocative, is "should Samza
> be a
> > > > > > > sub-project
> > > > > > > > >> of Kafka then?"
> > > > > > > > >> Or does it make sense to keep it as a separate project
> with
> > a
> > > > > > separate
> > > > > > > > >> governance?
> > > > > > > > >>
> > > > > > > > >> Cheers,
> > > > > > > > >>
> > > > > > > > >> --
> > > > > > > > >> Gianmarco
> > > > > > > > >>
> > > > > > > > >>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> > > > wrote:
> > > > > > > > >>>
> > > > > > > > >>> Overall, I agree to couple with Kafka more tightly.
> Because
> > > > Samza
> > > > > > de
> > > > > > > > >>> facto is based on Kafka, and it should leverage what
> Kafka
> > > has.
> > > > > At
> > > > > > > the
> > > > > > > > >>> same time, Kafka does not need to reinvent what Samza
> > already
> > > > > has.
> > > > > > I
> > > > > > > > >>> also like the idea of separating the ingestion and
> > > > > transformation.
> > > > > > > > >>>
> > > > > > > > >>> But it is a little difficult for me to image how the
> Samza
> > > will
> > > > > > look
> > > > > > > > >> like.
> > > > > > > > >>> And I feel Chris and Jay have a little difference in
> terms
> > of
> > > > how
> > > > > > > > >>> Samza should look like.
> > > > > > > > >>>
> > > > > > > > >>> *** Will it look like what Jay's code shows (A client of
> > > > Kakfa) ?
> > > > > > And
> > > > > > > > >>> user's application code calls this client?
> > > > > > > > >>>
> > > > > > > > >>> 1. If we make Samza be a library of Kafka (like what the
> > code
> > > > > > shows),
> > > > > > > > >>> how do we implement auto-balance and fault-tolerance? Are
> > > they
> > > > > > taken
> > > > > > > > >>> care by the Kafka broker or other mechanism, such as
> "Samza
> > > > > worker"
> > > > > > > > >>> (just make up the name) ?
> > > > > > > > >>>
> > > > > > > > >>> 2. What about other features, such as auto-scaling,
> shared
> > > > state,
> > > > > > > > >>> monitoring?
> > > > > > > > >>>
> > > > > > > > >>>
> > > > > > > > >>> *** If we have Samza standalone, (is this what Chris
> > > suggests?)
> > > > > > > > >>>
> > > > > > > > >>> 1. we still need to ingest data from Kakfa and produce to
> > it.
> > > > > Then
> > > > > > it
> > > > > > > > >>> becomes the same as what Samza looks like now, except it
> > does
> > > > not
> > > > > > > rely
> > > > > > > > >>> on Yarn anymore.
> > > > > > > > >>>
> > > > > > > > >>> 2. if it is standalone, how can it leverage Kafka's
> > metrics,
> > > > > logs,
> > > > > > > > >>> etc? Use Kafka code as the dependency?
> > > > > > > > >>>
> > > > > > > > >>>
> > > > > > > > >>> Thanks,
> > > > > > > > >>>
> > > > > > > > >>> Fang, Yan
> > > > > > > > >>> yanfang724@gmail.com
> > > > > > > > >>>
> > > > > > > > >>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > > > > wangguoz@gmail.com
> > > > > > >
> > > > > > > > >>> wrote:
> > > > > > > > >>>
> > > > > > > > >>>> Read through the code example and it looks good to me. A
> > few
> > > > > > > > >>>> thoughts regarding deployment:
> > > > > > > > >>>>
> > > > > > > > >>>> Today Samza deploys as executable runnable like:
> > > > > > > > >>>>
> > > > > > > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
> > > > > > > > >> --config-path=file://...
> > > > > > > > >>>>
> > > > > > > > >>>> And this proposal advocate for deploying Samza more as
> > > > embedded
> > > > > > > > >>>> libraries in user application code (ignoring the
> > terminology
> > > > > since
> > > > > > > > >>>> it is not the
> > > > > > > > >>> same
> > > > > > > > >>>> as the prototype code):
> > > > > > > > >>>>
> > > > > > > > >>>> StreamTask task = new MyStreamTask(configs); Thread
> > thread =
> > > > new
> > > > > > > > >>>> Thread(task); thread.start();
> > > > > > > > >>>>
> > > > > > > > >>>> I think both of these deployment modes are important for
> > > > > different
> > > > > > > > >>>> types
> > > > > > > > >>> of
> > > > > > > > >>>> users. That said, I think making Samza purely standalone
> > is
> > > > > still
> > > > > > > > >>>> sufficient for either runnable or library modes.
> > > > > > > > >>>>
> > > > > > > > >>>> Guozhang
> > > > > > > > >>>>
> > > > > > > > >>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> > > > jay@confluent.io>
> > > > > > > > wrote:
> > > > > > > > >>>>>
> > > > > > > > >>>>> Looks like gmail mangled the code example, it was
> > supposed
> > > to
> > > > > > look
> > > > > > > > >>>>> like
> > > > > > > > >>>>> this:
> > > > > > > > >>>>>
> > > > > > > > >>>>> Properties props = new Properties();
> > > > > > > > >>>>> props.put("bootstrap.servers", "localhost:4242");
> > > > > StreamingConfig
> > > > > > > > >>>>> config = new StreamingConfig(props);
> > > > > > > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
> > > > > > > > >>>>> config.processor(ExampleStreamProcessor.class);
> > > > > > > > >>>>> config.serialization(new StringSerializer(), new
> > > > > > > > >>>>> StringDeserializer()); KafkaStreaming container = new
> > > > > > > > >>>>> KafkaStreaming(config); container.run();
> > > > > > > > >>>>>
> > > > > > > > >>>>> -Jay
> > > > > > > > >>>>>
> > > > > > > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> > > > jay@confluent.io>
> > > > > > > > >> wrote:
> > > > > > > > >>>>>
> > > > > > > > >>>>>> Hey guys,
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> This came out of some conversations Chris and I were
> > > having
> > > > > > > > >>>>>> around
> > > > > > > > >>>>> whether
> > > > > > > > >>>>>> it would make sense to use Samza as a kind of data
> > > ingestion
> > > > > > > > >>> framework
> > > > > > > > >>>>> for
> > > > > > > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat").
> This
> > > kind
> > > > > of
> > > > > > > > >>>> combined
> > > > > > > > >>>>>> with complaints around config and YARN and the
> > discussion
> > > > > around
> > > > > > > > >>>>>> how
> > > > > > > > >>> to
> > > > > > > > >>>>>> best do a standalone mode.
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> So the thought experiment was, given that Samza was
> > > > basically
> > > > > > > > >>>>>> already totally Kafka specific, what if you just
> > embraced
> > > > that
> > > > > > > > >>>>>> and turned it
> > > > > > > > >>>> into
> > > > > > > > >>>>>> something less like a heavyweight framework and more
> > like
> > > a
> > > > > > > > >>>>>> third
> > > > > > > > >>> Kafka
> > > > > > > > >>>>>> client--a kind of "producing consumer" with state
> > > management
> > > > > > > > >>>> facilities.
> > > > > > > > >>>>>> Basically a library. Instead of a complex stream
> > > processing
> > > > > > > > >>>>>> framework
> > > > > > > > >>>>> this
> > > > > > > > >>>>>> would actually be a very simple thing, not much more
> > > > > complicated
> > > > > > > > >>>>>> to
> > > > > > > > >>> use
> > > > > > > > >>>>> or
> > > > > > > > >>>>>> operate than a Kafka consumer. As Chris said we
> thought
> > > > about
> > > > > it
> > > > > > > > >>>>>> a
> > > > > > > > >>> lot
> > > > > > > > >>>> of
> > > > > > > > >>>>>> what Samza (and the other stream processing systems
> were
> > > > > doing)
> > > > > > > > >>> seemed
> > > > > > > > >>>>> like
> > > > > > > > >>>>>> kind of a hangover from MapReduce.
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> Of course you need to ingest/output data to and from
> the
> > > > > stream
> > > > > > > > >>>>>> processing. But when we actually looked into how that
> > > would
> > > > > > > > >>>>>> work,
> > > > > > > > >>> Samza
> > > > > > > > >>>>>> isn't really an ideal data ingestion framework for a
> > bunch
> > > > of
> > > > > > > > >>> reasons.
> > > > > > > > >>>> To
> > > > > > > > >>>>>> really do that right you need a pretty different
> > internal
> > > > data
> > > > > > > > >>>>>> model
> > > > > > > > >>>> and
> > > > > > > > >>>>>> set of apis. So what if you split them and had an api
> > for
> > > > > Kafka
> > > > > > > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api
> > for
> > > > > Kafka
> > > > > > > > >>>>>> transformation (Samza).
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> This would also allow really embracing the same
> > > terminology
> > > > > and
> > > > > > > > >>>>>> conventions. One complaint about the current state is
> > that
> > > > the
> > > > > > > > >>>>>> two
> > > > > > > > >>>>> systems
> > > > > > > > >>>>>> kind of feel bolted on. Terminology like "stream" vs
> > > "topic"
> > > > > and
> > > > > > > > >>>>> different
> > > > > > > > >>>>>> config and monitoring systems means you kind of have
> to
> > > > learn
> > > > > > > > >>>>>> Kafka's
> > > > > > > > >>>>> way,
> > > > > > > > >>>>>> then learn Samza's slightly different way, then kind
> of
> > > > > > > > >>>>>> understand
> > > > > > > > >>> how
> > > > > > > > >>>>> they
> > > > > > > > >>>>>> map to each other, which having walked a few people
> > > through
> > > > > this
> > > > > > > > >>>>>> is surprisingly tricky for folks to get.
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> Since I have been spending a lot of time on airplanes
> I
> > > > hacked
> > > > > > > > >>>>>> up an ernest but still somewhat incomplete prototype
> of
> > > what
> > > > > > > > >>>>>> this would
> > > > > > > > >>> look
> > > > > > > > >>>>>> like. This is just unceremoniously dumped into Kafka
> as
> > it
> > > > > > > > >>>>>> required a
> > > > > > > > >>>> few
> > > > > > > > >>>>>> changes to the new consumer. Here is the code:
> > > > > > > > >>>
> > > > > > >
> > > >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > > > > > >>> /apache/kafka/clients/streaming
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> For the purpose of the prototype I just liberally
> > renamed
> > > > > > > > >>>>>> everything
> > > > > > > > >>> to
> > > > > > > > >>>>>> try to align it with Kafka with no regard for
> > > compatibility.
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> To use this would be something like this:
> > > > > > > > >>>>>> Properties props = new Properties();
> > > > > > > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > > > > > > >>>>>> StreamingConfig config = new
> > > > > > > > >>> StreamingConfig(props);
> > > > > > > > >>>>> config.subscribe("test-topic-1",
> > > > > > > > >>>>>> "test-topic-2");
> > > > > config.processor(ExampleStreamProcessor.class);
> > > > > > > > >>>>> config.serialization(new
> > > > > > > > >>>>>> StringSerializer(), new StringDeserializer());
> > > > KafkaStreaming
> > > > > > > > >>>> container =
> > > > > > > > >>>>>> new KafkaStreaming(config); container.run();
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> KafkaStreaming is basically the SamzaContainer;
> > > > > StreamProcessor
> > > > > > > > >>>>>> is basically StreamTask.
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> So rather than putting all the class names in a file
> and
> > > > then
> > > > > > > > >>>>>> having
> > > > > > > > >>>> the
> > > > > > > > >>>>>> job assembled by reflection, you just instantiate the
> > > > > container
> > > > > > > > >>>>>> programmatically. Work is balanced over however many
> > > > instances
> > > > > > > > >>>>>> of
> > > > > > > > >>> this
> > > > > > > > >>>>> are
> > > > > > > > >>>>>> alive at any time (i.e. if an instance dies, new tasks
> > are
> > > > > added
> > > > > > > > >>>>>> to
> > > > > > > > >>> the
> > > > > > > > >>>>>> existing containers without shutting them down).
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> We would provide some glue for running this stuff in
> > YARN
> > > > via
> > > > > > > > >>>>>> Slider, Mesos via Marathon, and AWS using some of
> their
> > > > tools
> > > > > > > > >>>>>> but from the
> > > > > > > > >>>> point
> > > > > > > > >>>>> of
> > > > > > > > >>>>>> view of these frameworks these stream processing jobs
> > are
> > > > just
> > > > > > > > >>>> stateless
> > > > > > > > >>>>>> services that can come and go and expand and contract
> at
> > > > will.
> > > > > > > > >>>>>> There
> > > > > > > > >>> is
> > > > > > > > >>>>> no
> > > > > > > > >>>>>> more custom scheduler.
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> Here are some relevant details:
> > > > > > > > >>>>>>
> > > > > > > > >>>>>>   1. It is only ~1300 lines of code, it would get
> larger
> > > if
> > > > we
> > > > > > > > >>>>>>   productionized but not vastly larger. We really do
> > get a
> > > > ton
> > > > > > > > >>>>>> of
> > > > > > > > >>>>> leverage
> > > > > > > > >>>>>>   out of Kafka.
> > > > > > > > >>>>>>   2. Partition management is fully delegated to the
> new
> > > > > > consumer.
> > > > > > > > >>> This
> > > > > > > > >>>>>>   is nice since now any partition management strategy
> > > > > available
> > > > > > > > >>>>>> to
> > > > > > > > >>>> Kafka
> > > > > > > > >>>>>>   consumer is also available to Samza (and vice versa)
> > and
> > > > > with
> > > > > > > > >>>>>> the
> > > > > > > > >>>>> exact
> > > > > > > > >>>>>>   same configs.
> > > > > > > > >>>>>>   3. It supports state as well as state reuse
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> -Jay
> > > > > > > > >>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > > > > > > >>>> criccomini@apache.org>
> > > > > > > > >>>>>> wrote:
> > > > > > > > >>>>>>
> > > > > > > > >>>>>>> Hey all,
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> I have had some discussions with Samza engineers at
> > > > LinkedIn
> > > > > > > > >>>>>>> and
> > > > > > > > >>>>> Confluent
> > > > > > > > >>>>>>> and we came up with a few observations and would like
> > to
> > > > > > > > >>>>>>> propose
> > > > > > > > >>> some
> > > > > > > > >>>>>>> changes.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> We've observed some things that I want to call out
> > about
> > > > > > > > >>>>>>> Samza's
> > > > > > > > >>>> design,
> > > > > > > > >>>>>>> and I'd like to propose some changes.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> * Samza is dependent upon a dynamic deployment
> system.
> > > > > > > > >>>>>>> * Samza is too pluggable.
> > > > > > > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> > > > consumer
> > > > > > > > >>>>>>> APIs
> > > > > > > > >>> are
> > > > > > > > >>>>>>> trying to solve a lot of the same problems.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> All three of these issues are related, but I'll
> address
> > > > them
> > > > > in
> > > > > > > > >>> order.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Deployment
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Samza strongly depends on the use of a dynamic
> > deployment
> > > > > > > > >>>>>>> scheduler
> > > > > > > > >>>> such
> > > > > > > > >>>>>>> as
> > > > > > > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we
> bet
> > > > that
> > > > > > > > >>>>>>> there
> > > > > > > > >>>> would
> > > > > > > > >>>>>>> be
> > > > > > > > >>>>>>> one or two winners in this area, and we could support
> > > them,
> > > > > and
> > > > > > > > >>>>>>> the
> > > > > > > > >>>> rest
> > > > > > > > >>>>>>> would go away. In reality, there are many variations.
> > > > > > > > >>>>>>> Furthermore,
> > > > > > > > >>>> many
> > > > > > > > >>>>>>> people still prefer to just start their processors
> like
> > > > > normal
> > > > > > > > >>>>>>> Java processes, and use traditional deployment
> scripts
> > > such
> > > > > as
> > > > > > > > >>>>>>> Fabric,
> > > > > > > > >>>> Chef,
> > > > > > > > >>>>>>> Ansible, etc. Forcing a deployment system on users
> > makes
> > > > the
> > > > > > > > >>>>>>> Samza start-up process really painful for first time
> > > users.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Dynamic deployment as a requirement was also a bit
> of a
> > > > > > > > >>>>>>> mis-fire
> > > > > > > > >>>> because
> > > > > > > > >>>>>>> of
> > > > > > > > >>>>>>> a fundamental misunderstanding between the nature of
> > > batch
> > > > > jobs
> > > > > > > > >>>>>>> and
> > > > > > > > >>>>> stream
> > > > > > > > >>>>>>> processing jobs. Early on, we made conscious effort
> to
> > > > favor
> > > > > > > > >>>>>>> the
> > > > > > > > >>>> Hadoop
> > > > > > > > >>>>>>> (Map/Reduce) way of doing things, since it worked and
> > was
> > > > > well
> > > > > > > > >>>>> understood.
> > > > > > > > >>>>>>> One thing that we missed was that batch jobs have a
> > > > definite
> > > > > > > > >>>> beginning,
> > > > > > > > >>>>>>> and
> > > > > > > > >>>>>>> end, and stream processing jobs don't (usually). This
> > > leads
> > > > > to
> > > > > > > > >>>>>>> a
> > > > > > > > >>> much
> > > > > > > > >>>>>>> simpler scheduling problem for stream processors. You
> > > > > basically
> > > > > > > > >>>>>>> just
> > > > > > > > >>>>> need
> > > > > > > > >>>>>>> to find a place to start the processor, and start it.
> > The
> > > > way
> > > > > > > > >>>>>>> we run grids, at LinkedIn, there's no concept of a
> > > cluster
> > > > > > > > >>>>>>> being "full". We always
> > > > > > > > >>>> add
> > > > > > > > >>>>>>> more machines. The problem with coupling Samza with a
> > > > > scheduler
> > > > > > > > >>>>>>> is
> > > > > > > > >>>> that
> > > > > > > > >>>>>>> Samza (as a framework) now has to handle deployment.
> > This
> > > > > pulls
> > > > > > > > >>>>>>> in a
> > > > > > > > >>>>> bunch
> > > > > > > > >>>>>>> of things such as configuration distribution (config
> > > > stream),
> > > > > > > > >>>>>>> shell
> > > > > > > > >>>>> scrips
> > > > > > > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> > > > stuff),
> > > > > > etc.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Another reason for requiring dynamic deployment was
> to
> > > > > support
> > > > > > > > >>>>>>> data locality. If you want to have locality, you need
> > to
> > > > put
> > > > > > > > >>>>>>> your
> > > > > > > > >>>> processors
> > > > > > > > >>>>>>> close to the data they're processing. Upon further
> > > > > > > > >>>>>>> investigation,
> > > > > > > > >>>>> though,
> > > > > > > > >>>>>>> this feature is not that beneficial. There is some
> good
> > > > > > > > >>>>>>> discussion
> > > > > > > > >>>> about
> > > > > > > > >>>>>>> some problems with it on SAMZA-335. Again, we took
> the
> > > > > > > > >>>>>>> Map/Reduce
> > > > > > > > >>>> path,
> > > > > > > > >>>>>>> but
> > > > > > > > >>>>>>> there are some fundamental differences between HDFS
> and
> > > > > Kafka.
> > > > > > > > >>>>>>> HDFS
> > > > > > > > >>>> has
> > > > > > > > >>>>>>> blocks, while Kafka has partitions. This leads to
> less
> > > > > > > > >>>>>>> optimization potential with stream processors on top
> of
> > > > > Kafka.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> This feature is also used as a crutch. Samza doesn't
> > have
> > > > any
> > > > > > > > >>>>>>> built
> > > > > > > > >>> in
> > > > > > > > >>>>>>> fault-tolerance logic. Instead, it depends on the
> > dynamic
> > > > > > > > >>>>>>> deployment scheduling system to handle restarts when
> a
> > > > > > > > >>>>>>> processor dies. This has
> > > > > > > > >>>>> made
> > > > > > > > >>>>>>> it very difficult to write a standalone Samza
> container
> > > > > > > > >> (SAMZA-516).
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Pluggability
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> In some cases pluggability is good, but I think that
> > > we've
> > > > > gone
> > > > > > > > >>>>>>> too
> > > > > > > > >>>> far
> > > > > > > > >>>>>>> with it. Currently, Samza has:
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> * Pluggable config.
> > > > > > > > >>>>>>> * Pluggable metrics.
> > > > > > > > >>>>>>> * Pluggable deployment systems.
> > > > > > > > >>>>>>> * Pluggable streaming systems (SystemConsumer,
> > > > > SystemProducer,
> > > > > > > > >> etc).
> > > > > > > > >>>>>>> * Pluggable serdes.
> > > > > > > > >>>>>>> * Pluggable storage engines.
> > > > > > > > >>>>>>> * Pluggable strategies for just about every component
> > > > > > > > >>> (MessageChooser,
> > > > > > > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> There's probably more that I've forgotten, as well.
> > Some
> > > of
> > > > > > > > >>>>>>> these
> > > > > > > > >>> are
> > > > > > > > >>>>>>> useful, but some have proven not to be. This all
> comes
> > > at a
> > > > > > cost:
> > > > > > > > >>>>>>> complexity. This complexity is making it harder for
> our
> > > > users
> > > > > > > > >>>>>>> to
> > > > > > > > >>> pick
> > > > > > > > >>>> up
> > > > > > > > >>>>>>> and use Samza out of the box. It also makes it
> > difficult
> > > > for
> > > > > > > > >>>>>>> Samza developers to reason about what the
> > characteristics
> > > > of
> > > > > > > > >>>>>>> the container (since the characteristics change
> > depending
> > > > on
> > > > > > > > >>>>>>> which plugins are use).
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> The issues with pluggability are most visible in the
> > > System
> > > > > > APIs.
> > > > > > > > >>> What
> > > > > > > > >>>>>>> Samza really requires to be functional is Kafka as
> its
> > > > > > > > >>>>>>> transport
> > > > > > > > >>>> layer.
> > > > > > > > >>>>>>> But
> > > > > > > > >>>>>>> we've conflated two unrelated use cases into one API:
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> 1. Get data into/out of Kafka.
> > > > > > > > >>>>>>> 2. Process the data in Kafka.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> The current System API supports both of these use
> > cases.
> > > > The
> > > > > > > > >>>>>>> problem
> > > > > > > > >>>> is,
> > > > > > > > >>>>>>> we
> > > > > > > > >>>>>>> actually want different features for each use case.
> By
> > > > > papering
> > > > > > > > >>>>>>> over
> > > > > > > > >>>>> these
> > > > > > > > >>>>>>> two use cases, and providing a single API, we've
> > > > introduced a
> > > > > > > > >>>>>>> ton of
> > > > > > > > >>>>> leaky
> > > > > > > > >>>>>>> abstractions.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> For example, what we'd really like in (2) is to have
> > > > > > > > >>>>>>> monotonically increasing longs for offsets (like
> > Kafka).
> > > > This
> > > > > > > > >>>>>>> would be at odds
> > > > > > > > >>> with
> > > > > > > > >>>>> (1),
> > > > > > > > >>>>>>> though, since different systems have different
> > > > > > > > >>>>> SCNs/Offsets/UUIDs/vectors.
> > > > > > > > >>>>>>> There was discussion both on the mailing list and the
> > SQL
> > > > > JIRAs
> > > > > > > > >>> about
> > > > > > > > >>>>> the
> > > > > > > > >>>>>>> need for this.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> The same thing holds true for replayability. Kafka
> > allows
> > > > us
> > > > > to
> > > > > > > > >>> rewind
> > > > > > > > >>>>>>> when
> > > > > > > > >>>>>>> we have a failure. Many other systems don't. In some
> > > cases,
> > > > > > > > >>>>>>> systems
> > > > > > > > >>>>> return
> > > > > > > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> > > > because
> > > > > > > > >>>>>>> they
> > > > > > > > >>>> have
> > > > > > > > >>>>> no
> > > > > > > > >>>>>>> offsets.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Partitioning is another example. Kafka supports
> > > > partitioning,
> > > > > > > > >>>>>>> but
> > > > > > > > >>> many
> > > > > > > > >>>>>>> systems don't. We model this by having a single
> > partition
> > > > for
> > > > > > > > >>>>>>> those systems. Still, other systems model
> partitioning
> > > > > > > > >> differently (e.g.
> > > > > > > > >>>>>>> Kinesis).
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> The SystemAdmin interface is also a mess. Creating
> > > streams
> > > > > in a
> > > > > > > > >>>>>>> system-agnostic way is almost impossible. As is
> > modeling
> > > > > > > > >>>>>>> metadata
> > > > > > > > >>> for
> > > > > > > > >>>>> the
> > > > > > > > >>>>>>> system (replication factor, partitions, location,
> etc).
> > > The
> > > > > > > > >>>>>>> list
> > > > > > > > >>> goes
> > > > > > > > >>>>> on.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Duplicate work
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> At the time that we began writing Samza, Kafka's
> > consumer
> > > > and
> > > > > > > > >>> producer
> > > > > > > > >>>>>>> APIs
> > > > > > > > >>>>>>> had a relatively weak feature set. On the
> > consumer-side,
> > > > you
> > > > > > > > >>>>>>> had two
> > > > > > > > >>>>>>> options: use the high level consumer, or the simple
> > > > consumer.
> > > > > > > > >>>>>>> The
> > > > > > > > >>>>> problem
> > > > > > > > >>>>>>> with the high-level consumer was that it controlled
> > your
> > > > > > > > >>>>>>> offsets, partition assignments, and the order in
> which
> > > you
> > > > > > > > >>>>>>> received messages. The
> > > > > > > > >>> problem
> > > > > > > > >>>>>>> with
> > > > > > > > >>>>>>> the simple consumer is that it's not simple. It's
> > basic.
> > > > You
> > > > > > > > >>>>>>> end up
> > > > > > > > >>>>> having
> > > > > > > > >>>>>>> to handle a lot of really low-level stuff that you
> > > > shouldn't.
> > > > > > > > >>>>>>> We
> > > > > > > > >>>> spent a
> > > > > > > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> > > > robust.
> > > > > It
> > > > > > > > >>>>>>> also allows us to support some cool features:
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> * Per-partition message ordering and prioritization.
> > > > > > > > >>>>>>> * Tight control over partition assignment to support
> > > joins,
> > > > > > > > >>>>>>> global
> > > > > > > > >>>> state
> > > > > > > > >>>>>>> (if we want to implement it :)), etc.
> > > > > > > > >>>>>>> * Tight control over offset checkpointing.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> What we didn't realize at the time is that these
> > features
> > > > > > > > >>>>>>> should
> > > > > > > > >>>>> actually
> > > > > > > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
> > > > stream
> > > > > > > > >>>> processors)
> > > > > > > > >>>>>>> end up wanting to do things like joins and partition
> > > > > > > > >>>>>>> assignment. The
> > > > > > > > >>>>> Kafka
> > > > > > > > >>>>>>> community has come to the same conclusion. They're
> > > adding a
> > > > > ton
> > > > > > > > >>>>>>> of upgrades into their new Kafka consumer
> > implementation.
> > > > To
> > > > > a
> > > > > > > > >>>>>>> large extent,
> > > > > > > > >>> it's
> > > > > > > > >>>>>>> duplicate work to what we've already done in Samza.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> On top of this, Kafka ended up taking a very similar
> > > > approach
> > > > > > > > >>>>>>> to
> > > > > > > > >>>> Samza's
> > > > > > > > >>>>>>> KafkaCheckpointManager implementation for handling
> > offset
> > > > > > > > >>>> checkpointing.
> > > > > > > > >>>>>>> Like Samza, Kafka's new offset management feature
> > stores
> > > > > offset
> > > > > > > > >>>>>>> checkpoints in a topic, and allows you to fetch them
> > from
> > > > the
> > > > > > > > >>>>>>> broker.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> A lot of this seems like a waste, since we could have
> > > > shared
> > > > > > > > >>>>>>> the
> > > > > > > > >>> work
> > > > > > > > >>>> if
> > > > > > > > >>>>>>> it
> > > > > > > > >>>>>>> had been done in Kafka from the get-go.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Vision
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> All of this leads me to a rather radical proposal.
> > Samza
> > > is
> > > > > > > > >>> relatively
> > > > > > > > >>>>>>> stable at this point. I'd venture to say that we're
> > near
> > > a
> > > > > 1.0
> > > > > > > > >>>> release.
> > > > > > > > >>>>>>> I'd
> > > > > > > > >>>>>>> like to propose that we take what we've learned, and
> > > begin
> > > > > > > > >>>>>>> thinking
> > > > > > > > >>>>> about
> > > > > > > > >>>>>>> Samza beyond 1.0. What would we change if we were
> > > starting
> > > > > from
> > > > > > > > >>>> scratch?
> > > > > > > > >>>>>>> My
> > > > > > > > >>>>>>> proposal is to:
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > > > > > > > >>>>>>> processors, and eliminate all direct dependences on
> > YARN,
> > > > > > Mesos,
> > > > > > > > >> etc.
> > > > > > > > >>>>>>> 2. Make a definitive call to support only Kafka as
> the
> > > > stream
> > > > > > > > >>>> processing
> > > > > > > > >>>>>>> layer.
> > > > > > > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization,
> > and
> > > > > > > > >>>>>>> config
> > > > > > > > >>>>> systems,
> > > > > > > > >>>>>>> and simply use Kafka's instead.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> This would fix all of the issues that I outlined
> above.
> > > It
> > > > > > > > >>>>>>> should
> > > > > > > > >>> also
> > > > > > > > >>>>>>> shrink the Samza code base pretty dramatically.
> > > Supporting
> > > > > only
> > > > > > > > >>>>>>> a standalone container will allow Samza to be
> executed
> > on
> > > > > YARN
> > > > > > > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or
> most
> > > > other
> > > > > > > > >>>>>>> in-house
> > > > > > > > >>>>> deployment
> > > > > > > > >>>>>>> systems. This should make life a lot easier for new
> > > users.
> > > > > > > > >>>>>>> Imagine
> > > > > > > > >>>>> having
> > > > > > > > >>>>>>> the hello-samza tutorial without YARN. The drop in
> > > mailing
> > > > > list
> > > > > > > > >>>> traffic
> > > > > > > > >>>>>>> will be pretty dramatic.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Coupling with Kafka seems long overdue to me. The
> > reality
> > > > is,
> > > > > > > > >>> everyone
> > > > > > > > >>>>>>> that
> > > > > > > > >>>>>>> I'm aware of is using Samza with Kafka. We basically
> > > > require
> > > > > it
> > > > > > > > >>>> already
> > > > > > > > >>>>> in
> > > > > > > > >>>>>>> order for most features to work. Those that are using
> > > other
> > > > > > > > >>>>>>> systems
> > > > > > > > >>>> are
> > > > > > > > >>>>>>> generally using it for ingest into Kafka (1), and
> then
> > > they
> > > > > do
> > > > > > > > >>>>>>> the processing on top. There is already discussion (
> > > > > > > > >>>
> > > > > > >
> > > >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > > > > > >>> 767
> > > > > > > > >>>>>>> )
> > > > > > > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Once we make the call to couple with Kafka, we can
> > > > leverage a
> > > > > > > > >>>>>>> ton of
> > > > > > > > >>>>> their
> > > > > > > > >>>>>>> ecosystem. We no longer have to maintain our own
> > config,
> > > > > > > > >>>>>>> metrics,
> > > > > > > > >>> etc.
> > > > > > > > >>>>> We
> > > > > > > > >>>>>>> can all share the same libraries, and make them
> better.
> > > > This
> > > > > > > > >>>>>>> will
> > > > > > > > >>> also
> > > > > > > > >>>>>>> allow us to share the consumer/producer APIs, and
> will
> > > let
> > > > us
> > > > > > > > >>> leverage
> > > > > > > > >>>>>>> their offset management and partition management,
> > rather
> > > > than
> > > > > > > > >>>>>>> having
> > > > > > > > >>>> our
> > > > > > > > >>>>>>> own. All of the coordinator stream code would go
> away,
> > as
> > > > > would
> > > > > > > > >>>>>>> most
> > > > > > > > >>>> of
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>>>> YARN AppMaster code. We'd probably have to push some
> > > > > partition
> > > > > > > > >>>>> management
> > > > > > > > >>>>>>> features into the Kafka broker, but they're already
> > > moving
> > > > in
> > > > > > > > >>>>>>> that direction with the new consumer API. The
> features
> > we
> > > > > have
> > > > > > > > >>>>>>> for
> > > > > > > > >>>> partition
> > > > > > > > >>>>>>> assignment aren't unique to Samza, and seem like they
> > > > should
> > > > > be
> > > > > > > > >>>>>>> in
> > > > > > > > >>>> Kafka
> > > > > > > > >>>>>>> anyway. There will always be some niche usages which
> > will
> > > > > > > > >>>>>>> require
> > > > > > > > >>>> extra
> > > > > > > > >>>>>>> care and hence full control over partition
> assignments
> > > much
> > > > > > > > >>>>>>> like the
> > > > > > > > >>>>> Kafka
> > > > > > > > >>>>>>> low level consumer api. These would continue to be
> > > > supported.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> These items will be good for the Samza community.
> > They'll
> > > > > make
> > > > > > > > >>>>>>> Samza easier to use, and make it easier for
> developers
> > to
> > > > add
> > > > > > > > >>>>>>> new features.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Obviously this is a fairly large (and somewhat
> > backwards
> > > > > > > > >>> incompatible
> > > > > > > > >>>>>>> change). If we choose to go this route, it's
> important
> > > that
> > > > > we
> > > > > > > > >>> openly
> > > > > > > > >>>>>>> communicate how we're going to provide a migration
> path
> > > > from
> > > > > > > > >>>>>>> the
> > > > > > > > >>>>> existing
> > > > > > > > >>>>>>> APIs to the new ones (if we make incompatible
> > changes). I
> > > > > think
> > > > > > > > >>>>>>> at a minimum, we'd probably need to provide a wrapper
> > to
> > > > > allow
> > > > > > > > >>>>>>> existing StreamTask implementations to continue
> running
> > > on
> > > > > the
> > > > > > > > >> new container.
> > > > > > > > >>>>> It's
> > > > > > > > >>>>>>> also important that we openly communicate about
> timing,
> > > and
> > > > > > > > >>>>>>> stages
> > > > > > > > >>> of
> > > > > > > > >>>>> the
> > > > > > > > >>>>>>> migration.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> If you made it this far, I'm sure you have opinions.
> :)
> > > > > Please
> > > > > > > > >>>>>>> send
> > > > > > > > >>>> your
> > > > > > > > >>>>>>> thoughts and feedback.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Cheers,
> > > > > > > > >>>>>>> Chris
> > > > > > > > >>>>
> > > > > > > > >>>>
> > > > > > > > >>>>
> > > > > > > > >>>> --
> > > > > > > > >>>> -- Guozhang
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
@Jay,

{quote}
I think it may be possible to rework the assignment feature in the consumer
to make this always be a client-side concern so that Samza, the Kafka
consumer, and Copycat can all use the same facility.
{quote}
Thanks! I like that idea.

{quote}
So it may make sense to revist this, I don't think it is necessarily a massive
change and would give more flexibility for the variety of cases.
{quote}
I totally agree.

P.S. just for my education,
{quote}
The original design for the kafka coordinator was that the coordinator would
just coordinate *group* membership and the actual assignment of partitions
to members of the group would be done client side.
{quote}
Please correct me if I am wrong. Is the link here still valid:
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Detailed+Consumer+Coordinator+Design
?
If yes, I thought that the assignment is done by the broker as in
KAFKA-167? Maybe we can discuss and clarify this in person.

Thanks a lot!

-Yi

On Thu, Jul 2, 2015 at 3:52 PM, Jay Kreps <ja...@confluent.io> wrote:

> Yeah, hey Yi, I get what you are saying now. At the risk of getting into
> the weeds a bit you are exactly right, a similar thing is needed for
> copycat/kip-26. At the risk of getting a bit into the weeds, I think it may
> be possible to rework the assignment feature in the consumer to make this
> always be a client-side concern so that Samza, the Kafka consumer, and
> Copycat can all use the same facility.
>
> The original design for the kafka coordinator was that the coordinator
> would just coordinate *group* membership and the actual assignment of
> partitions to members of the group would be done client side. The advantage
> of this was that it was more general the disadvantage was that the server
> couldn't really check or monitor the partition assignment. Since we didn't
> have any other use case for generic group management we went with the more
> specific partition assignment.
>
> However a few things have changed since that original design:
> 1. We now have the additional use cases of copycat and Samza
> 2. We now realize that the assignment strategies don't actually necessarily
> ensure each partition is assigned to only one consumer--there are really
> valid use cases for broadcast or multiple replica assignment schemes--so we
> can't actually make the a hard assertion on the server.
>
> So it may make sense to revist this, I don't think it is necessarily a
> massive change and would give more flexibility for the variety of cases.
>
> -Jay
>
> On Thu, Jul 2, 2015 at 3:38 PM, Yi Pan <ni...@gmail.com> wrote:
>
> > @Guozhang, yes, that's what I meant. From Kafka consumers' point of view,
> > it pretty much boils down to answer the following question:
> > 1. For Kafka consumer in each container (i.e. a Samza worker), which
> topic
> > partitions it should consume.
> > Samza's current StreamTask model still makes sense to me and the
> > partition-to-task mapping is the debatable point that whether that should
> > be in Kafka or stays in a separate module. As we discussed earlier, some
> > simple partition-to-task mapping maybe expressed as co-partition
> > distribution among different topics in Kafka (forgive me if I had make
> > mistakes here since I am not 100% sure about how partition distribution
> > policies work in Kafka). However, more complex and application-specific
> > partition-to-task mapping would need to stay outside of Kafka. One
> example
> > is the discussion on SQL tasks:
> >
> >
> https://issues.apache.org/jira/browse/SAMZA-516?focusedCommentId=14288685&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14288685
> >
> > On Thu, Jul 2, 2015 at 2:47 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Since the resource scheduling systems like YARN / Mesos only gives
> Samza
> > a
> > > couple of resource units (or "containers") to run processes, while
> Samza
> > > still needs to handle task assignment / scheduling like which tasks
> > should
> > > be allocated to which containers that consume from which partitions,
> > etc. I
> > > think this is want Yi meant for "partition management"?
> > >
> > > On Thu, Jul 2, 2015 at 2:35 PM, Yi Pan <ni...@gmail.com> wrote:
> > >
> > > > @Jay, yes, the current function in the JobCoordinator is just
> partition
> > > > management. Maybe we should just call it PartitionManager to make it
> > > > explicit.
> > > >
> > > > -Yi
> > > >
> > > > On Thu, Jul 2, 2015 at 2:24 PM, Jay Kreps <ja...@confluent.io> wrote:
> > > >
> > > > > Hey Yi,
> > > > >
> > > > > What does the JobCoordinator do? YARN/Mesos/etc would be doing the
> > > actual
> > > > > resource assignment, process restart, etc, right? Is the additional
> > > value
> > > > > add of the JobCoordinator just partition management?
> > > > >
> > > > > -Jay
> > > > >
> > > > > On Thu, Jul 2, 2015 at 11:32 AM, Yi Pan <ni...@gmail.com>
> wrote:
> > > > >
> > > > > > Hi, all,
> > > > > >
> > > > > >
> > > > > > Thanks Chris for sending out this proposal and Jay for sharing
> the
> > > > > > extremely illustrative prototype code.
> > > > > >
> > > > > >
> > > > > > I have been thinking it over many times and want to list out my
> > > > personal
> > > > > > opinions below:
> > > > > >
> > > > > > 1. Generally, I agree with most of the people here on the mailing
> > > list
> > > > on
> > > > > > two points:
> > > > > >
> > > > > >    a. Deeper integration w/ Kafka is great. No more confusing
> > mapping
> > > > > from
> > > > > > SystemStreamPartition to TopicPartition etc.
> > > > > >
> > > > > >    b. Separation the ingestion vs transformation greatly simplify
> > the
> > > > > > systems APIs
> > > > > >
> > > > > > Having the above two changes would allow us to remove many
> > > unnecessary
> > > > > > complexities introduced by those pluggable interfaces Chris’
> > pointed
> > > > out,
> > > > > > e.g. pluggable streaming systems and serde.
> > > > > >
> > > > > >
> > > > > > To recall one of Chris’s statement on difficulties in dynamic
> > > > > deployment, I
> > > > > > believe that the difficulties are mainly the result of
> > tight-coupling
> > > > of
> > > > > > partition assignment vs the container deployment in the current
> > > system.
> > > > > The
> > > > > > current container deployment requires a pre-defined partition
> > > > assignment
> > > > > > strategy coupled together w/ the deployment configuration before
> we
> > > can
> > > > > > submit to YARN and start the Samza container, which makes the
> > > launching
> > > > > > process super long. Also, fault-tolerance and the embedded
> > > > JobCoordinator
> > > > > > code in YARN AppMaster is another way of  making dynamic
> deployment
> > > > more
> > > > > > complex and difficult.
> > > > > >
> > > > > >
> > > > > > First, borrowing Yan’s term, let’s call the Samza standalone
> > process
> > > a
> > > > > > Samza worker. Here is what I have been thinking:
> > > > > >
> > > > > > 1. Separate the execution framework from partition
> assignment/load
> > > > > > balancing:
> > > > > >
> > > > > >     a. a Samza worker should be launched by execution framework
> > that
> > > > only
> > > > > > deals w/ process placement to available nodes. The execution
> > > framework
> > > > > now
> > > > > > should only deal w/ how many such processes are needed, where to
> > put
> > > > > them,
> > > > > > and how to keep them alive.
> > > > > >
> > > > > >     b. Partition assignment/load balancing can be a pluggable
> > > interface
> > > > > in
> > > > > > Samza that allows the Samza workers to ask for partition
> > assignments.
> > > > > Let’s
> > > > > > borrow the name JobCoordinator for now. To allow fault-tolerance
> in
> > > > case
> > > > > of
> > > > > > failure, the partition assignments to workers need to be dynamic.
> > > > Hence,
> > > > > > the abstract interface would be much like what Jay’s code
> > illustrate:
> > > > > > get()/onAssigned()/onRevoke(). The implementation of the
> partition
> > > > > > assignment can be either:
> > > > > >
> > > > > >         a) completely rely on Kafka.
> > > > > >
> > > > > >         b) explicit partition assignment via JobCoordinator.
> > Chris’s
> > > > work
> > > > > > in SAMZA-516 can be easily incorporated here. The use case in
> > > SAMZA-41
> > > > > that
> > > > > > runs Samza ProcessJob w/ static partition assignment can be
> > > implemented
> > > > > of
> > > > > > JobCoordinator via any home-grown implementation of distributed
> > > > > > coordinator. All the work we did in LinkedIn to support dynamic
> > > > partition
> > > > > > assignment and host-affinity SAMZA-617 can be nicely reused as an
> > > > > > implementation of JobCoordinator.
> > > > > >
> > > > > >
> > > > > > When we did the above work, I can see three usage patterns in
> > Samza:
> > > > > >
> > > > > >    a. Samza as a library: Samza has a set of libraries to provide
> > > > stream
> > > > > > processing, just like a third Kafka client (as illustrated in
> Jay’s
> > > > > > example). The execution/deployment is totally controlled by the
> > > > > application
> > > > > > and the partition coordination is done via Kafka
> > > > > >
> > > > > >    b. Samza as a process: Samza runs as a standalone process.
> There
> > > may
> > > > > not
> > > > > > be a execution framework to launch and deploy Samza processes.
> The
> > > > > > partition assignment is pluggable JobCoordinator.
> > > > > >
> > > > > >    c. Samza as a service: Samza runs as a collection of
> processes.
> > > > There
> > > > > > will be an execution framework to allocate resource, launch and
> > > deploy
> > > > > > Samza workers and keep them alive. The same pluggable
> > JobCoordinator
> > > is
> > > > > > desirable here as well.
> > > > > >
> > > > > >
> > > > > > Lastly, I would argue that CopyCat in KIP-26 should probably
> follow
> > > the
> > > > > > same model. Hence, in Samza as a service model as in LinkedIn, we
> > can
> > > > use
> > > > > > the same fault tolerance execution framework to run CopyCat and
> > Samza
> > > > w/o
> > > > > > the need to operate two service platforms, which should address
> > > > Sriram’s
> > > > > > comment in the email thread.
> > > > > >
> > > > > >
> > > > > > Hope the above makes sense. Thanks all!
> > > > > >
> > > > > >
> > > > > > -Yi
> > > > > >
> > > > > > On Thu, Jul 2, 2015 at 9:53 AM, Sriram <sr...@gmail.com>
> > wrote:
> > > > > >
> > > > > > > One thing that is worth exploring is to have a transformation
> and
> > > > > > > ingestion library in Kafka but use the same framework for fault
> > > > > > tolerance,
> > > > > > > resource isolation and management. The biggest difference I see
> > in
> > > > > these
> > > > > > > two use cases is the API and data model.
> > > > > > >
> > > > > > >
> > > > > > > > On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io>
> > wrote:
> > > > > > > >
> > > > > > > > Hey Garry,
> > > > > > > >
> > > > > > > > Yeah that's super frustrating. I'd be happy to chat more
> about
> > > this
> > > > > if
> > > > > > > > you'd be interested. I think Chris and I started with the
> idea
> > of
> > > > > "what
> > > > > > > > would it take to make Samza a kick-ass ingestion tool" but
> > > > ultimately
> > > > > > we
> > > > > > > > kind of came around to the idea that ingestion and
> > transformation
> > > > had
> > > > > > > > pretty different needs and coupling the two made things hard.
> > > > > > > >
> > > > > > > > For what it's worth I think copycat (KIP-26) actually will do
> > > what
> > > > > you
> > > > > > > are
> > > > > > > > looking for.
> > > > > > > >
> > > > > > > > With regard to your point about slider, I don't necessarily
> > > > disagree.
> > > > > > > But I
> > > > > > > > think getting good YARN support is quite doable and I think
> we
> > > can
> > > > > make
> > > > > > > > that work well. I think the issue this proposal solves is
> that
> > > > > > > technically
> > > > > > > > it is pretty hard to support multiple cluster management
> > systems
> > > > the
> > > > > > way
> > > > > > > > things are now, you need to write an "app master" or
> > "framework"
> > > > for
> > > > > > each
> > > > > > > > and they are all a little different so testing is really
> hard.
> > In
> > > > the
> > > > > > > > absence of this we have been stuck with just YARN which has
> > > > fantastic
> > > > > > > > penetration in the Hadoopy part of the org, but zero
> > penetration
> > > > > > > elsewhere.
> > > > > > > > Given the huge amount of work being put in to slider,
> marathon,
> > > aws
> > > > > > > > tooling, not to mention the umpteen related packaging
> > > technologies
> > > > > > people
> > > > > > > > want to use (Docker, Kubernetes, various cloud-specific
> deploy
> > > > tools,
> > > > > > > etc)
> > > > > > > > I really think it is important to get this right.
> > > > > > > >
> > > > > > > > -Jay
> > > > > > > >
> > > > > > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > > > > > > g.turkington@improvedigital.com> wrote:
> > > > > > > >
> > > > > > > >> Hi all,
> > > > > > > >>
> > > > > > > >> I think the question below re does Samza become a
> sub-project
> > of
> > > > > Kafka
> > > > > > > >> highlights the broader point around migration. Chris
> mentions
> > > > > Samza's
> > > > > > > >> maturity is heading towards a v1 release but I'm not sure it
> > > feels
> > > > > > > right to
> > > > > > > >> launch a v1 then immediately plan to deprecate most of it.
> > > > > > > >>
> > > > > > > >> From a selfish perspective I have some guys who have started
> > > > working
> > > > > > > with
> > > > > > > >> Samza and building some new consumers/producers was next up.
> > > > Sounds
> > > > > > like
> > > > > > > >> that is absolutely not the direction to go. I need to look
> > into
> > > > the
> > > > > > KIP
> > > > > > > in
> > > > > > > >> more detail but for me the attractiveness of adding new
> Samza
> > > > > > > >> consumer/producers -- even if yes all they were doing was
> > really
> > > > > > getting
> > > > > > > >> data into and out of Kafka --  was to avoid  having to worry
> > > about
> > > > > the
> > > > > > > >> lifecycle management of external clients. If there is a
> > generic
> > > > > Kafka
> > > > > > > >> ingress/egress layer that I can plug a new connector into
> and
> > > > have a
> > > > > > > lot of
> > > > > > > >> the heavy lifting re scale and reliability done for me then
> it
> > > > gives
> > > > > > me
> > > > > > > all
> > > > > > > >> the pushing new consumers/producers would. If not then it
> > > > > complicates
> > > > > > my
> > > > > > > >> operational deployments.
> > > > > > > >>
> > > > > > > >> Which is similar to my other question with the proposal --
> if
> > we
> > > > > > build a
> > > > > > > >> fully available/stand-alone Samza plus the requisite shims
> to
> > > > > > integrate
> > > > > > > >> with Slider etc I suspect the former may be a lot more work
> > than
> > > > we
> > > > > > > think.
> > > > > > > >> We may make it much easier for a newcomer to get something
> > > running
> > > > > but
> > > > > > > >> having them step up and get a reliable production deployment
> > may
> > > > > still
> > > > > > > >> dominate mailing list  traffic, if for different reasons
> than
> > > > today.
> > > > > > > >>
> > > > > > > >> Don't get me wrong -- I'm comfortable with making the Samza
> > > > > dependency
> > > > > > > on
> > > > > > > >> Kafka much more explicit and I absolutely see the benefits
> in
> > > the
> > > > > > > >> reduction of duplication and clashing
> > terminologies/abstractions
> > > > > that
> > > > > > > >> Chris/Jay describe. Samza as a library would likely be a
> very
> > > nice
> > > > > > tool
> > > > > > > to
> > > > > > > >> add to the Kafka ecosystem. I just have the concerns above
> re
> > > the
> > > > > > > >> operational side.
> > > > > > > >>
> > > > > > > >> Garry
> > > > > > > >>
> > > > > > > >> -----Original Message-----
> > > > > > > >> From: Gianmarco De Francisci Morales [mailto:
> gdfm@apache.org]
> > > > > > > >> Sent: 02 July 2015 12:56
> > > > > > > >> To: dev@samza.apache.org
> > > > > > > >> Subject: Re: Thoughts and obesrvations on Samza
> > > > > > > >>
> > > > > > > >> Very interesting thoughts.
> > > > > > > >> From outside, I have always perceived Samza as a computing
> > layer
> > > > > over
> > > > > > > >> Kafka.
> > > > > > > >>
> > > > > > > >> The question, maybe a bit provocative, is "should Samza be a
> > > > > > sub-project
> > > > > > > >> of Kafka then?"
> > > > > > > >> Or does it make sense to keep it as a separate project with
> a
> > > > > separate
> > > > > > > >> governance?
> > > > > > > >>
> > > > > > > >> Cheers,
> > > > > > > >>
> > > > > > > >> --
> > > > > > > >> Gianmarco
> > > > > > > >>
> > > > > > > >>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> > > wrote:
> > > > > > > >>>
> > > > > > > >>> Overall, I agree to couple with Kafka more tightly. Because
> > > Samza
> > > > > de
> > > > > > > >>> facto is based on Kafka, and it should leverage what Kafka
> > has.
> > > > At
> > > > > > the
> > > > > > > >>> same time, Kafka does not need to reinvent what Samza
> already
> > > > has.
> > > > > I
> > > > > > > >>> also like the idea of separating the ingestion and
> > > > transformation.
> > > > > > > >>>
> > > > > > > >>> But it is a little difficult for me to image how the Samza
> > will
> > > > > look
> > > > > > > >> like.
> > > > > > > >>> And I feel Chris and Jay have a little difference in terms
> of
> > > how
> > > > > > > >>> Samza should look like.
> > > > > > > >>>
> > > > > > > >>> *** Will it look like what Jay's code shows (A client of
> > > Kakfa) ?
> > > > > And
> > > > > > > >>> user's application code calls this client?
> > > > > > > >>>
> > > > > > > >>> 1. If we make Samza be a library of Kafka (like what the
> code
> > > > > shows),
> > > > > > > >>> how do we implement auto-balance and fault-tolerance? Are
> > they
> > > > > taken
> > > > > > > >>> care by the Kafka broker or other mechanism, such as "Samza
> > > > worker"
> > > > > > > >>> (just make up the name) ?
> > > > > > > >>>
> > > > > > > >>> 2. What about other features, such as auto-scaling, shared
> > > state,
> > > > > > > >>> monitoring?
> > > > > > > >>>
> > > > > > > >>>
> > > > > > > >>> *** If we have Samza standalone, (is this what Chris
> > suggests?)
> > > > > > > >>>
> > > > > > > >>> 1. we still need to ingest data from Kakfa and produce to
> it.
> > > > Then
> > > > > it
> > > > > > > >>> becomes the same as what Samza looks like now, except it
> does
> > > not
> > > > > > rely
> > > > > > > >>> on Yarn anymore.
> > > > > > > >>>
> > > > > > > >>> 2. if it is standalone, how can it leverage Kafka's
> metrics,
> > > > logs,
> > > > > > > >>> etc? Use Kafka code as the dependency?
> > > > > > > >>>
> > > > > > > >>>
> > > > > > > >>> Thanks,
> > > > > > > >>>
> > > > > > > >>> Fang, Yan
> > > > > > > >>> yanfang724@gmail.com
> > > > > > > >>>
> > > > > > > >>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > > > wangguoz@gmail.com
> > > > > >
> > > > > > > >>> wrote:
> > > > > > > >>>
> > > > > > > >>>> Read through the code example and it looks good to me. A
> few
> > > > > > > >>>> thoughts regarding deployment:
> > > > > > > >>>>
> > > > > > > >>>> Today Samza deploys as executable runnable like:
> > > > > > > >>>>
> > > > > > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
> > > > > > > >> --config-path=file://...
> > > > > > > >>>>
> > > > > > > >>>> And this proposal advocate for deploying Samza more as
> > > embedded
> > > > > > > >>>> libraries in user application code (ignoring the
> terminology
> > > > since
> > > > > > > >>>> it is not the
> > > > > > > >>> same
> > > > > > > >>>> as the prototype code):
> > > > > > > >>>>
> > > > > > > >>>> StreamTask task = new MyStreamTask(configs); Thread
> thread =
> > > new
> > > > > > > >>>> Thread(task); thread.start();
> > > > > > > >>>>
> > > > > > > >>>> I think both of these deployment modes are important for
> > > > different
> > > > > > > >>>> types
> > > > > > > >>> of
> > > > > > > >>>> users. That said, I think making Samza purely standalone
> is
> > > > still
> > > > > > > >>>> sufficient for either runnable or library modes.
> > > > > > > >>>>
> > > > > > > >>>> Guozhang
> > > > > > > >>>>
> > > > > > > >>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> > > jay@confluent.io>
> > > > > > > wrote:
> > > > > > > >>>>>
> > > > > > > >>>>> Looks like gmail mangled the code example, it was
> supposed
> > to
> > > > > look
> > > > > > > >>>>> like
> > > > > > > >>>>> this:
> > > > > > > >>>>>
> > > > > > > >>>>> Properties props = new Properties();
> > > > > > > >>>>> props.put("bootstrap.servers", "localhost:4242");
> > > > StreamingConfig
> > > > > > > >>>>> config = new StreamingConfig(props);
> > > > > > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
> > > > > > > >>>>> config.processor(ExampleStreamProcessor.class);
> > > > > > > >>>>> config.serialization(new StringSerializer(), new
> > > > > > > >>>>> StringDeserializer()); KafkaStreaming container = new
> > > > > > > >>>>> KafkaStreaming(config); container.run();
> > > > > > > >>>>>
> > > > > > > >>>>> -Jay
> > > > > > > >>>>>
> > > > > > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> > > jay@confluent.io>
> > > > > > > >> wrote:
> > > > > > > >>>>>
> > > > > > > >>>>>> Hey guys,
> > > > > > > >>>>>>
> > > > > > > >>>>>> This came out of some conversations Chris and I were
> > having
> > > > > > > >>>>>> around
> > > > > > > >>>>> whether
> > > > > > > >>>>>> it would make sense to use Samza as a kind of data
> > ingestion
> > > > > > > >>> framework
> > > > > > > >>>>> for
> > > > > > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This
> > kind
> > > > of
> > > > > > > >>>> combined
> > > > > > > >>>>>> with complaints around config and YARN and the
> discussion
> > > > around
> > > > > > > >>>>>> how
> > > > > > > >>> to
> > > > > > > >>>>>> best do a standalone mode.
> > > > > > > >>>>>>
> > > > > > > >>>>>> So the thought experiment was, given that Samza was
> > > basically
> > > > > > > >>>>>> already totally Kafka specific, what if you just
> embraced
> > > that
> > > > > > > >>>>>> and turned it
> > > > > > > >>>> into
> > > > > > > >>>>>> something less like a heavyweight framework and more
> like
> > a
> > > > > > > >>>>>> third
> > > > > > > >>> Kafka
> > > > > > > >>>>>> client--a kind of "producing consumer" with state
> > management
> > > > > > > >>>> facilities.
> > > > > > > >>>>>> Basically a library. Instead of a complex stream
> > processing
> > > > > > > >>>>>> framework
> > > > > > > >>>>> this
> > > > > > > >>>>>> would actually be a very simple thing, not much more
> > > > complicated
> > > > > > > >>>>>> to
> > > > > > > >>> use
> > > > > > > >>>>> or
> > > > > > > >>>>>> operate than a Kafka consumer. As Chris said we thought
> > > about
> > > > it
> > > > > > > >>>>>> a
> > > > > > > >>> lot
> > > > > > > >>>> of
> > > > > > > >>>>>> what Samza (and the other stream processing systems were
> > > > doing)
> > > > > > > >>> seemed
> > > > > > > >>>>> like
> > > > > > > >>>>>> kind of a hangover from MapReduce.
> > > > > > > >>>>>>
> > > > > > > >>>>>> Of course you need to ingest/output data to and from the
> > > > stream
> > > > > > > >>>>>> processing. But when we actually looked into how that
> > would
> > > > > > > >>>>>> work,
> > > > > > > >>> Samza
> > > > > > > >>>>>> isn't really an ideal data ingestion framework for a
> bunch
> > > of
> > > > > > > >>> reasons.
> > > > > > > >>>> To
> > > > > > > >>>>>> really do that right you need a pretty different
> internal
> > > data
> > > > > > > >>>>>> model
> > > > > > > >>>> and
> > > > > > > >>>>>> set of apis. So what if you split them and had an api
> for
> > > > Kafka
> > > > > > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api
> for
> > > > Kafka
> > > > > > > >>>>>> transformation (Samza).
> > > > > > > >>>>>>
> > > > > > > >>>>>> This would also allow really embracing the same
> > terminology
> > > > and
> > > > > > > >>>>>> conventions. One complaint about the current state is
> that
> > > the
> > > > > > > >>>>>> two
> > > > > > > >>>>> systems
> > > > > > > >>>>>> kind of feel bolted on. Terminology like "stream" vs
> > "topic"
> > > > and
> > > > > > > >>>>> different
> > > > > > > >>>>>> config and monitoring systems means you kind of have to
> > > learn
> > > > > > > >>>>>> Kafka's
> > > > > > > >>>>> way,
> > > > > > > >>>>>> then learn Samza's slightly different way, then kind of
> > > > > > > >>>>>> understand
> > > > > > > >>> how
> > > > > > > >>>>> they
> > > > > > > >>>>>> map to each other, which having walked a few people
> > through
> > > > this
> > > > > > > >>>>>> is surprisingly tricky for folks to get.
> > > > > > > >>>>>>
> > > > > > > >>>>>> Since I have been spending a lot of time on airplanes I
> > > hacked
> > > > > > > >>>>>> up an ernest but still somewhat incomplete prototype of
> > what
> > > > > > > >>>>>> this would
> > > > > > > >>> look
> > > > > > > >>>>>> like. This is just unceremoniously dumped into Kafka as
> it
> > > > > > > >>>>>> required a
> > > > > > > >>>> few
> > > > > > > >>>>>> changes to the new consumer. Here is the code:
> > > > > > > >>>
> > > > > >
> > > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > > > > >>> /apache/kafka/clients/streaming
> > > > > > > >>>>>>
> > > > > > > >>>>>> For the purpose of the prototype I just liberally
> renamed
> > > > > > > >>>>>> everything
> > > > > > > >>> to
> > > > > > > >>>>>> try to align it with Kafka with no regard for
> > compatibility.
> > > > > > > >>>>>>
> > > > > > > >>>>>> To use this would be something like this:
> > > > > > > >>>>>> Properties props = new Properties();
> > > > > > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > > > > > >>>>>> StreamingConfig config = new
> > > > > > > >>> StreamingConfig(props);
> > > > > > > >>>>> config.subscribe("test-topic-1",
> > > > > > > >>>>>> "test-topic-2");
> > > > config.processor(ExampleStreamProcessor.class);
> > > > > > > >>>>> config.serialization(new
> > > > > > > >>>>>> StringSerializer(), new StringDeserializer());
> > > KafkaStreaming
> > > > > > > >>>> container =
> > > > > > > >>>>>> new KafkaStreaming(config); container.run();
> > > > > > > >>>>>>
> > > > > > > >>>>>> KafkaStreaming is basically the SamzaContainer;
> > > > StreamProcessor
> > > > > > > >>>>>> is basically StreamTask.
> > > > > > > >>>>>>
> > > > > > > >>>>>> So rather than putting all the class names in a file and
> > > then
> > > > > > > >>>>>> having
> > > > > > > >>>> the
> > > > > > > >>>>>> job assembled by reflection, you just instantiate the
> > > > container
> > > > > > > >>>>>> programmatically. Work is balanced over however many
> > > instances
> > > > > > > >>>>>> of
> > > > > > > >>> this
> > > > > > > >>>>> are
> > > > > > > >>>>>> alive at any time (i.e. if an instance dies, new tasks
> are
> > > > added
> > > > > > > >>>>>> to
> > > > > > > >>> the
> > > > > > > >>>>>> existing containers without shutting them down).
> > > > > > > >>>>>>
> > > > > > > >>>>>> We would provide some glue for running this stuff in
> YARN
> > > via
> > > > > > > >>>>>> Slider, Mesos via Marathon, and AWS using some of their
> > > tools
> > > > > > > >>>>>> but from the
> > > > > > > >>>> point
> > > > > > > >>>>> of
> > > > > > > >>>>>> view of these frameworks these stream processing jobs
> are
> > > just
> > > > > > > >>>> stateless
> > > > > > > >>>>>> services that can come and go and expand and contract at
> > > will.
> > > > > > > >>>>>> There
> > > > > > > >>> is
> > > > > > > >>>>> no
> > > > > > > >>>>>> more custom scheduler.
> > > > > > > >>>>>>
> > > > > > > >>>>>> Here are some relevant details:
> > > > > > > >>>>>>
> > > > > > > >>>>>>   1. It is only ~1300 lines of code, it would get larger
> > if
> > > we
> > > > > > > >>>>>>   productionized but not vastly larger. We really do
> get a
> > > ton
> > > > > > > >>>>>> of
> > > > > > > >>>>> leverage
> > > > > > > >>>>>>   out of Kafka.
> > > > > > > >>>>>>   2. Partition management is fully delegated to the new
> > > > > consumer.
> > > > > > > >>> This
> > > > > > > >>>>>>   is nice since now any partition management strategy
> > > > available
> > > > > > > >>>>>> to
> > > > > > > >>>> Kafka
> > > > > > > >>>>>>   consumer is also available to Samza (and vice versa)
> and
> > > > with
> > > > > > > >>>>>> the
> > > > > > > >>>>> exact
> > > > > > > >>>>>>   same configs.
> > > > > > > >>>>>>   3. It supports state as well as state reuse
> > > > > > > >>>>>>
> > > > > > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
> > > > > > > >>>>>>
> > > > > > > >>>>>> -Jay
> > > > > > > >>>>>>
> > > > > > > >>>>>>
> > > > > > > >>>>>>
> > > > > > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > > > > > >>>> criccomini@apache.org>
> > > > > > > >>>>>> wrote:
> > > > > > > >>>>>>
> > > > > > > >>>>>>> Hey all,
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> I have had some discussions with Samza engineers at
> > > LinkedIn
> > > > > > > >>>>>>> and
> > > > > > > >>>>> Confluent
> > > > > > > >>>>>>> and we came up with a few observations and would like
> to
> > > > > > > >>>>>>> propose
> > > > > > > >>> some
> > > > > > > >>>>>>> changes.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> We've observed some things that I want to call out
> about
> > > > > > > >>>>>>> Samza's
> > > > > > > >>>> design,
> > > > > > > >>>>>>> and I'd like to propose some changes.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> * Samza is dependent upon a dynamic deployment system.
> > > > > > > >>>>>>> * Samza is too pluggable.
> > > > > > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> > > consumer
> > > > > > > >>>>>>> APIs
> > > > > > > >>> are
> > > > > > > >>>>>>> trying to solve a lot of the same problems.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> All three of these issues are related, but I'll address
> > > them
> > > > in
> > > > > > > >>> order.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Deployment
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Samza strongly depends on the use of a dynamic
> deployment
> > > > > > > >>>>>>> scheduler
> > > > > > > >>>> such
> > > > > > > >>>>>>> as
> > > > > > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet
> > > that
> > > > > > > >>>>>>> there
> > > > > > > >>>> would
> > > > > > > >>>>>>> be
> > > > > > > >>>>>>> one or two winners in this area, and we could support
> > them,
> > > > and
> > > > > > > >>>>>>> the
> > > > > > > >>>> rest
> > > > > > > >>>>>>> would go away. In reality, there are many variations.
> > > > > > > >>>>>>> Furthermore,
> > > > > > > >>>> many
> > > > > > > >>>>>>> people still prefer to just start their processors like
> > > > normal
> > > > > > > >>>>>>> Java processes, and use traditional deployment scripts
> > such
> > > > as
> > > > > > > >>>>>>> Fabric,
> > > > > > > >>>> Chef,
> > > > > > > >>>>>>> Ansible, etc. Forcing a deployment system on users
> makes
> > > the
> > > > > > > >>>>>>> Samza start-up process really painful for first time
> > users.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Dynamic deployment as a requirement was also a bit of a
> > > > > > > >>>>>>> mis-fire
> > > > > > > >>>> because
> > > > > > > >>>>>>> of
> > > > > > > >>>>>>> a fundamental misunderstanding between the nature of
> > batch
> > > > jobs
> > > > > > > >>>>>>> and
> > > > > > > >>>>> stream
> > > > > > > >>>>>>> processing jobs. Early on, we made conscious effort to
> > > favor
> > > > > > > >>>>>>> the
> > > > > > > >>>> Hadoop
> > > > > > > >>>>>>> (Map/Reduce) way of doing things, since it worked and
> was
> > > > well
> > > > > > > >>>>> understood.
> > > > > > > >>>>>>> One thing that we missed was that batch jobs have a
> > > definite
> > > > > > > >>>> beginning,
> > > > > > > >>>>>>> and
> > > > > > > >>>>>>> end, and stream processing jobs don't (usually). This
> > leads
> > > > to
> > > > > > > >>>>>>> a
> > > > > > > >>> much
> > > > > > > >>>>>>> simpler scheduling problem for stream processors. You
> > > > basically
> > > > > > > >>>>>>> just
> > > > > > > >>>>> need
> > > > > > > >>>>>>> to find a place to start the processor, and start it.
> The
> > > way
> > > > > > > >>>>>>> we run grids, at LinkedIn, there's no concept of a
> > cluster
> > > > > > > >>>>>>> being "full". We always
> > > > > > > >>>> add
> > > > > > > >>>>>>> more machines. The problem with coupling Samza with a
> > > > scheduler
> > > > > > > >>>>>>> is
> > > > > > > >>>> that
> > > > > > > >>>>>>> Samza (as a framework) now has to handle deployment.
> This
> > > > pulls
> > > > > > > >>>>>>> in a
> > > > > > > >>>>> bunch
> > > > > > > >>>>>>> of things such as configuration distribution (config
> > > stream),
> > > > > > > >>>>>>> shell
> > > > > > > >>>>> scrips
> > > > > > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> > > stuff),
> > > > > etc.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Another reason for requiring dynamic deployment was to
> > > > support
> > > > > > > >>>>>>> data locality. If you want to have locality, you need
> to
> > > put
> > > > > > > >>>>>>> your
> > > > > > > >>>> processors
> > > > > > > >>>>>>> close to the data they're processing. Upon further
> > > > > > > >>>>>>> investigation,
> > > > > > > >>>>> though,
> > > > > > > >>>>>>> this feature is not that beneficial. There is some good
> > > > > > > >>>>>>> discussion
> > > > > > > >>>> about
> > > > > > > >>>>>>> some problems with it on SAMZA-335. Again, we took the
> > > > > > > >>>>>>> Map/Reduce
> > > > > > > >>>> path,
> > > > > > > >>>>>>> but
> > > > > > > >>>>>>> there are some fundamental differences between HDFS and
> > > > Kafka.
> > > > > > > >>>>>>> HDFS
> > > > > > > >>>> has
> > > > > > > >>>>>>> blocks, while Kafka has partitions. This leads to less
> > > > > > > >>>>>>> optimization potential with stream processors on top of
> > > > Kafka.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> This feature is also used as a crutch. Samza doesn't
> have
> > > any
> > > > > > > >>>>>>> built
> > > > > > > >>> in
> > > > > > > >>>>>>> fault-tolerance logic. Instead, it depends on the
> dynamic
> > > > > > > >>>>>>> deployment scheduling system to handle restarts when a
> > > > > > > >>>>>>> processor dies. This has
> > > > > > > >>>>> made
> > > > > > > >>>>>>> it very difficult to write a standalone Samza container
> > > > > > > >> (SAMZA-516).
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Pluggability
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> In some cases pluggability is good, but I think that
> > we've
> > > > gone
> > > > > > > >>>>>>> too
> > > > > > > >>>> far
> > > > > > > >>>>>>> with it. Currently, Samza has:
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> * Pluggable config.
> > > > > > > >>>>>>> * Pluggable metrics.
> > > > > > > >>>>>>> * Pluggable deployment systems.
> > > > > > > >>>>>>> * Pluggable streaming systems (SystemConsumer,
> > > > SystemProducer,
> > > > > > > >> etc).
> > > > > > > >>>>>>> * Pluggable serdes.
> > > > > > > >>>>>>> * Pluggable storage engines.
> > > > > > > >>>>>>> * Pluggable strategies for just about every component
> > > > > > > >>> (MessageChooser,
> > > > > > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> There's probably more that I've forgotten, as well.
> Some
> > of
> > > > > > > >>>>>>> these
> > > > > > > >>> are
> > > > > > > >>>>>>> useful, but some have proven not to be. This all comes
> > at a
> > > > > cost:
> > > > > > > >>>>>>> complexity. This complexity is making it harder for our
> > > users
> > > > > > > >>>>>>> to
> > > > > > > >>> pick
> > > > > > > >>>> up
> > > > > > > >>>>>>> and use Samza out of the box. It also makes it
> difficult
> > > for
> > > > > > > >>>>>>> Samza developers to reason about what the
> characteristics
> > > of
> > > > > > > >>>>>>> the container (since the characteristics change
> depending
> > > on
> > > > > > > >>>>>>> which plugins are use).
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> The issues with pluggability are most visible in the
> > System
> > > > > APIs.
> > > > > > > >>> What
> > > > > > > >>>>>>> Samza really requires to be functional is Kafka as its
> > > > > > > >>>>>>> transport
> > > > > > > >>>> layer.
> > > > > > > >>>>>>> But
> > > > > > > >>>>>>> we've conflated two unrelated use cases into one API:
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> 1. Get data into/out of Kafka.
> > > > > > > >>>>>>> 2. Process the data in Kafka.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> The current System API supports both of these use
> cases.
> > > The
> > > > > > > >>>>>>> problem
> > > > > > > >>>> is,
> > > > > > > >>>>>>> we
> > > > > > > >>>>>>> actually want different features for each use case. By
> > > > papering
> > > > > > > >>>>>>> over
> > > > > > > >>>>> these
> > > > > > > >>>>>>> two use cases, and providing a single API, we've
> > > introduced a
> > > > > > > >>>>>>> ton of
> > > > > > > >>>>> leaky
> > > > > > > >>>>>>> abstractions.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> For example, what we'd really like in (2) is to have
> > > > > > > >>>>>>> monotonically increasing longs for offsets (like
> Kafka).
> > > This
> > > > > > > >>>>>>> would be at odds
> > > > > > > >>> with
> > > > > > > >>>>> (1),
> > > > > > > >>>>>>> though, since different systems have different
> > > > > > > >>>>> SCNs/Offsets/UUIDs/vectors.
> > > > > > > >>>>>>> There was discussion both on the mailing list and the
> SQL
> > > > JIRAs
> > > > > > > >>> about
> > > > > > > >>>>> the
> > > > > > > >>>>>>> need for this.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> The same thing holds true for replayability. Kafka
> allows
> > > us
> > > > to
> > > > > > > >>> rewind
> > > > > > > >>>>>>> when
> > > > > > > >>>>>>> we have a failure. Many other systems don't. In some
> > cases,
> > > > > > > >>>>>>> systems
> > > > > > > >>>>> return
> > > > > > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> > > because
> > > > > > > >>>>>>> they
> > > > > > > >>>> have
> > > > > > > >>>>> no
> > > > > > > >>>>>>> offsets.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Partitioning is another example. Kafka supports
> > > partitioning,
> > > > > > > >>>>>>> but
> > > > > > > >>> many
> > > > > > > >>>>>>> systems don't. We model this by having a single
> partition
> > > for
> > > > > > > >>>>>>> those systems. Still, other systems model partitioning
> > > > > > > >> differently (e.g.
> > > > > > > >>>>>>> Kinesis).
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> The SystemAdmin interface is also a mess. Creating
> > streams
> > > > in a
> > > > > > > >>>>>>> system-agnostic way is almost impossible. As is
> modeling
> > > > > > > >>>>>>> metadata
> > > > > > > >>> for
> > > > > > > >>>>> the
> > > > > > > >>>>>>> system (replication factor, partitions, location, etc).
> > The
> > > > > > > >>>>>>> list
> > > > > > > >>> goes
> > > > > > > >>>>> on.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Duplicate work
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> At the time that we began writing Samza, Kafka's
> consumer
> > > and
> > > > > > > >>> producer
> > > > > > > >>>>>>> APIs
> > > > > > > >>>>>>> had a relatively weak feature set. On the
> consumer-side,
> > > you
> > > > > > > >>>>>>> had two
> > > > > > > >>>>>>> options: use the high level consumer, or the simple
> > > consumer.
> > > > > > > >>>>>>> The
> > > > > > > >>>>> problem
> > > > > > > >>>>>>> with the high-level consumer was that it controlled
> your
> > > > > > > >>>>>>> offsets, partition assignments, and the order in which
> > you
> > > > > > > >>>>>>> received messages. The
> > > > > > > >>> problem
> > > > > > > >>>>>>> with
> > > > > > > >>>>>>> the simple consumer is that it's not simple. It's
> basic.
> > > You
> > > > > > > >>>>>>> end up
> > > > > > > >>>>> having
> > > > > > > >>>>>>> to handle a lot of really low-level stuff that you
> > > shouldn't.
> > > > > > > >>>>>>> We
> > > > > > > >>>> spent a
> > > > > > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> > > robust.
> > > > It
> > > > > > > >>>>>>> also allows us to support some cool features:
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> * Per-partition message ordering and prioritization.
> > > > > > > >>>>>>> * Tight control over partition assignment to support
> > joins,
> > > > > > > >>>>>>> global
> > > > > > > >>>> state
> > > > > > > >>>>>>> (if we want to implement it :)), etc.
> > > > > > > >>>>>>> * Tight control over offset checkpointing.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> What we didn't realize at the time is that these
> features
> > > > > > > >>>>>>> should
> > > > > > > >>>>> actually
> > > > > > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
> > > stream
> > > > > > > >>>> processors)
> > > > > > > >>>>>>> end up wanting to do things like joins and partition
> > > > > > > >>>>>>> assignment. The
> > > > > > > >>>>> Kafka
> > > > > > > >>>>>>> community has come to the same conclusion. They're
> > adding a
> > > > ton
> > > > > > > >>>>>>> of upgrades into their new Kafka consumer
> implementation.
> > > To
> > > > a
> > > > > > > >>>>>>> large extent,
> > > > > > > >>> it's
> > > > > > > >>>>>>> duplicate work to what we've already done in Samza.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> On top of this, Kafka ended up taking a very similar
> > > approach
> > > > > > > >>>>>>> to
> > > > > > > >>>> Samza's
> > > > > > > >>>>>>> KafkaCheckpointManager implementation for handling
> offset
> > > > > > > >>>> checkpointing.
> > > > > > > >>>>>>> Like Samza, Kafka's new offset management feature
> stores
> > > > offset
> > > > > > > >>>>>>> checkpoints in a topic, and allows you to fetch them
> from
> > > the
> > > > > > > >>>>>>> broker.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> A lot of this seems like a waste, since we could have
> > > shared
> > > > > > > >>>>>>> the
> > > > > > > >>> work
> > > > > > > >>>> if
> > > > > > > >>>>>>> it
> > > > > > > >>>>>>> had been done in Kafka from the get-go.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Vision
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> All of this leads me to a rather radical proposal.
> Samza
> > is
> > > > > > > >>> relatively
> > > > > > > >>>>>>> stable at this point. I'd venture to say that we're
> near
> > a
> > > > 1.0
> > > > > > > >>>> release.
> > > > > > > >>>>>>> I'd
> > > > > > > >>>>>>> like to propose that we take what we've learned, and
> > begin
> > > > > > > >>>>>>> thinking
> > > > > > > >>>>> about
> > > > > > > >>>>>>> Samza beyond 1.0. What would we change if we were
> > starting
> > > > from
> > > > > > > >>>> scratch?
> > > > > > > >>>>>>> My
> > > > > > > >>>>>>> proposal is to:
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > > > > > > >>>>>>> processors, and eliminate all direct dependences on
> YARN,
> > > > > Mesos,
> > > > > > > >> etc.
> > > > > > > >>>>>>> 2. Make a definitive call to support only Kafka as the
> > > stream
> > > > > > > >>>> processing
> > > > > > > >>>>>>> layer.
> > > > > > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization,
> and
> > > > > > > >>>>>>> config
> > > > > > > >>>>> systems,
> > > > > > > >>>>>>> and simply use Kafka's instead.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> This would fix all of the issues that I outlined above.
> > It
> > > > > > > >>>>>>> should
> > > > > > > >>> also
> > > > > > > >>>>>>> shrink the Samza code base pretty dramatically.
> > Supporting
> > > > only
> > > > > > > >>>>>>> a standalone container will allow Samza to be executed
> on
> > > > YARN
> > > > > > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most
> > > other
> > > > > > > >>>>>>> in-house
> > > > > > > >>>>> deployment
> > > > > > > >>>>>>> systems. This should make life a lot easier for new
> > users.
> > > > > > > >>>>>>> Imagine
> > > > > > > >>>>> having
> > > > > > > >>>>>>> the hello-samza tutorial without YARN. The drop in
> > mailing
> > > > list
> > > > > > > >>>> traffic
> > > > > > > >>>>>>> will be pretty dramatic.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Coupling with Kafka seems long overdue to me. The
> reality
> > > is,
> > > > > > > >>> everyone
> > > > > > > >>>>>>> that
> > > > > > > >>>>>>> I'm aware of is using Samza with Kafka. We basically
> > > require
> > > > it
> > > > > > > >>>> already
> > > > > > > >>>>> in
> > > > > > > >>>>>>> order for most features to work. Those that are using
> > other
> > > > > > > >>>>>>> systems
> > > > > > > >>>> are
> > > > > > > >>>>>>> generally using it for ingest into Kafka (1), and then
> > they
> > > > do
> > > > > > > >>>>>>> the processing on top. There is already discussion (
> > > > > > > >>>
> > > > > >
> > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > > > > >>> 767
> > > > > > > >>>>>>> )
> > > > > > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Once we make the call to couple with Kafka, we can
> > > leverage a
> > > > > > > >>>>>>> ton of
> > > > > > > >>>>> their
> > > > > > > >>>>>>> ecosystem. We no longer have to maintain our own
> config,
> > > > > > > >>>>>>> metrics,
> > > > > > > >>> etc.
> > > > > > > >>>>> We
> > > > > > > >>>>>>> can all share the same libraries, and make them better.
> > > This
> > > > > > > >>>>>>> will
> > > > > > > >>> also
> > > > > > > >>>>>>> allow us to share the consumer/producer APIs, and will
> > let
> > > us
> > > > > > > >>> leverage
> > > > > > > >>>>>>> their offset management and partition management,
> rather
> > > than
> > > > > > > >>>>>>> having
> > > > > > > >>>> our
> > > > > > > >>>>>>> own. All of the coordinator stream code would go away,
> as
> > > > would
> > > > > > > >>>>>>> most
> > > > > > > >>>> of
> > > > > > > >>>>>>> the
> > > > > > > >>>>>>> YARN AppMaster code. We'd probably have to push some
> > > > partition
> > > > > > > >>>>> management
> > > > > > > >>>>>>> features into the Kafka broker, but they're already
> > moving
> > > in
> > > > > > > >>>>>>> that direction with the new consumer API. The features
> we
> > > > have
> > > > > > > >>>>>>> for
> > > > > > > >>>> partition
> > > > > > > >>>>>>> assignment aren't unique to Samza, and seem like they
> > > should
> > > > be
> > > > > > > >>>>>>> in
> > > > > > > >>>> Kafka
> > > > > > > >>>>>>> anyway. There will always be some niche usages which
> will
> > > > > > > >>>>>>> require
> > > > > > > >>>> extra
> > > > > > > >>>>>>> care and hence full control over partition assignments
> > much
> > > > > > > >>>>>>> like the
> > > > > > > >>>>> Kafka
> > > > > > > >>>>>>> low level consumer api. These would continue to be
> > > supported.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> These items will be good for the Samza community.
> They'll
> > > > make
> > > > > > > >>>>>>> Samza easier to use, and make it easier for developers
> to
> > > add
> > > > > > > >>>>>>> new features.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Obviously this is a fairly large (and somewhat
> backwards
> > > > > > > >>> incompatible
> > > > > > > >>>>>>> change). If we choose to go this route, it's important
> > that
> > > > we
> > > > > > > >>> openly
> > > > > > > >>>>>>> communicate how we're going to provide a migration path
> > > from
> > > > > > > >>>>>>> the
> > > > > > > >>>>> existing
> > > > > > > >>>>>>> APIs to the new ones (if we make incompatible
> changes). I
> > > > think
> > > > > > > >>>>>>> at a minimum, we'd probably need to provide a wrapper
> to
> > > > allow
> > > > > > > >>>>>>> existing StreamTask implementations to continue running
> > on
> > > > the
> > > > > > > >> new container.
> > > > > > > >>>>> It's
> > > > > > > >>>>>>> also important that we openly communicate about timing,
> > and
> > > > > > > >>>>>>> stages
> > > > > > > >>> of
> > > > > > > >>>>> the
> > > > > > > >>>>>>> migration.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> If you made it this far, I'm sure you have opinions. :)
> > > > Please
> > > > > > > >>>>>>> send
> > > > > > > >>>> your
> > > > > > > >>>>>>> thoughts and feedback.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Cheers,
> > > > > > > >>>>>>> Chris
> > > > > > > >>>>
> > > > > > > >>>>
> > > > > > > >>>>
> > > > > > > >>>> --
> > > > > > > >>>> -- Guozhang
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
Hi, Guozhang,

{quote}
but I think if we decide to go this
route we'd better do it now than later as the protocol is not officially
"released" yet. This may delay the first release of the new consumer.
{quote}
I totally agree. Given that potential heavy migration cost later, I think
that a slight delay at beginning is worthwhile.

On Sun, Jul 5, 2015 at 10:49 PM, Guozhang Wang <wa...@gmail.com> wrote:

> 1. I am neutral to modifying the consumer rebalance protocol to move the
> logic pluggable to the client side, but I think if we decide to go this
> route we'd better do it now than later as the protocol is not officially
> "released" yet. This may delay the first release of the new consumer.
>
> 2. I like the idea of rebranding Samza as Kafka Messaging to keep the same
> API / project structure. But I think the Samza PMC / committers will have
> more saying in this manner.
>
> Guozhang
>
>
>
> On Fri, Jul 3, 2015 at 12:11 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey Gianmarco,
> >
> > To your broader point, I agree that having a close alignment with Kafka
> > would be a great thing in terms of adoption/discoverability/etc. There
> > areas where I think this matters a lot are:
> > 1. Website and docs: ideally when reading about Kafka you should be able
> to
> > find out about Samza.
> > 2. Api style and naming: ideally the various interfaces should feel
> similar
> > and use similar concepts and names. This is a bunch of little things
> > (calling topics and partitions in the same way, sharing metrics, sharing
> > partitioning strategies, etc).
> > 3. Release alignment--i.e. this set of versions all work together.
> > 4. Branding--I actually think if we go down that route it would be worth
> > considering just calling Samza something like "Kafka Streams" or "Kafka
> > Streaming" which I think would help a lot people to understand what it is
> > and since Kafka is heavily adopted would help with adoption. It always
> > seems silly to bother with naming, but I actually think this ends up
> > mattering a ton in how people understand the system (I guess as
> programmers
> > we kind of all intuitively understand the importance of good naming).
> >
> > WRT partition mapping, yeah I totally agree. I think in all proposals
> this
> > is left pluggable. And I think ideally the same set of assignment
> > strategies should be available either in the Kafka consumer or in Samza.
> I
> > think at this point the only debate is whether this is controlled client
> > side or server side.
> >
> > -Jay
> >
> > On Fri, Jul 3, 2015 at 1:40 AM, Gianmarco De Francisci Morales <
> > gdfm@apache.org> wrote:
> >
> > > Hi Jay,
> > >
> > > Thanks for your answer.
> > >
> > >
> > > > However a few things have changed since that original design:
> > > > 1. We now have the additional use cases of copycat and Samza
> > > > 2. We now realize that the assignment strategies don't actually
> > > necessarily
> > > > ensure each partition is assigned to only one consumer--there are
> > really
> > > > valid use cases for broadcast or multiple replica assignment
> > schemes--so
> > > we
> > > > can't actually make the a hard assertion on the server.
> > > >
> > > > So it may make sense to revist this, I don't think it is necessarily
> a
> > > > massive change and would give more flexibility for the variety of
> > cases.
> > > >
> > > > -Jay
> > >
> > >
> > > I totally agree, the 1-partition-1-task mapping is too restrictive.
> > > However, I think the fundamental operation that Samza, Copycat, and
> Kafka
> > > consumers should agree upon is "how can I specify in a simple and
> > > transparent way which partitions I want to consume, and how?".
> > > This means providing a mapping from partitions to consumer tasks,
> > possibly
> > > in a transparent way so as to allow for optimizations in placement,
> > > co-partitioning, etc...
> > > This issue has the potential of generating again a lot of duplicate
> work,
> > > and I think it should be solved at the Kafka level.
> > > Given that Copycat and normal consumers are already inside Kafka, I
> think
> > > having Samza there as well would simplify things a lot.
> > > The result is that Kafka would be a complete package for handling
> > streams:
> > > - Messaging, partitioning, and fault tolerance (Kafka core)
> > > - Ingestion (Copycat)
> > > - Lightweight processing (Samza)
> > > - Coupling with other systems (Kafka consumers)
> > >
> > > Cheers,
> > >
> > > --
> > > Gianmarco
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: Thoughts and obesrvations on Samza

Posted by Guozhang Wang <wa...@gmail.com>.
1. I am neutral to modifying the consumer rebalance protocol to move the
logic pluggable to the client side, but I think if we decide to go this
route we'd better do it now than later as the protocol is not officially
"released" yet. This may delay the first release of the new consumer.

2. I like the idea of rebranding Samza as Kafka Messaging to keep the same
API / project structure. But I think the Samza PMC / committers will have
more saying in this manner.

Guozhang



On Fri, Jul 3, 2015 at 12:11 PM, Jay Kreps <ja...@confluent.io> wrote:

> Hey Gianmarco,
>
> To your broader point, I agree that having a close alignment with Kafka
> would be a great thing in terms of adoption/discoverability/etc. There
> areas where I think this matters a lot are:
> 1. Website and docs: ideally when reading about Kafka you should be able to
> find out about Samza.
> 2. Api style and naming: ideally the various interfaces should feel similar
> and use similar concepts and names. This is a bunch of little things
> (calling topics and partitions in the same way, sharing metrics, sharing
> partitioning strategies, etc).
> 3. Release alignment--i.e. this set of versions all work together.
> 4. Branding--I actually think if we go down that route it would be worth
> considering just calling Samza something like "Kafka Streams" or "Kafka
> Streaming" which I think would help a lot people to understand what it is
> and since Kafka is heavily adopted would help with adoption. It always
> seems silly to bother with naming, but I actually think this ends up
> mattering a ton in how people understand the system (I guess as programmers
> we kind of all intuitively understand the importance of good naming).
>
> WRT partition mapping, yeah I totally agree. I think in all proposals this
> is left pluggable. And I think ideally the same set of assignment
> strategies should be available either in the Kafka consumer or in Samza. I
> think at this point the only debate is whether this is controlled client
> side or server side.
>
> -Jay
>
> On Fri, Jul 3, 2015 at 1:40 AM, Gianmarco De Francisci Morales <
> gdfm@apache.org> wrote:
>
> > Hi Jay,
> >
> > Thanks for your answer.
> >
> >
> > > However a few things have changed since that original design:
> > > 1. We now have the additional use cases of copycat and Samza
> > > 2. We now realize that the assignment strategies don't actually
> > necessarily
> > > ensure each partition is assigned to only one consumer--there are
> really
> > > valid use cases for broadcast or multiple replica assignment
> schemes--so
> > we
> > > can't actually make the a hard assertion on the server.
> > >
> > > So it may make sense to revist this, I don't think it is necessarily a
> > > massive change and would give more flexibility for the variety of
> cases.
> > >
> > > -Jay
> >
> >
> > I totally agree, the 1-partition-1-task mapping is too restrictive.
> > However, I think the fundamental operation that Samza, Copycat, and Kafka
> > consumers should agree upon is "how can I specify in a simple and
> > transparent way which partitions I want to consume, and how?".
> > This means providing a mapping from partitions to consumer tasks,
> possibly
> > in a transparent way so as to allow for optimizations in placement,
> > co-partitioning, etc...
> > This issue has the potential of generating again a lot of duplicate work,
> > and I think it should be solved at the Kafka level.
> > Given that Copycat and normal consumers are already inside Kafka, I think
> > having Samza there as well would simplify things a lot.
> > The result is that Kafka would be a complete package for handling
> streams:
> > - Messaging, partitioning, and fault tolerance (Kafka core)
> > - Ingestion (Copycat)
> > - Lightweight processing (Samza)
> > - Coupling with other systems (Kafka consumers)
> >
> > Cheers,
> >
> > --
> > Gianmarco
> >
>



-- 
-- Guozhang

Re: Thoughts and obesrvations on Samza

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

To your broader point, I agree that having a close alignment with Kafka
would be a great thing in terms of adoption/discoverability/etc. There
areas where I think this matters a lot are:
1. Website and docs: ideally when reading about Kafka you should be able to
find out about Samza.
2. Api style and naming: ideally the various interfaces should feel similar
and use similar concepts and names. This is a bunch of little things
(calling topics and partitions in the same way, sharing metrics, sharing
partitioning strategies, etc).
3. Release alignment--i.e. this set of versions all work together.
4. Branding--I actually think if we go down that route it would be worth
considering just calling Samza something like "Kafka Streams" or "Kafka
Streaming" which I think would help a lot people to understand what it is
and since Kafka is heavily adopted would help with adoption. It always
seems silly to bother with naming, but I actually think this ends up
mattering a ton in how people understand the system (I guess as programmers
we kind of all intuitively understand the importance of good naming).

WRT partition mapping, yeah I totally agree. I think in all proposals this
is left pluggable. And I think ideally the same set of assignment
strategies should be available either in the Kafka consumer or in Samza. I
think at this point the only debate is whether this is controlled client
side or server side.

-Jay

On Fri, Jul 3, 2015 at 1:40 AM, Gianmarco De Francisci Morales <
gdfm@apache.org> wrote:

> Hi Jay,
>
> Thanks for your answer.
>
>
> > However a few things have changed since that original design:
> > 1. We now have the additional use cases of copycat and Samza
> > 2. We now realize that the assignment strategies don't actually
> necessarily
> > ensure each partition is assigned to only one consumer--there are really
> > valid use cases for broadcast or multiple replica assignment schemes--so
> we
> > can't actually make the a hard assertion on the server.
> >
> > So it may make sense to revist this, I don't think it is necessarily a
> > massive change and would give more flexibility for the variety of cases.
> >
> > -Jay
>
>
> I totally agree, the 1-partition-1-task mapping is too restrictive.
> However, I think the fundamental operation that Samza, Copycat, and Kafka
> consumers should agree upon is "how can I specify in a simple and
> transparent way which partitions I want to consume, and how?".
> This means providing a mapping from partitions to consumer tasks, possibly
> in a transparent way so as to allow for optimizations in placement,
> co-partitioning, etc...
> This issue has the potential of generating again a lot of duplicate work,
> and I think it should be solved at the Kafka level.
> Given that Copycat and normal consumers are already inside Kafka, I think
> having Samza there as well would simplify things a lot.
> The result is that Kafka would be a complete package for handling streams:
> - Messaging, partitioning, and fault tolerance (Kafka core)
> - Ingestion (Copycat)
> - Lightweight processing (Samza)
> - Coupling with other systems (Kafka consumers)
>
> Cheers,
>
> --
> Gianmarco
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
Hi, Gianmarco,

{quote}
However, I think the fundamental operation that Samza, Copycat, and Kafka
consumers should agree upon is "how can I specify in a simple and
transparent way which partitions I want to consume, and how?".
{quote}

I agree that some basic partition distribution mechanism can be common and
those common use patterns should be provided / solved at Kafka level. I
would argue that a client-side pluggable logic is needed for the following
two reasons:
1. On the broker-side, the system does not have a view on client-side
resource/state (i.e. host-affinity of local state is a good example). When
the partition distribution/assignment needs to take client-side
resource/state into consideration, we need the client-side logic.
2. When we run Samza as a service, their might be additional resource/quote
related policies that requires an application-level decision, which the
information needed for decision is not visible at Kafka-level. In that
case, a pluggable client-side logic is useful.

Thanks!

On Fri, Jul 3, 2015 at 1:40 AM, Gianmarco De Francisci Morales <
gdfm@apache.org> wrote:

> Hi Jay,
>
> Thanks for your answer.
>
>
> > However a few things have changed since that original design:
> > 1. We now have the additional use cases of copycat and Samza
> > 2. We now realize that the assignment strategies don't actually
> necessarily
> > ensure each partition is assigned to only one consumer--there are really
> > valid use cases for broadcast or multiple replica assignment schemes--so
> we
> > can't actually make the a hard assertion on the server.
> >
> > So it may make sense to revist this, I don't think it is necessarily a
> > massive change and would give more flexibility for the variety of cases.
> >
> > -Jay
>
>
> I totally agree, the 1-partition-1-task mapping is too restrictive.
> However, I think the fundamental operation that Samza, Copycat, and Kafka
> consumers should agree upon is "how can I specify in a simple and
> transparent way which partitions I want to consume, and how?".
> This means providing a mapping from partitions to consumer tasks, possibly
> in a transparent way so as to allow for optimizations in placement,
> co-partitioning, etc...
> This issue has the potential of generating again a lot of duplicate work,
> and I think it should be solved at the Kafka level.
> Given that Copycat and normal consumers are already inside Kafka, I think
> having Samza there as well would simplify things a lot.
> The result is that Kafka would be a complete package for handling streams:
> - Messaging, partitioning, and fault tolerance (Kafka core)
> - Ingestion (Copycat)
> - Lightweight processing (Samza)
> - Coupling with other systems (Kafka consumers)
>
> Cheers,
>
> --
> Gianmarco
>

Re: Thoughts and obesrvations on Samza

Posted by Gianmarco De Francisci Morales <gd...@apache.org>.
Hi Jay,

Thanks for your answer.


> However a few things have changed since that original design:
> 1. We now have the additional use cases of copycat and Samza
> 2. We now realize that the assignment strategies don't actually necessarily
> ensure each partition is assigned to only one consumer--there are really
> valid use cases for broadcast or multiple replica assignment schemes--so we
> can't actually make the a hard assertion on the server.
>
> So it may make sense to revist this, I don't think it is necessarily a
> massive change and would give more flexibility for the variety of cases.
>
> -Jay


I totally agree, the 1-partition-1-task mapping is too restrictive.
However, I think the fundamental operation that Samza, Copycat, and Kafka
consumers should agree upon is "how can I specify in a simple and
transparent way which partitions I want to consume, and how?".
This means providing a mapping from partitions to consumer tasks, possibly
in a transparent way so as to allow for optimizations in placement,
co-partitioning, etc...
This issue has the potential of generating again a lot of duplicate work,
and I think it should be solved at the Kafka level.
Given that Copycat and normal consumers are already inside Kafka, I think
having Samza there as well would simplify things a lot.
The result is that Kafka would be a complete package for handling streams:
- Messaging, partitioning, and fault tolerance (Kafka core)
- Ingestion (Copycat)
- Lightweight processing (Samza)
- Coupling with other systems (Kafka consumers)

Cheers,

--
Gianmarco

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@confluent.io>.
Yeah, hey Yi, I get what you are saying now. At the risk of getting into
the weeds a bit you are exactly right, a similar thing is needed for
copycat/kip-26. At the risk of getting a bit into the weeds, I think it may
be possible to rework the assignment feature in the consumer to make this
always be a client-side concern so that Samza, the Kafka consumer, and
Copycat can all use the same facility.

The original design for the kafka coordinator was that the coordinator
would just coordinate *group* membership and the actual assignment of
partitions to members of the group would be done client side. The advantage
of this was that it was more general the disadvantage was that the server
couldn't really check or monitor the partition assignment. Since we didn't
have any other use case for generic group management we went with the more
specific partition assignment.

However a few things have changed since that original design:
1. We now have the additional use cases of copycat and Samza
2. We now realize that the assignment strategies don't actually necessarily
ensure each partition is assigned to only one consumer--there are really
valid use cases for broadcast or multiple replica assignment schemes--so we
can't actually make the a hard assertion on the server.

So it may make sense to revist this, I don't think it is necessarily a
massive change and would give more flexibility for the variety of cases.

-Jay

On Thu, Jul 2, 2015 at 3:38 PM, Yi Pan <ni...@gmail.com> wrote:

> @Guozhang, yes, that's what I meant. From Kafka consumers' point of view,
> it pretty much boils down to answer the following question:
> 1. For Kafka consumer in each container (i.e. a Samza worker), which topic
> partitions it should consume.
> Samza's current StreamTask model still makes sense to me and the
> partition-to-task mapping is the debatable point that whether that should
> be in Kafka or stays in a separate module. As we discussed earlier, some
> simple partition-to-task mapping maybe expressed as co-partition
> distribution among different topics in Kafka (forgive me if I had make
> mistakes here since I am not 100% sure about how partition distribution
> policies work in Kafka). However, more complex and application-specific
> partition-to-task mapping would need to stay outside of Kafka. One example
> is the discussion on SQL tasks:
>
> https://issues.apache.org/jira/browse/SAMZA-516?focusedCommentId=14288685&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14288685
>
> On Thu, Jul 2, 2015 at 2:47 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Since the resource scheduling systems like YARN / Mesos only gives Samza
> a
> > couple of resource units (or "containers") to run processes, while Samza
> > still needs to handle task assignment / scheduling like which tasks
> should
> > be allocated to which containers that consume from which partitions,
> etc. I
> > think this is want Yi meant for "partition management"?
> >
> > On Thu, Jul 2, 2015 at 2:35 PM, Yi Pan <ni...@gmail.com> wrote:
> >
> > > @Jay, yes, the current function in the JobCoordinator is just partition
> > > management. Maybe we should just call it PartitionManager to make it
> > > explicit.
> > >
> > > -Yi
> > >
> > > On Thu, Jul 2, 2015 at 2:24 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Hey Yi,
> > > >
> > > > What does the JobCoordinator do? YARN/Mesos/etc would be doing the
> > actual
> > > > resource assignment, process restart, etc, right? Is the additional
> > value
> > > > add of the JobCoordinator just partition management?
> > > >
> > > > -Jay
> > > >
> > > > On Thu, Jul 2, 2015 at 11:32 AM, Yi Pan <ni...@gmail.com> wrote:
> > > >
> > > > > Hi, all,
> > > > >
> > > > >
> > > > > Thanks Chris for sending out this proposal and Jay for sharing the
> > > > > extremely illustrative prototype code.
> > > > >
> > > > >
> > > > > I have been thinking it over many times and want to list out my
> > > personal
> > > > > opinions below:
> > > > >
> > > > > 1. Generally, I agree with most of the people here on the mailing
> > list
> > > on
> > > > > two points:
> > > > >
> > > > >    a. Deeper integration w/ Kafka is great. No more confusing
> mapping
> > > > from
> > > > > SystemStreamPartition to TopicPartition etc.
> > > > >
> > > > >    b. Separation the ingestion vs transformation greatly simplify
> the
> > > > > systems APIs
> > > > >
> > > > > Having the above two changes would allow us to remove many
> > unnecessary
> > > > > complexities introduced by those pluggable interfaces Chris’
> pointed
> > > out,
> > > > > e.g. pluggable streaming systems and serde.
> > > > >
> > > > >
> > > > > To recall one of Chris’s statement on difficulties in dynamic
> > > > deployment, I
> > > > > believe that the difficulties are mainly the result of
> tight-coupling
> > > of
> > > > > partition assignment vs the container deployment in the current
> > system.
> > > > The
> > > > > current container deployment requires a pre-defined partition
> > > assignment
> > > > > strategy coupled together w/ the deployment configuration before we
> > can
> > > > > submit to YARN and start the Samza container, which makes the
> > launching
> > > > > process super long. Also, fault-tolerance and the embedded
> > > JobCoordinator
> > > > > code in YARN AppMaster is another way of  making dynamic deployment
> > > more
> > > > > complex and difficult.
> > > > >
> > > > >
> > > > > First, borrowing Yan’s term, let’s call the Samza standalone
> process
> > a
> > > > > Samza worker. Here is what I have been thinking:
> > > > >
> > > > > 1. Separate the execution framework from partition assignment/load
> > > > > balancing:
> > > > >
> > > > >     a. a Samza worker should be launched by execution framework
> that
> > > only
> > > > > deals w/ process placement to available nodes. The execution
> > framework
> > > > now
> > > > > should only deal w/ how many such processes are needed, where to
> put
> > > > them,
> > > > > and how to keep them alive.
> > > > >
> > > > >     b. Partition assignment/load balancing can be a pluggable
> > interface
> > > > in
> > > > > Samza that allows the Samza workers to ask for partition
> assignments.
> > > > Let’s
> > > > > borrow the name JobCoordinator for now. To allow fault-tolerance in
> > > case
> > > > of
> > > > > failure, the partition assignments to workers need to be dynamic.
> > > Hence,
> > > > > the abstract interface would be much like what Jay’s code
> illustrate:
> > > > > get()/onAssigned()/onRevoke(). The implementation of the partition
> > > > > assignment can be either:
> > > > >
> > > > >         a) completely rely on Kafka.
> > > > >
> > > > >         b) explicit partition assignment via JobCoordinator.
> Chris’s
> > > work
> > > > > in SAMZA-516 can be easily incorporated here. The use case in
> > SAMZA-41
> > > > that
> > > > > runs Samza ProcessJob w/ static partition assignment can be
> > implemented
> > > > of
> > > > > JobCoordinator via any home-grown implementation of distributed
> > > > > coordinator. All the work we did in LinkedIn to support dynamic
> > > partition
> > > > > assignment and host-affinity SAMZA-617 can be nicely reused as an
> > > > > implementation of JobCoordinator.
> > > > >
> > > > >
> > > > > When we did the above work, I can see three usage patterns in
> Samza:
> > > > >
> > > > >    a. Samza as a library: Samza has a set of libraries to provide
> > > stream
> > > > > processing, just like a third Kafka client (as illustrated in Jay’s
> > > > > example). The execution/deployment is totally controlled by the
> > > > application
> > > > > and the partition coordination is done via Kafka
> > > > >
> > > > >    b. Samza as a process: Samza runs as a standalone process. There
> > may
> > > > not
> > > > > be a execution framework to launch and deploy Samza processes. The
> > > > > partition assignment is pluggable JobCoordinator.
> > > > >
> > > > >    c. Samza as a service: Samza runs as a collection of processes.
> > > There
> > > > > will be an execution framework to allocate resource, launch and
> > deploy
> > > > > Samza workers and keep them alive. The same pluggable
> JobCoordinator
> > is
> > > > > desirable here as well.
> > > > >
> > > > >
> > > > > Lastly, I would argue that CopyCat in KIP-26 should probably follow
> > the
> > > > > same model. Hence, in Samza as a service model as in LinkedIn, we
> can
> > > use
> > > > > the same fault tolerance execution framework to run CopyCat and
> Samza
> > > w/o
> > > > > the need to operate two service platforms, which should address
> > > Sriram’s
> > > > > comment in the email thread.
> > > > >
> > > > >
> > > > > Hope the above makes sense. Thanks all!
> > > > >
> > > > >
> > > > > -Yi
> > > > >
> > > > > On Thu, Jul 2, 2015 at 9:53 AM, Sriram <sr...@gmail.com>
> wrote:
> > > > >
> > > > > > One thing that is worth exploring is to have a transformation and
> > > > > > ingestion library in Kafka but use the same framework for fault
> > > > > tolerance,
> > > > > > resource isolation and management. The biggest difference I see
> in
> > > > these
> > > > > > two use cases is the API and data model.
> > > > > >
> > > > > >
> > > > > > > On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io>
> wrote:
> > > > > > >
> > > > > > > Hey Garry,
> > > > > > >
> > > > > > > Yeah that's super frustrating. I'd be happy to chat more about
> > this
> > > > if
> > > > > > > you'd be interested. I think Chris and I started with the idea
> of
> > > > "what
> > > > > > > would it take to make Samza a kick-ass ingestion tool" but
> > > ultimately
> > > > > we
> > > > > > > kind of came around to the idea that ingestion and
> transformation
> > > had
> > > > > > > pretty different needs and coupling the two made things hard.
> > > > > > >
> > > > > > > For what it's worth I think copycat (KIP-26) actually will do
> > what
> > > > you
> > > > > > are
> > > > > > > looking for.
> > > > > > >
> > > > > > > With regard to your point about slider, I don't necessarily
> > > disagree.
> > > > > > But I
> > > > > > > think getting good YARN support is quite doable and I think we
> > can
> > > > make
> > > > > > > that work well. I think the issue this proposal solves is that
> > > > > > technically
> > > > > > > it is pretty hard to support multiple cluster management
> systems
> > > the
> > > > > way
> > > > > > > things are now, you need to write an "app master" or
> "framework"
> > > for
> > > > > each
> > > > > > > and they are all a little different so testing is really hard.
> In
> > > the
> > > > > > > absence of this we have been stuck with just YARN which has
> > > fantastic
> > > > > > > penetration in the Hadoopy part of the org, but zero
> penetration
> > > > > > elsewhere.
> > > > > > > Given the huge amount of work being put in to slider, marathon,
> > aws
> > > > > > > tooling, not to mention the umpteen related packaging
> > technologies
> > > > > people
> > > > > > > want to use (Docker, Kubernetes, various cloud-specific deploy
> > > tools,
> > > > > > etc)
> > > > > > > I really think it is important to get this right.
> > > > > > >
> > > > > > > -Jay
> > > > > > >
> > > > > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > > > > > g.turkington@improvedigital.com> wrote:
> > > > > > >
> > > > > > >> Hi all,
> > > > > > >>
> > > > > > >> I think the question below re does Samza become a sub-project
> of
> > > > Kafka
> > > > > > >> highlights the broader point around migration. Chris mentions
> > > > Samza's
> > > > > > >> maturity is heading towards a v1 release but I'm not sure it
> > feels
> > > > > > right to
> > > > > > >> launch a v1 then immediately plan to deprecate most of it.
> > > > > > >>
> > > > > > >> From a selfish perspective I have some guys who have started
> > > working
> > > > > > with
> > > > > > >> Samza and building some new consumers/producers was next up.
> > > Sounds
> > > > > like
> > > > > > >> that is absolutely not the direction to go. I need to look
> into
> > > the
> > > > > KIP
> > > > > > in
> > > > > > >> more detail but for me the attractiveness of adding new Samza
> > > > > > >> consumer/producers -- even if yes all they were doing was
> really
> > > > > getting
> > > > > > >> data into and out of Kafka --  was to avoid  having to worry
> > about
> > > > the
> > > > > > >> lifecycle management of external clients. If there is a
> generic
> > > > Kafka
> > > > > > >> ingress/egress layer that I can plug a new connector into and
> > > have a
> > > > > > lot of
> > > > > > >> the heavy lifting re scale and reliability done for me then it
> > > gives
> > > > > me
> > > > > > all
> > > > > > >> the pushing new consumers/producers would. If not then it
> > > > complicates
> > > > > my
> > > > > > >> operational deployments.
> > > > > > >>
> > > > > > >> Which is similar to my other question with the proposal -- if
> we
> > > > > build a
> > > > > > >> fully available/stand-alone Samza plus the requisite shims to
> > > > > integrate
> > > > > > >> with Slider etc I suspect the former may be a lot more work
> than
> > > we
> > > > > > think.
> > > > > > >> We may make it much easier for a newcomer to get something
> > running
> > > > but
> > > > > > >> having them step up and get a reliable production deployment
> may
> > > > still
> > > > > > >> dominate mailing list  traffic, if for different reasons than
> > > today.
> > > > > > >>
> > > > > > >> Don't get me wrong -- I'm comfortable with making the Samza
> > > > dependency
> > > > > > on
> > > > > > >> Kafka much more explicit and I absolutely see the benefits  in
> > the
> > > > > > >> reduction of duplication and clashing
> terminologies/abstractions
> > > > that
> > > > > > >> Chris/Jay describe. Samza as a library would likely be a very
> > nice
> > > > > tool
> > > > > > to
> > > > > > >> add to the Kafka ecosystem. I just have the concerns above re
> > the
> > > > > > >> operational side.
> > > > > > >>
> > > > > > >> Garry
> > > > > > >>
> > > > > > >> -----Original Message-----
> > > > > > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> > > > > > >> Sent: 02 July 2015 12:56
> > > > > > >> To: dev@samza.apache.org
> > > > > > >> Subject: Re: Thoughts and obesrvations on Samza
> > > > > > >>
> > > > > > >> Very interesting thoughts.
> > > > > > >> From outside, I have always perceived Samza as a computing
> layer
> > > > over
> > > > > > >> Kafka.
> > > > > > >>
> > > > > > >> The question, maybe a bit provocative, is "should Samza be a
> > > > > sub-project
> > > > > > >> of Kafka then?"
> > > > > > >> Or does it make sense to keep it as a separate project with a
> > > > separate
> > > > > > >> governance?
> > > > > > >>
> > > > > > >> Cheers,
> > > > > > >>
> > > > > > >> --
> > > > > > >> Gianmarco
> > > > > > >>
> > > > > > >>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> > wrote:
> > > > > > >>>
> > > > > > >>> Overall, I agree to couple with Kafka more tightly. Because
> > Samza
> > > > de
> > > > > > >>> facto is based on Kafka, and it should leverage what Kafka
> has.
> > > At
> > > > > the
> > > > > > >>> same time, Kafka does not need to reinvent what Samza already
> > > has.
> > > > I
> > > > > > >>> also like the idea of separating the ingestion and
> > > transformation.
> > > > > > >>>
> > > > > > >>> But it is a little difficult for me to image how the Samza
> will
> > > > look
> > > > > > >> like.
> > > > > > >>> And I feel Chris and Jay have a little difference in terms of
> > how
> > > > > > >>> Samza should look like.
> > > > > > >>>
> > > > > > >>> *** Will it look like what Jay's code shows (A client of
> > Kakfa) ?
> > > > And
> > > > > > >>> user's application code calls this client?
> > > > > > >>>
> > > > > > >>> 1. If we make Samza be a library of Kafka (like what the code
> > > > shows),
> > > > > > >>> how do we implement auto-balance and fault-tolerance? Are
> they
> > > > taken
> > > > > > >>> care by the Kafka broker or other mechanism, such as "Samza
> > > worker"
> > > > > > >>> (just make up the name) ?
> > > > > > >>>
> > > > > > >>> 2. What about other features, such as auto-scaling, shared
> > state,
> > > > > > >>> monitoring?
> > > > > > >>>
> > > > > > >>>
> > > > > > >>> *** If we have Samza standalone, (is this what Chris
> suggests?)
> > > > > > >>>
> > > > > > >>> 1. we still need to ingest data from Kakfa and produce to it.
> > > Then
> > > > it
> > > > > > >>> becomes the same as what Samza looks like now, except it does
> > not
> > > > > rely
> > > > > > >>> on Yarn anymore.
> > > > > > >>>
> > > > > > >>> 2. if it is standalone, how can it leverage Kafka's metrics,
> > > logs,
> > > > > > >>> etc? Use Kafka code as the dependency?
> > > > > > >>>
> > > > > > >>>
> > > > > > >>> Thanks,
> > > > > > >>>
> > > > > > >>> Fang, Yan
> > > > > > >>> yanfang724@gmail.com
> > > > > > >>>
> > > > > > >>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > > wangguoz@gmail.com
> > > > >
> > > > > > >>> wrote:
> > > > > > >>>
> > > > > > >>>> Read through the code example and it looks good to me. A few
> > > > > > >>>> thoughts regarding deployment:
> > > > > > >>>>
> > > > > > >>>> Today Samza deploys as executable runnable like:
> > > > > > >>>>
> > > > > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
> > > > > > >> --config-path=file://...
> > > > > > >>>>
> > > > > > >>>> And this proposal advocate for deploying Samza more as
> > embedded
> > > > > > >>>> libraries in user application code (ignoring the terminology
> > > since
> > > > > > >>>> it is not the
> > > > > > >>> same
> > > > > > >>>> as the prototype code):
> > > > > > >>>>
> > > > > > >>>> StreamTask task = new MyStreamTask(configs); Thread thread =
> > new
> > > > > > >>>> Thread(task); thread.start();
> > > > > > >>>>
> > > > > > >>>> I think both of these deployment modes are important for
> > > different
> > > > > > >>>> types
> > > > > > >>> of
> > > > > > >>>> users. That said, I think making Samza purely standalone is
> > > still
> > > > > > >>>> sufficient for either runnable or library modes.
> > > > > > >>>>
> > > > > > >>>> Guozhang
> > > > > > >>>>
> > > > > > >>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> > jay@confluent.io>
> > > > > > wrote:
> > > > > > >>>>>
> > > > > > >>>>> Looks like gmail mangled the code example, it was supposed
> to
> > > > look
> > > > > > >>>>> like
> > > > > > >>>>> this:
> > > > > > >>>>>
> > > > > > >>>>> Properties props = new Properties();
> > > > > > >>>>> props.put("bootstrap.servers", "localhost:4242");
> > > StreamingConfig
> > > > > > >>>>> config = new StreamingConfig(props);
> > > > > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
> > > > > > >>>>> config.processor(ExampleStreamProcessor.class);
> > > > > > >>>>> config.serialization(new StringSerializer(), new
> > > > > > >>>>> StringDeserializer()); KafkaStreaming container = new
> > > > > > >>>>> KafkaStreaming(config); container.run();
> > > > > > >>>>>
> > > > > > >>>>> -Jay
> > > > > > >>>>>
> > > > > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> > jay@confluent.io>
> > > > > > >> wrote:
> > > > > > >>>>>
> > > > > > >>>>>> Hey guys,
> > > > > > >>>>>>
> > > > > > >>>>>> This came out of some conversations Chris and I were
> having
> > > > > > >>>>>> around
> > > > > > >>>>> whether
> > > > > > >>>>>> it would make sense to use Samza as a kind of data
> ingestion
> > > > > > >>> framework
> > > > > > >>>>> for
> > > > > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This
> kind
> > > of
> > > > > > >>>> combined
> > > > > > >>>>>> with complaints around config and YARN and the discussion
> > > around
> > > > > > >>>>>> how
> > > > > > >>> to
> > > > > > >>>>>> best do a standalone mode.
> > > > > > >>>>>>
> > > > > > >>>>>> So the thought experiment was, given that Samza was
> > basically
> > > > > > >>>>>> already totally Kafka specific, what if you just embraced
> > that
> > > > > > >>>>>> and turned it
> > > > > > >>>> into
> > > > > > >>>>>> something less like a heavyweight framework and more like
> a
> > > > > > >>>>>> third
> > > > > > >>> Kafka
> > > > > > >>>>>> client--a kind of "producing consumer" with state
> management
> > > > > > >>>> facilities.
> > > > > > >>>>>> Basically a library. Instead of a complex stream
> processing
> > > > > > >>>>>> framework
> > > > > > >>>>> this
> > > > > > >>>>>> would actually be a very simple thing, not much more
> > > complicated
> > > > > > >>>>>> to
> > > > > > >>> use
> > > > > > >>>>> or
> > > > > > >>>>>> operate than a Kafka consumer. As Chris said we thought
> > about
> > > it
> > > > > > >>>>>> a
> > > > > > >>> lot
> > > > > > >>>> of
> > > > > > >>>>>> what Samza (and the other stream processing systems were
> > > doing)
> > > > > > >>> seemed
> > > > > > >>>>> like
> > > > > > >>>>>> kind of a hangover from MapReduce.
> > > > > > >>>>>>
> > > > > > >>>>>> Of course you need to ingest/output data to and from the
> > > stream
> > > > > > >>>>>> processing. But when we actually looked into how that
> would
> > > > > > >>>>>> work,
> > > > > > >>> Samza
> > > > > > >>>>>> isn't really an ideal data ingestion framework for a bunch
> > of
> > > > > > >>> reasons.
> > > > > > >>>> To
> > > > > > >>>>>> really do that right you need a pretty different internal
> > data
> > > > > > >>>>>> model
> > > > > > >>>> and
> > > > > > >>>>>> set of apis. So what if you split them and had an api for
> > > Kafka
> > > > > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
> > > Kafka
> > > > > > >>>>>> transformation (Samza).
> > > > > > >>>>>>
> > > > > > >>>>>> This would also allow really embracing the same
> terminology
> > > and
> > > > > > >>>>>> conventions. One complaint about the current state is that
> > the
> > > > > > >>>>>> two
> > > > > > >>>>> systems
> > > > > > >>>>>> kind of feel bolted on. Terminology like "stream" vs
> "topic"
> > > and
> > > > > > >>>>> different
> > > > > > >>>>>> config and monitoring systems means you kind of have to
> > learn
> > > > > > >>>>>> Kafka's
> > > > > > >>>>> way,
> > > > > > >>>>>> then learn Samza's slightly different way, then kind of
> > > > > > >>>>>> understand
> > > > > > >>> how
> > > > > > >>>>> they
> > > > > > >>>>>> map to each other, which having walked a few people
> through
> > > this
> > > > > > >>>>>> is surprisingly tricky for folks to get.
> > > > > > >>>>>>
> > > > > > >>>>>> Since I have been spending a lot of time on airplanes I
> > hacked
> > > > > > >>>>>> up an ernest but still somewhat incomplete prototype of
> what
> > > > > > >>>>>> this would
> > > > > > >>> look
> > > > > > >>>>>> like. This is just unceremoniously dumped into Kafka as it
> > > > > > >>>>>> required a
> > > > > > >>>> few
> > > > > > >>>>>> changes to the new consumer. Here is the code:
> > > > > > >>>
> > > > >
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > > > >>> /apache/kafka/clients/streaming
> > > > > > >>>>>>
> > > > > > >>>>>> For the purpose of the prototype I just liberally renamed
> > > > > > >>>>>> everything
> > > > > > >>> to
> > > > > > >>>>>> try to align it with Kafka with no regard for
> compatibility.
> > > > > > >>>>>>
> > > > > > >>>>>> To use this would be something like this:
> > > > > > >>>>>> Properties props = new Properties();
> > > > > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > > > > >>>>>> StreamingConfig config = new
> > > > > > >>> StreamingConfig(props);
> > > > > > >>>>> config.subscribe("test-topic-1",
> > > > > > >>>>>> "test-topic-2");
> > > config.processor(ExampleStreamProcessor.class);
> > > > > > >>>>> config.serialization(new
> > > > > > >>>>>> StringSerializer(), new StringDeserializer());
> > KafkaStreaming
> > > > > > >>>> container =
> > > > > > >>>>>> new KafkaStreaming(config); container.run();
> > > > > > >>>>>>
> > > > > > >>>>>> KafkaStreaming is basically the SamzaContainer;
> > > StreamProcessor
> > > > > > >>>>>> is basically StreamTask.
> > > > > > >>>>>>
> > > > > > >>>>>> So rather than putting all the class names in a file and
> > then
> > > > > > >>>>>> having
> > > > > > >>>> the
> > > > > > >>>>>> job assembled by reflection, you just instantiate the
> > > container
> > > > > > >>>>>> programmatically. Work is balanced over however many
> > instances
> > > > > > >>>>>> of
> > > > > > >>> this
> > > > > > >>>>> are
> > > > > > >>>>>> alive at any time (i.e. if an instance dies, new tasks are
> > > added
> > > > > > >>>>>> to
> > > > > > >>> the
> > > > > > >>>>>> existing containers without shutting them down).
> > > > > > >>>>>>
> > > > > > >>>>>> We would provide some glue for running this stuff in YARN
> > via
> > > > > > >>>>>> Slider, Mesos via Marathon, and AWS using some of their
> > tools
> > > > > > >>>>>> but from the
> > > > > > >>>> point
> > > > > > >>>>> of
> > > > > > >>>>>> view of these frameworks these stream processing jobs are
> > just
> > > > > > >>>> stateless
> > > > > > >>>>>> services that can come and go and expand and contract at
> > will.
> > > > > > >>>>>> There
> > > > > > >>> is
> > > > > > >>>>> no
> > > > > > >>>>>> more custom scheduler.
> > > > > > >>>>>>
> > > > > > >>>>>> Here are some relevant details:
> > > > > > >>>>>>
> > > > > > >>>>>>   1. It is only ~1300 lines of code, it would get larger
> if
> > we
> > > > > > >>>>>>   productionized but not vastly larger. We really do get a
> > ton
> > > > > > >>>>>> of
> > > > > > >>>>> leverage
> > > > > > >>>>>>   out of Kafka.
> > > > > > >>>>>>   2. Partition management is fully delegated to the new
> > > > consumer.
> > > > > > >>> This
> > > > > > >>>>>>   is nice since now any partition management strategy
> > > available
> > > > > > >>>>>> to
> > > > > > >>>> Kafka
> > > > > > >>>>>>   consumer is also available to Samza (and vice versa) and
> > > with
> > > > > > >>>>>> the
> > > > > > >>>>> exact
> > > > > > >>>>>>   same configs.
> > > > > > >>>>>>   3. It supports state as well as state reuse
> > > > > > >>>>>>
> > > > > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
> > > > > > >>>>>>
> > > > > > >>>>>> -Jay
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > > > > >>>> criccomini@apache.org>
> > > > > > >>>>>> wrote:
> > > > > > >>>>>>
> > > > > > >>>>>>> Hey all,
> > > > > > >>>>>>>
> > > > > > >>>>>>> I have had some discussions with Samza engineers at
> > LinkedIn
> > > > > > >>>>>>> and
> > > > > > >>>>> Confluent
> > > > > > >>>>>>> and we came up with a few observations and would like to
> > > > > > >>>>>>> propose
> > > > > > >>> some
> > > > > > >>>>>>> changes.
> > > > > > >>>>>>>
> > > > > > >>>>>>> We've observed some things that I want to call out about
> > > > > > >>>>>>> Samza's
> > > > > > >>>> design,
> > > > > > >>>>>>> and I'd like to propose some changes.
> > > > > > >>>>>>>
> > > > > > >>>>>>> * Samza is dependent upon a dynamic deployment system.
> > > > > > >>>>>>> * Samza is too pluggable.
> > > > > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> > consumer
> > > > > > >>>>>>> APIs
> > > > > > >>> are
> > > > > > >>>>>>> trying to solve a lot of the same problems.
> > > > > > >>>>>>>
> > > > > > >>>>>>> All three of these issues are related, but I'll address
> > them
> > > in
> > > > > > >>> order.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Deployment
> > > > > > >>>>>>>
> > > > > > >>>>>>> Samza strongly depends on the use of a dynamic deployment
> > > > > > >>>>>>> scheduler
> > > > > > >>>> such
> > > > > > >>>>>>> as
> > > > > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet
> > that
> > > > > > >>>>>>> there
> > > > > > >>>> would
> > > > > > >>>>>>> be
> > > > > > >>>>>>> one or two winners in this area, and we could support
> them,
> > > and
> > > > > > >>>>>>> the
> > > > > > >>>> rest
> > > > > > >>>>>>> would go away. In reality, there are many variations.
> > > > > > >>>>>>> Furthermore,
> > > > > > >>>> many
> > > > > > >>>>>>> people still prefer to just start their processors like
> > > normal
> > > > > > >>>>>>> Java processes, and use traditional deployment scripts
> such
> > > as
> > > > > > >>>>>>> Fabric,
> > > > > > >>>> Chef,
> > > > > > >>>>>>> Ansible, etc. Forcing a deployment system on users makes
> > the
> > > > > > >>>>>>> Samza start-up process really painful for first time
> users.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Dynamic deployment as a requirement was also a bit of a
> > > > > > >>>>>>> mis-fire
> > > > > > >>>> because
> > > > > > >>>>>>> of
> > > > > > >>>>>>> a fundamental misunderstanding between the nature of
> batch
> > > jobs
> > > > > > >>>>>>> and
> > > > > > >>>>> stream
> > > > > > >>>>>>> processing jobs. Early on, we made conscious effort to
> > favor
> > > > > > >>>>>>> the
> > > > > > >>>> Hadoop
> > > > > > >>>>>>> (Map/Reduce) way of doing things, since it worked and was
> > > well
> > > > > > >>>>> understood.
> > > > > > >>>>>>> One thing that we missed was that batch jobs have a
> > definite
> > > > > > >>>> beginning,
> > > > > > >>>>>>> and
> > > > > > >>>>>>> end, and stream processing jobs don't (usually). This
> leads
> > > to
> > > > > > >>>>>>> a
> > > > > > >>> much
> > > > > > >>>>>>> simpler scheduling problem for stream processors. You
> > > basically
> > > > > > >>>>>>> just
> > > > > > >>>>> need
> > > > > > >>>>>>> to find a place to start the processor, and start it. The
> > way
> > > > > > >>>>>>> we run grids, at LinkedIn, there's no concept of a
> cluster
> > > > > > >>>>>>> being "full". We always
> > > > > > >>>> add
> > > > > > >>>>>>> more machines. The problem with coupling Samza with a
> > > scheduler
> > > > > > >>>>>>> is
> > > > > > >>>> that
> > > > > > >>>>>>> Samza (as a framework) now has to handle deployment. This
> > > pulls
> > > > > > >>>>>>> in a
> > > > > > >>>>> bunch
> > > > > > >>>>>>> of things such as configuration distribution (config
> > stream),
> > > > > > >>>>>>> shell
> > > > > > >>>>> scrips
> > > > > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> > stuff),
> > > > etc.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Another reason for requiring dynamic deployment was to
> > > support
> > > > > > >>>>>>> data locality. If you want to have locality, you need to
> > put
> > > > > > >>>>>>> your
> > > > > > >>>> processors
> > > > > > >>>>>>> close to the data they're processing. Upon further
> > > > > > >>>>>>> investigation,
> > > > > > >>>>> though,
> > > > > > >>>>>>> this feature is not that beneficial. There is some good
> > > > > > >>>>>>> discussion
> > > > > > >>>> about
> > > > > > >>>>>>> some problems with it on SAMZA-335. Again, we took the
> > > > > > >>>>>>> Map/Reduce
> > > > > > >>>> path,
> > > > > > >>>>>>> but
> > > > > > >>>>>>> there are some fundamental differences between HDFS and
> > > Kafka.
> > > > > > >>>>>>> HDFS
> > > > > > >>>> has
> > > > > > >>>>>>> blocks, while Kafka has partitions. This leads to less
> > > > > > >>>>>>> optimization potential with stream processors on top of
> > > Kafka.
> > > > > > >>>>>>>
> > > > > > >>>>>>> This feature is also used as a crutch. Samza doesn't have
> > any
> > > > > > >>>>>>> built
> > > > > > >>> in
> > > > > > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> > > > > > >>>>>>> deployment scheduling system to handle restarts when a
> > > > > > >>>>>>> processor dies. This has
> > > > > > >>>>> made
> > > > > > >>>>>>> it very difficult to write a standalone Samza container
> > > > > > >> (SAMZA-516).
> > > > > > >>>>>>>
> > > > > > >>>>>>> Pluggability
> > > > > > >>>>>>>
> > > > > > >>>>>>> In some cases pluggability is good, but I think that
> we've
> > > gone
> > > > > > >>>>>>> too
> > > > > > >>>> far
> > > > > > >>>>>>> with it. Currently, Samza has:
> > > > > > >>>>>>>
> > > > > > >>>>>>> * Pluggable config.
> > > > > > >>>>>>> * Pluggable metrics.
> > > > > > >>>>>>> * Pluggable deployment systems.
> > > > > > >>>>>>> * Pluggable streaming systems (SystemConsumer,
> > > SystemProducer,
> > > > > > >> etc).
> > > > > > >>>>>>> * Pluggable serdes.
> > > > > > >>>>>>> * Pluggable storage engines.
> > > > > > >>>>>>> * Pluggable strategies for just about every component
> > > > > > >>> (MessageChooser,
> > > > > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > > > > >>>>>>>
> > > > > > >>>>>>> There's probably more that I've forgotten, as well. Some
> of
> > > > > > >>>>>>> these
> > > > > > >>> are
> > > > > > >>>>>>> useful, but some have proven not to be. This all comes
> at a
> > > > cost:
> > > > > > >>>>>>> complexity. This complexity is making it harder for our
> > users
> > > > > > >>>>>>> to
> > > > > > >>> pick
> > > > > > >>>> up
> > > > > > >>>>>>> and use Samza out of the box. It also makes it difficult
> > for
> > > > > > >>>>>>> Samza developers to reason about what the characteristics
> > of
> > > > > > >>>>>>> the container (since the characteristics change depending
> > on
> > > > > > >>>>>>> which plugins are use).
> > > > > > >>>>>>>
> > > > > > >>>>>>> The issues with pluggability are most visible in the
> System
> > > > APIs.
> > > > > > >>> What
> > > > > > >>>>>>> Samza really requires to be functional is Kafka as its
> > > > > > >>>>>>> transport
> > > > > > >>>> layer.
> > > > > > >>>>>>> But
> > > > > > >>>>>>> we've conflated two unrelated use cases into one API:
> > > > > > >>>>>>>
> > > > > > >>>>>>> 1. Get data into/out of Kafka.
> > > > > > >>>>>>> 2. Process the data in Kafka.
> > > > > > >>>>>>>
> > > > > > >>>>>>> The current System API supports both of these use cases.
> > The
> > > > > > >>>>>>> problem
> > > > > > >>>> is,
> > > > > > >>>>>>> we
> > > > > > >>>>>>> actually want different features for each use case. By
> > > papering
> > > > > > >>>>>>> over
> > > > > > >>>>> these
> > > > > > >>>>>>> two use cases, and providing a single API, we've
> > introduced a
> > > > > > >>>>>>> ton of
> > > > > > >>>>> leaky
> > > > > > >>>>>>> abstractions.
> > > > > > >>>>>>>
> > > > > > >>>>>>> For example, what we'd really like in (2) is to have
> > > > > > >>>>>>> monotonically increasing longs for offsets (like Kafka).
> > This
> > > > > > >>>>>>> would be at odds
> > > > > > >>> with
> > > > > > >>>>> (1),
> > > > > > >>>>>>> though, since different systems have different
> > > > > > >>>>> SCNs/Offsets/UUIDs/vectors.
> > > > > > >>>>>>> There was discussion both on the mailing list and the SQL
> > > JIRAs
> > > > > > >>> about
> > > > > > >>>>> the
> > > > > > >>>>>>> need for this.
> > > > > > >>>>>>>
> > > > > > >>>>>>> The same thing holds true for replayability. Kafka allows
> > us
> > > to
> > > > > > >>> rewind
> > > > > > >>>>>>> when
> > > > > > >>>>>>> we have a failure. Many other systems don't. In some
> cases,
> > > > > > >>>>>>> systems
> > > > > > >>>>> return
> > > > > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> > because
> > > > > > >>>>>>> they
> > > > > > >>>> have
> > > > > > >>>>> no
> > > > > > >>>>>>> offsets.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Partitioning is another example. Kafka supports
> > partitioning,
> > > > > > >>>>>>> but
> > > > > > >>> many
> > > > > > >>>>>>> systems don't. We model this by having a single partition
> > for
> > > > > > >>>>>>> those systems. Still, other systems model partitioning
> > > > > > >> differently (e.g.
> > > > > > >>>>>>> Kinesis).
> > > > > > >>>>>>>
> > > > > > >>>>>>> The SystemAdmin interface is also a mess. Creating
> streams
> > > in a
> > > > > > >>>>>>> system-agnostic way is almost impossible. As is modeling
> > > > > > >>>>>>> metadata
> > > > > > >>> for
> > > > > > >>>>> the
> > > > > > >>>>>>> system (replication factor, partitions, location, etc).
> The
> > > > > > >>>>>>> list
> > > > > > >>> goes
> > > > > > >>>>> on.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Duplicate work
> > > > > > >>>>>>>
> > > > > > >>>>>>> At the time that we began writing Samza, Kafka's consumer
> > and
> > > > > > >>> producer
> > > > > > >>>>>>> APIs
> > > > > > >>>>>>> had a relatively weak feature set. On the consumer-side,
> > you
> > > > > > >>>>>>> had two
> > > > > > >>>>>>> options: use the high level consumer, or the simple
> > consumer.
> > > > > > >>>>>>> The
> > > > > > >>>>> problem
> > > > > > >>>>>>> with the high-level consumer was that it controlled your
> > > > > > >>>>>>> offsets, partition assignments, and the order in which
> you
> > > > > > >>>>>>> received messages. The
> > > > > > >>> problem
> > > > > > >>>>>>> with
> > > > > > >>>>>>> the simple consumer is that it's not simple. It's basic.
> > You
> > > > > > >>>>>>> end up
> > > > > > >>>>> having
> > > > > > >>>>>>> to handle a lot of really low-level stuff that you
> > shouldn't.
> > > > > > >>>>>>> We
> > > > > > >>>> spent a
> > > > > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> > robust.
> > > It
> > > > > > >>>>>>> also allows us to support some cool features:
> > > > > > >>>>>>>
> > > > > > >>>>>>> * Per-partition message ordering and prioritization.
> > > > > > >>>>>>> * Tight control over partition assignment to support
> joins,
> > > > > > >>>>>>> global
> > > > > > >>>> state
> > > > > > >>>>>>> (if we want to implement it :)), etc.
> > > > > > >>>>>>> * Tight control over offset checkpointing.
> > > > > > >>>>>>>
> > > > > > >>>>>>> What we didn't realize at the time is that these features
> > > > > > >>>>>>> should
> > > > > > >>>>> actually
> > > > > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
> > stream
> > > > > > >>>> processors)
> > > > > > >>>>>>> end up wanting to do things like joins and partition
> > > > > > >>>>>>> assignment. The
> > > > > > >>>>> Kafka
> > > > > > >>>>>>> community has come to the same conclusion. They're
> adding a
> > > ton
> > > > > > >>>>>>> of upgrades into their new Kafka consumer implementation.
> > To
> > > a
> > > > > > >>>>>>> large extent,
> > > > > > >>> it's
> > > > > > >>>>>>> duplicate work to what we've already done in Samza.
> > > > > > >>>>>>>
> > > > > > >>>>>>> On top of this, Kafka ended up taking a very similar
> > approach
> > > > > > >>>>>>> to
> > > > > > >>>> Samza's
> > > > > > >>>>>>> KafkaCheckpointManager implementation for handling offset
> > > > > > >>>> checkpointing.
> > > > > > >>>>>>> Like Samza, Kafka's new offset management feature stores
> > > offset
> > > > > > >>>>>>> checkpoints in a topic, and allows you to fetch them from
> > the
> > > > > > >>>>>>> broker.
> > > > > > >>>>>>>
> > > > > > >>>>>>> A lot of this seems like a waste, since we could have
> > shared
> > > > > > >>>>>>> the
> > > > > > >>> work
> > > > > > >>>> if
> > > > > > >>>>>>> it
> > > > > > >>>>>>> had been done in Kafka from the get-go.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Vision
> > > > > > >>>>>>>
> > > > > > >>>>>>> All of this leads me to a rather radical proposal. Samza
> is
> > > > > > >>> relatively
> > > > > > >>>>>>> stable at this point. I'd venture to say that we're near
> a
> > > 1.0
> > > > > > >>>> release.
> > > > > > >>>>>>> I'd
> > > > > > >>>>>>> like to propose that we take what we've learned, and
> begin
> > > > > > >>>>>>> thinking
> > > > > > >>>>> about
> > > > > > >>>>>>> Samza beyond 1.0. What would we change if we were
> starting
> > > from
> > > > > > >>>> scratch?
> > > > > > >>>>>>> My
> > > > > > >>>>>>> proposal is to:
> > > > > > >>>>>>>
> > > > > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > > > > > >>>>>>> processors, and eliminate all direct dependences on YARN,
> > > > Mesos,
> > > > > > >> etc.
> > > > > > >>>>>>> 2. Make a definitive call to support only Kafka as the
> > stream
> > > > > > >>>> processing
> > > > > > >>>>>>> layer.
> > > > > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> > > > > > >>>>>>> config
> > > > > > >>>>> systems,
> > > > > > >>>>>>> and simply use Kafka's instead.
> > > > > > >>>>>>>
> > > > > > >>>>>>> This would fix all of the issues that I outlined above.
> It
> > > > > > >>>>>>> should
> > > > > > >>> also
> > > > > > >>>>>>> shrink the Samza code base pretty dramatically.
> Supporting
> > > only
> > > > > > >>>>>>> a standalone container will allow Samza to be executed on
> > > YARN
> > > > > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most
> > other
> > > > > > >>>>>>> in-house
> > > > > > >>>>> deployment
> > > > > > >>>>>>> systems. This should make life a lot easier for new
> users.
> > > > > > >>>>>>> Imagine
> > > > > > >>>>> having
> > > > > > >>>>>>> the hello-samza tutorial without YARN. The drop in
> mailing
> > > list
> > > > > > >>>> traffic
> > > > > > >>>>>>> will be pretty dramatic.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Coupling with Kafka seems long overdue to me. The reality
> > is,
> > > > > > >>> everyone
> > > > > > >>>>>>> that
> > > > > > >>>>>>> I'm aware of is using Samza with Kafka. We basically
> > require
> > > it
> > > > > > >>>> already
> > > > > > >>>>> in
> > > > > > >>>>>>> order for most features to work. Those that are using
> other
> > > > > > >>>>>>> systems
> > > > > > >>>> are
> > > > > > >>>>>>> generally using it for ingest into Kafka (1), and then
> they
> > > do
> > > > > > >>>>>>> the processing on top. There is already discussion (
> > > > > > >>>
> > > > >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > > > >>> 767
> > > > > > >>>>>>> )
> > > > > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Once we make the call to couple with Kafka, we can
> > leverage a
> > > > > > >>>>>>> ton of
> > > > > > >>>>> their
> > > > > > >>>>>>> ecosystem. We no longer have to maintain our own config,
> > > > > > >>>>>>> metrics,
> > > > > > >>> etc.
> > > > > > >>>>> We
> > > > > > >>>>>>> can all share the same libraries, and make them better.
> > This
> > > > > > >>>>>>> will
> > > > > > >>> also
> > > > > > >>>>>>> allow us to share the consumer/producer APIs, and will
> let
> > us
> > > > > > >>> leverage
> > > > > > >>>>>>> their offset management and partition management, rather
> > than
> > > > > > >>>>>>> having
> > > > > > >>>> our
> > > > > > >>>>>>> own. All of the coordinator stream code would go away, as
> > > would
> > > > > > >>>>>>> most
> > > > > > >>>> of
> > > > > > >>>>>>> the
> > > > > > >>>>>>> YARN AppMaster code. We'd probably have to push some
> > > partition
> > > > > > >>>>> management
> > > > > > >>>>>>> features into the Kafka broker, but they're already
> moving
> > in
> > > > > > >>>>>>> that direction with the new consumer API. The features we
> > > have
> > > > > > >>>>>>> for
> > > > > > >>>> partition
> > > > > > >>>>>>> assignment aren't unique to Samza, and seem like they
> > should
> > > be
> > > > > > >>>>>>> in
> > > > > > >>>> Kafka
> > > > > > >>>>>>> anyway. There will always be some niche usages which will
> > > > > > >>>>>>> require
> > > > > > >>>> extra
> > > > > > >>>>>>> care and hence full control over partition assignments
> much
> > > > > > >>>>>>> like the
> > > > > > >>>>> Kafka
> > > > > > >>>>>>> low level consumer api. These would continue to be
> > supported.
> > > > > > >>>>>>>
> > > > > > >>>>>>> These items will be good for the Samza community. They'll
> > > make
> > > > > > >>>>>>> Samza easier to use, and make it easier for developers to
> > add
> > > > > > >>>>>>> new features.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Obviously this is a fairly large (and somewhat backwards
> > > > > > >>> incompatible
> > > > > > >>>>>>> change). If we choose to go this route, it's important
> that
> > > we
> > > > > > >>> openly
> > > > > > >>>>>>> communicate how we're going to provide a migration path
> > from
> > > > > > >>>>>>> the
> > > > > > >>>>> existing
> > > > > > >>>>>>> APIs to the new ones (if we make incompatible changes). I
> > > think
> > > > > > >>>>>>> at a minimum, we'd probably need to provide a wrapper to
> > > allow
> > > > > > >>>>>>> existing StreamTask implementations to continue running
> on
> > > the
> > > > > > >> new container.
> > > > > > >>>>> It's
> > > > > > >>>>>>> also important that we openly communicate about timing,
> and
> > > > > > >>>>>>> stages
> > > > > > >>> of
> > > > > > >>>>> the
> > > > > > >>>>>>> migration.
> > > > > > >>>>>>>
> > > > > > >>>>>>> If you made it this far, I'm sure you have opinions. :)
> > > Please
> > > > > > >>>>>>> send
> > > > > > >>>> your
> > > > > > >>>>>>> thoughts and feedback.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Cheers,
> > > > > > >>>>>>> Chris
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>>
> > > > > > >>>> --
> > > > > > >>>> -- Guozhang
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
@Guozhang, yes, that's what I meant. From Kafka consumers' point of view,
it pretty much boils down to answer the following question:
1. For Kafka consumer in each container (i.e. a Samza worker), which topic
partitions it should consume.
Samza's current StreamTask model still makes sense to me and the
partition-to-task mapping is the debatable point that whether that should
be in Kafka or stays in a separate module. As we discussed earlier, some
simple partition-to-task mapping maybe expressed as co-partition
distribution among different topics in Kafka (forgive me if I had make
mistakes here since I am not 100% sure about how partition distribution
policies work in Kafka). However, more complex and application-specific
partition-to-task mapping would need to stay outside of Kafka. One example
is the discussion on SQL tasks:
https://issues.apache.org/jira/browse/SAMZA-516?focusedCommentId=14288685&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14288685

On Thu, Jul 2, 2015 at 2:47 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Since the resource scheduling systems like YARN / Mesos only gives Samza a
> couple of resource units (or "containers") to run processes, while Samza
> still needs to handle task assignment / scheduling like which tasks should
> be allocated to which containers that consume from which partitions, etc. I
> think this is want Yi meant for "partition management"?
>
> On Thu, Jul 2, 2015 at 2:35 PM, Yi Pan <ni...@gmail.com> wrote:
>
> > @Jay, yes, the current function in the JobCoordinator is just partition
> > management. Maybe we should just call it PartitionManager to make it
> > explicit.
> >
> > -Yi
> >
> > On Thu, Jul 2, 2015 at 2:24 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Hey Yi,
> > >
> > > What does the JobCoordinator do? YARN/Mesos/etc would be doing the
> actual
> > > resource assignment, process restart, etc, right? Is the additional
> value
> > > add of the JobCoordinator just partition management?
> > >
> > > -Jay
> > >
> > > On Thu, Jul 2, 2015 at 11:32 AM, Yi Pan <ni...@gmail.com> wrote:
> > >
> > > > Hi, all,
> > > >
> > > >
> > > > Thanks Chris for sending out this proposal and Jay for sharing the
> > > > extremely illustrative prototype code.
> > > >
> > > >
> > > > I have been thinking it over many times and want to list out my
> > personal
> > > > opinions below:
> > > >
> > > > 1. Generally, I agree with most of the people here on the mailing
> list
> > on
> > > > two points:
> > > >
> > > >    a. Deeper integration w/ Kafka is great. No more confusing mapping
> > > from
> > > > SystemStreamPartition to TopicPartition etc.
> > > >
> > > >    b. Separation the ingestion vs transformation greatly simplify the
> > > > systems APIs
> > > >
> > > > Having the above two changes would allow us to remove many
> unnecessary
> > > > complexities introduced by those pluggable interfaces Chris’ pointed
> > out,
> > > > e.g. pluggable streaming systems and serde.
> > > >
> > > >
> > > > To recall one of Chris’s statement on difficulties in dynamic
> > > deployment, I
> > > > believe that the difficulties are mainly the result of tight-coupling
> > of
> > > > partition assignment vs the container deployment in the current
> system.
> > > The
> > > > current container deployment requires a pre-defined partition
> > assignment
> > > > strategy coupled together w/ the deployment configuration before we
> can
> > > > submit to YARN and start the Samza container, which makes the
> launching
> > > > process super long. Also, fault-tolerance and the embedded
> > JobCoordinator
> > > > code in YARN AppMaster is another way of  making dynamic deployment
> > more
> > > > complex and difficult.
> > > >
> > > >
> > > > First, borrowing Yan’s term, let’s call the Samza standalone process
> a
> > > > Samza worker. Here is what I have been thinking:
> > > >
> > > > 1. Separate the execution framework from partition assignment/load
> > > > balancing:
> > > >
> > > >     a. a Samza worker should be launched by execution framework that
> > only
> > > > deals w/ process placement to available nodes. The execution
> framework
> > > now
> > > > should only deal w/ how many such processes are needed, where to put
> > > them,
> > > > and how to keep them alive.
> > > >
> > > >     b. Partition assignment/load balancing can be a pluggable
> interface
> > > in
> > > > Samza that allows the Samza workers to ask for partition assignments.
> > > Let’s
> > > > borrow the name JobCoordinator for now. To allow fault-tolerance in
> > case
> > > of
> > > > failure, the partition assignments to workers need to be dynamic.
> > Hence,
> > > > the abstract interface would be much like what Jay’s code illustrate:
> > > > get()/onAssigned()/onRevoke(). The implementation of the partition
> > > > assignment can be either:
> > > >
> > > >         a) completely rely on Kafka.
> > > >
> > > >         b) explicit partition assignment via JobCoordinator. Chris’s
> > work
> > > > in SAMZA-516 can be easily incorporated here. The use case in
> SAMZA-41
> > > that
> > > > runs Samza ProcessJob w/ static partition assignment can be
> implemented
> > > of
> > > > JobCoordinator via any home-grown implementation of distributed
> > > > coordinator. All the work we did in LinkedIn to support dynamic
> > partition
> > > > assignment and host-affinity SAMZA-617 can be nicely reused as an
> > > > implementation of JobCoordinator.
> > > >
> > > >
> > > > When we did the above work, I can see three usage patterns in Samza:
> > > >
> > > >    a. Samza as a library: Samza has a set of libraries to provide
> > stream
> > > > processing, just like a third Kafka client (as illustrated in Jay’s
> > > > example). The execution/deployment is totally controlled by the
> > > application
> > > > and the partition coordination is done via Kafka
> > > >
> > > >    b. Samza as a process: Samza runs as a standalone process. There
> may
> > > not
> > > > be a execution framework to launch and deploy Samza processes. The
> > > > partition assignment is pluggable JobCoordinator.
> > > >
> > > >    c. Samza as a service: Samza runs as a collection of processes.
> > There
> > > > will be an execution framework to allocate resource, launch and
> deploy
> > > > Samza workers and keep them alive. The same pluggable JobCoordinator
> is
> > > > desirable here as well.
> > > >
> > > >
> > > > Lastly, I would argue that CopyCat in KIP-26 should probably follow
> the
> > > > same model. Hence, in Samza as a service model as in LinkedIn, we can
> > use
> > > > the same fault tolerance execution framework to run CopyCat and Samza
> > w/o
> > > > the need to operate two service platforms, which should address
> > Sriram’s
> > > > comment in the email thread.
> > > >
> > > >
> > > > Hope the above makes sense. Thanks all!
> > > >
> > > >
> > > > -Yi
> > > >
> > > > On Thu, Jul 2, 2015 at 9:53 AM, Sriram <sr...@gmail.com> wrote:
> > > >
> > > > > One thing that is worth exploring is to have a transformation and
> > > > > ingestion library in Kafka but use the same framework for fault
> > > > tolerance,
> > > > > resource isolation and management. The biggest difference I see in
> > > these
> > > > > two use cases is the API and data model.
> > > > >
> > > > >
> > > > > > On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io> wrote:
> > > > > >
> > > > > > Hey Garry,
> > > > > >
> > > > > > Yeah that's super frustrating. I'd be happy to chat more about
> this
> > > if
> > > > > > you'd be interested. I think Chris and I started with the idea of
> > > "what
> > > > > > would it take to make Samza a kick-ass ingestion tool" but
> > ultimately
> > > > we
> > > > > > kind of came around to the idea that ingestion and transformation
> > had
> > > > > > pretty different needs and coupling the two made things hard.
> > > > > >
> > > > > > For what it's worth I think copycat (KIP-26) actually will do
> what
> > > you
> > > > > are
> > > > > > looking for.
> > > > > >
> > > > > > With regard to your point about slider, I don't necessarily
> > disagree.
> > > > > But I
> > > > > > think getting good YARN support is quite doable and I think we
> can
> > > make
> > > > > > that work well. I think the issue this proposal solves is that
> > > > > technically
> > > > > > it is pretty hard to support multiple cluster management systems
> > the
> > > > way
> > > > > > things are now, you need to write an "app master" or "framework"
> > for
> > > > each
> > > > > > and they are all a little different so testing is really hard. In
> > the
> > > > > > absence of this we have been stuck with just YARN which has
> > fantastic
> > > > > > penetration in the Hadoopy part of the org, but zero penetration
> > > > > elsewhere.
> > > > > > Given the huge amount of work being put in to slider, marathon,
> aws
> > > > > > tooling, not to mention the umpteen related packaging
> technologies
> > > > people
> > > > > > want to use (Docker, Kubernetes, various cloud-specific deploy
> > tools,
> > > > > etc)
> > > > > > I really think it is important to get this right.
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > > > > g.turkington@improvedigital.com> wrote:
> > > > > >
> > > > > >> Hi all,
> > > > > >>
> > > > > >> I think the question below re does Samza become a sub-project of
> > > Kafka
> > > > > >> highlights the broader point around migration. Chris mentions
> > > Samza's
> > > > > >> maturity is heading towards a v1 release but I'm not sure it
> feels
> > > > > right to
> > > > > >> launch a v1 then immediately plan to deprecate most of it.
> > > > > >>
> > > > > >> From a selfish perspective I have some guys who have started
> > working
> > > > > with
> > > > > >> Samza and building some new consumers/producers was next up.
> > Sounds
> > > > like
> > > > > >> that is absolutely not the direction to go. I need to look into
> > the
> > > > KIP
> > > > > in
> > > > > >> more detail but for me the attractiveness of adding new Samza
> > > > > >> consumer/producers -- even if yes all they were doing was really
> > > > getting
> > > > > >> data into and out of Kafka --  was to avoid  having to worry
> about
> > > the
> > > > > >> lifecycle management of external clients. If there is a generic
> > > Kafka
> > > > > >> ingress/egress layer that I can plug a new connector into and
> > have a
> > > > > lot of
> > > > > >> the heavy lifting re scale and reliability done for me then it
> > gives
> > > > me
> > > > > all
> > > > > >> the pushing new consumers/producers would. If not then it
> > > complicates
> > > > my
> > > > > >> operational deployments.
> > > > > >>
> > > > > >> Which is similar to my other question with the proposal -- if we
> > > > build a
> > > > > >> fully available/stand-alone Samza plus the requisite shims to
> > > > integrate
> > > > > >> with Slider etc I suspect the former may be a lot more work than
> > we
> > > > > think.
> > > > > >> We may make it much easier for a newcomer to get something
> running
> > > but
> > > > > >> having them step up and get a reliable production deployment may
> > > still
> > > > > >> dominate mailing list  traffic, if for different reasons than
> > today.
> > > > > >>
> > > > > >> Don't get me wrong -- I'm comfortable with making the Samza
> > > dependency
> > > > > on
> > > > > >> Kafka much more explicit and I absolutely see the benefits  in
> the
> > > > > >> reduction of duplication and clashing terminologies/abstractions
> > > that
> > > > > >> Chris/Jay describe. Samza as a library would likely be a very
> nice
> > > > tool
> > > > > to
> > > > > >> add to the Kafka ecosystem. I just have the concerns above re
> the
> > > > > >> operational side.
> > > > > >>
> > > > > >> Garry
> > > > > >>
> > > > > >> -----Original Message-----
> > > > > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> > > > > >> Sent: 02 July 2015 12:56
> > > > > >> To: dev@samza.apache.org
> > > > > >> Subject: Re: Thoughts and obesrvations on Samza
> > > > > >>
> > > > > >> Very interesting thoughts.
> > > > > >> From outside, I have always perceived Samza as a computing layer
> > > over
> > > > > >> Kafka.
> > > > > >>
> > > > > >> The question, maybe a bit provocative, is "should Samza be a
> > > > sub-project
> > > > > >> of Kafka then?"
> > > > > >> Or does it make sense to keep it as a separate project with a
> > > separate
> > > > > >> governance?
> > > > > >>
> > > > > >> Cheers,
> > > > > >>
> > > > > >> --
> > > > > >> Gianmarco
> > > > > >>
> > > > > >>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> wrote:
> > > > > >>>
> > > > > >>> Overall, I agree to couple with Kafka more tightly. Because
> Samza
> > > de
> > > > > >>> facto is based on Kafka, and it should leverage what Kafka has.
> > At
> > > > the
> > > > > >>> same time, Kafka does not need to reinvent what Samza already
> > has.
> > > I
> > > > > >>> also like the idea of separating the ingestion and
> > transformation.
> > > > > >>>
> > > > > >>> But it is a little difficult for me to image how the Samza will
> > > look
> > > > > >> like.
> > > > > >>> And I feel Chris and Jay have a little difference in terms of
> how
> > > > > >>> Samza should look like.
> > > > > >>>
> > > > > >>> *** Will it look like what Jay's code shows (A client of
> Kakfa) ?
> > > And
> > > > > >>> user's application code calls this client?
> > > > > >>>
> > > > > >>> 1. If we make Samza be a library of Kafka (like what the code
> > > shows),
> > > > > >>> how do we implement auto-balance and fault-tolerance? Are they
> > > taken
> > > > > >>> care by the Kafka broker or other mechanism, such as "Samza
> > worker"
> > > > > >>> (just make up the name) ?
> > > > > >>>
> > > > > >>> 2. What about other features, such as auto-scaling, shared
> state,
> > > > > >>> monitoring?
> > > > > >>>
> > > > > >>>
> > > > > >>> *** If we have Samza standalone, (is this what Chris suggests?)
> > > > > >>>
> > > > > >>> 1. we still need to ingest data from Kakfa and produce to it.
> > Then
> > > it
> > > > > >>> becomes the same as what Samza looks like now, except it does
> not
> > > > rely
> > > > > >>> on Yarn anymore.
> > > > > >>>
> > > > > >>> 2. if it is standalone, how can it leverage Kafka's metrics,
> > logs,
> > > > > >>> etc? Use Kafka code as the dependency?
> > > > > >>>
> > > > > >>>
> > > > > >>> Thanks,
> > > > > >>>
> > > > > >>> Fang, Yan
> > > > > >>> yanfang724@gmail.com
> > > > > >>>
> > > > > >>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > wangguoz@gmail.com
> > > >
> > > > > >>> wrote:
> > > > > >>>
> > > > > >>>> Read through the code example and it looks good to me. A few
> > > > > >>>> thoughts regarding deployment:
> > > > > >>>>
> > > > > >>>> Today Samza deploys as executable runnable like:
> > > > > >>>>
> > > > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
> > > > > >> --config-path=file://...
> > > > > >>>>
> > > > > >>>> And this proposal advocate for deploying Samza more as
> embedded
> > > > > >>>> libraries in user application code (ignoring the terminology
> > since
> > > > > >>>> it is not the
> > > > > >>> same
> > > > > >>>> as the prototype code):
> > > > > >>>>
> > > > > >>>> StreamTask task = new MyStreamTask(configs); Thread thread =
> new
> > > > > >>>> Thread(task); thread.start();
> > > > > >>>>
> > > > > >>>> I think both of these deployment modes are important for
> > different
> > > > > >>>> types
> > > > > >>> of
> > > > > >>>> users. That said, I think making Samza purely standalone is
> > still
> > > > > >>>> sufficient for either runnable or library modes.
> > > > > >>>>
> > > > > >>>> Guozhang
> > > > > >>>>
> > > > > >>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> jay@confluent.io>
> > > > > wrote:
> > > > > >>>>>
> > > > > >>>>> Looks like gmail mangled the code example, it was supposed to
> > > look
> > > > > >>>>> like
> > > > > >>>>> this:
> > > > > >>>>>
> > > > > >>>>> Properties props = new Properties();
> > > > > >>>>> props.put("bootstrap.servers", "localhost:4242");
> > StreamingConfig
> > > > > >>>>> config = new StreamingConfig(props);
> > > > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
> > > > > >>>>> config.processor(ExampleStreamProcessor.class);
> > > > > >>>>> config.serialization(new StringSerializer(), new
> > > > > >>>>> StringDeserializer()); KafkaStreaming container = new
> > > > > >>>>> KafkaStreaming(config); container.run();
> > > > > >>>>>
> > > > > >>>>> -Jay
> > > > > >>>>>
> > > > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> jay@confluent.io>
> > > > > >> wrote:
> > > > > >>>>>
> > > > > >>>>>> Hey guys,
> > > > > >>>>>>
> > > > > >>>>>> This came out of some conversations Chris and I were having
> > > > > >>>>>> around
> > > > > >>>>> whether
> > > > > >>>>>> it would make sense to use Samza as a kind of data ingestion
> > > > > >>> framework
> > > > > >>>>> for
> > > > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind
> > of
> > > > > >>>> combined
> > > > > >>>>>> with complaints around config and YARN and the discussion
> > around
> > > > > >>>>>> how
> > > > > >>> to
> > > > > >>>>>> best do a standalone mode.
> > > > > >>>>>>
> > > > > >>>>>> So the thought experiment was, given that Samza was
> basically
> > > > > >>>>>> already totally Kafka specific, what if you just embraced
> that
> > > > > >>>>>> and turned it
> > > > > >>>> into
> > > > > >>>>>> something less like a heavyweight framework and more like a
> > > > > >>>>>> third
> > > > > >>> Kafka
> > > > > >>>>>> client--a kind of "producing consumer" with state management
> > > > > >>>> facilities.
> > > > > >>>>>> Basically a library. Instead of a complex stream processing
> > > > > >>>>>> framework
> > > > > >>>>> this
> > > > > >>>>>> would actually be a very simple thing, not much more
> > complicated
> > > > > >>>>>> to
> > > > > >>> use
> > > > > >>>>> or
> > > > > >>>>>> operate than a Kafka consumer. As Chris said we thought
> about
> > it
> > > > > >>>>>> a
> > > > > >>> lot
> > > > > >>>> of
> > > > > >>>>>> what Samza (and the other stream processing systems were
> > doing)
> > > > > >>> seemed
> > > > > >>>>> like
> > > > > >>>>>> kind of a hangover from MapReduce.
> > > > > >>>>>>
> > > > > >>>>>> Of course you need to ingest/output data to and from the
> > stream
> > > > > >>>>>> processing. But when we actually looked into how that would
> > > > > >>>>>> work,
> > > > > >>> Samza
> > > > > >>>>>> isn't really an ideal data ingestion framework for a bunch
> of
> > > > > >>> reasons.
> > > > > >>>> To
> > > > > >>>>>> really do that right you need a pretty different internal
> data
> > > > > >>>>>> model
> > > > > >>>> and
> > > > > >>>>>> set of apis. So what if you split them and had an api for
> > Kafka
> > > > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
> > Kafka
> > > > > >>>>>> transformation (Samza).
> > > > > >>>>>>
> > > > > >>>>>> This would also allow really embracing the same terminology
> > and
> > > > > >>>>>> conventions. One complaint about the current state is that
> the
> > > > > >>>>>> two
> > > > > >>>>> systems
> > > > > >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic"
> > and
> > > > > >>>>> different
> > > > > >>>>>> config and monitoring systems means you kind of have to
> learn
> > > > > >>>>>> Kafka's
> > > > > >>>>> way,
> > > > > >>>>>> then learn Samza's slightly different way, then kind of
> > > > > >>>>>> understand
> > > > > >>> how
> > > > > >>>>> they
> > > > > >>>>>> map to each other, which having walked a few people through
> > this
> > > > > >>>>>> is surprisingly tricky for folks to get.
> > > > > >>>>>>
> > > > > >>>>>> Since I have been spending a lot of time on airplanes I
> hacked
> > > > > >>>>>> up an ernest but still somewhat incomplete prototype of what
> > > > > >>>>>> this would
> > > > > >>> look
> > > > > >>>>>> like. This is just unceremoniously dumped into Kafka as it
> > > > > >>>>>> required a
> > > > > >>>> few
> > > > > >>>>>> changes to the new consumer. Here is the code:
> > > > > >>>
> > > >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > > >>> /apache/kafka/clients/streaming
> > > > > >>>>>>
> > > > > >>>>>> For the purpose of the prototype I just liberally renamed
> > > > > >>>>>> everything
> > > > > >>> to
> > > > > >>>>>> try to align it with Kafka with no regard for compatibility.
> > > > > >>>>>>
> > > > > >>>>>> To use this would be something like this:
> > > > > >>>>>> Properties props = new Properties();
> > > > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > > > >>>>>> StreamingConfig config = new
> > > > > >>> StreamingConfig(props);
> > > > > >>>>> config.subscribe("test-topic-1",
> > > > > >>>>>> "test-topic-2");
> > config.processor(ExampleStreamProcessor.class);
> > > > > >>>>> config.serialization(new
> > > > > >>>>>> StringSerializer(), new StringDeserializer());
> KafkaStreaming
> > > > > >>>> container =
> > > > > >>>>>> new KafkaStreaming(config); container.run();
> > > > > >>>>>>
> > > > > >>>>>> KafkaStreaming is basically the SamzaContainer;
> > StreamProcessor
> > > > > >>>>>> is basically StreamTask.
> > > > > >>>>>>
> > > > > >>>>>> So rather than putting all the class names in a file and
> then
> > > > > >>>>>> having
> > > > > >>>> the
> > > > > >>>>>> job assembled by reflection, you just instantiate the
> > container
> > > > > >>>>>> programmatically. Work is balanced over however many
> instances
> > > > > >>>>>> of
> > > > > >>> this
> > > > > >>>>> are
> > > > > >>>>>> alive at any time (i.e. if an instance dies, new tasks are
> > added
> > > > > >>>>>> to
> > > > > >>> the
> > > > > >>>>>> existing containers without shutting them down).
> > > > > >>>>>>
> > > > > >>>>>> We would provide some glue for running this stuff in YARN
> via
> > > > > >>>>>> Slider, Mesos via Marathon, and AWS using some of their
> tools
> > > > > >>>>>> but from the
> > > > > >>>> point
> > > > > >>>>> of
> > > > > >>>>>> view of these frameworks these stream processing jobs are
> just
> > > > > >>>> stateless
> > > > > >>>>>> services that can come and go and expand and contract at
> will.
> > > > > >>>>>> There
> > > > > >>> is
> > > > > >>>>> no
> > > > > >>>>>> more custom scheduler.
> > > > > >>>>>>
> > > > > >>>>>> Here are some relevant details:
> > > > > >>>>>>
> > > > > >>>>>>   1. It is only ~1300 lines of code, it would get larger if
> we
> > > > > >>>>>>   productionized but not vastly larger. We really do get a
> ton
> > > > > >>>>>> of
> > > > > >>>>> leverage
> > > > > >>>>>>   out of Kafka.
> > > > > >>>>>>   2. Partition management is fully delegated to the new
> > > consumer.
> > > > > >>> This
> > > > > >>>>>>   is nice since now any partition management strategy
> > available
> > > > > >>>>>> to
> > > > > >>>> Kafka
> > > > > >>>>>>   consumer is also available to Samza (and vice versa) and
> > with
> > > > > >>>>>> the
> > > > > >>>>> exact
> > > > > >>>>>>   same configs.
> > > > > >>>>>>   3. It supports state as well as state reuse
> > > > > >>>>>>
> > > > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
> > > > > >>>>>>
> > > > > >>>>>> -Jay
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>>
> > > > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > > > >>>> criccomini@apache.org>
> > > > > >>>>>> wrote:
> > > > > >>>>>>
> > > > > >>>>>>> Hey all,
> > > > > >>>>>>>
> > > > > >>>>>>> I have had some discussions with Samza engineers at
> LinkedIn
> > > > > >>>>>>> and
> > > > > >>>>> Confluent
> > > > > >>>>>>> and we came up with a few observations and would like to
> > > > > >>>>>>> propose
> > > > > >>> some
> > > > > >>>>>>> changes.
> > > > > >>>>>>>
> > > > > >>>>>>> We've observed some things that I want to call out about
> > > > > >>>>>>> Samza's
> > > > > >>>> design,
> > > > > >>>>>>> and I'd like to propose some changes.
> > > > > >>>>>>>
> > > > > >>>>>>> * Samza is dependent upon a dynamic deployment system.
> > > > > >>>>>>> * Samza is too pluggable.
> > > > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> consumer
> > > > > >>>>>>> APIs
> > > > > >>> are
> > > > > >>>>>>> trying to solve a lot of the same problems.
> > > > > >>>>>>>
> > > > > >>>>>>> All three of these issues are related, but I'll address
> them
> > in
> > > > > >>> order.
> > > > > >>>>>>>
> > > > > >>>>>>> Deployment
> > > > > >>>>>>>
> > > > > >>>>>>> Samza strongly depends on the use of a dynamic deployment
> > > > > >>>>>>> scheduler
> > > > > >>>> such
> > > > > >>>>>>> as
> > > > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet
> that
> > > > > >>>>>>> there
> > > > > >>>> would
> > > > > >>>>>>> be
> > > > > >>>>>>> one or two winners in this area, and we could support them,
> > and
> > > > > >>>>>>> the
> > > > > >>>> rest
> > > > > >>>>>>> would go away. In reality, there are many variations.
> > > > > >>>>>>> Furthermore,
> > > > > >>>> many
> > > > > >>>>>>> people still prefer to just start their processors like
> > normal
> > > > > >>>>>>> Java processes, and use traditional deployment scripts such
> > as
> > > > > >>>>>>> Fabric,
> > > > > >>>> Chef,
> > > > > >>>>>>> Ansible, etc. Forcing a deployment system on users makes
> the
> > > > > >>>>>>> Samza start-up process really painful for first time users.
> > > > > >>>>>>>
> > > > > >>>>>>> Dynamic deployment as a requirement was also a bit of a
> > > > > >>>>>>> mis-fire
> > > > > >>>> because
> > > > > >>>>>>> of
> > > > > >>>>>>> a fundamental misunderstanding between the nature of batch
> > jobs
> > > > > >>>>>>> and
> > > > > >>>>> stream
> > > > > >>>>>>> processing jobs. Early on, we made conscious effort to
> favor
> > > > > >>>>>>> the
> > > > > >>>> Hadoop
> > > > > >>>>>>> (Map/Reduce) way of doing things, since it worked and was
> > well
> > > > > >>>>> understood.
> > > > > >>>>>>> One thing that we missed was that batch jobs have a
> definite
> > > > > >>>> beginning,
> > > > > >>>>>>> and
> > > > > >>>>>>> end, and stream processing jobs don't (usually). This leads
> > to
> > > > > >>>>>>> a
> > > > > >>> much
> > > > > >>>>>>> simpler scheduling problem for stream processors. You
> > basically
> > > > > >>>>>>> just
> > > > > >>>>> need
> > > > > >>>>>>> to find a place to start the processor, and start it. The
> way
> > > > > >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> > > > > >>>>>>> being "full". We always
> > > > > >>>> add
> > > > > >>>>>>> more machines. The problem with coupling Samza with a
> > scheduler
> > > > > >>>>>>> is
> > > > > >>>> that
> > > > > >>>>>>> Samza (as a framework) now has to handle deployment. This
> > pulls
> > > > > >>>>>>> in a
> > > > > >>>>> bunch
> > > > > >>>>>>> of things such as configuration distribution (config
> stream),
> > > > > >>>>>>> shell
> > > > > >>>>> scrips
> > > > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> stuff),
> > > etc.
> > > > > >>>>>>>
> > > > > >>>>>>> Another reason for requiring dynamic deployment was to
> > support
> > > > > >>>>>>> data locality. If you want to have locality, you need to
> put
> > > > > >>>>>>> your
> > > > > >>>> processors
> > > > > >>>>>>> close to the data they're processing. Upon further
> > > > > >>>>>>> investigation,
> > > > > >>>>> though,
> > > > > >>>>>>> this feature is not that beneficial. There is some good
> > > > > >>>>>>> discussion
> > > > > >>>> about
> > > > > >>>>>>> some problems with it on SAMZA-335. Again, we took the
> > > > > >>>>>>> Map/Reduce
> > > > > >>>> path,
> > > > > >>>>>>> but
> > > > > >>>>>>> there are some fundamental differences between HDFS and
> > Kafka.
> > > > > >>>>>>> HDFS
> > > > > >>>> has
> > > > > >>>>>>> blocks, while Kafka has partitions. This leads to less
> > > > > >>>>>>> optimization potential with stream processors on top of
> > Kafka.
> > > > > >>>>>>>
> > > > > >>>>>>> This feature is also used as a crutch. Samza doesn't have
> any
> > > > > >>>>>>> built
> > > > > >>> in
> > > > > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> > > > > >>>>>>> deployment scheduling system to handle restarts when a
> > > > > >>>>>>> processor dies. This has
> > > > > >>>>> made
> > > > > >>>>>>> it very difficult to write a standalone Samza container
> > > > > >> (SAMZA-516).
> > > > > >>>>>>>
> > > > > >>>>>>> Pluggability
> > > > > >>>>>>>
> > > > > >>>>>>> In some cases pluggability is good, but I think that we've
> > gone
> > > > > >>>>>>> too
> > > > > >>>> far
> > > > > >>>>>>> with it. Currently, Samza has:
> > > > > >>>>>>>
> > > > > >>>>>>> * Pluggable config.
> > > > > >>>>>>> * Pluggable metrics.
> > > > > >>>>>>> * Pluggable deployment systems.
> > > > > >>>>>>> * Pluggable streaming systems (SystemConsumer,
> > SystemProducer,
> > > > > >> etc).
> > > > > >>>>>>> * Pluggable serdes.
> > > > > >>>>>>> * Pluggable storage engines.
> > > > > >>>>>>> * Pluggable strategies for just about every component
> > > > > >>> (MessageChooser,
> > > > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > > > >>>>>>>
> > > > > >>>>>>> There's probably more that I've forgotten, as well. Some of
> > > > > >>>>>>> these
> > > > > >>> are
> > > > > >>>>>>> useful, but some have proven not to be. This all comes at a
> > > cost:
> > > > > >>>>>>> complexity. This complexity is making it harder for our
> users
> > > > > >>>>>>> to
> > > > > >>> pick
> > > > > >>>> up
> > > > > >>>>>>> and use Samza out of the box. It also makes it difficult
> for
> > > > > >>>>>>> Samza developers to reason about what the characteristics
> of
> > > > > >>>>>>> the container (since the characteristics change depending
> on
> > > > > >>>>>>> which plugins are use).
> > > > > >>>>>>>
> > > > > >>>>>>> The issues with pluggability are most visible in the System
> > > APIs.
> > > > > >>> What
> > > > > >>>>>>> Samza really requires to be functional is Kafka as its
> > > > > >>>>>>> transport
> > > > > >>>> layer.
> > > > > >>>>>>> But
> > > > > >>>>>>> we've conflated two unrelated use cases into one API:
> > > > > >>>>>>>
> > > > > >>>>>>> 1. Get data into/out of Kafka.
> > > > > >>>>>>> 2. Process the data in Kafka.
> > > > > >>>>>>>
> > > > > >>>>>>> The current System API supports both of these use cases.
> The
> > > > > >>>>>>> problem
> > > > > >>>> is,
> > > > > >>>>>>> we
> > > > > >>>>>>> actually want different features for each use case. By
> > papering
> > > > > >>>>>>> over
> > > > > >>>>> these
> > > > > >>>>>>> two use cases, and providing a single API, we've
> introduced a
> > > > > >>>>>>> ton of
> > > > > >>>>> leaky
> > > > > >>>>>>> abstractions.
> > > > > >>>>>>>
> > > > > >>>>>>> For example, what we'd really like in (2) is to have
> > > > > >>>>>>> monotonically increasing longs for offsets (like Kafka).
> This
> > > > > >>>>>>> would be at odds
> > > > > >>> with
> > > > > >>>>> (1),
> > > > > >>>>>>> though, since different systems have different
> > > > > >>>>> SCNs/Offsets/UUIDs/vectors.
> > > > > >>>>>>> There was discussion both on the mailing list and the SQL
> > JIRAs
> > > > > >>> about
> > > > > >>>>> the
> > > > > >>>>>>> need for this.
> > > > > >>>>>>>
> > > > > >>>>>>> The same thing holds true for replayability. Kafka allows
> us
> > to
> > > > > >>> rewind
> > > > > >>>>>>> when
> > > > > >>>>>>> we have a failure. Many other systems don't. In some cases,
> > > > > >>>>>>> systems
> > > > > >>>>> return
> > > > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> because
> > > > > >>>>>>> they
> > > > > >>>> have
> > > > > >>>>> no
> > > > > >>>>>>> offsets.
> > > > > >>>>>>>
> > > > > >>>>>>> Partitioning is another example. Kafka supports
> partitioning,
> > > > > >>>>>>> but
> > > > > >>> many
> > > > > >>>>>>> systems don't. We model this by having a single partition
> for
> > > > > >>>>>>> those systems. Still, other systems model partitioning
> > > > > >> differently (e.g.
> > > > > >>>>>>> Kinesis).
> > > > > >>>>>>>
> > > > > >>>>>>> The SystemAdmin interface is also a mess. Creating streams
> > in a
> > > > > >>>>>>> system-agnostic way is almost impossible. As is modeling
> > > > > >>>>>>> metadata
> > > > > >>> for
> > > > > >>>>> the
> > > > > >>>>>>> system (replication factor, partitions, location, etc). The
> > > > > >>>>>>> list
> > > > > >>> goes
> > > > > >>>>> on.
> > > > > >>>>>>>
> > > > > >>>>>>> Duplicate work
> > > > > >>>>>>>
> > > > > >>>>>>> At the time that we began writing Samza, Kafka's consumer
> and
> > > > > >>> producer
> > > > > >>>>>>> APIs
> > > > > >>>>>>> had a relatively weak feature set. On the consumer-side,
> you
> > > > > >>>>>>> had two
> > > > > >>>>>>> options: use the high level consumer, or the simple
> consumer.
> > > > > >>>>>>> The
> > > > > >>>>> problem
> > > > > >>>>>>> with the high-level consumer was that it controlled your
> > > > > >>>>>>> offsets, partition assignments, and the order in which you
> > > > > >>>>>>> received messages. The
> > > > > >>> problem
> > > > > >>>>>>> with
> > > > > >>>>>>> the simple consumer is that it's not simple. It's basic.
> You
> > > > > >>>>>>> end up
> > > > > >>>>> having
> > > > > >>>>>>> to handle a lot of really low-level stuff that you
> shouldn't.
> > > > > >>>>>>> We
> > > > > >>>> spent a
> > > > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> robust.
> > It
> > > > > >>>>>>> also allows us to support some cool features:
> > > > > >>>>>>>
> > > > > >>>>>>> * Per-partition message ordering and prioritization.
> > > > > >>>>>>> * Tight control over partition assignment to support joins,
> > > > > >>>>>>> global
> > > > > >>>> state
> > > > > >>>>>>> (if we want to implement it :)), etc.
> > > > > >>>>>>> * Tight control over offset checkpointing.
> > > > > >>>>>>>
> > > > > >>>>>>> What we didn't realize at the time is that these features
> > > > > >>>>>>> should
> > > > > >>>>> actually
> > > > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
> stream
> > > > > >>>> processors)
> > > > > >>>>>>> end up wanting to do things like joins and partition
> > > > > >>>>>>> assignment. The
> > > > > >>>>> Kafka
> > > > > >>>>>>> community has come to the same conclusion. They're adding a
> > ton
> > > > > >>>>>>> of upgrades into their new Kafka consumer implementation.
> To
> > a
> > > > > >>>>>>> large extent,
> > > > > >>> it's
> > > > > >>>>>>> duplicate work to what we've already done in Samza.
> > > > > >>>>>>>
> > > > > >>>>>>> On top of this, Kafka ended up taking a very similar
> approach
> > > > > >>>>>>> to
> > > > > >>>> Samza's
> > > > > >>>>>>> KafkaCheckpointManager implementation for handling offset
> > > > > >>>> checkpointing.
> > > > > >>>>>>> Like Samza, Kafka's new offset management feature stores
> > offset
> > > > > >>>>>>> checkpoints in a topic, and allows you to fetch them from
> the
> > > > > >>>>>>> broker.
> > > > > >>>>>>>
> > > > > >>>>>>> A lot of this seems like a waste, since we could have
> shared
> > > > > >>>>>>> the
> > > > > >>> work
> > > > > >>>> if
> > > > > >>>>>>> it
> > > > > >>>>>>> had been done in Kafka from the get-go.
> > > > > >>>>>>>
> > > > > >>>>>>> Vision
> > > > > >>>>>>>
> > > > > >>>>>>> All of this leads me to a rather radical proposal. Samza is
> > > > > >>> relatively
> > > > > >>>>>>> stable at this point. I'd venture to say that we're near a
> > 1.0
> > > > > >>>> release.
> > > > > >>>>>>> I'd
> > > > > >>>>>>> like to propose that we take what we've learned, and begin
> > > > > >>>>>>> thinking
> > > > > >>>>> about
> > > > > >>>>>>> Samza beyond 1.0. What would we change if we were starting
> > from
> > > > > >>>> scratch?
> > > > > >>>>>>> My
> > > > > >>>>>>> proposal is to:
> > > > > >>>>>>>
> > > > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > > > > >>>>>>> processors, and eliminate all direct dependences on YARN,
> > > Mesos,
> > > > > >> etc.
> > > > > >>>>>>> 2. Make a definitive call to support only Kafka as the
> stream
> > > > > >>>> processing
> > > > > >>>>>>> layer.
> > > > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> > > > > >>>>>>> config
> > > > > >>>>> systems,
> > > > > >>>>>>> and simply use Kafka's instead.
> > > > > >>>>>>>
> > > > > >>>>>>> This would fix all of the issues that I outlined above. It
> > > > > >>>>>>> should
> > > > > >>> also
> > > > > >>>>>>> shrink the Samza code base pretty dramatically. Supporting
> > only
> > > > > >>>>>>> a standalone container will allow Samza to be executed on
> > YARN
> > > > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most
> other
> > > > > >>>>>>> in-house
> > > > > >>>>> deployment
> > > > > >>>>>>> systems. This should make life a lot easier for new users.
> > > > > >>>>>>> Imagine
> > > > > >>>>> having
> > > > > >>>>>>> the hello-samza tutorial without YARN. The drop in mailing
> > list
> > > > > >>>> traffic
> > > > > >>>>>>> will be pretty dramatic.
> > > > > >>>>>>>
> > > > > >>>>>>> Coupling with Kafka seems long overdue to me. The reality
> is,
> > > > > >>> everyone
> > > > > >>>>>>> that
> > > > > >>>>>>> I'm aware of is using Samza with Kafka. We basically
> require
> > it
> > > > > >>>> already
> > > > > >>>>> in
> > > > > >>>>>>> order for most features to work. Those that are using other
> > > > > >>>>>>> systems
> > > > > >>>> are
> > > > > >>>>>>> generally using it for ingest into Kafka (1), and then they
> > do
> > > > > >>>>>>> the processing on top. There is already discussion (
> > > > > >>>
> > > >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > > >>> 767
> > > > > >>>>>>> )
> > > > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > > > > >>>>>>>
> > > > > >>>>>>> Once we make the call to couple with Kafka, we can
> leverage a
> > > > > >>>>>>> ton of
> > > > > >>>>> their
> > > > > >>>>>>> ecosystem. We no longer have to maintain our own config,
> > > > > >>>>>>> metrics,
> > > > > >>> etc.
> > > > > >>>>> We
> > > > > >>>>>>> can all share the same libraries, and make them better.
> This
> > > > > >>>>>>> will
> > > > > >>> also
> > > > > >>>>>>> allow us to share the consumer/producer APIs, and will let
> us
> > > > > >>> leverage
> > > > > >>>>>>> their offset management and partition management, rather
> than
> > > > > >>>>>>> having
> > > > > >>>> our
> > > > > >>>>>>> own. All of the coordinator stream code would go away, as
> > would
> > > > > >>>>>>> most
> > > > > >>>> of
> > > > > >>>>>>> the
> > > > > >>>>>>> YARN AppMaster code. We'd probably have to push some
> > partition
> > > > > >>>>> management
> > > > > >>>>>>> features into the Kafka broker, but they're already moving
> in
> > > > > >>>>>>> that direction with the new consumer API. The features we
> > have
> > > > > >>>>>>> for
> > > > > >>>> partition
> > > > > >>>>>>> assignment aren't unique to Samza, and seem like they
> should
> > be
> > > > > >>>>>>> in
> > > > > >>>> Kafka
> > > > > >>>>>>> anyway. There will always be some niche usages which will
> > > > > >>>>>>> require
> > > > > >>>> extra
> > > > > >>>>>>> care and hence full control over partition assignments much
> > > > > >>>>>>> like the
> > > > > >>>>> Kafka
> > > > > >>>>>>> low level consumer api. These would continue to be
> supported.
> > > > > >>>>>>>
> > > > > >>>>>>> These items will be good for the Samza community. They'll
> > make
> > > > > >>>>>>> Samza easier to use, and make it easier for developers to
> add
> > > > > >>>>>>> new features.
> > > > > >>>>>>>
> > > > > >>>>>>> Obviously this is a fairly large (and somewhat backwards
> > > > > >>> incompatible
> > > > > >>>>>>> change). If we choose to go this route, it's important that
> > we
> > > > > >>> openly
> > > > > >>>>>>> communicate how we're going to provide a migration path
> from
> > > > > >>>>>>> the
> > > > > >>>>> existing
> > > > > >>>>>>> APIs to the new ones (if we make incompatible changes). I
> > think
> > > > > >>>>>>> at a minimum, we'd probably need to provide a wrapper to
> > allow
> > > > > >>>>>>> existing StreamTask implementations to continue running on
> > the
> > > > > >> new container.
> > > > > >>>>> It's
> > > > > >>>>>>> also important that we openly communicate about timing, and
> > > > > >>>>>>> stages
> > > > > >>> of
> > > > > >>>>> the
> > > > > >>>>>>> migration.
> > > > > >>>>>>>
> > > > > >>>>>>> If you made it this far, I'm sure you have opinions. :)
> > Please
> > > > > >>>>>>> send
> > > > > >>>> your
> > > > > >>>>>>> thoughts and feedback.
> > > > > >>>>>>>
> > > > > >>>>>>> Cheers,
> > > > > >>>>>>> Chris
> > > > > >>>>
> > > > > >>>>
> > > > > >>>>
> > > > > >>>> --
> > > > > >>>> -- Guozhang
> > > > > >>
> > > > >
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: Thoughts and obesrvations on Samza

Posted by Guozhang Wang <wa...@gmail.com>.
Since the resource scheduling systems like YARN / Mesos only gives Samza a
couple of resource units (or "containers") to run processes, while Samza
still needs to handle task assignment / scheduling like which tasks should
be allocated to which containers that consume from which partitions, etc. I
think this is want Yi meant for "partition management"?

On Thu, Jul 2, 2015 at 2:35 PM, Yi Pan <ni...@gmail.com> wrote:

> @Jay, yes, the current function in the JobCoordinator is just partition
> management. Maybe we should just call it PartitionManager to make it
> explicit.
>
> -Yi
>
> On Thu, Jul 2, 2015 at 2:24 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey Yi,
> >
> > What does the JobCoordinator do? YARN/Mesos/etc would be doing the actual
> > resource assignment, process restart, etc, right? Is the additional value
> > add of the JobCoordinator just partition management?
> >
> > -Jay
> >
> > On Thu, Jul 2, 2015 at 11:32 AM, Yi Pan <ni...@gmail.com> wrote:
> >
> > > Hi, all,
> > >
> > >
> > > Thanks Chris for sending out this proposal and Jay for sharing the
> > > extremely illustrative prototype code.
> > >
> > >
> > > I have been thinking it over many times and want to list out my
> personal
> > > opinions below:
> > >
> > > 1. Generally, I agree with most of the people here on the mailing list
> on
> > > two points:
> > >
> > >    a. Deeper integration w/ Kafka is great. No more confusing mapping
> > from
> > > SystemStreamPartition to TopicPartition etc.
> > >
> > >    b. Separation the ingestion vs transformation greatly simplify the
> > > systems APIs
> > >
> > > Having the above two changes would allow us to remove many unnecessary
> > > complexities introduced by those pluggable interfaces Chris’ pointed
> out,
> > > e.g. pluggable streaming systems and serde.
> > >
> > >
> > > To recall one of Chris’s statement on difficulties in dynamic
> > deployment, I
> > > believe that the difficulties are mainly the result of tight-coupling
> of
> > > partition assignment vs the container deployment in the current system.
> > The
> > > current container deployment requires a pre-defined partition
> assignment
> > > strategy coupled together w/ the deployment configuration before we can
> > > submit to YARN and start the Samza container, which makes the launching
> > > process super long. Also, fault-tolerance and the embedded
> JobCoordinator
> > > code in YARN AppMaster is another way of  making dynamic deployment
> more
> > > complex and difficult.
> > >
> > >
> > > First, borrowing Yan’s term, let’s call the Samza standalone process a
> > > Samza worker. Here is what I have been thinking:
> > >
> > > 1. Separate the execution framework from partition assignment/load
> > > balancing:
> > >
> > >     a. a Samza worker should be launched by execution framework that
> only
> > > deals w/ process placement to available nodes. The execution framework
> > now
> > > should only deal w/ how many such processes are needed, where to put
> > them,
> > > and how to keep them alive.
> > >
> > >     b. Partition assignment/load balancing can be a pluggable interface
> > in
> > > Samza that allows the Samza workers to ask for partition assignments.
> > Let’s
> > > borrow the name JobCoordinator for now. To allow fault-tolerance in
> case
> > of
> > > failure, the partition assignments to workers need to be dynamic.
> Hence,
> > > the abstract interface would be much like what Jay’s code illustrate:
> > > get()/onAssigned()/onRevoke(). The implementation of the partition
> > > assignment can be either:
> > >
> > >         a) completely rely on Kafka.
> > >
> > >         b) explicit partition assignment via JobCoordinator. Chris’s
> work
> > > in SAMZA-516 can be easily incorporated here. The use case in SAMZA-41
> > that
> > > runs Samza ProcessJob w/ static partition assignment can be implemented
> > of
> > > JobCoordinator via any home-grown implementation of distributed
> > > coordinator. All the work we did in LinkedIn to support dynamic
> partition
> > > assignment and host-affinity SAMZA-617 can be nicely reused as an
> > > implementation of JobCoordinator.
> > >
> > >
> > > When we did the above work, I can see three usage patterns in Samza:
> > >
> > >    a. Samza as a library: Samza has a set of libraries to provide
> stream
> > > processing, just like a third Kafka client (as illustrated in Jay’s
> > > example). The execution/deployment is totally controlled by the
> > application
> > > and the partition coordination is done via Kafka
> > >
> > >    b. Samza as a process: Samza runs as a standalone process. There may
> > not
> > > be a execution framework to launch and deploy Samza processes. The
> > > partition assignment is pluggable JobCoordinator.
> > >
> > >    c. Samza as a service: Samza runs as a collection of processes.
> There
> > > will be an execution framework to allocate resource, launch and deploy
> > > Samza workers and keep them alive. The same pluggable JobCoordinator is
> > > desirable here as well.
> > >
> > >
> > > Lastly, I would argue that CopyCat in KIP-26 should probably follow the
> > > same model. Hence, in Samza as a service model as in LinkedIn, we can
> use
> > > the same fault tolerance execution framework to run CopyCat and Samza
> w/o
> > > the need to operate two service platforms, which should address
> Sriram’s
> > > comment in the email thread.
> > >
> > >
> > > Hope the above makes sense. Thanks all!
> > >
> > >
> > > -Yi
> > >
> > > On Thu, Jul 2, 2015 at 9:53 AM, Sriram <sr...@gmail.com> wrote:
> > >
> > > > One thing that is worth exploring is to have a transformation and
> > > > ingestion library in Kafka but use the same framework for fault
> > > tolerance,
> > > > resource isolation and management. The biggest difference I see in
> > these
> > > > two use cases is the API and data model.
> > > >
> > > >
> > > > > On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io> wrote:
> > > > >
> > > > > Hey Garry,
> > > > >
> > > > > Yeah that's super frustrating. I'd be happy to chat more about this
> > if
> > > > > you'd be interested. I think Chris and I started with the idea of
> > "what
> > > > > would it take to make Samza a kick-ass ingestion tool" but
> ultimately
> > > we
> > > > > kind of came around to the idea that ingestion and transformation
> had
> > > > > pretty different needs and coupling the two made things hard.
> > > > >
> > > > > For what it's worth I think copycat (KIP-26) actually will do what
> > you
> > > > are
> > > > > looking for.
> > > > >
> > > > > With regard to your point about slider, I don't necessarily
> disagree.
> > > > But I
> > > > > think getting good YARN support is quite doable and I think we can
> > make
> > > > > that work well. I think the issue this proposal solves is that
> > > > technically
> > > > > it is pretty hard to support multiple cluster management systems
> the
> > > way
> > > > > things are now, you need to write an "app master" or "framework"
> for
> > > each
> > > > > and they are all a little different so testing is really hard. In
> the
> > > > > absence of this we have been stuck with just YARN which has
> fantastic
> > > > > penetration in the Hadoopy part of the org, but zero penetration
> > > > elsewhere.
> > > > > Given the huge amount of work being put in to slider, marathon, aws
> > > > > tooling, not to mention the umpteen related packaging technologies
> > > people
> > > > > want to use (Docker, Kubernetes, various cloud-specific deploy
> tools,
> > > > etc)
> > > > > I really think it is important to get this right.
> > > > >
> > > > > -Jay
> > > > >
> > > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > > > g.turkington@improvedigital.com> wrote:
> > > > >
> > > > >> Hi all,
> > > > >>
> > > > >> I think the question below re does Samza become a sub-project of
> > Kafka
> > > > >> highlights the broader point around migration. Chris mentions
> > Samza's
> > > > >> maturity is heading towards a v1 release but I'm not sure it feels
> > > > right to
> > > > >> launch a v1 then immediately plan to deprecate most of it.
> > > > >>
> > > > >> From a selfish perspective I have some guys who have started
> working
> > > > with
> > > > >> Samza and building some new consumers/producers was next up.
> Sounds
> > > like
> > > > >> that is absolutely not the direction to go. I need to look into
> the
> > > KIP
> > > > in
> > > > >> more detail but for me the attractiveness of adding new Samza
> > > > >> consumer/producers -- even if yes all they were doing was really
> > > getting
> > > > >> data into and out of Kafka --  was to avoid  having to worry about
> > the
> > > > >> lifecycle management of external clients. If there is a generic
> > Kafka
> > > > >> ingress/egress layer that I can plug a new connector into and
> have a
> > > > lot of
> > > > >> the heavy lifting re scale and reliability done for me then it
> gives
> > > me
> > > > all
> > > > >> the pushing new consumers/producers would. If not then it
> > complicates
> > > my
> > > > >> operational deployments.
> > > > >>
> > > > >> Which is similar to my other question with the proposal -- if we
> > > build a
> > > > >> fully available/stand-alone Samza plus the requisite shims to
> > > integrate
> > > > >> with Slider etc I suspect the former may be a lot more work than
> we
> > > > think.
> > > > >> We may make it much easier for a newcomer to get something running
> > but
> > > > >> having them step up and get a reliable production deployment may
> > still
> > > > >> dominate mailing list  traffic, if for different reasons than
> today.
> > > > >>
> > > > >> Don't get me wrong -- I'm comfortable with making the Samza
> > dependency
> > > > on
> > > > >> Kafka much more explicit and I absolutely see the benefits  in the
> > > > >> reduction of duplication and clashing terminologies/abstractions
> > that
> > > > >> Chris/Jay describe. Samza as a library would likely be a very nice
> > > tool
> > > > to
> > > > >> add to the Kafka ecosystem. I just have the concerns above re the
> > > > >> operational side.
> > > > >>
> > > > >> Garry
> > > > >>
> > > > >> -----Original Message-----
> > > > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> > > > >> Sent: 02 July 2015 12:56
> > > > >> To: dev@samza.apache.org
> > > > >> Subject: Re: Thoughts and obesrvations on Samza
> > > > >>
> > > > >> Very interesting thoughts.
> > > > >> From outside, I have always perceived Samza as a computing layer
> > over
> > > > >> Kafka.
> > > > >>
> > > > >> The question, maybe a bit provocative, is "should Samza be a
> > > sub-project
> > > > >> of Kafka then?"
> > > > >> Or does it make sense to keep it as a separate project with a
> > separate
> > > > >> governance?
> > > > >>
> > > > >> Cheers,
> > > > >>
> > > > >> --
> > > > >> Gianmarco
> > > > >>
> > > > >>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> > > > >>>
> > > > >>> Overall, I agree to couple with Kafka more tightly. Because Samza
> > de
> > > > >>> facto is based on Kafka, and it should leverage what Kafka has.
> At
> > > the
> > > > >>> same time, Kafka does not need to reinvent what Samza already
> has.
> > I
> > > > >>> also like the idea of separating the ingestion and
> transformation.
> > > > >>>
> > > > >>> But it is a little difficult for me to image how the Samza will
> > look
> > > > >> like.
> > > > >>> And I feel Chris and Jay have a little difference in terms of how
> > > > >>> Samza should look like.
> > > > >>>
> > > > >>> *** Will it look like what Jay's code shows (A client of Kakfa) ?
> > And
> > > > >>> user's application code calls this client?
> > > > >>>
> > > > >>> 1. If we make Samza be a library of Kafka (like what the code
> > shows),
> > > > >>> how do we implement auto-balance and fault-tolerance? Are they
> > taken
> > > > >>> care by the Kafka broker or other mechanism, such as "Samza
> worker"
> > > > >>> (just make up the name) ?
> > > > >>>
> > > > >>> 2. What about other features, such as auto-scaling, shared state,
> > > > >>> monitoring?
> > > > >>>
> > > > >>>
> > > > >>> *** If we have Samza standalone, (is this what Chris suggests?)
> > > > >>>
> > > > >>> 1. we still need to ingest data from Kakfa and produce to it.
> Then
> > it
> > > > >>> becomes the same as what Samza looks like now, except it does not
> > > rely
> > > > >>> on Yarn anymore.
> > > > >>>
> > > > >>> 2. if it is standalone, how can it leverage Kafka's metrics,
> logs,
> > > > >>> etc? Use Kafka code as the dependency?
> > > > >>>
> > > > >>>
> > > > >>> Thanks,
> > > > >>>
> > > > >>> Fang, Yan
> > > > >>> yanfang724@gmail.com
> > > > >>>
> > > > >>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> wangguoz@gmail.com
> > >
> > > > >>> wrote:
> > > > >>>
> > > > >>>> Read through the code example and it looks good to me. A few
> > > > >>>> thoughts regarding deployment:
> > > > >>>>
> > > > >>>> Today Samza deploys as executable runnable like:
> > > > >>>>
> > > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
> > > > >> --config-path=file://...
> > > > >>>>
> > > > >>>> And this proposal advocate for deploying Samza more as embedded
> > > > >>>> libraries in user application code (ignoring the terminology
> since
> > > > >>>> it is not the
> > > > >>> same
> > > > >>>> as the prototype code):
> > > > >>>>
> > > > >>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
> > > > >>>> Thread(task); thread.start();
> > > > >>>>
> > > > >>>> I think both of these deployment modes are important for
> different
> > > > >>>> types
> > > > >>> of
> > > > >>>> users. That said, I think making Samza purely standalone is
> still
> > > > >>>> sufficient for either runnable or library modes.
> > > > >>>>
> > > > >>>> Guozhang
> > > > >>>>
> > > > >>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
> > > > wrote:
> > > > >>>>>
> > > > >>>>> Looks like gmail mangled the code example, it was supposed to
> > look
> > > > >>>>> like
> > > > >>>>> this:
> > > > >>>>>
> > > > >>>>> Properties props = new Properties();
> > > > >>>>> props.put("bootstrap.servers", "localhost:4242");
> StreamingConfig
> > > > >>>>> config = new StreamingConfig(props);
> > > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
> > > > >>>>> config.processor(ExampleStreamProcessor.class);
> > > > >>>>> config.serialization(new StringSerializer(), new
> > > > >>>>> StringDeserializer()); KafkaStreaming container = new
> > > > >>>>> KafkaStreaming(config); container.run();
> > > > >>>>>
> > > > >>>>> -Jay
> > > > >>>>>
> > > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> > > > >> wrote:
> > > > >>>>>
> > > > >>>>>> Hey guys,
> > > > >>>>>>
> > > > >>>>>> This came out of some conversations Chris and I were having
> > > > >>>>>> around
> > > > >>>>> whether
> > > > >>>>>> it would make sense to use Samza as a kind of data ingestion
> > > > >>> framework
> > > > >>>>> for
> > > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind
> of
> > > > >>>> combined
> > > > >>>>>> with complaints around config and YARN and the discussion
> around
> > > > >>>>>> how
> > > > >>> to
> > > > >>>>>> best do a standalone mode.
> > > > >>>>>>
> > > > >>>>>> So the thought experiment was, given that Samza was basically
> > > > >>>>>> already totally Kafka specific, what if you just embraced that
> > > > >>>>>> and turned it
> > > > >>>> into
> > > > >>>>>> something less like a heavyweight framework and more like a
> > > > >>>>>> third
> > > > >>> Kafka
> > > > >>>>>> client--a kind of "producing consumer" with state management
> > > > >>>> facilities.
> > > > >>>>>> Basically a library. Instead of a complex stream processing
> > > > >>>>>> framework
> > > > >>>>> this
> > > > >>>>>> would actually be a very simple thing, not much more
> complicated
> > > > >>>>>> to
> > > > >>> use
> > > > >>>>> or
> > > > >>>>>> operate than a Kafka consumer. As Chris said we thought about
> it
> > > > >>>>>> a
> > > > >>> lot
> > > > >>>> of
> > > > >>>>>> what Samza (and the other stream processing systems were
> doing)
> > > > >>> seemed
> > > > >>>>> like
> > > > >>>>>> kind of a hangover from MapReduce.
> > > > >>>>>>
> > > > >>>>>> Of course you need to ingest/output data to and from the
> stream
> > > > >>>>>> processing. But when we actually looked into how that would
> > > > >>>>>> work,
> > > > >>> Samza
> > > > >>>>>> isn't really an ideal data ingestion framework for a bunch of
> > > > >>> reasons.
> > > > >>>> To
> > > > >>>>>> really do that right you need a pretty different internal data
> > > > >>>>>> model
> > > > >>>> and
> > > > >>>>>> set of apis. So what if you split them and had an api for
> Kafka
> > > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
> Kafka
> > > > >>>>>> transformation (Samza).
> > > > >>>>>>
> > > > >>>>>> This would also allow really embracing the same terminology
> and
> > > > >>>>>> conventions. One complaint about the current state is that the
> > > > >>>>>> two
> > > > >>>>> systems
> > > > >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic"
> and
> > > > >>>>> different
> > > > >>>>>> config and monitoring systems means you kind of have to learn
> > > > >>>>>> Kafka's
> > > > >>>>> way,
> > > > >>>>>> then learn Samza's slightly different way, then kind of
> > > > >>>>>> understand
> > > > >>> how
> > > > >>>>> they
> > > > >>>>>> map to each other, which having walked a few people through
> this
> > > > >>>>>> is surprisingly tricky for folks to get.
> > > > >>>>>>
> > > > >>>>>> Since I have been spending a lot of time on airplanes I hacked
> > > > >>>>>> up an ernest but still somewhat incomplete prototype of what
> > > > >>>>>> this would
> > > > >>> look
> > > > >>>>>> like. This is just unceremoniously dumped into Kafka as it
> > > > >>>>>> required a
> > > > >>>> few
> > > > >>>>>> changes to the new consumer. Here is the code:
> > > > >>>
> > > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > > >>> /apache/kafka/clients/streaming
> > > > >>>>>>
> > > > >>>>>> For the purpose of the prototype I just liberally renamed
> > > > >>>>>> everything
> > > > >>> to
> > > > >>>>>> try to align it with Kafka with no regard for compatibility.
> > > > >>>>>>
> > > > >>>>>> To use this would be something like this:
> > > > >>>>>> Properties props = new Properties();
> > > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > > >>>>>> StreamingConfig config = new
> > > > >>> StreamingConfig(props);
> > > > >>>>> config.subscribe("test-topic-1",
> > > > >>>>>> "test-topic-2");
> config.processor(ExampleStreamProcessor.class);
> > > > >>>>> config.serialization(new
> > > > >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> > > > >>>> container =
> > > > >>>>>> new KafkaStreaming(config); container.run();
> > > > >>>>>>
> > > > >>>>>> KafkaStreaming is basically the SamzaContainer;
> StreamProcessor
> > > > >>>>>> is basically StreamTask.
> > > > >>>>>>
> > > > >>>>>> So rather than putting all the class names in a file and then
> > > > >>>>>> having
> > > > >>>> the
> > > > >>>>>> job assembled by reflection, you just instantiate the
> container
> > > > >>>>>> programmatically. Work is balanced over however many instances
> > > > >>>>>> of
> > > > >>> this
> > > > >>>>> are
> > > > >>>>>> alive at any time (i.e. if an instance dies, new tasks are
> added
> > > > >>>>>> to
> > > > >>> the
> > > > >>>>>> existing containers without shutting them down).
> > > > >>>>>>
> > > > >>>>>> We would provide some glue for running this stuff in YARN via
> > > > >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> > > > >>>>>> but from the
> > > > >>>> point
> > > > >>>>> of
> > > > >>>>>> view of these frameworks these stream processing jobs are just
> > > > >>>> stateless
> > > > >>>>>> services that can come and go and expand and contract at will.
> > > > >>>>>> There
> > > > >>> is
> > > > >>>>> no
> > > > >>>>>> more custom scheduler.
> > > > >>>>>>
> > > > >>>>>> Here are some relevant details:
> > > > >>>>>>
> > > > >>>>>>   1. It is only ~1300 lines of code, it would get larger if we
> > > > >>>>>>   productionized but not vastly larger. We really do get a ton
> > > > >>>>>> of
> > > > >>>>> leverage
> > > > >>>>>>   out of Kafka.
> > > > >>>>>>   2. Partition management is fully delegated to the new
> > consumer.
> > > > >>> This
> > > > >>>>>>   is nice since now any partition management strategy
> available
> > > > >>>>>> to
> > > > >>>> Kafka
> > > > >>>>>>   consumer is also available to Samza (and vice versa) and
> with
> > > > >>>>>> the
> > > > >>>>> exact
> > > > >>>>>>   same configs.
> > > > >>>>>>   3. It supports state as well as state reuse
> > > > >>>>>>
> > > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
> > > > >>>>>>
> > > > >>>>>> -Jay
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > > >>>> criccomini@apache.org>
> > > > >>>>>> wrote:
> > > > >>>>>>
> > > > >>>>>>> Hey all,
> > > > >>>>>>>
> > > > >>>>>>> I have had some discussions with Samza engineers at LinkedIn
> > > > >>>>>>> and
> > > > >>>>> Confluent
> > > > >>>>>>> and we came up with a few observations and would like to
> > > > >>>>>>> propose
> > > > >>> some
> > > > >>>>>>> changes.
> > > > >>>>>>>
> > > > >>>>>>> We've observed some things that I want to call out about
> > > > >>>>>>> Samza's
> > > > >>>> design,
> > > > >>>>>>> and I'd like to propose some changes.
> > > > >>>>>>>
> > > > >>>>>>> * Samza is dependent upon a dynamic deployment system.
> > > > >>>>>>> * Samza is too pluggable.
> > > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> > > > >>>>>>> APIs
> > > > >>> are
> > > > >>>>>>> trying to solve a lot of the same problems.
> > > > >>>>>>>
> > > > >>>>>>> All three of these issues are related, but I'll address them
> in
> > > > >>> order.
> > > > >>>>>>>
> > > > >>>>>>> Deployment
> > > > >>>>>>>
> > > > >>>>>>> Samza strongly depends on the use of a dynamic deployment
> > > > >>>>>>> scheduler
> > > > >>>> such
> > > > >>>>>>> as
> > > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> > > > >>>>>>> there
> > > > >>>> would
> > > > >>>>>>> be
> > > > >>>>>>> one or two winners in this area, and we could support them,
> and
> > > > >>>>>>> the
> > > > >>>> rest
> > > > >>>>>>> would go away. In reality, there are many variations.
> > > > >>>>>>> Furthermore,
> > > > >>>> many
> > > > >>>>>>> people still prefer to just start their processors like
> normal
> > > > >>>>>>> Java processes, and use traditional deployment scripts such
> as
> > > > >>>>>>> Fabric,
> > > > >>>> Chef,
> > > > >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> > > > >>>>>>> Samza start-up process really painful for first time users.
> > > > >>>>>>>
> > > > >>>>>>> Dynamic deployment as a requirement was also a bit of a
> > > > >>>>>>> mis-fire
> > > > >>>> because
> > > > >>>>>>> of
> > > > >>>>>>> a fundamental misunderstanding between the nature of batch
> jobs
> > > > >>>>>>> and
> > > > >>>>> stream
> > > > >>>>>>> processing jobs. Early on, we made conscious effort to favor
> > > > >>>>>>> the
> > > > >>>> Hadoop
> > > > >>>>>>> (Map/Reduce) way of doing things, since it worked and was
> well
> > > > >>>>> understood.
> > > > >>>>>>> One thing that we missed was that batch jobs have a definite
> > > > >>>> beginning,
> > > > >>>>>>> and
> > > > >>>>>>> end, and stream processing jobs don't (usually). This leads
> to
> > > > >>>>>>> a
> > > > >>> much
> > > > >>>>>>> simpler scheduling problem for stream processors. You
> basically
> > > > >>>>>>> just
> > > > >>>>> need
> > > > >>>>>>> to find a place to start the processor, and start it. The way
> > > > >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> > > > >>>>>>> being "full". We always
> > > > >>>> add
> > > > >>>>>>> more machines. The problem with coupling Samza with a
> scheduler
> > > > >>>>>>> is
> > > > >>>> that
> > > > >>>>>>> Samza (as a framework) now has to handle deployment. This
> pulls
> > > > >>>>>>> in a
> > > > >>>>> bunch
> > > > >>>>>>> of things such as configuration distribution (config stream),
> > > > >>>>>>> shell
> > > > >>>>> scrips
> > > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff),
> > etc.
> > > > >>>>>>>
> > > > >>>>>>> Another reason for requiring dynamic deployment was to
> support
> > > > >>>>>>> data locality. If you want to have locality, you need to put
> > > > >>>>>>> your
> > > > >>>> processors
> > > > >>>>>>> close to the data they're processing. Upon further
> > > > >>>>>>> investigation,
> > > > >>>>> though,
> > > > >>>>>>> this feature is not that beneficial. There is some good
> > > > >>>>>>> discussion
> > > > >>>> about
> > > > >>>>>>> some problems with it on SAMZA-335. Again, we took the
> > > > >>>>>>> Map/Reduce
> > > > >>>> path,
> > > > >>>>>>> but
> > > > >>>>>>> there are some fundamental differences between HDFS and
> Kafka.
> > > > >>>>>>> HDFS
> > > > >>>> has
> > > > >>>>>>> blocks, while Kafka has partitions. This leads to less
> > > > >>>>>>> optimization potential with stream processors on top of
> Kafka.
> > > > >>>>>>>
> > > > >>>>>>> This feature is also used as a crutch. Samza doesn't have any
> > > > >>>>>>> built
> > > > >>> in
> > > > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> > > > >>>>>>> deployment scheduling system to handle restarts when a
> > > > >>>>>>> processor dies. This has
> > > > >>>>> made
> > > > >>>>>>> it very difficult to write a standalone Samza container
> > > > >> (SAMZA-516).
> > > > >>>>>>>
> > > > >>>>>>> Pluggability
> > > > >>>>>>>
> > > > >>>>>>> In some cases pluggability is good, but I think that we've
> gone
> > > > >>>>>>> too
> > > > >>>> far
> > > > >>>>>>> with it. Currently, Samza has:
> > > > >>>>>>>
> > > > >>>>>>> * Pluggable config.
> > > > >>>>>>> * Pluggable metrics.
> > > > >>>>>>> * Pluggable deployment systems.
> > > > >>>>>>> * Pluggable streaming systems (SystemConsumer,
> SystemProducer,
> > > > >> etc).
> > > > >>>>>>> * Pluggable serdes.
> > > > >>>>>>> * Pluggable storage engines.
> > > > >>>>>>> * Pluggable strategies for just about every component
> > > > >>> (MessageChooser,
> > > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > > >>>>>>>
> > > > >>>>>>> There's probably more that I've forgotten, as well. Some of
> > > > >>>>>>> these
> > > > >>> are
> > > > >>>>>>> useful, but some have proven not to be. This all comes at a
> > cost:
> > > > >>>>>>> complexity. This complexity is making it harder for our users
> > > > >>>>>>> to
> > > > >>> pick
> > > > >>>> up
> > > > >>>>>>> and use Samza out of the box. It also makes it difficult for
> > > > >>>>>>> Samza developers to reason about what the characteristics of
> > > > >>>>>>> the container (since the characteristics change depending on
> > > > >>>>>>> which plugins are use).
> > > > >>>>>>>
> > > > >>>>>>> The issues with pluggability are most visible in the System
> > APIs.
> > > > >>> What
> > > > >>>>>>> Samza really requires to be functional is Kafka as its
> > > > >>>>>>> transport
> > > > >>>> layer.
> > > > >>>>>>> But
> > > > >>>>>>> we've conflated two unrelated use cases into one API:
> > > > >>>>>>>
> > > > >>>>>>> 1. Get data into/out of Kafka.
> > > > >>>>>>> 2. Process the data in Kafka.
> > > > >>>>>>>
> > > > >>>>>>> The current System API supports both of these use cases. The
> > > > >>>>>>> problem
> > > > >>>> is,
> > > > >>>>>>> we
> > > > >>>>>>> actually want different features for each use case. By
> papering
> > > > >>>>>>> over
> > > > >>>>> these
> > > > >>>>>>> two use cases, and providing a single API, we've introduced a
> > > > >>>>>>> ton of
> > > > >>>>> leaky
> > > > >>>>>>> abstractions.
> > > > >>>>>>>
> > > > >>>>>>> For example, what we'd really like in (2) is to have
> > > > >>>>>>> monotonically increasing longs for offsets (like Kafka). This
> > > > >>>>>>> would be at odds
> > > > >>> with
> > > > >>>>> (1),
> > > > >>>>>>> though, since different systems have different
> > > > >>>>> SCNs/Offsets/UUIDs/vectors.
> > > > >>>>>>> There was discussion both on the mailing list and the SQL
> JIRAs
> > > > >>> about
> > > > >>>>> the
> > > > >>>>>>> need for this.
> > > > >>>>>>>
> > > > >>>>>>> The same thing holds true for replayability. Kafka allows us
> to
> > > > >>> rewind
> > > > >>>>>>> when
> > > > >>>>>>> we have a failure. Many other systems don't. In some cases,
> > > > >>>>>>> systems
> > > > >>>>> return
> > > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
> > > > >>>>>>> they
> > > > >>>> have
> > > > >>>>> no
> > > > >>>>>>> offsets.
> > > > >>>>>>>
> > > > >>>>>>> Partitioning is another example. Kafka supports partitioning,
> > > > >>>>>>> but
> > > > >>> many
> > > > >>>>>>> systems don't. We model this by having a single partition for
> > > > >>>>>>> those systems. Still, other systems model partitioning
> > > > >> differently (e.g.
> > > > >>>>>>> Kinesis).
> > > > >>>>>>>
> > > > >>>>>>> The SystemAdmin interface is also a mess. Creating streams
> in a
> > > > >>>>>>> system-agnostic way is almost impossible. As is modeling
> > > > >>>>>>> metadata
> > > > >>> for
> > > > >>>>> the
> > > > >>>>>>> system (replication factor, partitions, location, etc). The
> > > > >>>>>>> list
> > > > >>> goes
> > > > >>>>> on.
> > > > >>>>>>>
> > > > >>>>>>> Duplicate work
> > > > >>>>>>>
> > > > >>>>>>> At the time that we began writing Samza, Kafka's consumer and
> > > > >>> producer
> > > > >>>>>>> APIs
> > > > >>>>>>> had a relatively weak feature set. On the consumer-side, you
> > > > >>>>>>> had two
> > > > >>>>>>> options: use the high level consumer, or the simple consumer.
> > > > >>>>>>> The
> > > > >>>>> problem
> > > > >>>>>>> with the high-level consumer was that it controlled your
> > > > >>>>>>> offsets, partition assignments, and the order in which you
> > > > >>>>>>> received messages. The
> > > > >>> problem
> > > > >>>>>>> with
> > > > >>>>>>> the simple consumer is that it's not simple. It's basic. You
> > > > >>>>>>> end up
> > > > >>>>> having
> > > > >>>>>>> to handle a lot of really low-level stuff that you shouldn't.
> > > > >>>>>>> We
> > > > >>>> spent a
> > > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust.
> It
> > > > >>>>>>> also allows us to support some cool features:
> > > > >>>>>>>
> > > > >>>>>>> * Per-partition message ordering and prioritization.
> > > > >>>>>>> * Tight control over partition assignment to support joins,
> > > > >>>>>>> global
> > > > >>>> state
> > > > >>>>>>> (if we want to implement it :)), etc.
> > > > >>>>>>> * Tight control over offset checkpointing.
> > > > >>>>>>>
> > > > >>>>>>> What we didn't realize at the time is that these features
> > > > >>>>>>> should
> > > > >>>>> actually
> > > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> > > > >>>> processors)
> > > > >>>>>>> end up wanting to do things like joins and partition
> > > > >>>>>>> assignment. The
> > > > >>>>> Kafka
> > > > >>>>>>> community has come to the same conclusion. They're adding a
> ton
> > > > >>>>>>> of upgrades into their new Kafka consumer implementation. To
> a
> > > > >>>>>>> large extent,
> > > > >>> it's
> > > > >>>>>>> duplicate work to what we've already done in Samza.
> > > > >>>>>>>
> > > > >>>>>>> On top of this, Kafka ended up taking a very similar approach
> > > > >>>>>>> to
> > > > >>>> Samza's
> > > > >>>>>>> KafkaCheckpointManager implementation for handling offset
> > > > >>>> checkpointing.
> > > > >>>>>>> Like Samza, Kafka's new offset management feature stores
> offset
> > > > >>>>>>> checkpoints in a topic, and allows you to fetch them from the
> > > > >>>>>>> broker.
> > > > >>>>>>>
> > > > >>>>>>> A lot of this seems like a waste, since we could have shared
> > > > >>>>>>> the
> > > > >>> work
> > > > >>>> if
> > > > >>>>>>> it
> > > > >>>>>>> had been done in Kafka from the get-go.
> > > > >>>>>>>
> > > > >>>>>>> Vision
> > > > >>>>>>>
> > > > >>>>>>> All of this leads me to a rather radical proposal. Samza is
> > > > >>> relatively
> > > > >>>>>>> stable at this point. I'd venture to say that we're near a
> 1.0
> > > > >>>> release.
> > > > >>>>>>> I'd
> > > > >>>>>>> like to propose that we take what we've learned, and begin
> > > > >>>>>>> thinking
> > > > >>>>> about
> > > > >>>>>>> Samza beyond 1.0. What would we change if we were starting
> from
> > > > >>>> scratch?
> > > > >>>>>>> My
> > > > >>>>>>> proposal is to:
> > > > >>>>>>>
> > > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > > > >>>>>>> processors, and eliminate all direct dependences on YARN,
> > Mesos,
> > > > >> etc.
> > > > >>>>>>> 2. Make a definitive call to support only Kafka as the stream
> > > > >>>> processing
> > > > >>>>>>> layer.
> > > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> > > > >>>>>>> config
> > > > >>>>> systems,
> > > > >>>>>>> and simply use Kafka's instead.
> > > > >>>>>>>
> > > > >>>>>>> This would fix all of the issues that I outlined above. It
> > > > >>>>>>> should
> > > > >>> also
> > > > >>>>>>> shrink the Samza code base pretty dramatically. Supporting
> only
> > > > >>>>>>> a standalone container will allow Samza to be executed on
> YARN
> > > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> > > > >>>>>>> in-house
> > > > >>>>> deployment
> > > > >>>>>>> systems. This should make life a lot easier for new users.
> > > > >>>>>>> Imagine
> > > > >>>>> having
> > > > >>>>>>> the hello-samza tutorial without YARN. The drop in mailing
> list
> > > > >>>> traffic
> > > > >>>>>>> will be pretty dramatic.
> > > > >>>>>>>
> > > > >>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
> > > > >>> everyone
> > > > >>>>>>> that
> > > > >>>>>>> I'm aware of is using Samza with Kafka. We basically require
> it
> > > > >>>> already
> > > > >>>>> in
> > > > >>>>>>> order for most features to work. Those that are using other
> > > > >>>>>>> systems
> > > > >>>> are
> > > > >>>>>>> generally using it for ingest into Kafka (1), and then they
> do
> > > > >>>>>>> the processing on top. There is already discussion (
> > > > >>>
> > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > > >>> 767
> > > > >>>>>>> )
> > > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > > > >>>>>>>
> > > > >>>>>>> Once we make the call to couple with Kafka, we can leverage a
> > > > >>>>>>> ton of
> > > > >>>>> their
> > > > >>>>>>> ecosystem. We no longer have to maintain our own config,
> > > > >>>>>>> metrics,
> > > > >>> etc.
> > > > >>>>> We
> > > > >>>>>>> can all share the same libraries, and make them better. This
> > > > >>>>>>> will
> > > > >>> also
> > > > >>>>>>> allow us to share the consumer/producer APIs, and will let us
> > > > >>> leverage
> > > > >>>>>>> their offset management and partition management, rather than
> > > > >>>>>>> having
> > > > >>>> our
> > > > >>>>>>> own. All of the coordinator stream code would go away, as
> would
> > > > >>>>>>> most
> > > > >>>> of
> > > > >>>>>>> the
> > > > >>>>>>> YARN AppMaster code. We'd probably have to push some
> partition
> > > > >>>>> management
> > > > >>>>>>> features into the Kafka broker, but they're already moving in
> > > > >>>>>>> that direction with the new consumer API. The features we
> have
> > > > >>>>>>> for
> > > > >>>> partition
> > > > >>>>>>> assignment aren't unique to Samza, and seem like they should
> be
> > > > >>>>>>> in
> > > > >>>> Kafka
> > > > >>>>>>> anyway. There will always be some niche usages which will
> > > > >>>>>>> require
> > > > >>>> extra
> > > > >>>>>>> care and hence full control over partition assignments much
> > > > >>>>>>> like the
> > > > >>>>> Kafka
> > > > >>>>>>> low level consumer api. These would continue to be supported.
> > > > >>>>>>>
> > > > >>>>>>> These items will be good for the Samza community. They'll
> make
> > > > >>>>>>> Samza easier to use, and make it easier for developers to add
> > > > >>>>>>> new features.
> > > > >>>>>>>
> > > > >>>>>>> Obviously this is a fairly large (and somewhat backwards
> > > > >>> incompatible
> > > > >>>>>>> change). If we choose to go this route, it's important that
> we
> > > > >>> openly
> > > > >>>>>>> communicate how we're going to provide a migration path from
> > > > >>>>>>> the
> > > > >>>>> existing
> > > > >>>>>>> APIs to the new ones (if we make incompatible changes). I
> think
> > > > >>>>>>> at a minimum, we'd probably need to provide a wrapper to
> allow
> > > > >>>>>>> existing StreamTask implementations to continue running on
> the
> > > > >> new container.
> > > > >>>>> It's
> > > > >>>>>>> also important that we openly communicate about timing, and
> > > > >>>>>>> stages
> > > > >>> of
> > > > >>>>> the
> > > > >>>>>>> migration.
> > > > >>>>>>>
> > > > >>>>>>> If you made it this far, I'm sure you have opinions. :)
> Please
> > > > >>>>>>> send
> > > > >>>> your
> > > > >>>>>>> thoughts and feedback.
> > > > >>>>>>>
> > > > >>>>>>> Cheers,
> > > > >>>>>>> Chris
> > > > >>>>
> > > > >>>>
> > > > >>>>
> > > > >>>> --
> > > > >>>> -- Guozhang
> > > > >>
> > > >
> > >
> >
>



-- 
-- Guozhang

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
@Jay, yes, that's the point.
{quote}
allow the job to control partition assignment without having to deploy a
custom partition assignment strategy to the Kafka broker
{quote}

Thanks!

-Yi

On Thu, Jul 2, 2015 at 2:57 PM, Jay Kreps <ja...@confluent.io> wrote:

> Hey Yi,
>
> I think the goal of this is to allow the job to control partition
> assignment without having to deploy a custom partition assignment strategy
> to the Kafka broker, is that right?
>
> The regex support and dynamic topic discovery you get for free as the
> consumer needs to do that anyway.
>
> -Jay
>
> On Thu, Jul 2, 2015 at 2:52 PM, Yi Pan <ni...@gmail.com> wrote:
>
> > One more use case we had encountered that needs an explicit dynamic
> > PartitionManager/JobCoordinator outside Kafka broker is: there are use
> > cases that a Samza job needs to consume all Kafka topics matching a
> certain
> > regex, and users want the newly added topics to be assigned in runtime.
> > There is a need to have a dynamic discovery module for new topics and
> > assign the new topic partitions to the Samza workers. IMO, this should be
> > the functionality in a PartitionManager outside the Kafka broker, since
> it
> > is part of the application logic.
> >
> > Said all that, my main point is simple: I am proposing that we need a
> > pluggable partition management component, decoupled from the framework to
> > do resource assignment, process restart, etc.
> >
> > On Thu, Jul 2, 2015 at 2:35 PM, Yi Pan <ni...@gmail.com> wrote:
> >
> > > @Jay, yes, the current function in the JobCoordinator is just partition
> > > management. Maybe we should just call it PartitionManager to make it
> > > explicit.
> > >
> > > -Yi
> > >
> > > On Thu, Jul 2, 2015 at 2:24 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > >> Hey Yi,
> > >>
> > >> What does the JobCoordinator do? YARN/Mesos/etc would be doing the
> > actual
> > >> resource assignment, process restart, etc, right? Is the additional
> > value
> > >> add of the JobCoordinator just partition management?
> > >>
> > >> -Jay
> > >>
> > >> On Thu, Jul 2, 2015 at 11:32 AM, Yi Pan <ni...@gmail.com> wrote:
> > >>
> > >> > Hi, all,
> > >> >
> > >> >
> > >> > Thanks Chris for sending out this proposal and Jay for sharing the
> > >> > extremely illustrative prototype code.
> > >> >
> > >> >
> > >> > I have been thinking it over many times and want to list out my
> > personal
> > >> > opinions below:
> > >> >
> > >> > 1. Generally, I agree with most of the people here on the mailing
> list
> > >> on
> > >> > two points:
> > >> >
> > >> >    a. Deeper integration w/ Kafka is great. No more confusing
> mapping
> > >> from
> > >> > SystemStreamPartition to TopicPartition etc.
> > >> >
> > >> >    b. Separation the ingestion vs transformation greatly simplify
> the
> > >> > systems APIs
> > >> >
> > >> > Having the above two changes would allow us to remove many
> unnecessary
> > >> > complexities introduced by those pluggable interfaces Chris’ pointed
> > >> out,
> > >> > e.g. pluggable streaming systems and serde.
> > >> >
> > >> >
> > >> > To recall one of Chris’s statement on difficulties in dynamic
> > >> deployment, I
> > >> > believe that the difficulties are mainly the result of
> tight-coupling
> > of
> > >> > partition assignment vs the container deployment in the current
> > system.
> > >> The
> > >> > current container deployment requires a pre-defined partition
> > assignment
> > >> > strategy coupled together w/ the deployment configuration before we
> > can
> > >> > submit to YARN and start the Samza container, which makes the
> > launching
> > >> > process super long. Also, fault-tolerance and the embedded
> > >> JobCoordinator
> > >> > code in YARN AppMaster is another way of  making dynamic deployment
> > more
> > >> > complex and difficult.
> > >> >
> > >> >
> > >> > First, borrowing Yan’s term, let’s call the Samza standalone
> process a
> > >> > Samza worker. Here is what I have been thinking:
> > >> >
> > >> > 1. Separate the execution framework from partition assignment/load
> > >> > balancing:
> > >> >
> > >> >     a. a Samza worker should be launched by execution framework that
> > >> only
> > >> > deals w/ process placement to available nodes. The execution
> framework
> > >> now
> > >> > should only deal w/ how many such processes are needed, where to put
> > >> them,
> > >> > and how to keep them alive.
> > >> >
> > >> >     b. Partition assignment/load balancing can be a pluggable
> > interface
> > >> in
> > >> > Samza that allows the Samza workers to ask for partition
> assignments.
> > >> Let’s
> > >> > borrow the name JobCoordinator for now. To allow fault-tolerance in
> > >> case of
> > >> > failure, the partition assignments to workers need to be dynamic.
> > Hence,
> > >> > the abstract interface would be much like what Jay’s code
> illustrate:
> > >> > get()/onAssigned()/onRevoke(). The implementation of the partition
> > >> > assignment can be either:
> > >> >
> > >> >         a) completely rely on Kafka.
> > >> >
> > >> >         b) explicit partition assignment via JobCoordinator. Chris’s
> > >> work
> > >> > in SAMZA-516 can be easily incorporated here. The use case in
> SAMZA-41
> > >> that
> > >> > runs Samza ProcessJob w/ static partition assignment can be
> > implemented
> > >> of
> > >> > JobCoordinator via any home-grown implementation of distributed
> > >> > coordinator. All the work we did in LinkedIn to support dynamic
> > >> partition
> > >> > assignment and host-affinity SAMZA-617 can be nicely reused as an
> > >> > implementation of JobCoordinator.
> > >> >
> > >> >
> > >> > When we did the above work, I can see three usage patterns in Samza:
> > >> >
> > >> >    a. Samza as a library: Samza has a set of libraries to provide
> > stream
> > >> > processing, just like a third Kafka client (as illustrated in Jay’s
> > >> > example). The execution/deployment is totally controlled by the
> > >> application
> > >> > and the partition coordination is done via Kafka
> > >> >
> > >> >    b. Samza as a process: Samza runs as a standalone process. There
> > may
> > >> not
> > >> > be a execution framework to launch and deploy Samza processes. The
> > >> > partition assignment is pluggable JobCoordinator.
> > >> >
> > >> >    c. Samza as a service: Samza runs as a collection of processes.
> > There
> > >> > will be an execution framework to allocate resource, launch and
> deploy
> > >> > Samza workers and keep them alive. The same pluggable JobCoordinator
> > is
> > >> > desirable here as well.
> > >> >
> > >> >
> > >> > Lastly, I would argue that CopyCat in KIP-26 should probably follow
> > the
> > >> > same model. Hence, in Samza as a service model as in LinkedIn, we
> can
> > >> use
> > >> > the same fault tolerance execution framework to run CopyCat and
> Samza
> > >> w/o
> > >> > the need to operate two service platforms, which should address
> > Sriram’s
> > >> > comment in the email thread.
> > >> >
> > >> >
> > >> > Hope the above makes sense. Thanks all!
> > >> >
> > >> >
> > >> > -Yi
> > >> >
> > >> > On Thu, Jul 2, 2015 at 9:53 AM, Sriram <sr...@gmail.com>
> wrote:
> > >> >
> > >> > > One thing that is worth exploring is to have a transformation and
> > >> > > ingestion library in Kafka but use the same framework for fault
> > >> > tolerance,
> > >> > > resource isolation and management. The biggest difference I see in
> > >> these
> > >> > > two use cases is the API and data model.
> > >> > >
> > >> > >
> > >> > > > On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io> wrote:
> > >> > > >
> > >> > > > Hey Garry,
> > >> > > >
> > >> > > > Yeah that's super frustrating. I'd be happy to chat more about
> > this
> > >> if
> > >> > > > you'd be interested. I think Chris and I started with the idea
> of
> > >> "what
> > >> > > > would it take to make Samza a kick-ass ingestion tool" but
> > >> ultimately
> > >> > we
> > >> > > > kind of came around to the idea that ingestion and
> transformation
> > >> had
> > >> > > > pretty different needs and coupling the two made things hard.
> > >> > > >
> > >> > > > For what it's worth I think copycat (KIP-26) actually will do
> what
> > >> you
> > >> > > are
> > >> > > > looking for.
> > >> > > >
> > >> > > > With regard to your point about slider, I don't necessarily
> > >> disagree.
> > >> > > But I
> > >> > > > think getting good YARN support is quite doable and I think we
> can
> > >> make
> > >> > > > that work well. I think the issue this proposal solves is that
> > >> > > technically
> > >> > > > it is pretty hard to support multiple cluster management systems
> > the
> > >> > way
> > >> > > > things are now, you need to write an "app master" or "framework"
> > for
> > >> > each
> > >> > > > and they are all a little different so testing is really hard.
> In
> > >> the
> > >> > > > absence of this we have been stuck with just YARN which has
> > >> fantastic
> > >> > > > penetration in the Hadoopy part of the org, but zero penetration
> > >> > > elsewhere.
> > >> > > > Given the huge amount of work being put in to slider, marathon,
> > aws
> > >> > > > tooling, not to mention the umpteen related packaging
> technologies
> > >> > people
> > >> > > > want to use (Docker, Kubernetes, various cloud-specific deploy
> > >> tools,
> > >> > > etc)
> > >> > > > I really think it is important to get this right.
> > >> > > >
> > >> > > > -Jay
> > >> > > >
> > >> > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > >> > > > g.turkington@improvedigital.com> wrote:
> > >> > > >
> > >> > > >> Hi all,
> > >> > > >>
> > >> > > >> I think the question below re does Samza become a sub-project
> of
> > >> Kafka
> > >> > > >> highlights the broader point around migration. Chris mentions
> > >> Samza's
> > >> > > >> maturity is heading towards a v1 release but I'm not sure it
> > feels
> > >> > > right to
> > >> > > >> launch a v1 then immediately plan to deprecate most of it.
> > >> > > >>
> > >> > > >> From a selfish perspective I have some guys who have started
> > >> working
> > >> > > with
> > >> > > >> Samza and building some new consumers/producers was next up.
> > Sounds
> > >> > like
> > >> > > >> that is absolutely not the direction to go. I need to look into
> > the
> > >> > KIP
> > >> > > in
> > >> > > >> more detail but for me the attractiveness of adding new Samza
> > >> > > >> consumer/producers -- even if yes all they were doing was
> really
> > >> > getting
> > >> > > >> data into and out of Kafka --  was to avoid  having to worry
> > about
> > >> the
> > >> > > >> lifecycle management of external clients. If there is a generic
> > >> Kafka
> > >> > > >> ingress/egress layer that I can plug a new connector into and
> > have
> > >> a
> > >> > > lot of
> > >> > > >> the heavy lifting re scale and reliability done for me then it
> > >> gives
> > >> > me
> > >> > > all
> > >> > > >> the pushing new consumers/producers would. If not then it
> > >> complicates
> > >> > my
> > >> > > >> operational deployments.
> > >> > > >>
> > >> > > >> Which is similar to my other question with the proposal -- if
> we
> > >> > build a
> > >> > > >> fully available/stand-alone Samza plus the requisite shims to
> > >> > integrate
> > >> > > >> with Slider etc I suspect the former may be a lot more work
> than
> > we
> > >> > > think.
> > >> > > >> We may make it much easier for a newcomer to get something
> > running
> > >> but
> > >> > > >> having them step up and get a reliable production deployment
> may
> > >> still
> > >> > > >> dominate mailing list  traffic, if for different reasons than
> > >> today.
> > >> > > >>
> > >> > > >> Don't get me wrong -- I'm comfortable with making the Samza
> > >> dependency
> > >> > > on
> > >> > > >> Kafka much more explicit and I absolutely see the benefits  in
> > the
> > >> > > >> reduction of duplication and clashing
> terminologies/abstractions
> > >> that
> > >> > > >> Chris/Jay describe. Samza as a library would likely be a very
> > nice
> > >> > tool
> > >> > > to
> > >> > > >> add to the Kafka ecosystem. I just have the concerns above re
> the
> > >> > > >> operational side.
> > >> > > >>
> > >> > > >> Garry
> > >> > > >>
> > >> > > >> -----Original Message-----
> > >> > > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> > >> > > >> Sent: 02 July 2015 12:56
> > >> > > >> To: dev@samza.apache.org
> > >> > > >> Subject: Re: Thoughts and obesrvations on Samza
> > >> > > >>
> > >> > > >> Very interesting thoughts.
> > >> > > >> From outside, I have always perceived Samza as a computing
> layer
> > >> over
> > >> > > >> Kafka.
> > >> > > >>
> > >> > > >> The question, maybe a bit provocative, is "should Samza be a
> > >> > sub-project
> > >> > > >> of Kafka then?"
> > >> > > >> Or does it make sense to keep it as a separate project with a
> > >> separate
> > >> > > >> governance?
> > >> > > >>
> > >> > > >> Cheers,
> > >> > > >>
> > >> > > >> --
> > >> > > >> Gianmarco
> > >> > > >>
> > >> > > >>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com>
> wrote:
> > >> > > >>>
> > >> > > >>> Overall, I agree to couple with Kafka more tightly. Because
> > Samza
> > >> de
> > >> > > >>> facto is based on Kafka, and it should leverage what Kafka
> has.
> > At
> > >> > the
> > >> > > >>> same time, Kafka does not need to reinvent what Samza already
> > >> has. I
> > >> > > >>> also like the idea of separating the ingestion and
> > transformation.
> > >> > > >>>
> > >> > > >>> But it is a little difficult for me to image how the Samza
> will
> > >> look
> > >> > > >> like.
> > >> > > >>> And I feel Chris and Jay have a little difference in terms of
> > how
> > >> > > >>> Samza should look like.
> > >> > > >>>
> > >> > > >>> *** Will it look like what Jay's code shows (A client of
> Kakfa)
> > ?
> > >> And
> > >> > > >>> user's application code calls this client?
> > >> > > >>>
> > >> > > >>> 1. If we make Samza be a library of Kafka (like what the code
> > >> shows),
> > >> > > >>> how do we implement auto-balance and fault-tolerance? Are they
> > >> taken
> > >> > > >>> care by the Kafka broker or other mechanism, such as "Samza
> > >> worker"
> > >> > > >>> (just make up the name) ?
> > >> > > >>>
> > >> > > >>> 2. What about other features, such as auto-scaling, shared
> > state,
> > >> > > >>> monitoring?
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> *** If we have Samza standalone, (is this what Chris
> suggests?)
> > >> > > >>>
> > >> > > >>> 1. we still need to ingest data from Kakfa and produce to it.
> > >> Then it
> > >> > > >>> becomes the same as what Samza looks like now, except it does
> > not
> > >> > rely
> > >> > > >>> on Yarn anymore.
> > >> > > >>>
> > >> > > >>> 2. if it is standalone, how can it leverage Kafka's metrics,
> > logs,
> > >> > > >>> etc? Use Kafka code as the dependency?
> > >> > > >>>
> > >> > > >>>
> > >> > > >>> Thanks,
> > >> > > >>>
> > >> > > >>> Fang, Yan
> > >> > > >>> yanfang724@gmail.com
> > >> > > >>>
> > >> > > >>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> > >> wangguoz@gmail.com>
> > >> > > >>> wrote:
> > >> > > >>>
> > >> > > >>>> Read through the code example and it looks good to me. A few
> > >> > > >>>> thoughts regarding deployment:
> > >> > > >>>>
> > >> > > >>>> Today Samza deploys as executable runnable like:
> > >> > > >>>>
> > >> > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
> > >> > > >> --config-path=file://...
> > >> > > >>>>
> > >> > > >>>> And this proposal advocate for deploying Samza more as
> embedded
> > >> > > >>>> libraries in user application code (ignoring the terminology
> > >> since
> > >> > > >>>> it is not the
> > >> > > >>> same
> > >> > > >>>> as the prototype code):
> > >> > > >>>>
> > >> > > >>>> StreamTask task = new MyStreamTask(configs); Thread thread =
> > new
> > >> > > >>>> Thread(task); thread.start();
> > >> > > >>>>
> > >> > > >>>> I think both of these deployment modes are important for
> > >> different
> > >> > > >>>> types
> > >> > > >>> of
> > >> > > >>>> users. That said, I think making Samza purely standalone is
> > still
> > >> > > >>>> sufficient for either runnable or library modes.
> > >> > > >>>>
> > >> > > >>>> Guozhang
> > >> > > >>>>
> > >> > > >>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <
> jay@confluent.io
> > >
> > >> > > wrote:
> > >> > > >>>>>
> > >> > > >>>>> Looks like gmail mangled the code example, it was supposed
> to
> > >> look
> > >> > > >>>>> like
> > >> > > >>>>> this:
> > >> > > >>>>>
> > >> > > >>>>> Properties props = new Properties();
> > >> > > >>>>> props.put("bootstrap.servers", "localhost:4242");
> > >> StreamingConfig
> > >> > > >>>>> config = new StreamingConfig(props);
> > >> > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
> > >> > > >>>>> config.processor(ExampleStreamProcessor.class);
> > >> > > >>>>> config.serialization(new StringSerializer(), new
> > >> > > >>>>> StringDeserializer()); KafkaStreaming container = new
> > >> > > >>>>> KafkaStreaming(config); container.run();
> > >> > > >>>>>
> > >> > > >>>>> -Jay
> > >> > > >>>>>
> > >> > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <
> jay@confluent.io
> > >
> > >> > > >> wrote:
> > >> > > >>>>>
> > >> > > >>>>>> Hey guys,
> > >> > > >>>>>>
> > >> > > >>>>>> This came out of some conversations Chris and I were having
> > >> > > >>>>>> around
> > >> > > >>>>> whether
> > >> > > >>>>>> it would make sense to use Samza as a kind of data
> ingestion
> > >> > > >>> framework
> > >> > > >>>>> for
> > >> > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This
> kind
> > of
> > >> > > >>>> combined
> > >> > > >>>>>> with complaints around config and YARN and the discussion
> > >> around
> > >> > > >>>>>> how
> > >> > > >>> to
> > >> > > >>>>>> best do a standalone mode.
> > >> > > >>>>>>
> > >> > > >>>>>> So the thought experiment was, given that Samza was
> basically
> > >> > > >>>>>> already totally Kafka specific, what if you just embraced
> > that
> > >> > > >>>>>> and turned it
> > >> > > >>>> into
> > >> > > >>>>>> something less like a heavyweight framework and more like a
> > >> > > >>>>>> third
> > >> > > >>> Kafka
> > >> > > >>>>>> client--a kind of "producing consumer" with state
> management
> > >> > > >>>> facilities.
> > >> > > >>>>>> Basically a library. Instead of a complex stream processing
> > >> > > >>>>>> framework
> > >> > > >>>>> this
> > >> > > >>>>>> would actually be a very simple thing, not much more
> > >> complicated
> > >> > > >>>>>> to
> > >> > > >>> use
> > >> > > >>>>> or
> > >> > > >>>>>> operate than a Kafka consumer. As Chris said we thought
> about
> > >> it
> > >> > > >>>>>> a
> > >> > > >>> lot
> > >> > > >>>> of
> > >> > > >>>>>> what Samza (and the other stream processing systems were
> > doing)
> > >> > > >>> seemed
> > >> > > >>>>> like
> > >> > > >>>>>> kind of a hangover from MapReduce.
> > >> > > >>>>>>
> > >> > > >>>>>> Of course you need to ingest/output data to and from the
> > stream
> > >> > > >>>>>> processing. But when we actually looked into how that would
> > >> > > >>>>>> work,
> > >> > > >>> Samza
> > >> > > >>>>>> isn't really an ideal data ingestion framework for a bunch
> of
> > >> > > >>> reasons.
> > >> > > >>>> To
> > >> > > >>>>>> really do that right you need a pretty different internal
> > data
> > >> > > >>>>>> model
> > >> > > >>>> and
> > >> > > >>>>>> set of apis. So what if you split them and had an api for
> > Kafka
> > >> > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
> > >> Kafka
> > >> > > >>>>>> transformation (Samza).
> > >> > > >>>>>>
> > >> > > >>>>>> This would also allow really embracing the same terminology
> > and
> > >> > > >>>>>> conventions. One complaint about the current state is that
> > the
> > >> > > >>>>>> two
> > >> > > >>>>> systems
> > >> > > >>>>>> kind of feel bolted on. Terminology like "stream" vs
> "topic"
> > >> and
> > >> > > >>>>> different
> > >> > > >>>>>> config and monitoring systems means you kind of have to
> learn
> > >> > > >>>>>> Kafka's
> > >> > > >>>>> way,
> > >> > > >>>>>> then learn Samza's slightly different way, then kind of
> > >> > > >>>>>> understand
> > >> > > >>> how
> > >> > > >>>>> they
> > >> > > >>>>>> map to each other, which having walked a few people through
> > >> this
> > >> > > >>>>>> is surprisingly tricky for folks to get.
> > >> > > >>>>>>
> > >> > > >>>>>> Since I have been spending a lot of time on airplanes I
> > hacked
> > >> > > >>>>>> up an ernest but still somewhat incomplete prototype of
> what
> > >> > > >>>>>> this would
> > >> > > >>> look
> > >> > > >>>>>> like. This is just unceremoniously dumped into Kafka as it
> > >> > > >>>>>> required a
> > >> > > >>>> few
> > >> > > >>>>>> changes to the new consumer. Here is the code:
> > >> > > >>>
> > >> >
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >> > > >>> /apache/kafka/clients/streaming
> > >> > > >>>>>>
> > >> > > >>>>>> For the purpose of the prototype I just liberally renamed
> > >> > > >>>>>> everything
> > >> > > >>> to
> > >> > > >>>>>> try to align it with Kafka with no regard for
> compatibility.
> > >> > > >>>>>>
> > >> > > >>>>>> To use this would be something like this:
> > >> > > >>>>>> Properties props = new Properties();
> > >> > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
> > >> > > >>>>>> StreamingConfig config = new
> > >> > > >>> StreamingConfig(props);
> > >> > > >>>>> config.subscribe("test-topic-1",
> > >> > > >>>>>> "test-topic-2");
> > >> config.processor(ExampleStreamProcessor.class);
> > >> > > >>>>> config.serialization(new
> > >> > > >>>>>> StringSerializer(), new StringDeserializer());
> KafkaStreaming
> > >> > > >>>> container =
> > >> > > >>>>>> new KafkaStreaming(config); container.run();
> > >> > > >>>>>>
> > >> > > >>>>>> KafkaStreaming is basically the SamzaContainer;
> > StreamProcessor
> > >> > > >>>>>> is basically StreamTask.
> > >> > > >>>>>>
> > >> > > >>>>>> So rather than putting all the class names in a file and
> then
> > >> > > >>>>>> having
> > >> > > >>>> the
> > >> > > >>>>>> job assembled by reflection, you just instantiate the
> > container
> > >> > > >>>>>> programmatically. Work is balanced over however many
> > instances
> > >> > > >>>>>> of
> > >> > > >>> this
> > >> > > >>>>> are
> > >> > > >>>>>> alive at any time (i.e. if an instance dies, new tasks are
> > >> added
> > >> > > >>>>>> to
> > >> > > >>> the
> > >> > > >>>>>> existing containers without shutting them down).
> > >> > > >>>>>>
> > >> > > >>>>>> We would provide some glue for running this stuff in YARN
> via
> > >> > > >>>>>> Slider, Mesos via Marathon, and AWS using some of their
> tools
> > >> > > >>>>>> but from the
> > >> > > >>>> point
> > >> > > >>>>> of
> > >> > > >>>>>> view of these frameworks these stream processing jobs are
> > just
> > >> > > >>>> stateless
> > >> > > >>>>>> services that can come and go and expand and contract at
> > will.
> > >> > > >>>>>> There
> > >> > > >>> is
> > >> > > >>>>> no
> > >> > > >>>>>> more custom scheduler.
> > >> > > >>>>>>
> > >> > > >>>>>> Here are some relevant details:
> > >> > > >>>>>>
> > >> > > >>>>>>   1. It is only ~1300 lines of code, it would get larger if
> > we
> > >> > > >>>>>>   productionized but not vastly larger. We really do get a
> > ton
> > >> > > >>>>>> of
> > >> > > >>>>> leverage
> > >> > > >>>>>>   out of Kafka.
> > >> > > >>>>>>   2. Partition management is fully delegated to the new
> > >> consumer.
> > >> > > >>> This
> > >> > > >>>>>>   is nice since now any partition management strategy
> > available
> > >> > > >>>>>> to
> > >> > > >>>> Kafka
> > >> > > >>>>>>   consumer is also available to Samza (and vice versa) and
> > with
> > >> > > >>>>>> the
> > >> > > >>>>> exact
> > >> > > >>>>>>   same configs.
> > >> > > >>>>>>   3. It supports state as well as state reuse
> > >> > > >>>>>>
> > >> > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
> > >> > > >>>>>>
> > >> > > >>>>>> -Jay
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > >> > > >>>> criccomini@apache.org>
> > >> > > >>>>>> wrote:
> > >> > > >>>>>>
> > >> > > >>>>>>> Hey all,
> > >> > > >>>>>>>
> > >> > > >>>>>>> I have had some discussions with Samza engineers at
> LinkedIn
> > >> > > >>>>>>> and
> > >> > > >>>>> Confluent
> > >> > > >>>>>>> and we came up with a few observations and would like to
> > >> > > >>>>>>> propose
> > >> > > >>> some
> > >> > > >>>>>>> changes.
> > >> > > >>>>>>>
> > >> > > >>>>>>> We've observed some things that I want to call out about
> > >> > > >>>>>>> Samza's
> > >> > > >>>> design,
> > >> > > >>>>>>> and I'd like to propose some changes.
> > >> > > >>>>>>>
> > >> > > >>>>>>> * Samza is dependent upon a dynamic deployment system.
> > >> > > >>>>>>> * Samza is too pluggable.
> > >> > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's
> consumer
> > >> > > >>>>>>> APIs
> > >> > > >>> are
> > >> > > >>>>>>> trying to solve a lot of the same problems.
> > >> > > >>>>>>>
> > >> > > >>>>>>> All three of these issues are related, but I'll address
> them
> > >> in
> > >> > > >>> order.
> > >> > > >>>>>>>
> > >> > > >>>>>>> Deployment
> > >> > > >>>>>>>
> > >> > > >>>>>>> Samza strongly depends on the use of a dynamic deployment
> > >> > > >>>>>>> scheduler
> > >> > > >>>> such
> > >> > > >>>>>>> as
> > >> > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet
> that
> > >> > > >>>>>>> there
> > >> > > >>>> would
> > >> > > >>>>>>> be
> > >> > > >>>>>>> one or two winners in this area, and we could support
> them,
> > >> and
> > >> > > >>>>>>> the
> > >> > > >>>> rest
> > >> > > >>>>>>> would go away. In reality, there are many variations.
> > >> > > >>>>>>> Furthermore,
> > >> > > >>>> many
> > >> > > >>>>>>> people still prefer to just start their processors like
> > normal
> > >> > > >>>>>>> Java processes, and use traditional deployment scripts
> such
> > as
> > >> > > >>>>>>> Fabric,
> > >> > > >>>> Chef,
> > >> > > >>>>>>> Ansible, etc. Forcing a deployment system on users makes
> the
> > >> > > >>>>>>> Samza start-up process really painful for first time
> users.
> > >> > > >>>>>>>
> > >> > > >>>>>>> Dynamic deployment as a requirement was also a bit of a
> > >> > > >>>>>>> mis-fire
> > >> > > >>>> because
> > >> > > >>>>>>> of
> > >> > > >>>>>>> a fundamental misunderstanding between the nature of batch
> > >> jobs
> > >> > > >>>>>>> and
> > >> > > >>>>> stream
> > >> > > >>>>>>> processing jobs. Early on, we made conscious effort to
> favor
> > >> > > >>>>>>> the
> > >> > > >>>> Hadoop
> > >> > > >>>>>>> (Map/Reduce) way of doing things, since it worked and was
> > well
> > >> > > >>>>> understood.
> > >> > > >>>>>>> One thing that we missed was that batch jobs have a
> definite
> > >> > > >>>> beginning,
> > >> > > >>>>>>> and
> > >> > > >>>>>>> end, and stream processing jobs don't (usually). This
> leads
> > to
> > >> > > >>>>>>> a
> > >> > > >>> much
> > >> > > >>>>>>> simpler scheduling problem for stream processors. You
> > >> basically
> > >> > > >>>>>>> just
> > >> > > >>>>> need
> > >> > > >>>>>>> to find a place to start the processor, and start it. The
> > way
> > >> > > >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> > >> > > >>>>>>> being "full". We always
> > >> > > >>>> add
> > >> > > >>>>>>> more machines. The problem with coupling Samza with a
> > >> scheduler
> > >> > > >>>>>>> is
> > >> > > >>>> that
> > >> > > >>>>>>> Samza (as a framework) now has to handle deployment. This
> > >> pulls
> > >> > > >>>>>>> in a
> > >> > > >>>>> bunch
> > >> > > >>>>>>> of things such as configuration distribution (config
> > stream),
> > >> > > >>>>>>> shell
> > >> > > >>>>> scrips
> > >> > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz
> stuff),
> > >> etc.
> > >> > > >>>>>>>
> > >> > > >>>>>>> Another reason for requiring dynamic deployment was to
> > support
> > >> > > >>>>>>> data locality. If you want to have locality, you need to
> put
> > >> > > >>>>>>> your
> > >> > > >>>> processors
> > >> > > >>>>>>> close to the data they're processing. Upon further
> > >> > > >>>>>>> investigation,
> > >> > > >>>>> though,
> > >> > > >>>>>>> this feature is not that beneficial. There is some good
> > >> > > >>>>>>> discussion
> > >> > > >>>> about
> > >> > > >>>>>>> some problems with it on SAMZA-335. Again, we took the
> > >> > > >>>>>>> Map/Reduce
> > >> > > >>>> path,
> > >> > > >>>>>>> but
> > >> > > >>>>>>> there are some fundamental differences between HDFS and
> > Kafka.
> > >> > > >>>>>>> HDFS
> > >> > > >>>> has
> > >> > > >>>>>>> blocks, while Kafka has partitions. This leads to less
> > >> > > >>>>>>> optimization potential with stream processors on top of
> > Kafka.
> > >> > > >>>>>>>
> > >> > > >>>>>>> This feature is also used as a crutch. Samza doesn't have
> > any
> > >> > > >>>>>>> built
> > >> > > >>> in
> > >> > > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> > >> > > >>>>>>> deployment scheduling system to handle restarts when a
> > >> > > >>>>>>> processor dies. This has
> > >> > > >>>>> made
> > >> > > >>>>>>> it very difficult to write a standalone Samza container
> > >> > > >> (SAMZA-516).
> > >> > > >>>>>>>
> > >> > > >>>>>>> Pluggability
> > >> > > >>>>>>>
> > >> > > >>>>>>> In some cases pluggability is good, but I think that we've
> > >> gone
> > >> > > >>>>>>> too
> > >> > > >>>> far
> > >> > > >>>>>>> with it. Currently, Samza has:
> > >> > > >>>>>>>
> > >> > > >>>>>>> * Pluggable config.
> > >> > > >>>>>>> * Pluggable metrics.
> > >> > > >>>>>>> * Pluggable deployment systems.
> > >> > > >>>>>>> * Pluggable streaming systems (SystemConsumer,
> > SystemProducer,
> > >> > > >> etc).
> > >> > > >>>>>>> * Pluggable serdes.
> > >> > > >>>>>>> * Pluggable storage engines.
> > >> > > >>>>>>> * Pluggable strategies for just about every component
> > >> > > >>> (MessageChooser,
> > >> > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > >> > > >>>>>>>
> > >> > > >>>>>>> There's probably more that I've forgotten, as well. Some
> of
> > >> > > >>>>>>> these
> > >> > > >>> are
> > >> > > >>>>>>> useful, but some have proven not to be. This all comes at
> a
> > >> cost:
> > >> > > >>>>>>> complexity. This complexity is making it harder for our
> > users
> > >> > > >>>>>>> to
> > >> > > >>> pick
> > >> > > >>>> up
> > >> > > >>>>>>> and use Samza out of the box. It also makes it difficult
> for
> > >> > > >>>>>>> Samza developers to reason about what the characteristics
> of
> > >> > > >>>>>>> the container (since the characteristics change depending
> on
> > >> > > >>>>>>> which plugins are use).
> > >> > > >>>>>>>
> > >> > > >>>>>>> The issues with pluggability are most visible in the
> System
> > >> APIs.
> > >> > > >>> What
> > >> > > >>>>>>> Samza really requires to be functional is Kafka as its
> > >> > > >>>>>>> transport
> > >> > > >>>> layer.
> > >> > > >>>>>>> But
> > >> > > >>>>>>> we've conflated two unrelated use cases into one API:
> > >> > > >>>>>>>
> > >> > > >>>>>>> 1. Get data into/out of Kafka.
> > >> > > >>>>>>> 2. Process the data in Kafka.
> > >> > > >>>>>>>
> > >> > > >>>>>>> The current System API supports both of these use cases.
> The
> > >> > > >>>>>>> problem
> > >> > > >>>> is,
> > >> > > >>>>>>> we
> > >> > > >>>>>>> actually want different features for each use case. By
> > >> papering
> > >> > > >>>>>>> over
> > >> > > >>>>> these
> > >> > > >>>>>>> two use cases, and providing a single API, we've
> introduced
> > a
> > >> > > >>>>>>> ton of
> > >> > > >>>>> leaky
> > >> > > >>>>>>> abstractions.
> > >> > > >>>>>>>
> > >> > > >>>>>>> For example, what we'd really like in (2) is to have
> > >> > > >>>>>>> monotonically increasing longs for offsets (like Kafka).
> > This
> > >> > > >>>>>>> would be at odds
> > >> > > >>> with
> > >> > > >>>>> (1),
> > >> > > >>>>>>> though, since different systems have different
> > >> > > >>>>> SCNs/Offsets/UUIDs/vectors.
> > >> > > >>>>>>> There was discussion both on the mailing list and the SQL
> > >> JIRAs
> > >> > > >>> about
> > >> > > >>>>> the
> > >> > > >>>>>>> need for this.
> > >> > > >>>>>>>
> > >> > > >>>>>>> The same thing holds true for replayability. Kafka allows
> us
> > >> to
> > >> > > >>> rewind
> > >> > > >>>>>>> when
> > >> > > >>>>>>> we have a failure. Many other systems don't. In some
> cases,
> > >> > > >>>>>>> systems
> > >> > > >>>>> return
> > >> > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> > because
> > >> > > >>>>>>> they
> > >> > > >>>> have
> > >> > > >>>>> no
> > >> > > >>>>>>> offsets.
> > >> > > >>>>>>>
> > >> > > >>>>>>> Partitioning is another example. Kafka supports
> > partitioning,
> > >> > > >>>>>>> but
> > >> > > >>> many
> > >> > > >>>>>>> systems don't. We model this by having a single partition
> > for
> > >> > > >>>>>>> those systems. Still, other systems model partitioning
> > >> > > >> differently (e.g.
> > >> > > >>>>>>> Kinesis).
> > >> > > >>>>>>>
> > >> > > >>>>>>> The SystemAdmin interface is also a mess. Creating streams
> > in
> > >> a
> > >> > > >>>>>>> system-agnostic way is almost impossible. As is modeling
> > >> > > >>>>>>> metadata
> > >> > > >>> for
> > >> > > >>>>> the
> > >> > > >>>>>>> system (replication factor, partitions, location, etc).
> The
> > >> > > >>>>>>> list
> > >> > > >>> goes
> > >> > > >>>>> on.
> > >> > > >>>>>>>
> > >> > > >>>>>>> Duplicate work
> > >> > > >>>>>>>
> > >> > > >>>>>>> At the time that we began writing Samza, Kafka's consumer
> > and
> > >> > > >>> producer
> > >> > > >>>>>>> APIs
> > >> > > >>>>>>> had a relatively weak feature set. On the consumer-side,
> you
> > >> > > >>>>>>> had two
> > >> > > >>>>>>> options: use the high level consumer, or the simple
> > consumer.
> > >> > > >>>>>>> The
> > >> > > >>>>> problem
> > >> > > >>>>>>> with the high-level consumer was that it controlled your
> > >> > > >>>>>>> offsets, partition assignments, and the order in which you
> > >> > > >>>>>>> received messages. The
> > >> > > >>> problem
> > >> > > >>>>>>> with
> > >> > > >>>>>>> the simple consumer is that it's not simple. It's basic.
> You
> > >> > > >>>>>>> end up
> > >> > > >>>>> having
> > >> > > >>>>>>> to handle a lot of really low-level stuff that you
> > shouldn't.
> > >> > > >>>>>>> We
> > >> > > >>>> spent a
> > >> > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very
> robust.
> > >> It
> > >> > > >>>>>>> also allows us to support some cool features:
> > >> > > >>>>>>>
> > >> > > >>>>>>> * Per-partition message ordering and prioritization.
> > >> > > >>>>>>> * Tight control over partition assignment to support
> joins,
> > >> > > >>>>>>> global
> > >> > > >>>> state
> > >> > > >>>>>>> (if we want to implement it :)), etc.
> > >> > > >>>>>>> * Tight control over offset checkpointing.
> > >> > > >>>>>>>
> > >> > > >>>>>>> What we didn't realize at the time is that these features
> > >> > > >>>>>>> should
> > >> > > >>>>> actually
> > >> > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza
> stream
> > >> > > >>>> processors)
> > >> > > >>>>>>> end up wanting to do things like joins and partition
> > >> > > >>>>>>> assignment. The
> > >> > > >>>>> Kafka
> > >> > > >>>>>>> community has come to the same conclusion. They're adding
> a
> > >> ton
> > >> > > >>>>>>> of upgrades into their new Kafka consumer implementation.
> > To a
> > >> > > >>>>>>> large extent,
> > >> > > >>> it's
> > >> > > >>>>>>> duplicate work to what we've already done in Samza.
> > >> > > >>>>>>>
> > >> > > >>>>>>> On top of this, Kafka ended up taking a very similar
> > approach
> > >> > > >>>>>>> to
> > >> > > >>>> Samza's
> > >> > > >>>>>>> KafkaCheckpointManager implementation for handling offset
> > >> > > >>>> checkpointing.
> > >> > > >>>>>>> Like Samza, Kafka's new offset management feature stores
> > >> offset
> > >> > > >>>>>>> checkpoints in a topic, and allows you to fetch them from
> > the
> > >> > > >>>>>>> broker.
> > >> > > >>>>>>>
> > >> > > >>>>>>> A lot of this seems like a waste, since we could have
> shared
> > >> > > >>>>>>> the
> > >> > > >>> work
> > >> > > >>>> if
> > >> > > >>>>>>> it
> > >> > > >>>>>>> had been done in Kafka from the get-go.
> > >> > > >>>>>>>
> > >> > > >>>>>>> Vision
> > >> > > >>>>>>>
> > >> > > >>>>>>> All of this leads me to a rather radical proposal. Samza
> is
> > >> > > >>> relatively
> > >> > > >>>>>>> stable at this point. I'd venture to say that we're near a
> > 1.0
> > >> > > >>>> release.
> > >> > > >>>>>>> I'd
> > >> > > >>>>>>> like to propose that we take what we've learned, and begin
> > >> > > >>>>>>> thinking
> > >> > > >>>>> about
> > >> > > >>>>>>> Samza beyond 1.0. What would we change if we were starting
> > >> from
> > >> > > >>>> scratch?
> > >> > > >>>>>>> My
> > >> > > >>>>>>> proposal is to:
> > >> > > >>>>>>>
> > >> > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > >> > > >>>>>>> processors, and eliminate all direct dependences on YARN,
> > >> Mesos,
> > >> > > >> etc.
> > >> > > >>>>>>> 2. Make a definitive call to support only Kafka as the
> > stream
> > >> > > >>>> processing
> > >> > > >>>>>>> layer.
> > >> > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> > >> > > >>>>>>> config
> > >> > > >>>>> systems,
> > >> > > >>>>>>> and simply use Kafka's instead.
> > >> > > >>>>>>>
> > >> > > >>>>>>> This would fix all of the issues that I outlined above. It
> > >> > > >>>>>>> should
> > >> > > >>> also
> > >> > > >>>>>>> shrink the Samza code base pretty dramatically. Supporting
> > >> only
> > >> > > >>>>>>> a standalone container will allow Samza to be executed on
> > YARN
> > >> > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most
> other
> > >> > > >>>>>>> in-house
> > >> > > >>>>> deployment
> > >> > > >>>>>>> systems. This should make life a lot easier for new users.
> > >> > > >>>>>>> Imagine
> > >> > > >>>>> having
> > >> > > >>>>>>> the hello-samza tutorial without YARN. The drop in mailing
> > >> list
> > >> > > >>>> traffic
> > >> > > >>>>>>> will be pretty dramatic.
> > >> > > >>>>>>>
> > >> > > >>>>>>> Coupling with Kafka seems long overdue to me. The reality
> > is,
> > >> > > >>> everyone
> > >> > > >>>>>>> that
> > >> > > >>>>>>> I'm aware of is using Samza with Kafka. We basically
> require
> > >> it
> > >> > > >>>> already
> > >> > > >>>>> in
> > >> > > >>>>>>> order for most features to work. Those that are using
> other
> > >> > > >>>>>>> systems
> > >> > > >>>> are
> > >> > > >>>>>>> generally using it for ingest into Kafka (1), and then
> they
> > do
> > >> > > >>>>>>> the processing on top. There is already discussion (
> > >> > > >>>
> > >> >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >> > > >>> 767
> > >> > > >>>>>>> )
> > >> > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > >> > > >>>>>>>
> > >> > > >>>>>>> Once we make the call to couple with Kafka, we can
> leverage
> > a
> > >> > > >>>>>>> ton of
> > >> > > >>>>> their
> > >> > > >>>>>>> ecosystem. We no longer have to maintain our own config,
> > >> > > >>>>>>> metrics,
> > >> > > >>> etc.
> > >> > > >>>>> We
> > >> > > >>>>>>> can all share the same libraries, and make them better.
> This
> > >> > > >>>>>>> will
> > >> > > >>> also
> > >> > > >>>>>>> allow us to share the consumer/producer APIs, and will let
> > us
> > >> > > >>> leverage
> > >> > > >>>>>>> their offset management and partition management, rather
> > than
> > >> > > >>>>>>> having
> > >> > > >>>> our
> > >> > > >>>>>>> own. All of the coordinator stream code would go away, as
> > >> would
> > >> > > >>>>>>> most
> > >> > > >>>> of
> > >> > > >>>>>>> the
> > >> > > >>>>>>> YARN AppMaster code. We'd probably have to push some
> > partition
> > >> > > >>>>> management
> > >> > > >>>>>>> features into the Kafka broker, but they're already moving
> > in
> > >> > > >>>>>>> that direction with the new consumer API. The features we
> > have
> > >> > > >>>>>>> for
> > >> > > >>>> partition
> > >> > > >>>>>>> assignment aren't unique to Samza, and seem like they
> should
> > >> be
> > >> > > >>>>>>> in
> > >> > > >>>> Kafka
> > >> > > >>>>>>> anyway. There will always be some niche usages which will
> > >> > > >>>>>>> require
> > >> > > >>>> extra
> > >> > > >>>>>>> care and hence full control over partition assignments
> much
> > >> > > >>>>>>> like the
> > >> > > >>>>> Kafka
> > >> > > >>>>>>> low level consumer api. These would continue to be
> > supported.
> > >> > > >>>>>>>
> > >> > > >>>>>>> These items will be good for the Samza community. They'll
> > make
> > >> > > >>>>>>> Samza easier to use, and make it easier for developers to
> > add
> > >> > > >>>>>>> new features.
> > >> > > >>>>>>>
> > >> > > >>>>>>> Obviously this is a fairly large (and somewhat backwards
> > >> > > >>> incompatible
> > >> > > >>>>>>> change). If we choose to go this route, it's important
> that
> > we
> > >> > > >>> openly
> > >> > > >>>>>>> communicate how we're going to provide a migration path
> from
> > >> > > >>>>>>> the
> > >> > > >>>>> existing
> > >> > > >>>>>>> APIs to the new ones (if we make incompatible changes). I
> > >> think
> > >> > > >>>>>>> at a minimum, we'd probably need to provide a wrapper to
> > allow
> > >> > > >>>>>>> existing StreamTask implementations to continue running on
> > the
> > >> > > >> new container.
> > >> > > >>>>> It's
> > >> > > >>>>>>> also important that we openly communicate about timing,
> and
> > >> > > >>>>>>> stages
> > >> > > >>> of
> > >> > > >>>>> the
> > >> > > >>>>>>> migration.
> > >> > > >>>>>>>
> > >> > > >>>>>>> If you made it this far, I'm sure you have opinions. :)
> > Please
> > >> > > >>>>>>> send
> > >> > > >>>> your
> > >> > > >>>>>>> thoughts and feedback.
> > >> > > >>>>>>>
> > >> > > >>>>>>> Cheers,
> > >> > > >>>>>>> Chris
> > >> > > >>>>
> > >> > > >>>>
> > >> > > >>>>
> > >> > > >>>> --
> > >> > > >>>> -- Guozhang
> > >> > > >>
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: Thoughts and obesrvations on Samza

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

I think the goal of this is to allow the job to control partition
assignment without having to deploy a custom partition assignment strategy
to the Kafka broker, is that right?

The regex support and dynamic topic discovery you get for free as the
consumer needs to do that anyway.

-Jay

On Thu, Jul 2, 2015 at 2:52 PM, Yi Pan <ni...@gmail.com> wrote:

> One more use case we had encountered that needs an explicit dynamic
> PartitionManager/JobCoordinator outside Kafka broker is: there are use
> cases that a Samza job needs to consume all Kafka topics matching a certain
> regex, and users want the newly added topics to be assigned in runtime.
> There is a need to have a dynamic discovery module for new topics and
> assign the new topic partitions to the Samza workers. IMO, this should be
> the functionality in a PartitionManager outside the Kafka broker, since it
> is part of the application logic.
>
> Said all that, my main point is simple: I am proposing that we need a
> pluggable partition management component, decoupled from the framework to
> do resource assignment, process restart, etc.
>
> On Thu, Jul 2, 2015 at 2:35 PM, Yi Pan <ni...@gmail.com> wrote:
>
> > @Jay, yes, the current function in the JobCoordinator is just partition
> > management. Maybe we should just call it PartitionManager to make it
> > explicit.
> >
> > -Yi
> >
> > On Thu, Jul 2, 2015 at 2:24 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> >> Hey Yi,
> >>
> >> What does the JobCoordinator do? YARN/Mesos/etc would be doing the
> actual
> >> resource assignment, process restart, etc, right? Is the additional
> value
> >> add of the JobCoordinator just partition management?
> >>
> >> -Jay
> >>
> >> On Thu, Jul 2, 2015 at 11:32 AM, Yi Pan <ni...@gmail.com> wrote:
> >>
> >> > Hi, all,
> >> >
> >> >
> >> > Thanks Chris for sending out this proposal and Jay for sharing the
> >> > extremely illustrative prototype code.
> >> >
> >> >
> >> > I have been thinking it over many times and want to list out my
> personal
> >> > opinions below:
> >> >
> >> > 1. Generally, I agree with most of the people here on the mailing list
> >> on
> >> > two points:
> >> >
> >> >    a. Deeper integration w/ Kafka is great. No more confusing mapping
> >> from
> >> > SystemStreamPartition to TopicPartition etc.
> >> >
> >> >    b. Separation the ingestion vs transformation greatly simplify the
> >> > systems APIs
> >> >
> >> > Having the above two changes would allow us to remove many unnecessary
> >> > complexities introduced by those pluggable interfaces Chris’ pointed
> >> out,
> >> > e.g. pluggable streaming systems and serde.
> >> >
> >> >
> >> > To recall one of Chris’s statement on difficulties in dynamic
> >> deployment, I
> >> > believe that the difficulties are mainly the result of tight-coupling
> of
> >> > partition assignment vs the container deployment in the current
> system.
> >> The
> >> > current container deployment requires a pre-defined partition
> assignment
> >> > strategy coupled together w/ the deployment configuration before we
> can
> >> > submit to YARN and start the Samza container, which makes the
> launching
> >> > process super long. Also, fault-tolerance and the embedded
> >> JobCoordinator
> >> > code in YARN AppMaster is another way of  making dynamic deployment
> more
> >> > complex and difficult.
> >> >
> >> >
> >> > First, borrowing Yan’s term, let’s call the Samza standalone process a
> >> > Samza worker. Here is what I have been thinking:
> >> >
> >> > 1. Separate the execution framework from partition assignment/load
> >> > balancing:
> >> >
> >> >     a. a Samza worker should be launched by execution framework that
> >> only
> >> > deals w/ process placement to available nodes. The execution framework
> >> now
> >> > should only deal w/ how many such processes are needed, where to put
> >> them,
> >> > and how to keep them alive.
> >> >
> >> >     b. Partition assignment/load balancing can be a pluggable
> interface
> >> in
> >> > Samza that allows the Samza workers to ask for partition assignments.
> >> Let’s
> >> > borrow the name JobCoordinator for now. To allow fault-tolerance in
> >> case of
> >> > failure, the partition assignments to workers need to be dynamic.
> Hence,
> >> > the abstract interface would be much like what Jay’s code illustrate:
> >> > get()/onAssigned()/onRevoke(). The implementation of the partition
> >> > assignment can be either:
> >> >
> >> >         a) completely rely on Kafka.
> >> >
> >> >         b) explicit partition assignment via JobCoordinator. Chris’s
> >> work
> >> > in SAMZA-516 can be easily incorporated here. The use case in SAMZA-41
> >> that
> >> > runs Samza ProcessJob w/ static partition assignment can be
> implemented
> >> of
> >> > JobCoordinator via any home-grown implementation of distributed
> >> > coordinator. All the work we did in LinkedIn to support dynamic
> >> partition
> >> > assignment and host-affinity SAMZA-617 can be nicely reused as an
> >> > implementation of JobCoordinator.
> >> >
> >> >
> >> > When we did the above work, I can see three usage patterns in Samza:
> >> >
> >> >    a. Samza as a library: Samza has a set of libraries to provide
> stream
> >> > processing, just like a third Kafka client (as illustrated in Jay’s
> >> > example). The execution/deployment is totally controlled by the
> >> application
> >> > and the partition coordination is done via Kafka
> >> >
> >> >    b. Samza as a process: Samza runs as a standalone process. There
> may
> >> not
> >> > be a execution framework to launch and deploy Samza processes. The
> >> > partition assignment is pluggable JobCoordinator.
> >> >
> >> >    c. Samza as a service: Samza runs as a collection of processes.
> There
> >> > will be an execution framework to allocate resource, launch and deploy
> >> > Samza workers and keep them alive. The same pluggable JobCoordinator
> is
> >> > desirable here as well.
> >> >
> >> >
> >> > Lastly, I would argue that CopyCat in KIP-26 should probably follow
> the
> >> > same model. Hence, in Samza as a service model as in LinkedIn, we can
> >> use
> >> > the same fault tolerance execution framework to run CopyCat and Samza
> >> w/o
> >> > the need to operate two service platforms, which should address
> Sriram’s
> >> > comment in the email thread.
> >> >
> >> >
> >> > Hope the above makes sense. Thanks all!
> >> >
> >> >
> >> > -Yi
> >> >
> >> > On Thu, Jul 2, 2015 at 9:53 AM, Sriram <sr...@gmail.com> wrote:
> >> >
> >> > > One thing that is worth exploring is to have a transformation and
> >> > > ingestion library in Kafka but use the same framework for fault
> >> > tolerance,
> >> > > resource isolation and management. The biggest difference I see in
> >> these
> >> > > two use cases is the API and data model.
> >> > >
> >> > >
> >> > > > On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io> wrote:
> >> > > >
> >> > > > Hey Garry,
> >> > > >
> >> > > > Yeah that's super frustrating. I'd be happy to chat more about
> this
> >> if
> >> > > > you'd be interested. I think Chris and I started with the idea of
> >> "what
> >> > > > would it take to make Samza a kick-ass ingestion tool" but
> >> ultimately
> >> > we
> >> > > > kind of came around to the idea that ingestion and transformation
> >> had
> >> > > > pretty different needs and coupling the two made things hard.
> >> > > >
> >> > > > For what it's worth I think copycat (KIP-26) actually will do what
> >> you
> >> > > are
> >> > > > looking for.
> >> > > >
> >> > > > With regard to your point about slider, I don't necessarily
> >> disagree.
> >> > > But I
> >> > > > think getting good YARN support is quite doable and I think we can
> >> make
> >> > > > that work well. I think the issue this proposal solves is that
> >> > > technically
> >> > > > it is pretty hard to support multiple cluster management systems
> the
> >> > way
> >> > > > things are now, you need to write an "app master" or "framework"
> for
> >> > each
> >> > > > and they are all a little different so testing is really hard. In
> >> the
> >> > > > absence of this we have been stuck with just YARN which has
> >> fantastic
> >> > > > penetration in the Hadoopy part of the org, but zero penetration
> >> > > elsewhere.
> >> > > > Given the huge amount of work being put in to slider, marathon,
> aws
> >> > > > tooling, not to mention the umpteen related packaging technologies
> >> > people
> >> > > > want to use (Docker, Kubernetes, various cloud-specific deploy
> >> tools,
> >> > > etc)
> >> > > > I really think it is important to get this right.
> >> > > >
> >> > > > -Jay
> >> > > >
> >> > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> >> > > > g.turkington@improvedigital.com> wrote:
> >> > > >
> >> > > >> Hi all,
> >> > > >>
> >> > > >> I think the question below re does Samza become a sub-project of
> >> Kafka
> >> > > >> highlights the broader point around migration. Chris mentions
> >> Samza's
> >> > > >> maturity is heading towards a v1 release but I'm not sure it
> feels
> >> > > right to
> >> > > >> launch a v1 then immediately plan to deprecate most of it.
> >> > > >>
> >> > > >> From a selfish perspective I have some guys who have started
> >> working
> >> > > with
> >> > > >> Samza and building some new consumers/producers was next up.
> Sounds
> >> > like
> >> > > >> that is absolutely not the direction to go. I need to look into
> the
> >> > KIP
> >> > > in
> >> > > >> more detail but for me the attractiveness of adding new Samza
> >> > > >> consumer/producers -- even if yes all they were doing was really
> >> > getting
> >> > > >> data into and out of Kafka --  was to avoid  having to worry
> about
> >> the
> >> > > >> lifecycle management of external clients. If there is a generic
> >> Kafka
> >> > > >> ingress/egress layer that I can plug a new connector into and
> have
> >> a
> >> > > lot of
> >> > > >> the heavy lifting re scale and reliability done for me then it
> >> gives
> >> > me
> >> > > all
> >> > > >> the pushing new consumers/producers would. If not then it
> >> complicates
> >> > my
> >> > > >> operational deployments.
> >> > > >>
> >> > > >> Which is similar to my other question with the proposal -- if we
> >> > build a
> >> > > >> fully available/stand-alone Samza plus the requisite shims to
> >> > integrate
> >> > > >> with Slider etc I suspect the former may be a lot more work than
> we
> >> > > think.
> >> > > >> We may make it much easier for a newcomer to get something
> running
> >> but
> >> > > >> having them step up and get a reliable production deployment may
> >> still
> >> > > >> dominate mailing list  traffic, if for different reasons than
> >> today.
> >> > > >>
> >> > > >> Don't get me wrong -- I'm comfortable with making the Samza
> >> dependency
> >> > > on
> >> > > >> Kafka much more explicit and I absolutely see the benefits  in
> the
> >> > > >> reduction of duplication and clashing terminologies/abstractions
> >> that
> >> > > >> Chris/Jay describe. Samza as a library would likely be a very
> nice
> >> > tool
> >> > > to
> >> > > >> add to the Kafka ecosystem. I just have the concerns above re the
> >> > > >> operational side.
> >> > > >>
> >> > > >> Garry
> >> > > >>
> >> > > >> -----Original Message-----
> >> > > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> >> > > >> Sent: 02 July 2015 12:56
> >> > > >> To: dev@samza.apache.org
> >> > > >> Subject: Re: Thoughts and obesrvations on Samza
> >> > > >>
> >> > > >> Very interesting thoughts.
> >> > > >> From outside, I have always perceived Samza as a computing layer
> >> over
> >> > > >> Kafka.
> >> > > >>
> >> > > >> The question, maybe a bit provocative, is "should Samza be a
> >> > sub-project
> >> > > >> of Kafka then?"
> >> > > >> Or does it make sense to keep it as a separate project with a
> >> separate
> >> > > >> governance?
> >> > > >>
> >> > > >> Cheers,
> >> > > >>
> >> > > >> --
> >> > > >> Gianmarco
> >> > > >>
> >> > > >>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> >> > > >>>
> >> > > >>> Overall, I agree to couple with Kafka more tightly. Because
> Samza
> >> de
> >> > > >>> facto is based on Kafka, and it should leverage what Kafka has.
> At
> >> > the
> >> > > >>> same time, Kafka does not need to reinvent what Samza already
> >> has. I
> >> > > >>> also like the idea of separating the ingestion and
> transformation.
> >> > > >>>
> >> > > >>> But it is a little difficult for me to image how the Samza will
> >> look
> >> > > >> like.
> >> > > >>> And I feel Chris and Jay have a little difference in terms of
> how
> >> > > >>> Samza should look like.
> >> > > >>>
> >> > > >>> *** Will it look like what Jay's code shows (A client of Kakfa)
> ?
> >> And
> >> > > >>> user's application code calls this client?
> >> > > >>>
> >> > > >>> 1. If we make Samza be a library of Kafka (like what the code
> >> shows),
> >> > > >>> how do we implement auto-balance and fault-tolerance? Are they
> >> taken
> >> > > >>> care by the Kafka broker or other mechanism, such as "Samza
> >> worker"
> >> > > >>> (just make up the name) ?
> >> > > >>>
> >> > > >>> 2. What about other features, such as auto-scaling, shared
> state,
> >> > > >>> monitoring?
> >> > > >>>
> >> > > >>>
> >> > > >>> *** If we have Samza standalone, (is this what Chris suggests?)
> >> > > >>>
> >> > > >>> 1. we still need to ingest data from Kakfa and produce to it.
> >> Then it
> >> > > >>> becomes the same as what Samza looks like now, except it does
> not
> >> > rely
> >> > > >>> on Yarn anymore.
> >> > > >>>
> >> > > >>> 2. if it is standalone, how can it leverage Kafka's metrics,
> logs,
> >> > > >>> etc? Use Kafka code as the dependency?
> >> > > >>>
> >> > > >>>
> >> > > >>> Thanks,
> >> > > >>>
> >> > > >>> Fang, Yan
> >> > > >>> yanfang724@gmail.com
> >> > > >>>
> >> > > >>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
> >> wangguoz@gmail.com>
> >> > > >>> wrote:
> >> > > >>>
> >> > > >>>> Read through the code example and it looks good to me. A few
> >> > > >>>> thoughts regarding deployment:
> >> > > >>>>
> >> > > >>>> Today Samza deploys as executable runnable like:
> >> > > >>>>
> >> > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
> >> > > >> --config-path=file://...
> >> > > >>>>
> >> > > >>>> And this proposal advocate for deploying Samza more as embedded
> >> > > >>>> libraries in user application code (ignoring the terminology
> >> since
> >> > > >>>> it is not the
> >> > > >>> same
> >> > > >>>> as the prototype code):
> >> > > >>>>
> >> > > >>>> StreamTask task = new MyStreamTask(configs); Thread thread =
> new
> >> > > >>>> Thread(task); thread.start();
> >> > > >>>>
> >> > > >>>> I think both of these deployment modes are important for
> >> different
> >> > > >>>> types
> >> > > >>> of
> >> > > >>>> users. That said, I think making Samza purely standalone is
> still
> >> > > >>>> sufficient for either runnable or library modes.
> >> > > >>>>
> >> > > >>>> Guozhang
> >> > > >>>>
> >> > > >>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <jay@confluent.io
> >
> >> > > wrote:
> >> > > >>>>>
> >> > > >>>>> Looks like gmail mangled the code example, it was supposed to
> >> look
> >> > > >>>>> like
> >> > > >>>>> this:
> >> > > >>>>>
> >> > > >>>>> Properties props = new Properties();
> >> > > >>>>> props.put("bootstrap.servers", "localhost:4242");
> >> StreamingConfig
> >> > > >>>>> config = new StreamingConfig(props);
> >> > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
> >> > > >>>>> config.processor(ExampleStreamProcessor.class);
> >> > > >>>>> config.serialization(new StringSerializer(), new
> >> > > >>>>> StringDeserializer()); KafkaStreaming container = new
> >> > > >>>>> KafkaStreaming(config); container.run();
> >> > > >>>>>
> >> > > >>>>> -Jay
> >> > > >>>>>
> >> > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <jay@confluent.io
> >
> >> > > >> wrote:
> >> > > >>>>>
> >> > > >>>>>> Hey guys,
> >> > > >>>>>>
> >> > > >>>>>> This came out of some conversations Chris and I were having
> >> > > >>>>>> around
> >> > > >>>>> whether
> >> > > >>>>>> it would make sense to use Samza as a kind of data ingestion
> >> > > >>> framework
> >> > > >>>>> for
> >> > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind
> of
> >> > > >>>> combined
> >> > > >>>>>> with complaints around config and YARN and the discussion
> >> around
> >> > > >>>>>> how
> >> > > >>> to
> >> > > >>>>>> best do a standalone mode.
> >> > > >>>>>>
> >> > > >>>>>> So the thought experiment was, given that Samza was basically
> >> > > >>>>>> already totally Kafka specific, what if you just embraced
> that
> >> > > >>>>>> and turned it
> >> > > >>>> into
> >> > > >>>>>> something less like a heavyweight framework and more like a
> >> > > >>>>>> third
> >> > > >>> Kafka
> >> > > >>>>>> client--a kind of "producing consumer" with state management
> >> > > >>>> facilities.
> >> > > >>>>>> Basically a library. Instead of a complex stream processing
> >> > > >>>>>> framework
> >> > > >>>>> this
> >> > > >>>>>> would actually be a very simple thing, not much more
> >> complicated
> >> > > >>>>>> to
> >> > > >>> use
> >> > > >>>>> or
> >> > > >>>>>> operate than a Kafka consumer. As Chris said we thought about
> >> it
> >> > > >>>>>> a
> >> > > >>> lot
> >> > > >>>> of
> >> > > >>>>>> what Samza (and the other stream processing systems were
> doing)
> >> > > >>> seemed
> >> > > >>>>> like
> >> > > >>>>>> kind of a hangover from MapReduce.
> >> > > >>>>>>
> >> > > >>>>>> Of course you need to ingest/output data to and from the
> stream
> >> > > >>>>>> processing. But when we actually looked into how that would
> >> > > >>>>>> work,
> >> > > >>> Samza
> >> > > >>>>>> isn't really an ideal data ingestion framework for a bunch of
> >> > > >>> reasons.
> >> > > >>>> To
> >> > > >>>>>> really do that right you need a pretty different internal
> data
> >> > > >>>>>> model
> >> > > >>>> and
> >> > > >>>>>> set of apis. So what if you split them and had an api for
> Kafka
> >> > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
> >> Kafka
> >> > > >>>>>> transformation (Samza).
> >> > > >>>>>>
> >> > > >>>>>> This would also allow really embracing the same terminology
> and
> >> > > >>>>>> conventions. One complaint about the current state is that
> the
> >> > > >>>>>> two
> >> > > >>>>> systems
> >> > > >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic"
> >> and
> >> > > >>>>> different
> >> > > >>>>>> config and monitoring systems means you kind of have to learn
> >> > > >>>>>> Kafka's
> >> > > >>>>> way,
> >> > > >>>>>> then learn Samza's slightly different way, then kind of
> >> > > >>>>>> understand
> >> > > >>> how
> >> > > >>>>> they
> >> > > >>>>>> map to each other, which having walked a few people through
> >> this
> >> > > >>>>>> is surprisingly tricky for folks to get.
> >> > > >>>>>>
> >> > > >>>>>> Since I have been spending a lot of time on airplanes I
> hacked
> >> > > >>>>>> up an ernest but still somewhat incomplete prototype of what
> >> > > >>>>>> this would
> >> > > >>> look
> >> > > >>>>>> like. This is just unceremoniously dumped into Kafka as it
> >> > > >>>>>> required a
> >> > > >>>> few
> >> > > >>>>>> changes to the new consumer. Here is the code:
> >> > > >>>
> >> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >> > > >>> /apache/kafka/clients/streaming
> >> > > >>>>>>
> >> > > >>>>>> For the purpose of the prototype I just liberally renamed
> >> > > >>>>>> everything
> >> > > >>> to
> >> > > >>>>>> try to align it with Kafka with no regard for compatibility.
> >> > > >>>>>>
> >> > > >>>>>> To use this would be something like this:
> >> > > >>>>>> Properties props = new Properties();
> >> > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
> >> > > >>>>>> StreamingConfig config = new
> >> > > >>> StreamingConfig(props);
> >> > > >>>>> config.subscribe("test-topic-1",
> >> > > >>>>>> "test-topic-2");
> >> config.processor(ExampleStreamProcessor.class);
> >> > > >>>>> config.serialization(new
> >> > > >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> >> > > >>>> container =
> >> > > >>>>>> new KafkaStreaming(config); container.run();
> >> > > >>>>>>
> >> > > >>>>>> KafkaStreaming is basically the SamzaContainer;
> StreamProcessor
> >> > > >>>>>> is basically StreamTask.
> >> > > >>>>>>
> >> > > >>>>>> So rather than putting all the class names in a file and then
> >> > > >>>>>> having
> >> > > >>>> the
> >> > > >>>>>> job assembled by reflection, you just instantiate the
> container
> >> > > >>>>>> programmatically. Work is balanced over however many
> instances
> >> > > >>>>>> of
> >> > > >>> this
> >> > > >>>>> are
> >> > > >>>>>> alive at any time (i.e. if an instance dies, new tasks are
> >> added
> >> > > >>>>>> to
> >> > > >>> the
> >> > > >>>>>> existing containers without shutting them down).
> >> > > >>>>>>
> >> > > >>>>>> We would provide some glue for running this stuff in YARN via
> >> > > >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> >> > > >>>>>> but from the
> >> > > >>>> point
> >> > > >>>>> of
> >> > > >>>>>> view of these frameworks these stream processing jobs are
> just
> >> > > >>>> stateless
> >> > > >>>>>> services that can come and go and expand and contract at
> will.
> >> > > >>>>>> There
> >> > > >>> is
> >> > > >>>>> no
> >> > > >>>>>> more custom scheduler.
> >> > > >>>>>>
> >> > > >>>>>> Here are some relevant details:
> >> > > >>>>>>
> >> > > >>>>>>   1. It is only ~1300 lines of code, it would get larger if
> we
> >> > > >>>>>>   productionized but not vastly larger. We really do get a
> ton
> >> > > >>>>>> of
> >> > > >>>>> leverage
> >> > > >>>>>>   out of Kafka.
> >> > > >>>>>>   2. Partition management is fully delegated to the new
> >> consumer.
> >> > > >>> This
> >> > > >>>>>>   is nice since now any partition management strategy
> available
> >> > > >>>>>> to
> >> > > >>>> Kafka
> >> > > >>>>>>   consumer is also available to Samza (and vice versa) and
> with
> >> > > >>>>>> the
> >> > > >>>>> exact
> >> > > >>>>>>   same configs.
> >> > > >>>>>>   3. It supports state as well as state reuse
> >> > > >>>>>>
> >> > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
> >> > > >>>>>>
> >> > > >>>>>> -Jay
> >> > > >>>>>>
> >> > > >>>>>>
> >> > > >>>>>>
> >> > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> >> > > >>>> criccomini@apache.org>
> >> > > >>>>>> wrote:
> >> > > >>>>>>
> >> > > >>>>>>> Hey all,
> >> > > >>>>>>>
> >> > > >>>>>>> I have had some discussions with Samza engineers at LinkedIn
> >> > > >>>>>>> and
> >> > > >>>>> Confluent
> >> > > >>>>>>> and we came up with a few observations and would like to
> >> > > >>>>>>> propose
> >> > > >>> some
> >> > > >>>>>>> changes.
> >> > > >>>>>>>
> >> > > >>>>>>> We've observed some things that I want to call out about
> >> > > >>>>>>> Samza's
> >> > > >>>> design,
> >> > > >>>>>>> and I'd like to propose some changes.
> >> > > >>>>>>>
> >> > > >>>>>>> * Samza is dependent upon a dynamic deployment system.
> >> > > >>>>>>> * Samza is too pluggable.
> >> > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> >> > > >>>>>>> APIs
> >> > > >>> are
> >> > > >>>>>>> trying to solve a lot of the same problems.
> >> > > >>>>>>>
> >> > > >>>>>>> All three of these issues are related, but I'll address them
> >> in
> >> > > >>> order.
> >> > > >>>>>>>
> >> > > >>>>>>> Deployment
> >> > > >>>>>>>
> >> > > >>>>>>> Samza strongly depends on the use of a dynamic deployment
> >> > > >>>>>>> scheduler
> >> > > >>>> such
> >> > > >>>>>>> as
> >> > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> >> > > >>>>>>> there
> >> > > >>>> would
> >> > > >>>>>>> be
> >> > > >>>>>>> one or two winners in this area, and we could support them,
> >> and
> >> > > >>>>>>> the
> >> > > >>>> rest
> >> > > >>>>>>> would go away. In reality, there are many variations.
> >> > > >>>>>>> Furthermore,
> >> > > >>>> many
> >> > > >>>>>>> people still prefer to just start their processors like
> normal
> >> > > >>>>>>> Java processes, and use traditional deployment scripts such
> as
> >> > > >>>>>>> Fabric,
> >> > > >>>> Chef,
> >> > > >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> >> > > >>>>>>> Samza start-up process really painful for first time users.
> >> > > >>>>>>>
> >> > > >>>>>>> Dynamic deployment as a requirement was also a bit of a
> >> > > >>>>>>> mis-fire
> >> > > >>>> because
> >> > > >>>>>>> of
> >> > > >>>>>>> a fundamental misunderstanding between the nature of batch
> >> jobs
> >> > > >>>>>>> and
> >> > > >>>>> stream
> >> > > >>>>>>> processing jobs. Early on, we made conscious effort to favor
> >> > > >>>>>>> the
> >> > > >>>> Hadoop
> >> > > >>>>>>> (Map/Reduce) way of doing things, since it worked and was
> well
> >> > > >>>>> understood.
> >> > > >>>>>>> One thing that we missed was that batch jobs have a definite
> >> > > >>>> beginning,
> >> > > >>>>>>> and
> >> > > >>>>>>> end, and stream processing jobs don't (usually). This leads
> to
> >> > > >>>>>>> a
> >> > > >>> much
> >> > > >>>>>>> simpler scheduling problem for stream processors. You
> >> basically
> >> > > >>>>>>> just
> >> > > >>>>> need
> >> > > >>>>>>> to find a place to start the processor, and start it. The
> way
> >> > > >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> >> > > >>>>>>> being "full". We always
> >> > > >>>> add
> >> > > >>>>>>> more machines. The problem with coupling Samza with a
> >> scheduler
> >> > > >>>>>>> is
> >> > > >>>> that
> >> > > >>>>>>> Samza (as a framework) now has to handle deployment. This
> >> pulls
> >> > > >>>>>>> in a
> >> > > >>>>> bunch
> >> > > >>>>>>> of things such as configuration distribution (config
> stream),
> >> > > >>>>>>> shell
> >> > > >>>>> scrips
> >> > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff),
> >> etc.
> >> > > >>>>>>>
> >> > > >>>>>>> Another reason for requiring dynamic deployment was to
> support
> >> > > >>>>>>> data locality. If you want to have locality, you need to put
> >> > > >>>>>>> your
> >> > > >>>> processors
> >> > > >>>>>>> close to the data they're processing. Upon further
> >> > > >>>>>>> investigation,
> >> > > >>>>> though,
> >> > > >>>>>>> this feature is not that beneficial. There is some good
> >> > > >>>>>>> discussion
> >> > > >>>> about
> >> > > >>>>>>> some problems with it on SAMZA-335. Again, we took the
> >> > > >>>>>>> Map/Reduce
> >> > > >>>> path,
> >> > > >>>>>>> but
> >> > > >>>>>>> there are some fundamental differences between HDFS and
> Kafka.
> >> > > >>>>>>> HDFS
> >> > > >>>> has
> >> > > >>>>>>> blocks, while Kafka has partitions. This leads to less
> >> > > >>>>>>> optimization potential with stream processors on top of
> Kafka.
> >> > > >>>>>>>
> >> > > >>>>>>> This feature is also used as a crutch. Samza doesn't have
> any
> >> > > >>>>>>> built
> >> > > >>> in
> >> > > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> >> > > >>>>>>> deployment scheduling system to handle restarts when a
> >> > > >>>>>>> processor dies. This has
> >> > > >>>>> made
> >> > > >>>>>>> it very difficult to write a standalone Samza container
> >> > > >> (SAMZA-516).
> >> > > >>>>>>>
> >> > > >>>>>>> Pluggability
> >> > > >>>>>>>
> >> > > >>>>>>> In some cases pluggability is good, but I think that we've
> >> gone
> >> > > >>>>>>> too
> >> > > >>>> far
> >> > > >>>>>>> with it. Currently, Samza has:
> >> > > >>>>>>>
> >> > > >>>>>>> * Pluggable config.
> >> > > >>>>>>> * Pluggable metrics.
> >> > > >>>>>>> * Pluggable deployment systems.
> >> > > >>>>>>> * Pluggable streaming systems (SystemConsumer,
> SystemProducer,
> >> > > >> etc).
> >> > > >>>>>>> * Pluggable serdes.
> >> > > >>>>>>> * Pluggable storage engines.
> >> > > >>>>>>> * Pluggable strategies for just about every component
> >> > > >>> (MessageChooser,
> >> > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> >> > > >>>>>>>
> >> > > >>>>>>> There's probably more that I've forgotten, as well. Some of
> >> > > >>>>>>> these
> >> > > >>> are
> >> > > >>>>>>> useful, but some have proven not to be. This all comes at a
> >> cost:
> >> > > >>>>>>> complexity. This complexity is making it harder for our
> users
> >> > > >>>>>>> to
> >> > > >>> pick
> >> > > >>>> up
> >> > > >>>>>>> and use Samza out of the box. It also makes it difficult for
> >> > > >>>>>>> Samza developers to reason about what the characteristics of
> >> > > >>>>>>> the container (since the characteristics change depending on
> >> > > >>>>>>> which plugins are use).
> >> > > >>>>>>>
> >> > > >>>>>>> The issues with pluggability are most visible in the System
> >> APIs.
> >> > > >>> What
> >> > > >>>>>>> Samza really requires to be functional is Kafka as its
> >> > > >>>>>>> transport
> >> > > >>>> layer.
> >> > > >>>>>>> But
> >> > > >>>>>>> we've conflated two unrelated use cases into one API:
> >> > > >>>>>>>
> >> > > >>>>>>> 1. Get data into/out of Kafka.
> >> > > >>>>>>> 2. Process the data in Kafka.
> >> > > >>>>>>>
> >> > > >>>>>>> The current System API supports both of these use cases. The
> >> > > >>>>>>> problem
> >> > > >>>> is,
> >> > > >>>>>>> we
> >> > > >>>>>>> actually want different features for each use case. By
> >> papering
> >> > > >>>>>>> over
> >> > > >>>>> these
> >> > > >>>>>>> two use cases, and providing a single API, we've introduced
> a
> >> > > >>>>>>> ton of
> >> > > >>>>> leaky
> >> > > >>>>>>> abstractions.
> >> > > >>>>>>>
> >> > > >>>>>>> For example, what we'd really like in (2) is to have
> >> > > >>>>>>> monotonically increasing longs for offsets (like Kafka).
> This
> >> > > >>>>>>> would be at odds
> >> > > >>> with
> >> > > >>>>> (1),
> >> > > >>>>>>> though, since different systems have different
> >> > > >>>>> SCNs/Offsets/UUIDs/vectors.
> >> > > >>>>>>> There was discussion both on the mailing list and the SQL
> >> JIRAs
> >> > > >>> about
> >> > > >>>>> the
> >> > > >>>>>>> need for this.
> >> > > >>>>>>>
> >> > > >>>>>>> The same thing holds true for replayability. Kafka allows us
> >> to
> >> > > >>> rewind
> >> > > >>>>>>> when
> >> > > >>>>>>> we have a failure. Many other systems don't. In some cases,
> >> > > >>>>>>> systems
> >> > > >>>>> return
> >> > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer)
> because
> >> > > >>>>>>> they
> >> > > >>>> have
> >> > > >>>>> no
> >> > > >>>>>>> offsets.
> >> > > >>>>>>>
> >> > > >>>>>>> Partitioning is another example. Kafka supports
> partitioning,
> >> > > >>>>>>> but
> >> > > >>> many
> >> > > >>>>>>> systems don't. We model this by having a single partition
> for
> >> > > >>>>>>> those systems. Still, other systems model partitioning
> >> > > >> differently (e.g.
> >> > > >>>>>>> Kinesis).
> >> > > >>>>>>>
> >> > > >>>>>>> The SystemAdmin interface is also a mess. Creating streams
> in
> >> a
> >> > > >>>>>>> system-agnostic way is almost impossible. As is modeling
> >> > > >>>>>>> metadata
> >> > > >>> for
> >> > > >>>>> the
> >> > > >>>>>>> system (replication factor, partitions, location, etc). The
> >> > > >>>>>>> list
> >> > > >>> goes
> >> > > >>>>> on.
> >> > > >>>>>>>
> >> > > >>>>>>> Duplicate work
> >> > > >>>>>>>
> >> > > >>>>>>> At the time that we began writing Samza, Kafka's consumer
> and
> >> > > >>> producer
> >> > > >>>>>>> APIs
> >> > > >>>>>>> had a relatively weak feature set. On the consumer-side, you
> >> > > >>>>>>> had two
> >> > > >>>>>>> options: use the high level consumer, or the simple
> consumer.
> >> > > >>>>>>> The
> >> > > >>>>> problem
> >> > > >>>>>>> with the high-level consumer was that it controlled your
> >> > > >>>>>>> offsets, partition assignments, and the order in which you
> >> > > >>>>>>> received messages. The
> >> > > >>> problem
> >> > > >>>>>>> with
> >> > > >>>>>>> the simple consumer is that it's not simple. It's basic. You
> >> > > >>>>>>> end up
> >> > > >>>>> having
> >> > > >>>>>>> to handle a lot of really low-level stuff that you
> shouldn't.
> >> > > >>>>>>> We
> >> > > >>>> spent a
> >> > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust.
> >> It
> >> > > >>>>>>> also allows us to support some cool features:
> >> > > >>>>>>>
> >> > > >>>>>>> * Per-partition message ordering and prioritization.
> >> > > >>>>>>> * Tight control over partition assignment to support joins,
> >> > > >>>>>>> global
> >> > > >>>> state
> >> > > >>>>>>> (if we want to implement it :)), etc.
> >> > > >>>>>>> * Tight control over offset checkpointing.
> >> > > >>>>>>>
> >> > > >>>>>>> What we didn't realize at the time is that these features
> >> > > >>>>>>> should
> >> > > >>>>> actually
> >> > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> >> > > >>>> processors)
> >> > > >>>>>>> end up wanting to do things like joins and partition
> >> > > >>>>>>> assignment. The
> >> > > >>>>> Kafka
> >> > > >>>>>>> community has come to the same conclusion. They're adding a
> >> ton
> >> > > >>>>>>> of upgrades into their new Kafka consumer implementation.
> To a
> >> > > >>>>>>> large extent,
> >> > > >>> it's
> >> > > >>>>>>> duplicate work to what we've already done in Samza.
> >> > > >>>>>>>
> >> > > >>>>>>> On top of this, Kafka ended up taking a very similar
> approach
> >> > > >>>>>>> to
> >> > > >>>> Samza's
> >> > > >>>>>>> KafkaCheckpointManager implementation for handling offset
> >> > > >>>> checkpointing.
> >> > > >>>>>>> Like Samza, Kafka's new offset management feature stores
> >> offset
> >> > > >>>>>>> checkpoints in a topic, and allows you to fetch them from
> the
> >> > > >>>>>>> broker.
> >> > > >>>>>>>
> >> > > >>>>>>> A lot of this seems like a waste, since we could have shared
> >> > > >>>>>>> the
> >> > > >>> work
> >> > > >>>> if
> >> > > >>>>>>> it
> >> > > >>>>>>> had been done in Kafka from the get-go.
> >> > > >>>>>>>
> >> > > >>>>>>> Vision
> >> > > >>>>>>>
> >> > > >>>>>>> All of this leads me to a rather radical proposal. Samza is
> >> > > >>> relatively
> >> > > >>>>>>> stable at this point. I'd venture to say that we're near a
> 1.0
> >> > > >>>> release.
> >> > > >>>>>>> I'd
> >> > > >>>>>>> like to propose that we take what we've learned, and begin
> >> > > >>>>>>> thinking
> >> > > >>>>> about
> >> > > >>>>>>> Samza beyond 1.0. What would we change if we were starting
> >> from
> >> > > >>>> scratch?
> >> > > >>>>>>> My
> >> > > >>>>>>> proposal is to:
> >> > > >>>>>>>
> >> > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> >> > > >>>>>>> processors, and eliminate all direct dependences on YARN,
> >> Mesos,
> >> > > >> etc.
> >> > > >>>>>>> 2. Make a definitive call to support only Kafka as the
> stream
> >> > > >>>> processing
> >> > > >>>>>>> layer.
> >> > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> >> > > >>>>>>> config
> >> > > >>>>> systems,
> >> > > >>>>>>> and simply use Kafka's instead.
> >> > > >>>>>>>
> >> > > >>>>>>> This would fix all of the issues that I outlined above. It
> >> > > >>>>>>> should
> >> > > >>> also
> >> > > >>>>>>> shrink the Samza code base pretty dramatically. Supporting
> >> only
> >> > > >>>>>>> a standalone container will allow Samza to be executed on
> YARN
> >> > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> >> > > >>>>>>> in-house
> >> > > >>>>> deployment
> >> > > >>>>>>> systems. This should make life a lot easier for new users.
> >> > > >>>>>>> Imagine
> >> > > >>>>> having
> >> > > >>>>>>> the hello-samza tutorial without YARN. The drop in mailing
> >> list
> >> > > >>>> traffic
> >> > > >>>>>>> will be pretty dramatic.
> >> > > >>>>>>>
> >> > > >>>>>>> Coupling with Kafka seems long overdue to me. The reality
> is,
> >> > > >>> everyone
> >> > > >>>>>>> that
> >> > > >>>>>>> I'm aware of is using Samza with Kafka. We basically require
> >> it
> >> > > >>>> already
> >> > > >>>>> in
> >> > > >>>>>>> order for most features to work. Those that are using other
> >> > > >>>>>>> systems
> >> > > >>>> are
> >> > > >>>>>>> generally using it for ingest into Kafka (1), and then they
> do
> >> > > >>>>>>> the processing on top. There is already discussion (
> >> > > >>>
> >> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >> > > >>> 767
> >> > > >>>>>>> )
> >> > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> >> > > >>>>>>>
> >> > > >>>>>>> Once we make the call to couple with Kafka, we can leverage
> a
> >> > > >>>>>>> ton of
> >> > > >>>>> their
> >> > > >>>>>>> ecosystem. We no longer have to maintain our own config,
> >> > > >>>>>>> metrics,
> >> > > >>> etc.
> >> > > >>>>> We
> >> > > >>>>>>> can all share the same libraries, and make them better. This
> >> > > >>>>>>> will
> >> > > >>> also
> >> > > >>>>>>> allow us to share the consumer/producer APIs, and will let
> us
> >> > > >>> leverage
> >> > > >>>>>>> their offset management and partition management, rather
> than
> >> > > >>>>>>> having
> >> > > >>>> our
> >> > > >>>>>>> own. All of the coordinator stream code would go away, as
> >> would
> >> > > >>>>>>> most
> >> > > >>>> of
> >> > > >>>>>>> the
> >> > > >>>>>>> YARN AppMaster code. We'd probably have to push some
> partition
> >> > > >>>>> management
> >> > > >>>>>>> features into the Kafka broker, but they're already moving
> in
> >> > > >>>>>>> that direction with the new consumer API. The features we
> have
> >> > > >>>>>>> for
> >> > > >>>> partition
> >> > > >>>>>>> assignment aren't unique to Samza, and seem like they should
> >> be
> >> > > >>>>>>> in
> >> > > >>>> Kafka
> >> > > >>>>>>> anyway. There will always be some niche usages which will
> >> > > >>>>>>> require
> >> > > >>>> extra
> >> > > >>>>>>> care and hence full control over partition assignments much
> >> > > >>>>>>> like the
> >> > > >>>>> Kafka
> >> > > >>>>>>> low level consumer api. These would continue to be
> supported.
> >> > > >>>>>>>
> >> > > >>>>>>> These items will be good for the Samza community. They'll
> make
> >> > > >>>>>>> Samza easier to use, and make it easier for developers to
> add
> >> > > >>>>>>> new features.
> >> > > >>>>>>>
> >> > > >>>>>>> Obviously this is a fairly large (and somewhat backwards
> >> > > >>> incompatible
> >> > > >>>>>>> change). If we choose to go this route, it's important that
> we
> >> > > >>> openly
> >> > > >>>>>>> communicate how we're going to provide a migration path from
> >> > > >>>>>>> the
> >> > > >>>>> existing
> >> > > >>>>>>> APIs to the new ones (if we make incompatible changes). I
> >> think
> >> > > >>>>>>> at a minimum, we'd probably need to provide a wrapper to
> allow
> >> > > >>>>>>> existing StreamTask implementations to continue running on
> the
> >> > > >> new container.
> >> > > >>>>> It's
> >> > > >>>>>>> also important that we openly communicate about timing, and
> >> > > >>>>>>> stages
> >> > > >>> of
> >> > > >>>>> the
> >> > > >>>>>>> migration.
> >> > > >>>>>>>
> >> > > >>>>>>> If you made it this far, I'm sure you have opinions. :)
> Please
> >> > > >>>>>>> send
> >> > > >>>> your
> >> > > >>>>>>> thoughts and feedback.
> >> > > >>>>>>>
> >> > > >>>>>>> Cheers,
> >> > > >>>>>>> Chris
> >> > > >>>>
> >> > > >>>>
> >> > > >>>>
> >> > > >>>> --
> >> > > >>>> -- Guozhang
> >> > > >>
> >> > >
> >> >
> >>
> >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
One more use case we had encountered that needs an explicit dynamic
PartitionManager/JobCoordinator outside Kafka broker is: there are use
cases that a Samza job needs to consume all Kafka topics matching a certain
regex, and users want the newly added topics to be assigned in runtime.
There is a need to have a dynamic discovery module for new topics and
assign the new topic partitions to the Samza workers. IMO, this should be
the functionality in a PartitionManager outside the Kafka broker, since it
is part of the application logic.

Said all that, my main point is simple: I am proposing that we need a
pluggable partition management component, decoupled from the framework to
do resource assignment, process restart, etc.

On Thu, Jul 2, 2015 at 2:35 PM, Yi Pan <ni...@gmail.com> wrote:

> @Jay, yes, the current function in the JobCoordinator is just partition
> management. Maybe we should just call it PartitionManager to make it
> explicit.
>
> -Yi
>
> On Thu, Jul 2, 2015 at 2:24 PM, Jay Kreps <ja...@confluent.io> wrote:
>
>> Hey Yi,
>>
>> What does the JobCoordinator do? YARN/Mesos/etc would be doing the actual
>> resource assignment, process restart, etc, right? Is the additional value
>> add of the JobCoordinator just partition management?
>>
>> -Jay
>>
>> On Thu, Jul 2, 2015 at 11:32 AM, Yi Pan <ni...@gmail.com> wrote:
>>
>> > Hi, all,
>> >
>> >
>> > Thanks Chris for sending out this proposal and Jay for sharing the
>> > extremely illustrative prototype code.
>> >
>> >
>> > I have been thinking it over many times and want to list out my personal
>> > opinions below:
>> >
>> > 1. Generally, I agree with most of the people here on the mailing list
>> on
>> > two points:
>> >
>> >    a. Deeper integration w/ Kafka is great. No more confusing mapping
>> from
>> > SystemStreamPartition to TopicPartition etc.
>> >
>> >    b. Separation the ingestion vs transformation greatly simplify the
>> > systems APIs
>> >
>> > Having the above two changes would allow us to remove many unnecessary
>> > complexities introduced by those pluggable interfaces Chris’ pointed
>> out,
>> > e.g. pluggable streaming systems and serde.
>> >
>> >
>> > To recall one of Chris’s statement on difficulties in dynamic
>> deployment, I
>> > believe that the difficulties are mainly the result of tight-coupling of
>> > partition assignment vs the container deployment in the current system.
>> The
>> > current container deployment requires a pre-defined partition assignment
>> > strategy coupled together w/ the deployment configuration before we can
>> > submit to YARN and start the Samza container, which makes the launching
>> > process super long. Also, fault-tolerance and the embedded
>> JobCoordinator
>> > code in YARN AppMaster is another way of  making dynamic deployment more
>> > complex and difficult.
>> >
>> >
>> > First, borrowing Yan’s term, let’s call the Samza standalone process a
>> > Samza worker. Here is what I have been thinking:
>> >
>> > 1. Separate the execution framework from partition assignment/load
>> > balancing:
>> >
>> >     a. a Samza worker should be launched by execution framework that
>> only
>> > deals w/ process placement to available nodes. The execution framework
>> now
>> > should only deal w/ how many such processes are needed, where to put
>> them,
>> > and how to keep them alive.
>> >
>> >     b. Partition assignment/load balancing can be a pluggable interface
>> in
>> > Samza that allows the Samza workers to ask for partition assignments.
>> Let’s
>> > borrow the name JobCoordinator for now. To allow fault-tolerance in
>> case of
>> > failure, the partition assignments to workers need to be dynamic. Hence,
>> > the abstract interface would be much like what Jay’s code illustrate:
>> > get()/onAssigned()/onRevoke(). The implementation of the partition
>> > assignment can be either:
>> >
>> >         a) completely rely on Kafka.
>> >
>> >         b) explicit partition assignment via JobCoordinator. Chris’s
>> work
>> > in SAMZA-516 can be easily incorporated here. The use case in SAMZA-41
>> that
>> > runs Samza ProcessJob w/ static partition assignment can be implemented
>> of
>> > JobCoordinator via any home-grown implementation of distributed
>> > coordinator. All the work we did in LinkedIn to support dynamic
>> partition
>> > assignment and host-affinity SAMZA-617 can be nicely reused as an
>> > implementation of JobCoordinator.
>> >
>> >
>> > When we did the above work, I can see three usage patterns in Samza:
>> >
>> >    a. Samza as a library: Samza has a set of libraries to provide stream
>> > processing, just like a third Kafka client (as illustrated in Jay’s
>> > example). The execution/deployment is totally controlled by the
>> application
>> > and the partition coordination is done via Kafka
>> >
>> >    b. Samza as a process: Samza runs as a standalone process. There may
>> not
>> > be a execution framework to launch and deploy Samza processes. The
>> > partition assignment is pluggable JobCoordinator.
>> >
>> >    c. Samza as a service: Samza runs as a collection of processes. There
>> > will be an execution framework to allocate resource, launch and deploy
>> > Samza workers and keep them alive. The same pluggable JobCoordinator is
>> > desirable here as well.
>> >
>> >
>> > Lastly, I would argue that CopyCat in KIP-26 should probably follow the
>> > same model. Hence, in Samza as a service model as in LinkedIn, we can
>> use
>> > the same fault tolerance execution framework to run CopyCat and Samza
>> w/o
>> > the need to operate two service platforms, which should address Sriram’s
>> > comment in the email thread.
>> >
>> >
>> > Hope the above makes sense. Thanks all!
>> >
>> >
>> > -Yi
>> >
>> > On Thu, Jul 2, 2015 at 9:53 AM, Sriram <sr...@gmail.com> wrote:
>> >
>> > > One thing that is worth exploring is to have a transformation and
>> > > ingestion library in Kafka but use the same framework for fault
>> > tolerance,
>> > > resource isolation and management. The biggest difference I see in
>> these
>> > > two use cases is the API and data model.
>> > >
>> > >
>> > > > On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io> wrote:
>> > > >
>> > > > Hey Garry,
>> > > >
>> > > > Yeah that's super frustrating. I'd be happy to chat more about this
>> if
>> > > > you'd be interested. I think Chris and I started with the idea of
>> "what
>> > > > would it take to make Samza a kick-ass ingestion tool" but
>> ultimately
>> > we
>> > > > kind of came around to the idea that ingestion and transformation
>> had
>> > > > pretty different needs and coupling the two made things hard.
>> > > >
>> > > > For what it's worth I think copycat (KIP-26) actually will do what
>> you
>> > > are
>> > > > looking for.
>> > > >
>> > > > With regard to your point about slider, I don't necessarily
>> disagree.
>> > > But I
>> > > > think getting good YARN support is quite doable and I think we can
>> make
>> > > > that work well. I think the issue this proposal solves is that
>> > > technically
>> > > > it is pretty hard to support multiple cluster management systems the
>> > way
>> > > > things are now, you need to write an "app master" or "framework" for
>> > each
>> > > > and they are all a little different so testing is really hard. In
>> the
>> > > > absence of this we have been stuck with just YARN which has
>> fantastic
>> > > > penetration in the Hadoopy part of the org, but zero penetration
>> > > elsewhere.
>> > > > Given the huge amount of work being put in to slider, marathon, aws
>> > > > tooling, not to mention the umpteen related packaging technologies
>> > people
>> > > > want to use (Docker, Kubernetes, various cloud-specific deploy
>> tools,
>> > > etc)
>> > > > I really think it is important to get this right.
>> > > >
>> > > > -Jay
>> > > >
>> > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
>> > > > g.turkington@improvedigital.com> wrote:
>> > > >
>> > > >> Hi all,
>> > > >>
>> > > >> I think the question below re does Samza become a sub-project of
>> Kafka
>> > > >> highlights the broader point around migration. Chris mentions
>> Samza's
>> > > >> maturity is heading towards a v1 release but I'm not sure it feels
>> > > right to
>> > > >> launch a v1 then immediately plan to deprecate most of it.
>> > > >>
>> > > >> From a selfish perspective I have some guys who have started
>> working
>> > > with
>> > > >> Samza and building some new consumers/producers was next up. Sounds
>> > like
>> > > >> that is absolutely not the direction to go. I need to look into the
>> > KIP
>> > > in
>> > > >> more detail but for me the attractiveness of adding new Samza
>> > > >> consumer/producers -- even if yes all they were doing was really
>> > getting
>> > > >> data into and out of Kafka --  was to avoid  having to worry about
>> the
>> > > >> lifecycle management of external clients. If there is a generic
>> Kafka
>> > > >> ingress/egress layer that I can plug a new connector into and have
>> a
>> > > lot of
>> > > >> the heavy lifting re scale and reliability done for me then it
>> gives
>> > me
>> > > all
>> > > >> the pushing new consumers/producers would. If not then it
>> complicates
>> > my
>> > > >> operational deployments.
>> > > >>
>> > > >> Which is similar to my other question with the proposal -- if we
>> > build a
>> > > >> fully available/stand-alone Samza plus the requisite shims to
>> > integrate
>> > > >> with Slider etc I suspect the former may be a lot more work than we
>> > > think.
>> > > >> We may make it much easier for a newcomer to get something running
>> but
>> > > >> having them step up and get a reliable production deployment may
>> still
>> > > >> dominate mailing list  traffic, if for different reasons than
>> today.
>> > > >>
>> > > >> Don't get me wrong -- I'm comfortable with making the Samza
>> dependency
>> > > on
>> > > >> Kafka much more explicit and I absolutely see the benefits  in the
>> > > >> reduction of duplication and clashing terminologies/abstractions
>> that
>> > > >> Chris/Jay describe. Samza as a library would likely be a very nice
>> > tool
>> > > to
>> > > >> add to the Kafka ecosystem. I just have the concerns above re the
>> > > >> operational side.
>> > > >>
>> > > >> Garry
>> > > >>
>> > > >> -----Original Message-----
>> > > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
>> > > >> Sent: 02 July 2015 12:56
>> > > >> To: dev@samza.apache.org
>> > > >> Subject: Re: Thoughts and obesrvations on Samza
>> > > >>
>> > > >> Very interesting thoughts.
>> > > >> From outside, I have always perceived Samza as a computing layer
>> over
>> > > >> Kafka.
>> > > >>
>> > > >> The question, maybe a bit provocative, is "should Samza be a
>> > sub-project
>> > > >> of Kafka then?"
>> > > >> Or does it make sense to keep it as a separate project with a
>> separate
>> > > >> governance?
>> > > >>
>> > > >> Cheers,
>> > > >>
>> > > >> --
>> > > >> Gianmarco
>> > > >>
>> > > >>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
>> > > >>>
>> > > >>> Overall, I agree to couple with Kafka more tightly. Because Samza
>> de
>> > > >>> facto is based on Kafka, and it should leverage what Kafka has. At
>> > the
>> > > >>> same time, Kafka does not need to reinvent what Samza already
>> has. I
>> > > >>> also like the idea of separating the ingestion and transformation.
>> > > >>>
>> > > >>> But it is a little difficult for me to image how the Samza will
>> look
>> > > >> like.
>> > > >>> And I feel Chris and Jay have a little difference in terms of how
>> > > >>> Samza should look like.
>> > > >>>
>> > > >>> *** Will it look like what Jay's code shows (A client of Kakfa) ?
>> And
>> > > >>> user's application code calls this client?
>> > > >>>
>> > > >>> 1. If we make Samza be a library of Kafka (like what the code
>> shows),
>> > > >>> how do we implement auto-balance and fault-tolerance? Are they
>> taken
>> > > >>> care by the Kafka broker or other mechanism, such as "Samza
>> worker"
>> > > >>> (just make up the name) ?
>> > > >>>
>> > > >>> 2. What about other features, such as auto-scaling, shared state,
>> > > >>> monitoring?
>> > > >>>
>> > > >>>
>> > > >>> *** If we have Samza standalone, (is this what Chris suggests?)
>> > > >>>
>> > > >>> 1. we still need to ingest data from Kakfa and produce to it.
>> Then it
>> > > >>> becomes the same as what Samza looks like now, except it does not
>> > rely
>> > > >>> on Yarn anymore.
>> > > >>>
>> > > >>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
>> > > >>> etc? Use Kafka code as the dependency?
>> > > >>>
>> > > >>>
>> > > >>> Thanks,
>> > > >>>
>> > > >>> Fang, Yan
>> > > >>> yanfang724@gmail.com
>> > > >>>
>> > > >>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <
>> wangguoz@gmail.com>
>> > > >>> wrote:
>> > > >>>
>> > > >>>> Read through the code example and it looks good to me. A few
>> > > >>>> thoughts regarding deployment:
>> > > >>>>
>> > > >>>> Today Samza deploys as executable runnable like:
>> > > >>>>
>> > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
>> > > >> --config-path=file://...
>> > > >>>>
>> > > >>>> And this proposal advocate for deploying Samza more as embedded
>> > > >>>> libraries in user application code (ignoring the terminology
>> since
>> > > >>>> it is not the
>> > > >>> same
>> > > >>>> as the prototype code):
>> > > >>>>
>> > > >>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
>> > > >>>> Thread(task); thread.start();
>> > > >>>>
>> > > >>>> I think both of these deployment modes are important for
>> different
>> > > >>>> types
>> > > >>> of
>> > > >>>> users. That said, I think making Samza purely standalone is still
>> > > >>>> sufficient for either runnable or library modes.
>> > > >>>>
>> > > >>>> Guozhang
>> > > >>>>
>> > > >>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
>> > > wrote:
>> > > >>>>>
>> > > >>>>> Looks like gmail mangled the code example, it was supposed to
>> look
>> > > >>>>> like
>> > > >>>>> this:
>> > > >>>>>
>> > > >>>>> Properties props = new Properties();
>> > > >>>>> props.put("bootstrap.servers", "localhost:4242");
>> StreamingConfig
>> > > >>>>> config = new StreamingConfig(props);
>> > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
>> > > >>>>> config.processor(ExampleStreamProcessor.class);
>> > > >>>>> config.serialization(new StringSerializer(), new
>> > > >>>>> StringDeserializer()); KafkaStreaming container = new
>> > > >>>>> KafkaStreaming(config); container.run();
>> > > >>>>>
>> > > >>>>> -Jay
>> > > >>>>>
>> > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
>> > > >> wrote:
>> > > >>>>>
>> > > >>>>>> Hey guys,
>> > > >>>>>>
>> > > >>>>>> This came out of some conversations Chris and I were having
>> > > >>>>>> around
>> > > >>>>> whether
>> > > >>>>>> it would make sense to use Samza as a kind of data ingestion
>> > > >>> framework
>> > > >>>>> for
>> > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
>> > > >>>> combined
>> > > >>>>>> with complaints around config and YARN and the discussion
>> around
>> > > >>>>>> how
>> > > >>> to
>> > > >>>>>> best do a standalone mode.
>> > > >>>>>>
>> > > >>>>>> So the thought experiment was, given that Samza was basically
>> > > >>>>>> already totally Kafka specific, what if you just embraced that
>> > > >>>>>> and turned it
>> > > >>>> into
>> > > >>>>>> something less like a heavyweight framework and more like a
>> > > >>>>>> third
>> > > >>> Kafka
>> > > >>>>>> client--a kind of "producing consumer" with state management
>> > > >>>> facilities.
>> > > >>>>>> Basically a library. Instead of a complex stream processing
>> > > >>>>>> framework
>> > > >>>>> this
>> > > >>>>>> would actually be a very simple thing, not much more
>> complicated
>> > > >>>>>> to
>> > > >>> use
>> > > >>>>> or
>> > > >>>>>> operate than a Kafka consumer. As Chris said we thought about
>> it
>> > > >>>>>> a
>> > > >>> lot
>> > > >>>> of
>> > > >>>>>> what Samza (and the other stream processing systems were doing)
>> > > >>> seemed
>> > > >>>>> like
>> > > >>>>>> kind of a hangover from MapReduce.
>> > > >>>>>>
>> > > >>>>>> Of course you need to ingest/output data to and from the stream
>> > > >>>>>> processing. But when we actually looked into how that would
>> > > >>>>>> work,
>> > > >>> Samza
>> > > >>>>>> isn't really an ideal data ingestion framework for a bunch of
>> > > >>> reasons.
>> > > >>>> To
>> > > >>>>>> really do that right you need a pretty different internal data
>> > > >>>>>> model
>> > > >>>> and
>> > > >>>>>> set of apis. So what if you split them and had an api for Kafka
>> > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for
>> Kafka
>> > > >>>>>> transformation (Samza).
>> > > >>>>>>
>> > > >>>>>> This would also allow really embracing the same terminology and
>> > > >>>>>> conventions. One complaint about the current state is that the
>> > > >>>>>> two
>> > > >>>>> systems
>> > > >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic"
>> and
>> > > >>>>> different
>> > > >>>>>> config and monitoring systems means you kind of have to learn
>> > > >>>>>> Kafka's
>> > > >>>>> way,
>> > > >>>>>> then learn Samza's slightly different way, then kind of
>> > > >>>>>> understand
>> > > >>> how
>> > > >>>>> they
>> > > >>>>>> map to each other, which having walked a few people through
>> this
>> > > >>>>>> is surprisingly tricky for folks to get.
>> > > >>>>>>
>> > > >>>>>> Since I have been spending a lot of time on airplanes I hacked
>> > > >>>>>> up an ernest but still somewhat incomplete prototype of what
>> > > >>>>>> this would
>> > > >>> look
>> > > >>>>>> like. This is just unceremoniously dumped into Kafka as it
>> > > >>>>>> required a
>> > > >>>> few
>> > > >>>>>> changes to the new consumer. Here is the code:
>> > > >>>
>> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>> > > >>> /apache/kafka/clients/streaming
>> > > >>>>>>
>> > > >>>>>> For the purpose of the prototype I just liberally renamed
>> > > >>>>>> everything
>> > > >>> to
>> > > >>>>>> try to align it with Kafka with no regard for compatibility.
>> > > >>>>>>
>> > > >>>>>> To use this would be something like this:
>> > > >>>>>> Properties props = new Properties();
>> > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
>> > > >>>>>> StreamingConfig config = new
>> > > >>> StreamingConfig(props);
>> > > >>>>> config.subscribe("test-topic-1",
>> > > >>>>>> "test-topic-2");
>> config.processor(ExampleStreamProcessor.class);
>> > > >>>>> config.serialization(new
>> > > >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
>> > > >>>> container =
>> > > >>>>>> new KafkaStreaming(config); container.run();
>> > > >>>>>>
>> > > >>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
>> > > >>>>>> is basically StreamTask.
>> > > >>>>>>
>> > > >>>>>> So rather than putting all the class names in a file and then
>> > > >>>>>> having
>> > > >>>> the
>> > > >>>>>> job assembled by reflection, you just instantiate the container
>> > > >>>>>> programmatically. Work is balanced over however many instances
>> > > >>>>>> of
>> > > >>> this
>> > > >>>>> are
>> > > >>>>>> alive at any time (i.e. if an instance dies, new tasks are
>> added
>> > > >>>>>> to
>> > > >>> the
>> > > >>>>>> existing containers without shutting them down).
>> > > >>>>>>
>> > > >>>>>> We would provide some glue for running this stuff in YARN via
>> > > >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
>> > > >>>>>> but from the
>> > > >>>> point
>> > > >>>>> of
>> > > >>>>>> view of these frameworks these stream processing jobs are just
>> > > >>>> stateless
>> > > >>>>>> services that can come and go and expand and contract at will.
>> > > >>>>>> There
>> > > >>> is
>> > > >>>>> no
>> > > >>>>>> more custom scheduler.
>> > > >>>>>>
>> > > >>>>>> Here are some relevant details:
>> > > >>>>>>
>> > > >>>>>>   1. It is only ~1300 lines of code, it would get larger if we
>> > > >>>>>>   productionized but not vastly larger. We really do get a ton
>> > > >>>>>> of
>> > > >>>>> leverage
>> > > >>>>>>   out of Kafka.
>> > > >>>>>>   2. Partition management is fully delegated to the new
>> consumer.
>> > > >>> This
>> > > >>>>>>   is nice since now any partition management strategy available
>> > > >>>>>> to
>> > > >>>> Kafka
>> > > >>>>>>   consumer is also available to Samza (and vice versa) and with
>> > > >>>>>> the
>> > > >>>>> exact
>> > > >>>>>>   same configs.
>> > > >>>>>>   3. It supports state as well as state reuse
>> > > >>>>>>
>> > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
>> > > >>>>>>
>> > > >>>>>> -Jay
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>> > > >>>> criccomini@apache.org>
>> > > >>>>>> wrote:
>> > > >>>>>>
>> > > >>>>>>> Hey all,
>> > > >>>>>>>
>> > > >>>>>>> I have had some discussions with Samza engineers at LinkedIn
>> > > >>>>>>> and
>> > > >>>>> Confluent
>> > > >>>>>>> and we came up with a few observations and would like to
>> > > >>>>>>> propose
>> > > >>> some
>> > > >>>>>>> changes.
>> > > >>>>>>>
>> > > >>>>>>> We've observed some things that I want to call out about
>> > > >>>>>>> Samza's
>> > > >>>> design,
>> > > >>>>>>> and I'd like to propose some changes.
>> > > >>>>>>>
>> > > >>>>>>> * Samza is dependent upon a dynamic deployment system.
>> > > >>>>>>> * Samza is too pluggable.
>> > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
>> > > >>>>>>> APIs
>> > > >>> are
>> > > >>>>>>> trying to solve a lot of the same problems.
>> > > >>>>>>>
>> > > >>>>>>> All three of these issues are related, but I'll address them
>> in
>> > > >>> order.
>> > > >>>>>>>
>> > > >>>>>>> Deployment
>> > > >>>>>>>
>> > > >>>>>>> Samza strongly depends on the use of a dynamic deployment
>> > > >>>>>>> scheduler
>> > > >>>> such
>> > > >>>>>>> as
>> > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
>> > > >>>>>>> there
>> > > >>>> would
>> > > >>>>>>> be
>> > > >>>>>>> one or two winners in this area, and we could support them,
>> and
>> > > >>>>>>> the
>> > > >>>> rest
>> > > >>>>>>> would go away. In reality, there are many variations.
>> > > >>>>>>> Furthermore,
>> > > >>>> many
>> > > >>>>>>> people still prefer to just start their processors like normal
>> > > >>>>>>> Java processes, and use traditional deployment scripts such as
>> > > >>>>>>> Fabric,
>> > > >>>> Chef,
>> > > >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
>> > > >>>>>>> Samza start-up process really painful for first time users.
>> > > >>>>>>>
>> > > >>>>>>> Dynamic deployment as a requirement was also a bit of a
>> > > >>>>>>> mis-fire
>> > > >>>> because
>> > > >>>>>>> of
>> > > >>>>>>> a fundamental misunderstanding between the nature of batch
>> jobs
>> > > >>>>>>> and
>> > > >>>>> stream
>> > > >>>>>>> processing jobs. Early on, we made conscious effort to favor
>> > > >>>>>>> the
>> > > >>>> Hadoop
>> > > >>>>>>> (Map/Reduce) way of doing things, since it worked and was well
>> > > >>>>> understood.
>> > > >>>>>>> One thing that we missed was that batch jobs have a definite
>> > > >>>> beginning,
>> > > >>>>>>> and
>> > > >>>>>>> end, and stream processing jobs don't (usually). This leads to
>> > > >>>>>>> a
>> > > >>> much
>> > > >>>>>>> simpler scheduling problem for stream processors. You
>> basically
>> > > >>>>>>> just
>> > > >>>>> need
>> > > >>>>>>> to find a place to start the processor, and start it. The way
>> > > >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
>> > > >>>>>>> being "full". We always
>> > > >>>> add
>> > > >>>>>>> more machines. The problem with coupling Samza with a
>> scheduler
>> > > >>>>>>> is
>> > > >>>> that
>> > > >>>>>>> Samza (as a framework) now has to handle deployment. This
>> pulls
>> > > >>>>>>> in a
>> > > >>>>> bunch
>> > > >>>>>>> of things such as configuration distribution (config stream),
>> > > >>>>>>> shell
>> > > >>>>> scrips
>> > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff),
>> etc.
>> > > >>>>>>>
>> > > >>>>>>> Another reason for requiring dynamic deployment was to support
>> > > >>>>>>> data locality. If you want to have locality, you need to put
>> > > >>>>>>> your
>> > > >>>> processors
>> > > >>>>>>> close to the data they're processing. Upon further
>> > > >>>>>>> investigation,
>> > > >>>>> though,
>> > > >>>>>>> this feature is not that beneficial. There is some good
>> > > >>>>>>> discussion
>> > > >>>> about
>> > > >>>>>>> some problems with it on SAMZA-335. Again, we took the
>> > > >>>>>>> Map/Reduce
>> > > >>>> path,
>> > > >>>>>>> but
>> > > >>>>>>> there are some fundamental differences between HDFS and Kafka.
>> > > >>>>>>> HDFS
>> > > >>>> has
>> > > >>>>>>> blocks, while Kafka has partitions. This leads to less
>> > > >>>>>>> optimization potential with stream processors on top of Kafka.
>> > > >>>>>>>
>> > > >>>>>>> This feature is also used as a crutch. Samza doesn't have any
>> > > >>>>>>> built
>> > > >>> in
>> > > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
>> > > >>>>>>> deployment scheduling system to handle restarts when a
>> > > >>>>>>> processor dies. This has
>> > > >>>>> made
>> > > >>>>>>> it very difficult to write a standalone Samza container
>> > > >> (SAMZA-516).
>> > > >>>>>>>
>> > > >>>>>>> Pluggability
>> > > >>>>>>>
>> > > >>>>>>> In some cases pluggability is good, but I think that we've
>> gone
>> > > >>>>>>> too
>> > > >>>> far
>> > > >>>>>>> with it. Currently, Samza has:
>> > > >>>>>>>
>> > > >>>>>>> * Pluggable config.
>> > > >>>>>>> * Pluggable metrics.
>> > > >>>>>>> * Pluggable deployment systems.
>> > > >>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
>> > > >> etc).
>> > > >>>>>>> * Pluggable serdes.
>> > > >>>>>>> * Pluggable storage engines.
>> > > >>>>>>> * Pluggable strategies for just about every component
>> > > >>> (MessageChooser,
>> > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>> > > >>>>>>>
>> > > >>>>>>> There's probably more that I've forgotten, as well. Some of
>> > > >>>>>>> these
>> > > >>> are
>> > > >>>>>>> useful, but some have proven not to be. This all comes at a
>> cost:
>> > > >>>>>>> complexity. This complexity is making it harder for our users
>> > > >>>>>>> to
>> > > >>> pick
>> > > >>>> up
>> > > >>>>>>> and use Samza out of the box. It also makes it difficult for
>> > > >>>>>>> Samza developers to reason about what the characteristics of
>> > > >>>>>>> the container (since the characteristics change depending on
>> > > >>>>>>> which plugins are use).
>> > > >>>>>>>
>> > > >>>>>>> The issues with pluggability are most visible in the System
>> APIs.
>> > > >>> What
>> > > >>>>>>> Samza really requires to be functional is Kafka as its
>> > > >>>>>>> transport
>> > > >>>> layer.
>> > > >>>>>>> But
>> > > >>>>>>> we've conflated two unrelated use cases into one API:
>> > > >>>>>>>
>> > > >>>>>>> 1. Get data into/out of Kafka.
>> > > >>>>>>> 2. Process the data in Kafka.
>> > > >>>>>>>
>> > > >>>>>>> The current System API supports both of these use cases. The
>> > > >>>>>>> problem
>> > > >>>> is,
>> > > >>>>>>> we
>> > > >>>>>>> actually want different features for each use case. By
>> papering
>> > > >>>>>>> over
>> > > >>>>> these
>> > > >>>>>>> two use cases, and providing a single API, we've introduced a
>> > > >>>>>>> ton of
>> > > >>>>> leaky
>> > > >>>>>>> abstractions.
>> > > >>>>>>>
>> > > >>>>>>> For example, what we'd really like in (2) is to have
>> > > >>>>>>> monotonically increasing longs for offsets (like Kafka). This
>> > > >>>>>>> would be at odds
>> > > >>> with
>> > > >>>>> (1),
>> > > >>>>>>> though, since different systems have different
>> > > >>>>> SCNs/Offsets/UUIDs/vectors.
>> > > >>>>>>> There was discussion both on the mailing list and the SQL
>> JIRAs
>> > > >>> about
>> > > >>>>> the
>> > > >>>>>>> need for this.
>> > > >>>>>>>
>> > > >>>>>>> The same thing holds true for replayability. Kafka allows us
>> to
>> > > >>> rewind
>> > > >>>>>>> when
>> > > >>>>>>> we have a failure. Many other systems don't. In some cases,
>> > > >>>>>>> systems
>> > > >>>>> return
>> > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
>> > > >>>>>>> they
>> > > >>>> have
>> > > >>>>> no
>> > > >>>>>>> offsets.
>> > > >>>>>>>
>> > > >>>>>>> Partitioning is another example. Kafka supports partitioning,
>> > > >>>>>>> but
>> > > >>> many
>> > > >>>>>>> systems don't. We model this by having a single partition for
>> > > >>>>>>> those systems. Still, other systems model partitioning
>> > > >> differently (e.g.
>> > > >>>>>>> Kinesis).
>> > > >>>>>>>
>> > > >>>>>>> The SystemAdmin interface is also a mess. Creating streams in
>> a
>> > > >>>>>>> system-agnostic way is almost impossible. As is modeling
>> > > >>>>>>> metadata
>> > > >>> for
>> > > >>>>> the
>> > > >>>>>>> system (replication factor, partitions, location, etc). The
>> > > >>>>>>> list
>> > > >>> goes
>> > > >>>>> on.
>> > > >>>>>>>
>> > > >>>>>>> Duplicate work
>> > > >>>>>>>
>> > > >>>>>>> At the time that we began writing Samza, Kafka's consumer and
>> > > >>> producer
>> > > >>>>>>> APIs
>> > > >>>>>>> had a relatively weak feature set. On the consumer-side, you
>> > > >>>>>>> had two
>> > > >>>>>>> options: use the high level consumer, or the simple consumer.
>> > > >>>>>>> The
>> > > >>>>> problem
>> > > >>>>>>> with the high-level consumer was that it controlled your
>> > > >>>>>>> offsets, partition assignments, and the order in which you
>> > > >>>>>>> received messages. The
>> > > >>> problem
>> > > >>>>>>> with
>> > > >>>>>>> the simple consumer is that it's not simple. It's basic. You
>> > > >>>>>>> end up
>> > > >>>>> having
>> > > >>>>>>> to handle a lot of really low-level stuff that you shouldn't.
>> > > >>>>>>> We
>> > > >>>> spent a
>> > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust.
>> It
>> > > >>>>>>> also allows us to support some cool features:
>> > > >>>>>>>
>> > > >>>>>>> * Per-partition message ordering and prioritization.
>> > > >>>>>>> * Tight control over partition assignment to support joins,
>> > > >>>>>>> global
>> > > >>>> state
>> > > >>>>>>> (if we want to implement it :)), etc.
>> > > >>>>>>> * Tight control over offset checkpointing.
>> > > >>>>>>>
>> > > >>>>>>> What we didn't realize at the time is that these features
>> > > >>>>>>> should
>> > > >>>>> actually
>> > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
>> > > >>>> processors)
>> > > >>>>>>> end up wanting to do things like joins and partition
>> > > >>>>>>> assignment. The
>> > > >>>>> Kafka
>> > > >>>>>>> community has come to the same conclusion. They're adding a
>> ton
>> > > >>>>>>> of upgrades into their new Kafka consumer implementation. To a
>> > > >>>>>>> large extent,
>> > > >>> it's
>> > > >>>>>>> duplicate work to what we've already done in Samza.
>> > > >>>>>>>
>> > > >>>>>>> On top of this, Kafka ended up taking a very similar approach
>> > > >>>>>>> to
>> > > >>>> Samza's
>> > > >>>>>>> KafkaCheckpointManager implementation for handling offset
>> > > >>>> checkpointing.
>> > > >>>>>>> Like Samza, Kafka's new offset management feature stores
>> offset
>> > > >>>>>>> checkpoints in a topic, and allows you to fetch them from the
>> > > >>>>>>> broker.
>> > > >>>>>>>
>> > > >>>>>>> A lot of this seems like a waste, since we could have shared
>> > > >>>>>>> the
>> > > >>> work
>> > > >>>> if
>> > > >>>>>>> it
>> > > >>>>>>> had been done in Kafka from the get-go.
>> > > >>>>>>>
>> > > >>>>>>> Vision
>> > > >>>>>>>
>> > > >>>>>>> All of this leads me to a rather radical proposal. Samza is
>> > > >>> relatively
>> > > >>>>>>> stable at this point. I'd venture to say that we're near a 1.0
>> > > >>>> release.
>> > > >>>>>>> I'd
>> > > >>>>>>> like to propose that we take what we've learned, and begin
>> > > >>>>>>> thinking
>> > > >>>>> about
>> > > >>>>>>> Samza beyond 1.0. What would we change if we were starting
>> from
>> > > >>>> scratch?
>> > > >>>>>>> My
>> > > >>>>>>> proposal is to:
>> > > >>>>>>>
>> > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
>> > > >>>>>>> processors, and eliminate all direct dependences on YARN,
>> Mesos,
>> > > >> etc.
>> > > >>>>>>> 2. Make a definitive call to support only Kafka as the stream
>> > > >>>> processing
>> > > >>>>>>> layer.
>> > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
>> > > >>>>>>> config
>> > > >>>>> systems,
>> > > >>>>>>> and simply use Kafka's instead.
>> > > >>>>>>>
>> > > >>>>>>> This would fix all of the issues that I outlined above. It
>> > > >>>>>>> should
>> > > >>> also
>> > > >>>>>>> shrink the Samza code base pretty dramatically. Supporting
>> only
>> > > >>>>>>> a standalone container will allow Samza to be executed on YARN
>> > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
>> > > >>>>>>> in-house
>> > > >>>>> deployment
>> > > >>>>>>> systems. This should make life a lot easier for new users.
>> > > >>>>>>> Imagine
>> > > >>>>> having
>> > > >>>>>>> the hello-samza tutorial without YARN. The drop in mailing
>> list
>> > > >>>> traffic
>> > > >>>>>>> will be pretty dramatic.
>> > > >>>>>>>
>> > > >>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
>> > > >>> everyone
>> > > >>>>>>> that
>> > > >>>>>>> I'm aware of is using Samza with Kafka. We basically require
>> it
>> > > >>>> already
>> > > >>>>> in
>> > > >>>>>>> order for most features to work. Those that are using other
>> > > >>>>>>> systems
>> > > >>>> are
>> > > >>>>>>> generally using it for ingest into Kafka (1), and then they do
>> > > >>>>>>> the processing on top. There is already discussion (
>> > > >>>
>> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>> > > >>> 767
>> > > >>>>>>> )
>> > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
>> > > >>>>>>>
>> > > >>>>>>> Once we make the call to couple with Kafka, we can leverage a
>> > > >>>>>>> ton of
>> > > >>>>> their
>> > > >>>>>>> ecosystem. We no longer have to maintain our own config,
>> > > >>>>>>> metrics,
>> > > >>> etc.
>> > > >>>>> We
>> > > >>>>>>> can all share the same libraries, and make them better. This
>> > > >>>>>>> will
>> > > >>> also
>> > > >>>>>>> allow us to share the consumer/producer APIs, and will let us
>> > > >>> leverage
>> > > >>>>>>> their offset management and partition management, rather than
>> > > >>>>>>> having
>> > > >>>> our
>> > > >>>>>>> own. All of the coordinator stream code would go away, as
>> would
>> > > >>>>>>> most
>> > > >>>> of
>> > > >>>>>>> the
>> > > >>>>>>> YARN AppMaster code. We'd probably have to push some partition
>> > > >>>>> management
>> > > >>>>>>> features into the Kafka broker, but they're already moving in
>> > > >>>>>>> that direction with the new consumer API. The features we have
>> > > >>>>>>> for
>> > > >>>> partition
>> > > >>>>>>> assignment aren't unique to Samza, and seem like they should
>> be
>> > > >>>>>>> in
>> > > >>>> Kafka
>> > > >>>>>>> anyway. There will always be some niche usages which will
>> > > >>>>>>> require
>> > > >>>> extra
>> > > >>>>>>> care and hence full control over partition assignments much
>> > > >>>>>>> like the
>> > > >>>>> Kafka
>> > > >>>>>>> low level consumer api. These would continue to be supported.
>> > > >>>>>>>
>> > > >>>>>>> These items will be good for the Samza community. They'll make
>> > > >>>>>>> Samza easier to use, and make it easier for developers to add
>> > > >>>>>>> new features.
>> > > >>>>>>>
>> > > >>>>>>> Obviously this is a fairly large (and somewhat backwards
>> > > >>> incompatible
>> > > >>>>>>> change). If we choose to go this route, it's important that we
>> > > >>> openly
>> > > >>>>>>> communicate how we're going to provide a migration path from
>> > > >>>>>>> the
>> > > >>>>> existing
>> > > >>>>>>> APIs to the new ones (if we make incompatible changes). I
>> think
>> > > >>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
>> > > >>>>>>> existing StreamTask implementations to continue running on the
>> > > >> new container.
>> > > >>>>> It's
>> > > >>>>>>> also important that we openly communicate about timing, and
>> > > >>>>>>> stages
>> > > >>> of
>> > > >>>>> the
>> > > >>>>>>> migration.
>> > > >>>>>>>
>> > > >>>>>>> If you made it this far, I'm sure you have opinions. :) Please
>> > > >>>>>>> send
>> > > >>>> your
>> > > >>>>>>> thoughts and feedback.
>> > > >>>>>>>
>> > > >>>>>>> Cheers,
>> > > >>>>>>> Chris
>> > > >>>>
>> > > >>>>
>> > > >>>>
>> > > >>>> --
>> > > >>>> -- Guozhang
>> > > >>
>> > >
>> >
>>
>
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
@Jay, yes, the current function in the JobCoordinator is just partition
management. Maybe we should just call it PartitionManager to make it
explicit.

-Yi

On Thu, Jul 2, 2015 at 2:24 PM, Jay Kreps <ja...@confluent.io> wrote:

> Hey Yi,
>
> What does the JobCoordinator do? YARN/Mesos/etc would be doing the actual
> resource assignment, process restart, etc, right? Is the additional value
> add of the JobCoordinator just partition management?
>
> -Jay
>
> On Thu, Jul 2, 2015 at 11:32 AM, Yi Pan <ni...@gmail.com> wrote:
>
> > Hi, all,
> >
> >
> > Thanks Chris for sending out this proposal and Jay for sharing the
> > extremely illustrative prototype code.
> >
> >
> > I have been thinking it over many times and want to list out my personal
> > opinions below:
> >
> > 1. Generally, I agree with most of the people here on the mailing list on
> > two points:
> >
> >    a. Deeper integration w/ Kafka is great. No more confusing mapping
> from
> > SystemStreamPartition to TopicPartition etc.
> >
> >    b. Separation the ingestion vs transformation greatly simplify the
> > systems APIs
> >
> > Having the above two changes would allow us to remove many unnecessary
> > complexities introduced by those pluggable interfaces Chris’ pointed out,
> > e.g. pluggable streaming systems and serde.
> >
> >
> > To recall one of Chris’s statement on difficulties in dynamic
> deployment, I
> > believe that the difficulties are mainly the result of tight-coupling of
> > partition assignment vs the container deployment in the current system.
> The
> > current container deployment requires a pre-defined partition assignment
> > strategy coupled together w/ the deployment configuration before we can
> > submit to YARN and start the Samza container, which makes the launching
> > process super long. Also, fault-tolerance and the embedded JobCoordinator
> > code in YARN AppMaster is another way of  making dynamic deployment more
> > complex and difficult.
> >
> >
> > First, borrowing Yan’s term, let’s call the Samza standalone process a
> > Samza worker. Here is what I have been thinking:
> >
> > 1. Separate the execution framework from partition assignment/load
> > balancing:
> >
> >     a. a Samza worker should be launched by execution framework that only
> > deals w/ process placement to available nodes. The execution framework
> now
> > should only deal w/ how many such processes are needed, where to put
> them,
> > and how to keep them alive.
> >
> >     b. Partition assignment/load balancing can be a pluggable interface
> in
> > Samza that allows the Samza workers to ask for partition assignments.
> Let’s
> > borrow the name JobCoordinator for now. To allow fault-tolerance in case
> of
> > failure, the partition assignments to workers need to be dynamic. Hence,
> > the abstract interface would be much like what Jay’s code illustrate:
> > get()/onAssigned()/onRevoke(). The implementation of the partition
> > assignment can be either:
> >
> >         a) completely rely on Kafka.
> >
> >         b) explicit partition assignment via JobCoordinator. Chris’s work
> > in SAMZA-516 can be easily incorporated here. The use case in SAMZA-41
> that
> > runs Samza ProcessJob w/ static partition assignment can be implemented
> of
> > JobCoordinator via any home-grown implementation of distributed
> > coordinator. All the work we did in LinkedIn to support dynamic partition
> > assignment and host-affinity SAMZA-617 can be nicely reused as an
> > implementation of JobCoordinator.
> >
> >
> > When we did the above work, I can see three usage patterns in Samza:
> >
> >    a. Samza as a library: Samza has a set of libraries to provide stream
> > processing, just like a third Kafka client (as illustrated in Jay’s
> > example). The execution/deployment is totally controlled by the
> application
> > and the partition coordination is done via Kafka
> >
> >    b. Samza as a process: Samza runs as a standalone process. There may
> not
> > be a execution framework to launch and deploy Samza processes. The
> > partition assignment is pluggable JobCoordinator.
> >
> >    c. Samza as a service: Samza runs as a collection of processes. There
> > will be an execution framework to allocate resource, launch and deploy
> > Samza workers and keep them alive. The same pluggable JobCoordinator is
> > desirable here as well.
> >
> >
> > Lastly, I would argue that CopyCat in KIP-26 should probably follow the
> > same model. Hence, in Samza as a service model as in LinkedIn, we can use
> > the same fault tolerance execution framework to run CopyCat and Samza w/o
> > the need to operate two service platforms, which should address Sriram’s
> > comment in the email thread.
> >
> >
> > Hope the above makes sense. Thanks all!
> >
> >
> > -Yi
> >
> > On Thu, Jul 2, 2015 at 9:53 AM, Sriram <sr...@gmail.com> wrote:
> >
> > > One thing that is worth exploring is to have a transformation and
> > > ingestion library in Kafka but use the same framework for fault
> > tolerance,
> > > resource isolation and management. The biggest difference I see in
> these
> > > two use cases is the API and data model.
> > >
> > >
> > > > On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io> wrote:
> > > >
> > > > Hey Garry,
> > > >
> > > > Yeah that's super frustrating. I'd be happy to chat more about this
> if
> > > > you'd be interested. I think Chris and I started with the idea of
> "what
> > > > would it take to make Samza a kick-ass ingestion tool" but ultimately
> > we
> > > > kind of came around to the idea that ingestion and transformation had
> > > > pretty different needs and coupling the two made things hard.
> > > >
> > > > For what it's worth I think copycat (KIP-26) actually will do what
> you
> > > are
> > > > looking for.
> > > >
> > > > With regard to your point about slider, I don't necessarily disagree.
> > > But I
> > > > think getting good YARN support is quite doable and I think we can
> make
> > > > that work well. I think the issue this proposal solves is that
> > > technically
> > > > it is pretty hard to support multiple cluster management systems the
> > way
> > > > things are now, you need to write an "app master" or "framework" for
> > each
> > > > and they are all a little different so testing is really hard. In the
> > > > absence of this we have been stuck with just YARN which has fantastic
> > > > penetration in the Hadoopy part of the org, but zero penetration
> > > elsewhere.
> > > > Given the huge amount of work being put in to slider, marathon, aws
> > > > tooling, not to mention the umpteen related packaging technologies
> > people
> > > > want to use (Docker, Kubernetes, various cloud-specific deploy tools,
> > > etc)
> > > > I really think it is important to get this right.
> > > >
> > > > -Jay
> > > >
> > > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > > g.turkington@improvedigital.com> wrote:
> > > >
> > > >> Hi all,
> > > >>
> > > >> I think the question below re does Samza become a sub-project of
> Kafka
> > > >> highlights the broader point around migration. Chris mentions
> Samza's
> > > >> maturity is heading towards a v1 release but I'm not sure it feels
> > > right to
> > > >> launch a v1 then immediately plan to deprecate most of it.
> > > >>
> > > >> From a selfish perspective I have some guys who have started working
> > > with
> > > >> Samza and building some new consumers/producers was next up. Sounds
> > like
> > > >> that is absolutely not the direction to go. I need to look into the
> > KIP
> > > in
> > > >> more detail but for me the attractiveness of adding new Samza
> > > >> consumer/producers -- even if yes all they were doing was really
> > getting
> > > >> data into and out of Kafka --  was to avoid  having to worry about
> the
> > > >> lifecycle management of external clients. If there is a generic
> Kafka
> > > >> ingress/egress layer that I can plug a new connector into and have a
> > > lot of
> > > >> the heavy lifting re scale and reliability done for me then it gives
> > me
> > > all
> > > >> the pushing new consumers/producers would. If not then it
> complicates
> > my
> > > >> operational deployments.
> > > >>
> > > >> Which is similar to my other question with the proposal -- if we
> > build a
> > > >> fully available/stand-alone Samza plus the requisite shims to
> > integrate
> > > >> with Slider etc I suspect the former may be a lot more work than we
> > > think.
> > > >> We may make it much easier for a newcomer to get something running
> but
> > > >> having them step up and get a reliable production deployment may
> still
> > > >> dominate mailing list  traffic, if for different reasons than today.
> > > >>
> > > >> Don't get me wrong -- I'm comfortable with making the Samza
> dependency
> > > on
> > > >> Kafka much more explicit and I absolutely see the benefits  in the
> > > >> reduction of duplication and clashing terminologies/abstractions
> that
> > > >> Chris/Jay describe. Samza as a library would likely be a very nice
> > tool
> > > to
> > > >> add to the Kafka ecosystem. I just have the concerns above re the
> > > >> operational side.
> > > >>
> > > >> Garry
> > > >>
> > > >> -----Original Message-----
> > > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> > > >> Sent: 02 July 2015 12:56
> > > >> To: dev@samza.apache.org
> > > >> Subject: Re: Thoughts and obesrvations on Samza
> > > >>
> > > >> Very interesting thoughts.
> > > >> From outside, I have always perceived Samza as a computing layer
> over
> > > >> Kafka.
> > > >>
> > > >> The question, maybe a bit provocative, is "should Samza be a
> > sub-project
> > > >> of Kafka then?"
> > > >> Or does it make sense to keep it as a separate project with a
> separate
> > > >> governance?
> > > >>
> > > >> Cheers,
> > > >>
> > > >> --
> > > >> Gianmarco
> > > >>
> > > >>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> > > >>>
> > > >>> Overall, I agree to couple with Kafka more tightly. Because Samza
> de
> > > >>> facto is based on Kafka, and it should leverage what Kafka has. At
> > the
> > > >>> same time, Kafka does not need to reinvent what Samza already has.
> I
> > > >>> also like the idea of separating the ingestion and transformation.
> > > >>>
> > > >>> But it is a little difficult for me to image how the Samza will
> look
> > > >> like.
> > > >>> And I feel Chris and Jay have a little difference in terms of how
> > > >>> Samza should look like.
> > > >>>
> > > >>> *** Will it look like what Jay's code shows (A client of Kakfa) ?
> And
> > > >>> user's application code calls this client?
> > > >>>
> > > >>> 1. If we make Samza be a library of Kafka (like what the code
> shows),
> > > >>> how do we implement auto-balance and fault-tolerance? Are they
> taken
> > > >>> care by the Kafka broker or other mechanism, such as "Samza worker"
> > > >>> (just make up the name) ?
> > > >>>
> > > >>> 2. What about other features, such as auto-scaling, shared state,
> > > >>> monitoring?
> > > >>>
> > > >>>
> > > >>> *** If we have Samza standalone, (is this what Chris suggests?)
> > > >>>
> > > >>> 1. we still need to ingest data from Kakfa and produce to it. Then
> it
> > > >>> becomes the same as what Samza looks like now, except it does not
> > rely
> > > >>> on Yarn anymore.
> > > >>>
> > > >>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
> > > >>> etc? Use Kafka code as the dependency?
> > > >>>
> > > >>>
> > > >>> Thanks,
> > > >>>
> > > >>> Fang, Yan
> > > >>> yanfang724@gmail.com
> > > >>>
> > > >>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wangguoz@gmail.com
> >
> > > >>> wrote:
> > > >>>
> > > >>>> Read through the code example and it looks good to me. A few
> > > >>>> thoughts regarding deployment:
> > > >>>>
> > > >>>> Today Samza deploys as executable runnable like:
> > > >>>>
> > > >>>> deploy/samza/bin/run-job.sh --config-factory=...
> > > >> --config-path=file://...
> > > >>>>
> > > >>>> And this proposal advocate for deploying Samza more as embedded
> > > >>>> libraries in user application code (ignoring the terminology since
> > > >>>> it is not the
> > > >>> same
> > > >>>> as the prototype code):
> > > >>>>
> > > >>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
> > > >>>> Thread(task); thread.start();
> > > >>>>
> > > >>>> I think both of these deployment modes are important for different
> > > >>>> types
> > > >>> of
> > > >>>> users. That said, I think making Samza purely standalone is still
> > > >>>> sufficient for either runnable or library modes.
> > > >>>>
> > > >>>> Guozhang
> > > >>>>
> > > >>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
> > > wrote:
> > > >>>>>
> > > >>>>> Looks like gmail mangled the code example, it was supposed to
> look
> > > >>>>> like
> > > >>>>> this:
> > > >>>>>
> > > >>>>> Properties props = new Properties();
> > > >>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
> > > >>>>> config = new StreamingConfig(props);
> > > >>>>> config.subscribe("test-topic-1", "test-topic-2");
> > > >>>>> config.processor(ExampleStreamProcessor.class);
> > > >>>>> config.serialization(new StringSerializer(), new
> > > >>>>> StringDeserializer()); KafkaStreaming container = new
> > > >>>>> KafkaStreaming(config); container.run();
> > > >>>>>
> > > >>>>> -Jay
> > > >>>>>
> > > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> > > >> wrote:
> > > >>>>>
> > > >>>>>> Hey guys,
> > > >>>>>>
> > > >>>>>> This came out of some conversations Chris and I were having
> > > >>>>>> around
> > > >>>>> whether
> > > >>>>>> it would make sense to use Samza as a kind of data ingestion
> > > >>> framework
> > > >>>>> for
> > > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> > > >>>> combined
> > > >>>>>> with complaints around config and YARN and the discussion around
> > > >>>>>> how
> > > >>> to
> > > >>>>>> best do a standalone mode.
> > > >>>>>>
> > > >>>>>> So the thought experiment was, given that Samza was basically
> > > >>>>>> already totally Kafka specific, what if you just embraced that
> > > >>>>>> and turned it
> > > >>>> into
> > > >>>>>> something less like a heavyweight framework and more like a
> > > >>>>>> third
> > > >>> Kafka
> > > >>>>>> client--a kind of "producing consumer" with state management
> > > >>>> facilities.
> > > >>>>>> Basically a library. Instead of a complex stream processing
> > > >>>>>> framework
> > > >>>>> this
> > > >>>>>> would actually be a very simple thing, not much more complicated
> > > >>>>>> to
> > > >>> use
> > > >>>>> or
> > > >>>>>> operate than a Kafka consumer. As Chris said we thought about it
> > > >>>>>> a
> > > >>> lot
> > > >>>> of
> > > >>>>>> what Samza (and the other stream processing systems were doing)
> > > >>> seemed
> > > >>>>> like
> > > >>>>>> kind of a hangover from MapReduce.
> > > >>>>>>
> > > >>>>>> Of course you need to ingest/output data to and from the stream
> > > >>>>>> processing. But when we actually looked into how that would
> > > >>>>>> work,
> > > >>> Samza
> > > >>>>>> isn't really an ideal data ingestion framework for a bunch of
> > > >>> reasons.
> > > >>>> To
> > > >>>>>> really do that right you need a pretty different internal data
> > > >>>>>> model
> > > >>>> and
> > > >>>>>> set of apis. So what if you split them and had an api for Kafka
> > > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > > >>>>>> transformation (Samza).
> > > >>>>>>
> > > >>>>>> This would also allow really embracing the same terminology and
> > > >>>>>> conventions. One complaint about the current state is that the
> > > >>>>>> two
> > > >>>>> systems
> > > >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
> > > >>>>> different
> > > >>>>>> config and monitoring systems means you kind of have to learn
> > > >>>>>> Kafka's
> > > >>>>> way,
> > > >>>>>> then learn Samza's slightly different way, then kind of
> > > >>>>>> understand
> > > >>> how
> > > >>>>> they
> > > >>>>>> map to each other, which having walked a few people through this
> > > >>>>>> is surprisingly tricky for folks to get.
> > > >>>>>>
> > > >>>>>> Since I have been spending a lot of time on airplanes I hacked
> > > >>>>>> up an ernest but still somewhat incomplete prototype of what
> > > >>>>>> this would
> > > >>> look
> > > >>>>>> like. This is just unceremoniously dumped into Kafka as it
> > > >>>>>> required a
> > > >>>> few
> > > >>>>>> changes to the new consumer. Here is the code:
> > > >>>
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > > >>> /apache/kafka/clients/streaming
> > > >>>>>>
> > > >>>>>> For the purpose of the prototype I just liberally renamed
> > > >>>>>> everything
> > > >>> to
> > > >>>>>> try to align it with Kafka with no regard for compatibility.
> > > >>>>>>
> > > >>>>>> To use this would be something like this:
> > > >>>>>> Properties props = new Properties();
> > > >>>>>> props.put("bootstrap.servers", "localhost:4242");
> > > >>>>>> StreamingConfig config = new
> > > >>> StreamingConfig(props);
> > > >>>>> config.subscribe("test-topic-1",
> > > >>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > > >>>>> config.serialization(new
> > > >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> > > >>>> container =
> > > >>>>>> new KafkaStreaming(config); container.run();
> > > >>>>>>
> > > >>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
> > > >>>>>> is basically StreamTask.
> > > >>>>>>
> > > >>>>>> So rather than putting all the class names in a file and then
> > > >>>>>> having
> > > >>>> the
> > > >>>>>> job assembled by reflection, you just instantiate the container
> > > >>>>>> programmatically. Work is balanced over however many instances
> > > >>>>>> of
> > > >>> this
> > > >>>>> are
> > > >>>>>> alive at any time (i.e. if an instance dies, new tasks are added
> > > >>>>>> to
> > > >>> the
> > > >>>>>> existing containers without shutting them down).
> > > >>>>>>
> > > >>>>>> We would provide some glue for running this stuff in YARN via
> > > >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> > > >>>>>> but from the
> > > >>>> point
> > > >>>>> of
> > > >>>>>> view of these frameworks these stream processing jobs are just
> > > >>>> stateless
> > > >>>>>> services that can come and go and expand and contract at will.
> > > >>>>>> There
> > > >>> is
> > > >>>>> no
> > > >>>>>> more custom scheduler.
> > > >>>>>>
> > > >>>>>> Here are some relevant details:
> > > >>>>>>
> > > >>>>>>   1. It is only ~1300 lines of code, it would get larger if we
> > > >>>>>>   productionized but not vastly larger. We really do get a ton
> > > >>>>>> of
> > > >>>>> leverage
> > > >>>>>>   out of Kafka.
> > > >>>>>>   2. Partition management is fully delegated to the new
> consumer.
> > > >>> This
> > > >>>>>>   is nice since now any partition management strategy available
> > > >>>>>> to
> > > >>>> Kafka
> > > >>>>>>   consumer is also available to Samza (and vice versa) and with
> > > >>>>>> the
> > > >>>>> exact
> > > >>>>>>   same configs.
> > > >>>>>>   3. It supports state as well as state reuse
> > > >>>>>>
> > > >>>>>> Anyhow take a look, hopefully it is thought provoking.
> > > >>>>>>
> > > >>>>>> -Jay
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > >>>> criccomini@apache.org>
> > > >>>>>> wrote:
> > > >>>>>>
> > > >>>>>>> Hey all,
> > > >>>>>>>
> > > >>>>>>> I have had some discussions with Samza engineers at LinkedIn
> > > >>>>>>> and
> > > >>>>> Confluent
> > > >>>>>>> and we came up with a few observations and would like to
> > > >>>>>>> propose
> > > >>> some
> > > >>>>>>> changes.
> > > >>>>>>>
> > > >>>>>>> We've observed some things that I want to call out about
> > > >>>>>>> Samza's
> > > >>>> design,
> > > >>>>>>> and I'd like to propose some changes.
> > > >>>>>>>
> > > >>>>>>> * Samza is dependent upon a dynamic deployment system.
> > > >>>>>>> * Samza is too pluggable.
> > > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> > > >>>>>>> APIs
> > > >>> are
> > > >>>>>>> trying to solve a lot of the same problems.
> > > >>>>>>>
> > > >>>>>>> All three of these issues are related, but I'll address them in
> > > >>> order.
> > > >>>>>>>
> > > >>>>>>> Deployment
> > > >>>>>>>
> > > >>>>>>> Samza strongly depends on the use of a dynamic deployment
> > > >>>>>>> scheduler
> > > >>>> such
> > > >>>>>>> as
> > > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> > > >>>>>>> there
> > > >>>> would
> > > >>>>>>> be
> > > >>>>>>> one or two winners in this area, and we could support them, and
> > > >>>>>>> the
> > > >>>> rest
> > > >>>>>>> would go away. In reality, there are many variations.
> > > >>>>>>> Furthermore,
> > > >>>> many
> > > >>>>>>> people still prefer to just start their processors like normal
> > > >>>>>>> Java processes, and use traditional deployment scripts such as
> > > >>>>>>> Fabric,
> > > >>>> Chef,
> > > >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> > > >>>>>>> Samza start-up process really painful for first time users.
> > > >>>>>>>
> > > >>>>>>> Dynamic deployment as a requirement was also a bit of a
> > > >>>>>>> mis-fire
> > > >>>> because
> > > >>>>>>> of
> > > >>>>>>> a fundamental misunderstanding between the nature of batch jobs
> > > >>>>>>> and
> > > >>>>> stream
> > > >>>>>>> processing jobs. Early on, we made conscious effort to favor
> > > >>>>>>> the
> > > >>>> Hadoop
> > > >>>>>>> (Map/Reduce) way of doing things, since it worked and was well
> > > >>>>> understood.
> > > >>>>>>> One thing that we missed was that batch jobs have a definite
> > > >>>> beginning,
> > > >>>>>>> and
> > > >>>>>>> end, and stream processing jobs don't (usually). This leads to
> > > >>>>>>> a
> > > >>> much
> > > >>>>>>> simpler scheduling problem for stream processors. You basically
> > > >>>>>>> just
> > > >>>>> need
> > > >>>>>>> to find a place to start the processor, and start it. The way
> > > >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> > > >>>>>>> being "full". We always
> > > >>>> add
> > > >>>>>>> more machines. The problem with coupling Samza with a scheduler
> > > >>>>>>> is
> > > >>>> that
> > > >>>>>>> Samza (as a framework) now has to handle deployment. This pulls
> > > >>>>>>> in a
> > > >>>>> bunch
> > > >>>>>>> of things such as configuration distribution (config stream),
> > > >>>>>>> shell
> > > >>>>> scrips
> > > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff),
> etc.
> > > >>>>>>>
> > > >>>>>>> Another reason for requiring dynamic deployment was to support
> > > >>>>>>> data locality. If you want to have locality, you need to put
> > > >>>>>>> your
> > > >>>> processors
> > > >>>>>>> close to the data they're processing. Upon further
> > > >>>>>>> investigation,
> > > >>>>> though,
> > > >>>>>>> this feature is not that beneficial. There is some good
> > > >>>>>>> discussion
> > > >>>> about
> > > >>>>>>> some problems with it on SAMZA-335. Again, we took the
> > > >>>>>>> Map/Reduce
> > > >>>> path,
> > > >>>>>>> but
> > > >>>>>>> there are some fundamental differences between HDFS and Kafka.
> > > >>>>>>> HDFS
> > > >>>> has
> > > >>>>>>> blocks, while Kafka has partitions. This leads to less
> > > >>>>>>> optimization potential with stream processors on top of Kafka.
> > > >>>>>>>
> > > >>>>>>> This feature is also used as a crutch. Samza doesn't have any
> > > >>>>>>> built
> > > >>> in
> > > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> > > >>>>>>> deployment scheduling system to handle restarts when a
> > > >>>>>>> processor dies. This has
> > > >>>>> made
> > > >>>>>>> it very difficult to write a standalone Samza container
> > > >> (SAMZA-516).
> > > >>>>>>>
> > > >>>>>>> Pluggability
> > > >>>>>>>
> > > >>>>>>> In some cases pluggability is good, but I think that we've gone
> > > >>>>>>> too
> > > >>>> far
> > > >>>>>>> with it. Currently, Samza has:
> > > >>>>>>>
> > > >>>>>>> * Pluggable config.
> > > >>>>>>> * Pluggable metrics.
> > > >>>>>>> * Pluggable deployment systems.
> > > >>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
> > > >> etc).
> > > >>>>>>> * Pluggable serdes.
> > > >>>>>>> * Pluggable storage engines.
> > > >>>>>>> * Pluggable strategies for just about every component
> > > >>> (MessageChooser,
> > > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > >>>>>>>
> > > >>>>>>> There's probably more that I've forgotten, as well. Some of
> > > >>>>>>> these
> > > >>> are
> > > >>>>>>> useful, but some have proven not to be. This all comes at a
> cost:
> > > >>>>>>> complexity. This complexity is making it harder for our users
> > > >>>>>>> to
> > > >>> pick
> > > >>>> up
> > > >>>>>>> and use Samza out of the box. It also makes it difficult for
> > > >>>>>>> Samza developers to reason about what the characteristics of
> > > >>>>>>> the container (since the characteristics change depending on
> > > >>>>>>> which plugins are use).
> > > >>>>>>>
> > > >>>>>>> The issues with pluggability are most visible in the System
> APIs.
> > > >>> What
> > > >>>>>>> Samza really requires to be functional is Kafka as its
> > > >>>>>>> transport
> > > >>>> layer.
> > > >>>>>>> But
> > > >>>>>>> we've conflated two unrelated use cases into one API:
> > > >>>>>>>
> > > >>>>>>> 1. Get data into/out of Kafka.
> > > >>>>>>> 2. Process the data in Kafka.
> > > >>>>>>>
> > > >>>>>>> The current System API supports both of these use cases. The
> > > >>>>>>> problem
> > > >>>> is,
> > > >>>>>>> we
> > > >>>>>>> actually want different features for each use case. By papering
> > > >>>>>>> over
> > > >>>>> these
> > > >>>>>>> two use cases, and providing a single API, we've introduced a
> > > >>>>>>> ton of
> > > >>>>> leaky
> > > >>>>>>> abstractions.
> > > >>>>>>>
> > > >>>>>>> For example, what we'd really like in (2) is to have
> > > >>>>>>> monotonically increasing longs for offsets (like Kafka). This
> > > >>>>>>> would be at odds
> > > >>> with
> > > >>>>> (1),
> > > >>>>>>> though, since different systems have different
> > > >>>>> SCNs/Offsets/UUIDs/vectors.
> > > >>>>>>> There was discussion both on the mailing list and the SQL JIRAs
> > > >>> about
> > > >>>>> the
> > > >>>>>>> need for this.
> > > >>>>>>>
> > > >>>>>>> The same thing holds true for replayability. Kafka allows us to
> > > >>> rewind
> > > >>>>>>> when
> > > >>>>>>> we have a failure. Many other systems don't. In some cases,
> > > >>>>>>> systems
> > > >>>>> return
> > > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
> > > >>>>>>> they
> > > >>>> have
> > > >>>>> no
> > > >>>>>>> offsets.
> > > >>>>>>>
> > > >>>>>>> Partitioning is another example. Kafka supports partitioning,
> > > >>>>>>> but
> > > >>> many
> > > >>>>>>> systems don't. We model this by having a single partition for
> > > >>>>>>> those systems. Still, other systems model partitioning
> > > >> differently (e.g.
> > > >>>>>>> Kinesis).
> > > >>>>>>>
> > > >>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
> > > >>>>>>> system-agnostic way is almost impossible. As is modeling
> > > >>>>>>> metadata
> > > >>> for
> > > >>>>> the
> > > >>>>>>> system (replication factor, partitions, location, etc). The
> > > >>>>>>> list
> > > >>> goes
> > > >>>>> on.
> > > >>>>>>>
> > > >>>>>>> Duplicate work
> > > >>>>>>>
> > > >>>>>>> At the time that we began writing Samza, Kafka's consumer and
> > > >>> producer
> > > >>>>>>> APIs
> > > >>>>>>> had a relatively weak feature set. On the consumer-side, you
> > > >>>>>>> had two
> > > >>>>>>> options: use the high level consumer, or the simple consumer.
> > > >>>>>>> The
> > > >>>>> problem
> > > >>>>>>> with the high-level consumer was that it controlled your
> > > >>>>>>> offsets, partition assignments, and the order in which you
> > > >>>>>>> received messages. The
> > > >>> problem
> > > >>>>>>> with
> > > >>>>>>> the simple consumer is that it's not simple. It's basic. You
> > > >>>>>>> end up
> > > >>>>> having
> > > >>>>>>> to handle a lot of really low-level stuff that you shouldn't.
> > > >>>>>>> We
> > > >>>> spent a
> > > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
> > > >>>>>>> also allows us to support some cool features:
> > > >>>>>>>
> > > >>>>>>> * Per-partition message ordering and prioritization.
> > > >>>>>>> * Tight control over partition assignment to support joins,
> > > >>>>>>> global
> > > >>>> state
> > > >>>>>>> (if we want to implement it :)), etc.
> > > >>>>>>> * Tight control over offset checkpointing.
> > > >>>>>>>
> > > >>>>>>> What we didn't realize at the time is that these features
> > > >>>>>>> should
> > > >>>>> actually
> > > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> > > >>>> processors)
> > > >>>>>>> end up wanting to do things like joins and partition
> > > >>>>>>> assignment. The
> > > >>>>> Kafka
> > > >>>>>>> community has come to the same conclusion. They're adding a ton
> > > >>>>>>> of upgrades into their new Kafka consumer implementation. To a
> > > >>>>>>> large extent,
> > > >>> it's
> > > >>>>>>> duplicate work to what we've already done in Samza.
> > > >>>>>>>
> > > >>>>>>> On top of this, Kafka ended up taking a very similar approach
> > > >>>>>>> to
> > > >>>> Samza's
> > > >>>>>>> KafkaCheckpointManager implementation for handling offset
> > > >>>> checkpointing.
> > > >>>>>>> Like Samza, Kafka's new offset management feature stores offset
> > > >>>>>>> checkpoints in a topic, and allows you to fetch them from the
> > > >>>>>>> broker.
> > > >>>>>>>
> > > >>>>>>> A lot of this seems like a waste, since we could have shared
> > > >>>>>>> the
> > > >>> work
> > > >>>> if
> > > >>>>>>> it
> > > >>>>>>> had been done in Kafka from the get-go.
> > > >>>>>>>
> > > >>>>>>> Vision
> > > >>>>>>>
> > > >>>>>>> All of this leads me to a rather radical proposal. Samza is
> > > >>> relatively
> > > >>>>>>> stable at this point. I'd venture to say that we're near a 1.0
> > > >>>> release.
> > > >>>>>>> I'd
> > > >>>>>>> like to propose that we take what we've learned, and begin
> > > >>>>>>> thinking
> > > >>>>> about
> > > >>>>>>> Samza beyond 1.0. What would we change if we were starting from
> > > >>>> scratch?
> > > >>>>>>> My
> > > >>>>>>> proposal is to:
> > > >>>>>>>
> > > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > > >>>>>>> processors, and eliminate all direct dependences on YARN,
> Mesos,
> > > >> etc.
> > > >>>>>>> 2. Make a definitive call to support only Kafka as the stream
> > > >>>> processing
> > > >>>>>>> layer.
> > > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> > > >>>>>>> config
> > > >>>>> systems,
> > > >>>>>>> and simply use Kafka's instead.
> > > >>>>>>>
> > > >>>>>>> This would fix all of the issues that I outlined above. It
> > > >>>>>>> should
> > > >>> also
> > > >>>>>>> shrink the Samza code base pretty dramatically. Supporting only
> > > >>>>>>> a standalone container will allow Samza to be executed on YARN
> > > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> > > >>>>>>> in-house
> > > >>>>> deployment
> > > >>>>>>> systems. This should make life a lot easier for new users.
> > > >>>>>>> Imagine
> > > >>>>> having
> > > >>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
> > > >>>> traffic
> > > >>>>>>> will be pretty dramatic.
> > > >>>>>>>
> > > >>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
> > > >>> everyone
> > > >>>>>>> that
> > > >>>>>>> I'm aware of is using Samza with Kafka. We basically require it
> > > >>>> already
> > > >>>>> in
> > > >>>>>>> order for most features to work. Those that are using other
> > > >>>>>>> systems
> > > >>>> are
> > > >>>>>>> generally using it for ingest into Kafka (1), and then they do
> > > >>>>>>> the processing on top. There is already discussion (
> > > >>>
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > > >>> 767
> > > >>>>>>> )
> > > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > > >>>>>>>
> > > >>>>>>> Once we make the call to couple with Kafka, we can leverage a
> > > >>>>>>> ton of
> > > >>>>> their
> > > >>>>>>> ecosystem. We no longer have to maintain our own config,
> > > >>>>>>> metrics,
> > > >>> etc.
> > > >>>>> We
> > > >>>>>>> can all share the same libraries, and make them better. This
> > > >>>>>>> will
> > > >>> also
> > > >>>>>>> allow us to share the consumer/producer APIs, and will let us
> > > >>> leverage
> > > >>>>>>> their offset management and partition management, rather than
> > > >>>>>>> having
> > > >>>> our
> > > >>>>>>> own. All of the coordinator stream code would go away, as would
> > > >>>>>>> most
> > > >>>> of
> > > >>>>>>> the
> > > >>>>>>> YARN AppMaster code. We'd probably have to push some partition
> > > >>>>> management
> > > >>>>>>> features into the Kafka broker, but they're already moving in
> > > >>>>>>> that direction with the new consumer API. The features we have
> > > >>>>>>> for
> > > >>>> partition
> > > >>>>>>> assignment aren't unique to Samza, and seem like they should be
> > > >>>>>>> in
> > > >>>> Kafka
> > > >>>>>>> anyway. There will always be some niche usages which will
> > > >>>>>>> require
> > > >>>> extra
> > > >>>>>>> care and hence full control over partition assignments much
> > > >>>>>>> like the
> > > >>>>> Kafka
> > > >>>>>>> low level consumer api. These would continue to be supported.
> > > >>>>>>>
> > > >>>>>>> These items will be good for the Samza community. They'll make
> > > >>>>>>> Samza easier to use, and make it easier for developers to add
> > > >>>>>>> new features.
> > > >>>>>>>
> > > >>>>>>> Obviously this is a fairly large (and somewhat backwards
> > > >>> incompatible
> > > >>>>>>> change). If we choose to go this route, it's important that we
> > > >>> openly
> > > >>>>>>> communicate how we're going to provide a migration path from
> > > >>>>>>> the
> > > >>>>> existing
> > > >>>>>>> APIs to the new ones (if we make incompatible changes). I think
> > > >>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
> > > >>>>>>> existing StreamTask implementations to continue running on the
> > > >> new container.
> > > >>>>> It's
> > > >>>>>>> also important that we openly communicate about timing, and
> > > >>>>>>> stages
> > > >>> of
> > > >>>>> the
> > > >>>>>>> migration.
> > > >>>>>>>
> > > >>>>>>> If you made it this far, I'm sure you have opinions. :) Please
> > > >>>>>>> send
> > > >>>> your
> > > >>>>>>> thoughts and feedback.
> > > >>>>>>>
> > > >>>>>>> Cheers,
> > > >>>>>>> Chris
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>> --
> > > >>>> -- Guozhang
> > > >>
> > >
> >
>

Re: Thoughts and obesrvations on Samza

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

What does the JobCoordinator do? YARN/Mesos/etc would be doing the actual
resource assignment, process restart, etc, right? Is the additional value
add of the JobCoordinator just partition management?

-Jay

On Thu, Jul 2, 2015 at 11:32 AM, Yi Pan <ni...@gmail.com> wrote:

> Hi, all,
>
>
> Thanks Chris for sending out this proposal and Jay for sharing the
> extremely illustrative prototype code.
>
>
> I have been thinking it over many times and want to list out my personal
> opinions below:
>
> 1. Generally, I agree with most of the people here on the mailing list on
> two points:
>
>    a. Deeper integration w/ Kafka is great. No more confusing mapping from
> SystemStreamPartition to TopicPartition etc.
>
>    b. Separation the ingestion vs transformation greatly simplify the
> systems APIs
>
> Having the above two changes would allow us to remove many unnecessary
> complexities introduced by those pluggable interfaces Chris’ pointed out,
> e.g. pluggable streaming systems and serde.
>
>
> To recall one of Chris’s statement on difficulties in dynamic deployment, I
> believe that the difficulties are mainly the result of tight-coupling of
> partition assignment vs the container deployment in the current system. The
> current container deployment requires a pre-defined partition assignment
> strategy coupled together w/ the deployment configuration before we can
> submit to YARN and start the Samza container, which makes the launching
> process super long. Also, fault-tolerance and the embedded JobCoordinator
> code in YARN AppMaster is another way of  making dynamic deployment more
> complex and difficult.
>
>
> First, borrowing Yan’s term, let’s call the Samza standalone process a
> Samza worker. Here is what I have been thinking:
>
> 1. Separate the execution framework from partition assignment/load
> balancing:
>
>     a. a Samza worker should be launched by execution framework that only
> deals w/ process placement to available nodes. The execution framework now
> should only deal w/ how many such processes are needed, where to put them,
> and how to keep them alive.
>
>     b. Partition assignment/load balancing can be a pluggable interface in
> Samza that allows the Samza workers to ask for partition assignments. Let’s
> borrow the name JobCoordinator for now. To allow fault-tolerance in case of
> failure, the partition assignments to workers need to be dynamic. Hence,
> the abstract interface would be much like what Jay’s code illustrate:
> get()/onAssigned()/onRevoke(). The implementation of the partition
> assignment can be either:
>
>         a) completely rely on Kafka.
>
>         b) explicit partition assignment via JobCoordinator. Chris’s work
> in SAMZA-516 can be easily incorporated here. The use case in SAMZA-41 that
> runs Samza ProcessJob w/ static partition assignment can be implemented of
> JobCoordinator via any home-grown implementation of distributed
> coordinator. All the work we did in LinkedIn to support dynamic partition
> assignment and host-affinity SAMZA-617 can be nicely reused as an
> implementation of JobCoordinator.
>
>
> When we did the above work, I can see three usage patterns in Samza:
>
>    a. Samza as a library: Samza has a set of libraries to provide stream
> processing, just like a third Kafka client (as illustrated in Jay’s
> example). The execution/deployment is totally controlled by the application
> and the partition coordination is done via Kafka
>
>    b. Samza as a process: Samza runs as a standalone process. There may not
> be a execution framework to launch and deploy Samza processes. The
> partition assignment is pluggable JobCoordinator.
>
>    c. Samza as a service: Samza runs as a collection of processes. There
> will be an execution framework to allocate resource, launch and deploy
> Samza workers and keep them alive. The same pluggable JobCoordinator is
> desirable here as well.
>
>
> Lastly, I would argue that CopyCat in KIP-26 should probably follow the
> same model. Hence, in Samza as a service model as in LinkedIn, we can use
> the same fault tolerance execution framework to run CopyCat and Samza w/o
> the need to operate two service platforms, which should address Sriram’s
> comment in the email thread.
>
>
> Hope the above makes sense. Thanks all!
>
>
> -Yi
>
> On Thu, Jul 2, 2015 at 9:53 AM, Sriram <sr...@gmail.com> wrote:
>
> > One thing that is worth exploring is to have a transformation and
> > ingestion library in Kafka but use the same framework for fault
> tolerance,
> > resource isolation and management. The biggest difference I see in these
> > two use cases is the API and data model.
> >
> >
> > > On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > Hey Garry,
> > >
> > > Yeah that's super frustrating. I'd be happy to chat more about this if
> > > you'd be interested. I think Chris and I started with the idea of "what
> > > would it take to make Samza a kick-ass ingestion tool" but ultimately
> we
> > > kind of came around to the idea that ingestion and transformation had
> > > pretty different needs and coupling the two made things hard.
> > >
> > > For what it's worth I think copycat (KIP-26) actually will do what you
> > are
> > > looking for.
> > >
> > > With regard to your point about slider, I don't necessarily disagree.
> > But I
> > > think getting good YARN support is quite doable and I think we can make
> > > that work well. I think the issue this proposal solves is that
> > technically
> > > it is pretty hard to support multiple cluster management systems the
> way
> > > things are now, you need to write an "app master" or "framework" for
> each
> > > and they are all a little different so testing is really hard. In the
> > > absence of this we have been stuck with just YARN which has fantastic
> > > penetration in the Hadoopy part of the org, but zero penetration
> > elsewhere.
> > > Given the huge amount of work being put in to slider, marathon, aws
> > > tooling, not to mention the umpteen related packaging technologies
> people
> > > want to use (Docker, Kubernetes, various cloud-specific deploy tools,
> > etc)
> > > I really think it is important to get this right.
> > >
> > > -Jay
> > >
> > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > g.turkington@improvedigital.com> wrote:
> > >
> > >> Hi all,
> > >>
> > >> I think the question below re does Samza become a sub-project of Kafka
> > >> highlights the broader point around migration. Chris mentions Samza's
> > >> maturity is heading towards a v1 release but I'm not sure it feels
> > right to
> > >> launch a v1 then immediately plan to deprecate most of it.
> > >>
> > >> From a selfish perspective I have some guys who have started working
> > with
> > >> Samza and building some new consumers/producers was next up. Sounds
> like
> > >> that is absolutely not the direction to go. I need to look into the
> KIP
> > in
> > >> more detail but for me the attractiveness of adding new Samza
> > >> consumer/producers -- even if yes all they were doing was really
> getting
> > >> data into and out of Kafka --  was to avoid  having to worry about the
> > >> lifecycle management of external clients. If there is a generic Kafka
> > >> ingress/egress layer that I can plug a new connector into and have a
> > lot of
> > >> the heavy lifting re scale and reliability done for me then it gives
> me
> > all
> > >> the pushing new consumers/producers would. If not then it complicates
> my
> > >> operational deployments.
> > >>
> > >> Which is similar to my other question with the proposal -- if we
> build a
> > >> fully available/stand-alone Samza plus the requisite shims to
> integrate
> > >> with Slider etc I suspect the former may be a lot more work than we
> > think.
> > >> We may make it much easier for a newcomer to get something running but
> > >> having them step up and get a reliable production deployment may still
> > >> dominate mailing list  traffic, if for different reasons than today.
> > >>
> > >> Don't get me wrong -- I'm comfortable with making the Samza dependency
> > on
> > >> Kafka much more explicit and I absolutely see the benefits  in the
> > >> reduction of duplication and clashing terminologies/abstractions that
> > >> Chris/Jay describe. Samza as a library would likely be a very nice
> tool
> > to
> > >> add to the Kafka ecosystem. I just have the concerns above re the
> > >> operational side.
> > >>
> > >> Garry
> > >>
> > >> -----Original Message-----
> > >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> > >> Sent: 02 July 2015 12:56
> > >> To: dev@samza.apache.org
> > >> Subject: Re: Thoughts and obesrvations on Samza
> > >>
> > >> Very interesting thoughts.
> > >> From outside, I have always perceived Samza as a computing layer over
> > >> Kafka.
> > >>
> > >> The question, maybe a bit provocative, is "should Samza be a
> sub-project
> > >> of Kafka then?"
> > >> Or does it make sense to keep it as a separate project with a separate
> > >> governance?
> > >>
> > >> Cheers,
> > >>
> > >> --
> > >> Gianmarco
> > >>
> > >>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> > >>>
> > >>> Overall, I agree to couple with Kafka more tightly. Because Samza de
> > >>> facto is based on Kafka, and it should leverage what Kafka has. At
> the
> > >>> same time, Kafka does not need to reinvent what Samza already has. I
> > >>> also like the idea of separating the ingestion and transformation.
> > >>>
> > >>> But it is a little difficult for me to image how the Samza will look
> > >> like.
> > >>> And I feel Chris and Jay have a little difference in terms of how
> > >>> Samza should look like.
> > >>>
> > >>> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
> > >>> user's application code calls this client?
> > >>>
> > >>> 1. If we make Samza be a library of Kafka (like what the code shows),
> > >>> how do we implement auto-balance and fault-tolerance? Are they taken
> > >>> care by the Kafka broker or other mechanism, such as "Samza worker"
> > >>> (just make up the name) ?
> > >>>
> > >>> 2. What about other features, such as auto-scaling, shared state,
> > >>> monitoring?
> > >>>
> > >>>
> > >>> *** If we have Samza standalone, (is this what Chris suggests?)
> > >>>
> > >>> 1. we still need to ingest data from Kakfa and produce to it. Then it
> > >>> becomes the same as what Samza looks like now, except it does not
> rely
> > >>> on Yarn anymore.
> > >>>
> > >>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
> > >>> etc? Use Kafka code as the dependency?
> > >>>
> > >>>
> > >>> Thanks,
> > >>>
> > >>> Fang, Yan
> > >>> yanfang724@gmail.com
> > >>>
> > >>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
> > >>> wrote:
> > >>>
> > >>>> Read through the code example and it looks good to me. A few
> > >>>> thoughts regarding deployment:
> > >>>>
> > >>>> Today Samza deploys as executable runnable like:
> > >>>>
> > >>>> deploy/samza/bin/run-job.sh --config-factory=...
> > >> --config-path=file://...
> > >>>>
> > >>>> And this proposal advocate for deploying Samza more as embedded
> > >>>> libraries in user application code (ignoring the terminology since
> > >>>> it is not the
> > >>> same
> > >>>> as the prototype code):
> > >>>>
> > >>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
> > >>>> Thread(task); thread.start();
> > >>>>
> > >>>> I think both of these deployment modes are important for different
> > >>>> types
> > >>> of
> > >>>> users. That said, I think making Samza purely standalone is still
> > >>>> sufficient for either runnable or library modes.
> > >>>>
> > >>>> Guozhang
> > >>>>
> > >>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
> > wrote:
> > >>>>>
> > >>>>> Looks like gmail mangled the code example, it was supposed to look
> > >>>>> like
> > >>>>> this:
> > >>>>>
> > >>>>> Properties props = new Properties();
> > >>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
> > >>>>> config = new StreamingConfig(props);
> > >>>>> config.subscribe("test-topic-1", "test-topic-2");
> > >>>>> config.processor(ExampleStreamProcessor.class);
> > >>>>> config.serialization(new StringSerializer(), new
> > >>>>> StringDeserializer()); KafkaStreaming container = new
> > >>>>> KafkaStreaming(config); container.run();
> > >>>>>
> > >>>>> -Jay
> > >>>>>
> > >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> > >> wrote:
> > >>>>>
> > >>>>>> Hey guys,
> > >>>>>>
> > >>>>>> This came out of some conversations Chris and I were having
> > >>>>>> around
> > >>>>> whether
> > >>>>>> it would make sense to use Samza as a kind of data ingestion
> > >>> framework
> > >>>>> for
> > >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> > >>>> combined
> > >>>>>> with complaints around config and YARN and the discussion around
> > >>>>>> how
> > >>> to
> > >>>>>> best do a standalone mode.
> > >>>>>>
> > >>>>>> So the thought experiment was, given that Samza was basically
> > >>>>>> already totally Kafka specific, what if you just embraced that
> > >>>>>> and turned it
> > >>>> into
> > >>>>>> something less like a heavyweight framework and more like a
> > >>>>>> third
> > >>> Kafka
> > >>>>>> client--a kind of "producing consumer" with state management
> > >>>> facilities.
> > >>>>>> Basically a library. Instead of a complex stream processing
> > >>>>>> framework
> > >>>>> this
> > >>>>>> would actually be a very simple thing, not much more complicated
> > >>>>>> to
> > >>> use
> > >>>>> or
> > >>>>>> operate than a Kafka consumer. As Chris said we thought about it
> > >>>>>> a
> > >>> lot
> > >>>> of
> > >>>>>> what Samza (and the other stream processing systems were doing)
> > >>> seemed
> > >>>>> like
> > >>>>>> kind of a hangover from MapReduce.
> > >>>>>>
> > >>>>>> Of course you need to ingest/output data to and from the stream
> > >>>>>> processing. But when we actually looked into how that would
> > >>>>>> work,
> > >>> Samza
> > >>>>>> isn't really an ideal data ingestion framework for a bunch of
> > >>> reasons.
> > >>>> To
> > >>>>>> really do that right you need a pretty different internal data
> > >>>>>> model
> > >>>> and
> > >>>>>> set of apis. So what if you split them and had an api for Kafka
> > >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > >>>>>> transformation (Samza).
> > >>>>>>
> > >>>>>> This would also allow really embracing the same terminology and
> > >>>>>> conventions. One complaint about the current state is that the
> > >>>>>> two
> > >>>>> systems
> > >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
> > >>>>> different
> > >>>>>> config and monitoring systems means you kind of have to learn
> > >>>>>> Kafka's
> > >>>>> way,
> > >>>>>> then learn Samza's slightly different way, then kind of
> > >>>>>> understand
> > >>> how
> > >>>>> they
> > >>>>>> map to each other, which having walked a few people through this
> > >>>>>> is surprisingly tricky for folks to get.
> > >>>>>>
> > >>>>>> Since I have been spending a lot of time on airplanes I hacked
> > >>>>>> up an ernest but still somewhat incomplete prototype of what
> > >>>>>> this would
> > >>> look
> > >>>>>> like. This is just unceremoniously dumped into Kafka as it
> > >>>>>> required a
> > >>>> few
> > >>>>>> changes to the new consumer. Here is the code:
> > >>>
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > >>> /apache/kafka/clients/streaming
> > >>>>>>
> > >>>>>> For the purpose of the prototype I just liberally renamed
> > >>>>>> everything
> > >>> to
> > >>>>>> try to align it with Kafka with no regard for compatibility.
> > >>>>>>
> > >>>>>> To use this would be something like this:
> > >>>>>> Properties props = new Properties();
> > >>>>>> props.put("bootstrap.servers", "localhost:4242");
> > >>>>>> StreamingConfig config = new
> > >>> StreamingConfig(props);
> > >>>>> config.subscribe("test-topic-1",
> > >>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > >>>>> config.serialization(new
> > >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> > >>>> container =
> > >>>>>> new KafkaStreaming(config); container.run();
> > >>>>>>
> > >>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
> > >>>>>> is basically StreamTask.
> > >>>>>>
> > >>>>>> So rather than putting all the class names in a file and then
> > >>>>>> having
> > >>>> the
> > >>>>>> job assembled by reflection, you just instantiate the container
> > >>>>>> programmatically. Work is balanced over however many instances
> > >>>>>> of
> > >>> this
> > >>>>> are
> > >>>>>> alive at any time (i.e. if an instance dies, new tasks are added
> > >>>>>> to
> > >>> the
> > >>>>>> existing containers without shutting them down).
> > >>>>>>
> > >>>>>> We would provide some glue for running this stuff in YARN via
> > >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> > >>>>>> but from the
> > >>>> point
> > >>>>> of
> > >>>>>> view of these frameworks these stream processing jobs are just
> > >>>> stateless
> > >>>>>> services that can come and go and expand and contract at will.
> > >>>>>> There
> > >>> is
> > >>>>> no
> > >>>>>> more custom scheduler.
> > >>>>>>
> > >>>>>> Here are some relevant details:
> > >>>>>>
> > >>>>>>   1. It is only ~1300 lines of code, it would get larger if we
> > >>>>>>   productionized but not vastly larger. We really do get a ton
> > >>>>>> of
> > >>>>> leverage
> > >>>>>>   out of Kafka.
> > >>>>>>   2. Partition management is fully delegated to the new consumer.
> > >>> This
> > >>>>>>   is nice since now any partition management strategy available
> > >>>>>> to
> > >>>> Kafka
> > >>>>>>   consumer is also available to Samza (and vice versa) and with
> > >>>>>> the
> > >>>>> exact
> > >>>>>>   same configs.
> > >>>>>>   3. It supports state as well as state reuse
> > >>>>>>
> > >>>>>> Anyhow take a look, hopefully it is thought provoking.
> > >>>>>>
> > >>>>>> -Jay
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > >>>> criccomini@apache.org>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>> Hey all,
> > >>>>>>>
> > >>>>>>> I have had some discussions with Samza engineers at LinkedIn
> > >>>>>>> and
> > >>>>> Confluent
> > >>>>>>> and we came up with a few observations and would like to
> > >>>>>>> propose
> > >>> some
> > >>>>>>> changes.
> > >>>>>>>
> > >>>>>>> We've observed some things that I want to call out about
> > >>>>>>> Samza's
> > >>>> design,
> > >>>>>>> and I'd like to propose some changes.
> > >>>>>>>
> > >>>>>>> * Samza is dependent upon a dynamic deployment system.
> > >>>>>>> * Samza is too pluggable.
> > >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> > >>>>>>> APIs
> > >>> are
> > >>>>>>> trying to solve a lot of the same problems.
> > >>>>>>>
> > >>>>>>> All three of these issues are related, but I'll address them in
> > >>> order.
> > >>>>>>>
> > >>>>>>> Deployment
> > >>>>>>>
> > >>>>>>> Samza strongly depends on the use of a dynamic deployment
> > >>>>>>> scheduler
> > >>>> such
> > >>>>>>> as
> > >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> > >>>>>>> there
> > >>>> would
> > >>>>>>> be
> > >>>>>>> one or two winners in this area, and we could support them, and
> > >>>>>>> the
> > >>>> rest
> > >>>>>>> would go away. In reality, there are many variations.
> > >>>>>>> Furthermore,
> > >>>> many
> > >>>>>>> people still prefer to just start their processors like normal
> > >>>>>>> Java processes, and use traditional deployment scripts such as
> > >>>>>>> Fabric,
> > >>>> Chef,
> > >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> > >>>>>>> Samza start-up process really painful for first time users.
> > >>>>>>>
> > >>>>>>> Dynamic deployment as a requirement was also a bit of a
> > >>>>>>> mis-fire
> > >>>> because
> > >>>>>>> of
> > >>>>>>> a fundamental misunderstanding between the nature of batch jobs
> > >>>>>>> and
> > >>>>> stream
> > >>>>>>> processing jobs. Early on, we made conscious effort to favor
> > >>>>>>> the
> > >>>> Hadoop
> > >>>>>>> (Map/Reduce) way of doing things, since it worked and was well
> > >>>>> understood.
> > >>>>>>> One thing that we missed was that batch jobs have a definite
> > >>>> beginning,
> > >>>>>>> and
> > >>>>>>> end, and stream processing jobs don't (usually). This leads to
> > >>>>>>> a
> > >>> much
> > >>>>>>> simpler scheduling problem for stream processors. You basically
> > >>>>>>> just
> > >>>>> need
> > >>>>>>> to find a place to start the processor, and start it. The way
> > >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> > >>>>>>> being "full". We always
> > >>>> add
> > >>>>>>> more machines. The problem with coupling Samza with a scheduler
> > >>>>>>> is
> > >>>> that
> > >>>>>>> Samza (as a framework) now has to handle deployment. This pulls
> > >>>>>>> in a
> > >>>>> bunch
> > >>>>>>> of things such as configuration distribution (config stream),
> > >>>>>>> shell
> > >>>>> scrips
> > >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> > >>>>>>>
> > >>>>>>> Another reason for requiring dynamic deployment was to support
> > >>>>>>> data locality. If you want to have locality, you need to put
> > >>>>>>> your
> > >>>> processors
> > >>>>>>> close to the data they're processing. Upon further
> > >>>>>>> investigation,
> > >>>>> though,
> > >>>>>>> this feature is not that beneficial. There is some good
> > >>>>>>> discussion
> > >>>> about
> > >>>>>>> some problems with it on SAMZA-335. Again, we took the
> > >>>>>>> Map/Reduce
> > >>>> path,
> > >>>>>>> but
> > >>>>>>> there are some fundamental differences between HDFS and Kafka.
> > >>>>>>> HDFS
> > >>>> has
> > >>>>>>> blocks, while Kafka has partitions. This leads to less
> > >>>>>>> optimization potential with stream processors on top of Kafka.
> > >>>>>>>
> > >>>>>>> This feature is also used as a crutch. Samza doesn't have any
> > >>>>>>> built
> > >>> in
> > >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> > >>>>>>> deployment scheduling system to handle restarts when a
> > >>>>>>> processor dies. This has
> > >>>>> made
> > >>>>>>> it very difficult to write a standalone Samza container
> > >> (SAMZA-516).
> > >>>>>>>
> > >>>>>>> Pluggability
> > >>>>>>>
> > >>>>>>> In some cases pluggability is good, but I think that we've gone
> > >>>>>>> too
> > >>>> far
> > >>>>>>> with it. Currently, Samza has:
> > >>>>>>>
> > >>>>>>> * Pluggable config.
> > >>>>>>> * Pluggable metrics.
> > >>>>>>> * Pluggable deployment systems.
> > >>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
> > >> etc).
> > >>>>>>> * Pluggable serdes.
> > >>>>>>> * Pluggable storage engines.
> > >>>>>>> * Pluggable strategies for just about every component
> > >>> (MessageChooser,
> > >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > >>>>>>>
> > >>>>>>> There's probably more that I've forgotten, as well. Some of
> > >>>>>>> these
> > >>> are
> > >>>>>>> useful, but some have proven not to be. This all comes at a cost:
> > >>>>>>> complexity. This complexity is making it harder for our users
> > >>>>>>> to
> > >>> pick
> > >>>> up
> > >>>>>>> and use Samza out of the box. It also makes it difficult for
> > >>>>>>> Samza developers to reason about what the characteristics of
> > >>>>>>> the container (since the characteristics change depending on
> > >>>>>>> which plugins are use).
> > >>>>>>>
> > >>>>>>> The issues with pluggability are most visible in the System APIs.
> > >>> What
> > >>>>>>> Samza really requires to be functional is Kafka as its
> > >>>>>>> transport
> > >>>> layer.
> > >>>>>>> But
> > >>>>>>> we've conflated two unrelated use cases into one API:
> > >>>>>>>
> > >>>>>>> 1. Get data into/out of Kafka.
> > >>>>>>> 2. Process the data in Kafka.
> > >>>>>>>
> > >>>>>>> The current System API supports both of these use cases. The
> > >>>>>>> problem
> > >>>> is,
> > >>>>>>> we
> > >>>>>>> actually want different features for each use case. By papering
> > >>>>>>> over
> > >>>>> these
> > >>>>>>> two use cases, and providing a single API, we've introduced a
> > >>>>>>> ton of
> > >>>>> leaky
> > >>>>>>> abstractions.
> > >>>>>>>
> > >>>>>>> For example, what we'd really like in (2) is to have
> > >>>>>>> monotonically increasing longs for offsets (like Kafka). This
> > >>>>>>> would be at odds
> > >>> with
> > >>>>> (1),
> > >>>>>>> though, since different systems have different
> > >>>>> SCNs/Offsets/UUIDs/vectors.
> > >>>>>>> There was discussion both on the mailing list and the SQL JIRAs
> > >>> about
> > >>>>> the
> > >>>>>>> need for this.
> > >>>>>>>
> > >>>>>>> The same thing holds true for replayability. Kafka allows us to
> > >>> rewind
> > >>>>>>> when
> > >>>>>>> we have a failure. Many other systems don't. In some cases,
> > >>>>>>> systems
> > >>>>> return
> > >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
> > >>>>>>> they
> > >>>> have
> > >>>>> no
> > >>>>>>> offsets.
> > >>>>>>>
> > >>>>>>> Partitioning is another example. Kafka supports partitioning,
> > >>>>>>> but
> > >>> many
> > >>>>>>> systems don't. We model this by having a single partition for
> > >>>>>>> those systems. Still, other systems model partitioning
> > >> differently (e.g.
> > >>>>>>> Kinesis).
> > >>>>>>>
> > >>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
> > >>>>>>> system-agnostic way is almost impossible. As is modeling
> > >>>>>>> metadata
> > >>> for
> > >>>>> the
> > >>>>>>> system (replication factor, partitions, location, etc). The
> > >>>>>>> list
> > >>> goes
> > >>>>> on.
> > >>>>>>>
> > >>>>>>> Duplicate work
> > >>>>>>>
> > >>>>>>> At the time that we began writing Samza, Kafka's consumer and
> > >>> producer
> > >>>>>>> APIs
> > >>>>>>> had a relatively weak feature set. On the consumer-side, you
> > >>>>>>> had two
> > >>>>>>> options: use the high level consumer, or the simple consumer.
> > >>>>>>> The
> > >>>>> problem
> > >>>>>>> with the high-level consumer was that it controlled your
> > >>>>>>> offsets, partition assignments, and the order in which you
> > >>>>>>> received messages. The
> > >>> problem
> > >>>>>>> with
> > >>>>>>> the simple consumer is that it's not simple. It's basic. You
> > >>>>>>> end up
> > >>>>> having
> > >>>>>>> to handle a lot of really low-level stuff that you shouldn't.
> > >>>>>>> We
> > >>>> spent a
> > >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
> > >>>>>>> also allows us to support some cool features:
> > >>>>>>>
> > >>>>>>> * Per-partition message ordering and prioritization.
> > >>>>>>> * Tight control over partition assignment to support joins,
> > >>>>>>> global
> > >>>> state
> > >>>>>>> (if we want to implement it :)), etc.
> > >>>>>>> * Tight control over offset checkpointing.
> > >>>>>>>
> > >>>>>>> What we didn't realize at the time is that these features
> > >>>>>>> should
> > >>>>> actually
> > >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> > >>>> processors)
> > >>>>>>> end up wanting to do things like joins and partition
> > >>>>>>> assignment. The
> > >>>>> Kafka
> > >>>>>>> community has come to the same conclusion. They're adding a ton
> > >>>>>>> of upgrades into their new Kafka consumer implementation. To a
> > >>>>>>> large extent,
> > >>> it's
> > >>>>>>> duplicate work to what we've already done in Samza.
> > >>>>>>>
> > >>>>>>> On top of this, Kafka ended up taking a very similar approach
> > >>>>>>> to
> > >>>> Samza's
> > >>>>>>> KafkaCheckpointManager implementation for handling offset
> > >>>> checkpointing.
> > >>>>>>> Like Samza, Kafka's new offset management feature stores offset
> > >>>>>>> checkpoints in a topic, and allows you to fetch them from the
> > >>>>>>> broker.
> > >>>>>>>
> > >>>>>>> A lot of this seems like a waste, since we could have shared
> > >>>>>>> the
> > >>> work
> > >>>> if
> > >>>>>>> it
> > >>>>>>> had been done in Kafka from the get-go.
> > >>>>>>>
> > >>>>>>> Vision
> > >>>>>>>
> > >>>>>>> All of this leads me to a rather radical proposal. Samza is
> > >>> relatively
> > >>>>>>> stable at this point. I'd venture to say that we're near a 1.0
> > >>>> release.
> > >>>>>>> I'd
> > >>>>>>> like to propose that we take what we've learned, and begin
> > >>>>>>> thinking
> > >>>>> about
> > >>>>>>> Samza beyond 1.0. What would we change if we were starting from
> > >>>> scratch?
> > >>>>>>> My
> > >>>>>>> proposal is to:
> > >>>>>>>
> > >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> > >>>>>>> processors, and eliminate all direct dependences on YARN, Mesos,
> > >> etc.
> > >>>>>>> 2. Make a definitive call to support only Kafka as the stream
> > >>>> processing
> > >>>>>>> layer.
> > >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> > >>>>>>> config
> > >>>>> systems,
> > >>>>>>> and simply use Kafka's instead.
> > >>>>>>>
> > >>>>>>> This would fix all of the issues that I outlined above. It
> > >>>>>>> should
> > >>> also
> > >>>>>>> shrink the Samza code base pretty dramatically. Supporting only
> > >>>>>>> a standalone container will allow Samza to be executed on YARN
> > >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> > >>>>>>> in-house
> > >>>>> deployment
> > >>>>>>> systems. This should make life a lot easier for new users.
> > >>>>>>> Imagine
> > >>>>> having
> > >>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
> > >>>> traffic
> > >>>>>>> will be pretty dramatic.
> > >>>>>>>
> > >>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
> > >>> everyone
> > >>>>>>> that
> > >>>>>>> I'm aware of is using Samza with Kafka. We basically require it
> > >>>> already
> > >>>>> in
> > >>>>>>> order for most features to work. Those that are using other
> > >>>>>>> systems
> > >>>> are
> > >>>>>>> generally using it for ingest into Kafka (1), and then they do
> > >>>>>>> the processing on top. There is already discussion (
> > >>>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > >>> 767
> > >>>>>>> )
> > >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> > >>>>>>>
> > >>>>>>> Once we make the call to couple with Kafka, we can leverage a
> > >>>>>>> ton of
> > >>>>> their
> > >>>>>>> ecosystem. We no longer have to maintain our own config,
> > >>>>>>> metrics,
> > >>> etc.
> > >>>>> We
> > >>>>>>> can all share the same libraries, and make them better. This
> > >>>>>>> will
> > >>> also
> > >>>>>>> allow us to share the consumer/producer APIs, and will let us
> > >>> leverage
> > >>>>>>> their offset management and partition management, rather than
> > >>>>>>> having
> > >>>> our
> > >>>>>>> own. All of the coordinator stream code would go away, as would
> > >>>>>>> most
> > >>>> of
> > >>>>>>> the
> > >>>>>>> YARN AppMaster code. We'd probably have to push some partition
> > >>>>> management
> > >>>>>>> features into the Kafka broker, but they're already moving in
> > >>>>>>> that direction with the new consumer API. The features we have
> > >>>>>>> for
> > >>>> partition
> > >>>>>>> assignment aren't unique to Samza, and seem like they should be
> > >>>>>>> in
> > >>>> Kafka
> > >>>>>>> anyway. There will always be some niche usages which will
> > >>>>>>> require
> > >>>> extra
> > >>>>>>> care and hence full control over partition assignments much
> > >>>>>>> like the
> > >>>>> Kafka
> > >>>>>>> low level consumer api. These would continue to be supported.
> > >>>>>>>
> > >>>>>>> These items will be good for the Samza community. They'll make
> > >>>>>>> Samza easier to use, and make it easier for developers to add
> > >>>>>>> new features.
> > >>>>>>>
> > >>>>>>> Obviously this is a fairly large (and somewhat backwards
> > >>> incompatible
> > >>>>>>> change). If we choose to go this route, it's important that we
> > >>> openly
> > >>>>>>> communicate how we're going to provide a migration path from
> > >>>>>>> the
> > >>>>> existing
> > >>>>>>> APIs to the new ones (if we make incompatible changes). I think
> > >>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
> > >>>>>>> existing StreamTask implementations to continue running on the
> > >> new container.
> > >>>>> It's
> > >>>>>>> also important that we openly communicate about timing, and
> > >>>>>>> stages
> > >>> of
> > >>>>> the
> > >>>>>>> migration.
> > >>>>>>>
> > >>>>>>> If you made it this far, I'm sure you have opinions. :) Please
> > >>>>>>> send
> > >>>> your
> > >>>>>>> thoughts and feedback.
> > >>>>>>>
> > >>>>>>> Cheers,
> > >>>>>>> Chris
> > >>>>
> > >>>>
> > >>>>
> > >>>> --
> > >>>> -- Guozhang
> > >>
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yi Pan <ni...@gmail.com>.
Hi, all,


Thanks Chris for sending out this proposal and Jay for sharing the
extremely illustrative prototype code.


I have been thinking it over many times and want to list out my personal
opinions below:

1. Generally, I agree with most of the people here on the mailing list on
two points:

   a. Deeper integration w/ Kafka is great. No more confusing mapping from
SystemStreamPartition to TopicPartition etc.

   b. Separation the ingestion vs transformation greatly simplify the
systems APIs

Having the above two changes would allow us to remove many unnecessary
complexities introduced by those pluggable interfaces Chris’ pointed out,
e.g. pluggable streaming systems and serde.


To recall one of Chris’s statement on difficulties in dynamic deployment, I
believe that the difficulties are mainly the result of tight-coupling of
partition assignment vs the container deployment in the current system. The
current container deployment requires a pre-defined partition assignment
strategy coupled together w/ the deployment configuration before we can
submit to YARN and start the Samza container, which makes the launching
process super long. Also, fault-tolerance and the embedded JobCoordinator
code in YARN AppMaster is another way of  making dynamic deployment more
complex and difficult.


First, borrowing Yan’s term, let’s call the Samza standalone process a
Samza worker. Here is what I have been thinking:

1. Separate the execution framework from partition assignment/load
balancing:

    a. a Samza worker should be launched by execution framework that only
deals w/ process placement to available nodes. The execution framework now
should only deal w/ how many such processes are needed, where to put them,
and how to keep them alive.

    b. Partition assignment/load balancing can be a pluggable interface in
Samza that allows the Samza workers to ask for partition assignments. Let’s
borrow the name JobCoordinator for now. To allow fault-tolerance in case of
failure, the partition assignments to workers need to be dynamic. Hence,
the abstract interface would be much like what Jay’s code illustrate:
get()/onAssigned()/onRevoke(). The implementation of the partition
assignment can be either:

        a) completely rely on Kafka.

        b) explicit partition assignment via JobCoordinator. Chris’s work
in SAMZA-516 can be easily incorporated here. The use case in SAMZA-41 that
runs Samza ProcessJob w/ static partition assignment can be implemented of
JobCoordinator via any home-grown implementation of distributed
coordinator. All the work we did in LinkedIn to support dynamic partition
assignment and host-affinity SAMZA-617 can be nicely reused as an
implementation of JobCoordinator.


When we did the above work, I can see three usage patterns in Samza:

   a. Samza as a library: Samza has a set of libraries to provide stream
processing, just like a third Kafka client (as illustrated in Jay’s
example). The execution/deployment is totally controlled by the application
and the partition coordination is done via Kafka

   b. Samza as a process: Samza runs as a standalone process. There may not
be a execution framework to launch and deploy Samza processes. The
partition assignment is pluggable JobCoordinator.

   c. Samza as a service: Samza runs as a collection of processes. There
will be an execution framework to allocate resource, launch and deploy
Samza workers and keep them alive. The same pluggable JobCoordinator is
desirable here as well.


Lastly, I would argue that CopyCat in KIP-26 should probably follow the
same model. Hence, in Samza as a service model as in LinkedIn, we can use
the same fault tolerance execution framework to run CopyCat and Samza w/o
the need to operate two service platforms, which should address Sriram’s
comment in the email thread.


Hope the above makes sense. Thanks all!


-Yi

On Thu, Jul 2, 2015 at 9:53 AM, Sriram <sr...@gmail.com> wrote:

> One thing that is worth exploring is to have a transformation and
> ingestion library in Kafka but use the same framework for fault tolerance,
> resource isolation and management. The biggest difference I see in these
> two use cases is the API and data model.
>
>
> > On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > Hey Garry,
> >
> > Yeah that's super frustrating. I'd be happy to chat more about this if
> > you'd be interested. I think Chris and I started with the idea of "what
> > would it take to make Samza a kick-ass ingestion tool" but ultimately we
> > kind of came around to the idea that ingestion and transformation had
> > pretty different needs and coupling the two made things hard.
> >
> > For what it's worth I think copycat (KIP-26) actually will do what you
> are
> > looking for.
> >
> > With regard to your point about slider, I don't necessarily disagree.
> But I
> > think getting good YARN support is quite doable and I think we can make
> > that work well. I think the issue this proposal solves is that
> technically
> > it is pretty hard to support multiple cluster management systems the way
> > things are now, you need to write an "app master" or "framework" for each
> > and they are all a little different so testing is really hard. In the
> > absence of this we have been stuck with just YARN which has fantastic
> > penetration in the Hadoopy part of the org, but zero penetration
> elsewhere.
> > Given the huge amount of work being put in to slider, marathon, aws
> > tooling, not to mention the umpteen related packaging technologies people
> > want to use (Docker, Kubernetes, various cloud-specific deploy tools,
> etc)
> > I really think it is important to get this right.
> >
> > -Jay
> >
> > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > g.turkington@improvedigital.com> wrote:
> >
> >> Hi all,
> >>
> >> I think the question below re does Samza become a sub-project of Kafka
> >> highlights the broader point around migration. Chris mentions Samza's
> >> maturity is heading towards a v1 release but I'm not sure it feels
> right to
> >> launch a v1 then immediately plan to deprecate most of it.
> >>
> >> From a selfish perspective I have some guys who have started working
> with
> >> Samza and building some new consumers/producers was next up. Sounds like
> >> that is absolutely not the direction to go. I need to look into the KIP
> in
> >> more detail but for me the attractiveness of adding new Samza
> >> consumer/producers -- even if yes all they were doing was really getting
> >> data into and out of Kafka --  was to avoid  having to worry about the
> >> lifecycle management of external clients. If there is a generic Kafka
> >> ingress/egress layer that I can plug a new connector into and have a
> lot of
> >> the heavy lifting re scale and reliability done for me then it gives me
> all
> >> the pushing new consumers/producers would. If not then it complicates my
> >> operational deployments.
> >>
> >> Which is similar to my other question with the proposal -- if we build a
> >> fully available/stand-alone Samza plus the requisite shims to integrate
> >> with Slider etc I suspect the former may be a lot more work than we
> think.
> >> We may make it much easier for a newcomer to get something running but
> >> having them step up and get a reliable production deployment may still
> >> dominate mailing list  traffic, if for different reasons than today.
> >>
> >> Don't get me wrong -- I'm comfortable with making the Samza dependency
> on
> >> Kafka much more explicit and I absolutely see the benefits  in the
> >> reduction of duplication and clashing terminologies/abstractions that
> >> Chris/Jay describe. Samza as a library would likely be a very nice tool
> to
> >> add to the Kafka ecosystem. I just have the concerns above re the
> >> operational side.
> >>
> >> Garry
> >>
> >> -----Original Message-----
> >> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> >> Sent: 02 July 2015 12:56
> >> To: dev@samza.apache.org
> >> Subject: Re: Thoughts and obesrvations on Samza
> >>
> >> Very interesting thoughts.
> >> From outside, I have always perceived Samza as a computing layer over
> >> Kafka.
> >>
> >> The question, maybe a bit provocative, is "should Samza be a sub-project
> >> of Kafka then?"
> >> Or does it make sense to keep it as a separate project with a separate
> >> governance?
> >>
> >> Cheers,
> >>
> >> --
> >> Gianmarco
> >>
> >>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
> >>>
> >>> Overall, I agree to couple with Kafka more tightly. Because Samza de
> >>> facto is based on Kafka, and it should leverage what Kafka has. At the
> >>> same time, Kafka does not need to reinvent what Samza already has. I
> >>> also like the idea of separating the ingestion and transformation.
> >>>
> >>> But it is a little difficult for me to image how the Samza will look
> >> like.
> >>> And I feel Chris and Jay have a little difference in terms of how
> >>> Samza should look like.
> >>>
> >>> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
> >>> user's application code calls this client?
> >>>
> >>> 1. If we make Samza be a library of Kafka (like what the code shows),
> >>> how do we implement auto-balance and fault-tolerance? Are they taken
> >>> care by the Kafka broker or other mechanism, such as "Samza worker"
> >>> (just make up the name) ?
> >>>
> >>> 2. What about other features, such as auto-scaling, shared state,
> >>> monitoring?
> >>>
> >>>
> >>> *** If we have Samza standalone, (is this what Chris suggests?)
> >>>
> >>> 1. we still need to ingest data from Kakfa and produce to it. Then it
> >>> becomes the same as what Samza looks like now, except it does not rely
> >>> on Yarn anymore.
> >>>
> >>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
> >>> etc? Use Kafka code as the dependency?
> >>>
> >>>
> >>> Thanks,
> >>>
> >>> Fang, Yan
> >>> yanfang724@gmail.com
> >>>
> >>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
> >>> wrote:
> >>>
> >>>> Read through the code example and it looks good to me. A few
> >>>> thoughts regarding deployment:
> >>>>
> >>>> Today Samza deploys as executable runnable like:
> >>>>
> >>>> deploy/samza/bin/run-job.sh --config-factory=...
> >> --config-path=file://...
> >>>>
> >>>> And this proposal advocate for deploying Samza more as embedded
> >>>> libraries in user application code (ignoring the terminology since
> >>>> it is not the
> >>> same
> >>>> as the prototype code):
> >>>>
> >>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
> >>>> Thread(task); thread.start();
> >>>>
> >>>> I think both of these deployment modes are important for different
> >>>> types
> >>> of
> >>>> users. That said, I think making Samza purely standalone is still
> >>>> sufficient for either runnable or library modes.
> >>>>
> >>>> Guozhang
> >>>>
> >>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> >>>>>
> >>>>> Looks like gmail mangled the code example, it was supposed to look
> >>>>> like
> >>>>> this:
> >>>>>
> >>>>> Properties props = new Properties();
> >>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
> >>>>> config = new StreamingConfig(props);
> >>>>> config.subscribe("test-topic-1", "test-topic-2");
> >>>>> config.processor(ExampleStreamProcessor.class);
> >>>>> config.serialization(new StringSerializer(), new
> >>>>> StringDeserializer()); KafkaStreaming container = new
> >>>>> KafkaStreaming(config); container.run();
> >>>>>
> >>>>> -Jay
> >>>>>
> >>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> >> wrote:
> >>>>>
> >>>>>> Hey guys,
> >>>>>>
> >>>>>> This came out of some conversations Chris and I were having
> >>>>>> around
> >>>>> whether
> >>>>>> it would make sense to use Samza as a kind of data ingestion
> >>> framework
> >>>>> for
> >>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> >>>> combined
> >>>>>> with complaints around config and YARN and the discussion around
> >>>>>> how
> >>> to
> >>>>>> best do a standalone mode.
> >>>>>>
> >>>>>> So the thought experiment was, given that Samza was basically
> >>>>>> already totally Kafka specific, what if you just embraced that
> >>>>>> and turned it
> >>>> into
> >>>>>> something less like a heavyweight framework and more like a
> >>>>>> third
> >>> Kafka
> >>>>>> client--a kind of "producing consumer" with state management
> >>>> facilities.
> >>>>>> Basically a library. Instead of a complex stream processing
> >>>>>> framework
> >>>>> this
> >>>>>> would actually be a very simple thing, not much more complicated
> >>>>>> to
> >>> use
> >>>>> or
> >>>>>> operate than a Kafka consumer. As Chris said we thought about it
> >>>>>> a
> >>> lot
> >>>> of
> >>>>>> what Samza (and the other stream processing systems were doing)
> >>> seemed
> >>>>> like
> >>>>>> kind of a hangover from MapReduce.
> >>>>>>
> >>>>>> Of course you need to ingest/output data to and from the stream
> >>>>>> processing. But when we actually looked into how that would
> >>>>>> work,
> >>> Samza
> >>>>>> isn't really an ideal data ingestion framework for a bunch of
> >>> reasons.
> >>>> To
> >>>>>> really do that right you need a pretty different internal data
> >>>>>> model
> >>>> and
> >>>>>> set of apis. So what if you split them and had an api for Kafka
> >>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> >>>>>> transformation (Samza).
> >>>>>>
> >>>>>> This would also allow really embracing the same terminology and
> >>>>>> conventions. One complaint about the current state is that the
> >>>>>> two
> >>>>> systems
> >>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
> >>>>> different
> >>>>>> config and monitoring systems means you kind of have to learn
> >>>>>> Kafka's
> >>>>> way,
> >>>>>> then learn Samza's slightly different way, then kind of
> >>>>>> understand
> >>> how
> >>>>> they
> >>>>>> map to each other, which having walked a few people through this
> >>>>>> is surprisingly tricky for folks to get.
> >>>>>>
> >>>>>> Since I have been spending a lot of time on airplanes I hacked
> >>>>>> up an ernest but still somewhat incomplete prototype of what
> >>>>>> this would
> >>> look
> >>>>>> like. This is just unceremoniously dumped into Kafka as it
> >>>>>> required a
> >>>> few
> >>>>>> changes to the new consumer. Here is the code:
> >>> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> >>> /apache/kafka/clients/streaming
> >>>>>>
> >>>>>> For the purpose of the prototype I just liberally renamed
> >>>>>> everything
> >>> to
> >>>>>> try to align it with Kafka with no regard for compatibility.
> >>>>>>
> >>>>>> To use this would be something like this:
> >>>>>> Properties props = new Properties();
> >>>>>> props.put("bootstrap.servers", "localhost:4242");
> >>>>>> StreamingConfig config = new
> >>> StreamingConfig(props);
> >>>>> config.subscribe("test-topic-1",
> >>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> >>>>> config.serialization(new
> >>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
> >>>> container =
> >>>>>> new KafkaStreaming(config); container.run();
> >>>>>>
> >>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
> >>>>>> is basically StreamTask.
> >>>>>>
> >>>>>> So rather than putting all the class names in a file and then
> >>>>>> having
> >>>> the
> >>>>>> job assembled by reflection, you just instantiate the container
> >>>>>> programmatically. Work is balanced over however many instances
> >>>>>> of
> >>> this
> >>>>> are
> >>>>>> alive at any time (i.e. if an instance dies, new tasks are added
> >>>>>> to
> >>> the
> >>>>>> existing containers without shutting them down).
> >>>>>>
> >>>>>> We would provide some glue for running this stuff in YARN via
> >>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
> >>>>>> but from the
> >>>> point
> >>>>> of
> >>>>>> view of these frameworks these stream processing jobs are just
> >>>> stateless
> >>>>>> services that can come and go and expand and contract at will.
> >>>>>> There
> >>> is
> >>>>> no
> >>>>>> more custom scheduler.
> >>>>>>
> >>>>>> Here are some relevant details:
> >>>>>>
> >>>>>>   1. It is only ~1300 lines of code, it would get larger if we
> >>>>>>   productionized but not vastly larger. We really do get a ton
> >>>>>> of
> >>>>> leverage
> >>>>>>   out of Kafka.
> >>>>>>   2. Partition management is fully delegated to the new consumer.
> >>> This
> >>>>>>   is nice since now any partition management strategy available
> >>>>>> to
> >>>> Kafka
> >>>>>>   consumer is also available to Samza (and vice versa) and with
> >>>>>> the
> >>>>> exact
> >>>>>>   same configs.
> >>>>>>   3. It supports state as well as state reuse
> >>>>>>
> >>>>>> Anyhow take a look, hopefully it is thought provoking.
> >>>>>>
> >>>>>> -Jay
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> >>>> criccomini@apache.org>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Hey all,
> >>>>>>>
> >>>>>>> I have had some discussions with Samza engineers at LinkedIn
> >>>>>>> and
> >>>>> Confluent
> >>>>>>> and we came up with a few observations and would like to
> >>>>>>> propose
> >>> some
> >>>>>>> changes.
> >>>>>>>
> >>>>>>> We've observed some things that I want to call out about
> >>>>>>> Samza's
> >>>> design,
> >>>>>>> and I'd like to propose some changes.
> >>>>>>>
> >>>>>>> * Samza is dependent upon a dynamic deployment system.
> >>>>>>> * Samza is too pluggable.
> >>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> >>>>>>> APIs
> >>> are
> >>>>>>> trying to solve a lot of the same problems.
> >>>>>>>
> >>>>>>> All three of these issues are related, but I'll address them in
> >>> order.
> >>>>>>>
> >>>>>>> Deployment
> >>>>>>>
> >>>>>>> Samza strongly depends on the use of a dynamic deployment
> >>>>>>> scheduler
> >>>> such
> >>>>>>> as
> >>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
> >>>>>>> there
> >>>> would
> >>>>>>> be
> >>>>>>> one or two winners in this area, and we could support them, and
> >>>>>>> the
> >>>> rest
> >>>>>>> would go away. In reality, there are many variations.
> >>>>>>> Furthermore,
> >>>> many
> >>>>>>> people still prefer to just start their processors like normal
> >>>>>>> Java processes, and use traditional deployment scripts such as
> >>>>>>> Fabric,
> >>>> Chef,
> >>>>>>> Ansible, etc. Forcing a deployment system on users makes the
> >>>>>>> Samza start-up process really painful for first time users.
> >>>>>>>
> >>>>>>> Dynamic deployment as a requirement was also a bit of a
> >>>>>>> mis-fire
> >>>> because
> >>>>>>> of
> >>>>>>> a fundamental misunderstanding between the nature of batch jobs
> >>>>>>> and
> >>>>> stream
> >>>>>>> processing jobs. Early on, we made conscious effort to favor
> >>>>>>> the
> >>>> Hadoop
> >>>>>>> (Map/Reduce) way of doing things, since it worked and was well
> >>>>> understood.
> >>>>>>> One thing that we missed was that batch jobs have a definite
> >>>> beginning,
> >>>>>>> and
> >>>>>>> end, and stream processing jobs don't (usually). This leads to
> >>>>>>> a
> >>> much
> >>>>>>> simpler scheduling problem for stream processors. You basically
> >>>>>>> just
> >>>>> need
> >>>>>>> to find a place to start the processor, and start it. The way
> >>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
> >>>>>>> being "full". We always
> >>>> add
> >>>>>>> more machines. The problem with coupling Samza with a scheduler
> >>>>>>> is
> >>>> that
> >>>>>>> Samza (as a framework) now has to handle deployment. This pulls
> >>>>>>> in a
> >>>>> bunch
> >>>>>>> of things such as configuration distribution (config stream),
> >>>>>>> shell
> >>>>> scrips
> >>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> >>>>>>>
> >>>>>>> Another reason for requiring dynamic deployment was to support
> >>>>>>> data locality. If you want to have locality, you need to put
> >>>>>>> your
> >>>> processors
> >>>>>>> close to the data they're processing. Upon further
> >>>>>>> investigation,
> >>>>> though,
> >>>>>>> this feature is not that beneficial. There is some good
> >>>>>>> discussion
> >>>> about
> >>>>>>> some problems with it on SAMZA-335. Again, we took the
> >>>>>>> Map/Reduce
> >>>> path,
> >>>>>>> but
> >>>>>>> there are some fundamental differences between HDFS and Kafka.
> >>>>>>> HDFS
> >>>> has
> >>>>>>> blocks, while Kafka has partitions. This leads to less
> >>>>>>> optimization potential with stream processors on top of Kafka.
> >>>>>>>
> >>>>>>> This feature is also used as a crutch. Samza doesn't have any
> >>>>>>> built
> >>> in
> >>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
> >>>>>>> deployment scheduling system to handle restarts when a
> >>>>>>> processor dies. This has
> >>>>> made
> >>>>>>> it very difficult to write a standalone Samza container
> >> (SAMZA-516).
> >>>>>>>
> >>>>>>> Pluggability
> >>>>>>>
> >>>>>>> In some cases pluggability is good, but I think that we've gone
> >>>>>>> too
> >>>> far
> >>>>>>> with it. Currently, Samza has:
> >>>>>>>
> >>>>>>> * Pluggable config.
> >>>>>>> * Pluggable metrics.
> >>>>>>> * Pluggable deployment systems.
> >>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
> >> etc).
> >>>>>>> * Pluggable serdes.
> >>>>>>> * Pluggable storage engines.
> >>>>>>> * Pluggable strategies for just about every component
> >>> (MessageChooser,
> >>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> >>>>>>>
> >>>>>>> There's probably more that I've forgotten, as well. Some of
> >>>>>>> these
> >>> are
> >>>>>>> useful, but some have proven not to be. This all comes at a cost:
> >>>>>>> complexity. This complexity is making it harder for our users
> >>>>>>> to
> >>> pick
> >>>> up
> >>>>>>> and use Samza out of the box. It also makes it difficult for
> >>>>>>> Samza developers to reason about what the characteristics of
> >>>>>>> the container (since the characteristics change depending on
> >>>>>>> which plugins are use).
> >>>>>>>
> >>>>>>> The issues with pluggability are most visible in the System APIs.
> >>> What
> >>>>>>> Samza really requires to be functional is Kafka as its
> >>>>>>> transport
> >>>> layer.
> >>>>>>> But
> >>>>>>> we've conflated two unrelated use cases into one API:
> >>>>>>>
> >>>>>>> 1. Get data into/out of Kafka.
> >>>>>>> 2. Process the data in Kafka.
> >>>>>>>
> >>>>>>> The current System API supports both of these use cases. The
> >>>>>>> problem
> >>>> is,
> >>>>>>> we
> >>>>>>> actually want different features for each use case. By papering
> >>>>>>> over
> >>>>> these
> >>>>>>> two use cases, and providing a single API, we've introduced a
> >>>>>>> ton of
> >>>>> leaky
> >>>>>>> abstractions.
> >>>>>>>
> >>>>>>> For example, what we'd really like in (2) is to have
> >>>>>>> monotonically increasing longs for offsets (like Kafka). This
> >>>>>>> would be at odds
> >>> with
> >>>>> (1),
> >>>>>>> though, since different systems have different
> >>>>> SCNs/Offsets/UUIDs/vectors.
> >>>>>>> There was discussion both on the mailing list and the SQL JIRAs
> >>> about
> >>>>> the
> >>>>>>> need for this.
> >>>>>>>
> >>>>>>> The same thing holds true for replayability. Kafka allows us to
> >>> rewind
> >>>>>>> when
> >>>>>>> we have a failure. Many other systems don't. In some cases,
> >>>>>>> systems
> >>>>> return
> >>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
> >>>>>>> they
> >>>> have
> >>>>> no
> >>>>>>> offsets.
> >>>>>>>
> >>>>>>> Partitioning is another example. Kafka supports partitioning,
> >>>>>>> but
> >>> many
> >>>>>>> systems don't. We model this by having a single partition for
> >>>>>>> those systems. Still, other systems model partitioning
> >> differently (e.g.
> >>>>>>> Kinesis).
> >>>>>>>
> >>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
> >>>>>>> system-agnostic way is almost impossible. As is modeling
> >>>>>>> metadata
> >>> for
> >>>>> the
> >>>>>>> system (replication factor, partitions, location, etc). The
> >>>>>>> list
> >>> goes
> >>>>> on.
> >>>>>>>
> >>>>>>> Duplicate work
> >>>>>>>
> >>>>>>> At the time that we began writing Samza, Kafka's consumer and
> >>> producer
> >>>>>>> APIs
> >>>>>>> had a relatively weak feature set. On the consumer-side, you
> >>>>>>> had two
> >>>>>>> options: use the high level consumer, or the simple consumer.
> >>>>>>> The
> >>>>> problem
> >>>>>>> with the high-level consumer was that it controlled your
> >>>>>>> offsets, partition assignments, and the order in which you
> >>>>>>> received messages. The
> >>> problem
> >>>>>>> with
> >>>>>>> the simple consumer is that it's not simple. It's basic. You
> >>>>>>> end up
> >>>>> having
> >>>>>>> to handle a lot of really low-level stuff that you shouldn't.
> >>>>>>> We
> >>>> spent a
> >>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
> >>>>>>> also allows us to support some cool features:
> >>>>>>>
> >>>>>>> * Per-partition message ordering and prioritization.
> >>>>>>> * Tight control over partition assignment to support joins,
> >>>>>>> global
> >>>> state
> >>>>>>> (if we want to implement it :)), etc.
> >>>>>>> * Tight control over offset checkpointing.
> >>>>>>>
> >>>>>>> What we didn't realize at the time is that these features
> >>>>>>> should
> >>>>> actually
> >>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
> >>>> processors)
> >>>>>>> end up wanting to do things like joins and partition
> >>>>>>> assignment. The
> >>>>> Kafka
> >>>>>>> community has come to the same conclusion. They're adding a ton
> >>>>>>> of upgrades into their new Kafka consumer implementation. To a
> >>>>>>> large extent,
> >>> it's
> >>>>>>> duplicate work to what we've already done in Samza.
> >>>>>>>
> >>>>>>> On top of this, Kafka ended up taking a very similar approach
> >>>>>>> to
> >>>> Samza's
> >>>>>>> KafkaCheckpointManager implementation for handling offset
> >>>> checkpointing.
> >>>>>>> Like Samza, Kafka's new offset management feature stores offset
> >>>>>>> checkpoints in a topic, and allows you to fetch them from the
> >>>>>>> broker.
> >>>>>>>
> >>>>>>> A lot of this seems like a waste, since we could have shared
> >>>>>>> the
> >>> work
> >>>> if
> >>>>>>> it
> >>>>>>> had been done in Kafka from the get-go.
> >>>>>>>
> >>>>>>> Vision
> >>>>>>>
> >>>>>>> All of this leads me to a rather radical proposal. Samza is
> >>> relatively
> >>>>>>> stable at this point. I'd venture to say that we're near a 1.0
> >>>> release.
> >>>>>>> I'd
> >>>>>>> like to propose that we take what we've learned, and begin
> >>>>>>> thinking
> >>>>> about
> >>>>>>> Samza beyond 1.0. What would we change if we were starting from
> >>>> scratch?
> >>>>>>> My
> >>>>>>> proposal is to:
> >>>>>>>
> >>>>>>> 1. Make Samza standalone the *only* way to run Samza
> >>>>>>> processors, and eliminate all direct dependences on YARN, Mesos,
> >> etc.
> >>>>>>> 2. Make a definitive call to support only Kafka as the stream
> >>>> processing
> >>>>>>> layer.
> >>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
> >>>>>>> config
> >>>>> systems,
> >>>>>>> and simply use Kafka's instead.
> >>>>>>>
> >>>>>>> This would fix all of the issues that I outlined above. It
> >>>>>>> should
> >>> also
> >>>>>>> shrink the Samza code base pretty dramatically. Supporting only
> >>>>>>> a standalone container will allow Samza to be executed on YARN
> >>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
> >>>>>>> in-house
> >>>>> deployment
> >>>>>>> systems. This should make life a lot easier for new users.
> >>>>>>> Imagine
> >>>>> having
> >>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
> >>>> traffic
> >>>>>>> will be pretty dramatic.
> >>>>>>>
> >>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
> >>> everyone
> >>>>>>> that
> >>>>>>> I'm aware of is using Samza with Kafka. We basically require it
> >>>> already
> >>>>> in
> >>>>>>> order for most features to work. Those that are using other
> >>>>>>> systems
> >>>> are
> >>>>>>> generally using it for ingest into Kafka (1), and then they do
> >>>>>>> the processing on top. There is already discussion (
> >>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> >>> 767
> >>>>>>> )
> >>>>>>> in Kafka to make ingesting into Kafka extremely easy.
> >>>>>>>
> >>>>>>> Once we make the call to couple with Kafka, we can leverage a
> >>>>>>> ton of
> >>>>> their
> >>>>>>> ecosystem. We no longer have to maintain our own config,
> >>>>>>> metrics,
> >>> etc.
> >>>>> We
> >>>>>>> can all share the same libraries, and make them better. This
> >>>>>>> will
> >>> also
> >>>>>>> allow us to share the consumer/producer APIs, and will let us
> >>> leverage
> >>>>>>> their offset management and partition management, rather than
> >>>>>>> having
> >>>> our
> >>>>>>> own. All of the coordinator stream code would go away, as would
> >>>>>>> most
> >>>> of
> >>>>>>> the
> >>>>>>> YARN AppMaster code. We'd probably have to push some partition
> >>>>> management
> >>>>>>> features into the Kafka broker, but they're already moving in
> >>>>>>> that direction with the new consumer API. The features we have
> >>>>>>> for
> >>>> partition
> >>>>>>> assignment aren't unique to Samza, and seem like they should be
> >>>>>>> in
> >>>> Kafka
> >>>>>>> anyway. There will always be some niche usages which will
> >>>>>>> require
> >>>> extra
> >>>>>>> care and hence full control over partition assignments much
> >>>>>>> like the
> >>>>> Kafka
> >>>>>>> low level consumer api. These would continue to be supported.
> >>>>>>>
> >>>>>>> These items will be good for the Samza community. They'll make
> >>>>>>> Samza easier to use, and make it easier for developers to add
> >>>>>>> new features.
> >>>>>>>
> >>>>>>> Obviously this is a fairly large (and somewhat backwards
> >>> incompatible
> >>>>>>> change). If we choose to go this route, it's important that we
> >>> openly
> >>>>>>> communicate how we're going to provide a migration path from
> >>>>>>> the
> >>>>> existing
> >>>>>>> APIs to the new ones (if we make incompatible changes). I think
> >>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
> >>>>>>> existing StreamTask implementations to continue running on the
> >> new container.
> >>>>> It's
> >>>>>>> also important that we openly communicate about timing, and
> >>>>>>> stages
> >>> of
> >>>>> the
> >>>>>>> migration.
> >>>>>>>
> >>>>>>> If you made it this far, I'm sure you have opinions. :) Please
> >>>>>>> send
> >>>> your
> >>>>>>> thoughts and feedback.
> >>>>>>>
> >>>>>>> Cheers,
> >>>>>>> Chris
> >>>>
> >>>>
> >>>>
> >>>> --
> >>>> -- Guozhang
> >>
>

Re: Thoughts and obesrvations on Samza

Posted by Sriram <sr...@gmail.com>.
One thing that is worth exploring is to have a transformation and ingestion library in Kafka but use the same framework for fault tolerance, resource isolation and management. The biggest difference I see in these two use cases is the API and data model.


> On Jul 2, 2015, at 8:59 AM, Jay Kreps <ja...@confluent.io> wrote:
> 
> Hey Garry,
> 
> Yeah that's super frustrating. I'd be happy to chat more about this if
> you'd be interested. I think Chris and I started with the idea of "what
> would it take to make Samza a kick-ass ingestion tool" but ultimately we
> kind of came around to the idea that ingestion and transformation had
> pretty different needs and coupling the two made things hard.
> 
> For what it's worth I think copycat (KIP-26) actually will do what you are
> looking for.
> 
> With regard to your point about slider, I don't necessarily disagree. But I
> think getting good YARN support is quite doable and I think we can make
> that work well. I think the issue this proposal solves is that technically
> it is pretty hard to support multiple cluster management systems the way
> things are now, you need to write an "app master" or "framework" for each
> and they are all a little different so testing is really hard. In the
> absence of this we have been stuck with just YARN which has fantastic
> penetration in the Hadoopy part of the org, but zero penetration elsewhere.
> Given the huge amount of work being put in to slider, marathon, aws
> tooling, not to mention the umpteen related packaging technologies people
> want to use (Docker, Kubernetes, various cloud-specific deploy tools, etc)
> I really think it is important to get this right.
> 
> -Jay
> 
> On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> g.turkington@improvedigital.com> wrote:
> 
>> Hi all,
>> 
>> I think the question below re does Samza become a sub-project of Kafka
>> highlights the broader point around migration. Chris mentions Samza's
>> maturity is heading towards a v1 release but I'm not sure it feels right to
>> launch a v1 then immediately plan to deprecate most of it.
>> 
>> From a selfish perspective I have some guys who have started working with
>> Samza and building some new consumers/producers was next up. Sounds like
>> that is absolutely not the direction to go. I need to look into the KIP in
>> more detail but for me the attractiveness of adding new Samza
>> consumer/producers -- even if yes all they were doing was really getting
>> data into and out of Kafka --  was to avoid  having to worry about the
>> lifecycle management of external clients. If there is a generic Kafka
>> ingress/egress layer that I can plug a new connector into and have a lot of
>> the heavy lifting re scale and reliability done for me then it gives me all
>> the pushing new consumers/producers would. If not then it complicates my
>> operational deployments.
>> 
>> Which is similar to my other question with the proposal -- if we build a
>> fully available/stand-alone Samza plus the requisite shims to integrate
>> with Slider etc I suspect the former may be a lot more work than we think.
>> We may make it much easier for a newcomer to get something running but
>> having them step up and get a reliable production deployment may still
>> dominate mailing list  traffic, if for different reasons than today.
>> 
>> Don't get me wrong -- I'm comfortable with making the Samza dependency on
>> Kafka much more explicit and I absolutely see the benefits  in the
>> reduction of duplication and clashing terminologies/abstractions that
>> Chris/Jay describe. Samza as a library would likely be a very nice tool to
>> add to the Kafka ecosystem. I just have the concerns above re the
>> operational side.
>> 
>> Garry
>> 
>> -----Original Message-----
>> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
>> Sent: 02 July 2015 12:56
>> To: dev@samza.apache.org
>> Subject: Re: Thoughts and obesrvations on Samza
>> 
>> Very interesting thoughts.
>> From outside, I have always perceived Samza as a computing layer over
>> Kafka.
>> 
>> The question, maybe a bit provocative, is "should Samza be a sub-project
>> of Kafka then?"
>> Or does it make sense to keep it as a separate project with a separate
>> governance?
>> 
>> Cheers,
>> 
>> --
>> Gianmarco
>> 
>>> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
>>> 
>>> Overall, I agree to couple with Kafka more tightly. Because Samza de
>>> facto is based on Kafka, and it should leverage what Kafka has. At the
>>> same time, Kafka does not need to reinvent what Samza already has. I
>>> also like the idea of separating the ingestion and transformation.
>>> 
>>> But it is a little difficult for me to image how the Samza will look
>> like.
>>> And I feel Chris and Jay have a little difference in terms of how
>>> Samza should look like.
>>> 
>>> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
>>> user's application code calls this client?
>>> 
>>> 1. If we make Samza be a library of Kafka (like what the code shows),
>>> how do we implement auto-balance and fault-tolerance? Are they taken
>>> care by the Kafka broker or other mechanism, such as "Samza worker"
>>> (just make up the name) ?
>>> 
>>> 2. What about other features, such as auto-scaling, shared state,
>>> monitoring?
>>> 
>>> 
>>> *** If we have Samza standalone, (is this what Chris suggests?)
>>> 
>>> 1. we still need to ingest data from Kakfa and produce to it. Then it
>>> becomes the same as what Samza looks like now, except it does not rely
>>> on Yarn anymore.
>>> 
>>> 2. if it is standalone, how can it leverage Kafka's metrics, logs,
>>> etc? Use Kafka code as the dependency?
>>> 
>>> 
>>> Thanks,
>>> 
>>> Fang, Yan
>>> yanfang724@gmail.com
>>> 
>>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
>>> wrote:
>>> 
>>>> Read through the code example and it looks good to me. A few
>>>> thoughts regarding deployment:
>>>> 
>>>> Today Samza deploys as executable runnable like:
>>>> 
>>>> deploy/samza/bin/run-job.sh --config-factory=...
>> --config-path=file://...
>>>> 
>>>> And this proposal advocate for deploying Samza more as embedded
>>>> libraries in user application code (ignoring the terminology since
>>>> it is not the
>>> same
>>>> as the prototype code):
>>>> 
>>>> StreamTask task = new MyStreamTask(configs); Thread thread = new
>>>> Thread(task); thread.start();
>>>> 
>>>> I think both of these deployment modes are important for different
>>>> types
>>> of
>>>> users. That said, I think making Samza purely standalone is still
>>>> sufficient for either runnable or library modes.
>>>> 
>>>> Guozhang
>>>> 
>>>>> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
>>>>> 
>>>>> Looks like gmail mangled the code example, it was supposed to look
>>>>> like
>>>>> this:
>>>>> 
>>>>> Properties props = new Properties();
>>>>> props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
>>>>> config = new StreamingConfig(props);
>>>>> config.subscribe("test-topic-1", "test-topic-2");
>>>>> config.processor(ExampleStreamProcessor.class);
>>>>> config.serialization(new StringSerializer(), new
>>>>> StringDeserializer()); KafkaStreaming container = new
>>>>> KafkaStreaming(config); container.run();
>>>>> 
>>>>> -Jay
>>>>> 
>>>>> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
>> wrote:
>>>>> 
>>>>>> Hey guys,
>>>>>> 
>>>>>> This came out of some conversations Chris and I were having
>>>>>> around
>>>>> whether
>>>>>> it would make sense to use Samza as a kind of data ingestion
>>> framework
>>>>> for
>>>>>> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
>>>> combined
>>>>>> with complaints around config and YARN and the discussion around
>>>>>> how
>>> to
>>>>>> best do a standalone mode.
>>>>>> 
>>>>>> So the thought experiment was, given that Samza was basically
>>>>>> already totally Kafka specific, what if you just embraced that
>>>>>> and turned it
>>>> into
>>>>>> something less like a heavyweight framework and more like a
>>>>>> third
>>> Kafka
>>>>>> client--a kind of "producing consumer" with state management
>>>> facilities.
>>>>>> Basically a library. Instead of a complex stream processing
>>>>>> framework
>>>>> this
>>>>>> would actually be a very simple thing, not much more complicated
>>>>>> to
>>> use
>>>>> or
>>>>>> operate than a Kafka consumer. As Chris said we thought about it
>>>>>> a
>>> lot
>>>> of
>>>>>> what Samza (and the other stream processing systems were doing)
>>> seemed
>>>>> like
>>>>>> kind of a hangover from MapReduce.
>>>>>> 
>>>>>> Of course you need to ingest/output data to and from the stream
>>>>>> processing. But when we actually looked into how that would
>>>>>> work,
>>> Samza
>>>>>> isn't really an ideal data ingestion framework for a bunch of
>>> reasons.
>>>> To
>>>>>> really do that right you need a pretty different internal data
>>>>>> model
>>>> and
>>>>>> set of apis. So what if you split them and had an api for Kafka
>>>>>> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
>>>>>> transformation (Samza).
>>>>>> 
>>>>>> This would also allow really embracing the same terminology and
>>>>>> conventions. One complaint about the current state is that the
>>>>>> two
>>>>> systems
>>>>>> kind of feel bolted on. Terminology like "stream" vs "topic" and
>>>>> different
>>>>>> config and monitoring systems means you kind of have to learn
>>>>>> Kafka's
>>>>> way,
>>>>>> then learn Samza's slightly different way, then kind of
>>>>>> understand
>>> how
>>>>> they
>>>>>> map to each other, which having walked a few people through this
>>>>>> is surprisingly tricky for folks to get.
>>>>>> 
>>>>>> Since I have been spending a lot of time on airplanes I hacked
>>>>>> up an ernest but still somewhat incomplete prototype of what
>>>>>> this would
>>> look
>>>>>> like. This is just unceremoniously dumped into Kafka as it
>>>>>> required a
>>>> few
>>>>>> changes to the new consumer. Here is the code:
>>> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
>>> /apache/kafka/clients/streaming
>>>>>> 
>>>>>> For the purpose of the prototype I just liberally renamed
>>>>>> everything
>>> to
>>>>>> try to align it with Kafka with no regard for compatibility.
>>>>>> 
>>>>>> To use this would be something like this:
>>>>>> Properties props = new Properties();
>>>>>> props.put("bootstrap.servers", "localhost:4242");
>>>>>> StreamingConfig config = new
>>> StreamingConfig(props);
>>>>> config.subscribe("test-topic-1",
>>>>>> "test-topic-2"); config.processor(ExampleStreamProcessor.class);
>>>>> config.serialization(new
>>>>>> StringSerializer(), new StringDeserializer()); KafkaStreaming
>>>> container =
>>>>>> new KafkaStreaming(config); container.run();
>>>>>> 
>>>>>> KafkaStreaming is basically the SamzaContainer; StreamProcessor
>>>>>> is basically StreamTask.
>>>>>> 
>>>>>> So rather than putting all the class names in a file and then
>>>>>> having
>>>> the
>>>>>> job assembled by reflection, you just instantiate the container
>>>>>> programmatically. Work is balanced over however many instances
>>>>>> of
>>> this
>>>>> are
>>>>>> alive at any time (i.e. if an instance dies, new tasks are added
>>>>>> to
>>> the
>>>>>> existing containers without shutting them down).
>>>>>> 
>>>>>> We would provide some glue for running this stuff in YARN via
>>>>>> Slider, Mesos via Marathon, and AWS using some of their tools
>>>>>> but from the
>>>> point
>>>>> of
>>>>>> view of these frameworks these stream processing jobs are just
>>>> stateless
>>>>>> services that can come and go and expand and contract at will.
>>>>>> There
>>> is
>>>>> no
>>>>>> more custom scheduler.
>>>>>> 
>>>>>> Here are some relevant details:
>>>>>> 
>>>>>>   1. It is only ~1300 lines of code, it would get larger if we
>>>>>>   productionized but not vastly larger. We really do get a ton
>>>>>> of
>>>>> leverage
>>>>>>   out of Kafka.
>>>>>>   2. Partition management is fully delegated to the new consumer.
>>> This
>>>>>>   is nice since now any partition management strategy available
>>>>>> to
>>>> Kafka
>>>>>>   consumer is also available to Samza (and vice versa) and with
>>>>>> the
>>>>> exact
>>>>>>   same configs.
>>>>>>   3. It supports state as well as state reuse
>>>>>> 
>>>>>> Anyhow take a look, hopefully it is thought provoking.
>>>>>> 
>>>>>> -Jay
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
>>>> criccomini@apache.org>
>>>>>> wrote:
>>>>>> 
>>>>>>> Hey all,
>>>>>>> 
>>>>>>> I have had some discussions with Samza engineers at LinkedIn
>>>>>>> and
>>>>> Confluent
>>>>>>> and we came up with a few observations and would like to
>>>>>>> propose
>>> some
>>>>>>> changes.
>>>>>>> 
>>>>>>> We've observed some things that I want to call out about
>>>>>>> Samza's
>>>> design,
>>>>>>> and I'd like to propose some changes.
>>>>>>> 
>>>>>>> * Samza is dependent upon a dynamic deployment system.
>>>>>>> * Samza is too pluggable.
>>>>>>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
>>>>>>> APIs
>>> are
>>>>>>> trying to solve a lot of the same problems.
>>>>>>> 
>>>>>>> All three of these issues are related, but I'll address them in
>>> order.
>>>>>>> 
>>>>>>> Deployment
>>>>>>> 
>>>>>>> Samza strongly depends on the use of a dynamic deployment
>>>>>>> scheduler
>>>> such
>>>>>>> as
>>>>>>> YARN, Mesos, etc. When we initially built Samza, we bet that
>>>>>>> there
>>>> would
>>>>>>> be
>>>>>>> one or two winners in this area, and we could support them, and
>>>>>>> the
>>>> rest
>>>>>>> would go away. In reality, there are many variations.
>>>>>>> Furthermore,
>>>> many
>>>>>>> people still prefer to just start their processors like normal
>>>>>>> Java processes, and use traditional deployment scripts such as
>>>>>>> Fabric,
>>>> Chef,
>>>>>>> Ansible, etc. Forcing a deployment system on users makes the
>>>>>>> Samza start-up process really painful for first time users.
>>>>>>> 
>>>>>>> Dynamic deployment as a requirement was also a bit of a
>>>>>>> mis-fire
>>>> because
>>>>>>> of
>>>>>>> a fundamental misunderstanding between the nature of batch jobs
>>>>>>> and
>>>>> stream
>>>>>>> processing jobs. Early on, we made conscious effort to favor
>>>>>>> the
>>>> Hadoop
>>>>>>> (Map/Reduce) way of doing things, since it worked and was well
>>>>> understood.
>>>>>>> One thing that we missed was that batch jobs have a definite
>>>> beginning,
>>>>>>> and
>>>>>>> end, and stream processing jobs don't (usually). This leads to
>>>>>>> a
>>> much
>>>>>>> simpler scheduling problem for stream processors. You basically
>>>>>>> just
>>>>> need
>>>>>>> to find a place to start the processor, and start it. The way
>>>>>>> we run grids, at LinkedIn, there's no concept of a cluster
>>>>>>> being "full". We always
>>>> add
>>>>>>> more machines. The problem with coupling Samza with a scheduler
>>>>>>> is
>>>> that
>>>>>>> Samza (as a framework) now has to handle deployment. This pulls
>>>>>>> in a
>>>>> bunch
>>>>>>> of things such as configuration distribution (config stream),
>>>>>>> shell
>>>>> scrips
>>>>>>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
>>>>>>> 
>>>>>>> Another reason for requiring dynamic deployment was to support
>>>>>>> data locality. If you want to have locality, you need to put
>>>>>>> your
>>>> processors
>>>>>>> close to the data they're processing. Upon further
>>>>>>> investigation,
>>>>> though,
>>>>>>> this feature is not that beneficial. There is some good
>>>>>>> discussion
>>>> about
>>>>>>> some problems with it on SAMZA-335. Again, we took the
>>>>>>> Map/Reduce
>>>> path,
>>>>>>> but
>>>>>>> there are some fundamental differences between HDFS and Kafka.
>>>>>>> HDFS
>>>> has
>>>>>>> blocks, while Kafka has partitions. This leads to less
>>>>>>> optimization potential with stream processors on top of Kafka.
>>>>>>> 
>>>>>>> This feature is also used as a crutch. Samza doesn't have any
>>>>>>> built
>>> in
>>>>>>> fault-tolerance logic. Instead, it depends on the dynamic
>>>>>>> deployment scheduling system to handle restarts when a
>>>>>>> processor dies. This has
>>>>> made
>>>>>>> it very difficult to write a standalone Samza container
>> (SAMZA-516).
>>>>>>> 
>>>>>>> Pluggability
>>>>>>> 
>>>>>>> In some cases pluggability is good, but I think that we've gone
>>>>>>> too
>>>> far
>>>>>>> with it. Currently, Samza has:
>>>>>>> 
>>>>>>> * Pluggable config.
>>>>>>> * Pluggable metrics.
>>>>>>> * Pluggable deployment systems.
>>>>>>> * Pluggable streaming systems (SystemConsumer, SystemProducer,
>> etc).
>>>>>>> * Pluggable serdes.
>>>>>>> * Pluggable storage engines.
>>>>>>> * Pluggable strategies for just about every component
>>> (MessageChooser,
>>>>>>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>>>>>>> 
>>>>>>> There's probably more that I've forgotten, as well. Some of
>>>>>>> these
>>> are
>>>>>>> useful, but some have proven not to be. This all comes at a cost:
>>>>>>> complexity. This complexity is making it harder for our users
>>>>>>> to
>>> pick
>>>> up
>>>>>>> and use Samza out of the box. It also makes it difficult for
>>>>>>> Samza developers to reason about what the characteristics of
>>>>>>> the container (since the characteristics change depending on
>>>>>>> which plugins are use).
>>>>>>> 
>>>>>>> The issues with pluggability are most visible in the System APIs.
>>> What
>>>>>>> Samza really requires to be functional is Kafka as its
>>>>>>> transport
>>>> layer.
>>>>>>> But
>>>>>>> we've conflated two unrelated use cases into one API:
>>>>>>> 
>>>>>>> 1. Get data into/out of Kafka.
>>>>>>> 2. Process the data in Kafka.
>>>>>>> 
>>>>>>> The current System API supports both of these use cases. The
>>>>>>> problem
>>>> is,
>>>>>>> we
>>>>>>> actually want different features for each use case. By papering
>>>>>>> over
>>>>> these
>>>>>>> two use cases, and providing a single API, we've introduced a
>>>>>>> ton of
>>>>> leaky
>>>>>>> abstractions.
>>>>>>> 
>>>>>>> For example, what we'd really like in (2) is to have
>>>>>>> monotonically increasing longs for offsets (like Kafka). This
>>>>>>> would be at odds
>>> with
>>>>> (1),
>>>>>>> though, since different systems have different
>>>>> SCNs/Offsets/UUIDs/vectors.
>>>>>>> There was discussion both on the mailing list and the SQL JIRAs
>>> about
>>>>> the
>>>>>>> need for this.
>>>>>>> 
>>>>>>> The same thing holds true for replayability. Kafka allows us to
>>> rewind
>>>>>>> when
>>>>>>> we have a failure. Many other systems don't. In some cases,
>>>>>>> systems
>>>>> return
>>>>>>> null for their offsets (e.g. WikipediaSystemConsumer) because
>>>>>>> they
>>>> have
>>>>> no
>>>>>>> offsets.
>>>>>>> 
>>>>>>> Partitioning is another example. Kafka supports partitioning,
>>>>>>> but
>>> many
>>>>>>> systems don't. We model this by having a single partition for
>>>>>>> those systems. Still, other systems model partitioning
>> differently (e.g.
>>>>>>> Kinesis).
>>>>>>> 
>>>>>>> The SystemAdmin interface is also a mess. Creating streams in a
>>>>>>> system-agnostic way is almost impossible. As is modeling
>>>>>>> metadata
>>> for
>>>>> the
>>>>>>> system (replication factor, partitions, location, etc). The
>>>>>>> list
>>> goes
>>>>> on.
>>>>>>> 
>>>>>>> Duplicate work
>>>>>>> 
>>>>>>> At the time that we began writing Samza, Kafka's consumer and
>>> producer
>>>>>>> APIs
>>>>>>> had a relatively weak feature set. On the consumer-side, you
>>>>>>> had two
>>>>>>> options: use the high level consumer, or the simple consumer.
>>>>>>> The
>>>>> problem
>>>>>>> with the high-level consumer was that it controlled your
>>>>>>> offsets, partition assignments, and the order in which you
>>>>>>> received messages. The
>>> problem
>>>>>>> with
>>>>>>> the simple consumer is that it's not simple. It's basic. You
>>>>>>> end up
>>>>> having
>>>>>>> to handle a lot of really low-level stuff that you shouldn't.
>>>>>>> We
>>>> spent a
>>>>>>> lot of time to make Samza's KafkaSystemConsumer very robust. It
>>>>>>> also allows us to support some cool features:
>>>>>>> 
>>>>>>> * Per-partition message ordering and prioritization.
>>>>>>> * Tight control over partition assignment to support joins,
>>>>>>> global
>>>> state
>>>>>>> (if we want to implement it :)), etc.
>>>>>>> * Tight control over offset checkpointing.
>>>>>>> 
>>>>>>> What we didn't realize at the time is that these features
>>>>>>> should
>>>>> actually
>>>>>>> be in Kafka. A lot of Kafka consumers (not just Samza stream
>>>> processors)
>>>>>>> end up wanting to do things like joins and partition
>>>>>>> assignment. The
>>>>> Kafka
>>>>>>> community has come to the same conclusion. They're adding a ton
>>>>>>> of upgrades into their new Kafka consumer implementation. To a
>>>>>>> large extent,
>>> it's
>>>>>>> duplicate work to what we've already done in Samza.
>>>>>>> 
>>>>>>> On top of this, Kafka ended up taking a very similar approach
>>>>>>> to
>>>> Samza's
>>>>>>> KafkaCheckpointManager implementation for handling offset
>>>> checkpointing.
>>>>>>> Like Samza, Kafka's new offset management feature stores offset
>>>>>>> checkpoints in a topic, and allows you to fetch them from the
>>>>>>> broker.
>>>>>>> 
>>>>>>> A lot of this seems like a waste, since we could have shared
>>>>>>> the
>>> work
>>>> if
>>>>>>> it
>>>>>>> had been done in Kafka from the get-go.
>>>>>>> 
>>>>>>> Vision
>>>>>>> 
>>>>>>> All of this leads me to a rather radical proposal. Samza is
>>> relatively
>>>>>>> stable at this point. I'd venture to say that we're near a 1.0
>>>> release.
>>>>>>> I'd
>>>>>>> like to propose that we take what we've learned, and begin
>>>>>>> thinking
>>>>> about
>>>>>>> Samza beyond 1.0. What would we change if we were starting from
>>>> scratch?
>>>>>>> My
>>>>>>> proposal is to:
>>>>>>> 
>>>>>>> 1. Make Samza standalone the *only* way to run Samza
>>>>>>> processors, and eliminate all direct dependences on YARN, Mesos,
>> etc.
>>>>>>> 2. Make a definitive call to support only Kafka as the stream
>>>> processing
>>>>>>> layer.
>>>>>>> 3. Eliminate Samza's metrics, logging, serialization, and
>>>>>>> config
>>>>> systems,
>>>>>>> and simply use Kafka's instead.
>>>>>>> 
>>>>>>> This would fix all of the issues that I outlined above. It
>>>>>>> should
>>> also
>>>>>>> shrink the Samza code base pretty dramatically. Supporting only
>>>>>>> a standalone container will allow Samza to be executed on YARN
>>>>>>> (using Slider), Mesos (using Marathon/Aurora), or most other
>>>>>>> in-house
>>>>> deployment
>>>>>>> systems. This should make life a lot easier for new users.
>>>>>>> Imagine
>>>>> having
>>>>>>> the hello-samza tutorial without YARN. The drop in mailing list
>>>> traffic
>>>>>>> will be pretty dramatic.
>>>>>>> 
>>>>>>> Coupling with Kafka seems long overdue to me. The reality is,
>>> everyone
>>>>>>> that
>>>>>>> I'm aware of is using Samza with Kafka. We basically require it
>>>> already
>>>>> in
>>>>>>> order for most features to work. Those that are using other
>>>>>>> systems
>>>> are
>>>>>>> generally using it for ingest into Kafka (1), and then they do
>>>>>>> the processing on top. There is already discussion (
>>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
>>> 767
>>>>>>> )
>>>>>>> in Kafka to make ingesting into Kafka extremely easy.
>>>>>>> 
>>>>>>> Once we make the call to couple with Kafka, we can leverage a
>>>>>>> ton of
>>>>> their
>>>>>>> ecosystem. We no longer have to maintain our own config,
>>>>>>> metrics,
>>> etc.
>>>>> We
>>>>>>> can all share the same libraries, and make them better. This
>>>>>>> will
>>> also
>>>>>>> allow us to share the consumer/producer APIs, and will let us
>>> leverage
>>>>>>> their offset management and partition management, rather than
>>>>>>> having
>>>> our
>>>>>>> own. All of the coordinator stream code would go away, as would
>>>>>>> most
>>>> of
>>>>>>> the
>>>>>>> YARN AppMaster code. We'd probably have to push some partition
>>>>> management
>>>>>>> features into the Kafka broker, but they're already moving in
>>>>>>> that direction with the new consumer API. The features we have
>>>>>>> for
>>>> partition
>>>>>>> assignment aren't unique to Samza, and seem like they should be
>>>>>>> in
>>>> Kafka
>>>>>>> anyway. There will always be some niche usages which will
>>>>>>> require
>>>> extra
>>>>>>> care and hence full control over partition assignments much
>>>>>>> like the
>>>>> Kafka
>>>>>>> low level consumer api. These would continue to be supported.
>>>>>>> 
>>>>>>> These items will be good for the Samza community. They'll make
>>>>>>> Samza easier to use, and make it easier for developers to add
>>>>>>> new features.
>>>>>>> 
>>>>>>> Obviously this is a fairly large (and somewhat backwards
>>> incompatible
>>>>>>> change). If we choose to go this route, it's important that we
>>> openly
>>>>>>> communicate how we're going to provide a migration path from
>>>>>>> the
>>>>> existing
>>>>>>> APIs to the new ones (if we make incompatible changes). I think
>>>>>>> at a minimum, we'd probably need to provide a wrapper to allow
>>>>>>> existing StreamTask implementations to continue running on the
>> new container.
>>>>> It's
>>>>>>> also important that we openly communicate about timing, and
>>>>>>> stages
>>> of
>>>>> the
>>>>>>> migration.
>>>>>>> 
>>>>>>> If you made it this far, I'm sure you have opinions. :) Please
>>>>>>> send
>>>> your
>>>>>>> thoughts and feedback.
>>>>>>> 
>>>>>>> Cheers,
>>>>>>> Chris
>>>> 
>>>> 
>>>> 
>>>> --
>>>> -- Guozhang
>> 

Re: Thoughts and obesrvations on Samza

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

Yeah that's super frustrating. I'd be happy to chat more about this if
you'd be interested. I think Chris and I started with the idea of "what
would it take to make Samza a kick-ass ingestion tool" but ultimately we
kind of came around to the idea that ingestion and transformation had
pretty different needs and coupling the two made things hard.

For what it's worth I think copycat (KIP-26) actually will do what you are
looking for.

With regard to your point about slider, I don't necessarily disagree. But I
think getting good YARN support is quite doable and I think we can make
that work well. I think the issue this proposal solves is that technically
it is pretty hard to support multiple cluster management systems the way
things are now, you need to write an "app master" or "framework" for each
and they are all a little different so testing is really hard. In the
absence of this we have been stuck with just YARN which has fantastic
penetration in the Hadoopy part of the org, but zero penetration elsewhere.
Given the huge amount of work being put in to slider, marathon, aws
tooling, not to mention the umpteen related packaging technologies people
want to use (Docker, Kubernetes, various cloud-specific deploy tools, etc)
I really think it is important to get this right.

-Jay

On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
g.turkington@improvedigital.com> wrote:

> Hi all,
>
> I think the question below re does Samza become a sub-project of Kafka
> highlights the broader point around migration. Chris mentions Samza's
> maturity is heading towards a v1 release but I'm not sure it feels right to
> launch a v1 then immediately plan to deprecate most of it.
>
> From a selfish perspective I have some guys who have started working with
> Samza and building some new consumers/producers was next up. Sounds like
> that is absolutely not the direction to go. I need to look into the KIP in
> more detail but for me the attractiveness of adding new Samza
> consumer/producers -- even if yes all they were doing was really getting
> data into and out of Kafka --  was to avoid  having to worry about the
> lifecycle management of external clients. If there is a generic Kafka
> ingress/egress layer that I can plug a new connector into and have a lot of
> the heavy lifting re scale and reliability done for me then it gives me all
> the pushing new consumers/producers would. If not then it complicates my
> operational deployments.
>
> Which is similar to my other question with the proposal -- if we build a
> fully available/stand-alone Samza plus the requisite shims to integrate
> with Slider etc I suspect the former may be a lot more work than we think.
> We may make it much easier for a newcomer to get something running but
> having them step up and get a reliable production deployment may still
> dominate mailing list  traffic, if for different reasons than today.
>
> Don't get me wrong -- I'm comfortable with making the Samza dependency on
> Kafka much more explicit and I absolutely see the benefits  in the
> reduction of duplication and clashing terminologies/abstractions that
> Chris/Jay describe. Samza as a library would likely be a very nice tool to
> add to the Kafka ecosystem. I just have the concerns above re the
> operational side.
>
> Garry
>
> -----Original Message-----
> From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org]
> Sent: 02 July 2015 12:56
> To: dev@samza.apache.org
> Subject: Re: Thoughts and obesrvations on Samza
>
> Very interesting thoughts.
> From outside, I have always perceived Samza as a computing layer over
> Kafka.
>
> The question, maybe a bit provocative, is "should Samza be a sub-project
> of Kafka then?"
> Or does it make sense to keep it as a separate project with a separate
> governance?
>
> Cheers,
>
> --
> Gianmarco
>
> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
>
> > Overall, I agree to couple with Kafka more tightly. Because Samza de
> > facto is based on Kafka, and it should leverage what Kafka has. At the
> > same time, Kafka does not need to reinvent what Samza already has. I
> > also like the idea of separating the ingestion and transformation.
> >
> > But it is a little difficult for me to image how the Samza will look
> like.
> > And I feel Chris and Jay have a little difference in terms of how
> > Samza should look like.
> >
> > *** Will it look like what Jay's code shows (A client of Kakfa) ? And
> > user's application code calls this client?
> >
> > 1. If we make Samza be a library of Kafka (like what the code shows),
> > how do we implement auto-balance and fault-tolerance? Are they taken
> > care by the Kafka broker or other mechanism, such as "Samza worker"
> > (just make up the name) ?
> >
> > 2. What about other features, such as auto-scaling, shared state,
> > monitoring?
> >
> >
> > *** If we have Samza standalone, (is this what Chris suggests?)
> >
> > 1. we still need to ingest data from Kakfa and produce to it. Then it
> > becomes the same as what Samza looks like now, except it does not rely
> > on Yarn anymore.
> >
> > 2. if it is standalone, how can it leverage Kafka's metrics, logs,
> > etc? Use Kafka code as the dependency?
> >
> >
> > Thanks,
> >
> > Fang, Yan
> > yanfang724@gmail.com
> >
> > On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Read through the code example and it looks good to me. A few
> > > thoughts regarding deployment:
> > >
> > > Today Samza deploys as executable runnable like:
> > >
> > > deploy/samza/bin/run-job.sh --config-factory=...
> --config-path=file://...
> > >
> > > And this proposal advocate for deploying Samza more as embedded
> > > libraries in user application code (ignoring the terminology since
> > > it is not the
> > same
> > > as the prototype code):
> > >
> > > StreamTask task = new MyStreamTask(configs); Thread thread = new
> > > Thread(task); thread.start();
> > >
> > > I think both of these deployment modes are important for different
> > > types
> > of
> > > users. That said, I think making Samza purely standalone is still
> > > sufficient for either runnable or library modes.
> > >
> > > Guozhang
> > >
> > > On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Looks like gmail mangled the code example, it was supposed to look
> > > > like
> > > > this:
> > > >
> > > > Properties props = new Properties();
> > > > props.put("bootstrap.servers", "localhost:4242"); StreamingConfig
> > > > config = new StreamingConfig(props);
> > > > config.subscribe("test-topic-1", "test-topic-2");
> > > > config.processor(ExampleStreamProcessor.class);
> > > > config.serialization(new StringSerializer(), new
> > > > StringDeserializer()); KafkaStreaming container = new
> > > > KafkaStreaming(config); container.run();
> > > >
> > > > -Jay
> > > >
> > > > On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > > >
> > > > > Hey guys,
> > > > >
> > > > > This came out of some conversations Chris and I were having
> > > > > around
> > > > whether
> > > > > it would make sense to use Samza as a kind of data ingestion
> > framework
> > > > for
> > > > > Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> > > combined
> > > > > with complaints around config and YARN and the discussion around
> > > > > how
> > to
> > > > > best do a standalone mode.
> > > > >
> > > > > So the thought experiment was, given that Samza was basically
> > > > > already totally Kafka specific, what if you just embraced that
> > > > > and turned it
> > > into
> > > > > something less like a heavyweight framework and more like a
> > > > > third
> > Kafka
> > > > > client--a kind of "producing consumer" with state management
> > > facilities.
> > > > > Basically a library. Instead of a complex stream processing
> > > > > framework
> > > > this
> > > > > would actually be a very simple thing, not much more complicated
> > > > > to
> > use
> > > > or
> > > > > operate than a Kafka consumer. As Chris said we thought about it
> > > > > a
> > lot
> > > of
> > > > > what Samza (and the other stream processing systems were doing)
> > seemed
> > > > like
> > > > > kind of a hangover from MapReduce.
> > > > >
> > > > > Of course you need to ingest/output data to and from the stream
> > > > > processing. But when we actually looked into how that would
> > > > > work,
> > Samza
> > > > > isn't really an ideal data ingestion framework for a bunch of
> > reasons.
> > > To
> > > > > really do that right you need a pretty different internal data
> > > > > model
> > > and
> > > > > set of apis. So what if you split them and had an api for Kafka
> > > > > ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > > > > transformation (Samza).
> > > > >
> > > > > This would also allow really embracing the same terminology and
> > > > > conventions. One complaint about the current state is that the
> > > > > two
> > > > systems
> > > > > kind of feel bolted on. Terminology like "stream" vs "topic" and
> > > > different
> > > > > config and monitoring systems means you kind of have to learn
> > > > > Kafka's
> > > > way,
> > > > > then learn Samza's slightly different way, then kind of
> > > > > understand
> > how
> > > > they
> > > > > map to each other, which having walked a few people through this
> > > > > is surprisingly tricky for folks to get.
> > > > >
> > > > > Since I have been spending a lot of time on airplanes I hacked
> > > > > up an ernest but still somewhat incomplete prototype of what
> > > > > this would
> > look
> > > > > like. This is just unceremoniously dumped into Kafka as it
> > > > > required a
> > > few
> > > > > changes to the new consumer. Here is the code:
> > > > >
> > > > >
> > > >
> > >
> > https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> > /apache/kafka/clients/streaming
> > > > >
> > > > > For the purpose of the prototype I just liberally renamed
> > > > > everything
> > to
> > > > > try to align it with Kafka with no regard for compatibility.
> > > > >
> > > > > To use this would be something like this:
> > > > > Properties props = new Properties();
> > > > > props.put("bootstrap.servers", "localhost:4242");
> > > > > StreamingConfig config = new
> > StreamingConfig(props);
> > > > config.subscribe("test-topic-1",
> > > > > "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > > > config.serialization(new
> > > > > StringSerializer(), new StringDeserializer()); KafkaStreaming
> > > container =
> > > > > new KafkaStreaming(config); container.run();
> > > > >
> > > > > KafkaStreaming is basically the SamzaContainer; StreamProcessor
> > > > > is basically StreamTask.
> > > > >
> > > > > So rather than putting all the class names in a file and then
> > > > > having
> > > the
> > > > > job assembled by reflection, you just instantiate the container
> > > > > programmatically. Work is balanced over however many instances
> > > > > of
> > this
> > > > are
> > > > > alive at any time (i.e. if an instance dies, new tasks are added
> > > > > to
> > the
> > > > > existing containers without shutting them down).
> > > > >
> > > > > We would provide some glue for running this stuff in YARN via
> > > > > Slider, Mesos via Marathon, and AWS using some of their tools
> > > > > but from the
> > > point
> > > > of
> > > > > view of these frameworks these stream processing jobs are just
> > > stateless
> > > > > services that can come and go and expand and contract at will.
> > > > > There
> > is
> > > > no
> > > > > more custom scheduler.
> > > > >
> > > > > Here are some relevant details:
> > > > >
> > > > >    1. It is only ~1300 lines of code, it would get larger if we
> > > > >    productionized but not vastly larger. We really do get a ton
> > > > > of
> > > > leverage
> > > > >    out of Kafka.
> > > > >    2. Partition management is fully delegated to the new consumer.
> > This
> > > > >    is nice since now any partition management strategy available
> > > > > to
> > > Kafka
> > > > >    consumer is also available to Samza (and vice versa) and with
> > > > > the
> > > > exact
> > > > >    same configs.
> > > > >    3. It supports state as well as state reuse
> > > > >
> > > > > Anyhow take a look, hopefully it is thought provoking.
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > criccomini@apache.org>
> > > > > wrote:
> > > > >
> > > > >> Hey all,
> > > > >>
> > > > >> I have had some discussions with Samza engineers at LinkedIn
> > > > >> and
> > > > Confluent
> > > > >> and we came up with a few observations and would like to
> > > > >> propose
> > some
> > > > >> changes.
> > > > >>
> > > > >> We've observed some things that I want to call out about
> > > > >> Samza's
> > > design,
> > > > >> and I'd like to propose some changes.
> > > > >>
> > > > >> * Samza is dependent upon a dynamic deployment system.
> > > > >> * Samza is too pluggable.
> > > > >> * Samza's SystemConsumer/SystemProducer and Kafka's consumer
> > > > >> APIs
> > are
> > > > >> trying to solve a lot of the same problems.
> > > > >>
> > > > >> All three of these issues are related, but I'll address them in
> > order.
> > > > >>
> > > > >> Deployment
> > > > >>
> > > > >> Samza strongly depends on the use of a dynamic deployment
> > > > >> scheduler
> > > such
> > > > >> as
> > > > >> YARN, Mesos, etc. When we initially built Samza, we bet that
> > > > >> there
> > > would
> > > > >> be
> > > > >> one or two winners in this area, and we could support them, and
> > > > >> the
> > > rest
> > > > >> would go away. In reality, there are many variations.
> > > > >> Furthermore,
> > > many
> > > > >> people still prefer to just start their processors like normal
> > > > >> Java processes, and use traditional deployment scripts such as
> > > > >> Fabric,
> > > Chef,
> > > > >> Ansible, etc. Forcing a deployment system on users makes the
> > > > >> Samza start-up process really painful for first time users.
> > > > >>
> > > > >> Dynamic deployment as a requirement was also a bit of a
> > > > >> mis-fire
> > > because
> > > > >> of
> > > > >> a fundamental misunderstanding between the nature of batch jobs
> > > > >> and
> > > > stream
> > > > >> processing jobs. Early on, we made conscious effort to favor
> > > > >> the
> > > Hadoop
> > > > >> (Map/Reduce) way of doing things, since it worked and was well
> > > > understood.
> > > > >> One thing that we missed was that batch jobs have a definite
> > > beginning,
> > > > >> and
> > > > >> end, and stream processing jobs don't (usually). This leads to
> > > > >> a
> > much
> > > > >> simpler scheduling problem for stream processors. You basically
> > > > >> just
> > > > need
> > > > >> to find a place to start the processor, and start it. The way
> > > > >> we run grids, at LinkedIn, there's no concept of a cluster
> > > > >> being "full". We always
> > > add
> > > > >> more machines. The problem with coupling Samza with a scheduler
> > > > >> is
> > > that
> > > > >> Samza (as a framework) now has to handle deployment. This pulls
> > > > >> in a
> > > > bunch
> > > > >> of things such as configuration distribution (config stream),
> > > > >> shell
> > > > scrips
> > > > >> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> > > > >>
> > > > >> Another reason for requiring dynamic deployment was to support
> > > > >> data locality. If you want to have locality, you need to put
> > > > >> your
> > > processors
> > > > >> close to the data they're processing. Upon further
> > > > >> investigation,
> > > > though,
> > > > >> this feature is not that beneficial. There is some good
> > > > >> discussion
> > > about
> > > > >> some problems with it on SAMZA-335. Again, we took the
> > > > >> Map/Reduce
> > > path,
> > > > >> but
> > > > >> there are some fundamental differences between HDFS and Kafka.
> > > > >> HDFS
> > > has
> > > > >> blocks, while Kafka has partitions. This leads to less
> > > > >> optimization potential with stream processors on top of Kafka.
> > > > >>
> > > > >> This feature is also used as a crutch. Samza doesn't have any
> > > > >> built
> > in
> > > > >> fault-tolerance logic. Instead, it depends on the dynamic
> > > > >> deployment scheduling system to handle restarts when a
> > > > >> processor dies. This has
> > > > made
> > > > >> it very difficult to write a standalone Samza container
> (SAMZA-516).
> > > > >>
> > > > >> Pluggability
> > > > >>
> > > > >> In some cases pluggability is good, but I think that we've gone
> > > > >> too
> > > far
> > > > >> with it. Currently, Samza has:
> > > > >>
> > > > >> * Pluggable config.
> > > > >> * Pluggable metrics.
> > > > >> * Pluggable deployment systems.
> > > > >> * Pluggable streaming systems (SystemConsumer, SystemProducer,
> etc).
> > > > >> * Pluggable serdes.
> > > > >> * Pluggable storage engines.
> > > > >> * Pluggable strategies for just about every component
> > (MessageChooser,
> > > > >> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > > >>
> > > > >> There's probably more that I've forgotten, as well. Some of
> > > > >> these
> > are
> > > > >> useful, but some have proven not to be. This all comes at a cost:
> > > > >> complexity. This complexity is making it harder for our users
> > > > >> to
> > pick
> > > up
> > > > >> and use Samza out of the box. It also makes it difficult for
> > > > >> Samza developers to reason about what the characteristics of
> > > > >> the container (since the characteristics change depending on
> > > > >> which plugins are use).
> > > > >>
> > > > >> The issues with pluggability are most visible in the System APIs.
> > What
> > > > >> Samza really requires to be functional is Kafka as its
> > > > >> transport
> > > layer.
> > > > >> But
> > > > >> we've conflated two unrelated use cases into one API:
> > > > >>
> > > > >> 1. Get data into/out of Kafka.
> > > > >> 2. Process the data in Kafka.
> > > > >>
> > > > >> The current System API supports both of these use cases. The
> > > > >> problem
> > > is,
> > > > >> we
> > > > >> actually want different features for each use case. By papering
> > > > >> over
> > > > these
> > > > >> two use cases, and providing a single API, we've introduced a
> > > > >> ton of
> > > > leaky
> > > > >> abstractions.
> > > > >>
> > > > >> For example, what we'd really like in (2) is to have
> > > > >> monotonically increasing longs for offsets (like Kafka). This
> > > > >> would be at odds
> > with
> > > > (1),
> > > > >> though, since different systems have different
> > > > SCNs/Offsets/UUIDs/vectors.
> > > > >> There was discussion both on the mailing list and the SQL JIRAs
> > about
> > > > the
> > > > >> need for this.
> > > > >>
> > > > >> The same thing holds true for replayability. Kafka allows us to
> > rewind
> > > > >> when
> > > > >> we have a failure. Many other systems don't. In some cases,
> > > > >> systems
> > > > return
> > > > >> null for their offsets (e.g. WikipediaSystemConsumer) because
> > > > >> they
> > > have
> > > > no
> > > > >> offsets.
> > > > >>
> > > > >> Partitioning is another example. Kafka supports partitioning,
> > > > >> but
> > many
> > > > >> systems don't. We model this by having a single partition for
> > > > >> those systems. Still, other systems model partitioning
> differently (e.g.
> > > > >> Kinesis).
> > > > >>
> > > > >> The SystemAdmin interface is also a mess. Creating streams in a
> > > > >> system-agnostic way is almost impossible. As is modeling
> > > > >> metadata
> > for
> > > > the
> > > > >> system (replication factor, partitions, location, etc). The
> > > > >> list
> > goes
> > > > on.
> > > > >>
> > > > >> Duplicate work
> > > > >>
> > > > >> At the time that we began writing Samza, Kafka's consumer and
> > producer
> > > > >> APIs
> > > > >> had a relatively weak feature set. On the consumer-side, you
> > > > >> had two
> > > > >> options: use the high level consumer, or the simple consumer.
> > > > >> The
> > > > problem
> > > > >> with the high-level consumer was that it controlled your
> > > > >> offsets, partition assignments, and the order in which you
> > > > >> received messages. The
> > problem
> > > > >> with
> > > > >> the simple consumer is that it's not simple. It's basic. You
> > > > >> end up
> > > > having
> > > > >> to handle a lot of really low-level stuff that you shouldn't.
> > > > >> We
> > > spent a
> > > > >> lot of time to make Samza's KafkaSystemConsumer very robust. It
> > > > >> also allows us to support some cool features:
> > > > >>
> > > > >> * Per-partition message ordering and prioritization.
> > > > >> * Tight control over partition assignment to support joins,
> > > > >> global
> > > state
> > > > >> (if we want to implement it :)), etc.
> > > > >> * Tight control over offset checkpointing.
> > > > >>
> > > > >> What we didn't realize at the time is that these features
> > > > >> should
> > > > actually
> > > > >> be in Kafka. A lot of Kafka consumers (not just Samza stream
> > > processors)
> > > > >> end up wanting to do things like joins and partition
> > > > >> assignment. The
> > > > Kafka
> > > > >> community has come to the same conclusion. They're adding a ton
> > > > >> of upgrades into their new Kafka consumer implementation. To a
> > > > >> large extent,
> > it's
> > > > >> duplicate work to what we've already done in Samza.
> > > > >>
> > > > >> On top of this, Kafka ended up taking a very similar approach
> > > > >> to
> > > Samza's
> > > > >> KafkaCheckpointManager implementation for handling offset
> > > checkpointing.
> > > > >> Like Samza, Kafka's new offset management feature stores offset
> > > > >> checkpoints in a topic, and allows you to fetch them from the
> > > > >> broker.
> > > > >>
> > > > >> A lot of this seems like a waste, since we could have shared
> > > > >> the
> > work
> > > if
> > > > >> it
> > > > >> had been done in Kafka from the get-go.
> > > > >>
> > > > >> Vision
> > > > >>
> > > > >> All of this leads me to a rather radical proposal. Samza is
> > relatively
> > > > >> stable at this point. I'd venture to say that we're near a 1.0
> > > release.
> > > > >> I'd
> > > > >> like to propose that we take what we've learned, and begin
> > > > >> thinking
> > > > about
> > > > >> Samza beyond 1.0. What would we change if we were starting from
> > > scratch?
> > > > >> My
> > > > >> proposal is to:
> > > > >>
> > > > >> 1. Make Samza standalone the *only* way to run Samza
> > > > >> processors, and eliminate all direct dependences on YARN, Mesos,
> etc.
> > > > >> 2. Make a definitive call to support only Kafka as the stream
> > > processing
> > > > >> layer.
> > > > >> 3. Eliminate Samza's metrics, logging, serialization, and
> > > > >> config
> > > > systems,
> > > > >> and simply use Kafka's instead.
> > > > >>
> > > > >> This would fix all of the issues that I outlined above. It
> > > > >> should
> > also
> > > > >> shrink the Samza code base pretty dramatically. Supporting only
> > > > >> a standalone container will allow Samza to be executed on YARN
> > > > >> (using Slider), Mesos (using Marathon/Aurora), or most other
> > > > >> in-house
> > > > deployment
> > > > >> systems. This should make life a lot easier for new users.
> > > > >> Imagine
> > > > having
> > > > >> the hello-samza tutorial without YARN. The drop in mailing list
> > > traffic
> > > > >> will be pretty dramatic.
> > > > >>
> > > > >> Coupling with Kafka seems long overdue to me. The reality is,
> > everyone
> > > > >> that
> > > > >> I'm aware of is using Samza with Kafka. We basically require it
> > > already
> > > > in
> > > > >> order for most features to work. Those that are using other
> > > > >> systems
> > > are
> > > > >> generally using it for ingest into Kafka (1), and then they do
> > > > >> the processing on top. There is already discussion (
> > > > >>
> > > >
> > >
> > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> > 767
> > > > >> )
> > > > >> in Kafka to make ingesting into Kafka extremely easy.
> > > > >>
> > > > >> Once we make the call to couple with Kafka, we can leverage a
> > > > >> ton of
> > > > their
> > > > >> ecosystem. We no longer have to maintain our own config,
> > > > >> metrics,
> > etc.
> > > > We
> > > > >> can all share the same libraries, and make them better. This
> > > > >> will
> > also
> > > > >> allow us to share the consumer/producer APIs, and will let us
> > leverage
> > > > >> their offset management and partition management, rather than
> > > > >> having
> > > our
> > > > >> own. All of the coordinator stream code would go away, as would
> > > > >> most
> > > of
> > > > >> the
> > > > >> YARN AppMaster code. We'd probably have to push some partition
> > > > management
> > > > >> features into the Kafka broker, but they're already moving in
> > > > >> that direction with the new consumer API. The features we have
> > > > >> for
> > > partition
> > > > >> assignment aren't unique to Samza, and seem like they should be
> > > > >> in
> > > Kafka
> > > > >> anyway. There will always be some niche usages which will
> > > > >> require
> > > extra
> > > > >> care and hence full control over partition assignments much
> > > > >> like the
> > > > Kafka
> > > > >> low level consumer api. These would continue to be supported.
> > > > >>
> > > > >> These items will be good for the Samza community. They'll make
> > > > >> Samza easier to use, and make it easier for developers to add
> > > > >> new features.
> > > > >>
> > > > >> Obviously this is a fairly large (and somewhat backwards
> > incompatible
> > > > >> change). If we choose to go this route, it's important that we
> > openly
> > > > >> communicate how we're going to provide a migration path from
> > > > >> the
> > > > existing
> > > > >> APIs to the new ones (if we make incompatible changes). I think
> > > > >> at a minimum, we'd probably need to provide a wrapper to allow
> > > > >> existing StreamTask implementations to continue running on the
> new container.
> > > > It's
> > > > >> also important that we openly communicate about timing, and
> > > > >> stages
> > of
> > > > the
> > > > >> migration.
> > > > >>
> > > > >> If you made it this far, I'm sure you have opinions. :) Please
> > > > >> send
> > > your
> > > > >> thoughts and feedback.
> > > > >>
> > > > >> Cheers,
> > > > >> Chris
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

RE: Thoughts and obesrvations on Samza

Posted by Garry Turkington <g....@improvedigital.com>.
Hi all,

I think the question below re does Samza become a sub-project of Kafka highlights the broader point around migration. Chris mentions Samza's maturity is heading towards a v1 release but I'm not sure it feels right to launch a v1 then immediately plan to deprecate most of it.

From a selfish perspective I have some guys who have started working with Samza and building some new consumers/producers was next up. Sounds like that is absolutely not the direction to go. I need to look into the KIP in more detail but for me the attractiveness of adding new Samza consumer/producers -- even if yes all they were doing was really getting data into and out of Kafka --  was to avoid  having to worry about the lifecycle management of external clients. If there is a generic Kafka ingress/egress layer that I can plug a new connector into and have a lot of the heavy lifting re scale and reliability done for me then it gives me all the pushing new consumers/producers would. If not then it complicates my operational deployments.

Which is similar to my other question with the proposal -- if we build a fully available/stand-alone Samza plus the requisite shims to integrate with Slider etc I suspect the former may be a lot more work than we think. We may make it much easier for a newcomer to get something running but having them step up and get a reliable production deployment may still dominate mailing list  traffic, if for different reasons than today.

Don't get me wrong -- I'm comfortable with making the Samza dependency on Kafka much more explicit and I absolutely see the benefits  in the reduction of duplication and clashing terminologies/abstractions that Chris/Jay describe. Samza as a library would likely be a very nice tool to add to the Kafka ecosystem. I just have the concerns above re the operational side.

Garry

-----Original Message-----
From: Gianmarco De Francisci Morales [mailto:gdfm@apache.org] 
Sent: 02 July 2015 12:56
To: dev@samza.apache.org
Subject: Re: Thoughts and obesrvations on Samza

Very interesting thoughts.
From outside, I have always perceived Samza as a computing layer over Kafka.

The question, maybe a bit provocative, is "should Samza be a sub-project of Kafka then?"
Or does it make sense to keep it as a separate project with a separate governance?

Cheers,

--
Gianmarco

On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:

> Overall, I agree to couple with Kafka more tightly. Because Samza de 
> facto is based on Kafka, and it should leverage what Kafka has. At the 
> same time, Kafka does not need to reinvent what Samza already has. I 
> also like the idea of separating the ingestion and transformation.
>
> But it is a little difficult for me to image how the Samza will look like.
> And I feel Chris and Jay have a little difference in terms of how 
> Samza should look like.
>
> *** Will it look like what Jay's code shows (A client of Kakfa) ? And 
> user's application code calls this client?
>
> 1. If we make Samza be a library of Kafka (like what the code shows), 
> how do we implement auto-balance and fault-tolerance? Are they taken 
> care by the Kafka broker or other mechanism, such as "Samza worker" 
> (just make up the name) ?
>
> 2. What about other features, such as auto-scaling, shared state, 
> monitoring?
>
>
> *** If we have Samza standalone, (is this what Chris suggests?)
>
> 1. we still need to ingest data from Kakfa and produce to it. Then it 
> becomes the same as what Samza looks like now, except it does not rely 
> on Yarn anymore.
>
> 2. if it is standalone, how can it leverage Kafka's metrics, logs, 
> etc? Use Kafka code as the dependency?
>
>
> Thanks,
>
> Fang, Yan
> yanfang724@gmail.com
>
> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Read through the code example and it looks good to me. A few 
> > thoughts regarding deployment:
> >
> > Today Samza deploys as executable runnable like:
> >
> > deploy/samza/bin/run-job.sh --config-factory=... --config-path=file://...
> >
> > And this proposal advocate for deploying Samza more as embedded 
> > libraries in user application code (ignoring the terminology since 
> > it is not the
> same
> > as the prototype code):
> >
> > StreamTask task = new MyStreamTask(configs); Thread thread = new 
> > Thread(task); thread.start();
> >
> > I think both of these deployment modes are important for different 
> > types
> of
> > users. That said, I think making Samza purely standalone is still 
> > sufficient for either runnable or library modes.
> >
> > Guozhang
> >
> > On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Looks like gmail mangled the code example, it was supposed to look 
> > > like
> > > this:
> > >
> > > Properties props = new Properties(); 
> > > props.put("bootstrap.servers", "localhost:4242"); StreamingConfig 
> > > config = new StreamingConfig(props); 
> > > config.subscribe("test-topic-1", "test-topic-2"); 
> > > config.processor(ExampleStreamProcessor.class);
> > > config.serialization(new StringSerializer(), new 
> > > StringDeserializer()); KafkaStreaming container = new 
> > > KafkaStreaming(config); container.run();
> > >
> > > -Jay
> > >
> > > On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Hey guys,
> > > >
> > > > This came out of some conversations Chris and I were having 
> > > > around
> > > whether
> > > > it would make sense to use Samza as a kind of data ingestion
> framework
> > > for
> > > > Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> > combined
> > > > with complaints around config and YARN and the discussion around 
> > > > how
> to
> > > > best do a standalone mode.
> > > >
> > > > So the thought experiment was, given that Samza was basically 
> > > > already totally Kafka specific, what if you just embraced that 
> > > > and turned it
> > into
> > > > something less like a heavyweight framework and more like a 
> > > > third
> Kafka
> > > > client--a kind of "producing consumer" with state management
> > facilities.
> > > > Basically a library. Instead of a complex stream processing 
> > > > framework
> > > this
> > > > would actually be a very simple thing, not much more complicated 
> > > > to
> use
> > > or
> > > > operate than a Kafka consumer. As Chris said we thought about it 
> > > > a
> lot
> > of
> > > > what Samza (and the other stream processing systems were doing)
> seemed
> > > like
> > > > kind of a hangover from MapReduce.
> > > >
> > > > Of course you need to ingest/output data to and from the stream 
> > > > processing. But when we actually looked into how that would 
> > > > work,
> Samza
> > > > isn't really an ideal data ingestion framework for a bunch of
> reasons.
> > To
> > > > really do that right you need a pretty different internal data 
> > > > model
> > and
> > > > set of apis. So what if you split them and had an api for Kafka 
> > > > ingress/egress (copycat AKA KIP-26) and a separate api for Kafka 
> > > > transformation (Samza).
> > > >
> > > > This would also allow really embracing the same terminology and 
> > > > conventions. One complaint about the current state is that the 
> > > > two
> > > systems
> > > > kind of feel bolted on. Terminology like "stream" vs "topic" and
> > > different
> > > > config and monitoring systems means you kind of have to learn 
> > > > Kafka's
> > > way,
> > > > then learn Samza's slightly different way, then kind of 
> > > > understand
> how
> > > they
> > > > map to each other, which having walked a few people through this 
> > > > is surprisingly tricky for folks to get.
> > > >
> > > > Since I have been spending a lot of time on airplanes I hacked 
> > > > up an ernest but still somewhat incomplete prototype of what 
> > > > this would
> look
> > > > like. This is just unceremoniously dumped into Kafka as it 
> > > > required a
> > few
> > > > changes to the new consumer. Here is the code:
> > > >
> > > >
> > >
> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org
> /apache/kafka/clients/streaming
> > > >
> > > > For the purpose of the prototype I just liberally renamed 
> > > > everything
> to
> > > > try to align it with Kafka with no regard for compatibility.
> > > >
> > > > To use this would be something like this:
> > > > Properties props = new Properties(); 
> > > > props.put("bootstrap.servers", "localhost:4242"); 
> > > > StreamingConfig config = new
> StreamingConfig(props);
> > > config.subscribe("test-topic-1",
> > > > "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > > config.serialization(new
> > > > StringSerializer(), new StringDeserializer()); KafkaStreaming
> > container =
> > > > new KafkaStreaming(config); container.run();
> > > >
> > > > KafkaStreaming is basically the SamzaContainer; StreamProcessor 
> > > > is basically StreamTask.
> > > >
> > > > So rather than putting all the class names in a file and then 
> > > > having
> > the
> > > > job assembled by reflection, you just instantiate the container 
> > > > programmatically. Work is balanced over however many instances 
> > > > of
> this
> > > are
> > > > alive at any time (i.e. if an instance dies, new tasks are added 
> > > > to
> the
> > > > existing containers without shutting them down).
> > > >
> > > > We would provide some glue for running this stuff in YARN via 
> > > > Slider, Mesos via Marathon, and AWS using some of their tools 
> > > > but from the
> > point
> > > of
> > > > view of these frameworks these stream processing jobs are just
> > stateless
> > > > services that can come and go and expand and contract at will. 
> > > > There
> is
> > > no
> > > > more custom scheduler.
> > > >
> > > > Here are some relevant details:
> > > >
> > > >    1. It is only ~1300 lines of code, it would get larger if we
> > > >    productionized but not vastly larger. We really do get a ton 
> > > > of
> > > leverage
> > > >    out of Kafka.
> > > >    2. Partition management is fully delegated to the new consumer.
> This
> > > >    is nice since now any partition management strategy available 
> > > > to
> > Kafka
> > > >    consumer is also available to Samza (and vice versa) and with 
> > > > the
> > > exact
> > > >    same configs.
> > > >    3. It supports state as well as state reuse
> > > >
> > > > Anyhow take a look, hopefully it is thought provoking.
> > > >
> > > > -Jay
> > > >
> > > >
> > > >
> > > > On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > criccomini@apache.org>
> > > > wrote:
> > > >
> > > >> Hey all,
> > > >>
> > > >> I have had some discussions with Samza engineers at LinkedIn 
> > > >> and
> > > Confluent
> > > >> and we came up with a few observations and would like to 
> > > >> propose
> some
> > > >> changes.
> > > >>
> > > >> We've observed some things that I want to call out about 
> > > >> Samza's
> > design,
> > > >> and I'd like to propose some changes.
> > > >>
> > > >> * Samza is dependent upon a dynamic deployment system.
> > > >> * Samza is too pluggable.
> > > >> * Samza's SystemConsumer/SystemProducer and Kafka's consumer 
> > > >> APIs
> are
> > > >> trying to solve a lot of the same problems.
> > > >>
> > > >> All three of these issues are related, but I'll address them in
> order.
> > > >>
> > > >> Deployment
> > > >>
> > > >> Samza strongly depends on the use of a dynamic deployment 
> > > >> scheduler
> > such
> > > >> as
> > > >> YARN, Mesos, etc. When we initially built Samza, we bet that 
> > > >> there
> > would
> > > >> be
> > > >> one or two winners in this area, and we could support them, and 
> > > >> the
> > rest
> > > >> would go away. In reality, there are many variations. 
> > > >> Furthermore,
> > many
> > > >> people still prefer to just start their processors like normal 
> > > >> Java processes, and use traditional deployment scripts such as 
> > > >> Fabric,
> > Chef,
> > > >> Ansible, etc. Forcing a deployment system on users makes the 
> > > >> Samza start-up process really painful for first time users.
> > > >>
> > > >> Dynamic deployment as a requirement was also a bit of a 
> > > >> mis-fire
> > because
> > > >> of
> > > >> a fundamental misunderstanding between the nature of batch jobs 
> > > >> and
> > > stream
> > > >> processing jobs. Early on, we made conscious effort to favor 
> > > >> the
> > Hadoop
> > > >> (Map/Reduce) way of doing things, since it worked and was well
> > > understood.
> > > >> One thing that we missed was that batch jobs have a definite
> > beginning,
> > > >> and
> > > >> end, and stream processing jobs don't (usually). This leads to 
> > > >> a
> much
> > > >> simpler scheduling problem for stream processors. You basically 
> > > >> just
> > > need
> > > >> to find a place to start the processor, and start it. The way 
> > > >> we run grids, at LinkedIn, there's no concept of a cluster 
> > > >> being "full". We always
> > add
> > > >> more machines. The problem with coupling Samza with a scheduler 
> > > >> is
> > that
> > > >> Samza (as a framework) now has to handle deployment. This pulls 
> > > >> in a
> > > bunch
> > > >> of things such as configuration distribution (config stream), 
> > > >> shell
> > > scrips
> > > >> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> > > >>
> > > >> Another reason for requiring dynamic deployment was to support 
> > > >> data locality. If you want to have locality, you need to put 
> > > >> your
> > processors
> > > >> close to the data they're processing. Upon further 
> > > >> investigation,
> > > though,
> > > >> this feature is not that beneficial. There is some good 
> > > >> discussion
> > about
> > > >> some problems with it on SAMZA-335. Again, we took the 
> > > >> Map/Reduce
> > path,
> > > >> but
> > > >> there are some fundamental differences between HDFS and Kafka. 
> > > >> HDFS
> > has
> > > >> blocks, while Kafka has partitions. This leads to less 
> > > >> optimization potential with stream processors on top of Kafka.
> > > >>
> > > >> This feature is also used as a crutch. Samza doesn't have any 
> > > >> built
> in
> > > >> fault-tolerance logic. Instead, it depends on the dynamic 
> > > >> deployment scheduling system to handle restarts when a 
> > > >> processor dies. This has
> > > made
> > > >> it very difficult to write a standalone Samza container (SAMZA-516).
> > > >>
> > > >> Pluggability
> > > >>
> > > >> In some cases pluggability is good, but I think that we've gone 
> > > >> too
> > far
> > > >> with it. Currently, Samza has:
> > > >>
> > > >> * Pluggable config.
> > > >> * Pluggable metrics.
> > > >> * Pluggable deployment systems.
> > > >> * Pluggable streaming systems (SystemConsumer, SystemProducer, etc).
> > > >> * Pluggable serdes.
> > > >> * Pluggable storage engines.
> > > >> * Pluggable strategies for just about every component
> (MessageChooser,
> > > >> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > >>
> > > >> There's probably more that I've forgotten, as well. Some of 
> > > >> these
> are
> > > >> useful, but some have proven not to be. This all comes at a cost:
> > > >> complexity. This complexity is making it harder for our users 
> > > >> to
> pick
> > up
> > > >> and use Samza out of the box. It also makes it difficult for 
> > > >> Samza developers to reason about what the characteristics of 
> > > >> the container (since the characteristics change depending on 
> > > >> which plugins are use).
> > > >>
> > > >> The issues with pluggability are most visible in the System APIs.
> What
> > > >> Samza really requires to be functional is Kafka as its 
> > > >> transport
> > layer.
> > > >> But
> > > >> we've conflated two unrelated use cases into one API:
> > > >>
> > > >> 1. Get data into/out of Kafka.
> > > >> 2. Process the data in Kafka.
> > > >>
> > > >> The current System API supports both of these use cases. The 
> > > >> problem
> > is,
> > > >> we
> > > >> actually want different features for each use case. By papering 
> > > >> over
> > > these
> > > >> two use cases, and providing a single API, we've introduced a 
> > > >> ton of
> > > leaky
> > > >> abstractions.
> > > >>
> > > >> For example, what we'd really like in (2) is to have 
> > > >> monotonically increasing longs for offsets (like Kafka). This 
> > > >> would be at odds
> with
> > > (1),
> > > >> though, since different systems have different
> > > SCNs/Offsets/UUIDs/vectors.
> > > >> There was discussion both on the mailing list and the SQL JIRAs
> about
> > > the
> > > >> need for this.
> > > >>
> > > >> The same thing holds true for replayability. Kafka allows us to
> rewind
> > > >> when
> > > >> we have a failure. Many other systems don't. In some cases, 
> > > >> systems
> > > return
> > > >> null for their offsets (e.g. WikipediaSystemConsumer) because 
> > > >> they
> > have
> > > no
> > > >> offsets.
> > > >>
> > > >> Partitioning is another example. Kafka supports partitioning, 
> > > >> but
> many
> > > >> systems don't. We model this by having a single partition for 
> > > >> those systems. Still, other systems model partitioning differently (e.g.
> > > >> Kinesis).
> > > >>
> > > >> The SystemAdmin interface is also a mess. Creating streams in a 
> > > >> system-agnostic way is almost impossible. As is modeling 
> > > >> metadata
> for
> > > the
> > > >> system (replication factor, partitions, location, etc). The 
> > > >> list
> goes
> > > on.
> > > >>
> > > >> Duplicate work
> > > >>
> > > >> At the time that we began writing Samza, Kafka's consumer and
> producer
> > > >> APIs
> > > >> had a relatively weak feature set. On the consumer-side, you 
> > > >> had two
> > > >> options: use the high level consumer, or the simple consumer. 
> > > >> The
> > > problem
> > > >> with the high-level consumer was that it controlled your 
> > > >> offsets, partition assignments, and the order in which you 
> > > >> received messages. The
> problem
> > > >> with
> > > >> the simple consumer is that it's not simple. It's basic. You 
> > > >> end up
> > > having
> > > >> to handle a lot of really low-level stuff that you shouldn't. 
> > > >> We
> > spent a
> > > >> lot of time to make Samza's KafkaSystemConsumer very robust. It 
> > > >> also allows us to support some cool features:
> > > >>
> > > >> * Per-partition message ordering and prioritization.
> > > >> * Tight control over partition assignment to support joins, 
> > > >> global
> > state
> > > >> (if we want to implement it :)), etc.
> > > >> * Tight control over offset checkpointing.
> > > >>
> > > >> What we didn't realize at the time is that these features 
> > > >> should
> > > actually
> > > >> be in Kafka. A lot of Kafka consumers (not just Samza stream
> > processors)
> > > >> end up wanting to do things like joins and partition 
> > > >> assignment. The
> > > Kafka
> > > >> community has come to the same conclusion. They're adding a ton 
> > > >> of upgrades into their new Kafka consumer implementation. To a 
> > > >> large extent,
> it's
> > > >> duplicate work to what we've already done in Samza.
> > > >>
> > > >> On top of this, Kafka ended up taking a very similar approach 
> > > >> to
> > Samza's
> > > >> KafkaCheckpointManager implementation for handling offset
> > checkpointing.
> > > >> Like Samza, Kafka's new offset management feature stores offset 
> > > >> checkpoints in a topic, and allows you to fetch them from the 
> > > >> broker.
> > > >>
> > > >> A lot of this seems like a waste, since we could have shared 
> > > >> the
> work
> > if
> > > >> it
> > > >> had been done in Kafka from the get-go.
> > > >>
> > > >> Vision
> > > >>
> > > >> All of this leads me to a rather radical proposal. Samza is
> relatively
> > > >> stable at this point. I'd venture to say that we're near a 1.0
> > release.
> > > >> I'd
> > > >> like to propose that we take what we've learned, and begin 
> > > >> thinking
> > > about
> > > >> Samza beyond 1.0. What would we change if we were starting from
> > scratch?
> > > >> My
> > > >> proposal is to:
> > > >>
> > > >> 1. Make Samza standalone the *only* way to run Samza 
> > > >> processors, and eliminate all direct dependences on YARN, Mesos, etc.
> > > >> 2. Make a definitive call to support only Kafka as the stream
> > processing
> > > >> layer.
> > > >> 3. Eliminate Samza's metrics, logging, serialization, and 
> > > >> config
> > > systems,
> > > >> and simply use Kafka's instead.
> > > >>
> > > >> This would fix all of the issues that I outlined above. It 
> > > >> should
> also
> > > >> shrink the Samza code base pretty dramatically. Supporting only 
> > > >> a standalone container will allow Samza to be executed on YARN 
> > > >> (using Slider), Mesos (using Marathon/Aurora), or most other 
> > > >> in-house
> > > deployment
> > > >> systems. This should make life a lot easier for new users. 
> > > >> Imagine
> > > having
> > > >> the hello-samza tutorial without YARN. The drop in mailing list
> > traffic
> > > >> will be pretty dramatic.
> > > >>
> > > >> Coupling with Kafka seems long overdue to me. The reality is,
> everyone
> > > >> that
> > > >> I'm aware of is using Samza with Kafka. We basically require it
> > already
> > > in
> > > >> order for most features to work. Those that are using other 
> > > >> systems
> > are
> > > >> generally using it for ingest into Kafka (1), and then they do 
> > > >> the processing on top. There is already discussion (
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851
> 767
> > > >> )
> > > >> in Kafka to make ingesting into Kafka extremely easy.
> > > >>
> > > >> Once we make the call to couple with Kafka, we can leverage a 
> > > >> ton of
> > > their
> > > >> ecosystem. We no longer have to maintain our own config, 
> > > >> metrics,
> etc.
> > > We
> > > >> can all share the same libraries, and make them better. This 
> > > >> will
> also
> > > >> allow us to share the consumer/producer APIs, and will let us
> leverage
> > > >> their offset management and partition management, rather than 
> > > >> having
> > our
> > > >> own. All of the coordinator stream code would go away, as would 
> > > >> most
> > of
> > > >> the
> > > >> YARN AppMaster code. We'd probably have to push some partition
> > > management
> > > >> features into the Kafka broker, but they're already moving in 
> > > >> that direction with the new consumer API. The features we have 
> > > >> for
> > partition
> > > >> assignment aren't unique to Samza, and seem like they should be 
> > > >> in
> > Kafka
> > > >> anyway. There will always be some niche usages which will 
> > > >> require
> > extra
> > > >> care and hence full control over partition assignments much 
> > > >> like the
> > > Kafka
> > > >> low level consumer api. These would continue to be supported.
> > > >>
> > > >> These items will be good for the Samza community. They'll make 
> > > >> Samza easier to use, and make it easier for developers to add 
> > > >> new features.
> > > >>
> > > >> Obviously this is a fairly large (and somewhat backwards
> incompatible
> > > >> change). If we choose to go this route, it's important that we
> openly
> > > >> communicate how we're going to provide a migration path from 
> > > >> the
> > > existing
> > > >> APIs to the new ones (if we make incompatible changes). I think 
> > > >> at a minimum, we'd probably need to provide a wrapper to allow 
> > > >> existing StreamTask implementations to continue running on the new container.
> > > It's
> > > >> also important that we openly communicate about timing, and 
> > > >> stages
> of
> > > the
> > > >> migration.
> > > >>
> > > >> If you made it this far, I'm sure you have opinions. :) Please 
> > > >> send
> > your
> > > >> thoughts and feedback.
> > > >>
> > > >> Cheers,
> > > >> Chris
> > > >>
> > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: Thoughts and obesrvations on Samza

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

I agree that most people view Samza as a compute layer on top of Kafka and
that is not actually a bad thing. We have kind of built things as if they
were totally separate which kind of makes things harder for people which
is, I think, the important thing to correct.

As to your question about whether Samza should be a sub-project of Kafka. I
don't know, but it is worth thinking about.

I think there are a lot of good software engineering reasons to want a
separate repository and committer base. I think the prototype I showed
demonstrates that core Samza could be quite compact if it kind of embraced
Kafka but there is a bunch of stuff going on around SQL support that is
pretty extensive in its own right. So that kind of argues for keeping
things separate.

But from a branding and user experience point of view I think Samza would
really benefit from closer alignment. If it were just a light
transformation library that was configured, monitored, etc just like Kafka
that would make it a very light-weight adoption decision for processing if
you are going to be using Kafka for data. I think calling it something like
"Kafka Streams" would really help express what it is, and I think it would
be great to integrate the documentation with the main Kafka docs so people
could discover it in the natural course of things. I think this would help
a ton with adoption and really sell the point that it is a light-weight
adoption decision once you have Kafka.

-Jay



On Thu, Jul 2, 2015 at 3:55 AM, Gianmarco De Francisci Morales <
gdfm@apache.org> wrote:

> Very interesting thoughts.
> From outside, I have always perceived Samza as a computing layer over
> Kafka.
>
> The question, maybe a bit provocative, is "should Samza be a sub-project of
> Kafka then?"
> Or does it make sense to keep it as a separate project with a separate
> governance?
>
> Cheers,
>
> --
> Gianmarco
>
> On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:
>
> > Overall, I agree to couple with Kafka more tightly. Because Samza de
> facto
> > is based on Kafka, and it should leverage what Kafka has. At the same
> time,
> > Kafka does not need to reinvent what Samza already has. I also like the
> > idea of separating the ingestion and transformation.
> >
> > But it is a little difficult for me to image how the Samza will look
> like.
> > And I feel Chris and Jay have a little difference in terms of how Samza
> > should look like.
> >
> > *** Will it look like what Jay's code shows (A client of Kakfa) ? And
> > user's application code calls this client?
> >
> > 1. If we make Samza be a library of Kafka (like what the code shows), how
> > do we implement auto-balance and fault-tolerance? Are they taken care by
> > the Kafka broker or other mechanism, such as "Samza worker" (just make up
> > the name) ?
> >
> > 2. What about other features, such as auto-scaling, shared state,
> > monitoring?
> >
> >
> > *** If we have Samza standalone, (is this what Chris suggests?)
> >
> > 1. we still need to ingest data from Kakfa and produce to it. Then it
> > becomes the same as what Samza looks like now, except it does not rely on
> > Yarn anymore.
> >
> > 2. if it is standalone, how can it leverage Kafka's metrics, logs, etc?
> Use
> > Kafka code as the dependency?
> >
> >
> > Thanks,
> >
> > Fang, Yan
> > yanfang724@gmail.com
> >
> > On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Read through the code example and it looks good to me. A few thoughts
> > > regarding deployment:
> > >
> > > Today Samza deploys as executable runnable like:
> > >
> > > deploy/samza/bin/run-job.sh --config-factory=...
> --config-path=file://...
> > >
> > > And this proposal advocate for deploying Samza more as embedded
> libraries
> > > in user application code (ignoring the terminology since it is not the
> > same
> > > as the prototype code):
> > >
> > > StreamTask task = new MyStreamTask(configs);
> > > Thread thread = new Thread(task);
> > > thread.start();
> > >
> > > I think both of these deployment modes are important for different
> types
> > of
> > > users. That said, I think making Samza purely standalone is still
> > > sufficient for either runnable or library modes.
> > >
> > > Guozhang
> > >
> > > On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Looks like gmail mangled the code example, it was supposed to look
> like
> > > > this:
> > > >
> > > > Properties props = new Properties();
> > > > props.put("bootstrap.servers", "localhost:4242");
> > > > StreamingConfig config = new StreamingConfig(props);
> > > > config.subscribe("test-topic-1", "test-topic-2");
> > > > config.processor(ExampleStreamProcessor.class);
> > > > config.serialization(new StringSerializer(), new
> StringDeserializer());
> > > > KafkaStreaming container = new KafkaStreaming(config);
> > > > container.run();
> > > >
> > > > -Jay
> > > >
> > > > On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io>
> wrote:
> > > >
> > > > > Hey guys,
> > > > >
> > > > > This came out of some conversations Chris and I were having around
> > > > whether
> > > > > it would make sense to use Samza as a kind of data ingestion
> > framework
> > > > for
> > > > > Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> > > combined
> > > > > with complaints around config and YARN and the discussion around
> how
> > to
> > > > > best do a standalone mode.
> > > > >
> > > > > So the thought experiment was, given that Samza was basically
> already
> > > > > totally Kafka specific, what if you just embraced that and turned
> it
> > > into
> > > > > something less like a heavyweight framework and more like a third
> > Kafka
> > > > > client--a kind of "producing consumer" with state management
> > > facilities.
> > > > > Basically a library. Instead of a complex stream processing
> framework
> > > > this
> > > > > would actually be a very simple thing, not much more complicated to
> > use
> > > > or
> > > > > operate than a Kafka consumer. As Chris said we thought about it a
> > lot
> > > of
> > > > > what Samza (and the other stream processing systems were doing)
> > seemed
> > > > like
> > > > > kind of a hangover from MapReduce.
> > > > >
> > > > > Of course you need to ingest/output data to and from the stream
> > > > > processing. But when we actually looked into how that would work,
> > Samza
> > > > > isn't really an ideal data ingestion framework for a bunch of
> > reasons.
> > > To
> > > > > really do that right you need a pretty different internal data
> model
> > > and
> > > > > set of apis. So what if you split them and had an api for Kafka
> > > > > ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > > > > transformation (Samza).
> > > > >
> > > > > This would also allow really embracing the same terminology and
> > > > > conventions. One complaint about the current state is that the two
> > > > systems
> > > > > kind of feel bolted on. Terminology like "stream" vs "topic" and
> > > > different
> > > > > config and monitoring systems means you kind of have to learn
> Kafka's
> > > > way,
> > > > > then learn Samza's slightly different way, then kind of understand
> > how
> > > > they
> > > > > map to each other, which having walked a few people through this is
> > > > > surprisingly tricky for folks to get.
> > > > >
> > > > > Since I have been spending a lot of time on airplanes I hacked up
> an
> > > > > ernest but still somewhat incomplete prototype of what this would
> > look
> > > > > like. This is just unceremoniously dumped into Kafka as it
> required a
> > > few
> > > > > changes to the new consumer. Here is the code:
> > > > >
> > > > >
> > > >
> > >
> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org/apache/kafka/clients/streaming
> > > > >
> > > > > For the purpose of the prototype I just liberally renamed
> everything
> > to
> > > > > try to align it with Kafka with no regard for compatibility.
> > > > >
> > > > > To use this would be something like this:
> > > > > Properties props = new Properties(); props.put("bootstrap.servers",
> > > > > "localhost:4242"); StreamingConfig config = new
> > StreamingConfig(props);
> > > > config.subscribe("test-topic-1",
> > > > > "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > > > config.serialization(new
> > > > > StringSerializer(), new StringDeserializer()); KafkaStreaming
> > > container =
> > > > > new KafkaStreaming(config); container.run();
> > > > >
> > > > > KafkaStreaming is basically the SamzaContainer; StreamProcessor is
> > > > > basically StreamTask.
> > > > >
> > > > > So rather than putting all the class names in a file and then
> having
> > > the
> > > > > job assembled by reflection, you just instantiate the container
> > > > > programmatically. Work is balanced over however many instances of
> > this
> > > > are
> > > > > alive at any time (i.e. if an instance dies, new tasks are added to
> > the
> > > > > existing containers without shutting them down).
> > > > >
> > > > > We would provide some glue for running this stuff in YARN via
> Slider,
> > > > > Mesos via Marathon, and AWS using some of their tools but from the
> > > point
> > > > of
> > > > > view of these frameworks these stream processing jobs are just
> > > stateless
> > > > > services that can come and go and expand and contract at will.
> There
> > is
> > > > no
> > > > > more custom scheduler.
> > > > >
> > > > > Here are some relevant details:
> > > > >
> > > > >    1. It is only ~1300 lines of code, it would get larger if we
> > > > >    productionized but not vastly larger. We really do get a ton of
> > > > leverage
> > > > >    out of Kafka.
> > > > >    2. Partition management is fully delegated to the new consumer.
> > This
> > > > >    is nice since now any partition management strategy available to
> > > Kafka
> > > > >    consumer is also available to Samza (and vice versa) and with
> the
> > > > exact
> > > > >    same configs.
> > > > >    3. It supports state as well as state reuse
> > > > >
> > > > > Anyhow take a look, hopefully it is thought provoking.
> > > > >
> > > > > -Jay
> > > > >
> > > > >
> > > > >
> > > > > On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > > criccomini@apache.org>
> > > > > wrote:
> > > > >
> > > > >> Hey all,
> > > > >>
> > > > >> I have had some discussions with Samza engineers at LinkedIn and
> > > > Confluent
> > > > >> and we came up with a few observations and would like to propose
> > some
> > > > >> changes.
> > > > >>
> > > > >> We've observed some things that I want to call out about Samza's
> > > design,
> > > > >> and I'd like to propose some changes.
> > > > >>
> > > > >> * Samza is dependent upon a dynamic deployment system.
> > > > >> * Samza is too pluggable.
> > > > >> * Samza's SystemConsumer/SystemProducer and Kafka's consumer APIs
> > are
> > > > >> trying to solve a lot of the same problems.
> > > > >>
> > > > >> All three of these issues are related, but I'll address them in
> > order.
> > > > >>
> > > > >> Deployment
> > > > >>
> > > > >> Samza strongly depends on the use of a dynamic deployment
> scheduler
> > > such
> > > > >> as
> > > > >> YARN, Mesos, etc. When we initially built Samza, we bet that there
> > > would
> > > > >> be
> > > > >> one or two winners in this area, and we could support them, and
> the
> > > rest
> > > > >> would go away. In reality, there are many variations. Furthermore,
> > > many
> > > > >> people still prefer to just start their processors like normal
> Java
> > > > >> processes, and use traditional deployment scripts such as Fabric,
> > > Chef,
> > > > >> Ansible, etc. Forcing a deployment system on users makes the Samza
> > > > >> start-up
> > > > >> process really painful for first time users.
> > > > >>
> > > > >> Dynamic deployment as a requirement was also a bit of a mis-fire
> > > because
> > > > >> of
> > > > >> a fundamental misunderstanding between the nature of batch jobs
> and
> > > > stream
> > > > >> processing jobs. Early on, we made conscious effort to favor the
> > > Hadoop
> > > > >> (Map/Reduce) way of doing things, since it worked and was well
> > > > understood.
> > > > >> One thing that we missed was that batch jobs have a definite
> > > beginning,
> > > > >> and
> > > > >> end, and stream processing jobs don't (usually). This leads to a
> > much
> > > > >> simpler scheduling problem for stream processors. You basically
> just
> > > > need
> > > > >> to find a place to start the processor, and start it. The way we
> run
> > > > >> grids,
> > > > >> at LinkedIn, there's no concept of a cluster being "full". We
> always
> > > add
> > > > >> more machines. The problem with coupling Samza with a scheduler is
> > > that
> > > > >> Samza (as a framework) now has to handle deployment. This pulls
> in a
> > > > bunch
> > > > >> of things such as configuration distribution (config stream),
> shell
> > > > scrips
> > > > >> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> > > > >>
> > > > >> Another reason for requiring dynamic deployment was to support
> data
> > > > >> locality. If you want to have locality, you need to put your
> > > processors
> > > > >> close to the data they're processing. Upon further investigation,
> > > > though,
> > > > >> this feature is not that beneficial. There is some good discussion
> > > about
> > > > >> some problems with it on SAMZA-335. Again, we took the Map/Reduce
> > > path,
> > > > >> but
> > > > >> there are some fundamental differences between HDFS and Kafka.
> HDFS
> > > has
> > > > >> blocks, while Kafka has partitions. This leads to less
> optimization
> > > > >> potential with stream processors on top of Kafka.
> > > > >>
> > > > >> This feature is also used as a crutch. Samza doesn't have any
> built
> > in
> > > > >> fault-tolerance logic. Instead, it depends on the dynamic
> deployment
> > > > >> scheduling system to handle restarts when a processor dies. This
> has
> > > > made
> > > > >> it very difficult to write a standalone Samza container
> (SAMZA-516).
> > > > >>
> > > > >> Pluggability
> > > > >>
> > > > >> In some cases pluggability is good, but I think that we've gone
> too
> > > far
> > > > >> with it. Currently, Samza has:
> > > > >>
> > > > >> * Pluggable config.
> > > > >> * Pluggable metrics.
> > > > >> * Pluggable deployment systems.
> > > > >> * Pluggable streaming systems (SystemConsumer, SystemProducer,
> etc).
> > > > >> * Pluggable serdes.
> > > > >> * Pluggable storage engines.
> > > > >> * Pluggable strategies for just about every component
> > (MessageChooser,
> > > > >> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > > >>
> > > > >> There's probably more that I've forgotten, as well. Some of these
> > are
> > > > >> useful, but some have proven not to be. This all comes at a cost:
> > > > >> complexity. This complexity is making it harder for our users to
> > pick
> > > up
> > > > >> and use Samza out of the box. It also makes it difficult for Samza
> > > > >> developers to reason about what the characteristics of the
> container
> > > > >> (since
> > > > >> the characteristics change depending on which plugins are use).
> > > > >>
> > > > >> The issues with pluggability are most visible in the System APIs.
> > What
> > > > >> Samza really requires to be functional is Kafka as its transport
> > > layer.
> > > > >> But
> > > > >> we've conflated two unrelated use cases into one API:
> > > > >>
> > > > >> 1. Get data into/out of Kafka.
> > > > >> 2. Process the data in Kafka.
> > > > >>
> > > > >> The current System API supports both of these use cases. The
> problem
> > > is,
> > > > >> we
> > > > >> actually want different features for each use case. By papering
> over
> > > > these
> > > > >> two use cases, and providing a single API, we've introduced a ton
> of
> > > > leaky
> > > > >> abstractions.
> > > > >>
> > > > >> For example, what we'd really like in (2) is to have monotonically
> > > > >> increasing longs for offsets (like Kafka). This would be at odds
> > with
> > > > (1),
> > > > >> though, since different systems have different
> > > > SCNs/Offsets/UUIDs/vectors.
> > > > >> There was discussion both on the mailing list and the SQL JIRAs
> > about
> > > > the
> > > > >> need for this.
> > > > >>
> > > > >> The same thing holds true for replayability. Kafka allows us to
> > rewind
> > > > >> when
> > > > >> we have a failure. Many other systems don't. In some cases,
> systems
> > > > return
> > > > >> null for their offsets (e.g. WikipediaSystemConsumer) because they
> > > have
> > > > no
> > > > >> offsets.
> > > > >>
> > > > >> Partitioning is another example. Kafka supports partitioning, but
> > many
> > > > >> systems don't. We model this by having a single partition for
> those
> > > > >> systems. Still, other systems model partitioning differently (e.g.
> > > > >> Kinesis).
> > > > >>
> > > > >> The SystemAdmin interface is also a mess. Creating streams in a
> > > > >> system-agnostic way is almost impossible. As is modeling metadata
> > for
> > > > the
> > > > >> system (replication factor, partitions, location, etc). The list
> > goes
> > > > on.
> > > > >>
> > > > >> Duplicate work
> > > > >>
> > > > >> At the time that we began writing Samza, Kafka's consumer and
> > producer
> > > > >> APIs
> > > > >> had a relatively weak feature set. On the consumer-side, you had
> two
> > > > >> options: use the high level consumer, or the simple consumer. The
> > > > problem
> > > > >> with the high-level consumer was that it controlled your offsets,
> > > > >> partition
> > > > >> assignments, and the order in which you received messages. The
> > problem
> > > > >> with
> > > > >> the simple consumer is that it's not simple. It's basic. You end
> up
> > > > having
> > > > >> to handle a lot of really low-level stuff that you shouldn't. We
> > > spent a
> > > > >> lot of time to make Samza's KafkaSystemConsumer very robust. It
> also
> > > > >> allows
> > > > >> us to support some cool features:
> > > > >>
> > > > >> * Per-partition message ordering and prioritization.
> > > > >> * Tight control over partition assignment to support joins, global
> > > state
> > > > >> (if we want to implement it :)), etc.
> > > > >> * Tight control over offset checkpointing.
> > > > >>
> > > > >> What we didn't realize at the time is that these features should
> > > > actually
> > > > >> be in Kafka. A lot of Kafka consumers (not just Samza stream
> > > processors)
> > > > >> end up wanting to do things like joins and partition assignment.
> The
> > > > Kafka
> > > > >> community has come to the same conclusion. They're adding a ton of
> > > > >> upgrades
> > > > >> into their new Kafka consumer implementation. To a large extent,
> > it's
> > > > >> duplicate work to what we've already done in Samza.
> > > > >>
> > > > >> On top of this, Kafka ended up taking a very similar approach to
> > > Samza's
> > > > >> KafkaCheckpointManager implementation for handling offset
> > > checkpointing.
> > > > >> Like Samza, Kafka's new offset management feature stores offset
> > > > >> checkpoints
> > > > >> in a topic, and allows you to fetch them from the broker.
> > > > >>
> > > > >> A lot of this seems like a waste, since we could have shared the
> > work
> > > if
> > > > >> it
> > > > >> had been done in Kafka from the get-go.
> > > > >>
> > > > >> Vision
> > > > >>
> > > > >> All of this leads me to a rather radical proposal. Samza is
> > relatively
> > > > >> stable at this point. I'd venture to say that we're near a 1.0
> > > release.
> > > > >> I'd
> > > > >> like to propose that we take what we've learned, and begin
> thinking
> > > > about
> > > > >> Samza beyond 1.0. What would we change if we were starting from
> > > scratch?
> > > > >> My
> > > > >> proposal is to:
> > > > >>
> > > > >> 1. Make Samza standalone the *only* way to run Samza processors,
> and
> > > > >> eliminate all direct dependences on YARN, Mesos, etc.
> > > > >> 2. Make a definitive call to support only Kafka as the stream
> > > processing
> > > > >> layer.
> > > > >> 3. Eliminate Samza's metrics, logging, serialization, and config
> > > > systems,
> > > > >> and simply use Kafka's instead.
> > > > >>
> > > > >> This would fix all of the issues that I outlined above. It should
> > also
> > > > >> shrink the Samza code base pretty dramatically. Supporting only a
> > > > >> standalone container will allow Samza to be executed on YARN
> (using
> > > > >> Slider), Mesos (using Marathon/Aurora), or most other in-house
> > > > deployment
> > > > >> systems. This should make life a lot easier for new users. Imagine
> > > > having
> > > > >> the hello-samza tutorial without YARN. The drop in mailing list
> > > traffic
> > > > >> will be pretty dramatic.
> > > > >>
> > > > >> Coupling with Kafka seems long overdue to me. The reality is,
> > everyone
> > > > >> that
> > > > >> I'm aware of is using Samza with Kafka. We basically require it
> > > already
> > > > in
> > > > >> order for most features to work. Those that are using other
> systems
> > > are
> > > > >> generally using it for ingest into Kafka (1), and then they do the
> > > > >> processing on top. There is already discussion (
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> > > > >> )
> > > > >> in Kafka to make ingesting into Kafka extremely easy.
> > > > >>
> > > > >> Once we make the call to couple with Kafka, we can leverage a ton
> of
> > > > their
> > > > >> ecosystem. We no longer have to maintain our own config, metrics,
> > etc.
> > > > We
> > > > >> can all share the same libraries, and make them better. This will
> > also
> > > > >> allow us to share the consumer/producer APIs, and will let us
> > leverage
> > > > >> their offset management and partition management, rather than
> having
> > > our
> > > > >> own. All of the coordinator stream code would go away, as would
> most
> > > of
> > > > >> the
> > > > >> YARN AppMaster code. We'd probably have to push some partition
> > > > management
> > > > >> features into the Kafka broker, but they're already moving in that
> > > > >> direction with the new consumer API. The features we have for
> > > partition
> > > > >> assignment aren't unique to Samza, and seem like they should be in
> > > Kafka
> > > > >> anyway. There will always be some niche usages which will require
> > > extra
> > > > >> care and hence full control over partition assignments much like
> the
> > > > Kafka
> > > > >> low level consumer api. These would continue to be supported.
> > > > >>
> > > > >> These items will be good for the Samza community. They'll make
> Samza
> > > > >> easier
> > > > >> to use, and make it easier for developers to add new features.
> > > > >>
> > > > >> Obviously this is a fairly large (and somewhat backwards
> > incompatible
> > > > >> change). If we choose to go this route, it's important that we
> > openly
> > > > >> communicate how we're going to provide a migration path from the
> > > > existing
> > > > >> APIs to the new ones (if we make incompatible changes). I think
> at a
> > > > >> minimum, we'd probably need to provide a wrapper to allow existing
> > > > >> StreamTask implementations to continue running on the new
> container.
> > > > It's
> > > > >> also important that we openly communicate about timing, and stages
> > of
> > > > the
> > > > >> migration.
> > > > >>
> > > > >> If you made it this far, I'm sure you have opinions. :) Please
> send
> > > your
> > > > >> thoughts and feedback.
> > > > >>
> > > > >> Cheers,
> > > > >> Chris
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Gianmarco De Francisci Morales <gd...@apache.org>.
Very interesting thoughts.
>From outside, I have always perceived Samza as a computing layer over Kafka.

The question, maybe a bit provocative, is "should Samza be a sub-project of
Kafka then?"
Or does it make sense to keep it as a separate project with a separate
governance?

Cheers,

--
Gianmarco

On 2 July 2015 at 08:59, Yan Fang <ya...@gmail.com> wrote:

> Overall, I agree to couple with Kafka more tightly. Because Samza de facto
> is based on Kafka, and it should leverage what Kafka has. At the same time,
> Kafka does not need to reinvent what Samza already has. I also like the
> idea of separating the ingestion and transformation.
>
> But it is a little difficult for me to image how the Samza will look like.
> And I feel Chris and Jay have a little difference in terms of how Samza
> should look like.
>
> *** Will it look like what Jay's code shows (A client of Kakfa) ? And
> user's application code calls this client?
>
> 1. If we make Samza be a library of Kafka (like what the code shows), how
> do we implement auto-balance and fault-tolerance? Are they taken care by
> the Kafka broker or other mechanism, such as "Samza worker" (just make up
> the name) ?
>
> 2. What about other features, such as auto-scaling, shared state,
> monitoring?
>
>
> *** If we have Samza standalone, (is this what Chris suggests?)
>
> 1. we still need to ingest data from Kakfa and produce to it. Then it
> becomes the same as what Samza looks like now, except it does not rely on
> Yarn anymore.
>
> 2. if it is standalone, how can it leverage Kafka's metrics, logs, etc? Use
> Kafka code as the dependency?
>
>
> Thanks,
>
> Fang, Yan
> yanfang724@gmail.com
>
> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Read through the code example and it looks good to me. A few thoughts
> > regarding deployment:
> >
> > Today Samza deploys as executable runnable like:
> >
> > deploy/samza/bin/run-job.sh --config-factory=... --config-path=file://...
> >
> > And this proposal advocate for deploying Samza more as embedded libraries
> > in user application code (ignoring the terminology since it is not the
> same
> > as the prototype code):
> >
> > StreamTask task = new MyStreamTask(configs);
> > Thread thread = new Thread(task);
> > thread.start();
> >
> > I think both of these deployment modes are important for different types
> of
> > users. That said, I think making Samza purely standalone is still
> > sufficient for either runnable or library modes.
> >
> > Guozhang
> >
> > On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Looks like gmail mangled the code example, it was supposed to look like
> > > this:
> > >
> > > Properties props = new Properties();
> > > props.put("bootstrap.servers", "localhost:4242");
> > > StreamingConfig config = new StreamingConfig(props);
> > > config.subscribe("test-topic-1", "test-topic-2");
> > > config.processor(ExampleStreamProcessor.class);
> > > config.serialization(new StringSerializer(), new StringDeserializer());
> > > KafkaStreaming container = new KafkaStreaming(config);
> > > container.run();
> > >
> > > -Jay
> > >
> > > On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io> wrote:
> > >
> > > > Hey guys,
> > > >
> > > > This came out of some conversations Chris and I were having around
> > > whether
> > > > it would make sense to use Samza as a kind of data ingestion
> framework
> > > for
> > > > Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> > combined
> > > > with complaints around config and YARN and the discussion around how
> to
> > > > best do a standalone mode.
> > > >
> > > > So the thought experiment was, given that Samza was basically already
> > > > totally Kafka specific, what if you just embraced that and turned it
> > into
> > > > something less like a heavyweight framework and more like a third
> Kafka
> > > > client--a kind of "producing consumer" with state management
> > facilities.
> > > > Basically a library. Instead of a complex stream processing framework
> > > this
> > > > would actually be a very simple thing, not much more complicated to
> use
> > > or
> > > > operate than a Kafka consumer. As Chris said we thought about it a
> lot
> > of
> > > > what Samza (and the other stream processing systems were doing)
> seemed
> > > like
> > > > kind of a hangover from MapReduce.
> > > >
> > > > Of course you need to ingest/output data to and from the stream
> > > > processing. But when we actually looked into how that would work,
> Samza
> > > > isn't really an ideal data ingestion framework for a bunch of
> reasons.
> > To
> > > > really do that right you need a pretty different internal data model
> > and
> > > > set of apis. So what if you split them and had an api for Kafka
> > > > ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > > > transformation (Samza).
> > > >
> > > > This would also allow really embracing the same terminology and
> > > > conventions. One complaint about the current state is that the two
> > > systems
> > > > kind of feel bolted on. Terminology like "stream" vs "topic" and
> > > different
> > > > config and monitoring systems means you kind of have to learn Kafka's
> > > way,
> > > > then learn Samza's slightly different way, then kind of understand
> how
> > > they
> > > > map to each other, which having walked a few people through this is
> > > > surprisingly tricky for folks to get.
> > > >
> > > > Since I have been spending a lot of time on airplanes I hacked up an
> > > > ernest but still somewhat incomplete prototype of what this would
> look
> > > > like. This is just unceremoniously dumped into Kafka as it required a
> > few
> > > > changes to the new consumer. Here is the code:
> > > >
> > > >
> > >
> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org/apache/kafka/clients/streaming
> > > >
> > > > For the purpose of the prototype I just liberally renamed everything
> to
> > > > try to align it with Kafka with no regard for compatibility.
> > > >
> > > > To use this would be something like this:
> > > > Properties props = new Properties(); props.put("bootstrap.servers",
> > > > "localhost:4242"); StreamingConfig config = new
> StreamingConfig(props);
> > > config.subscribe("test-topic-1",
> > > > "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > > config.serialization(new
> > > > StringSerializer(), new StringDeserializer()); KafkaStreaming
> > container =
> > > > new KafkaStreaming(config); container.run();
> > > >
> > > > KafkaStreaming is basically the SamzaContainer; StreamProcessor is
> > > > basically StreamTask.
> > > >
> > > > So rather than putting all the class names in a file and then having
> > the
> > > > job assembled by reflection, you just instantiate the container
> > > > programmatically. Work is balanced over however many instances of
> this
> > > are
> > > > alive at any time (i.e. if an instance dies, new tasks are added to
> the
> > > > existing containers without shutting them down).
> > > >
> > > > We would provide some glue for running this stuff in YARN via Slider,
> > > > Mesos via Marathon, and AWS using some of their tools but from the
> > point
> > > of
> > > > view of these frameworks these stream processing jobs are just
> > stateless
> > > > services that can come and go and expand and contract at will. There
> is
> > > no
> > > > more custom scheduler.
> > > >
> > > > Here are some relevant details:
> > > >
> > > >    1. It is only ~1300 lines of code, it would get larger if we
> > > >    productionized but not vastly larger. We really do get a ton of
> > > leverage
> > > >    out of Kafka.
> > > >    2. Partition management is fully delegated to the new consumer.
> This
> > > >    is nice since now any partition management strategy available to
> > Kafka
> > > >    consumer is also available to Samza (and vice versa) and with the
> > > exact
> > > >    same configs.
> > > >    3. It supports state as well as state reuse
> > > >
> > > > Anyhow take a look, hopefully it is thought provoking.
> > > >
> > > > -Jay
> > > >
> > > >
> > > >
> > > > On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> > criccomini@apache.org>
> > > > wrote:
> > > >
> > > >> Hey all,
> > > >>
> > > >> I have had some discussions with Samza engineers at LinkedIn and
> > > Confluent
> > > >> and we came up with a few observations and would like to propose
> some
> > > >> changes.
> > > >>
> > > >> We've observed some things that I want to call out about Samza's
> > design,
> > > >> and I'd like to propose some changes.
> > > >>
> > > >> * Samza is dependent upon a dynamic deployment system.
> > > >> * Samza is too pluggable.
> > > >> * Samza's SystemConsumer/SystemProducer and Kafka's consumer APIs
> are
> > > >> trying to solve a lot of the same problems.
> > > >>
> > > >> All three of these issues are related, but I'll address them in
> order.
> > > >>
> > > >> Deployment
> > > >>
> > > >> Samza strongly depends on the use of a dynamic deployment scheduler
> > such
> > > >> as
> > > >> YARN, Mesos, etc. When we initially built Samza, we bet that there
> > would
> > > >> be
> > > >> one or two winners in this area, and we could support them, and the
> > rest
> > > >> would go away. In reality, there are many variations. Furthermore,
> > many
> > > >> people still prefer to just start their processors like normal Java
> > > >> processes, and use traditional deployment scripts such as Fabric,
> > Chef,
> > > >> Ansible, etc. Forcing a deployment system on users makes the Samza
> > > >> start-up
> > > >> process really painful for first time users.
> > > >>
> > > >> Dynamic deployment as a requirement was also a bit of a mis-fire
> > because
> > > >> of
> > > >> a fundamental misunderstanding between the nature of batch jobs and
> > > stream
> > > >> processing jobs. Early on, we made conscious effort to favor the
> > Hadoop
> > > >> (Map/Reduce) way of doing things, since it worked and was well
> > > understood.
> > > >> One thing that we missed was that batch jobs have a definite
> > beginning,
> > > >> and
> > > >> end, and stream processing jobs don't (usually). This leads to a
> much
> > > >> simpler scheduling problem for stream processors. You basically just
> > > need
> > > >> to find a place to start the processor, and start it. The way we run
> > > >> grids,
> > > >> at LinkedIn, there's no concept of a cluster being "full". We always
> > add
> > > >> more machines. The problem with coupling Samza with a scheduler is
> > that
> > > >> Samza (as a framework) now has to handle deployment. This pulls in a
> > > bunch
> > > >> of things such as configuration distribution (config stream), shell
> > > scrips
> > > >> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> > > >>
> > > >> Another reason for requiring dynamic deployment was to support data
> > > >> locality. If you want to have locality, you need to put your
> > processors
> > > >> close to the data they're processing. Upon further investigation,
> > > though,
> > > >> this feature is not that beneficial. There is some good discussion
> > about
> > > >> some problems with it on SAMZA-335. Again, we took the Map/Reduce
> > path,
> > > >> but
> > > >> there are some fundamental differences between HDFS and Kafka. HDFS
> > has
> > > >> blocks, while Kafka has partitions. This leads to less optimization
> > > >> potential with stream processors on top of Kafka.
> > > >>
> > > >> This feature is also used as a crutch. Samza doesn't have any built
> in
> > > >> fault-tolerance logic. Instead, it depends on the dynamic deployment
> > > >> scheduling system to handle restarts when a processor dies. This has
> > > made
> > > >> it very difficult to write a standalone Samza container (SAMZA-516).
> > > >>
> > > >> Pluggability
> > > >>
> > > >> In some cases pluggability is good, but I think that we've gone too
> > far
> > > >> with it. Currently, Samza has:
> > > >>
> > > >> * Pluggable config.
> > > >> * Pluggable metrics.
> > > >> * Pluggable deployment systems.
> > > >> * Pluggable streaming systems (SystemConsumer, SystemProducer, etc).
> > > >> * Pluggable serdes.
> > > >> * Pluggable storage engines.
> > > >> * Pluggable strategies for just about every component
> (MessageChooser,
> > > >> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > > >>
> > > >> There's probably more that I've forgotten, as well. Some of these
> are
> > > >> useful, but some have proven not to be. This all comes at a cost:
> > > >> complexity. This complexity is making it harder for our users to
> pick
> > up
> > > >> and use Samza out of the box. It also makes it difficult for Samza
> > > >> developers to reason about what the characteristics of the container
> > > >> (since
> > > >> the characteristics change depending on which plugins are use).
> > > >>
> > > >> The issues with pluggability are most visible in the System APIs.
> What
> > > >> Samza really requires to be functional is Kafka as its transport
> > layer.
> > > >> But
> > > >> we've conflated two unrelated use cases into one API:
> > > >>
> > > >> 1. Get data into/out of Kafka.
> > > >> 2. Process the data in Kafka.
> > > >>
> > > >> The current System API supports both of these use cases. The problem
> > is,
> > > >> we
> > > >> actually want different features for each use case. By papering over
> > > these
> > > >> two use cases, and providing a single API, we've introduced a ton of
> > > leaky
> > > >> abstractions.
> > > >>
> > > >> For example, what we'd really like in (2) is to have monotonically
> > > >> increasing longs for offsets (like Kafka). This would be at odds
> with
> > > (1),
> > > >> though, since different systems have different
> > > SCNs/Offsets/UUIDs/vectors.
> > > >> There was discussion both on the mailing list and the SQL JIRAs
> about
> > > the
> > > >> need for this.
> > > >>
> > > >> The same thing holds true for replayability. Kafka allows us to
> rewind
> > > >> when
> > > >> we have a failure. Many other systems don't. In some cases, systems
> > > return
> > > >> null for their offsets (e.g. WikipediaSystemConsumer) because they
> > have
> > > no
> > > >> offsets.
> > > >>
> > > >> Partitioning is another example. Kafka supports partitioning, but
> many
> > > >> systems don't. We model this by having a single partition for those
> > > >> systems. Still, other systems model partitioning differently (e.g.
> > > >> Kinesis).
> > > >>
> > > >> The SystemAdmin interface is also a mess. Creating streams in a
> > > >> system-agnostic way is almost impossible. As is modeling metadata
> for
> > > the
> > > >> system (replication factor, partitions, location, etc). The list
> goes
> > > on.
> > > >>
> > > >> Duplicate work
> > > >>
> > > >> At the time that we began writing Samza, Kafka's consumer and
> producer
> > > >> APIs
> > > >> had a relatively weak feature set. On the consumer-side, you had two
> > > >> options: use the high level consumer, or the simple consumer. The
> > > problem
> > > >> with the high-level consumer was that it controlled your offsets,
> > > >> partition
> > > >> assignments, and the order in which you received messages. The
> problem
> > > >> with
> > > >> the simple consumer is that it's not simple. It's basic. You end up
> > > having
> > > >> to handle a lot of really low-level stuff that you shouldn't. We
> > spent a
> > > >> lot of time to make Samza's KafkaSystemConsumer very robust. It also
> > > >> allows
> > > >> us to support some cool features:
> > > >>
> > > >> * Per-partition message ordering and prioritization.
> > > >> * Tight control over partition assignment to support joins, global
> > state
> > > >> (if we want to implement it :)), etc.
> > > >> * Tight control over offset checkpointing.
> > > >>
> > > >> What we didn't realize at the time is that these features should
> > > actually
> > > >> be in Kafka. A lot of Kafka consumers (not just Samza stream
> > processors)
> > > >> end up wanting to do things like joins and partition assignment. The
> > > Kafka
> > > >> community has come to the same conclusion. They're adding a ton of
> > > >> upgrades
> > > >> into their new Kafka consumer implementation. To a large extent,
> it's
> > > >> duplicate work to what we've already done in Samza.
> > > >>
> > > >> On top of this, Kafka ended up taking a very similar approach to
> > Samza's
> > > >> KafkaCheckpointManager implementation for handling offset
> > checkpointing.
> > > >> Like Samza, Kafka's new offset management feature stores offset
> > > >> checkpoints
> > > >> in a topic, and allows you to fetch them from the broker.
> > > >>
> > > >> A lot of this seems like a waste, since we could have shared the
> work
> > if
> > > >> it
> > > >> had been done in Kafka from the get-go.
> > > >>
> > > >> Vision
> > > >>
> > > >> All of this leads me to a rather radical proposal. Samza is
> relatively
> > > >> stable at this point. I'd venture to say that we're near a 1.0
> > release.
> > > >> I'd
> > > >> like to propose that we take what we've learned, and begin thinking
> > > about
> > > >> Samza beyond 1.0. What would we change if we were starting from
> > scratch?
> > > >> My
> > > >> proposal is to:
> > > >>
> > > >> 1. Make Samza standalone the *only* way to run Samza processors, and
> > > >> eliminate all direct dependences on YARN, Mesos, etc.
> > > >> 2. Make a definitive call to support only Kafka as the stream
> > processing
> > > >> layer.
> > > >> 3. Eliminate Samza's metrics, logging, serialization, and config
> > > systems,
> > > >> and simply use Kafka's instead.
> > > >>
> > > >> This would fix all of the issues that I outlined above. It should
> also
> > > >> shrink the Samza code base pretty dramatically. Supporting only a
> > > >> standalone container will allow Samza to be executed on YARN (using
> > > >> Slider), Mesos (using Marathon/Aurora), or most other in-house
> > > deployment
> > > >> systems. This should make life a lot easier for new users. Imagine
> > > having
> > > >> the hello-samza tutorial without YARN. The drop in mailing list
> > traffic
> > > >> will be pretty dramatic.
> > > >>
> > > >> Coupling with Kafka seems long overdue to me. The reality is,
> everyone
> > > >> that
> > > >> I'm aware of is using Samza with Kafka. We basically require it
> > already
> > > in
> > > >> order for most features to work. Those that are using other systems
> > are
> > > >> generally using it for ingest into Kafka (1), and then they do the
> > > >> processing on top. There is already discussion (
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> > > >> )
> > > >> in Kafka to make ingesting into Kafka extremely easy.
> > > >>
> > > >> Once we make the call to couple with Kafka, we can leverage a ton of
> > > their
> > > >> ecosystem. We no longer have to maintain our own config, metrics,
> etc.
> > > We
> > > >> can all share the same libraries, and make them better. This will
> also
> > > >> allow us to share the consumer/producer APIs, and will let us
> leverage
> > > >> their offset management and partition management, rather than having
> > our
> > > >> own. All of the coordinator stream code would go away, as would most
> > of
> > > >> the
> > > >> YARN AppMaster code. We'd probably have to push some partition
> > > management
> > > >> features into the Kafka broker, but they're already moving in that
> > > >> direction with the new consumer API. The features we have for
> > partition
> > > >> assignment aren't unique to Samza, and seem like they should be in
> > Kafka
> > > >> anyway. There will always be some niche usages which will require
> > extra
> > > >> care and hence full control over partition assignments much like the
> > > Kafka
> > > >> low level consumer api. These would continue to be supported.
> > > >>
> > > >> These items will be good for the Samza community. They'll make Samza
> > > >> easier
> > > >> to use, and make it easier for developers to add new features.
> > > >>
> > > >> Obviously this is a fairly large (and somewhat backwards
> incompatible
> > > >> change). If we choose to go this route, it's important that we
> openly
> > > >> communicate how we're going to provide a migration path from the
> > > existing
> > > >> APIs to the new ones (if we make incompatible changes). I think at a
> > > >> minimum, we'd probably need to provide a wrapper to allow existing
> > > >> StreamTask implementations to continue running on the new container.
> > > It's
> > > >> also important that we openly communicate about timing, and stages
> of
> > > the
> > > >> migration.
> > > >>
> > > >> If you made it this far, I'm sure you have opinions. :) Please send
> > your
> > > >> thoughts and feedback.
> > > >>
> > > >> Cheers,
> > > >> Chris
> > > >>
> > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: Thoughts and obesrvations on Samza

Posted by Yan Fang <ya...@gmail.com>.
Overall, I agree to couple with Kafka more tightly. Because Samza de facto
is based on Kafka, and it should leverage what Kafka has. At the same time,
Kafka does not need to reinvent what Samza already has. I also like the
idea of separating the ingestion and transformation.

But it is a little difficult for me to image how the Samza will look like.
And I feel Chris and Jay have a little difference in terms of how Samza
should look like.

*** Will it look like what Jay's code shows (A client of Kakfa) ? And
user's application code calls this client?

1. If we make Samza be a library of Kafka (like what the code shows), how
do we implement auto-balance and fault-tolerance? Are they taken care by
the Kafka broker or other mechanism, such as "Samza worker" (just make up
the name) ?

2. What about other features, such as auto-scaling, shared state,
monitoring?


*** If we have Samza standalone, (is this what Chris suggests?)

1. we still need to ingest data from Kakfa and produce to it. Then it
becomes the same as what Samza looks like now, except it does not rely on
Yarn anymore.

2. if it is standalone, how can it leverage Kafka's metrics, logs, etc? Use
Kafka code as the dependency?


Thanks,

Fang, Yan
yanfang724@gmail.com

On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Read through the code example and it looks good to me. A few thoughts
> regarding deployment:
>
> Today Samza deploys as executable runnable like:
>
> deploy/samza/bin/run-job.sh --config-factory=... --config-path=file://...
>
> And this proposal advocate for deploying Samza more as embedded libraries
> in user application code (ignoring the terminology since it is not the same
> as the prototype code):
>
> StreamTask task = new MyStreamTask(configs);
> Thread thread = new Thread(task);
> thread.start();
>
> I think both of these deployment modes are important for different types of
> users. That said, I think making Samza purely standalone is still
> sufficient for either runnable or library modes.
>
> Guozhang
>
> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Looks like gmail mangled the code example, it was supposed to look like
> > this:
> >
> > Properties props = new Properties();
> > props.put("bootstrap.servers", "localhost:4242");
> > StreamingConfig config = new StreamingConfig(props);
> > config.subscribe("test-topic-1", "test-topic-2");
> > config.processor(ExampleStreamProcessor.class);
> > config.serialization(new StringSerializer(), new StringDeserializer());
> > KafkaStreaming container = new KafkaStreaming(config);
> > container.run();
> >
> > -Jay
> >
> > On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Hey guys,
> > >
> > > This came out of some conversations Chris and I were having around
> > whether
> > > it would make sense to use Samza as a kind of data ingestion framework
> > for
> > > Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> combined
> > > with complaints around config and YARN and the discussion around how to
> > > best do a standalone mode.
> > >
> > > So the thought experiment was, given that Samza was basically already
> > > totally Kafka specific, what if you just embraced that and turned it
> into
> > > something less like a heavyweight framework and more like a third Kafka
> > > client--a kind of "producing consumer" with state management
> facilities.
> > > Basically a library. Instead of a complex stream processing framework
> > this
> > > would actually be a very simple thing, not much more complicated to use
> > or
> > > operate than a Kafka consumer. As Chris said we thought about it a lot
> of
> > > what Samza (and the other stream processing systems were doing) seemed
> > like
> > > kind of a hangover from MapReduce.
> > >
> > > Of course you need to ingest/output data to and from the stream
> > > processing. But when we actually looked into how that would work, Samza
> > > isn't really an ideal data ingestion framework for a bunch of reasons.
> To
> > > really do that right you need a pretty different internal data model
> and
> > > set of apis. So what if you split them and had an api for Kafka
> > > ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > > transformation (Samza).
> > >
> > > This would also allow really embracing the same terminology and
> > > conventions. One complaint about the current state is that the two
> > systems
> > > kind of feel bolted on. Terminology like "stream" vs "topic" and
> > different
> > > config and monitoring systems means you kind of have to learn Kafka's
> > way,
> > > then learn Samza's slightly different way, then kind of understand how
> > they
> > > map to each other, which having walked a few people through this is
> > > surprisingly tricky for folks to get.
> > >
> > > Since I have been spending a lot of time on airplanes I hacked up an
> > > ernest but still somewhat incomplete prototype of what this would look
> > > like. This is just unceremoniously dumped into Kafka as it required a
> few
> > > changes to the new consumer. Here is the code:
> > >
> > >
> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org/apache/kafka/clients/streaming
> > >
> > > For the purpose of the prototype I just liberally renamed everything to
> > > try to align it with Kafka with no regard for compatibility.
> > >
> > > To use this would be something like this:
> > > Properties props = new Properties(); props.put("bootstrap.servers",
> > > "localhost:4242"); StreamingConfig config = new StreamingConfig(props);
> > config.subscribe("test-topic-1",
> > > "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > config.serialization(new
> > > StringSerializer(), new StringDeserializer()); KafkaStreaming
> container =
> > > new KafkaStreaming(config); container.run();
> > >
> > > KafkaStreaming is basically the SamzaContainer; StreamProcessor is
> > > basically StreamTask.
> > >
> > > So rather than putting all the class names in a file and then having
> the
> > > job assembled by reflection, you just instantiate the container
> > > programmatically. Work is balanced over however many instances of this
> > are
> > > alive at any time (i.e. if an instance dies, new tasks are added to the
> > > existing containers without shutting them down).
> > >
> > > We would provide some glue for running this stuff in YARN via Slider,
> > > Mesos via Marathon, and AWS using some of their tools but from the
> point
> > of
> > > view of these frameworks these stream processing jobs are just
> stateless
> > > services that can come and go and expand and contract at will. There is
> > no
> > > more custom scheduler.
> > >
> > > Here are some relevant details:
> > >
> > >    1. It is only ~1300 lines of code, it would get larger if we
> > >    productionized but not vastly larger. We really do get a ton of
> > leverage
> > >    out of Kafka.
> > >    2. Partition management is fully delegated to the new consumer. This
> > >    is nice since now any partition management strategy available to
> Kafka
> > >    consumer is also available to Samza (and vice versa) and with the
> > exact
> > >    same configs.
> > >    3. It supports state as well as state reuse
> > >
> > > Anyhow take a look, hopefully it is thought provoking.
> > >
> > > -Jay
> > >
> > >
> > >
> > > On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> criccomini@apache.org>
> > > wrote:
> > >
> > >> Hey all,
> > >>
> > >> I have had some discussions with Samza engineers at LinkedIn and
> > Confluent
> > >> and we came up with a few observations and would like to propose some
> > >> changes.
> > >>
> > >> We've observed some things that I want to call out about Samza's
> design,
> > >> and I'd like to propose some changes.
> > >>
> > >> * Samza is dependent upon a dynamic deployment system.
> > >> * Samza is too pluggable.
> > >> * Samza's SystemConsumer/SystemProducer and Kafka's consumer APIs are
> > >> trying to solve a lot of the same problems.
> > >>
> > >> All three of these issues are related, but I'll address them in order.
> > >>
> > >> Deployment
> > >>
> > >> Samza strongly depends on the use of a dynamic deployment scheduler
> such
> > >> as
> > >> YARN, Mesos, etc. When we initially built Samza, we bet that there
> would
> > >> be
> > >> one or two winners in this area, and we could support them, and the
> rest
> > >> would go away. In reality, there are many variations. Furthermore,
> many
> > >> people still prefer to just start their processors like normal Java
> > >> processes, and use traditional deployment scripts such as Fabric,
> Chef,
> > >> Ansible, etc. Forcing a deployment system on users makes the Samza
> > >> start-up
> > >> process really painful for first time users.
> > >>
> > >> Dynamic deployment as a requirement was also a bit of a mis-fire
> because
> > >> of
> > >> a fundamental misunderstanding between the nature of batch jobs and
> > stream
> > >> processing jobs. Early on, we made conscious effort to favor the
> Hadoop
> > >> (Map/Reduce) way of doing things, since it worked and was well
> > understood.
> > >> One thing that we missed was that batch jobs have a definite
> beginning,
> > >> and
> > >> end, and stream processing jobs don't (usually). This leads to a much
> > >> simpler scheduling problem for stream processors. You basically just
> > need
> > >> to find a place to start the processor, and start it. The way we run
> > >> grids,
> > >> at LinkedIn, there's no concept of a cluster being "full". We always
> add
> > >> more machines. The problem with coupling Samza with a scheduler is
> that
> > >> Samza (as a framework) now has to handle deployment. This pulls in a
> > bunch
> > >> of things such as configuration distribution (config stream), shell
> > scrips
> > >> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> > >>
> > >> Another reason for requiring dynamic deployment was to support data
> > >> locality. If you want to have locality, you need to put your
> processors
> > >> close to the data they're processing. Upon further investigation,
> > though,
> > >> this feature is not that beneficial. There is some good discussion
> about
> > >> some problems with it on SAMZA-335. Again, we took the Map/Reduce
> path,
> > >> but
> > >> there are some fundamental differences between HDFS and Kafka. HDFS
> has
> > >> blocks, while Kafka has partitions. This leads to less optimization
> > >> potential with stream processors on top of Kafka.
> > >>
> > >> This feature is also used as a crutch. Samza doesn't have any built in
> > >> fault-tolerance logic. Instead, it depends on the dynamic deployment
> > >> scheduling system to handle restarts when a processor dies. This has
> > made
> > >> it very difficult to write a standalone Samza container (SAMZA-516).
> > >>
> > >> Pluggability
> > >>
> > >> In some cases pluggability is good, but I think that we've gone too
> far
> > >> with it. Currently, Samza has:
> > >>
> > >> * Pluggable config.
> > >> * Pluggable metrics.
> > >> * Pluggable deployment systems.
> > >> * Pluggable streaming systems (SystemConsumer, SystemProducer, etc).
> > >> * Pluggable serdes.
> > >> * Pluggable storage engines.
> > >> * Pluggable strategies for just about every component (MessageChooser,
> > >> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > >>
> > >> There's probably more that I've forgotten, as well. Some of these are
> > >> useful, but some have proven not to be. This all comes at a cost:
> > >> complexity. This complexity is making it harder for our users to pick
> up
> > >> and use Samza out of the box. It also makes it difficult for Samza
> > >> developers to reason about what the characteristics of the container
> > >> (since
> > >> the characteristics change depending on which plugins are use).
> > >>
> > >> The issues with pluggability are most visible in the System APIs. What
> > >> Samza really requires to be functional is Kafka as its transport
> layer.
> > >> But
> > >> we've conflated two unrelated use cases into one API:
> > >>
> > >> 1. Get data into/out of Kafka.
> > >> 2. Process the data in Kafka.
> > >>
> > >> The current System API supports both of these use cases. The problem
> is,
> > >> we
> > >> actually want different features for each use case. By papering over
> > these
> > >> two use cases, and providing a single API, we've introduced a ton of
> > leaky
> > >> abstractions.
> > >>
> > >> For example, what we'd really like in (2) is to have monotonically
> > >> increasing longs for offsets (like Kafka). This would be at odds with
> > (1),
> > >> though, since different systems have different
> > SCNs/Offsets/UUIDs/vectors.
> > >> There was discussion both on the mailing list and the SQL JIRAs about
> > the
> > >> need for this.
> > >>
> > >> The same thing holds true for replayability. Kafka allows us to rewind
> > >> when
> > >> we have a failure. Many other systems don't. In some cases, systems
> > return
> > >> null for their offsets (e.g. WikipediaSystemConsumer) because they
> have
> > no
> > >> offsets.
> > >>
> > >> Partitioning is another example. Kafka supports partitioning, but many
> > >> systems don't. We model this by having a single partition for those
> > >> systems. Still, other systems model partitioning differently (e.g.
> > >> Kinesis).
> > >>
> > >> The SystemAdmin interface is also a mess. Creating streams in a
> > >> system-agnostic way is almost impossible. As is modeling metadata for
> > the
> > >> system (replication factor, partitions, location, etc). The list goes
> > on.
> > >>
> > >> Duplicate work
> > >>
> > >> At the time that we began writing Samza, Kafka's consumer and producer
> > >> APIs
> > >> had a relatively weak feature set. On the consumer-side, you had two
> > >> options: use the high level consumer, or the simple consumer. The
> > problem
> > >> with the high-level consumer was that it controlled your offsets,
> > >> partition
> > >> assignments, and the order in which you received messages. The problem
> > >> with
> > >> the simple consumer is that it's not simple. It's basic. You end up
> > having
> > >> to handle a lot of really low-level stuff that you shouldn't. We
> spent a
> > >> lot of time to make Samza's KafkaSystemConsumer very robust. It also
> > >> allows
> > >> us to support some cool features:
> > >>
> > >> * Per-partition message ordering and prioritization.
> > >> * Tight control over partition assignment to support joins, global
> state
> > >> (if we want to implement it :)), etc.
> > >> * Tight control over offset checkpointing.
> > >>
> > >> What we didn't realize at the time is that these features should
> > actually
> > >> be in Kafka. A lot of Kafka consumers (not just Samza stream
> processors)
> > >> end up wanting to do things like joins and partition assignment. The
> > Kafka
> > >> community has come to the same conclusion. They're adding a ton of
> > >> upgrades
> > >> into their new Kafka consumer implementation. To a large extent, it's
> > >> duplicate work to what we've already done in Samza.
> > >>
> > >> On top of this, Kafka ended up taking a very similar approach to
> Samza's
> > >> KafkaCheckpointManager implementation for handling offset
> checkpointing.
> > >> Like Samza, Kafka's new offset management feature stores offset
> > >> checkpoints
> > >> in a topic, and allows you to fetch them from the broker.
> > >>
> > >> A lot of this seems like a waste, since we could have shared the work
> if
> > >> it
> > >> had been done in Kafka from the get-go.
> > >>
> > >> Vision
> > >>
> > >> All of this leads me to a rather radical proposal. Samza is relatively
> > >> stable at this point. I'd venture to say that we're near a 1.0
> release.
> > >> I'd
> > >> like to propose that we take what we've learned, and begin thinking
> > about
> > >> Samza beyond 1.0. What would we change if we were starting from
> scratch?
> > >> My
> > >> proposal is to:
> > >>
> > >> 1. Make Samza standalone the *only* way to run Samza processors, and
> > >> eliminate all direct dependences on YARN, Mesos, etc.
> > >> 2. Make a definitive call to support only Kafka as the stream
> processing
> > >> layer.
> > >> 3. Eliminate Samza's metrics, logging, serialization, and config
> > systems,
> > >> and simply use Kafka's instead.
> > >>
> > >> This would fix all of the issues that I outlined above. It should also
> > >> shrink the Samza code base pretty dramatically. Supporting only a
> > >> standalone container will allow Samza to be executed on YARN (using
> > >> Slider), Mesos (using Marathon/Aurora), or most other in-house
> > deployment
> > >> systems. This should make life a lot easier for new users. Imagine
> > having
> > >> the hello-samza tutorial without YARN. The drop in mailing list
> traffic
> > >> will be pretty dramatic.
> > >>
> > >> Coupling with Kafka seems long overdue to me. The reality is, everyone
> > >> that
> > >> I'm aware of is using Samza with Kafka. We basically require it
> already
> > in
> > >> order for most features to work. Those that are using other systems
> are
> > >> generally using it for ingest into Kafka (1), and then they do the
> > >> processing on top. There is already discussion (
> > >>
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> > >> )
> > >> in Kafka to make ingesting into Kafka extremely easy.
> > >>
> > >> Once we make the call to couple with Kafka, we can leverage a ton of
> > their
> > >> ecosystem. We no longer have to maintain our own config, metrics, etc.
> > We
> > >> can all share the same libraries, and make them better. This will also
> > >> allow us to share the consumer/producer APIs, and will let us leverage
> > >> their offset management and partition management, rather than having
> our
> > >> own. All of the coordinator stream code would go away, as would most
> of
> > >> the
> > >> YARN AppMaster code. We'd probably have to push some partition
> > management
> > >> features into the Kafka broker, but they're already moving in that
> > >> direction with the new consumer API. The features we have for
> partition
> > >> assignment aren't unique to Samza, and seem like they should be in
> Kafka
> > >> anyway. There will always be some niche usages which will require
> extra
> > >> care and hence full control over partition assignments much like the
> > Kafka
> > >> low level consumer api. These would continue to be supported.
> > >>
> > >> These items will be good for the Samza community. They'll make Samza
> > >> easier
> > >> to use, and make it easier for developers to add new features.
> > >>
> > >> Obviously this is a fairly large (and somewhat backwards incompatible
> > >> change). If we choose to go this route, it's important that we openly
> > >> communicate how we're going to provide a migration path from the
> > existing
> > >> APIs to the new ones (if we make incompatible changes). I think at a
> > >> minimum, we'd probably need to provide a wrapper to allow existing
> > >> StreamTask implementations to continue running on the new container.
> > It's
> > >> also important that we openly communicate about timing, and stages of
> > the
> > >> migration.
> > >>
> > >> If you made it this far, I'm sure you have opinions. :) Please send
> your
> > >> thoughts and feedback.
> > >>
> > >> Cheers,
> > >> Chris
> > >>
> > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: Thoughts and obesrvations on Samza

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

Yeah I agree. Being able to run in YARN/Mesos is definitely doable and
perhaps easier. Having a generic command line run script should be possible
too but the question is how the wiring would work (e.g. how the config maps
to instantiated java objects). The current mechanism is pretty hard for
people to use and I think ultimately harder than just writing a ten-line
main method.

-Jay

On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Read through the code example and it looks good to me. A few thoughts
> regarding deployment:
>
> Today Samza deploys as executable runnable like:
>
> deploy/samza/bin/run-job.sh --config-factory=... --config-path=file://...
>
> And this proposal advocate for deploying Samza more as embedded libraries
> in user application code (ignoring the terminology since it is not the same
> as the prototype code):
>
> StreamTask task = new MyStreamTask(configs);
> Thread thread = new Thread(task);
> thread.start();
>
> I think both of these deployment modes are important for different types of
> users. That said, I think making Samza purely standalone is still
> sufficient for either runnable or library modes.
>
> Guozhang
>
> On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Looks like gmail mangled the code example, it was supposed to look like
> > this:
> >
> > Properties props = new Properties();
> > props.put("bootstrap.servers", "localhost:4242");
> > StreamingConfig config = new StreamingConfig(props);
> > config.subscribe("test-topic-1", "test-topic-2");
> > config.processor(ExampleStreamProcessor.class);
> > config.serialization(new StringSerializer(), new StringDeserializer());
> > KafkaStreaming container = new KafkaStreaming(config);
> > container.run();
> >
> > -Jay
> >
> > On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io> wrote:
> >
> > > Hey guys,
> > >
> > > This came out of some conversations Chris and I were having around
> > whether
> > > it would make sense to use Samza as a kind of data ingestion framework
> > for
> > > Kafka (which ultimately lead to KIP-26 "copycat"). This kind of
> combined
> > > with complaints around config and YARN and the discussion around how to
> > > best do a standalone mode.
> > >
> > > So the thought experiment was, given that Samza was basically already
> > > totally Kafka specific, what if you just embraced that and turned it
> into
> > > something less like a heavyweight framework and more like a third Kafka
> > > client--a kind of "producing consumer" with state management
> facilities.
> > > Basically a library. Instead of a complex stream processing framework
> > this
> > > would actually be a very simple thing, not much more complicated to use
> > or
> > > operate than a Kafka consumer. As Chris said we thought about it a lot
> of
> > > what Samza (and the other stream processing systems were doing) seemed
> > like
> > > kind of a hangover from MapReduce.
> > >
> > > Of course you need to ingest/output data to and from the stream
> > > processing. But when we actually looked into how that would work, Samza
> > > isn't really an ideal data ingestion framework for a bunch of reasons.
> To
> > > really do that right you need a pretty different internal data model
> and
> > > set of apis. So what if you split them and had an api for Kafka
> > > ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > > transformation (Samza).
> > >
> > > This would also allow really embracing the same terminology and
> > > conventions. One complaint about the current state is that the two
> > systems
> > > kind of feel bolted on. Terminology like "stream" vs "topic" and
> > different
> > > config and monitoring systems means you kind of have to learn Kafka's
> > way,
> > > then learn Samza's slightly different way, then kind of understand how
> > they
> > > map to each other, which having walked a few people through this is
> > > surprisingly tricky for folks to get.
> > >
> > > Since I have been spending a lot of time on airplanes I hacked up an
> > > ernest but still somewhat incomplete prototype of what this would look
> > > like. This is just unceremoniously dumped into Kafka as it required a
> few
> > > changes to the new consumer. Here is the code:
> > >
> > >
> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org/apache/kafka/clients/streaming
> > >
> > > For the purpose of the prototype I just liberally renamed everything to
> > > try to align it with Kafka with no regard for compatibility.
> > >
> > > To use this would be something like this:
> > > Properties props = new Properties(); props.put("bootstrap.servers",
> > > "localhost:4242"); StreamingConfig config = new StreamingConfig(props);
> > config.subscribe("test-topic-1",
> > > "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> > config.serialization(new
> > > StringSerializer(), new StringDeserializer()); KafkaStreaming
> container =
> > > new KafkaStreaming(config); container.run();
> > >
> > > KafkaStreaming is basically the SamzaContainer; StreamProcessor is
> > > basically StreamTask.
> > >
> > > So rather than putting all the class names in a file and then having
> the
> > > job assembled by reflection, you just instantiate the container
> > > programmatically. Work is balanced over however many instances of this
> > are
> > > alive at any time (i.e. if an instance dies, new tasks are added to the
> > > existing containers without shutting them down).
> > >
> > > We would provide some glue for running this stuff in YARN via Slider,
> > > Mesos via Marathon, and AWS using some of their tools but from the
> point
> > of
> > > view of these frameworks these stream processing jobs are just
> stateless
> > > services that can come and go and expand and contract at will. There is
> > no
> > > more custom scheduler.
> > >
> > > Here are some relevant details:
> > >
> > >    1. It is only ~1300 lines of code, it would get larger if we
> > >    productionized but not vastly larger. We really do get a ton of
> > leverage
> > >    out of Kafka.
> > >    2. Partition management is fully delegated to the new consumer. This
> > >    is nice since now any partition management strategy available to
> Kafka
> > >    consumer is also available to Samza (and vice versa) and with the
> > exact
> > >    same configs.
> > >    3. It supports state as well as state reuse
> > >
> > > Anyhow take a look, hopefully it is thought provoking.
> > >
> > > -Jay
> > >
> > >
> > >
> > > On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <
> criccomini@apache.org>
> > > wrote:
> > >
> > >> Hey all,
> > >>
> > >> I have had some discussions with Samza engineers at LinkedIn and
> > Confluent
> > >> and we came up with a few observations and would like to propose some
> > >> changes.
> > >>
> > >> We've observed some things that I want to call out about Samza's
> design,
> > >> and I'd like to propose some changes.
> > >>
> > >> * Samza is dependent upon a dynamic deployment system.
> > >> * Samza is too pluggable.
> > >> * Samza's SystemConsumer/SystemProducer and Kafka's consumer APIs are
> > >> trying to solve a lot of the same problems.
> > >>
> > >> All three of these issues are related, but I'll address them in order.
> > >>
> > >> Deployment
> > >>
> > >> Samza strongly depends on the use of a dynamic deployment scheduler
> such
> > >> as
> > >> YARN, Mesos, etc. When we initially built Samza, we bet that there
> would
> > >> be
> > >> one or two winners in this area, and we could support them, and the
> rest
> > >> would go away. In reality, there are many variations. Furthermore,
> many
> > >> people still prefer to just start their processors like normal Java
> > >> processes, and use traditional deployment scripts such as Fabric,
> Chef,
> > >> Ansible, etc. Forcing a deployment system on users makes the Samza
> > >> start-up
> > >> process really painful for first time users.
> > >>
> > >> Dynamic deployment as a requirement was also a bit of a mis-fire
> because
> > >> of
> > >> a fundamental misunderstanding between the nature of batch jobs and
> > stream
> > >> processing jobs. Early on, we made conscious effort to favor the
> Hadoop
> > >> (Map/Reduce) way of doing things, since it worked and was well
> > understood.
> > >> One thing that we missed was that batch jobs have a definite
> beginning,
> > >> and
> > >> end, and stream processing jobs don't (usually). This leads to a much
> > >> simpler scheduling problem for stream processors. You basically just
> > need
> > >> to find a place to start the processor, and start it. The way we run
> > >> grids,
> > >> at LinkedIn, there's no concept of a cluster being "full". We always
> add
> > >> more machines. The problem with coupling Samza with a scheduler is
> that
> > >> Samza (as a framework) now has to handle deployment. This pulls in a
> > bunch
> > >> of things such as configuration distribution (config stream), shell
> > scrips
> > >> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> > >>
> > >> Another reason for requiring dynamic deployment was to support data
> > >> locality. If you want to have locality, you need to put your
> processors
> > >> close to the data they're processing. Upon further investigation,
> > though,
> > >> this feature is not that beneficial. There is some good discussion
> about
> > >> some problems with it on SAMZA-335. Again, we took the Map/Reduce
> path,
> > >> but
> > >> there are some fundamental differences between HDFS and Kafka. HDFS
> has
> > >> blocks, while Kafka has partitions. This leads to less optimization
> > >> potential with stream processors on top of Kafka.
> > >>
> > >> This feature is also used as a crutch. Samza doesn't have any built in
> > >> fault-tolerance logic. Instead, it depends on the dynamic deployment
> > >> scheduling system to handle restarts when a processor dies. This has
> > made
> > >> it very difficult to write a standalone Samza container (SAMZA-516).
> > >>
> > >> Pluggability
> > >>
> > >> In some cases pluggability is good, but I think that we've gone too
> far
> > >> with it. Currently, Samza has:
> > >>
> > >> * Pluggable config.
> > >> * Pluggable metrics.
> > >> * Pluggable deployment systems.
> > >> * Pluggable streaming systems (SystemConsumer, SystemProducer, etc).
> > >> * Pluggable serdes.
> > >> * Pluggable storage engines.
> > >> * Pluggable strategies for just about every component (MessageChooser,
> > >> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> > >>
> > >> There's probably more that I've forgotten, as well. Some of these are
> > >> useful, but some have proven not to be. This all comes at a cost:
> > >> complexity. This complexity is making it harder for our users to pick
> up
> > >> and use Samza out of the box. It also makes it difficult for Samza
> > >> developers to reason about what the characteristics of the container
> > >> (since
> > >> the characteristics change depending on which plugins are use).
> > >>
> > >> The issues with pluggability are most visible in the System APIs. What
> > >> Samza really requires to be functional is Kafka as its transport
> layer.
> > >> But
> > >> we've conflated two unrelated use cases into one API:
> > >>
> > >> 1. Get data into/out of Kafka.
> > >> 2. Process the data in Kafka.
> > >>
> > >> The current System API supports both of these use cases. The problem
> is,
> > >> we
> > >> actually want different features for each use case. By papering over
> > these
> > >> two use cases, and providing a single API, we've introduced a ton of
> > leaky
> > >> abstractions.
> > >>
> > >> For example, what we'd really like in (2) is to have monotonically
> > >> increasing longs for offsets (like Kafka). This would be at odds with
> > (1),
> > >> though, since different systems have different
> > SCNs/Offsets/UUIDs/vectors.
> > >> There was discussion both on the mailing list and the SQL JIRAs about
> > the
> > >> need for this.
> > >>
> > >> The same thing holds true for replayability. Kafka allows us to rewind
> > >> when
> > >> we have a failure. Many other systems don't. In some cases, systems
> > return
> > >> null for their offsets (e.g. WikipediaSystemConsumer) because they
> have
> > no
> > >> offsets.
> > >>
> > >> Partitioning is another example. Kafka supports partitioning, but many
> > >> systems don't. We model this by having a single partition for those
> > >> systems. Still, other systems model partitioning differently (e.g.
> > >> Kinesis).
> > >>
> > >> The SystemAdmin interface is also a mess. Creating streams in a
> > >> system-agnostic way is almost impossible. As is modeling metadata for
> > the
> > >> system (replication factor, partitions, location, etc). The list goes
> > on.
> > >>
> > >> Duplicate work
> > >>
> > >> At the time that we began writing Samza, Kafka's consumer and producer
> > >> APIs
> > >> had a relatively weak feature set. On the consumer-side, you had two
> > >> options: use the high level consumer, or the simple consumer. The
> > problem
> > >> with the high-level consumer was that it controlled your offsets,
> > >> partition
> > >> assignments, and the order in which you received messages. The problem
> > >> with
> > >> the simple consumer is that it's not simple. It's basic. You end up
> > having
> > >> to handle a lot of really low-level stuff that you shouldn't. We
> spent a
> > >> lot of time to make Samza's KafkaSystemConsumer very robust. It also
> > >> allows
> > >> us to support some cool features:
> > >>
> > >> * Per-partition message ordering and prioritization.
> > >> * Tight control over partition assignment to support joins, global
> state
> > >> (if we want to implement it :)), etc.
> > >> * Tight control over offset checkpointing.
> > >>
> > >> What we didn't realize at the time is that these features should
> > actually
> > >> be in Kafka. A lot of Kafka consumers (not just Samza stream
> processors)
> > >> end up wanting to do things like joins and partition assignment. The
> > Kafka
> > >> community has come to the same conclusion. They're adding a ton of
> > >> upgrades
> > >> into their new Kafka consumer implementation. To a large extent, it's
> > >> duplicate work to what we've already done in Samza.
> > >>
> > >> On top of this, Kafka ended up taking a very similar approach to
> Samza's
> > >> KafkaCheckpointManager implementation for handling offset
> checkpointing.
> > >> Like Samza, Kafka's new offset management feature stores offset
> > >> checkpoints
> > >> in a topic, and allows you to fetch them from the broker.
> > >>
> > >> A lot of this seems like a waste, since we could have shared the work
> if
> > >> it
> > >> had been done in Kafka from the get-go.
> > >>
> > >> Vision
> > >>
> > >> All of this leads me to a rather radical proposal. Samza is relatively
> > >> stable at this point. I'd venture to say that we're near a 1.0
> release.
> > >> I'd
> > >> like to propose that we take what we've learned, and begin thinking
> > about
> > >> Samza beyond 1.0. What would we change if we were starting from
> scratch?
> > >> My
> > >> proposal is to:
> > >>
> > >> 1. Make Samza standalone the *only* way to run Samza processors, and
> > >> eliminate all direct dependences on YARN, Mesos, etc.
> > >> 2. Make a definitive call to support only Kafka as the stream
> processing
> > >> layer.
> > >> 3. Eliminate Samza's metrics, logging, serialization, and config
> > systems,
> > >> and simply use Kafka's instead.
> > >>
> > >> This would fix all of the issues that I outlined above. It should also
> > >> shrink the Samza code base pretty dramatically. Supporting only a
> > >> standalone container will allow Samza to be executed on YARN (using
> > >> Slider), Mesos (using Marathon/Aurora), or most other in-house
> > deployment
> > >> systems. This should make life a lot easier for new users. Imagine
> > having
> > >> the hello-samza tutorial without YARN. The drop in mailing list
> traffic
> > >> will be pretty dramatic.
> > >>
> > >> Coupling with Kafka seems long overdue to me. The reality is, everyone
> > >> that
> > >> I'm aware of is using Samza with Kafka. We basically require it
> already
> > in
> > >> order for most features to work. Those that are using other systems
> are
> > >> generally using it for ingest into Kafka (1), and then they do the
> > >> processing on top. There is already discussion (
> > >>
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> > >> )
> > >> in Kafka to make ingesting into Kafka extremely easy.
> > >>
> > >> Once we make the call to couple with Kafka, we can leverage a ton of
> > their
> > >> ecosystem. We no longer have to maintain our own config, metrics, etc.
> > We
> > >> can all share the same libraries, and make them better. This will also
> > >> allow us to share the consumer/producer APIs, and will let us leverage
> > >> their offset management and partition management, rather than having
> our
> > >> own. All of the coordinator stream code would go away, as would most
> of
> > >> the
> > >> YARN AppMaster code. We'd probably have to push some partition
> > management
> > >> features into the Kafka broker, but they're already moving in that
> > >> direction with the new consumer API. The features we have for
> partition
> > >> assignment aren't unique to Samza, and seem like they should be in
> Kafka
> > >> anyway. There will always be some niche usages which will require
> extra
> > >> care and hence full control over partition assignments much like the
> > Kafka
> > >> low level consumer api. These would continue to be supported.
> > >>
> > >> These items will be good for the Samza community. They'll make Samza
> > >> easier
> > >> to use, and make it easier for developers to add new features.
> > >>
> > >> Obviously this is a fairly large (and somewhat backwards incompatible
> > >> change). If we choose to go this route, it's important that we openly
> > >> communicate how we're going to provide a migration path from the
> > existing
> > >> APIs to the new ones (if we make incompatible changes). I think at a
> > >> minimum, we'd probably need to provide a wrapper to allow existing
> > >> StreamTask implementations to continue running on the new container.
> > It's
> > >> also important that we openly communicate about timing, and stages of
> > the
> > >> migration.
> > >>
> > >> If you made it this far, I'm sure you have opinions. :) Please send
> your
> > >> thoughts and feedback.
> > >>
> > >> Cheers,
> > >> Chris
> > >>
> > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: Thoughts and obesrvations on Samza

Posted by Guozhang Wang <wa...@gmail.com>.
Read through the code example and it looks good to me. A few thoughts
regarding deployment:

Today Samza deploys as executable runnable like:

deploy/samza/bin/run-job.sh --config-factory=... --config-path=file://...

And this proposal advocate for deploying Samza more as embedded libraries
in user application code (ignoring the terminology since it is not the same
as the prototype code):

StreamTask task = new MyStreamTask(configs);
Thread thread = new Thread(task);
thread.start();

I think both of these deployment modes are important for different types of
users. That said, I think making Samza purely standalone is still
sufficient for either runnable or library modes.

Guozhang

On Tue, Jun 30, 2015 at 11:33 PM, Jay Kreps <ja...@confluent.io> wrote:

> Looks like gmail mangled the code example, it was supposed to look like
> this:
>
> Properties props = new Properties();
> props.put("bootstrap.servers", "localhost:4242");
> StreamingConfig config = new StreamingConfig(props);
> config.subscribe("test-topic-1", "test-topic-2");
> config.processor(ExampleStreamProcessor.class);
> config.serialization(new StringSerializer(), new StringDeserializer());
> KafkaStreaming container = new KafkaStreaming(config);
> container.run();
>
> -Jay
>
> On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io> wrote:
>
> > Hey guys,
> >
> > This came out of some conversations Chris and I were having around
> whether
> > it would make sense to use Samza as a kind of data ingestion framework
> for
> > Kafka (which ultimately lead to KIP-26 "copycat"). This kind of combined
> > with complaints around config and YARN and the discussion around how to
> > best do a standalone mode.
> >
> > So the thought experiment was, given that Samza was basically already
> > totally Kafka specific, what if you just embraced that and turned it into
> > something less like a heavyweight framework and more like a third Kafka
> > client--a kind of "producing consumer" with state management facilities.
> > Basically a library. Instead of a complex stream processing framework
> this
> > would actually be a very simple thing, not much more complicated to use
> or
> > operate than a Kafka consumer. As Chris said we thought about it a lot of
> > what Samza (and the other stream processing systems were doing) seemed
> like
> > kind of a hangover from MapReduce.
> >
> > Of course you need to ingest/output data to and from the stream
> > processing. But when we actually looked into how that would work, Samza
> > isn't really an ideal data ingestion framework for a bunch of reasons. To
> > really do that right you need a pretty different internal data model and
> > set of apis. So what if you split them and had an api for Kafka
> > ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> > transformation (Samza).
> >
> > This would also allow really embracing the same terminology and
> > conventions. One complaint about the current state is that the two
> systems
> > kind of feel bolted on. Terminology like "stream" vs "topic" and
> different
> > config and monitoring systems means you kind of have to learn Kafka's
> way,
> > then learn Samza's slightly different way, then kind of understand how
> they
> > map to each other, which having walked a few people through this is
> > surprisingly tricky for folks to get.
> >
> > Since I have been spending a lot of time on airplanes I hacked up an
> > ernest but still somewhat incomplete prototype of what this would look
> > like. This is just unceremoniously dumped into Kafka as it required a few
> > changes to the new consumer. Here is the code:
> >
> >
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org/apache/kafka/clients/streaming
> >
> > For the purpose of the prototype I just liberally renamed everything to
> > try to align it with Kafka with no regard for compatibility.
> >
> > To use this would be something like this:
> > Properties props = new Properties(); props.put("bootstrap.servers",
> > "localhost:4242"); StreamingConfig config = new StreamingConfig(props);
> config.subscribe("test-topic-1",
> > "test-topic-2"); config.processor(ExampleStreamProcessor.class);
> config.serialization(new
> > StringSerializer(), new StringDeserializer()); KafkaStreaming container =
> > new KafkaStreaming(config); container.run();
> >
> > KafkaStreaming is basically the SamzaContainer; StreamProcessor is
> > basically StreamTask.
> >
> > So rather than putting all the class names in a file and then having the
> > job assembled by reflection, you just instantiate the container
> > programmatically. Work is balanced over however many instances of this
> are
> > alive at any time (i.e. if an instance dies, new tasks are added to the
> > existing containers without shutting them down).
> >
> > We would provide some glue for running this stuff in YARN via Slider,
> > Mesos via Marathon, and AWS using some of their tools but from the point
> of
> > view of these frameworks these stream processing jobs are just stateless
> > services that can come and go and expand and contract at will. There is
> no
> > more custom scheduler.
> >
> > Here are some relevant details:
> >
> >    1. It is only ~1300 lines of code, it would get larger if we
> >    productionized but not vastly larger. We really do get a ton of
> leverage
> >    out of Kafka.
> >    2. Partition management is fully delegated to the new consumer. This
> >    is nice since now any partition management strategy available to Kafka
> >    consumer is also available to Samza (and vice versa) and with the
> exact
> >    same configs.
> >    3. It supports state as well as state reuse
> >
> > Anyhow take a look, hopefully it is thought provoking.
> >
> > -Jay
> >
> >
> >
> > On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <cr...@apache.org>
> > wrote:
> >
> >> Hey all,
> >>
> >> I have had some discussions with Samza engineers at LinkedIn and
> Confluent
> >> and we came up with a few observations and would like to propose some
> >> changes.
> >>
> >> We've observed some things that I want to call out about Samza's design,
> >> and I'd like to propose some changes.
> >>
> >> * Samza is dependent upon a dynamic deployment system.
> >> * Samza is too pluggable.
> >> * Samza's SystemConsumer/SystemProducer and Kafka's consumer APIs are
> >> trying to solve a lot of the same problems.
> >>
> >> All three of these issues are related, but I'll address them in order.
> >>
> >> Deployment
> >>
> >> Samza strongly depends on the use of a dynamic deployment scheduler such
> >> as
> >> YARN, Mesos, etc. When we initially built Samza, we bet that there would
> >> be
> >> one or two winners in this area, and we could support them, and the rest
> >> would go away. In reality, there are many variations. Furthermore, many
> >> people still prefer to just start their processors like normal Java
> >> processes, and use traditional deployment scripts such as Fabric, Chef,
> >> Ansible, etc. Forcing a deployment system on users makes the Samza
> >> start-up
> >> process really painful for first time users.
> >>
> >> Dynamic deployment as a requirement was also a bit of a mis-fire because
> >> of
> >> a fundamental misunderstanding between the nature of batch jobs and
> stream
> >> processing jobs. Early on, we made conscious effort to favor the Hadoop
> >> (Map/Reduce) way of doing things, since it worked and was well
> understood.
> >> One thing that we missed was that batch jobs have a definite beginning,
> >> and
> >> end, and stream processing jobs don't (usually). This leads to a much
> >> simpler scheduling problem for stream processors. You basically just
> need
> >> to find a place to start the processor, and start it. The way we run
> >> grids,
> >> at LinkedIn, there's no concept of a cluster being "full". We always add
> >> more machines. The problem with coupling Samza with a scheduler is that
> >> Samza (as a framework) now has to handle deployment. This pulls in a
> bunch
> >> of things such as configuration distribution (config stream), shell
> scrips
> >> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
> >>
> >> Another reason for requiring dynamic deployment was to support data
> >> locality. If you want to have locality, you need to put your processors
> >> close to the data they're processing. Upon further investigation,
> though,
> >> this feature is not that beneficial. There is some good discussion about
> >> some problems with it on SAMZA-335. Again, we took the Map/Reduce path,
> >> but
> >> there are some fundamental differences between HDFS and Kafka. HDFS has
> >> blocks, while Kafka has partitions. This leads to less optimization
> >> potential with stream processors on top of Kafka.
> >>
> >> This feature is also used as a crutch. Samza doesn't have any built in
> >> fault-tolerance logic. Instead, it depends on the dynamic deployment
> >> scheduling system to handle restarts when a processor dies. This has
> made
> >> it very difficult to write a standalone Samza container (SAMZA-516).
> >>
> >> Pluggability
> >>
> >> In some cases pluggability is good, but I think that we've gone too far
> >> with it. Currently, Samza has:
> >>
> >> * Pluggable config.
> >> * Pluggable metrics.
> >> * Pluggable deployment systems.
> >> * Pluggable streaming systems (SystemConsumer, SystemProducer, etc).
> >> * Pluggable serdes.
> >> * Pluggable storage engines.
> >> * Pluggable strategies for just about every component (MessageChooser,
> >> SystemStreamPartitionGrouper, ConfigRewriter, etc).
> >>
> >> There's probably more that I've forgotten, as well. Some of these are
> >> useful, but some have proven not to be. This all comes at a cost:
> >> complexity. This complexity is making it harder for our users to pick up
> >> and use Samza out of the box. It also makes it difficult for Samza
> >> developers to reason about what the characteristics of the container
> >> (since
> >> the characteristics change depending on which plugins are use).
> >>
> >> The issues with pluggability are most visible in the System APIs. What
> >> Samza really requires to be functional is Kafka as its transport layer.
> >> But
> >> we've conflated two unrelated use cases into one API:
> >>
> >> 1. Get data into/out of Kafka.
> >> 2. Process the data in Kafka.
> >>
> >> The current System API supports both of these use cases. The problem is,
> >> we
> >> actually want different features for each use case. By papering over
> these
> >> two use cases, and providing a single API, we've introduced a ton of
> leaky
> >> abstractions.
> >>
> >> For example, what we'd really like in (2) is to have monotonically
> >> increasing longs for offsets (like Kafka). This would be at odds with
> (1),
> >> though, since different systems have different
> SCNs/Offsets/UUIDs/vectors.
> >> There was discussion both on the mailing list and the SQL JIRAs about
> the
> >> need for this.
> >>
> >> The same thing holds true for replayability. Kafka allows us to rewind
> >> when
> >> we have a failure. Many other systems don't. In some cases, systems
> return
> >> null for their offsets (e.g. WikipediaSystemConsumer) because they have
> no
> >> offsets.
> >>
> >> Partitioning is another example. Kafka supports partitioning, but many
> >> systems don't. We model this by having a single partition for those
> >> systems. Still, other systems model partitioning differently (e.g.
> >> Kinesis).
> >>
> >> The SystemAdmin interface is also a mess. Creating streams in a
> >> system-agnostic way is almost impossible. As is modeling metadata for
> the
> >> system (replication factor, partitions, location, etc). The list goes
> on.
> >>
> >> Duplicate work
> >>
> >> At the time that we began writing Samza, Kafka's consumer and producer
> >> APIs
> >> had a relatively weak feature set. On the consumer-side, you had two
> >> options: use the high level consumer, or the simple consumer. The
> problem
> >> with the high-level consumer was that it controlled your offsets,
> >> partition
> >> assignments, and the order in which you received messages. The problem
> >> with
> >> the simple consumer is that it's not simple. It's basic. You end up
> having
> >> to handle a lot of really low-level stuff that you shouldn't. We spent a
> >> lot of time to make Samza's KafkaSystemConsumer very robust. It also
> >> allows
> >> us to support some cool features:
> >>
> >> * Per-partition message ordering and prioritization.
> >> * Tight control over partition assignment to support joins, global state
> >> (if we want to implement it :)), etc.
> >> * Tight control over offset checkpointing.
> >>
> >> What we didn't realize at the time is that these features should
> actually
> >> be in Kafka. A lot of Kafka consumers (not just Samza stream processors)
> >> end up wanting to do things like joins and partition assignment. The
> Kafka
> >> community has come to the same conclusion. They're adding a ton of
> >> upgrades
> >> into their new Kafka consumer implementation. To a large extent, it's
> >> duplicate work to what we've already done in Samza.
> >>
> >> On top of this, Kafka ended up taking a very similar approach to Samza's
> >> KafkaCheckpointManager implementation for handling offset checkpointing.
> >> Like Samza, Kafka's new offset management feature stores offset
> >> checkpoints
> >> in a topic, and allows you to fetch them from the broker.
> >>
> >> A lot of this seems like a waste, since we could have shared the work if
> >> it
> >> had been done in Kafka from the get-go.
> >>
> >> Vision
> >>
> >> All of this leads me to a rather radical proposal. Samza is relatively
> >> stable at this point. I'd venture to say that we're near a 1.0 release.
> >> I'd
> >> like to propose that we take what we've learned, and begin thinking
> about
> >> Samza beyond 1.0. What would we change if we were starting from scratch?
> >> My
> >> proposal is to:
> >>
> >> 1. Make Samza standalone the *only* way to run Samza processors, and
> >> eliminate all direct dependences on YARN, Mesos, etc.
> >> 2. Make a definitive call to support only Kafka as the stream processing
> >> layer.
> >> 3. Eliminate Samza's metrics, logging, serialization, and config
> systems,
> >> and simply use Kafka's instead.
> >>
> >> This would fix all of the issues that I outlined above. It should also
> >> shrink the Samza code base pretty dramatically. Supporting only a
> >> standalone container will allow Samza to be executed on YARN (using
> >> Slider), Mesos (using Marathon/Aurora), or most other in-house
> deployment
> >> systems. This should make life a lot easier for new users. Imagine
> having
> >> the hello-samza tutorial without YARN. The drop in mailing list traffic
> >> will be pretty dramatic.
> >>
> >> Coupling with Kafka seems long overdue to me. The reality is, everyone
> >> that
> >> I'm aware of is using Samza with Kafka. We basically require it already
> in
> >> order for most features to work. Those that are using other systems are
> >> generally using it for ingest into Kafka (1), and then they do the
> >> processing on top. There is already discussion (
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
> >> )
> >> in Kafka to make ingesting into Kafka extremely easy.
> >>
> >> Once we make the call to couple with Kafka, we can leverage a ton of
> their
> >> ecosystem. We no longer have to maintain our own config, metrics, etc.
> We
> >> can all share the same libraries, and make them better. This will also
> >> allow us to share the consumer/producer APIs, and will let us leverage
> >> their offset management and partition management, rather than having our
> >> own. All of the coordinator stream code would go away, as would most of
> >> the
> >> YARN AppMaster code. We'd probably have to push some partition
> management
> >> features into the Kafka broker, but they're already moving in that
> >> direction with the new consumer API. The features we have for partition
> >> assignment aren't unique to Samza, and seem like they should be in Kafka
> >> anyway. There will always be some niche usages which will require extra
> >> care and hence full control over partition assignments much like the
> Kafka
> >> low level consumer api. These would continue to be supported.
> >>
> >> These items will be good for the Samza community. They'll make Samza
> >> easier
> >> to use, and make it easier for developers to add new features.
> >>
> >> Obviously this is a fairly large (and somewhat backwards incompatible
> >> change). If we choose to go this route, it's important that we openly
> >> communicate how we're going to provide a migration path from the
> existing
> >> APIs to the new ones (if we make incompatible changes). I think at a
> >> minimum, we'd probably need to provide a wrapper to allow existing
> >> StreamTask implementations to continue running on the new container.
> It's
> >> also important that we openly communicate about timing, and stages of
> the
> >> migration.
> >>
> >> If you made it this far, I'm sure you have opinions. :) Please send your
> >> thoughts and feedback.
> >>
> >> Cheers,
> >> Chris
> >>
> >
> >
>



-- 
-- Guozhang

Re: Thoughts and obesrvations on Samza

Posted by Jay Kreps <ja...@confluent.io>.
Looks like gmail mangled the code example, it was supposed to look like
this:

Properties props = new Properties();
props.put("bootstrap.servers", "localhost:4242");
StreamingConfig config = new StreamingConfig(props);
config.subscribe("test-topic-1", "test-topic-2");
config.processor(ExampleStreamProcessor.class);
config.serialization(new StringSerializer(), new StringDeserializer());
KafkaStreaming container = new KafkaStreaming(config);
container.run();

-Jay

On Tue, Jun 30, 2015 at 11:32 PM, Jay Kreps <ja...@confluent.io> wrote:

> Hey guys,
>
> This came out of some conversations Chris and I were having around whether
> it would make sense to use Samza as a kind of data ingestion framework for
> Kafka (which ultimately lead to KIP-26 "copycat"). This kind of combined
> with complaints around config and YARN and the discussion around how to
> best do a standalone mode.
>
> So the thought experiment was, given that Samza was basically already
> totally Kafka specific, what if you just embraced that and turned it into
> something less like a heavyweight framework and more like a third Kafka
> client--a kind of "producing consumer" with state management facilities.
> Basically a library. Instead of a complex stream processing framework this
> would actually be a very simple thing, not much more complicated to use or
> operate than a Kafka consumer. As Chris said we thought about it a lot of
> what Samza (and the other stream processing systems were doing) seemed like
> kind of a hangover from MapReduce.
>
> Of course you need to ingest/output data to and from the stream
> processing. But when we actually looked into how that would work, Samza
> isn't really an ideal data ingestion framework for a bunch of reasons. To
> really do that right you need a pretty different internal data model and
> set of apis. So what if you split them and had an api for Kafka
> ingress/egress (copycat AKA KIP-26) and a separate api for Kafka
> transformation (Samza).
>
> This would also allow really embracing the same terminology and
> conventions. One complaint about the current state is that the two systems
> kind of feel bolted on. Terminology like "stream" vs "topic" and different
> config and monitoring systems means you kind of have to learn Kafka's way,
> then learn Samza's slightly different way, then kind of understand how they
> map to each other, which having walked a few people through this is
> surprisingly tricky for folks to get.
>
> Since I have been spending a lot of time on airplanes I hacked up an
> ernest but still somewhat incomplete prototype of what this would look
> like. This is just unceremoniously dumped into Kafka as it required a few
> changes to the new consumer. Here is the code:
>
> https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org/apache/kafka/clients/streaming
>
> For the purpose of the prototype I just liberally renamed everything to
> try to align it with Kafka with no regard for compatibility.
>
> To use this would be something like this:
> Properties props = new Properties(); props.put("bootstrap.servers",
> "localhost:4242"); StreamingConfig config = new StreamingConfig(props); config.subscribe("test-topic-1",
> "test-topic-2"); config.processor(ExampleStreamProcessor.class); config.serialization(new
> StringSerializer(), new StringDeserializer()); KafkaStreaming container =
> new KafkaStreaming(config); container.run();
>
> KafkaStreaming is basically the SamzaContainer; StreamProcessor is
> basically StreamTask.
>
> So rather than putting all the class names in a file and then having the
> job assembled by reflection, you just instantiate the container
> programmatically. Work is balanced over however many instances of this are
> alive at any time (i.e. if an instance dies, new tasks are added to the
> existing containers without shutting them down).
>
> We would provide some glue for running this stuff in YARN via Slider,
> Mesos via Marathon, and AWS using some of their tools but from the point of
> view of these frameworks these stream processing jobs are just stateless
> services that can come and go and expand and contract at will. There is no
> more custom scheduler.
>
> Here are some relevant details:
>
>    1. It is only ~1300 lines of code, it would get larger if we
>    productionized but not vastly larger. We really do get a ton of leverage
>    out of Kafka.
>    2. Partition management is fully delegated to the new consumer. This
>    is nice since now any partition management strategy available to Kafka
>    consumer is also available to Samza (and vice versa) and with the exact
>    same configs.
>    3. It supports state as well as state reuse
>
> Anyhow take a look, hopefully it is thought provoking.
>
> -Jay
>
>
>
> On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <cr...@apache.org>
> wrote:
>
>> Hey all,
>>
>> I have had some discussions with Samza engineers at LinkedIn and Confluent
>> and we came up with a few observations and would like to propose some
>> changes.
>>
>> We've observed some things that I want to call out about Samza's design,
>> and I'd like to propose some changes.
>>
>> * Samza is dependent upon a dynamic deployment system.
>> * Samza is too pluggable.
>> * Samza's SystemConsumer/SystemProducer and Kafka's consumer APIs are
>> trying to solve a lot of the same problems.
>>
>> All three of these issues are related, but I'll address them in order.
>>
>> Deployment
>>
>> Samza strongly depends on the use of a dynamic deployment scheduler such
>> as
>> YARN, Mesos, etc. When we initially built Samza, we bet that there would
>> be
>> one or two winners in this area, and we could support them, and the rest
>> would go away. In reality, there are many variations. Furthermore, many
>> people still prefer to just start their processors like normal Java
>> processes, and use traditional deployment scripts such as Fabric, Chef,
>> Ansible, etc. Forcing a deployment system on users makes the Samza
>> start-up
>> process really painful for first time users.
>>
>> Dynamic deployment as a requirement was also a bit of a mis-fire because
>> of
>> a fundamental misunderstanding between the nature of batch jobs and stream
>> processing jobs. Early on, we made conscious effort to favor the Hadoop
>> (Map/Reduce) way of doing things, since it worked and was well understood.
>> One thing that we missed was that batch jobs have a definite beginning,
>> and
>> end, and stream processing jobs don't (usually). This leads to a much
>> simpler scheduling problem for stream processors. You basically just need
>> to find a place to start the processor, and start it. The way we run
>> grids,
>> at LinkedIn, there's no concept of a cluster being "full". We always add
>> more machines. The problem with coupling Samza with a scheduler is that
>> Samza (as a framework) now has to handle deployment. This pulls in a bunch
>> of things such as configuration distribution (config stream), shell scrips
>> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
>>
>> Another reason for requiring dynamic deployment was to support data
>> locality. If you want to have locality, you need to put your processors
>> close to the data they're processing. Upon further investigation, though,
>> this feature is not that beneficial. There is some good discussion about
>> some problems with it on SAMZA-335. Again, we took the Map/Reduce path,
>> but
>> there are some fundamental differences between HDFS and Kafka. HDFS has
>> blocks, while Kafka has partitions. This leads to less optimization
>> potential with stream processors on top of Kafka.
>>
>> This feature is also used as a crutch. Samza doesn't have any built in
>> fault-tolerance logic. Instead, it depends on the dynamic deployment
>> scheduling system to handle restarts when a processor dies. This has made
>> it very difficult to write a standalone Samza container (SAMZA-516).
>>
>> Pluggability
>>
>> In some cases pluggability is good, but I think that we've gone too far
>> with it. Currently, Samza has:
>>
>> * Pluggable config.
>> * Pluggable metrics.
>> * Pluggable deployment systems.
>> * Pluggable streaming systems (SystemConsumer, SystemProducer, etc).
>> * Pluggable serdes.
>> * Pluggable storage engines.
>> * Pluggable strategies for just about every component (MessageChooser,
>> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>>
>> There's probably more that I've forgotten, as well. Some of these are
>> useful, but some have proven not to be. This all comes at a cost:
>> complexity. This complexity is making it harder for our users to pick up
>> and use Samza out of the box. It also makes it difficult for Samza
>> developers to reason about what the characteristics of the container
>> (since
>> the characteristics change depending on which plugins are use).
>>
>> The issues with pluggability are most visible in the System APIs. What
>> Samza really requires to be functional is Kafka as its transport layer.
>> But
>> we've conflated two unrelated use cases into one API:
>>
>> 1. Get data into/out of Kafka.
>> 2. Process the data in Kafka.
>>
>> The current System API supports both of these use cases. The problem is,
>> we
>> actually want different features for each use case. By papering over these
>> two use cases, and providing a single API, we've introduced a ton of leaky
>> abstractions.
>>
>> For example, what we'd really like in (2) is to have monotonically
>> increasing longs for offsets (like Kafka). This would be at odds with (1),
>> though, since different systems have different SCNs/Offsets/UUIDs/vectors.
>> There was discussion both on the mailing list and the SQL JIRAs about the
>> need for this.
>>
>> The same thing holds true for replayability. Kafka allows us to rewind
>> when
>> we have a failure. Many other systems don't. In some cases, systems return
>> null for their offsets (e.g. WikipediaSystemConsumer) because they have no
>> offsets.
>>
>> Partitioning is another example. Kafka supports partitioning, but many
>> systems don't. We model this by having a single partition for those
>> systems. Still, other systems model partitioning differently (e.g.
>> Kinesis).
>>
>> The SystemAdmin interface is also a mess. Creating streams in a
>> system-agnostic way is almost impossible. As is modeling metadata for the
>> system (replication factor, partitions, location, etc). The list goes on.
>>
>> Duplicate work
>>
>> At the time that we began writing Samza, Kafka's consumer and producer
>> APIs
>> had a relatively weak feature set. On the consumer-side, you had two
>> options: use the high level consumer, or the simple consumer. The problem
>> with the high-level consumer was that it controlled your offsets,
>> partition
>> assignments, and the order in which you received messages. The problem
>> with
>> the simple consumer is that it's not simple. It's basic. You end up having
>> to handle a lot of really low-level stuff that you shouldn't. We spent a
>> lot of time to make Samza's KafkaSystemConsumer very robust. It also
>> allows
>> us to support some cool features:
>>
>> * Per-partition message ordering and prioritization.
>> * Tight control over partition assignment to support joins, global state
>> (if we want to implement it :)), etc.
>> * Tight control over offset checkpointing.
>>
>> What we didn't realize at the time is that these features should actually
>> be in Kafka. A lot of Kafka consumers (not just Samza stream processors)
>> end up wanting to do things like joins and partition assignment. The Kafka
>> community has come to the same conclusion. They're adding a ton of
>> upgrades
>> into their new Kafka consumer implementation. To a large extent, it's
>> duplicate work to what we've already done in Samza.
>>
>> On top of this, Kafka ended up taking a very similar approach to Samza's
>> KafkaCheckpointManager implementation for handling offset checkpointing.
>> Like Samza, Kafka's new offset management feature stores offset
>> checkpoints
>> in a topic, and allows you to fetch them from the broker.
>>
>> A lot of this seems like a waste, since we could have shared the work if
>> it
>> had been done in Kafka from the get-go.
>>
>> Vision
>>
>> All of this leads me to a rather radical proposal. Samza is relatively
>> stable at this point. I'd venture to say that we're near a 1.0 release.
>> I'd
>> like to propose that we take what we've learned, and begin thinking about
>> Samza beyond 1.0. What would we change if we were starting from scratch?
>> My
>> proposal is to:
>>
>> 1. Make Samza standalone the *only* way to run Samza processors, and
>> eliminate all direct dependences on YARN, Mesos, etc.
>> 2. Make a definitive call to support only Kafka as the stream processing
>> layer.
>> 3. Eliminate Samza's metrics, logging, serialization, and config systems,
>> and simply use Kafka's instead.
>>
>> This would fix all of the issues that I outlined above. It should also
>> shrink the Samza code base pretty dramatically. Supporting only a
>> standalone container will allow Samza to be executed on YARN (using
>> Slider), Mesos (using Marathon/Aurora), or most other in-house deployment
>> systems. This should make life a lot easier for new users. Imagine having
>> the hello-samza tutorial without YARN. The drop in mailing list traffic
>> will be pretty dramatic.
>>
>> Coupling with Kafka seems long overdue to me. The reality is, everyone
>> that
>> I'm aware of is using Samza with Kafka. We basically require it already in
>> order for most features to work. Those that are using other systems are
>> generally using it for ingest into Kafka (1), and then they do the
>> processing on top. There is already discussion (
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767
>> )
>> in Kafka to make ingesting into Kafka extremely easy.
>>
>> Once we make the call to couple with Kafka, we can leverage a ton of their
>> ecosystem. We no longer have to maintain our own config, metrics, etc. We
>> can all share the same libraries, and make them better. This will also
>> allow us to share the consumer/producer APIs, and will let us leverage
>> their offset management and partition management, rather than having our
>> own. All of the coordinator stream code would go away, as would most of
>> the
>> YARN AppMaster code. We'd probably have to push some partition management
>> features into the Kafka broker, but they're already moving in that
>> direction with the new consumer API. The features we have for partition
>> assignment aren't unique to Samza, and seem like they should be in Kafka
>> anyway. There will always be some niche usages which will require extra
>> care and hence full control over partition assignments much like the Kafka
>> low level consumer api. These would continue to be supported.
>>
>> These items will be good for the Samza community. They'll make Samza
>> easier
>> to use, and make it easier for developers to add new features.
>>
>> Obviously this is a fairly large (and somewhat backwards incompatible
>> change). If we choose to go this route, it's important that we openly
>> communicate how we're going to provide a migration path from the existing
>> APIs to the new ones (if we make incompatible changes). I think at a
>> minimum, we'd probably need to provide a wrapper to allow existing
>> StreamTask implementations to continue running on the new container. It's
>> also important that we openly communicate about timing, and stages of the
>> migration.
>>
>> If you made it this far, I'm sure you have opinions. :) Please send your
>> thoughts and feedback.
>>
>> Cheers,
>> Chris
>>
>
>

Re: Thoughts and obesrvations on Samza

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

This came out of some conversations Chris and I were having around whether
it would make sense to use Samza as a kind of data ingestion framework for
Kafka (which ultimately lead to KIP-26 "copycat"). This kind of combined
with complaints around config and YARN and the discussion around how to
best do a standalone mode.

So the thought experiment was, given that Samza was basically already
totally Kafka specific, what if you just embraced that and turned it into
something less like a heavyweight framework and more like a third Kafka
client--a kind of "producing consumer" with state management facilities.
Basically a library. Instead of a complex stream processing framework this
would actually be a very simple thing, not much more complicated to use or
operate than a Kafka consumer. As Chris said we thought about it a lot of
what Samza (and the other stream processing systems were doing) seemed like
kind of a hangover from MapReduce.

Of course you need to ingest/output data to and from the stream processing.
But when we actually looked into how that would work, Samza isn't really an
ideal data ingestion framework for a bunch of reasons. To really do that
right you need a pretty different internal data model and set of apis. So
what if you split them and had an api for Kafka ingress/egress (copycat AKA
KIP-26) and a separate api for Kafka transformation (Samza).

This would also allow really embracing the same terminology and
conventions. One complaint about the current state is that the two systems
kind of feel bolted on. Terminology like "stream" vs "topic" and different
config and monitoring systems means you kind of have to learn Kafka's way,
then learn Samza's slightly different way, then kind of understand how they
map to each other, which having walked a few people through this is
surprisingly tricky for folks to get.

Since I have been spending a lot of time on airplanes I hacked up an ernest
but still somewhat incomplete prototype of what this would look like. This
is just unceremoniously dumped into Kafka as it required a few changes to
the new consumer. Here is the code:
https://github.com/jkreps/kafka/tree/streams/clients/src/main/java/org/apache/kafka/clients/streaming

For the purpose of the prototype I just liberally renamed everything to try
to align it with Kafka with no regard for compatibility.

To use this would be something like this:
Properties props = new Properties(); props.put("bootstrap.servers",
"localhost:4242"); StreamingConfig config = new
StreamingConfig(props); config.subscribe("test-topic-1",
"test-topic-2"); config.processor(ExampleStreamProcessor.class);
config.serialization(new
StringSerializer(), new StringDeserializer()); KafkaStreaming container =
new KafkaStreaming(config); container.run();

KafkaStreaming is basically the SamzaContainer; StreamProcessor is
basically StreamTask.

So rather than putting all the class names in a file and then having the
job assembled by reflection, you just instantiate the container
programmatically. Work is balanced over however many instances of this are
alive at any time (i.e. if an instance dies, new tasks are added to the
existing containers without shutting them down).

We would provide some glue for running this stuff in YARN via Slider, Mesos
via Marathon, and AWS using some of their tools but from the point of view
of these frameworks these stream processing jobs are just stateless
services that can come and go and expand and contract at will. There is no
more custom scheduler.

Here are some relevant details:

   1. It is only ~1300 lines of code, it would get larger if we
   productionized but not vastly larger. We really do get a ton of leverage
   out of Kafka.
   2. Partition management is fully delegated to the new consumer. This is
   nice since now any partition management strategy available to Kafka
   consumer is also available to Samza (and vice versa) and with the exact
   same configs.
   3. It supports state as well as state reuse

Anyhow take a look, hopefully it is thought provoking.

-Jay



On Tue, Jun 30, 2015 at 6:55 PM, Chris Riccomini <cr...@apache.org>
wrote:

> Hey all,
>
> I have had some discussions with Samza engineers at LinkedIn and Confluent
> and we came up with a few observations and would like to propose some
> changes.
>
> We've observed some things that I want to call out about Samza's design,
> and I'd like to propose some changes.
>
> * Samza is dependent upon a dynamic deployment system.
> * Samza is too pluggable.
> * Samza's SystemConsumer/SystemProducer and Kafka's consumer APIs are
> trying to solve a lot of the same problems.
>
> All three of these issues are related, but I'll address them in order.
>
> Deployment
>
> Samza strongly depends on the use of a dynamic deployment scheduler such as
> YARN, Mesos, etc. When we initially built Samza, we bet that there would be
> one or two winners in this area, and we could support them, and the rest
> would go away. In reality, there are many variations. Furthermore, many
> people still prefer to just start their processors like normal Java
> processes, and use traditional deployment scripts such as Fabric, Chef,
> Ansible, etc. Forcing a deployment system on users makes the Samza start-up
> process really painful for first time users.
>
> Dynamic deployment as a requirement was also a bit of a mis-fire because of
> a fundamental misunderstanding between the nature of batch jobs and stream
> processing jobs. Early on, we made conscious effort to favor the Hadoop
> (Map/Reduce) way of doing things, since it worked and was well understood.
> One thing that we missed was that batch jobs have a definite beginning, and
> end, and stream processing jobs don't (usually). This leads to a much
> simpler scheduling problem for stream processors. You basically just need
> to find a place to start the processor, and start it. The way we run grids,
> at LinkedIn, there's no concept of a cluster being "full". We always add
> more machines. The problem with coupling Samza with a scheduler is that
> Samza (as a framework) now has to handle deployment. This pulls in a bunch
> of things such as configuration distribution (config stream), shell scrips
> (bin/run-job.sh, JobRunner), packaging (all the .tgz stuff), etc.
>
> Another reason for requiring dynamic deployment was to support data
> locality. If you want to have locality, you need to put your processors
> close to the data they're processing. Upon further investigation, though,
> this feature is not that beneficial. There is some good discussion about
> some problems with it on SAMZA-335. Again, we took the Map/Reduce path, but
> there are some fundamental differences between HDFS and Kafka. HDFS has
> blocks, while Kafka has partitions. This leads to less optimization
> potential with stream processors on top of Kafka.
>
> This feature is also used as a crutch. Samza doesn't have any built in
> fault-tolerance logic. Instead, it depends on the dynamic deployment
> scheduling system to handle restarts when a processor dies. This has made
> it very difficult to write a standalone Samza container (SAMZA-516).
>
> Pluggability
>
> In some cases pluggability is good, but I think that we've gone too far
> with it. Currently, Samza has:
>
> * Pluggable config.
> * Pluggable metrics.
> * Pluggable deployment systems.
> * Pluggable streaming systems (SystemConsumer, SystemProducer, etc).
> * Pluggable serdes.
> * Pluggable storage engines.
> * Pluggable strategies for just about every component (MessageChooser,
> SystemStreamPartitionGrouper, ConfigRewriter, etc).
>
> There's probably more that I've forgotten, as well. Some of these are
> useful, but some have proven not to be. This all comes at a cost:
> complexity. This complexity is making it harder for our users to pick up
> and use Samza out of the box. It also makes it difficult for Samza
> developers to reason about what the characteristics of the container (since
> the characteristics change depending on which plugins are use).
>
> The issues with pluggability are most visible in the System APIs. What
> Samza really requires to be functional is Kafka as its transport layer. But
> we've conflated two unrelated use cases into one API:
>
> 1. Get data into/out of Kafka.
> 2. Process the data in Kafka.
>
> The current System API supports both of these use cases. The problem is, we
> actually want different features for each use case. By papering over these
> two use cases, and providing a single API, we've introduced a ton of leaky
> abstractions.
>
> For example, what we'd really like in (2) is to have monotonically
> increasing longs for offsets (like Kafka). This would be at odds with (1),
> though, since different systems have different SCNs/Offsets/UUIDs/vectors.
> There was discussion both on the mailing list and the SQL JIRAs about the
> need for this.
>
> The same thing holds true for replayability. Kafka allows us to rewind when
> we have a failure. Many other systems don't. In some cases, systems return
> null for their offsets (e.g. WikipediaSystemConsumer) because they have no
> offsets.
>
> Partitioning is another example. Kafka supports partitioning, but many
> systems don't. We model this by having a single partition for those
> systems. Still, other systems model partitioning differently (e.g.
> Kinesis).
>
> The SystemAdmin interface is also a mess. Creating streams in a
> system-agnostic way is almost impossible. As is modeling metadata for the
> system (replication factor, partitions, location, etc). The list goes on.
>
> Duplicate work
>
> At the time that we began writing Samza, Kafka's consumer and producer APIs
> had a relatively weak feature set. On the consumer-side, you had two
> options: use the high level consumer, or the simple consumer. The problem
> with the high-level consumer was that it controlled your offsets, partition
> assignments, and the order in which you received messages. The problem with
> the simple consumer is that it's not simple. It's basic. You end up having
> to handle a lot of really low-level stuff that you shouldn't. We spent a
> lot of time to make Samza's KafkaSystemConsumer very robust. It also allows
> us to support some cool features:
>
> * Per-partition message ordering and prioritization.
> * Tight control over partition assignment to support joins, global state
> (if we want to implement it :)), etc.
> * Tight control over offset checkpointing.
>
> What we didn't realize at the time is that these features should actually
> be in Kafka. A lot of Kafka consumers (not just Samza stream processors)
> end up wanting to do things like joins and partition assignment. The Kafka
> community has come to the same conclusion. They're adding a ton of upgrades
> into their new Kafka consumer implementation. To a large extent, it's
> duplicate work to what we've already done in Samza.
>
> On top of this, Kafka ended up taking a very similar approach to Samza's
> KafkaCheckpointManager implementation for handling offset checkpointing.
> Like Samza, Kafka's new offset management feature stores offset checkpoints
> in a topic, and allows you to fetch them from the broker.
>
> A lot of this seems like a waste, since we could have shared the work if it
> had been done in Kafka from the get-go.
>
> Vision
>
> All of this leads me to a rather radical proposal. Samza is relatively
> stable at this point. I'd venture to say that we're near a 1.0 release. I'd
> like to propose that we take what we've learned, and begin thinking about
> Samza beyond 1.0. What would we change if we were starting from scratch? My
> proposal is to:
>
> 1. Make Samza standalone the *only* way to run Samza processors, and
> eliminate all direct dependences on YARN, Mesos, etc.
> 2. Make a definitive call to support only Kafka as the stream processing
> layer.
> 3. Eliminate Samza's metrics, logging, serialization, and config systems,
> and simply use Kafka's instead.
>
> This would fix all of the issues that I outlined above. It should also
> shrink the Samza code base pretty dramatically. Supporting only a
> standalone container will allow Samza to be executed on YARN (using
> Slider), Mesos (using Marathon/Aurora), or most other in-house deployment
> systems. This should make life a lot easier for new users. Imagine having
> the hello-samza tutorial without YARN. The drop in mailing list traffic
> will be pretty dramatic.
>
> Coupling with Kafka seems long overdue to me. The reality is, everyone that
> I'm aware of is using Samza with Kafka. We basically require it already in
> order for most features to work. Those that are using other systems are
> generally using it for ingest into Kafka (1), and then they do the
> processing on top. There is already discussion (
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767)
> in Kafka to make ingesting into Kafka extremely easy.
>
> Once we make the call to couple with Kafka, we can leverage a ton of their
> ecosystem. We no longer have to maintain our own config, metrics, etc. We
> can all share the same libraries, and make them better. This will also
> allow us to share the consumer/producer APIs, and will let us leverage
> their offset management and partition management, rather than having our
> own. All of the coordinator stream code would go away, as would most of the
> YARN AppMaster code. We'd probably have to push some partition management
> features into the Kafka broker, but they're already moving in that
> direction with the new consumer API. The features we have for partition
> assignment aren't unique to Samza, and seem like they should be in Kafka
> anyway. There will always be some niche usages which will require extra
> care and hence full control over partition assignments much like the Kafka
> low level consumer api. These would continue to be supported.
>
> These items will be good for the Samza community. They'll make Samza easier
> to use, and make it easier for developers to add new features.
>
> Obviously this is a fairly large (and somewhat backwards incompatible
> change). If we choose to go this route, it's important that we openly
> communicate how we're going to provide a migration path from the existing
> APIs to the new ones (if we make incompatible changes). I think at a
> minimum, we'd probably need to provide a wrapper to allow existing
> StreamTask implementations to continue running on the new container. It's
> also important that we openly communicate about timing, and stages of the
> migration.
>
> If you made it this far, I'm sure you have opinions. :) Please send your
> thoughts and feedback.
>
> Cheers,
> Chris
>