Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-18 Thread Jim Jagielski
Very cool. > On Jul 16, 2016, at 9:55 AM, Damian Guy wrote: > > Hi, > The vote is now complete and KIP-67 has been accepted and adopted. > Thanks everyone for the input etc. > > Regards, > Damian > > On Sat, 16 Jul 2016 at 06:53 Damian Guy wrote: >

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-16 Thread Damian Guy
Hi, The vote is now complete and KIP-67 has been accepted and adopted. Thanks everyone for the input etc. Regards, Damian On Sat, 16 Jul 2016 at 06:53 Damian Guy wrote: > Hi, > Jay's interpretation is correct. > Thanks, > Damian > > > On Fri, 15 Jul 2016 at 16:10, Jay

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-16 Thread Damian Guy
Hi, Jay's interpretation is correct. Thanks, Damian On Fri, 15 Jul 2016 at 16:10, Jay Kreps wrote: > My interpretation was that you need an implementation of > QueryableStoreType which anyone can do and QueryableStoreTypes is just a > place to put the type objects for the

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-15 Thread Jay Kreps
My interpretation was that you need an implementation of QueryableStoreType which anyone can do and QueryableStoreTypes is just a place to put the type objects for the types we ship with Kafka. -Jay On Fri, Jul 15, 2016 at 4:04 PM, Sriram Subramanian wrote: > So, it looks

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-15 Thread Sriram Subramanian
So, it looks like QueryableStoreTypes would be part of the streams library, right? If a developer needs to support a new store, would they need to change code in streams? On Fri, Jul 15, 2016 at 3:15 PM, Jay Kreps wrote: > Cool, I'm +1 after the updates. > > -Jay > > On Fri,

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-15 Thread Jay Kreps
Cool, I'm +1 after the updates. -Jay On Fri, Jul 15, 2016 at 1:50 PM, Damian Guy wrote: > Hi Guozhang, KIP updated. > > Thanks, > Damian > > On Fri, 15 Jul 2016 at 13:15 Guozhang Wang wrote: > > > Hi Damian, > > > > Since the StateStoreProvider is

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-15 Thread Damian Guy
Hi Guozhang, KIP updated. Thanks, Damian On Fri, 15 Jul 2016 at 13:15 Guozhang Wang wrote: > Hi Damian, > > Since the StateStoreProvider is moved into internal packages, how about > just keeping the ReadOnlyXXStores interface for the queryAPI, and > "QueryableStoreType" in

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-15 Thread Guozhang Wang
Hi Damian, Since the StateStoreProvider is moved into internal packages, how about just keeping the ReadOnlyXXStores interface for the queryAPI, and "QueryableStoreType" in the discoverAPI, and move the StateStoreProvider / QueryableStoreTypeMatcher and different implementations of the matcher

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-15 Thread Damian Guy
Hi All, I've updated the KIP with changes as discussed in this Thread. Thanks, Damian On Wed, 13 Jul 2016 at 16:51 Ismael Juma wrote: > I think it's important to distinguish the use cases of defining new stores > (somewhat rare) versus using the `store` method (very

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Ismael Juma
I think it's important to distinguish the use cases of defining new stores (somewhat rare) versus using the `store` method (very common). The strategy employed here is a common way to use generics to ensure type safety for the latter case. In the former case, there are all sorts of weird things

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Damian Guy
Yes, you get compile time errors On Wed, 13 Jul 2016 at 16:22 Damian Guy wrote: > You wont get a runtime error as you wouldn't find a store of that type. > The API would return null > > On Wed, 13 Jul 2016 at 16:22 Jay Kreps wrote: > >> But if

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Damian Guy
You wont get a runtime error as you wouldn't find a store of that type. The API would return null On Wed, 13 Jul 2016 at 16:22 Jay Kreps wrote: > But if "my-store" is not of type MyStoreType don't you still get a run time > error that in effect is the same as the class cast

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Jay Kreps
But if "my-store" is not of type MyStoreType don't you still get a run time error that in effect is the same as the class cast would be? Basically the question I'm asking is whether this added complexity is actually moving runtime errors to compile time errors. -Jay On Wed, Jul 13, 2016 at 4:16

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Damian Guy
You create your custom Store, i.e,: /** * An interface your custom store provides * @param * @param */ interface MyStoreType { V get(K key); void put(K key, V value); } /** * Implement your store * @param * @param */ public class MyStoreImpl implements StateStore,

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Jay Kreps
But to avoid the cast you introduce a bunch of magic that doesn't really bring type safety, right? Or possibly I'm misunderstanding, how do I plug in a new store type and get access to it? Can you give the steps for that? -Jay On Wed, Jul 13, 2016 at 10:47 AM, Guozhang Wang

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Ismael Juma
I agree too. Casting also makes it harder to rely on static analysis tools to detect unsafe code usage. Ismael On Wed, Jul 13, 2016 at 7:37 PM, Damian Guy wrote: > I'm in agreement with Gouzhang. Further, from a "Developer Experience" PoV > IMO we should avoid forcing

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Jim Jagielski
+1 > On Jul 13, 2016, at 2:37 PM, Damian Guy wrote: > > I'm in agreement with Gouzhang. Further, from a "Developer Experience" PoV > IMO we should avoid forcing developers to cast - it doesn't make for nice > and easy to use API, introduces possible runtime errors due to

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Damian Guy
I'm in agreement with Gouzhang. Further, from a "Developer Experience" PoV IMO we should avoid forcing developers to cast - it doesn't make for nice and easy to use API, introduces possible runtime errors due to invalid casts, developers need to know what they are casting to. On Wed, 13 Jul 2016

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Guozhang Wang
Personally I think the additional complexity of the introduced " QueryableStoreType" interface is still acceptable from a user's point of view: this is the only interface we are exposing to users, and other wrappers are all internal classes. Regarding "QueryableStoreTypes", maybe we can consider

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Neha Narkhede
Damian -- appreciate the example code and you convinced me. Agree that the class approach is better and renaming to KafkaStreamsMetadata along with renaming the API methods will address the issues I was referring to. One other thing I wanted to get people's thoughts on was the way we are

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-13 Thread Jim Jagielski
IMO, that makes the most sense. > On Jul 12, 2016, at 5:11 PM, Ismael Juma wrote: > > Hi Damian, > > How about StreamsMetadata instead? The general naming pattern seems to > avoid the `Kafka` prefix for everything outside of `KafkaStreams` itself. > > Ismael > > On Tue,

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-12 Thread Damian Guy
Ismael - that is fine with me. Thanks On Tue, 12 Jul 2016 at 14:11 Ismael Juma wrote: > Hi Damian, > > How about StreamsMetadata instead? The general naming pattern seems to > avoid the `Kafka` prefix for everything outside of `KafkaStreams` itself. > > Ismael > > On Tue, Jul

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-12 Thread Ismael Juma
Hi Damian, How about StreamsMetadata instead? The general naming pattern seems to avoid the `Kafka` prefix for everything outside of `KafkaStreams` itself. Ismael On Tue, Jul 12, 2016 at 7:14 PM, Damian Guy wrote: > Hi, > > I agree with point 1. application.server is a

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-12 Thread Guozhang Wang
Hi Damian, After running the example code in mind with these two approaches I'm sold on your arguments :) Guozhang On Tue, Jul 12, 2016 at 12:49 PM, Sriram Subramanian wrote: > I liked abstracting the metadata approach as well. It also helps to evolve > in the future. > >

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-12 Thread Sriram Subramanian
I liked abstracting the metadata approach as well. It also helps to evolve in the future. On Tue, Jul 12, 2016 at 12:42 PM, Michael Noll wrote: > Like Damian I'd also favor a proper type (KafkaStreamsMetadata) rather than > a Map-based construct. > > -Michael > > On Tue,

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-12 Thread Michael Noll
Like Damian I'd also favor a proper type (KafkaStreamsMetadata) rather than a Map-based construct. -Michael On Tue, Jul 12, 2016 at 8:45 PM, Damian Guy wrote: > One more thing on the above, the methods on KafkaStreams should be changed > to something like: > > Collection

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-12 Thread Damian Guy
One more thing on the above, the methods on KafkaStreams should be changed to something like: Collection allMetadata() Collection allMetadataForStore(final String storeName) KafkaStreamsMetadata metadataWithKey(final String storeName, final K key,

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-12 Thread Damian Guy
Hi, I agree with point 1. application.server is a better name for the config (we'll change this). However, on point 2 I think we should stick mostly with what we already have. I've tried both ways of doing this when working on the JIRA and building examples and I find the current approach more

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-12 Thread Michael Noll
> 1. Re StreamsConfig.USER_ENDPOINT_CONFIG: > > I agree with Neha that Kafka Streams can provide the bare minimum APIs just > for host/port, and user's implemented layer can provide URL / proxy address > they want to build on top of it. In this case I second Neha's suggestion to give that

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-11 Thread Guozhang Wang
1. Re StreamsConfig.USER_ENDPOINT_CONFIG: I agree with Neha that Kafka Streams can provide the bare minimum APIs just for host/port, and user's implemented layer can provide URL / proxy address they want to build on top of it. 2. Re Improving KafkaStreamsInstance interface: Users are indeed

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-10 Thread Neha Narkhede
Few thoughts that became apparent after observing example code of what an application architecture and code might look like with these changes. Apologize for the late realization hence. 1. "user.endpoint" will be very differently defined for respective applications. I don't think Kafka Streams

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-08 Thread Michael Noll
Damian, about the rolling upgrade comment: An instance A will contact another instance B by the latter's endpoint, right? So if A has no further information available than B's host and port, then how should instance A know whether it should call B at /v1/ or at /v2/? I agree that my suggestion

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-08 Thread Michael Noll
Addendum in case my previous email wasn't clear: > So for any given instance of a streams application there will never be both a v1 and v2 alive at the same time That's right. But the current live instance will be able to tell other instances, via its endpoint setting, whether it wants to be

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-08 Thread Damian Guy
Michael - i'm ok with changing it to a string. Any one else have a strong opinion on this? FWIW - i don't think it will work fine as is during the rolling upgrade scenario as the service that is listening on the port needs to be embedded within each instance. So for any given instance of a

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-08 Thread Michael Noll
I have one further comment about `StreamsConfig.USER_ENDPOINT_CONFIG`. I think we should consider to not restricting the value of this setting to only `host:port` pairs. By design, this setting is capturing user-driven metadata to define an endpoint, so why restrict the creativity or flexibility

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-08 Thread Matthias J. Sax
+1 On 07/08/2016 11:03 AM, Eno Thereska wrote: > +1 (non-binding) > >> On 7 Jul 2016, at 18:31, Sriram Subramanian wrote: >> >> +1 >> >> On Thu, Jul 7, 2016 at 9:53 AM, Henry Cai >> wrote: >> >>> +1 >>> >>> On Thu, Jul 7, 2016 at 6:48 AM, Michael

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-08 Thread Eno Thereska
+1 (non-binding) > On 7 Jul 2016, at 18:31, Sriram Subramanian wrote: > > +1 > > On Thu, Jul 7, 2016 at 9:53 AM, Henry Cai > wrote: > >> +1 >> >> On Thu, Jul 7, 2016 at 6:48 AM, Michael Noll wrote: >> >>> +1

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-07 Thread Sriram Subramanian
+1 On Thu, Jul 7, 2016 at 9:53 AM, Henry Cai wrote: > +1 > > On Thu, Jul 7, 2016 at 6:48 AM, Michael Noll wrote: > > > +1 (non-binding) > > > > On Thu, Jul 7, 2016 at 10:24 AM, Damian Guy > wrote: > > > > > Thanks Henry -

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-07 Thread Henry Cai
+1 On Thu, Jul 7, 2016 at 6:48 AM, Michael Noll wrote: > +1 (non-binding) > > On Thu, Jul 7, 2016 at 10:24 AM, Damian Guy wrote: > > > Thanks Henry - we've updated the KIP with an example and the new config > > parameter required. FWIW the user

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-07 Thread Michael Noll
+1 (non-binding) On Thu, Jul 7, 2016 at 10:24 AM, Damian Guy wrote: > Thanks Henry - we've updated the KIP with an example and the new config > parameter required. FWIW the user doesn't register a listener, they provide > a host:port in config. It is expected they will

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-07 Thread Damian Guy
Thanks Henry - we've updated the KIP with an example and the new config parameter required. FWIW the user doesn't register a listener, they provide a host:port in config. It is expected they will start a service running on that host:port that they can use to connect to the running KafkaStreams

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-06 Thread Henry Cai
It wasn't quite clear to me how the user program interacts with the discovery API, especially on the user supplied listener part, how does the user program supply that listener to KafkaStreams and how does KafkaStreams know which port the user listener is running, maybe a more complete end-to-end

Re: [VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-06 Thread Guozhang Wang
+1 On Wed, Jul 6, 2016 at 12:44 PM, Damian Guy wrote: > Hi all, > > I'd like to initiate the voting process for KIP-67 > < > https://cwiki.apache.org/confluence/display/KAFKA/KIP-67%3A+Queryable+state+for+Kafka+Streams > > > > KAFKA-3909

[VOTE] KIP-67: Queryable state for Kafka Streams

2016-07-06 Thread Damian Guy
Hi all, I'd like to initiate the voting process for KIP-67 KAFKA-3909 is the top level JIRA for this effort. Initial PRs for Step 1 of the process