Thoughts and obesrvations on Samza

2015-06-30 Thread Chris Riccomini
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 dif

Re: Thoughts and obesrvations on Samza

2015-06-30 Thread Jay Kreps
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 
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 variat

Re: Thoughts and obesrvations on Samza

2015-06-30 Thread Jay Kreps
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  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 
> 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'

Re: Thoughts and obesrvations on Samza

2015-07-01 Thread Guozhang Wang
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  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  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
> 

Re: Thoughts and obesrvations on Samza

2015-07-01 Thread Yan Fang
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
yanfang...@gmail.com

On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang  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  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  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 requi

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Gianmarco De Francisci Morales
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  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
> yanfang...@gmail.com
>
> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang  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  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  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 

RE: Thoughts and obesrvations on Samza

2015-07-02 Thread Garry Turkington
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:g...@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  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
> yanfang...@gmail.com
>
> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang  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  wrote:
> >
> > > Looks like gmail mangled the code exam

RE: Thoughts and obesrvations on Samza

2015-07-02 Thread Kartik Paramasivam
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 [j...@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  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"

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Jay Kreps
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  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  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  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 bas

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Jay Kreps
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
s 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  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
> yanfang...@gmail.com
>
> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang  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  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  wrote:
> > >
> > > > Hey guys,
> 

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Jay Kreps
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 <
g...@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  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
> > yanfang...@gmail.com
> >
> > On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang 
> 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  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 
> wrote:
> > > >
> > > > > Hey guys,
> > > > >
> > > > > This came out of some 

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Jay Kreps
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.turking...@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:g...@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  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
> &g

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Sriram
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  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.turking...@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:g...@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  wrote:
>>> 
>>> Overall, I agree to couple with Kafka more tightly. Because Samza de
>>> facto is based on Kafka, and it should leverag

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Yi Pan
ort 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.turking...@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:g...@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  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
> &

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Jay Kreps
 Jay Kreps  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.turking...@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:g...@apache.org]
> > >> Sent: 02 July 2015 12:56
> > >> To: dev@samza.apache.org
> > >> Subject: Re: Thoughts and obesrvations on Samza
> > >>
> > >> Very interesting thoughts.
>

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Yi Pan
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  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  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.turking...@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

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Guozhang Wang
ost 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:g...@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  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 

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Yi Pan
ading 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:g...@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  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) ?
>>

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Jay Kreps
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.turking...@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:g...@apache.org]
> >> > > >> Sent: 02 July 2015 12:56
> >> > > >> To: de

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Yi Pan
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.turking...@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. 

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Yi Pan
> 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.turking...@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.
> > >> > > >>
> > >> > > >&g

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Jay Kreps
hem 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 
> 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 
> 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.turking...@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
> > > > > > >> m

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Yi Pan
; > > 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.turking...@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
> > > > 

Re: Thoughts and obesrvations on Samza

2015-07-03 Thread Gianmarco De Francisci Morales
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

2015-07-03 Thread Jay Kreps
; > > > 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 
> > > 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 
> > > 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 

Re: Thoughts and obesrvations on Samza

2015-07-03 Thread Jay Kreps
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 <
g...@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

2015-07-05 Thread Guozhang Wang
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  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 <
> g...@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

2015-07-06 Thread Martin Kleppmann
 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:g...@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  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
>>> yanfang...@gmail.com
>>> 
>>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang 
>> 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

Re: Thoughts and obesrvations on Samza

2015-07-06 Thread Yi Pan
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 <
g...@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

2015-07-06 Thread Yi Pan
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  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  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 <
> > g...@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

2015-07-06 Thread Jay Kreps
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:g...@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  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
> >>> yanfang...@gmail.com
> >>>
> >>> On Wed, Jul 1, 2015 at 5:46 PM, Guozhang Wang 
> >> 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.
> >&g

Re: Thoughts and obesrvations on Samza

2015-07-06 Thread Yi Pan
s 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  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.turking...@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 M

Re: Thoughts and obesrvations on Samza

2015-07-06 Thread Martin Kleppmann
ately 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.turking...@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:g...@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

Re: Thoughts and obesrvations on Samza

2015-07-06 Thread Jay Kreps
> > >
> > > -Jay
> > >
> > > On Thu, Jul 2, 2015 at 4:17 AM, Garry Turkington <
> > > g.turking...@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:g...@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  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) ?
> > >>&g

RE: Thoughts and obesrvations on Samza

2015-07-06 Thread Ken Krugler
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  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 

Re: Thoughts and obesrvations on Samza

2015-07-06 Thread Yi Pan
t; > Best,
> > > Martin
> > >
> > > On 2 Jul 2015, at 16:59, Jay Kreps  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.turking...@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

Re: Thoughts and obesrvations on Samza

2015-07-06 Thread Timothy Chen
> > > 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  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.turking...@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
>> >

Re: Thoughts and obesrvations on Samza

2015-07-06 Thread Yi Pan
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  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.turking...@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/produc

Re: Thoughts and obesrvations on Samza

2015-07-06 Thread Jay Kreps
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:g...@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  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 hav

Re: Thoughts and obesrvations on Samza

2015-07-07 Thread Gianmarco De Francisci Morales
a 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.turking...@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.
> > >>>>
>

Re: Thoughts and obesrvations on Samza

2015-07-07 Thread Gianmarco De Francisci Morales
 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  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.turking...@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 

Re: Thoughts and obesrvations on Samza

2015-07-07 Thread Jay Kreps
er,
> >> 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  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 s

Re: Thoughts and obesrvations on Samza

2015-07-07 Thread Roger Hoover
ive, 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  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.turking...@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
> >>

Re: Thoughts and obesrvations on Samza

2015-07-07 Thread Jay Kreps
; > >> 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  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)

Re: Thoughts and obesrvations on Samza

2015-07-08 Thread Ben Kirwin
gt;> > >> > >>
>> > >> > >> 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  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 

Re: Thoughts and obesrvations on Samza

2015-07-08 Thread Jakob Homan
known 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 <
>>> > >> mar...@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
>>> > >&g

Re: Thoughts and obesrvations on Samza

2015-07-08 Thread Jay Kreps
ctually 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 <
> >>> > >> mar...@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
> >>> >

RE: Thoughts and obesrvations on Samza

2015-07-08 Thread Thomas Becker
>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 [j...@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 
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 impor

Re: Thoughts and obesrvations on Samza

2015-07-08 Thread Jordan Shaw
f 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 <
> > >>> > >> mar...@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,
> > &g

Re: Thoughts and obesrvations on Samza

2015-07-09 Thread Julian Hyde
nment, 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.
>> &

Re: Thoughts and obesrvations on Samza

2015-07-09 Thread Martin Kleppmann
t;>>>>>>>>> "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

Re: Thoughts and obesrvations on Samza

2015-07-09 Thread Yi Pan
t; 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

Re: Thoughts and obesrvations on Samza

2015-07-09 Thread Tim Williams
Some thoughts from the peanut gallery...

On Thu, Jul 9, 2015 at 5:14 PM, Martin Kleppmann  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

2015-07-10 Thread Roger Hoover
> 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  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
> > > >> >

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Jay Kreps
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
> 

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Roger Hoover
gt; > >> > >> 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
&

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Jakob Homan
;> > > 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 <
>> > > > > >> mar...@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 a

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Jay Kreps
> > > > >> 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 <
> >> > > > > >> mar...@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 th

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Yan Fang
t; > > > > >> > 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 <
> > >> > > > > >> mar...@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
> &

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Julian Hyde
>> 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 <
>> >> > > > > >> mar...@kleppmann.com>
>> >> > > > > >> > > wrote:
>> >> > > > > >> > >
>> >> > > &

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Jay Kreps
f 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 <
> >> >> > > > > >> mar...@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 o

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Jay Kreps
gt; >> > > 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
> > > >> > > > > >> > 

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Yan Fang
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
> > > > >> > > &

Re: Thoughts and obesrvations on Samza

2015-07-12 Thread Chris Riccomini
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  wrote:

> Thanks, Jay. This argument persuaded me actually. :)
>
> Fang, Yan
> yanfang...@gmail.com
>
> On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps  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

Re: Thoughts and obesrvations on Samza

2015-07-12 Thread Chris Riccomini
That was meant to be "thread" not "threat". lol. :)

On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini 
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  wrote:
>
>> Thanks, Jay. This argument persuaded me actually. :)
>>
>> Fang, Yan
>> yanfang...@gmail.com
>>
>> On Fri, Jul 10, 2015 at 2:33 PM, Jay Kreps  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 technica

Re: Thoughts and obesrvations on Samza

2015-07-12 Thread Tim Williams
On Sun, Jul 12, 2015 at 8:54 PM, Chris Riccomini  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 muc

Re: Thoughts and obesrvations on Samza

2015-07-12 Thread Garrett Barton
​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 
wrote:

> That was meant to be "thread" not "threat". lol. :)
>
> On Sun, Jul 12, 2015 at 5:54 PM, Chris Riccomini 
> 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

Re: Thoughts and obesrvations on Samza

2015-07-12 Thread Yi Pan
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_dt19v0znmmhb...@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-2716167c3...@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 
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] Su

Re: Thoughts and obesrvations on Samza

2015-07-12 Thread Yi Pan
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  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_dt19v0znmmhb...@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-2716167c3...@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 ke

Re: Thoughts and obesrvations on Samza

2015-07-12 Thread Jakob Homan
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  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

2015-07-12 Thread Garrett Barton
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  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  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_dt19v0znmmhb...@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 que

Re: Thoughts and obesrvations on Samza

2015-07-12 Thread Jay Kreps
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 
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  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 framewo

Re: Thoughts and obesrvations on Samza

2015-07-12 Thread Jay Kreps
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 
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,

Re: Thoughts and obesrvations on Samza

2015-07-13 Thread Jay Kreps
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  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 
> 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/%3CCADiKvVtWPjHLLDsmxvz9

Re: Thoughts and obesrvations on Samza

2015-07-13 Thread Jordan Shaw
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  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  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 
> > 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.ap

Re: Thoughts and obesrvations on Samza

2015-07-13 Thread Yan Fang
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
yanfang...@gmail.com

On Mon, Jul 13, 2015 at 11:25 AM, Jordan Shaw  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  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  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
> > > discou

Re: Thoughts and obesrvations on Samza

2015-07-13 Thread Guozhang Wang
>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  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  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
>

Re: Thoughts and obesrvations on Samza

2015-07-13 Thread Yi Pan
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.turking...@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:yanfang...@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" 

Re: Thoughts and obesrvations on Samza

2015-07-13 Thread Yi Pan
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.turking...@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:yanfang...@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
> yanfang...@gmail.com
>
> On Mon, Jul 13, 2015 at 11:25 AM, Jordan Shaw  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