[jira] [Commented] (KAFKA-3775) Throttle maximum number of tasks assigned to a single KafkaStreams

2016-06-09 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15322254#comment-15322254 ] Matthias J. Sax commented on KAFKA-3775: I want to push this discussion further. As [~guozhang

Kafka Wiki

2016-05-26 Thread Matthias J. Sax
Hi, can I get write access to the Kafka Wiki? Username is: mjsax Thanks. -Matthias signature.asc Description: OpenPGP digital signature

[jira] [Commented] (KAFKA-3775) Throttle maximum number of tasks assigned to a single KafkaStreams

2016-06-13 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15327326#comment-15327326 ] Matthias J. Sax commented on KAFKA-3775: [~tenggyut] [~wushujames] Kafka Streams allow to specify

Re: [DISCUSS] KIP-63: Unify store and downstream caching in streams

2016-06-13 Thread Matthias J. Sax
I am just catching up on this thread. From my point of view, easy tuning for the user is the most important thing, because Kafka Streams is a library. Thus, a global cache size parameter should be the best. About dividing the memory vs a single global cache. I would argue that in the first place

[jira] [Commented] (KAFKA-3576) Unify KStream and KTable API

2016-06-13 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3576?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15328242#comment-15328242 ] Matthias J. Sax commented on KAFKA-3576: I see two points: (1) this is similar to SQL / Pig

[jira] [Commented] (KAFKA-3775) Throttle maximum number of tasks assigned to a single KafkaStreams

2016-06-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3775?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15312250#comment-15312250 ] Matthias J. Sax commented on KAFKA-3775: I have some concerns about this: 1) a KStreams

Re: KStreams Rewind Offset

2016-06-02 Thread Matthias J. Sax
Hi Mike, currently, this is not possible. We are already discussing some changes with regard to reprocess. However, I doubt that going back to a specific offset of a specific partition will be supported as it would be too difficult to reset the internal data structures and intermediate results

Re: Reduce function Null checks

2016-06-19 Thread Matthias J. Sax
Hi Jeyhun, thanks for reporting! It got already fixed in trunk version. https://issues.apache.org/jira/browse/KAFKA-3589 https://github.com/apache/kafka/pull/1246/files -Matthias On 06/19/2016 02:46 PM, Jeyhun Karimov wrote: > Hi community, > > When using, reduce(Reducer,Reducer,

Re: Operator order

2016-06-19 Thread Matthias J. Sax
of some tuples for some operators (in runtime) to get approximate results. > However, it iseems the possible solution can bring extra overhead to system > in some cases. > > Jeyhun > > On Sat, Jun 18, 2016 at 7:36 PM Matthias J. Sax <matth...@confluent.io> > wrote: >

[jira] [Created] (KAFKA-3880) Disallow Join Windows with size zero

2016-06-20 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-3880: -- Summary: Disallow Join Windows with size zero Key: KAFKA-3880 URL: https://issues.apache.org/jira/browse/KAFKA-3880 Project: Kafka Issue Type

Re: Operator order

2016-06-18 Thread Matthias J. Sax
Hi Jeyhun, there is no support by the library itself. But you could build a custom solution by building the DAG with all required edges (ie, additional edges from A->E, and B->sink etc.). For this, each output message from A would be duplicate and send to B and E. Therefore, A should "tag" each

[jira] [Assigned] (KAFKA-3185) Allow users to cleanup internal data

2016-06-17 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3185?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-3185: -- Assignee: Matthias J. Sax > Allow users to cleanup internal d

[jira] [Assigned] (KAFKA-3337) Extract selector as a separate groupBy operator for KTable aggregations

2016-04-06 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-3337: -- Assignee: Matthias J. Sax > Extract selector as a separate groupBy operator for KTa

Re: KStream Close Processor

2016-04-11 Thread Matthias J. Sax
What about extending the API with a method beforeClose() that enables the user to flush buffered data? Maybe we can also rename close() to afterClose(), to make the difference clear. At least, we should document when close() is called -- from a user point of view, I would expect that close()

[jira] [Assigned] (KAFKA-3477) Add customizable StreamPartition into #to functions of Streams DSL

2016-04-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3477?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-3477: -- Assignee: Matthias J. Sax > Add customizable StreamPartition into #to functi

Jira

2016-03-31 Thread Matthias J. Sax
Hi, I was browsing Kafka JIRAs and realized, that I cannot assign JIRAs to myself. Can anyone give me credentials for this, please? username: mjsax Thanks. :) -Matthias signature.asc Description: OpenPGP digital signature

[jira] [Updated] (KAFKA-3598) Improve JavaDoc of public API

2016-04-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3598?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3598: --- Status: Patch Available (was: In Progress) > Improve JavaDoc of public

[jira] [Updated] (KAFKA-3599) Minor code cleanup

2016-04-24 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3599?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3599: --- Status: Patch Available (was: In Progress) > Minor code clea

[jira] [Updated] (KAFKA-3599) Minor code cleanup

2016-04-24 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3599?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3599: --- Description: - remove {{WindowStoreUtils}} from public API, ie, move to sub-package

[jira] [Updated] (KAFKA-3599) Move WindowStoreUtils to package "internals"

2016-04-24 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3599?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3599: --- Summary: Move WindowStoreUtils to package "internals" (was: Minor code cleanu

[jira] [Commented] (KAFKA-3599) Minor code cleanup

2016-04-24 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15255733#comment-15255733 ] Matthias J. Sax commented on KAFKA-3599: Removing the `equals()` issue from this JIRA because

[jira] [Updated] (KAFKA-3440) Add Javadoc for KTable (changelog stream) and KStream (record stream)

2016-04-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3440: --- Status: Patch Available (was: In Progress) > Add Javadoc for KTable (changelog str

[jira] [Updated] (KAFKA-3440) Add Javadoc for KTable (changelog stream) and KStream (record stream)

2016-04-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3440: --- Description: Currently we only have a 1-liner in {code}KTable{code} and {code}KStream{code

[jira] [Updated] (KAFKA-3440) Add Javadoc for KTable (changelog stream) and KStream (record stream)

2016-04-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3440: --- Description: Currently we only have a 1-liner in {code}KTable{code} and {code}KStream{code

[jira] [Work started] (KAFKA-3440) Add Javadoc for KTable (changelog stream) and KStream (record stream)

2016-04-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3440 started by Matthias J. Sax. -- > Add Javadoc for KTable (changelog stream) and KStream (record str

[jira] [Assigned] (KAFKA-3574) Add missing @returns tags for JavaDoc

2016-04-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3574?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-3574: -- Assignee: Matthias J. Sax (was: Guozhang Wang) > Add missing @returns t

[jira] [Assigned] (KAFKA-3440) Add Javadoc for KTable (changelog stream) and KStream (record stream)

2016-04-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-3440: -- Assignee: Matthias J. Sax > Add Javadoc for KTable (changelog stream) and KStr

[jira] [Closed] (KAFKA-3574) Add missing @returns tags for JavaDoc

2016-04-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3574?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax closed KAFKA-3574. -- > Add missing @returns tags for Java

[jira] [Resolved] (KAFKA-3574) Add missing @returns tags for JavaDoc

2016-04-28 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3574?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-3574. Resolution: Duplicate > Add missing @returns tags for Java

Re: [ANNOUNCE] New committer: Ismael Juma

2016-04-26 Thread Matthias J. Sax
Congrats Ismael! On 04/26/2016 07:52 AM, Neha Narkhede wrote: > The PMC for Apache Kafka has invited Ismael Juma to join as a committer and > we are pleased to announce that he has accepted! > > Ismael has contributed 121 commits > to a wide

Re: [COMMERCIAL] Re: download - 0.10.0.0 RC6

2016-05-18 Thread Matthias J. Sax
It's for different Scala versions, ie, 2.10 and 2.11, respectively. -Matthias On 05/18/2016 07:28 PM, Ramanan, Buvana (Nokia - US) wrote: > Ian, > > Thanks a lot for the prompt response. > > What is the difference between the following? > > 1) kafka-0.10.0.0-src.tgz > 2)

[jira] [Work started] (KAFKA-3429) Remove Serdes needed for repartitioning in KTable stateful operations

2016-04-18 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3429?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3429 started by Matthias J. Sax. -- > Remove Serdes needed for repartitioning in KTable stateful operati

[jira] [Created] (KAFKA-3576) Unify KStream and KTable API

2016-04-18 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-3576: -- Summary: Unify KStream and KTable API Key: KAFKA-3576 URL: https://issues.apache.org/jira/browse/KAFKA-3576 Project: Kafka Issue Type: Sub-task

[jira] [Updated] (KAFKA-3576) Unify KStream and KTable API

2016-04-18 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3576?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3576: --- Fix Version/s: 0.10.1.0 > Unify KStream and KTable

Re: kafka streaming: passing config string to Processor

2016-04-17 Thread Matthias J. Sax
I guess the simplest way would be to use a constructor parameter: > public static class CampaignProcessor implements > ProcessorSupplier> > { > private final String jedis_server; > > public CampaignProcessor(String jedisServer) { >

[jira] [Assigned] (KAFKA-3429) Remove Serdes needed for repartitioning in KTable stateful operations

2016-04-17 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3429?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-3429: -- Assignee: Matthias J. Sax > Remove Serdes needed for repartitioning in KTable state

Re: kafka streaming: passing config string to Processor

2016-04-17 Thread Matthias J. Sax
t; Thanks > > 2016-04-17 12:38 GMT+02:00 Matthias J. Sax <matth...@confluent.io>: > >> I guess the simplest way would be to use a constructor parameter: >> >>> public static class CampaignProcessor implements >> ProcessorSupplier<Windowe

[jira] [Created] (KAFKA-3574) Add missing @returns tags for JavaDoc

2016-04-17 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-3574: -- Summary: Add missing @returns tags for JavaDoc Key: KAFKA-3574 URL: https://issues.apache.org/jira/browse/KAFKA-3574 Project: Kafka Issue Type

[jira] [Updated] (KAFKA-3574) Add missing @returns tags for JavaDoc

2016-04-17 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3574?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3574: --- Issue Type: Sub-task (was: Improvement) Parent: KAFKA-2590 > Add missing @retu

[jira] [Commented] (KAFKA-3429) Remove Serdes needed for repartitioning in KTable stateful operations

2016-04-18 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15245384#comment-15245384 ] Matthias J. Sax commented on KAFKA-3429: Ok. This does not contradict my thoughts. In general, we

[jira] [Commented] (KAFKA-3429) Remove Serdes needed for repartitioning in KTable stateful operations

2016-04-17 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3429?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15244771#comment-15244771 ] Matthias J. Sax commented on KAFKA-3429: [~guozhang] I am a little confused here: {{originalSerde

[jira] [Work started] (KAFKA-3337) Extract selector as a separate groupBy operator for KTable aggregations

2016-04-14 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3337 started by Matthias J. Sax. -- > Extract selector as a separate groupBy operator for KTable aggregati

[jira] [Work started] (KAFKA-3598) Improve JavaDoc of public API

2016-04-21 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3598?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3598 started by Matthias J. Sax. -- > Improve JavaDoc of public

[jira] [Updated] (KAFKA-3598) Improve JavaDoc of public API

2016-04-21 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3598?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3598: --- Description: Add missing JavaDoc to all {{public}} methods of public API. Is related to KAFKA

[jira] [Created] (KAFKA-3599) Minor code cleanup

2016-04-21 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-3599: -- Summary: Minor code cleanup Key: KAFKA-3599 URL: https://issues.apache.org/jira/browse/KAFKA-3599 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-3598) Improve JavaDoc of public API

2016-04-21 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-3598: -- Summary: Improve JavaDoc of public API Key: KAFKA-3598 URL: https://issues.apache.org/jira/browse/KAFKA-3598 Project: Kafka Issue Type: Sub-task

[jira] [Work started] (KAFKA-3599) Minor code cleanup

2016-04-21 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3599?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3599 started by Matthias J. Sax. -- > Minor code cleanup > -- > >

[jira] [Commented] (KAFKA-3599) Minor code cleanup

2016-04-21 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15251933#comment-15251933 ] Matthias J. Sax commented on KAFKA-3599: [~guozhang] if I understand correctly, we remove

[jira] [Resolved] (KAFKA-3981) Possible race condition between controller cache and ZK on topic delete

2016-07-25 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3981?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-3981. Resolution: Not A Problem > Possible race condition between controller cache and

[jira] [Created] (KAFKA-4001) Improving join semantics in Kafka Stremas

2016-07-27 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4001: -- Summary: Improving join semantics in Kafka Stremas Key: KAFKA-4001 URL: https://issues.apache.org/jira/browse/KAFKA-4001 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-4008) Module "tools" should ne be dependent on "core"

2016-07-29 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4008: -- Summary: Module "tools" should ne be dependent on "core" Key: KAFKA-4008 URL: https://issues.apache.org/jira/browse/KAFKA-4008 Project: Kafka

Re: [VOTE] 0.10.0.1 RC0

2016-07-29 Thread Matthias J. Sax
Hi, I think we need a new RC because of https://issues.apache.org/jira/browse/KAFKA-4008 -Matthias On 07/29/2016 04:59 PM, Harsha Chintalapani wrote: > Hi Ismael, > I would like this JIRA to be included in the minor release > https://issues.apache.org/jira/browse/KAFKA-3950 > Thanks,

Re: Kafka Streams question

2016-07-14 Thread Matthias J. Sax
Hi, you can manually create a topic with the number of partitions you want to have and use this topic via through() KStream input = ... input.map().through("manually-created-topic").join(...) However, both KStream and KTable need to have the same number of partitions for perform the join.

[jira] [Created] (KAFKA-3981) Possible race condition between controller cache and ZK on topic delete

2016-07-20 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-3981: -- Summary: Possible race condition between controller cache and ZK on topic delete Key: KAFKA-3981 URL: https://issues.apache.org/jira/browse/KAFKA-3981 Project

[jira] [Updated] (KAFKA-3185) Allow users to cleanup internal data

2016-07-19 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3185?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3185: --- Status: Patch Available (was: Open) > Allow users to cleanup internal d

[jira] [Updated] (KAFKA-3478) Finer Stream Flow Control

2016-07-15 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3478?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3478: --- Description: Today we have a event-time based flow control mechanism in order to synchronize

Re: join in the dev group

2016-07-07 Thread Matthias J. Sax
Hi, if you want to subscribe to dev list, you need to send an email to dev-subscr...@kafka.apache.org See: https://kafka.apache.org/contact.html -Matthias On 07/07/2016 06:04 AM, caizhiqi...@kingsoft.com wrote: > > hello, > i wanna join and learn more kafka. thanks so much . > > >

[jira] [Created] (KAFKA-3936) Validate user parameters as early as possible

2016-07-08 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-3936: -- Summary: Validate user parameters as early as possible Key: KAFKA-3936 URL: https://issues.apache.org/jira/browse/KAFKA-3936 Project: Kafka Issue Type

[jira] [Updated] (KAFKA-3936) Validate user parameters as early as possible

2016-07-08 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3936?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3936: --- Description: Currently, parameters handed in by the user via public API, are not validated

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: Parallelisation factor in kafka streams

2016-07-06 Thread Matthias J. Sax
Hi Jeyhun, the number of partitions determine the number of tasks within a Kafka Streams application and thus, the maximum number of parallelism for your application. For more details see http://docs.confluent.io/3.0.0/streams/architecture.html#parallelism-model You can set the number of

Re: Parallelisation factor in kafka streams

2016-07-06 Thread Matthias J. Sax
you for your answer Matthias. > Is it possible to change the parallelism in runtime? Or do we have to > restart the cluster? > > > On Wed, 6 Jul 2016 at 19:08, Matthias J. Sax <matth...@confluent.io> wrote: > >> Hi Jeyhun, >> >> the number of partitions

Re: [Kafka Streams] Source Nodes

2016-06-29 Thread Matthias J. Sax
Hi, for joins, data of both inputs must be co-located, ie, partitioned on the same key and have the same number of partitions: See "Note" box at: http://docs.confluent.io/3.0.0/streams/developer-guide.html?highlight=join#joining-streams From an older email thread about the same issue: >>> If

[jira] [Updated] (KAFKA-3880) Disallow Join Windows with size zero

2016-06-20 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3880?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3880: --- Status: Patch Available (was: Open) https://github.com/apache/kafka/pull/1529 > Disal

[jira] [Work stopped] (KAFKA-3429) Remove Serdes needed for repartitioning in KTable stateful operations

2016-07-01 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3429?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3429 stopped by Matthias J. Sax. -- > Remove Serdes needed for repartitioning in KTable stateful operati

Re: [VOTE] KIP:71 Enable log compaction and deletion to co-exist

2016-08-15 Thread Matthias J. Sax
+1 On 08/15/2016 06:00 PM, Eno Thereska wrote: > +1 (non-binding) > >> On 15 Aug 2016, at 14:20, Damian Guy wrote: >> >> I would like to initiate the voting process for KIP-71 ( >>

[jira] [Commented] (KAFKA-4722) StreamThread should allow customization of thread prefix

2017-02-01 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15849195#comment-15849195 ] Matthias J. Sax commented on KAFKA-4722: I see. I guess, we could add the {{application.id

[jira] [Commented] (KAFKA-3729) Auto-configure non-default SerDes passed alongside the topology builder

2017-02-01 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15848947#comment-15848947 ] Matthias J. Sax commented on KAFKA-3729: Code freeze for {{0.10.2}} was last Friday. No change

[jira] [Commented] (KAFKA-4722) StreamThread should allow customization of thread prefix

2017-02-01 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15849057#comment-15849057 ] Matthias J. Sax commented on KAFKA-4722: How does this relate to multiple KStream / KTables

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-01 Thread Matthias J. Sax
Thanks for the KIP Damian. I am wondering about two things: 1. what should be the default value for the new parameter? 2. why is the new parameter provided in ms? About (2): because "the minimum checkpoint interval will be the value of commit.interval.ms. In effect the actual checkpoint

[jira] [Commented] (KAFKA-4722) Add application.id to StreamThread name

2017-02-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15850331#comment-15850331 ] Matthias J. Sax commented on KAFKA-4722: Thanks for the hint. Removing the duplicate app-Id from

[jira] [Created] (KAFKA-4732) Unstable test: KStreamKTableJoinIntegrationTest.shouldCountClicksPerRegion[1]

2017-02-03 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4732: -- Summary: Unstable test: KStreamKTableJoinIntegrationTest.shouldCountClicksPerRegion[1] Key: KAFKA-4732 URL: https://issues.apache.org/jira/browse/KAFKA-4732

[jira] [Created] (KAFKA-4731) Add event-based session windows

2017-02-03 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4731: -- Summary: Add event-based session windows Key: KAFKA-4731 URL: https://issues.apache.org/jira/browse/KAFKA-4731 Project: Kafka Issue Type: New Feature

[DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-03 Thread Matthias J. Sax
Hi All, I did prepare a KIP to do some cleanup some of Kafka's Streaming API. Please have a look here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-120%3A+Cleanup+Kafka+Streams+builder+API Looking forward to your feedback! -Matthias signature.asc Description: OpenPGP digital

[jira] [Updated] (KAFKA-3856) Cleanup Kafka Streams builder API

2017-02-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3856?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3856: --- Summary: Cleanup Kafka Streams builder API (was: Move inner classes accessible only

[jira] [Updated] (KAFKA-3856) Move inner classes accessible only functions in TopologyBuilder out of public APIs

2017-02-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3856?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-3856: --- Description: https://cwiki.apache.org/confluence/display/KAFKA/KIP-120%3A+Cleanup+Kafka

[jira] [Created] (KAFKA-4733) Improve Streams Reset Tool console output

2017-02-03 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4733: -- Summary: Improve Streams Reset Tool console output Key: KAFKA-4733 URL: https://issues.apache.org/jira/browse/KAFKA-4733 Project: Kafka Issue Type

[jira] [Updated] (KAFKA-4720) Add KStream.peek(ForeachAction<K,V>)

2017-02-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4720?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4720: --- Labels: needs-kip (was: ) > Add KStream.peek(ForeachAction&

[jira] [Commented] (KAFKA-3543) Allow a variant of transform() which can emit multiple values

2017-01-31 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3543?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15847802#comment-15847802 ] Matthias J. Sax commented on KAFKA-3543: [~gfodor] I close this as duplicate. Nevertheless one

[jira] [Resolved] (KAFKA-3543) Allow a variant of transform() which can emit multiple values

2017-01-31 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3543?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-3543. Resolution: Duplicate > Allow a variant of transform() which can emit multiple val

[jira] [Created] (KAFKA-4718) Revisit DSL partitioning assumption for KStream source topics

2017-01-31 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4718: -- Summary: Revisit DSL partitioning assumption for KStream source topics Key: KAFKA-4718 URL: https://issues.apache.org/jira/browse/KAFKA-4718 Project: Kafka

[jira] (KAFKA-4317) RocksDB checkpoint files lost on kill -9

2017-01-31 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4317?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4317: --- Labels: architecture needs-kip user-experience (was: architecture user-experience

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Matthias J. Sax
quot; belong in two different levels of the hierarchy. They both > transform two (or more) streams into one. > > Gwen > > On Fri, Feb 3, 2017 at 3:33 PM, Matthias J. Sax <matth...@confluent.io> wrote: >> Hi All, >> >> I did prepare a KIP to do some cleanup som

Re: KIP-121 [Discuss]: Add KStream peek method

2017-02-06 Thread Matthias J. Sax
Steven, Thanks for your KIP. I move this discussion to dev mailing list -- KIPs need to be discussed there (and can be cc'ed to user list). Can you also add the KIP to the table "KIPs under discussion":

[jira] [Commented] (KAFKA-4731) Add event-based session windows

2017-02-06 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4731?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15855287#comment-15855287 ] Matthias J. Sax commented on KAFKA-4731: Feel free to take it. I don't see any conflict

[jira] [Commented] (KAFKA-4738) Remove generic type of class ClientState

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15856683#comment-15856683 ] Matthias J. Sax commented on KAFKA-4738: Maybe. But we should have a new JIRA for this case. It's

[jira] [Updated] (KAFKA-4721) KafkaStreams (and possibly others) should inherit Closeable

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4721?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4721: --- Labels: needs-kip (was: ) > KafkaStreams (and possibly others) should inherit Closea

[jira] [Created] (KAFKA-4742) ResetTool does not commit offsets correclty

2017-02-07 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4742: -- Summary: ResetTool does not commit offsets correclty Key: KAFKA-4742 URL: https://issues.apache.org/jira/browse/KAFKA-4742 Project: Kafka Issue Type

[jira] [Updated] (KAFKA-4697) Simplify Streams Reset Tool

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4697?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4697: --- Affects Version/s: 0.10.2.0 > Simplify Streams Reset T

Re: KIP-121 [VOTE]: Add KStream peek method

2017-02-08 Thread Matthias J. Sax
+1 On 2/8/17 4:51 PM, Gwen Shapira wrote: > +1 (binding) > > On Wed, Feb 8, 2017 at 4:45 PM, Steven Schlansker > wrote: >> Hi everyone, >> >> Thank you for constructive feedback on KIP-121, >> KStream.peek(ForeachAction) ; >> it seems like it is time to call a

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-02 Thread Matthias J. Sax
so need to change the checkpoint setting (i.e, you > still only want to checkpoint every n minutes). > > On Wed, 1 Feb 2017 at 23:46 Matthias J. Sax <matth...@confluent.io> wrote: > >> Thanks for the KIP Damian. >> >> I am wondering about two things: >>

[jira] [Updated] (KAFKA-4317) RocksDB checkpoint files lost on kill -9

2017-02-02 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4317?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4317: --- Description: Right now, the checkpoint files for logged RocksDB stores are written during

[jira] [Created] (KAFKA-4738) Remove generic type of class ClientState

2017-02-06 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4738: -- Summary: Remove generic type of class ClientState Key: KAFKA-4738 URL: https://issues.apache.org/jira/browse/KAFKA-4738 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-4564) When the destination brokers are down or misconfigured in config, Streams should fail fast

2017-02-06 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15854436#comment-15854436 ] Matthias J. Sax commented on KAFKA-4564: With new client compatibility in {{0.10.2}} we did add

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-02-06 Thread Matthias J. Sax
native approach with this new API. >> >> KStreamBuilder.newName -- Similar to addInternalTopic, I use this to create >> processor names in reusable components. Lacking this method would be >> fairly easy to work around. >> >> Mathieu >> >> >>

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Matthias J. Sax
Jorge, thanks for you KIP. I like it a lot and think it will be a nice addition! -Matthias On 2/7/17 7:04 PM, Dong Lin wrote: > Hey Jorge, > > Thanks for the KIP. I have some quick comments: > > - Should we allow user to use wildcard to reset offset of all groups for a > given topic as

Re: [DISCUSS] KIP-116 - Add State Store Checkpoint Interval Configuration

2017-02-07 Thread Matthias J. Sax
sn't make sense to disable it, but then i'm sure someone > will come up with a reason they don't want it! > I'm happy to change it such that the checkpoint interval must be > 0. > > Cheers, > Damian > > On Fri, 3 Feb 2017 at 01:29 Matthias J. Sax <matth...@confluent.io>

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Matthias J. Sax
Jorge, can you please add your KIP to this table: https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals#KafkaImprovementProposals-KIPsunderdiscussion Thanks! -Matthias On 2/7/17 9:29 PM, Matthias J. Sax wrote: > Jorge, > > thanks for you KIP. I like

[jira] [Assigned] (KAFKA-4743) Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4743?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-4743: -- Assignee: Matthias J. Sax > Add a tool to Reset Consumer Group Offs

[jira] [Updated] (KAFKA-4743) Add a tool to Reset Consumer Group Offsets

2017-02-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4743?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4743: --- Labels: kip (was: ) > Add a tool to Reset Consumer Group Offs

  1   2   3   4   5   6   7   8   9   10   >