[jira] [Created] (KAFKA-6191) Consumers are staying blocked on rebalancing

2017-11-09 Thread Idriss Neumann (JIRA)
Idriss Neumann created KAFKA-6191: - Summary: Consumers are staying blocked on rebalancing Key: KAFKA-6191 URL: https://issues.apache.org/jira/browse/KAFKA-6191 Project: Kafka Issue Type: Bug

Re: [DISCUSS] Kafka 2.0.0 in June 2018

2017-11-09 Thread Tom Bentley
Hi Stephane, I think the version number rules are based on semantic versioning, so Kafka can't remove even deprecated APIs in a minor release (it is a breaking change, after all). Therefore until Kafka 2.0 we will have to carry the weight of the deprecated APIs, and Java 7. Cheers, Tom On 9

[DISCUSS]KIP-223 - Add per-topic min lead and per-partition lead metrics to KafkaConsumer

2017-11-09 Thread Hu Xi
Hi all, As per Jun Rao's suggestion, I opened up the KIP-223(https://cwiki.apache.org/confluence/display/KAFKA/KIP-223+-+Add+per-topic+min+lead+and+per-partition+lead+metrics+to+KafkaConsumer) concerning adding new kinds of lag metrics for KafkaConsumer. Be free to leave your comments here.

[DISCUSS] Kafka 2.0.0 in June 2018

2017-11-09 Thread Ismael Juma
Hi all, I'm starting this discussion early because of the potential impact. Kafka 1.0.0 was just released and the focus was on achieving the original project vision in terms of features provided while maintaining compatibility for the most part (i.e. we did not remove deprecated components like

Re: [DISCUSS] KIP-224: Add configuration parameters `retries` and `retry.backoff.ms` to Streams API

2017-11-09 Thread Damian Guy
Thanks Matthias, LGTM On Thu, 9 Nov 2017 at 11:13 Matthias J. Sax wrote: > Hi, > > I want to propose a new KIP to make Streams API more resilient to broker > disconnections. > > >

[DISCUSS] KIP-224: Add configuration parameters `retries` and `retry.backoff.ms` to Streams API

2017-11-09 Thread Matthias J. Sax
Hi, I want to propose a new KIP to make Streams API more resilient to broker disconnections. https://cwiki.apache.org/confluence/display/KAFKA/KIP-224%3A+Add+configuration+parameters+%60retries%60+and+%60retry.backoff.ms%60+to+Streams+API -Matthias signature.asc Description: OpenPGP digital

Re: [DISCUSS] Kafka 2.0.0 in June 2018

2017-11-09 Thread Stephane Maarek
I'm very happy with the milestones but worried about the versioning number. It seems it will mostly bring stuff out of deprecation vs actually bringing in breaking features. A 2.0 to me should bring something major to the table, possibly breaking, which would justify a big number hop. I'm still

Re: [DISCUSS] Kafka 2.0.0 in June 2018

2017-11-09 Thread Ismael Juma
That's correct, Tom. We can only remove deprecated APIs in major releases since it's a breaking change. Ismael On 9 Nov 2017 11:48 am, "Tom Bentley" wrote: Hi Stephane, I think the version number rules are based on semantic versioning, so Kafka can't remove even

Build failed in Jenkins: kafka-1.0-jdk7 #75

2017-11-09 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Update Scala 2.11 to 2.11.12 -- [...truncated 25.11 KB...] (topicPartition, new ListOffsetResponse.PartitionData(Errors.NONE,

[GitHub] kafka pull request #4185: MINOR: Update Scala 2.11 to 2.11.12

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

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

2017-11-09 Thread Apache Jenkins Server
See

[DISCUSS] KIP-225 - Use tags for consumer “records.lag” metrics

2017-11-09 Thread charly molter
Hi, I'd like to start the discussion on KIP-225. This KIP tries to correct the way the consumer lag metrics are reported to use built in tags from MetricName. Here's the link: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=74686649 Thanks! -- Charly Molter

Re: [VOTE] KIP-215: Add topic regex support for Connect sinks

2017-11-09 Thread Randall Hauch
Jeff, This KIP does pass with 3 binding +1s and no other binding votes, and with more than 72 hour for voting. Do you want to update the KIP-215 and KIP list pages accordingly? We can also merge the PR. Thanks, and very nice work! Randall On Fri, Nov 3, 2017 at 8:12 PM, Stephane Maarek <

Kafka 0.9.0.1 partitions shrink and expand frequently after restart the broker

2017-11-09 Thread Json Tu
Hi, we have a kafka cluster which is made of 6 brokers, with 8 cpu and 16G memory on each broker’s machine, and we have about 1600 topics in the cluster,about 1700 partitions’ leader and 1600 partitions' replica on each broker. when we restart a normal broke, we find that there are

Re: Kafka 0.9.0.1 partitions shrink and expand frequently after restart the broker

2017-11-09 Thread Viktor Somogyi
I'm happy that it's solved :) On Thu, Nov 9, 2017 at 3:32 PM, John Yost wrote: > Excellent points Viktor! Also, the reason I mistakenly went > 8 GB memory > heap was due to OOM errors that were being thrown when I upgraded from > 0.9.0.1 to 0.10.0.0 and forgot to

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

2017-11-09 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Update Scala 2.11 to 2.11.12 -- [...truncated 385.80 KB...] kafka.security.auth.ResourceTypeTest > testJavaConversions STARTED

[GitHub] kafka pull request #4181: KAFKA-6164: Shutdown quota managers if other compo...

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

Re: [DISCUSS] KIP-224: Add configuration parameters `retries` and `retry.backoff.ms` to Streams API

2017-11-09 Thread Ted Yu
lgtm bq. pass both parameter parameter should be in plural. Same with 'two new configuration parameter' Cheers On Thu, Nov 9, 2017 at 4:20 AM, Damian Guy wrote: > Thanks Matthias, LGTM > > On Thu, 9 Nov 2017 at 11:13 Matthias J. Sax wrote: > > >

Re: Kafka 0.9.0.1 partitions shrink and expand frequently after restart the broker

2017-11-09 Thread Viktor Somogyi
Hi Json. John might have a point. It is not reasonable to have more than 6-8GB of heap provided for the JVM that's running Kafka. One of the reason is GC time and the other is that Kafka relies heavily on the OS' disk read/write in-memory caching. Also there were a few synchronization bugs in 0.9

[jira] [Created] (KAFKA-6192) In Config always transform Properties to Map

2017-11-09 Thread Charly Molter (JIRA)
Charly Molter created KAFKA-6192: Summary: In Config always transform Properties to Map Key: KAFKA-6192 URL: https://issues.apache.org/jira/browse/KAFKA-6192 Project: Kafka Issue Type:

Re: Kafka 0.9.0.1 partitions shrink and expand frequently after restart the broker

2017-11-09 Thread John Yost
Yep, the team here, including Ismael, pointed me in the right direction, which was much appreciated. :) On Thu, Nov 9, 2017 at 10:02 AM, Viktor Somogyi wrote: > I'm happy that it's solved :) > > On Thu, Nov 9, 2017 at 3:32 PM, John Yost wrote: > >

Re: Kafka 0.9.0.1 partitions shrink and expand frequently after restart the broker

2017-11-09 Thread John Yost
Excellent points Viktor! Also, the reason I mistakenly went > 8 GB memory heap was due to OOM errors that were being thrown when I upgraded from 0.9.0.1 to 0.10.0.0 and forgot to explicitly set the message format to 0.9.0.1 because we needed to support the older clients and the corresponding

[GitHub] kafka pull request #4193: KAFKA-6185: Remove channels from explictlyMutedCha...

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

Re: Kafka 0.9.0.1 partitions shrink and expand frequently after restart the broker

2017-11-09 Thread John Yost
I've seen this before and it was due to long GC pauses due in large part to a memory heap > 8 GB. --John On Thu, Nov 9, 2017 at 8:17 AM, Json Tu wrote: > Hi, > we have a kafka cluster which is made of 6 brokers, with 8 cpu and > 16G memory on each broker’s machine, and

[GitHub] kafka pull request #4184: MINOR: Remove FanoutIntegrationTest.java

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

[GitHub] kafka pull request #4200: MINOR: Update to Scala 2.11.12 (part 2)

2017-11-09 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/4200 MINOR: Update to Scala 2.11.12 (part 2) Missed a few places in the previous commit. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka

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

2017-11-09 Thread Apache Jenkins Server
See

Jenkins build is back to normal : kafka-1.0-jdk7 #76

2017-11-09 Thread Apache Jenkins Server
See

Regarding kafka produce Image(18mb)

2017-11-09 Thread SADHNA
Hi Team, I am trying to send images using kafka. I have used value_serializer=lambda v: json.dumps(v).encode('utf-8') and sending the messages like message = {"cam_id" : 1, "frame" : image, "frame_no": i} but getting TypeError: array([[[ 26, 36, 31], [ 16, 26, 21], [ 11,

Re: Regarding kafka produce Image(18mb)

2017-11-09 Thread Ted Yu
bq. Each image if of 18mb in size. Seems the image size is non-trivial. Can you tell us more about use case ? e.g. what systems (Apache projects) are involved. Just want to see if there is another route for sending the images. On Thu, Nov 9, 2017 at 2:15 AM, SADHNA

Build failed in Jenkins: kafka-1.0-jdk7 #77

2017-11-09 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-6185; Remove channels from explictlyMutedChannels set when closed -- [...truncated 371.72 KB...] kafka.server.ServerGenerateBrokerIdTest >

[jira] [Created] (KAFKA-6194) Server crash

2017-11-09 Thread Ben Corlett (JIRA)
Ben Corlett created KAFKA-6194: -- Summary: Server crash Key: KAFKA-6194 URL: https://issues.apache.org/jira/browse/KAFKA-6194 Project: Kafka Issue Type: Bug Components: core

Re: [DISCUSS] KIP-224: Add configuration parameters `retries` and `retry.backoff.ms` to Streams API

2017-11-09 Thread Guozhang Wang
Instead of restoring the global store during registration, could we also do this after the rebalance callback as in the main loop? By doing this we can effectively swallow-and-retry-in-next-loop as we did for non-global stores. Since global stores are per task not per thread, we would not process

[GitHub] kafka pull request #4072: MINOR: Add HttpMetricsReporter for system tests

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

[GitHub] kafka pull request #4199: KAFKA-6164; Shutdown quota managers if other compo...

2017-11-09 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/4199 KAFKA-6164; Shutdown quota managers if other components fail to start You can merge this pull request into a Git repository by running: $ git pull https://github.com/rajinisivaram/kafka

Re: [DISCUSS]KIP-223 - Add per-topic min lead and per-partition lead metrics to KafkaConsumer

2017-11-09 Thread Ted Yu
Can you fill out JIRA number ? "TOPIC-PARTITION_ID.record-lead" In the code sample, "records-" is used. Please make them consistent. On Thu, Nov 9, 2017 at 1:03 AM, Hu Xi wrote: > Hi all, > > > As per Jun Rao's suggestion, I opened up the KIP-223(https://cwiki.apache. >

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

2017-11-09 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-6164; Shutdown quota managers if other components fail to start -- [...truncated 1.95 MB...] org.apache.kafka.streams.KafkaStreamsTest >

[jira] [Created] (KAFKA-6193) ReassignPartitionsClusterTest.shouldPerformMultipleReassignmentOperationsOverVariousTopics fails sometimes

2017-11-09 Thread Ted Yu (JIRA)
Ted Yu created KAFKA-6193: - Summary: ReassignPartitionsClusterTest.shouldPerformMultipleReassignmentOperationsOverVariousTopics fails sometimes Key: KAFKA-6193 URL: https://issues.apache.org/jira/browse/KAFKA-6193

[jira] [Resolved] (KAFKA-6185) Selector memory leak with high likelihood of OOM in case of down conversion

2017-11-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6185?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-6185. Resolution: Fixed > Selector memory leak with high likelihood of OOM in case of down conversion >

Re: [DISCUSS] KIP-224: Add configuration parameters `retries` and `retry.backoff.ms` to Streams API

2017-11-09 Thread Kamal
Matthias, Could you describe how the current 'fail-fast' strategy (default: 1 retry) is useful since it's kills the stream instance ? Does the plan is to keep infinite retry or users need to configure it ? On Fri, Nov 10, 2017 at 12:57 AM, Damian Guy wrote: >

Re: [DISCUSS] KIP-212: Enforce set of legal characters for connector names

2017-11-09 Thread Randall Hauch
Any progress on updating the PR and withdrawing KIP-212? On Fri, Oct 27, 2017 at 5:19 PM, Randall Hauch wrote: > Yes, connector names should not be blank or contain just whitespace. In > fact, I might recommend that we trim whitespace at the front and rear of > new connector

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

2017-11-09 Thread Apache Jenkins Server
See

Build failed in Jenkins: kafka-trunk-jdk9 #183

2017-11-09 Thread Apache Jenkins Server
See Changes: [me] MINOR: Add HttpMetricsReporter for system tests -- [...truncated 1.90 MB...] org.apache.kafka.connect.runtime.distributed.DistributedHerderTest >

Re: [DISCUSS] KIP-224: Add configuration parameters `retries` and `retry.backoff.ms` to Streams API

2017-11-09 Thread Damian Guy
Guozhang, i'm not sure i follow... Global stores aren't per task, they are per application instance and should be fully restored before the stream threads start processing. They don't go through a rebalance as it is manual assignment of all partitions in the topic. On Thu, 9 Nov 2017 at 17:43

Jenkins build is back to normal : kafka-1.0-jdk7 #78

2017-11-09 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #4200: MINOR: Update to Scala 2.11.12 (part 2)

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

[jira] [Created] (KAFKA-6195) DNS alias support for secured connections

2017-11-09 Thread Jonathan Skrzypek (JIRA)
Jonathan Skrzypek created KAFKA-6195: Summary: DNS alias support for secured connections Key: KAFKA-6195 URL: https://issues.apache.org/jira/browse/KAFKA-6195 Project: Kafka Issue Type:

[jira] [Resolved] (KAFKA-6164) ClientQuotaManager threads prevent shutdown when encountering an error loading logs

2017-11-09 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6164?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-6164. --- Resolution: Fixed Issue resolved by pull request 4199

Jenkins build is back to normal : kafka-trunk-jdk9 #184

2017-11-09 Thread Apache Jenkins Server
See

[jira] [Reopened] (KAFKA-5669) Define allowed characters for connector names

2017-11-09 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5669?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch reopened KAFKA-5669: -- > Define allowed characters for connector names > - > >

[jira] [Resolved] (KAFKA-5669) Define allowed characters for connector names

2017-11-09 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5669?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-5669. -- Resolution: Duplicate > Define allowed characters for connector names >

[jira] [Resolved] (KAFKA-5669) Define allowed characters for connector names

2017-11-09 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5669?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-5669. -- Resolution: Won't Fix I'm closing this issue, since KAFKA-4930 is actually closer to the heart

[jira] [Resolved] (KAFKA-4938) Creating a connector with missing name parameter throws a NullPointerException

2017-11-09 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4938?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-4938. -- Resolution: Duplicate I'm closing this issue, since KAFKA-4930 is actually closer to the heart

[jira] [Resolved] (KAFKA-4827) Kafka connect: error with special characters in connector name

2017-11-09 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4827?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-4827. -- Resolution: Fixed I'm closing this issue, since KAFKA-4930 is actually closer to the heart of

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

2017-11-09 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Update to Scala 2.11.12 (part 2) -- [...truncated 383.01 KB...] kafka.security.auth.ResourceTypeTest > testFromString STARTED

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

2017-11-09 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Update to Scala 2.11.12 (part 2) -- [...truncated 381.79 KB...] kafka.server.ServerGenerateBrokerIdTest > testUserConfigAndGeneratedBrokerId

[jira] [Reopened] (KAFKA-4827) Kafka connect: error with special characters in connector name

2017-11-09 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4827?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch reopened KAFKA-4827: -- > Kafka connect: error with special characters in connector name >

[GitHub] kafka pull request #4198: MINOR: make controller helper methods private

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

[jira] [Resolved] (KAFKA-4827) Kafka connect: error with special characters in connector name

2017-11-09 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4827?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch resolved KAFKA-4827. -- Resolution: Won't Fix To be clear, connector names should be *URL-encoded* in the REST request

[jira] [Reopened] (KAFKA-4827) Kafka connect: error with special characters in connector name

2017-11-09 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4827?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch reopened KAFKA-4827: -- > Kafka connect: error with special characters in connector name >

Re: [DISCUSS] KIP-224: Add configuration parameters `retries` and `retry.backoff.ms` to Streams API

2017-11-09 Thread Guozhang Wang
Damian, You are right! I was dreaming at the wrong class :) Guozhang On Thu, Nov 9, 2017 at 11:27 AM, Damian Guy wrote: > Guozhang, i'm not sure i follow... Global stores aren't per task, they are > per application instance and should be fully restored before the stream

Re: [VOTE] KIP-159: Introducing Rich functions to Streams

2017-11-09 Thread Guozhang Wang
Hello Jan, Regarding your question about caching: today we keep the record context with the cached entry already so when we flush the cache which may generate new records forwarding we will set the record context appropriately; and then after the flush is completed we will reset the context to

答复: [DISCUSS]KIP-223 - Add per-topic min lead and per-partition lead metrics to KafkaConsumer

2017-11-09 Thread Hu Xi
Yu, thanks for your comments. Already explicitly showed the jira number and refined the exact metric names. 发件人: Ted Yu 发送时间: 2017年11月10日 1:51 收件人: dev@kafka.apache.org 主题: Re: [DISCUSS]KIP-223 - Add per-topic min lead and per-partition

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

2017-11-09 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #4197: KAFKA-6190 GlobalKTable never finishes restoring w...

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

Build failed in Jenkins: kafka-trunk-jdk9 #186

2017-11-09 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6190: Use consumer.position() instead of record.offset() to -- [...truncated 1.42 MB...] kafka.javaapi.message.ByteBufferMessageSetTest >

Re: [VOTE] KIP-159: Introducing Rich functions to Streams

2017-11-09 Thread Guozhang Wang
With our current state store implementation, when we are doing the two-way join operators like Stream-Stream or Table-Table, where each stream's record may trigger the join, it is hard to retrieve the record context for the matched record from the other stream's materialized state since we do not

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

2017-11-09 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-6135) TransactionsTest#testFencingOnCommit may fail due to unexpected KafkaException

2017-11-09 Thread Ted Yu (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6135?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Yu resolved KAFKA-6135. --- Resolution: Cannot Reproduce > TransactionsTest#testFencingOnCommit may fail due to unexpected KafkaException

Re: [DISCUSS] Kafka 2.0.0 in June 2018

2017-11-09 Thread Apurva Mehta
I think this is a good idea and your proposed changes look good. I also think that this might be a good time to adopt KIP-185 ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-185%3A+Make+exactly+once+in+order+delivery+per+partition+the+default+producer+setting), and make the idempotent

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

2017-11-09 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6190: Use consumer.position() instead of record.offset() to -- [...truncated 1.40 MB...]

Re: [DISCUSS] Kafka 2.0.0 in June 2018

2017-11-09 Thread Ismael Juma
Hi Apurva, I agree about KIP-185 (assuming the vote passes). To clarify, my list was not meant to be exhaustive, just the items with highest compatibility impact justifying the major bump. I expect we will have many other great KIPs. :) Ismael On 10 Nov 2017 12:57 am, "Apurva Mehta"

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

2017-11-09 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6190: Use consumer.position() instead of record.offset() to -- [...truncated 1.44 MB...] K extends Object declared in interface

[jira] [Resolved] (KAFKA-6155) Found Dead Lock Error on broker

2017-11-09 Thread Chengxiang Li (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6155?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chengxiang Li resolved KAFKA-6155. -- Resolution: Duplicate > Found Dead Lock Error on broker > --- > >

Re: Kafka 0.9.0.1 partitions shrink and expand frequently after restart the broker

2017-11-09 Thread Json Tu
I‘m so sorry for my poor english. what I really means is my broker machine is configured as 8 core 16G. but my jvm configure is as below. java -Xmx1G -Xms1G -server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+DisableExplicitGC -Djava.awt.headless=true

[jira] [Created] (KAFKA-6197) Difficult to get to the Kafka Streams javadocs

2017-11-09 Thread James Cheng (JIRA)
James Cheng created KAFKA-6197: -- Summary: Difficult to get to the Kafka Streams javadocs Key: KAFKA-6197 URL: https://issues.apache.org/jira/browse/KAFKA-6197 Project: Kafka Issue Type:

[GitHub] kafka pull request #4179: KAFKA-6175; AbstractIndex should cache index file ...

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

Re: Kafka 0.9.0.1 partitions shrink and expand frequently after restart the broker

2017-11-09 Thread Json Tu
The broker with broker id 4759750 is just restart,and there are 500+ replica partitions shrink and expand frequently,and there leader partition is distributed in the other 5 brokers. the log is pulled from one broker,and extract logs related to 1 partition. > 在 2017年11月10日,下午12:06,Hu Xi

Re: [DISCUSS] KIP-224: Add configuration parameters `retries` and `retry.backoff.ms` to Streams API

2017-11-09 Thread Bill Bejeck
Thanks for the KIP Matthias, +1 from me. -Bill On Thu, Nov 9, 2017 at 8:40 AM, Ted Yu wrote: > lgtm > > bq. pass both parameter > > parameter should be in plural. > Same with 'two new configuration parameter' > > Cheers > > On Thu, Nov 9, 2017 at 4:20 AM, Damian Guy

Re: [VOTE] KIP-159: Introducing Rich functions to Streams

2017-11-09 Thread Jan Filipiak
Okay, looks like it would _at least work_ for Cached KTableSources . But we make it harder to the user to make mistakes by putting features into places where they don't make sense and don't help anyone. I once again think that my suggestion is easier to implement and more correct. I will use

[jira] [Created] (KAFKA-6196) Kafka Transactional producer with broker on windows

2017-11-09 Thread Abhishek Verma (JIRA)
Abhishek Verma created KAFKA-6196: - Summary: Kafka Transactional producer with broker on windows Key: KAFKA-6196 URL: https://issues.apache.org/jira/browse/KAFKA-6196 Project: Kafka Issue

Re: [DISCUSS] KIP-221: Repartition Topic Hints in Streams

2017-11-09 Thread Guozhang Wang
A few clarification questions on the proposal details. 1. API: although the repartition only happens at the final stateful operations like agg / join, the repartition flag info was actually passed from an earlier operator like map / groupBy. So what should be the new API look like? For example,

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

2017-11-09 Thread Apache Jenkins Server
See Changes: [becket.qin] KAFKA-6175; AbstractIndex should cache index file to avoid unnecessary -- [...truncated 385.32 KB...] kafka.security.auth.ResourceTypeTest >

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

2017-11-09 Thread Apache Jenkins Server
See Changes: [becket.qin] KAFKA-6175; AbstractIndex should cache index file to avoid unnecessary -- [...truncated 384.55 KB...] kafka.log.ProducerStateManagerTest >

Build failed in Jenkins: kafka-trunk-jdk9 #187

2017-11-09 Thread Apache Jenkins Server
See Changes: [becket.qin] KAFKA-6175; AbstractIndex should cache index file to avoid unnecessary -- [...truncated 1.42 MB...] kafka.message.MessageTest >