Re: KIP-28 kafka processor

2015-08-18 Thread Jay Kreps
mail.com > > On Mon, Aug 17, 2015 at 11:05 PM, Yi Pan wrote: > > > Hi, Chris and Jay, > > > > Thanks for the reminder. I plan to follow up this week. > > > > Cheers! > > > > -Yi > > > > On Sun, Aug 16, 2015 at 12:27 PM, Jay Kreps wrote: &

Re: KIP-28 kafka processor

2015-08-16 Thread Jay Kreps
+1 Any feedback would be appreciated! -Jay On Sat, Aug 15, 2015 at 3:55 PM, Chris Riccomini wrote: > Hey all, > > I wanted to call attention to KIP-28: > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-28+-+Add+a+processor+client > > This is the result of the last conversation that we

Re: Question about sub-projects and project merging

2015-07-13 Thread Jay Kreps
My advice is the same as others -- keep the two projects separate, but > encourage individual Samza committers join as Kafka committers if they > feel the need to do so. > > On Mon, Jul 13, 2015 at 1:37 AM, Jay Kreps wrote: > > Hey board members, > > > > There is a lo

Re: Thoughts and obesrvations on Samza

2015-07-13 Thread Jay Kreps
l 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. Thou

Re: Thoughts and obesrvations on Samza

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

Question about sub-projects and project merging

2015-07-12 Thread Jay Kreps
Hey board members, There is a longish thread on the Apache Samza mailing list on the relationship between Kafka and Samza and whether they wouldn't make a lot more sense as a single project. This raised some questions I was hoping to get advice on. Discussion thread (warning: super long, I attemp

Re: Thoughts and obesrvations on Samza

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

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Jay Kreps
ka -> transformation -> copyCat, the thing is how we persuade them to > do this long chain. If yes, it will also be a win for Kafka too. Or if > there is no one in this community actually doing this so far, maybe ok to > not support the group 4 directly. > > Thanks, > > Fang

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Jay Kreps
for "simpler" large components to install. Avoid > at all costs! > > > On Fri, Jul 10, 2015 at 12:58 PM, Jay Kreps wrote: > > Yeah I agree with this summary. I think there are kind of two questions > > here: > > 1. Technically does alignment/reliance on Kafka mak

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Jay Kreps
y went > forward with 'Hey Samza!' My points above are directed entirely at > the community aspect of these choices. > -Jakob > > On 10 July 2015 at 09:10, Roger Hoover wrote: > > That's great. Thanks, Jay. > > > > On Fri, Jul 10, 2015 at 8:46 AM, J

Re: Thoughts and obesrvations on Samza

2015-07-10 Thread Jay Kreps
already but I thought it might be worth > calling out again. > > Cheers, > > Roger > > > On Tue, Jul 7, 2015 at 11:35 AM, Jay Kreps wrote: > > > Hey Roger, > > > > I couldn't agree more. We spent a bunch of time talking to people and > that &g

Re: Thoughts and obesrvations on Samza

2015-07-08 Thread Jay Kreps
Kafka has chosen (or > > switched to) 'static' partitioning, where each logical task consumes a > > fixed set of partitions. When 'dynamic deploying' (a la Storm / Mesos > > / Yarn) the underlying system is already doing failure detection and > > transfe

Re: Thoughts and obesrvations on Samza

2015-07-07 Thread Jay Kreps
ple, would it be better if the MR and HDFS components of > > > Hadoop had different release schedules? > > > > > > Actually, keeping the discussion in a single place would make agreeing > on > > > releases (and backwards compatibility) much easier, as everybody

Re: Thoughts and obesrvations on Samza

2015-07-07 Thread Jay Kreps
for the whole codebase. > > > > That said, I like the idea of absorbing samza-core as a sub-project, and > > leave the fancy stuff separate. > > It probably gives 90% of the benefits we have been discussing here. > > > > Cheers, > > > > -- > > Gianmarco &

Re: Thoughts and obesrvations on Samza

2015-07-06 Thread Jay Kreps
e 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, J

Re: Thoughts and obesrvations on Samza

2015-07-06 Thread Jay Kreps
tream 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 cu

Re: Thoughts and obesrvations on Samza

2015-07-06 Thread Jay Kreps
rom 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 ne

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

Re: Thoughts and obesrvations on Samza

2015-07-03 Thread Jay Kreps
; If yes, I thought that the assignment is done by the broker as in > KAFKA-167? Maybe we can discuss and clarify this in person. > > Thanks a lot! > > -Yi > > On Thu, Jul 2, 2015 at 3:52 PM, Jay Kreps wrote: > > > Yeah, hey Yi, I get what you are saying now. At the r

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Jay Kreps
from which partitions, > etc. I > > think this is want Yi meant for "partition management"? > > > > On Thu, Jul 2, 2015 at 2:35 PM, Yi Pan wrote: > > > > > @Jay, yes, the current function in the JobCoordinator is just partition > > > management.

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Jay Kreps
t; > > @Jay, yes, the current function in the JobCoordinator is just partition > > management. Maybe we should just call it PartitionManager to make it > > explicit. > > > > -Yi > > > > On Thu, Jul 2, 2015 at 2:24 PM, Jay Kreps wrote: > > >

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Jay Kreps
d. > > > 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 > toleran

Re: Thoughts and obesrvations on Samza

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

Re: Thoughts and obesrvations on Samza

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

Re: Thoughts and obesrvations on Samza

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

Re: Thoughts and obesrvations on Samza

2015-07-02 Thread Jay Kreps
able 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.se

Re: Thoughts and obesrvations on Samza

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

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

Fwd: About Spark Streaming overview in Samza docs

2015-06-02 Thread Jay Kreps
Hey guys, Here were some critiques of our system comparison page from Tathagata at Databricks. -Jay -- Forwarded message -- From: Tathagata Das Date: Thu, May 14, 2015 at 1:15 PM Subject: About Spark Streaming overview in Samza docs To: Jay Kreps Hello Jay, I am not sure if

Re: Re-processing a la Kappa/Liquid

2015-02-23 Thread Jay Kreps
en each operator propagates the looser requirements to its > input > > operator(s), and this allows garbage to be collected. > > > > I don't know whether checkpoints and retention are spelled out in > > Kappa/Liquid, but if not, they seem a natural and useful extensio

Re: Re-processing a la Kappa/Liquid

2015-02-23 Thread Jay Kreps
d this allows garbage to be collected. > > > > I don't know whether checkpoints and retention are spelled out in > > Kappa/Liquid, but if not, they seem a natural and useful extension to the > > theory. > > > > Julian > > > > > > > On Feb 21, 20

Re: Re-processing a la Kappa/Liquid

2015-02-21 Thread Jay Kreps
rom the local store. > > If you want to kick off another version of this job that starts back in > time, the new job cannot reliably recreate the same state of the local > store that the original had because old values may have been compacted > away. > > Does that make sense? >

Re: Re-processing a la Kappa/Liquid

2015-02-20 Thread Jay Kreps
Hey Roger, I'm not sure if I understand the case you are describing. As Chris says we don't yet give you fined grained control over when history starts to disappear (though we designed with the intention of making that configurable later). However I'm not sure if you need that for the case you de

Re: Collocating Samza(YARN) and Kafka/ZK clusters

2015-02-09 Thread Jay Kreps
It may or may not be due to colocating Kafka and Samza but you are probably tripping the failure detection in Kafka which considers a replica out of sync if it falls more than N messages behind. Can you try tuning this setting as described here: https://cwiki.apache.org/confluence/display/KAFKA/FAQ

Re: Streaming SQL - object models, ASTs and algebras

2015-02-01 Thread Jay Kreps
suggest that we only want to support one schema per > stream. Since Yi's stuff is more general, it could be made to work with > just one schema per-stream, but I just wanted to call it out as a conscious > decision. This seems as though it would prevent us from being able to >

Re: Streaming SQL - object models, ASTs and algebras

2015-01-30 Thread Jay Kreps
Chris, I think the schema repository acts as the stand in for the database catalog or hcat. People who don't have that will have to give their schema with the query. I think that can be some kind of plugin to provide the schema so it is automatically inferred for the avro people and manually provi

Re: Streaming SQL - object models, ASTs and algebras

2015-01-30 Thread Jay Kreps
I am not convinced that the partitioning and partition key issue is quite equivalent to partitions in a relational database. In a relational database partitions have no semantic meaning and are just a way to organize data for query efficiency. That is not the case in Kafka. Let my try to argue that

Re: Local state write throughput

2015-01-25 Thread Jay Kreps
> know what I find. > > On Sun, Jan 25, 2015 at 2:40 PM, Chris Riccomini > wrote: > > > Awesome, I'll have a look at this. @Roger, did setting this improve your > > RocksDB throughput? > > > > On Sun, Jan 25, 2015 at 12:53 PM, Jay Kreps wrote: > >

Re: Local state write throughput

2015-01-25 Thread Jay Kreps
I have seen a similar thing from the OS tunable readahead. I think Linux defaults to reading a full 128K into pagecache with every read. This is sensible for spinning disks where maybe blowing 500us may mean you get lucky and save a 10ms seek. But for SSDs, especially a key-value store doing purely

Re: Samza has graduated!

2015-01-23 Thread Jay Kreps
Nicely done everyone! Really excited to have been a part of this. :-) -Jay On Fri, Jan 23, 2015 at 2:46 PM, Shekar Tippur wrote: > Fantastic news. > A big thanks to Chris and team. > > - Shekar > > On Thu, Jan 22, 2015 at 1:58 PM, Yan Fang wrote: > > > This is great !! So happy to see it grow