Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-03-12 Thread Matthias J. Sax
@John: yes, we would throw if configs are missing (it's an implementation details IMHO and thus I did not include it in the KIP) @Guozhang: 1) I understand know what you mean. We can certainly, allow all values "0.10.0.x", "0.10.1.x", "0.10.2.x", ... "1.1.x" for `upgrade.from` parameter. I had a

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

2018-03-12 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-6624; Prevent concurrent log flush and log deletion (#4663) -- [...truncated 418.30 KB...] kafka.network.SocketServerTest > testConnectionIdRe

Re: How to set offset of a (topic,partition,group) without closing other consumers?

2018-03-12 Thread Matthias J. Sax
The consumer that owns the partition must do a `seek()` to the corresponding offsets. It's not possible from "outside" to manipulate the offset (if would also be dangerous if this would be allowed as a consumer could get into a bad state). Also note, even if you could commit a offset for the parti

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-12 Thread Jan Filipiak
Hi Jun, I will focus on point 61 as I think its _the_ fundamental part that I cant get across at the moment. Kafka is the platform to have state materialized multiple times from one input. I emphasize this: It is the building block in architectures that allow you to have your state maintaine

[jira] [Resolved] (KAFKA-6024) Consider moving validation in KafkaConsumer ahead of call to acquireAndEnsureOpen()

2018-03-12 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6024?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-6024. Resolution: Fixed Fix Version/s: 1.2.0 > Consider moving validation in KafkaConsumer

[jira] [Created] (KAFKA-6643) Warm up new replicas from scratch when changelog topic has retention time

2018-03-12 Thread Navinder Brar (JIRA)
Navinder Brar created KAFKA-6643: Summary: Warm up new replicas from scratch when changelog topic has retention time Key: KAFKA-6643 URL: https://issues.apache.org/jira/browse/KAFKA-6643 Project: Kafk

[jira] [Resolved] (KAFKA-6624) log segment deletion could cause a disk to be marked offline incorrectly

2018-03-12 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6624?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-6624. Resolution: Fixed Fix Version/s: 1.1.0 > log segment deletion could cause a disk to be marked offline

[jira] [Created] (KAFKA-6642) Rack aware replica assignment in kafka streams

2018-03-12 Thread Ashish Surana (JIRA)
Ashish Surana created KAFKA-6642: Summary: Rack aware replica assignment in kafka streams Key: KAFKA-6642 URL: https://issues.apache.org/jira/browse/KAFKA-6642 Project: Kafka Issue Type: New

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

2018-03-12 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix incorrect JavaDoc (type mismatch) (#4632) -- [...truncated 1.48 MB...] kafka.log.BrokerCompressionTest > testBrokerSideCompression[17] ST

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

2018-03-12 Thread Apache Jenkins Server
See

How to set offset of a (topic,partition,group) without closing other consumers?

2018-03-12 Thread York Zhang
I have some running consumers. I want to set the offset of a specific topic-partition-group with a new consumer instance in the same group. I have tried to use subscribe(), but kafka randomly assign partition to the new consumer, I can't operate the partition I want. Then I tried to use assign(),

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

2018-03-12 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Fix incorrect JavaDoc (type mismatch) (#4632) -- [...truncated 417.57 KB...] kafka.api.SaslClientsWithInvalidCredentialsTest > testKafkaAd

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-12 Thread Jun Rao
Hi, Jan, Thanks for the reply. A few more comments below. 50. Ok, we can think a bit harder for supporting compacted topics. 51. This is a fundamental design question. In the more common case, the reason why someone wants to increase the number of partitions is that the consumer application is s

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

2018-03-12 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-6641) Consider auto repartitioning for Stream.transform() API

2018-03-12 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-6641: Summary: Consider auto repartitioning for Stream.transform() API Key: KAFKA-6641 URL: https://issues.apache.org/jira/browse/KAFKA-6641 Project: Kafka Issue T

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

2018-03-12 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Fix wrong message in `bin/kafka-run-class.sh` (#4682) [jason] KAFKA-3368; Add documentation for old message format (#3425) -- [...truncated 41

Jenkins build is back to normal : kafka-1.1-jdk7 #75

2018-03-12 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-6640) Improve efficiency of KafkaAdminClient.describeTopics()

2018-03-12 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-6640: --- Summary: Improve efficiency of KafkaAdminClient.describeTopics() Key: KAFKA-6640 URL: https://issues.apache.org/jira/browse/KAFKA-6640 Project: Kafka Issue Type: Impro

[jira] [Resolved] (KAFKA-3368) Add the Message/Record set protocol to the protocol docs

2018-03-12 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3368?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-3368. Resolution: Fixed Fix Version/s: 1.2.0 > Add the Message/Record set protocol to the p

[jira] [Created] (KAFKA-6639) Follower may have sparse index if catching up

2018-03-12 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-6639: -- Summary: Follower may have sparse index if catching up Key: KAFKA-6639 URL: https://issues.apache.org/jira/browse/KAFKA-6639 Project: Kafka Issue Type: B

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

2018-03-12 Thread Apache Jenkins Server
See Changes: [github] KAFKA-6560: Add docs for KIP-261 (#4685) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H24 (ubunt

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

2018-03-12 Thread Apache Jenkins Server
See Changes: [github] KAFKA-6560: Add docs for KIP-261 (#4685) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H29 (ubun

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

2018-03-12 Thread Apache Jenkins Server
See Changes: [github] KAFKA-6560: Add docs for KIP-261 (#4685) -- Started by an SCM change Started by an SCM change [EnvInject] - Loading node environment variables. Build

Re: Log Retention Period of Kafka Messages

2018-03-12 Thread James Cheng
> On Mar 12, 2018, at 10:13 AM, Kyle Tinker > wrote: > > You have a couple options: > 1) You can adjust log.segment.bytes to make the segments smaller so that > individual segments can be trimmed > 2) You can set log.roll.hours, which will roll to a new log segment even if > the size hasn't

Re: [VOTE] KIP-267: Add Processor Unit Test Support to Kafka Streams Test Utils

2018-03-12 Thread Damian Guy
+1 On Fri, 9 Mar 2018 at 21:30, Matthias J. Sax wrote: > Guozhang, renaming the class is an internal change and I think it's not > required to cover it in the KIP. > > > +1 (binding) > > > -Matthias > > On 3/9/18 1:26 PM, Guozhang Wang wrote: > > Hi John, > > > > You mentioned you are going to do

[jira] [Created] (KAFKA-6638) Controller should remove replica from ISR if the replica is removed from the replica set

2018-03-12 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-6638: --- Summary: Controller should remove replica from ISR if the replica is removed from the replica set Key: KAFKA-6638 URL: https://issues.apache.org/jira/browse/KAFKA-6638 Project:

[jira] [Created] (KAFKA-6637) if set topic config segment.ms=0 Kafka broker won't be able to start

2018-03-12 Thread Chong Wang (JIRA)
Chong Wang created KAFKA-6637: - Summary: if set topic config segment.ms=0 Kafka broker won't be able to start Key: KAFKA-6637 URL: https://issues.apache.org/jira/browse/KAFKA-6637 Project: Kafka

[jira] [Created] (KAFKA-6636) ReplicaFetcherThread should not die if hw < 0

2018-03-12 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-6636: --- Summary: ReplicaFetcherThread should not die if hw < 0 Key: KAFKA-6636 URL: https://issues.apache.org/jira/browse/KAFKA-6636 Project: Kafka Issue Type: Improvement

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-03-12 Thread Guozhang Wang
Hello Matthias, thanks for your replies. 1) About the config names: actually I was trying to not expose implementation details :) My main concern was that in your proposal the values need to cover the span of all the versions that are actually using the same version, i.e. "0.10.1.x-1.1.x". So if

[jira] [Resolved] (KAFKA-4984) Unable to produce or consume when enabling authentication SASL/Kerberos

2018-03-12 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4984?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4984. -- Resolution: Cannot Reproduce {color:#00}Closing inactive issue. Please reopen if you think the issue

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

2018-03-12 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5504?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-5504. -- Resolution: Not A Problem Closing as per above comments. > Kafka controller is not getting elected > --

[jira] [Resolved] (KAFKA-6022) mirror maker stores offset in zookeeper

2018-03-12 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6022?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-6022. -- Resolution: Not A Problem Please reopen if you think the issue still exists > mirror maker stores offse

RE: Log Retention Period of Kafka Messages

2018-03-12 Thread Kyle Tinker
You have a couple options: 1) You can adjust log.segment.bytes to make the segments smaller so that individual segments can be trimmed 2) You can set log.roll.hours, which will roll to a new log segment even if the size hasn't been reached Note: #1 and #2 also have per-topic setting controls htt

[jira] [Resolved] (KAFKA-5467) setting offset retention minutes to a lower value is not reflecting

2018-03-12 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5467?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-5467. -- Resolution: Cannot Reproduce Closing inactive issue. Please reopen if you think the issue exists. > set

Log Retention Period of Kafka Messages

2018-03-12 Thread Muruganandham, Ashokkumar
Hi Team, Form the documentation I could see that log retention period is relative to the partition segments. So unless the segment is closed (1Gb/1 Week) the message will never be deleted. Consider I have a very critical use case where I will need to delete the data every day and my segment re

Dots in topic names in metric labels

2018-03-12 Thread Reftel, Magnus
Hi, Dots in topic names in labels are replaced with underscores in consumer metrics (it appears it has been that way all the way back since KAFKA-3233), but not in broker or producer metrics. We'd like to make visualizations that are parameterized by topic name, and show overview information ab

Re: [DISCUSS] KIP-258: Allow to Store Record Timestamps in RocksDB

2018-03-12 Thread John Roesler
This all sounds awesome to me. One (very minor) thought about the config parameters. You might consider throwing an exception if "upgrade.from" is set but "upgrade.mode" is not. This would eliminate the risk that folks forget to unset "upgrade.from" and just leave it in their configs indefinitely.

[jira] [Resolved] (KAFKA-6337) Error for partition [__consumer_offsets,15] to broker

2018-03-12 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-6337. -- Resolution: Cannot Reproduce Please reopen if you think the issue still exists > Error for partition [_

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

2018-03-12 Thread Apache Jenkins Server
See Changes: [github] MINOR: Revert to ZooKeeper 3.4.10 due to ZOOKEEPER-2960 (#4678) -- [...truncated 417.97 KB...] kafka.tools.ConsoleConsumerTest > shouldStopWhenOutpu

Build failed in Jenkins: kafka-1.1-jdk7 #74

2018-03-12 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Tag AWS instances with Jenkins build url (#4657) [ismael] MINOR: Revert to ZooKeeper 3.4.10 due to ZOOKEEPER-2960 (#4678) -- [...truncated 416.60

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

2018-03-12 Thread Apache Jenkins Server
See

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

2018-03-12 Thread Apache Jenkins Server
See

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

2018-03-12 Thread Apache Jenkins Server
See Changes: [github] MINOR: Revert to ZooKeeper 3.4.10 due to ZOOKEEPER-2960 (#4678) -- [...truncated 416.54 KB...] kafka.server.epoch.LeaderEpochFileCacheTest > shouldR

Re: [DISCUSS] KIP-253: Support in-order message delivery with partition expansion

2018-03-12 Thread Jan Filipiak
Hey, just want to throw in my question from the longer email in the other Thread here. How will the bloom filter help a new consumer to decide to apply the key or not? Why can we afford having a topic where its apparently not possible to start a new application on? I think this is an overall