Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

2017-09-11 Thread James Cheng
Thanks for the KIP, Randall. The KIP has one MBean per metric name. Can I suggest an alternate grouping? kafka.connect:type=connector-metrics,connector=([-.\w]+) connector-type connector-class connector-version status kafka.connect:type=task-metrics,connector=([-.

[GitHub] kafka pull request #3835: MINOR: update operations doc on topic deletion

2017-09-11 Thread omkreddy
GitHub user omkreddy opened a pull request: https://github.com/apache/kafka/pull/3835 MINOR: update operations doc on topic deletion You can merge this pull request into a Git repository by running: $ git pull https://github.com/omkreddy/kafka update-delete-topic-doc Alternat

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

2017-09-11 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Tweak detection of kafka server start-up in system tests -- [...truncated 2.50 MB...] org.apache.kafka.streams.integration.KTableKTableJoinIn

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

2017-09-11 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Tweak detection of kafka server start-up in system tests -- [...truncated 2.03 MB...] org.apache.kafka.common.security.ssl.SslFactoryTest > te

[GitHub] kafka pull request #3834: MINOR: Tweak detection of kafka server start-up in...

2017-09-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3834 ---

Re: [DISCUSS] KIP-199: Add Kafka Connect offset reset tool

2017-09-11 Thread Ewen Cheslack-Postava
A couple of comments: * I made some minor, non-critical updates to the motivation section to add a bit more color/background/clarity. In particular, clarifying how things are connected to consumer groups for sinks. Still, the motivation isn't entirely clear about how this works -- it is definitely

Re: [DISCUSS] KIP-197: Include Connector type in Connector REST API

2017-09-11 Thread Ismael Juma
Thanks for the KIP, +1 (binding). Ismael On Fri, Sep 8, 2017 at 6:00 PM, Ted Yu wrote: > Hi, > Please take a look at: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 197+Connect+REST+API+should+include+the+connector+type+ > when+describing+a+connector > > Thanks >

[GitHub] kafka pull request #3834: MINOR: Tweak detection of kafka server start-up in...

2017-09-11 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/3834 MINOR: Tweak detection of kafka server start-up in system tests You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka tweak-system-test-reg

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-11 Thread Apurva Mehta
Hi Becket, You are right: the calculations are per partition produced to by each idempotent producer. I actually think this makes the problem more acute when we actually end up enabling the idempotent producer by default. However, even the most optimized version will still result in an overhead of

Re: [DISCUSS] KIP-197: Include Connector type in Connector REST API

2017-09-11 Thread Ewen Cheslack-Postava
Awesome turnaround time :) Trying to get another committer to take a look so we can get this in for the next release. -Ewen On Mon, Sep 11, 2017 at 8:30 PM, Ted Yu wrote: > Updated KIP-197 with the reference to KIP-151 > > On Mon, Sep 11, 2017 at 8:24 PM, Ewen Cheslack-Postava > wrote: > > > Y

Re: [DISCUSS] KIP-197: Include Connector type in Connector REST API

2017-09-11 Thread Ted Yu
Updated KIP-197 with the reference to KIP-151 On Mon, Sep 11, 2017 at 8:24 PM, Ewen Cheslack-Postava wrote: > Yeah, this all seems reasonable to me as well. Even if there are other > places we should add this info, these seem like the really useful ones. > > re: the enum generating lowercase, th

Re: [VOTE] KIP-197: Include Connector type in Connector REST API

2017-09-11 Thread Ewen Cheslack-Postava
+1 binding. Thanks for the contribution Ted! Simple addition, but makes the API significantly more usable. -Ewen On Fri, Sep 8, 2017 at 7:46 PM, Sriram Subramanian wrote: > +1 > > On Fri, Sep 8, 2017 at 4:33 PM, Randall Hauch wrote: > > > +1 (non-binding) > > > > Randall > > > > On Fri, Sep 8

Re: [DISCUSS] KIP-197: Include Connector type in Connector REST API

2017-09-11 Thread Ewen Cheslack-Postava
Yeah, this all seems reasonable to me as well. Even if there are other places we should add this info, these seem like the really useful ones. re: the enum generating lowercase, that came in KIP-151 (which was the first to expose this info, and technically is enough to not *require* this change, b

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

2017-09-11 Thread Ewen Cheslack-Postava
re: questions about additional metrics, I think we'll undoubtedly find more that people want in practice, but as I mentioned earlier I think it's better to add the ones we know we need and then fill out the rest as we figure it out. So, e.g., batch size metrics sound like they could be useful, but

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

2017-09-11 Thread Ewen Cheslack-Postava
On Mon, Sep 11, 2017 at 4:50 PM, Randall Hauch wrote: > Thanks, Ewen. Comments inline below. > > On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava > wrote: > > > Randall, > > > > A couple of questions: > > > > * Some metrics don't seem to have unique names? e.g. > > source-record-produce-ra

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-11 Thread Becket Qin
Hi Apurva, Thanks for the explanation. I think the STO will be per producer/partition, right? Am I missing something? You are right that the proposal does not strengthen the semantic. The goal is more about trying to bound the memory consumption to some reasonable number and use the memory more e

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

2017-09-11 Thread Randall Hauch
Based on Roger and Ewen's feedback, I removed the aggregate metrics as they would be difficult to make use of without extra work. This simplified things a great deal, and I took the opportunity to reorganize the groups of metrics. Also, based upon Ewen's concerns regarding measuring times/durations

Re: [DISCUSS] KIP-195: AdminClient.increasePartitions

2017-09-11 Thread Ismael Juma
Hi Tom, A couple of comments: 1. "This API is synchronous in the sense that the client can assume that the partition count has been changed (or the request was rejected) once they have obtained the result for the topic from the CreatePartitionsResult." If you want to do this, I think you'd have t

[jira] [Created] (KAFKA-5871) bound the throttle time in byte rate quota

2017-09-11 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-5871: -- Summary: bound the throttle time in byte rate quota Key: KAFKA-5871 URL: https://issues.apache.org/jira/browse/KAFKA-5871 Project: Kafka Issue Type: Improvement

Re: [VOTE] KIP-199: Add Kafka Connect Offset Tool

2017-09-11 Thread Sriram Subramanian
+1 On Mon, Sep 11, 2017 at 2:56 PM, Gwen Shapira wrote: > +1 > > On Mon, Sep 11, 2017 at 1:33 PM Ted Yu wrote: > > > +1 > > > > On Mon, Sep 11, 2017 at 7:43 AM, Randall Hauch wrote: > > > > > I'd like to start the vote on KIP-199 to add a command line tool that > > will > > > allow Connect ope

Re: [VOTE] KIP-196: Add metrics to Kafka Connect framework

2017-09-11 Thread Sriram Subramanian
+1 On Mon, Sep 11, 2017 at 2:56 PM, Gwen Shapira wrote: > +1 > > Thanks for this. Can't wait for more complete monitoring for Connect. > > On Mon, Sep 11, 2017 at 7:40 AM Randall Hauch wrote: > > > I'd like to start the vote on KIP-196 to add metrics to the Kafka Connect > > framework so the wo

Understanding the semantics of Selector.poll()

2017-09-11 Thread Javed, Haseeb
Hello, I was looking into Kafka's network layer code have a few questions regarding the Selector class, particularly who the poll() method is implemented. The poll() method goes something like this: /* check ready keys */ long startSelect = time.nanoseconds(); int ready

[GitHub] kafka pull request #3833: MINOR: refactor build method to extract methods fr...

2017-09-11 Thread bbejeck
GitHub user bbejeck opened a pull request: https://github.com/apache/kafka/pull/3833 MINOR: refactor build method to extract methods from if statements You can merge this pull request into a Git repository by running: $ git pull https://github.com/bbejeck/kafka MINOR_extract_m

[GitHub] kafka pull request #3832: KAFKA-5854: (WIP) Handle SASL authentication failu...

2017-09-11 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/3832 KAFKA-5854: (WIP) Handle SASL authentication failures as non-retriable exceptions in clients This PR depends on the in progress [PR for KAFKA-4764](https://github.com/apache/kafka/pull/3708)

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

2017-09-11 Thread Randall Hauch
Thanks, Ewen. Comments inline below. On Mon, Sep 11, 2017 at 5:46 PM, Ewen Cheslack-Postava wrote: > Randall, > > A couple of questions: > > * Some metrics don't seem to have unique names? e.g. > source-record-produce-rate and source-record-produce-total seem like they > are duplicated. Looks li

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

2017-09-11 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Implement toString for NetworkClient#InFlightRequest -- [...truncated 2.50 MB...] org.apache.kafka.streams.integration.KTableKTableJoinIntegra

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

2017-09-11 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Implement toString for NetworkClient#InFlightRequest -- [...truncated 2.03 MB...] org.apache.kafka.common.security.ssl.SslFactoryTest > testCl

[GitHub] kafka pull request #3831: KAFKA-5520: KIP-171 - Extend Consumer Group Reset ...

2017-09-11 Thread jeqo
GitHub user jeqo opened a pull request: https://github.com/apache/kafka/pull/3831 KAFKA-5520: KIP-171 - Extend Consumer Group Reset Offset for Stream Application KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-171+-+Extend+Consumer+Group+Reset+Offset+for+Stream+Applicati

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-11 Thread Apurva Mehta
Hi Becket, Regarding the current implementation: we opted for a simpler server side implementation where we _don't_ snapshot the metadata of the last 5 batches to disk. So if a broker fails, comes back online, and is the leader again, it will only have the last batch in memory. With max.in.flight

[GitHub] kafka pull request #3805: MINOR: Implement toString for NetworkClient#InFlig...

2017-09-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3805 ---

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

2017-09-11 Thread Ewen Cheslack-Postava
Randall, A couple of questions: * Some metrics don't seem to have unique names? e.g. source-record-produce-rate and source-record-produce-total seem like they are duplicated. Looks like maybe just an oversight that the second ones should be changed from "produce" to "write". * I think there's a s

Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-11 Thread Ted Yu
To me, the validation you mentioned is not user friendly, assuming the values of linger.ms, request.timeout.ms, retry.backoff.ms are from cluster running previous release of Kafka. For fresh installation, the validation is fine. On Mon, Sep 11, 2017 at 2:50 PM, Sumant Tambe wrote: > @Ted, We th

[VOTE] KIP-171 - Extend Consumer Group Reset Offset for Stream Application

2017-09-11 Thread Jorge Esteban Quilcate Otoya
Hi All, It seems that there is no further concern with the KIP-171. At this point we would like to start the voting process. The KIP can be found here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-171+-+Extend+Consumer+Group+Reset+Offset+for+Stream+Application Thanks!

Re: [DISCUSS] KIP-171: Extend Consumer Group Reset Offset for Stream Application

2017-09-11 Thread Jorge Esteban Quilcate Otoya
Thanks Guozhang! I have updated the KIP to: 1. Only one scenario param is allowed. If none, `to-earliest` will be used, behaving as the current version. 2. 1. An exception will be printed mentioning that there is no existing offsets registered. 2. inputTopics format could support define part

Re: [VOTE] KIP-196: Add metrics to Kafka Connect framework

2017-09-11 Thread Gwen Shapira
+1 Thanks for this. Can't wait for more complete monitoring for Connect. On Mon, Sep 11, 2017 at 7:40 AM Randall Hauch wrote: > I'd like to start the vote on KIP-196 to add metrics to the Kafka Connect > framework so the worker processes can be measured. Details are here: > > > https://cwiki.ap

Re: [VOTE] KIP-199: Add Kafka Connect Offset Tool

2017-09-11 Thread Gwen Shapira
+1 On Mon, Sep 11, 2017 at 1:33 PM Ted Yu wrote: > +1 > > On Mon, Sep 11, 2017 at 7:43 AM, Randall Hauch wrote: > > > I'd like to start the vote on KIP-199 to add a command line tool that > will > > allow Connect operators to read, modify, and update source connector > > offsets. Details are he

Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-11 Thread Sumant Tambe
@Ted, We throw a ConfigException when user-configured values of linger.ms, request.timeout.ms, retry.backoff.ms add up to more than delivery.timeout.ms . The kip mentions this in the Validation section. On 11 September 2017 at 14:31, Ted Yu wrote: > bq. larger than default (linger.ms + request.t

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-11 Thread Becket Qin
Hi Apurva, Sorry for being late on this thread. I am trying to understand the implementation of case that we will throw DuplicateSequenceException. My understanding is the following: 1. On the broker side, we will cache 5 most recent sequence/timestamp/offset (STO) for each of the producer ID. 2.

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

2017-09-11 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-5817: [FOLLOW-UP] add SerializedInternal -- [...truncated 2.51 MB...] org.apache.kafka.streams.KafkaStreamsTest > shouldThrowExceptionSettin

Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-11 Thread Ted Yu
bq. larger than default (linger.ms + request.timeout.ms + retry.backoff.ms) I was not referring to the sum of default values for the above parameters. I was referring to the sum of user configured values for these parameters (since we don't know whether that sum is higher than 120 seconds or not)

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

2017-09-11 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-5817: [FOLLOW-UP] add SerializedInternal -- [...truncated 2.52 MB...] org.apache.kafka.streams.processor.UsePreviousTimeOnInvalidTimestampTes

Re: [VOTE] 0.11.0.1 RC0

2017-09-11 Thread Jun Rao
Hi, Damian, Thanks for running the release. Verified the quickstart from the src distribution. +1 Jun On Tue, Sep 5, 2017 at 1:34 PM, Damian Guy wrote: > Hello Kafka users, developers and client-developers, > > This is the first candidate for release of Apache Kafka 0.11.0.1. > > This is a bug

[jira] [Created] (KAFKA-5870) Idempotent producer: a producerId reset causes undesirable behavior for inflight batches to other partitions

2017-09-11 Thread Apurva Mehta (JIRA)
Apurva Mehta created KAFKA-5870: --- Summary: Idempotent producer: a producerId reset causes undesirable behavior for inflight batches to other partitions Key: KAFKA-5870 URL: https://issues.apache.org/jira/browse/KAFK

[jira] [Created] (KAFKA-5869) SMT to select Kafka topic based on field in message key or value

2017-09-11 Thread Yeva Byzek (JIRA)
Yeva Byzek created KAFKA-5869: - Summary: SMT to select Kafka topic based on field in message key or value Key: KAFKA-5869 URL: https://issues.apache.org/jira/browse/KAFKA-5869 Project: Kafka Iss

Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-11 Thread Jun Rao
Sumant, Thanks for the explanation. Since we default retries to MAX_INT, it seems that by default, the expiration shouldn't be out of order. Jun On Mon, Sep 11, 2017 at 10:06 AM, Sumant Tambe wrote: > @Jun, Until we make idempotent producer the default (kip-185), this kip is > sensitive to ret

Re: [VOTE] KIP-199: Add Kafka Connect Offset Tool

2017-09-11 Thread Ted Yu
+1 On Mon, Sep 11, 2017 at 7:43 AM, Randall Hauch wrote: > I'd like to start the vote on KIP-199 to add a command line tool that will > allow Connect operators to read, modify, and update source connector > offsets. Details are here: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 1

[GitHub] kafka pull request #3825: KAFKA-5817: [FOLLOW-UP] add SerializedInternal

2017-09-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3825 ---

[jira] [Resolved] (KAFKA-4747) add metrics for KafkaConsumer.poll

2017-09-11 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4747?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Onur Karaman resolved KAFKA-4747. - Resolution: Won't Fix [~junrao] pointed out that the distinction between tim-in-poll and time-in-

[GitHub] kafka pull request #3830: MINOR: Created convenience method to create ZkUtil...

2017-09-11 Thread bbaugher
GitHub user bbaugher opened a pull request: https://github.com/apache/kafka/pull/3830 MINOR: Created convenience method to create ZkUtils You can merge this pull request into a Git repository by running: $ git pull https://github.com/bbaugher/kafka zkutils-method Alternativel

[GitHub] kafka pull request #2959: Created convenience method to create ZkUtils

2017-09-11 Thread bbaugher
Github user bbaugher closed the pull request at: https://github.com/apache/kafka/pull/2959 ---

Re: 1.0.0 KIPs Update

2017-09-11 Thread Guozhang Wang
Sure! Please feel free to update the wiki. Guozhang On Mon, Sep 11, 2017 at 9:28 AM, Rajini Sivaram wrote: > Hi Guozhang, > > Can KIP-188 be added to the list, please? The vote has passed and PR should > be ready soon. > > Thank you, > > Rajini > > On Thu, Sep 7, 2017 at 10:28 PM, Guozhang Wan

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-11 Thread Apurva Mehta
Thanks for the votes everyone. One of the proposals here was to raise a 'DuplicateSequenceException' to the user if the broker detected that one of the internal retries resulted in the duplicate, and the metadata for the original batch was no longer cached. However, when implementing this change,

[GitHub] kafka pull request #2518: KAFKA-4747: add metrics for KafkaConsumer.poll

2017-09-11 Thread onurkaraman
Github user onurkaraman closed the pull request at: https://github.com/apache/kafka/pull/2518 ---

[GitHub] kafka pull request #3803: KAFKA-5845; KafkaController should send LeaderAndI...

2017-09-11 Thread lindong28
Github user lindong28 closed the pull request at: https://github.com/apache/kafka/pull/3803 ---

Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-11 Thread Sumant Tambe
@Jun, Until we make idempotent producer the default (kip-185), this kip is sensitive to retries. I.e., we expire batches either delivery.timeout.ms passes or all retries are exhausted, whichever comes first. In cases where retries exhaust first due to linger.ms + retries * (request.timeout.ms + ret

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

2017-09-11 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

2017-09-11 Thread Randall Hauch
Thanks for reviewing. Responses inline below. On Mon, Sep 11, 2017 at 11:22 AM, Roger Hoover wrote: > Randall, > > Thank you for the KIP. This should improve visibility greatly. I had a > few questions/ideas for more metrics. > > >1. What's the relationship between the worker state and the

[GitHub] kafka pull request #3829: KAFKA-5655: materialized count, aggregate, reduce ...

2017-09-11 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3829 KAFKA-5655: materialized count, aggregate, reduce to KGroupedTable Add overloads of `count`, `aggregate`, `reduce` using `Materialized` to `KGroupedTable` deprecate other overloads You can merge t

Re: 1.0.0 KIPs Update

2017-09-11 Thread Rajini Sivaram
Hi Guozhang, Can KIP-188 be added to the list, please? The vote has passed and PR should be ready soon. Thank you, Rajini On Thu, Sep 7, 2017 at 10:28 PM, Guozhang Wang wrote: > Actually my bad, there is already a voting thread and you asked people to > recast a vote on a small change. > > On

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

2017-09-11 Thread Roger Hoover
Randall, Thank you for the KIP. This should improve visibility greatly. I had a few questions/ideas for more metrics. 1. What's the relationship between the worker state and the connector status? Does the 'paused' status at the Connector level include the time that worker is 'rebalan

Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-11 Thread Jun Rao
Hi, Sumant, Thanks for the KIP. +1. Just a minor clarification. The KIP says "Batches expire in order when max.in.flight.request.per.connection==1". Is that true? It seems that even with max.in.flight.request.per.connection > 1, batches should still expire in order. Jun On Sat, Sep 9, 2017 at 6

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

2017-09-11 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5763; Use LogContext in NetworkClient, Selector and broker -- [...truncated 2.03 MB...] org.apache.kafka.common.ClusterTest > testBootstrap STA

[GitHub] kafka pull request #3828: Minor update processor topology test driver

2017-09-11 Thread bbejeck
GitHub user bbejeck opened a pull request: https://github.com/apache/kafka/pull/3828 Minor update processor topology test driver You can merge this pull request into a Git repository by running: $ git pull https://github.com/bbejeck/kafka MINOR_update_processor_topology_test_

[jira] [Resolved] (KAFKA-5763) Refactor NetworkClient to use LogContext

2017-09-11 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5763. Resolution: Fixed > Refactor NetworkClient to use LogContext > -

[VOTE] KIP-199: Add Kafka Connect Offset Tool

2017-09-11 Thread Randall Hauch
I'd like to start the vote on KIP-199 to add a command line tool that will allow Connect operators to read, modify, and update source connector offsets. Details are here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-199%3A+Add+Kafka+Connect+offset+tool Thanks, and best regards. Randall

[VOTE] KIP-196: Add metrics to Kafka Connect framework

2017-09-11 Thread Randall Hauch
I'd like to start the vote on KIP-196 to add metrics to the Kafka Connect framework so the worker processes can be measured. Details are here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework Thanks, and best regards. Randall

[GitHub] kafka pull request #3761: KAFKA-5763: Refactor NetworkClient to use LogConte...

2017-09-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3761 ---

[GitHub] kafka pull request #3827: KAFKA-5654: add materialized count, reduce, aggreg...

2017-09-11 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3827 KAFKA-5654: add materialized count, reduce, aggregate to KGroupedStream Add overloads of `count`, `reduce`, and `aggregate` that are `Materialized` to `KGroupedStream`. Refactor common parts betwee

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

2017-09-11 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5816; [FOLLOW UP] create ProducedInternal class -- [...truncated 2.51 MB...] org.apache.kafka.streams.KafkaStreamsTest > testStateGlobalThr

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

2017-09-11 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5816; [FOLLOW UP] create ProducedInternal class -- [...truncated 2.69 MB...] org.apache.kafka.connect.runtime.rest.resources.ConnectorsReso

[GitHub] kafka pull request #3810: KAFKA-5816: [FOLLOW UP] - create ProducedInternal ...

2017-09-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3810 ---

[GitHub] kafka pull request #3826: KAFKA:5653: add join overloads to KTable

2017-09-11 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3826 KAFKA:5653: add join overloads to KTable Add `join`, `leftJoin`, `outerJoin` overloads that use `Materialized` to `KTable` You can merge this pull request into a Git repository by running: $ git

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

2017-09-11 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5531; throw concrete exceptions in streams tests -- [...truncated 2.51 MB...] org.apache.kafka.streams.KafkaStreamsTest > testStateGlobalTh

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

2017-09-11 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5531; throw concrete exceptions in streams tests -- [...truncated 2.03 MB...] org.apache.kafka.common.security.ssl.SslFactoryTest > testCli

[jira] [Resolved] (KAFKA-5855) records-lag is always zero

2017-09-11 Thread Mohsen Zainalpour (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5855?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mohsen Zainalpour resolved KAFKA-5855. -- Resolution: Invalid > records-lag is always zero > -- > >

[GitHub] kafka pull request #3825: KAFKA-5817: [FOLLOW-UP] add SerializedInternal

2017-09-11 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3825 KAFKA-5817: [FOLLOW-UP] add SerializedInternal Add `SerializedInternal` class and remove getters from `Serialized` You can merge this pull request into a Git repository by running: $ git pull http

[GitHub] kafka pull request #3485: KAFKA-5531: throw concrete exceptions in streams t...

2017-09-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3485 ---

[jira] [Created] (KAFKA-5868) Kafka Consumer Rebalancing takes too long

2017-09-11 Thread Nandish Kotadia (JIRA)
Nandish Kotadia created KAFKA-5868: -- Summary: Kafka Consumer Rebalancing takes too long Key: KAFKA-5868 URL: https://issues.apache.org/jira/browse/KAFKA-5868 Project: Kafka Issue Type: Bug