Re: [VOTE] KIP-160: Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-06-22 Thread James Chain
Hi all, I apply original idea on KStream#writeAsText() and also update my pull request. Please re-review and re-cast the vote. James Chien

Re: Kafka streams KStream and ktable join issue

2017-06-22 Thread Matthias J. Sax
Hi, can you reproduce the error reliably? Are use using 0.10.2.0 or 0.10.2.1? It's unclear to me, how an NPE can occur. It seems to happen within Streams library. Might be a bug. Not sure atm. -Matthias On 6/22/17 9:43 AM, Shekar Tippur wrote: > Hello, > > I am trying to perform a simple join

[GitHub] kafka pull request #3417: MINOR: Correct the ConsumerPerformance print forma...

2017-06-22 Thread ConeyLiu
GitHub user ConeyLiu opened a pull request: https://github.com/apache/kafka/pull/3417 MINOR: Correct the ConsumerPerformance print format Currently, the output of `ConsumerPerformance` looks strange. Before the `header` format as follow: ``` "time, threadId, data.consumed.in

Re: [VOTE] 0.11.0.0 RC2

2017-06-22 Thread Ismael Juma
A quick note on notable changes since rc1: 1. A significant performance improvement if transactions are enabled: https://github.com/apache/kafka/commit/f239f1f839f8bcbd80cce2a4a8643e15d340be8e 2. Fixed a controller regression if many brokers are started simultaneously: https://github.com/apache/ka

[VOTE] 0.11.0.0 RC2

2017-06-22 Thread Ismael Juma
Hello Kafka users, developers and client-developers, This is the third candidate for release of Apache Kafka 0.11.0.0. This is a major version release of Apache Kafka. It includes 32 new KIPs. See the release notes and release plan ( https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-22 Thread Matthias J. Sax
I also think, that one config is better, with two default implementations: failing and log-and-continue However, I think we should fail by default. Similar to timestamp extractor as "silent" data loss is no good default behavior IMHO. -Matthias On 6/22/17 11:00 AM, Eno Thereska wrote: > Answers

Jenkins build is back to normal : kafka-0.11.0-jdk7 #187

2017-06-22 Thread Apache Jenkins Server
See

Minimum Replication Factor

2017-06-22 Thread Stephane Maarek
Hi all, Interested in getting people’s opinion on something. The problem I have is that some people launch streams app in our cluster but forget to set a replication factor > 1. Then it’s a pain to increase the topic’s RF, when we do notice some topic partitions go offline because we reboot

Re: CreateTopicResponse Error Code 41

2017-06-22 Thread Ismael Juma
Yes, it needs to be sent to the Controller. Metadata response has the controller id. Ismael On Fri, Jun 23, 2017 at 12:45 AM, Vineet Goel wrote: > Hi, > > I get an Error (Code 41) when sending a CreateTopicRequest to any of the > brokers except 1. Why might this be? Does this request need to be

CreateTopicResponse Error Code 41

2017-06-22 Thread Vineet Goel
Hi, I get an Error (Code 41) when sending a CreateTopicRequest to any of the brokers except 1. Why might this be? Does this request need to be sent to a specific broker? Best, Vineet

[GitHub] kafka pull request #2575: MINOR: update AWS test setup guide

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2575 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[GitHub] kafka pull request #3416: MINOR: improve test README

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3416 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[GitHub] kafka pull request #3416: MINOR: improve test README

2017-06-22 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/3416 MINOR: improve test README You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka minor-aws Alternatively you can review and apply these cha

Jenkins build is back to normal : kafka-trunk-jdk7 #2450

2017-06-22 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #3414: HOTFIX: reduce log verbosity on commit

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3414 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Created] (KAFKA-5504) Kafka controller is not getting elected

2017-06-22 Thread Ashish Kumar (JIRA)
Ashish Kumar created KAFKA-5504: --- Summary: Kafka controller is not getting elected Key: KAFKA-5504 URL: https://issues.apache.org/jira/browse/KAFKA-5504 Project: Kafka Issue Type: Bug

Build failed in Jenkins: kafka-trunk-jdk7 #2449

2017-06-22 Thread Apache Jenkins Server
See -- [...truncated 2.42 MB...] org.apache.kafka.streams.integration.QueryableStateIntegrationTest > shouldBeAbleToQueryStateWithNonZeroSizedCache PASSED org.apache.kafka.streams.inte

[GitHub] kafka pull request #3413: KAFKA-5502: read current brokers from zookeeper up...

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3413 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[GitHub] kafka pull request #3415: MINOR: Make 'Topic-Level Configs' a doc section fo...

2017-06-22 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/3415 MINOR: Make 'Topic-Level Configs' a doc section for easier access You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian/kafka d

Jenkins build is back to normal : kafka-trunk-jdk8 #1747

2017-06-22 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-5503) Idempotent producer ignores shutdown while fetching ProducerId

2017-06-22 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5503: -- Summary: Idempotent producer ignores shutdown while fetching ProducerId Key: KAFKA-5503 URL: https://issues.apache.org/jira/browse/KAFKA-5503 Project: Kafka

Build failed in Jenkins: kafka-trunk-jdk7 #2448

2017-06-22 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-5498: ConfigDef derived from another ConfigDef did not correctly -- [...truncated 967.04 KB...] kafka.log.ProducerStateManagerTest > testTrunc

[GitHub] kafka pull request #3414: HOTFIX: reduce log verbosity on commit

2017-06-22 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/3414 HOTFIX: reduce log verbosity on commit You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka hotfix-commit-logging Alternatively you can re

[GitHub] kafka pull request #3413: KAFKA-5502: read current brokers from zookeeper up...

2017-06-22 Thread onurkaraman
GitHub user onurkaraman opened a pull request: https://github.com/apache/kafka/pull/3413 KAFKA-5502: read current brokers from zookeeper upon processing broker change Dong Lin's testing of the 0.11.0 release revealed a controller-side performance regression in clusters with many br

[GitHub] kafka pull request #3412: KAFKA-5498: ConfigDef derived from another ConfigD...

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3412 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Resolved] (KAFKA-5498) Connect validation API stops returning recommendations for some fields after the right sequence of requests

2017-06-22 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5498?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira resolved KAFKA-5498. - Resolution: Fixed Fix Version/s: 0.11.1.0 Issue resolved by pull request 3412 [https://gith

[jira] [Created] (KAFKA-5502) read current brokers from zookeeper upon processing broker change

2017-06-22 Thread Onur Karaman (JIRA)
Onur Karaman created KAFKA-5502: --- Summary: read current brokers from zookeeper upon processing broker change Key: KAFKA-5502 URL: https://issues.apache.org/jira/browse/KAFKA-5502 Project: Kafka

[jira] [Created] (KAFKA-5501) use async zookeeper apis everywhere

2017-06-22 Thread Onur Karaman (JIRA)
Onur Karaman created KAFKA-5501: --- Summary: use async zookeeper apis everywhere Key: KAFKA-5501 URL: https://issues.apache.org/jira/browse/KAFKA-5501 Project: Kafka Issue Type: Sub-task

[GitHub] kafka pull request #3412: KAFKA-5498: ConfigDef derived from another ConfigD...

2017-06-22 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/3412 KAFKA-5498: ConfigDef derived from another ConfigDef did not correctly compute parentless configs You can merge this pull request into a Git repository by running: $ git pull https://github.co

Build failed in Jenkins: kafka-trunk-jdk8 #1746

2017-06-22 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5490; Skip empty record batches in the consumer [ismael] MINOR: Switch ZK client logging to INFO -- [...truncated 992.95 KB...] kafka.utils.Co

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-22 Thread Eno Thereska
Answers inline: > On 22 Jun 2017, at 03:26, Guozhang Wang wrote: > > Thanks for the updated KIP, some more comments: > > 1.The config name is "default.deserialization.exception.handler" while the > interface class name is "RecordExceptionHandler", which is more general > than the intended purp

[jira] [Resolved] (KAFKA-2611) Document MirrorMaker

2017-06-22 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2611?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira resolved KAFKA-2611. - Resolution: Fixed MirrorMaker is actually documented. > Document MirrorMaker > --

[jira] [Created] (KAFKA-5500) it is impossible to have custom Login Modules for PLAIN SASL mechanism

2017-06-22 Thread Anton Patrushev (JIRA)
Anton Patrushev created KAFKA-5500: -- Summary: it is impossible to have custom Login Modules for PLAIN SASL mechanism Key: KAFKA-5500 URL: https://issues.apache.org/jira/browse/KAFKA-5500 Project: Kaf

Re: [VOTE] KIP-168: Add TotalTopicCount metric per cluster

2017-06-22 Thread Onur Karaman
+1 On Thu, Jun 22, 2017 at 10:05 AM, Dong Lin wrote: > Thanks for the KIP. +1 (non-binding) > > On Wed, Jun 21, 2017 at 1:17 PM, Abhishek Mendhekar < > abhishek.mendhe...@gmail.com> wrote: > > > Hi Kafka Dev, > > > > I did like to start the voting on - > > https://cwiki.apache.org/confluence/dis

Re: [VOTE] KIP-168: Add TotalTopicCount metric per cluster

2017-06-22 Thread Dong Lin
Thanks for the KIP. +1 (non-binding) On Wed, Jun 21, 2017 at 1:17 PM, Abhishek Mendhekar < abhishek.mendhe...@gmail.com> wrote: > Hi Kafka Dev, > > I did like to start the voting on - > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 168%3A+Add+TotalTopicCount+metric+per+cluster > > Disc

[jira] [Created] (KAFKA-5499) Double check how we handle exceptions when commits fail

2017-06-22 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5499: --- Summary: Double check how we handle exceptions when commits fail Key: KAFKA-5499 URL: https://issues.apache.org/jira/browse/KAFKA-5499 Project: Kafka Issue Typ

Jenkins build is back to normal : kafka-trunk-jdk7 #2446

2017-06-22 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-0.11.0-jdk7 #185

2017-06-22 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5490; Skip empty record batches in the consumer [ismael] MINOR: Switch ZK client logging to INFO -- [...truncated 966.79 KB...] kafka.utils.Ut

[jira] [Created] (KAFKA-5498) Connect validation API stops returning recommendations for some fields after the right sequence of requests

2017-06-22 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-5498: Summary: Connect validation API stops returning recommendations for some fields after the right sequence of requests Key: KAFKA-5498 URL: https://issues.apache.org

Kafka streams KStream and ktable join issue

2017-06-22 Thread Shekar Tippur
Hello, I am trying to perform a simple join operation. I am using Kafka 0.10.2 I have a "raw" table and a "cache" topics and just 1 partition in my local environment. ktable has these entries {"Joe": {"location": "US", "gender": "male"}} {"Julie": {"location": "US", "gender": "female"}} {"Kawas

[GitHub] kafka pull request #3411: KAFKA-5487: upgrade and downgrade streams app syst...

2017-06-22 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/3411 KAFKA-5487: upgrade and downgrade streams app system test You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka KAFKA-5487-upgr

Re: [DISCUSS] KIP-168: Add TotalTopicCount metric per cluster

2017-06-22 Thread Abhishek Mendhekar
Hi Kafka Dev, Below is the link to the update KIP proposal. https://cwiki.apache.org/confluence/display/KAFKA/KIP-168%3A+Add+TopicCount+metric+per+cluster Thanks, Abhishek On Wed, Jun 21, 2017 at 3:55 PM, Abhishek Mendhekar < abhishek.mendhe...@gmail.com> wrote: > Hi Dong, > > Thanks for the su

Re: [VOTE] 0.11.0.0 RC1

2017-06-22 Thread Tom Crayford
That's fair, and nice find with the transaction performance improvement! Once the RC is out, we'll do a final round of performance testing with the new ProducerPerformance changes enabled. I think it's fair that this shouldn't delay the release. Is there an official stance on what should and shou

[GitHub] kafka pull request #3409: MINOR: Switch ZK client logging to INFO

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3409 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[GitHub] kafka pull request #3408: KAFKA-5490: Skip empty record batches in the consu...

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3408 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

Re: [VOTE] 0.11.0.0 RC1

2017-06-22 Thread Ismael Juma
Hi Tom, We are going to do another RC to include Apurva's significant performance improvement when transactions are enabled: https://github.com/apache/kafka/commit/f239f1f839f8bcbd80cce2a4a8643e15d340be8e Given that, we can also include the ProducerPerformance changes that Apurva did to find and

Re: [VOTE] 0.11.0.0 RC0

2017-06-22 Thread Ismael Juma
Thanks Tom. Sounds good. :) Ismael On Thu, Jun 22, 2017 at 4:21 PM, Tom Crayford wrote: > Hi Ismal, > > Sure. It's a standard heroku plan, the `extended-2`, which has 8 brokers. > We did several rounds of performance testing, some with low numbers of > partitions and topics (16 partitions with

[GitHub] kafka pull request #3401: MINOR: explain producer naming within Streams

2017-06-22 Thread mjsax
Github user mjsax closed the pull request at: https://github.com/apache/kafka/pull/3401 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabl

[GitHub] kafka pull request #3410: KAFKA-4913: prevent creation of window stores with...

2017-06-22 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3410 KAFKA-4913: prevent creation of window stores with less than 2 segments Throw IllegalArgumentException when attempting to create a `WindowStore` via `Stores` or directly with `RocksDBWindowStoreSupplie

Build failed in Jenkins: kafka-trunk-jdk7 #2445

2017-06-22 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-4785; Records from internal repartitioning topics should always -- [...truncated 895.30 KB...] kafka.server.epoch.LeaderEpochFileCacheTes

Re: [VOTE] 0.11.0.0 RC0

2017-06-22 Thread Tom Crayford
Hi Ismal, Sure. It's a standard heroku plan, the `extended-2`, which has 8 brokers. We did several rounds of performance testing, some with low numbers of partitions and topics (16 partitions with one topic, which we typically see the highest throughput on) and some with many more (many thousands

Re: [VOTE] 0.11.0.0 RC0

2017-06-22 Thread Ismael Juma
Hi Tom, Would you be able to share the details of the Kafka Cluster that you used for performance testing? We are interested in the number of brokers, topics, partitions per broker and replication factor. Thanks! Ismael On Mon, Jun 19, 2017 at 3:15 PM, Tom Crayford wrote: > Hello, > > Heroku h

[GitHub] kafka pull request #3409: MINOR: Switch ZK client logging to INFO

2017-06-22 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/3409 MINOR: Switch ZK client logging to INFO You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka tweak-log-config Alternatively you can review

[GitHub] kafka pull request #3293: KAFKA-4658: Improve test coverage InMemoryKeyValue...

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3293 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

Re: [DISCUSS] KIP-170: Enhanced TopicCreatePolicy and introduction of TopicDeletePolicy

2017-06-22 Thread Ismael Juma
Thanks for the KIP, Edoardo. A few comments: 1. Have you considered extending RequestMetadata with the additional information you need? We could add Cluster to it, which has topic assignment information, for example. This way, there would be no need for a V2 interface. 2. Something else that coul

[GitHub] kafka pull request #3290: KAFKA-4655: Improve test coverage of CompositeRead...

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3290 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

Long awaiting pull request

2017-06-22 Thread Arseniy Tashoyan
Friends, I have created a pull request 1.5 months ago and asked 5 or 6 people for review: https://github.com/apache/kafka/pull/3051 Nobody answers. Maybe I am doing something wrong? I understand, everybody is busy, but it would be great to receive any communication. I cannot also assign the corre

[DISCUSS] KIP-170: Enhanced TopicCreatePolicy and introduction of TopicDeletePolicy

2017-06-22 Thread Edoardo Comar
Hi all, We've drafted "KIP-170: Enhanced TopicCreatePolicy and introduction of TopicDeletePolicy" for discussion: https://cwiki.apache.org/confluence/display/KAFKA/KIP-170%3A+Enhanced+TopicCreatePolicy+and+introduction+of+TopicDeletePolicy Please take a look. Your feedback is welcome and much n

[jira] [Resolved] (KAFKA-4059) Documentation still refers to AsyncProducer and SyncProducer

2017-06-22 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4059?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4059. Resolution: Fixed Fix Version/s: 0.11.0.0 > Documentation still refers to AsyncProducer and S

[GitHub] kafka pull request #3372: Provide link to ZooKeeper within Quickstart

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3372 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Created] (KAFKA-5497) KIP-170: Enhanced CreateTopicPolicy and DeleteTopicPolicy

2017-06-22 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-5497: Summary: KIP-170: Enhanced CreateTopicPolicy and DeleteTopicPolicy Key: KAFKA-5497 URL: https://issues.apache.org/jira/browse/KAFKA-5497 Project: Kafka Issue

[GitHub] kafka pull request #3385: KAFKA-4059: Documentation still refers to AsyncPro...

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3385 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[GitHub] kafka pull request #1535: KAFKA-3727 - ConsumerListener for UnknownTopicOrPa...

2017-06-22 Thread edoardocomar
Github user edoardocomar closed the pull request at: https://github.com/apache/kafka/pull/1535 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature i

[GitHub] kafka pull request #1545: KAFKA-3727 - ClientListener for UnknownTopicOrPart...

2017-06-22 Thread edoardocomar
Github user edoardocomar closed the pull request at: https://github.com/apache/kafka/pull/1545 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature i

[GitHub] kafka pull request #3408: KAFKA-5490: Skip empty record batches in the consu...

2017-06-22 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/3408 KAFKA-5490: Skip empty record batches in the consumer The proper fix for KAFKA-5490 (including tests) is in https://github.com/apache/kafka/pull/3406. This is just the consumer change that

[GitHub] kafka pull request #3402: KAFKA-5486: org.apache.kafka logging should go to ...

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3402 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

Re: Handling 2 to 3 Million Events before Kafka

2017-06-22 Thread SenthilKumar K
Hi Barton - I think we can use Async Producer with Call Back api(s) to keep track on which event failed .. --Senthil On Thu, Jun 22, 2017 at 4:58 PM, SenthilKumar K wrote: > Thanks Barton.. I'll look into these .. > > On Thu, Jun 22, 2017 at 7:12 AM, Garrett Barton > wrote: > >> Getting good

Re: Handling 2 to 3 Million Events before Kafka

2017-06-22 Thread SenthilKumar K
Thanks Barton.. I'll look into these .. On Thu, Jun 22, 2017 at 7:12 AM, Garrett Barton wrote: > Getting good concurrency in a webapp is more than doable. Check out these > benchmarks: > https://www.techempower.com/benchmarks/#section=data-r14&hw=ph&test=db > I linked to the single query one be

[GitHub] kafka pull request #3106: KAFKA-4785: Records from internal repartitioning t...

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3106 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-22 Thread Damian Guy
Thanks everyone. My latest attempt is below. It builds on the fluent approach, but i think it is slightly nicer. I agree with some of what Eno said about mixing configy stuff in the DSL, but i think that enabling caching and enabling logging are things that aren't actually config. I'd probably not

[jira] [Created] (KAFKA-5496) Consistency in documentation

2017-06-22 Thread Tom Bentley (JIRA)
Tom Bentley created KAFKA-5496: -- Summary: Consistency in documentation Key: KAFKA-5496 URL: https://issues.apache.org/jira/browse/KAFKA-5496 Project: Kafka Issue Type: Improvement Comp

[GitHub] kafka pull request #3407: KAFKA-3881: Remove the replacing logic from "." to...

2017-06-22 Thread tombentley
GitHub user tombentley opened a pull request: https://github.com/apache/kafka/pull/3407 KAFKA-3881: Remove the replacing logic from "." to "_" in Fetcher You can merge this pull request into a Git repository by running: $ git pull https://github.com/tombentley/kafka KAFKA-3881

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-22 Thread Ismael Juma
Thanks Damian. I think both options have pros and cons. And both are better than overload abuse. The fluent API approach reads better, no mention of builder or build anywhere. The main downside is that the method signatures are a little less clear. By reading the method signature, one doesn't nece

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-22 Thread Jan Filipiak
Hi Eno, I am less interested in the user facing interface but more in the actual implementation. Any hints where I can follow the discussion on this? As I still want to discuss upstreaming of KAFKA-3705 with someone Best Jan On 21.06.2017 17:24, Eno Thereska wrote: (cc’ing user-list too)

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-22 Thread Eno Thereska
Note that while I agree with the initial proposal (withKeySerdes, withJoinType, etc), I don't agree with things like .materialize(), .enableCaching(), .enableLogging(). The former maintain the declarative DSL, while the later break the declarative part by mixing system decisions in the DSL. I

Build failed in Jenkins: kafka-trunk-jdk7 #2444

2017-06-22 Thread Apache Jenkins Server
See Changes: [damian.guy] MINOR: Turn off caching in demos for more understandable outputs -- [...truncated 1.16 MB...] kafka.server.ClientQuotaManagerTest > testUserClien

[GitHub] kafka pull request #3403: MINOR: Turn off caching in demos for more understa...

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3403 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[GitHub] kafka pull request #3291: KAFKA-4659: Improve test coverage of CachingKeyVal...

2017-06-22 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3291 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab