Re: [DISCUSS] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-03-03 Thread Dong Lin
Hey Becket, I haven't looked at the patch yet. But since we are going to try the split-on-oversize solution, should the KIP also add a sensor that shows the rate of split per second and the probability of split? Thanks, Dong On Fri, Mar 3, 2017 at 6:39 PM, Becket Qin

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

2017-03-03 Thread Apache Jenkins Server
See

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

2017-03-03 Thread Apache Jenkins Server
See Changes: [me] MINOR: Fix tests/docker/Dockerfile [ismael] KAFKA-2857; MINOR: Follow up to MINOR: Fix ResetIntegrationTest test [ismael] KAFKA-4796; Fix some findbugs warnings in Kafka Java client

Re: [DISCUSS] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-03-03 Thread Becket Qin
Just to clarify, the implementation is basically what I mentioned above (split/resend + adjusted estimation evolving algorithm) and changing the compression ratio estimation to be per topic. Thanks, Jiangjie (Becket) Qin On Fri, Mar 3, 2017 at 6:36 PM, Becket Qin wrote:

Re: [DISCUSS] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-03-03 Thread Becket Qin
I went ahead and have a patch submitted here: https://github.com/apache/kafka/pull/2638 Per Joel's suggestion, I changed the compression ratio to be per topic as well. It seems working well. Since there is an important behavior change and a new sensor is added, I'll keep the KIP and update it

[jira] [Commented] (KAFKA-3995) Add a new configuration "enable.compression.ratio.estimation" to the producer config

2017-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3995?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15895443#comment-15895443 ] ASF GitHub Bot commented on KAFKA-3995: --- GitHub user becketqin opened a pull request:

[GitHub] kafka pull request #2638: KAFKA-3995: fix compression ratio estimation.

2017-03-03 Thread becketqin
GitHub user becketqin opened a pull request: https://github.com/apache/kafka/pull/2638 KAFKA-3995: fix compression ratio estimation. You can merge this pull request into a Git repository by running: $ git pull https://github.com/becketqin/kafka KAFKA-3995 Alternatively you

[jira] [Created] (KAFKA-4840) There are are still cases where producer buffer pool will not remove waiters.

2017-03-03 Thread Sean McCauliff (JIRA)
Sean McCauliff created KAFKA-4840: - Summary: There are are still cases where producer buffer pool will not remove waiters. Key: KAFKA-4840 URL: https://issues.apache.org/jira/browse/KAFKA-4840

[GitHub] kafka pull request #2637: throw NoOffsetForPartitionException from poll once...

2017-03-03 Thread radai-rosenblatt
GitHub user radai-rosenblatt opened a pull request: https://github.com/apache/kafka/pull/2637 throw NoOffsetForPartitionException from poll once for all TopicPartitions affected Signed-off-by: radai-rosenblatt You can merge this pull request

[jira] [Created] (KAFKA-4839) throw NoOffsetForPartitionException once for all assigned partitions from poll

2017-03-03 Thread radai rosenblatt (JIRA)
radai rosenblatt created KAFKA-4839: --- Summary: throw NoOffsetForPartitionException once for all assigned partitions from poll Key: KAFKA-4839 URL: https://issues.apache.org/jira/browse/KAFKA-4839

[jira] [Updated] (KAFKA-4796) Fix some findbugs warnings in Kafka Java client

2017-03-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4796?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4796: --- Fix Version/s: 0.11.0.0 > Fix some findbugs warnings in Kafka Java client >

[jira] [Resolved] (KAFKA-4796) Fix some findbugs warnings in Kafka Java client

2017-03-03 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4796?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4796. Resolution: Fixed > Fix some findbugs warnings in Kafka Java client >

[jira] [Commented] (KAFKA-4796) Fix some findbugs warnings in Kafka Java client

2017-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4796?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15895339#comment-15895339 ] ASF GitHub Bot commented on KAFKA-4796: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #2593: KAFKA-4796: Fix some findbugs warnings in Kafka Ja...

2017-03-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2593 --- 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

[GitHub] kafka pull request #2636: MINOR: Follow up to KAFKA-2857

2017-03-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2636 --- 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

[jira] [Commented] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2017-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15895333#comment-15895333 ] ASF GitHub Bot commented on KAFKA-2857: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #2613: MINOR. Fix tests/docker/Dockerfile

2017-03-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2613 --- 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

[GitHub] kafka pull request #2628: MINOR: Bump version to 0.11.0.0-SNAPSHOT

2017-03-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2628 --- 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

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

2017-03-03 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Bump version to 0.11.0.0-SNAPSHOT -- [...truncated 264.82 KB...] kafka.api.RackAwareAutoTopicCreationTest > testAutoCreateTopic PASSED

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

2017-03-03 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Bump version to 0.11.0.0-SNAPSHOT -- [...truncated 847.12 KB...] org.apache.kafka.streams.kstream.internals.KStreamKStreamJoinTest > testJoin

[jira] [Commented] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2017-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15895265#comment-15895265 ] ASF GitHub Bot commented on KAFKA-2857: --- GitHub user vahidhashemian opened a pull request:

[GitHub] kafka pull request #2636: MINOR: Follow up to KAFKA-2857

2017-03-03 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2636 MINOR: Follow up to KAFKA-2857 You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian/kafka minor/kafka-2857-followup

[jira] [Updated] (KAFKA-4795) Confusion around topic deletion

2017-03-03 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4795?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-4795: --- Description: The topic deletion works like this in 0.10.2.0: #

[jira] [Commented] (KAFKA-4795) Confusion around topic deletion

2017-03-03 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4795?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15895220#comment-15895220 ] Vahid Hashemian commented on KAFKA-4795: [~omkreddy] Thanks for your comment. I had looked at the

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

2017-03-03 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-2857; Retry querying the consumer group while initializing -- [...truncated 321.30 KB...] kafka.security.auth.SimpleAclAuthorizerTest >

[jira] [Updated] (KAFKA-4790) Kafka cannot recover after a disk full

2017-03-03 Thread Jeff Widman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4790?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Widman updated KAFKA-4790: --- Fix Version/s: (was: 0.10.2.1) > Kafka cannot recover after a disk full >

[GitHub] kafka pull request #2635: MINOR: additional refactoring around the use of Er...

2017-03-03 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2635 MINOR: additional refactoring around the use of Errors A couple of updates were missed in the [PR](https://github.com/apache/kafka/pull/2475) that replaced the use of error codes with

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

2017-03-03 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-2857; Retry querying the consumer group while initializing -- [...truncated 847.00 KB...]

[jira] [Commented] (KAFKA-4787) KafkaStreams close() is not reentrant

2017-03-03 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4787?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15895074#comment-15895074 ] Colin P. McCabe commented on KAFKA-4787: Hmm. It seems bad if the code hangs when close() is

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-03 Thread Colin McCabe
On Fri, Mar 3, 2017, at 06:41, Ismael Juma wrote: > Hi Colin, > > I still need to do a detailed review, but I have a couple of > comments/questions: > > 1. I am not sure about having the options/response classes as inner > classes > of the interface. It means that file containing the interface

[jira] [Commented] (KAFKA-4830) Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-03-03 Thread Marc Juchli (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4830?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15895048#comment-15895048 ] Marc Juchli commented on KAFKA-4830: [~mjsax] that works for me, will catch up on Monday with that.

[GitHub] kafka pull request #2634: MINOR: Standardised benchmark params for consumer ...

2017-03-03 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/2634 MINOR: Standardised benchmark params for consumer and streams There were some minor differences in the basic consumer config and streams config that are now rectified. In addition, in AWS

Re: [VOTE] KIP-119: Drop Support for Scala 2.10 in Kafka 0.11

2017-03-03 Thread Mickael Maison
+1 non-binding On Fri, Mar 3, 2017 at 7:26 PM, Dong Lin wrote: > +1 (non-binding) > > On Thu, Mar 2, 2017 at 11:18 AM, Becket Qin wrote: > >> Thanks for the clarification, Ismael. In that case, it is reasonable to >> drop support for Scala 2.10.

Re: [VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-03-03 Thread Dong Lin
re: Purge vs PurgeMessages/Records - I think we agree that PurgeRequest is more consistent with current requests while PurgeRecords is more explicit by itself. The question is whether it is more important to be consistent or explicit. My opinion is that consistency is more important here. In

[jira] [Commented] (KAFKA-4838) Export buffer cache metrics in JMX

2017-03-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4838?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894969#comment-15894969 ] Eno Thereska commented on KAFKA-4838: - Note that we do have one cache sensor, the hitRatioSensor,

[jira] [Comment Edited] (KAFKA-4838) Export buffer cache metrics in JMX

2017-03-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4838?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894969#comment-15894969 ] Eno Thereska edited comment on KAFKA-4838 at 3/3/17 8:17 PM: - Note that we do

[jira] [Created] (KAFKA-4838) Export buffer cache metrics in JMX

2017-03-03 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-4838: --- Summary: Export buffer cache metrics in JMX Key: KAFKA-4838 URL: https://issues.apache.org/jira/browse/KAFKA-4838 Project: Kafka Issue Type: Improvement

[jira] [Commented] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2017-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894950#comment-15894950 ] ASF GitHub Bot commented on KAFKA-2857: --- Github user asfgit closed the pull request at:

[GitHub] kafka pull request #2538: KAFKA-2857: Retry querying the consumer group whil...

2017-03-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2538 --- 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

[jira] [Updated] (KAFKA-2857) ConsumerGroupCommand throws GroupCoordinatorNotAvailableException when describing a non-existent group before the offset topic is created

2017-03-03 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2857?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2857: --- Resolution: Fixed Fix Version/s: 0.11.0.0 Status: Resolved (was: Patch

[jira] [Commented] (KAFKA-4639) Kafka Streams metrics are undocumented

2017-03-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4639?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894924#comment-15894924 ] Eno Thereska commented on KAFKA-4639: - This is now documented in 0.10.2 :

[jira] [Resolved] (KAFKA-4639) Kafka Streams metrics are undocumented

2017-03-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4639?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska resolved KAFKA-4639. - Resolution: Fixed > Kafka Streams metrics are undocumented >

[jira] [Resolved] (KAFKA-4755) SimpleBenchmark consume test fails for streams

2017-03-03 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4755?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska resolved KAFKA-4755. - Resolution: Cannot Reproduce > SimpleBenchmark consume test fails for streams >

Re: [VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-03-03 Thread Jeff Widman
re: Purge vs PurgeMessages/Records - I also prefer that it be more explicit about what is being purged. Despite the inconsistency with Fetch/Produce, because it's explicit about what's being purged there shouldn't be additional confusion. Who knows what in the future might need purging? Adding the

Re: [VOTE] KIP-119: Drop Support for Scala 2.10 in Kafka 0.11

2017-03-03 Thread Dong Lin
+1 (non-binding) On Thu, Mar 2, 2017 at 11:18 AM, Becket Qin wrote: > Thanks for the clarification, Ismael. In that case, it is reasonable to > drop support for Scala 2.10. LinkedIn is probably fine with this change. > > I did not notice we have recommended Scala version

Re: [VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-03-03 Thread Dong Lin
Hey Ismael, Thank for the detailed explanation. Here is my thought: 1. purge vs. delete We have originally considered purge, delete, truncate and remove. I don't have a strong preference among them and would be OK with any choice here. That is why I didn't provide specific reasoning for

[jira] [Commented] (KAFKA-4830) Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-03-03 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4830?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894842#comment-15894842 ] Guozhang Wang commented on KAFKA-4830: -- [~mjuchli] As for this JIRA itself, the goal is to allow

[jira] [Commented] (KAFKA-4767) KafkaProducer is not joining its IO thread properly

2017-03-03 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4767?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894818#comment-15894818 ] Colin P. McCabe commented on KAFKA-4767: I agree that the interrupt state of the thread should be

[jira] [Updated] (KAFKA-4835) Allow users control over repartitioning

2017-03-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4835?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4835: --- Labels: needs-kip (was: ) > Allow users control over repartitioning >

[jira] [Commented] (KAFKA-4835) Allow users control over repartitioning

2017-03-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894817#comment-15894817 ] Matthias J. Sax commented on KAFKA-4835: I guess, we need API change for this, and thus, a KIP is

[jira] [Commented] (KAFKA-4830) Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-03-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4830?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894780#comment-15894780 ] Matthias J. Sax commented on KAFKA-4830: Thank for tackling this! Two comments: (1) this JIRA is

[jira] [Updated] (KAFKA-4830) Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-03-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4830?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4830: --- Description: Today {{KStream.print}} use the hard-coded result string as: {code} "[" +

[jira] [Updated] (KAFKA-4830) Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-03-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4830?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-4830: --- Labels: needs-kip newbie (was: newbie) > Augment KStream.print() to allow users pass in

[jira] [Commented] (KAFKA-4601) Avoid duplicated repartitioning in KStream DSL

2017-03-03 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4601?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894770#comment-15894770 ] Matthias J. Sax commented on KAFKA-4601: Thanks! > Avoid duplicated repartitioning in KStream DSL

[jira] [Updated] (KAFKA-4790) Kafka cannot recover after a disk full

2017-03-03 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4790?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin P. McCabe updated KAFKA-4790: --- Summary: Kafka cannot recover after a disk full (was: Kafka can't not recover after a disk

[jira] [Created] (KAFKA-4837) Config validation in Connector plugins need to compare against both canonical and simple class names

2017-03-03 Thread Konstantine Karantasis (JIRA)
Konstantine Karantasis created KAFKA-4837: - Summary: Config validation in Connector plugins need to compare against both canonical and simple class names Key: KAFKA-4837 URL:

[jira] [Commented] (KAFKA-4802) Support direct ByteBuffer serializers/deserializers in clients

2017-03-03 Thread Matt Sicker (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4802?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894657#comment-15894657 ] Matt Sicker commented on KAFKA-4802: We also found a related issue in the inability to reuse

[jira] [Commented] (KAFKA-4821) 9244L

2017-03-03 Thread Vamsi Jakkula (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894622#comment-15894622 ] Vamsi Jakkula commented on KAFKA-4821: -- this is a test comment > 9244L > - > >

[jira] [Updated] (KAFKA-3832) Kafka Connect's JSON Converter never outputs a null value

2017-03-03 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3832?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch updated KAFKA-3832: - Description: Kafka Connect's JSON Converter will never output a null value when

[jira] [Comment Edited] (KAFKA-4830) Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-03-03 Thread Marc Juchli (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4830?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894483#comment-15894483 ] Marc Juchli edited comment on KAFKA-4830 at 3/3/17 3:18 PM: I'd like to tackle

[jira] [Created] (KAFKA-4836) Kafka Streams app reset tool: support wildcards for topics

2017-03-03 Thread Michael Noll (JIRA)
Michael Noll created KAFKA-4836: --- Summary: Kafka Streams app reset tool: support wildcards for topics Key: KAFKA-4836 URL: https://issues.apache.org/jira/browse/KAFKA-4836 Project: Kafka Issue

[jira] [Commented] (KAFKA-4601) Avoid duplicated repartitioning in KStream DSL

2017-03-03 Thread Michal Borowiecki (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4601?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894498#comment-15894498 ] Michal Borowiecki commented on KAFKA-4601: -- Created KAFKA-4835. > Avoid duplicated

[jira] [Created] (KAFKA-4835) Allow users control over repartitioning

2017-03-03 Thread Michal Borowiecki (JIRA)
Michal Borowiecki created KAFKA-4835: Summary: Allow users control over repartitioning Key: KAFKA-4835 URL: https://issues.apache.org/jira/browse/KAFKA-4835 Project: Kafka Issue Type:

[GitHub] kafka pull request #2633: MINOR: Add simplified aggregate methods to KGroupe...

2017-03-03 Thread KyleWinkelman
GitHub user KyleWinkelman opened a pull request: https://github.com/apache/kafka/pull/2633 MINOR: Add simplified aggregate methods to KGroupedStream Currently the KGroupedStream aggregate methods requires users to supply (Serde) null if they intend to use the default value serde.

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-03 Thread Ismael Juma
Hi Colin, I still need to do a detailed review, but I have a couple of comments/questions: 1. I am not sure about having the options/response classes as inner classes of the interface. It means that file containing the interface will be huge eventually. And the classes are not necessarily

[jira] [Comment Edited] (KAFKA-4830) Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-03-03 Thread Marc Juchli (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4830?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894483#comment-15894483 ] Marc Juchli edited comment on KAFKA-4830 at 3/3/17 2:41 PM: I'd like to tackle

[jira] [Commented] (KAFKA-4830) Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-03-03 Thread Marc Juchli (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4830?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894483#comment-15894483 ] Marc Juchli commented on KAFKA-4830: I'd like to tackle this, but beforehand I have a question. I see

Re: [VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-03-03 Thread Ismael Juma
First of all, sorry to arrive late on this. Jun, do you have a reference that states that "purge" means to remove a portion? If I do "define: purge" on Google, one of the definitions is "physically remove (something) completely." In the PR, I was asking about the reasoning more than suggesting a

[jira] [Updated] (KAFKA-4710) Logging trait breaking logging location information retrival by log4j

2017-03-03 Thread Yuto Kawamura (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4710?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuto Kawamura updated KAFKA-4710: - Status: Open (was: Patch Available) > Logging trait breaking logging location information

[GitHub] kafka pull request #2465: KAFKA-4710: Interpolate log4j's logging source int...

2017-03-03 Thread kawamuray
Github user kawamuray closed the pull request at: https://github.com/apache/kafka/pull/2465 --- 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

[jira] [Commented] (KAFKA-4710) Logging trait breaking logging location information retrival by log4j

2017-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4710?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894376#comment-15894376 ] ASF GitHub Bot commented on KAFKA-4710: --- Github user kawamuray closed the pull request at:

Re: [DISCUSS] KIP-124: Request rate quotas

2017-03-03 Thread Rajini Sivaram
Jun, Agree about the two scenarios. But still not sure about a single quota covering both network threads and I/O threads with per-thread quota. If there are 10 I/O threads and 5 network threads and I want to assign half the quota to userA, the quota would be 750%. I imagine, internally, we

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-03 Thread Eno Thereska
Another question: The KIP doesn’t exactly spell out how it uses the idempotence guarantee from KIP-98. It seems that only the transactional part is needed. Or is the idempotence guarantee working behind the scenes and helping for some scenarios for which it is not worthwhile aborting a

[jira] [Comment Edited] (KAFKA-4795) Confusion around topic deletion

2017-03-03 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4795?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894178#comment-15894178 ] Manikumar edited comment on KAFKA-4795 at 3/3/17 11:26 AM: --- [~vahid] If we

[jira] [Commented] (KAFKA-4795) Confusion around topic deletion

2017-03-03 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4795?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894178#comment-15894178 ] Manikumar commented on KAFKA-4795: -- [~vahid] If we disable topic deletion then we clearing the delete

[jira] [Commented] (KAFKA-4807) Kafka broker fail over bug in zookeeper

2017-03-03 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4807?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894097#comment-15894097 ] Manikumar commented on KAFKA-4807: -- This is related to KAFKA-3959. offsets.topic.replication.factor

[jira] [Issue Comment Deleted] (KAFKA-4823) Creating Kafka Producer on application running on Java older version

2017-03-03 Thread Aneesh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4823?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aneesh updated KAFKA-4823: -- Comment: was deleted (was: [~ijuma] Kafka 0.8.2.0 worked for me.I did go through REST proxy , but couldn't

[jira] [Work started] (KAFKA-3182) Failure in kafka.network.SocketServerTest.testSocketsCloseOnShutdown

2017-03-03 Thread Armin Braun (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3182?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3182 started by Armin Braun. -- > Failure in kafka.network.SocketServerTest.testSocketsCloseOnShutdown >

[jira] [Commented] (KAFKA-3182) Failure in kafka.network.SocketServerTest.testSocketsCloseOnShutdown

2017-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3182?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15893974#comment-15893974 ] ASF GitHub Bot commented on KAFKA-3182: --- GitHub user original-brownbear opened a pull request:

[GitHub] kafka pull request #2632: KAFKA-3182: Fix Transient Test Failure (kafka.netw...

2017-03-03 Thread original-brownbear
GitHub user original-brownbear opened a pull request: https://github.com/apache/kafka/pull/2632 KAFKA-3182: Fix Transient Test Failure (kafka.network.SocketServerTest.testSocketsCloseOnShutdown) Fixes https://issues.apache.org/jira/browse/KAFKA-3182 by: * Turning off Nagle on

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

2017-03-03 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4631; Request metadata in consumer if topic/partitions unavailable -- [...truncated 835.01 KB...]