Re: [DISCUSS] KIP-143: Controller Health Metrics

2017-05-08 Thread Becket Qin
@Ismael, About the stage and event type. Yes, I think each event handling should have those stages covered. It is similar to what we are doing for the requests on the broker side. We have benefited from such systematic metric structure a lot so I think it would be worth following the same way in

Re: [VOTE] KIP-133: List and Alter Configs Admin APIs (second attempt)

2017-05-08 Thread Robert Benson
+1 (non-binding) On Sun, May 7, 2017 at 9:01 PM, Ismael Juma wrote: > [Seems like the original message ended up in the discuss thread in GMail, > so trying again] > > Hi everyone, > > I believe I addressed the comments in the discussion thread and given the > impending KIP

Re: [VOTE] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread BigData dev
+1 (non-binding) On Mon, May 8, 2017 at 3:25 PM, Dongjin Lee wrote: > +1 > > On Tue, May 9, 2017 at 7:24 AM, Sriram Subramanian > wrote: > > > +1 > > > > On Mon, May 8, 2017 at 2:14 PM, Konstantine Karantasis < > > konstant...@confluent.io> wrote: > > > >

Re: [DISCUSS] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread BigData dev
Thank You got it. On Mon, May 8, 2017 at 8:34 PM, Randall Hauch wrote: > Yes, that's the approach I'm suggesting and that is mentioned in the KIP. I > also propose that the distributed configuration provided in the examples > set the replication factor to one but include a

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

2017-05-08 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5176; AdminClient: add controller and clusterId methods to -- [...truncated 1.67 MB...] kafka.integration.SaslPlaintextTopicMetadataTest >

Re: [DISCUSS] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread Randall Hauch
Yes, that's the approach I'm suggesting and that is mentioned in the KIP. I also propose that the distributed configuration provided in the examples set the replication factor to one but include a relevant comment. On Mon, May 8, 2017 at 11:14 PM, BigData dev wrote: >

Re: [DISCUSS] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread BigData dev
So, when Kafka broker is less than 3, and the user has not set the replication configuration it will throw an error to the user, to correct the configuration according to his setup? Is this the approach you are suggesting here? On Mon, May 8, 2017 at 7:13 PM, Randall Hauch

Two problems looking for KIPs

2017-05-08 Thread Edoardo Comar
Thanks to all the people at the Kafka Summit who spent their time discussing with me a couple of edge problem cases. I'd be happy to collaborate to KIPs for these two problems, for which I just opened JIRAs as the motivation is still missing a proposed solution.

[jira] [Updated] (KAFKA-5201) Compacted topic could be misused to fill up a disk but deletion policy can't retain legitimate keys

2017-05-08 Thread Edoardo Comar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5201?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edoardo Comar updated KAFKA-5201: - Summary: Compacted topic could be misused to fill up a disk but deletion policy can't retain

[jira] [Created] (KAFKA-5201) Compacted topic could be misused up to fill a disk;

2017-05-08 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-5201: Summary: Compacted topic could be misused up to fill a disk; Key: KAFKA-5201 URL: https://issues.apache.org/jira/browse/KAFKA-5201 Project: Kafka Issue

[jira] [Created] (KAFKA-5200) Deleting topic when one broker is down will prevent topic to be re-creatable

2017-05-08 Thread Edoardo Comar (JIRA)
Edoardo Comar created KAFKA-5200: Summary: Deleting topic when one broker is down will prevent topic to be re-creatable Key: KAFKA-5200 URL: https://issues.apache.org/jira/browse/KAFKA-5200 Project:

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

2017-05-08 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5176; AdminClient: add controller and clusterId methods to -- [...truncated 851.62 KB...] kafka.integration.SaslPlaintextTopicMetadataTest >

Re: [DISCUSS] KIP-143: Controller Health Metrics

2017-05-08 Thread Onur Karaman
I had a similar comment to Becket but accidentally posted it on the vote thread last Friday. From that thread: "I noticed that both the ControllerState metric and the *RateAndTimeMs metrics only cover a subset of the controller event types. Was this intentional?" I think it makes most sense to

Re: [DISCUSS] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread Randall Hauch
One of the "Rejected Alternatives" was to do something "smarter" by automatically reducing the replication factor when the cluster size is smaller than the replication factor. However, this is extremely unintuitive, and in rare cases (e.g., during a partial outage) might even result in internal

[jira] [Updated] (KAFKA-5160) KIP-98 : broker side handling for the TxnOffsetCommitRequest

2017-05-08 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5160?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-5160: --- Issue Type: Sub-task (was: Task) Parent: KAFKA-4815 > KIP-98 : broker side handling

[jira] [Resolved] (KAFKA-5176) AdminClient: add controller and clusterId methods to DescribeClusterResults

2017-05-08 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5176?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5176. Resolution: Fixed > AdminClient: add controller and clusterId methods to DescribeClusterResults >

[jira] [Commented] (KAFKA-5176) AdminClient: add controller and clusterId methods to DescribeClusterResults

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

[GitHub] kafka pull request #2977: KAFKA-5176: AdminClient: add controller and cluste...

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

Re: [DISCUSS] KIP-144: Exponential backoff for broker reconnect attempts

2017-05-08 Thread Dana Powers
s/back-office/backoff/ Final note: although the goal here is not to resolve contention (as in the aws article), I think we do still want a relatively smooth rate of reconnects across all clients to avoid storm spikes. Full Jitter does that. I expect that narrower jitter bands will lead to more

Re: [DISCUSS] KIP-144: Exponential backoff for broker reconnect attempts

2017-05-08 Thread Dana Powers
For some discussion of jitter and exponential back-office, I found this article useful: https://www.awsarchitectureblog.com/2015/03/backoff.html My initial POC used the "Full Jitter" approach described therein. Equal Jitter is good too, and may perform a little better. It is random distribution

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

2017-05-08 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4839; Throw NoOffsetForPartitionException from poll once for all [jason] KAFKA-3353; Remove deprecated producer configs [jason] KAFKA-3763; Remove deprecated APIs for 0.11.0.0

Re: [VOTE] KIP-151: Expose Connector type in REST API (first attempt :)

2017-05-08 Thread BigData dev
+1 (non-binding) Thanks, Bharat On Mon, May 8, 2017 at 4:39 PM, Konstantine Karantasis < konstant...@confluent.io> wrote: > +1 (non-binding) > > On Mon, May 8, 2017 at 3:39 PM, dan wrote: > > > i'd like to begin voting on > >

[jira] [Commented] (KAFKA-5061) client.id should be set for Connect producers/consumers

2017-05-08 Thread Bharat Viswanadham (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5061?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001795#comment-16001795 ] Bharat Viswanadham commented on KAFKA-5061: --- [~ewencp] So you are suggesting to add client.id

Re: [VOTE] KIP-151: Expose Connector type in REST API (first attempt :)

2017-05-08 Thread Konstantine Karantasis
+1 (non-binding) On Mon, May 8, 2017 at 3:39 PM, dan wrote: > i'd like to begin voting on > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 151+Expose+Connector+type+in+REST+API > > discussion should remain on >

Re: [DISCUSS] KIP-143: Controller Health Metrics

2017-05-08 Thread Ismael Juma
Hi Becket, Thanks for the feedback. Comments inline. On Tue, May 9, 2017 at 12:19 AM, Becket Qin wrote: > > q10. With event loop based controller design, it seems natrural to have the > processing time for each controller event type. In that case, the current > metrics

[jira] [Commented] (KAFKA-3554) Generate actual data with specific compression ratio and add multi-thread support in the ProducerPerformance tool.

2017-05-08 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3554?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001780#comment-16001780 ] Jiangjie Qin commented on KAFKA-3554: - [~airbots] I lose track of this ticket... It was not checked in

Re: [DISCUSS] KIP-143: Controller Health Metrics

2017-05-08 Thread Becket Qin
Hi Ismael and Jun, Thanks for the KIP. It is very useful. A couple of comments: q10. With event loop based controller design, it seems natrural to have the processing time for each controller event type. In that case, the current metrics seem not covering all the event processing time? e.g.

Re: [VOTE] KIP-140: Add administrative RPCs for adding, deleting, and listing ACLs

2017-05-08 Thread Colin McCabe
Just a quick note: earlier Magnus and I talked about what happens if you attempt to add a duplicate ACL. It appears that SimpleAuthorizer silently allows re-adding an ACL that already exists. There is no error raised. However, the ACLs are deduplicated on the back end (only one ACL will show up

[jira] [Commented] (KAFKA-5117) Kafka Connect REST endpoints reveal Password typed values

2017-05-08 Thread Bharat Viswanadham (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5117?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001717#comment-16001717 ] Bharat Viswanadham commented on KAFKA-5117: --- [~tholmes] are you still working on this? > Kafka

Re: [DISCUSS]: KIP-149: Enabling key access in ValueTransformer, ValueMapper, and ValueJoiner

2017-05-08 Thread Matthias J. Sax
Hi, I was reading the updated KIP and I am wondering, if we should do the design a little different. Instead of distinguishing between a RichFunction and non-RichFunction at runtime level, we would use RichFunctions all the time. Thus, on the DSL entry level, if a user provides a

[jira] [Updated] (KAFKA-3623) Make KStreamTestDriver extending from ExternalResource

2017-05-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3623?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3623: - Labels: newbie test (was: test) > Make KStreamTestDriver extending from ExternalResource >

[VOTE] KIP-151: Expose Connector type in REST API (first attempt :)

2017-05-08 Thread dan
i'd like to begin voting on https://cwiki.apache.org/confluence/display/KAFKA/KIP-151+Expose+Connector+type+in+REST+API discussion should remain on http://mail-archives.apache.org/mod_mbox/kafka-dev/201705.mbox/%3ccafjy-u-pf7yxsradx_zaqycx2+sswmvpsbca4tdmpp5834s...@mail.gmail.com%3E This voting

[jira] [Updated] (KAFKA-5198) RocksDbStore#openIterators should be synchronized, since it is accessed from multiple threads

2017-05-08 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5198?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-5198: --- Component/s: streams > RocksDbStore#openIterators should be synchronized, since it is

[jira] [Commented] (KAFKA-5079) ProducerBounceTest fails occasionally with a SocketTimeoutException

2017-05-08 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5079?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001669#comment-16001669 ] Colin P. McCabe commented on KAFKA-5079: [~xvrl]: I filed KAFKA-5199 to investigate running the

Re: [VOTE] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread Dongjin Lee
+1 On Tue, May 9, 2017 at 7:24 AM, Sriram Subramanian wrote: > +1 > > On Mon, May 8, 2017 at 2:14 PM, Konstantine Karantasis < > konstant...@confluent.io> wrote: > > > +1 (non binding) > > > > On Mon, May 8, 2017 at 1:33 PM, Stephane Maarek < > >

Re: [VOTE] KIP-133: List and Alter Configs Admin APIs (second attempt)

2017-05-08 Thread Guozhang Wang
+1 On Mon, May 8, 2017 at 10:48 AM, Colin McCabe wrote: > +1 (non-binding) > > best, > Colin > > > On Mon, May 8, 2017, at 08:52, Ismael Juma wrote: > > Quick update, I renamed ListConfigs to DescribeConfigs (and related > > classes > > and methods) as that is more

[jira] [Created] (KAFKA-5199) Run Kafka JUnit build inside Docker

2017-05-08 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-5199: -- Summary: Run Kafka JUnit build inside Docker Key: KAFKA-5199 URL: https://issues.apache.org/jira/browse/KAFKA-5199 Project: Kafka Issue Type: Bug

Re: [VOTE] KIP-144: Exponential backoff for broker reconnect attempts

2017-05-08 Thread Dongjin Lee
+1 On Tue, May 9, 2017 at 6:48 AM, Joel Koshy wrote: > +1 > > On Mon, May 8, 2017 at 11:07 AM, Colin McCabe wrote: > > > +1 (non-binding) > > > > > > > > On Sat, May 6, 2017, at 11:13, Dana Powers wrote: > > > +1 ! > > > > > > On May 6, 2017 4:49 AM,

Re: [VOTE] KIP-146: Isolation of dependencies and classes in Kafka Connect (restarted voting thread)

2017-05-08 Thread Guozhang Wang
+1 On Mon, May 8, 2017 at 1:36 PM, Stephane Maarek < steph...@simplemachines.com.au> wrote: > +1 > Thanks heaps I can’t wait! > > > On 9/5/17, 4:48 am, "Konstantine Karantasis" > wrote: > > ** Restarting the voting thread here, with a different title to avoid >

Re: [VOTE] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread Sriram Subramanian
+1 On Mon, May 8, 2017 at 2:14 PM, Konstantine Karantasis < konstant...@confluent.io> wrote: > +1 (non binding) > > On Mon, May 8, 2017 at 1:33 PM, Stephane Maarek < > steph...@simplemachines.com.au> wrote: > > > +1 (non binding) > > > > > > > > On 9/5/17, 5:51 am, "Randall Hauch"

Re: [VOTE] KIP-153 (separating replication traffic from BytesOutPerSec metric)

2017-05-08 Thread Guozhang Wang
+1 On Mon, May 8, 2017 at 2:25 PM, Joel Koshy wrote: > +1 > > On Mon, May 8, 2017 at 8:30 AM, Roger Hoover > wrote: > > > +1 > > > > Sent from my iPhone > > > > > On May 8, 2017, at 5:00 AM, Edoardo Comar wrote: > > > > > > +1 >

Re: [DISCUSS] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread BigData dev
Hi, I liked the KIP, as it will avoid so many errors which user can make during setup. I have 1 questions here. 1. As default replication factor is set to 3, but if Kafka cluster is setup for one node, then the user needs to override the default configuraion, till then topics will not be created.

[jira] [Commented] (KAFKA-3502) Build is killed during kafka streams tests due to `pure virtual method called` error

2017-05-08 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3502?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001632#comment-16001632 ] Colin P. McCabe commented on KAFKA-3502: Possibly relevant: KAFKA-5189 > Build is killed during

[GitHub] kafka pull request #3000: KAFKA-5198. RocksDbStore#openIterators should be s...

2017-05-08 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/3000 KAFKA-5198. RocksDbStore#openIterators should be synchronized, since … …it is accessed from multiple threads You can merge this pull request into a Git repository by running: $ git pull

[jira] [Commented] (KAFKA-5198) RocksDbStore#openIterators should be synchronized, since it is accessed from multiple threads

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

[jira] [Created] (KAFKA-5198) RocksDbStore#openIterators should be synchronized, since it is accessed from multiple threads

2017-05-08 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-5198: -- Summary: RocksDbStore#openIterators should be synchronized, since it is accessed from multiple threads Key: KAFKA-5198 URL: https://issues.apache.org/jira/browse/KAFKA-5198

Re: [VOTE] KIP-144: Exponential backoff for broker reconnect attempts

2017-05-08 Thread Joel Koshy
+1 On Mon, May 8, 2017 at 11:07 AM, Colin McCabe wrote: > +1 (non-binding) > > > > On Sat, May 6, 2017, at 11:13, Dana Powers wrote: > > +1 ! > > > > On May 6, 2017 4:49 AM, "Edoardo Comar" wrote: > > > > > +1 (non binding) > > > thanks > > >

Re: [DISCUSS] KIP-143: Controller Health Metrics

2017-05-08 Thread Joel Koshy
Hi Ismael, > > What about a broker that is not the controller? Would you need a separate > > idle-not-controller state? > > > Do we need a separate state or can users just use the ActiveControllerCount > metric to check if the broker is the controller? > Sure - the ACC metric should be

[jira] [Updated] (KAFKA-5197) add a tool analyzing zookeeper client performance across its various apis

2017-05-08 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5197?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Onur Karaman updated KAFKA-5197: Status: Patch Available (was: In Progress) > add a tool analyzing zookeeper client performance

[GitHub] kafka pull request #2999: KAFKA-5197: add a tool analyzing zookeeper client ...

2017-05-08 Thread onurkaraman
GitHub user onurkaraman opened a pull request: https://github.com/apache/kafka/pull/2999 KAFKA-5197: add a tool analyzing zookeeper client performance across its various apis The raw zookeeper client offers various means of getting and setting znodes. It would be useful to have a

[jira] [Work started] (KAFKA-5197) add a tool analyzing zookeeper client performance across its various apis

2017-05-08 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5197?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5197 started by Onur Karaman. --- > add a tool analyzing zookeeper client performance across its various apis >

[jira] [Commented] (KAFKA-5197) add a tool analyzing zookeeper client performance across its various apis

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

[jira] [Created] (KAFKA-5197) add a tool analyzing zookeeper client performance across its various apis

2017-05-08 Thread Onur Karaman (JIRA)
Onur Karaman created KAFKA-5197: --- Summary: add a tool analyzing zookeeper client performance across its various apis Key: KAFKA-5197 URL: https://issues.apache.org/jira/browse/KAFKA-5197 Project: Kafka

Re: [VOTE] KIP-153 : Include only client traffic in BytesOutPerSec metric

2017-05-08 Thread Guozhang Wang
+1 On Sun, May 7, 2017 at 7:32 PM, Jun Rao wrote: > Hi, Everyone, > > I would like to start the voting process for KIP-153 : Include only client > traffic in BytesOutPerSec metric. > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- >

Re: How do I generate/view the site docs, when developing?

2017-05-08 Thread Guozhang Wang
Hello James, There are two options: you can setup a local apache server to render the docs (what I did when did the refactoring) or had some manual edits, which are summarized here (see Kafka Website Repository):

Re: [DISCUSS] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread Konstantine Karantasis
Thanks a lot for the KIP Randall. This improvement should simplify both regular deployments and testing! A minor comment. Maybe it would be nice to add a note about why there's no need for the property: config.storage.partitions I'm mentioning this for the sake of completeness, in case someone

Re: [VOTE] KIP-153 (separating replication traffic from BytesOutPerSec metric)

2017-05-08 Thread Joel Koshy
+1 On Mon, May 8, 2017 at 8:30 AM, Roger Hoover wrote: > +1 > > Sent from my iPhone > > > On May 8, 2017, at 5:00 AM, Edoardo Comar wrote: > > > > +1 > > Many thanks Jun > > -- > > Edoardo Comar > > IBM

Re: [VOTE] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread Konstantine Karantasis
+1 (non binding) On Mon, May 8, 2017 at 1:33 PM, Stephane Maarek < steph...@simplemachines.com.au> wrote: > +1 (non binding) > > > > On 9/5/17, 5:51 am, "Randall Hauch" wrote: > > Hi, everyone. > > Given the simple and non-controversial nature of the KIP, I would like

[jira] [Commented] (KAFKA-3554) Generate actual data with specific compression ratio and add multi-thread support in the ProducerPerformance tool.

2017-05-08 Thread Chen He (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3554?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001535#comment-16001535 ] Chen He commented on KAFKA-3554: This contribution is really valuable. Why it is not checked-in? If not

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

2017-05-08 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4839; Throw NoOffsetForPartitionException from poll once for all [jason] KAFKA-3353; Remove deprecated producer configs [jason] KAFKA-3763; Remove deprecated APIs for 0.11.0.0

[jira] [Commented] (KAFKA-5166) Add option "dry run" to Streams application reset tool

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

[GitHub] kafka pull request #2998: KAFKA-5166: Add option dry run to Streams applicat...

2017-05-08 Thread bharatviswa504
GitHub user bharatviswa504 opened a pull request: https://github.com/apache/kafka/pull/2998 KAFKA-5166: Add option dry run to Streams application reset tool Add an option dry-run that only prints what topics would get modified/deleted without actually applying any actions. You can

[jira] [Commented] (KAFKA-5154) Kafka Streams throws NPE during rebalance

2017-05-08 Thread Lukas Gemela (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001501#comment-16001501 ] Lukas Gemela commented on KAFKA-5154: - [~guozhang] Yes I know log does not contain NPE, I just wanted

Re: [VOTE] KIP-146: Isolation of dependencies and classes in Kafka Connect (restarted voting thread)

2017-05-08 Thread Stephane Maarek
+1 Thanks heaps I can’t wait! On 9/5/17, 4:48 am, "Konstantine Karantasis" wrote: ** Restarting the voting thread here, with a different title to avoid collapsing this thread's messages with the discussion thread's messages in mail clients. Apologies for

Re: [VOTE] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread Stephane Maarek
+1 (non binding) On 9/5/17, 5:51 am, "Randall Hauch" wrote: Hi, everyone. Given the simple and non-controversial nature of the KIP, I would like to start the voting process for KIP-154: Add Kafka Connect configuration properties for creating internal

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

2017-05-08 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-5174: Have at least 2 threads for compaction and flushing in -- [...truncated 3.91 MB...]

[jira] [Commented] (KAFKA-5154) Kafka Streams throws NPE during rebalance

2017-05-08 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001459#comment-16001459 ] Matthias J. Sax commented on KAFKA-5154: [~guozhang] Ah thanks. I was confused as the comment

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

2017-05-08 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-5174: Have at least 2 threads for compaction and flushing in -- [...truncated 2.48 MB...]

[VOTE] KIP-154 Add Kafka Connect configuration properties for creating internal topics

2017-05-08 Thread Randall Hauch
Hi, everyone. Given the simple and non-controversial nature of the KIP, I would like to start the voting process for KIP-154: Add Kafka Connect configuration properties for creating internal topics:

Re: [VOTE] KIP-126 - Allow KafkaProducer to split and resend oversized batches.

2017-05-08 Thread Becket Qin
Thanks everyone for voting. KIP-126 has passed with +3 binding (Joel, Jason and me) and +3 non-binding (Dong, Bill, Michael) Thanks, Jiangjie (Becket) Qin On Mon, May 8, 2017 at 10:06 AM, Jason Gustafson wrote: > +1 > > On Thu, May 4, 2017 at 9:11 PM, Michael Pearce

[jira] [Commented] (KAFKA-5154) Kafka Streams throws NPE during rebalance

2017-05-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001376#comment-16001376 ] Guozhang Wang commented on KAFKA-5154: -- [~mjsax] Not sure I understand your last sentence? I do see

[jira] [Updated] (KAFKA-4479) Streams tests should pass without hardcoded Time.SYSTEM in GroupCoordinator

2017-05-08 Thread Amit Daga (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4479?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Amit Daga updated KAFKA-4479: - Attachment: test.zip > Streams tests should pass without hardcoded Time.SYSTEM in GroupCoordinator >

[jira] [Commented] (KAFKA-4479) Streams tests should pass without hardcoded Time.SYSTEM in GroupCoordinator

2017-05-08 Thread Amit Daga (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4479?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001370#comment-16001370 ] Amit Daga commented on KAFKA-4479: -- [~ijuma] Initial findings: After trying to debug for a while, I found

Re: [VOTE] KIP-146: Classloading Isolation in Connect

2017-05-08 Thread Konstantine Karantasis
Because of interference with the discussion thread in some mail clients (e.g. gmail), voting thread has moved here: https://www.mail-archive.com/dev@kafka.apache.org/msg72274.html Please cast your votes on this thread instead. -Konstantine On Mon, May 8, 2017 at 11:30 AM, Guozhang Wang

[jira] [Assigned] (KAFKA-5167) streams task gets stuck after re-balance due to LockException

2017-05-08 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5167?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-5167: -- Assignee: Matthias J. Sax > streams task gets stuck after re-balance due to

[jira] [Work started] (KAFKA-5167) streams task gets stuck after re-balance due to LockException

2017-05-08 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5167?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5167 started by Matthias J. Sax. -- > streams task gets stuck after re-balance due to LockException >

[jira] [Assigned] (KAFKA-5070) org.apache.kafka.streams.errors.LockException: task [0_18] Failed to lock the state directory: /opt/rocksdb/pulse10/0_18

2017-05-08 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5070?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax reassigned KAFKA-5070: -- Assignee: Matthias J. Sax > org.apache.kafka.streams.errors.LockException: task [0_18]

[jira] [Work started] (KAFKA-5070) org.apache.kafka.streams.errors.LockException: task [0_18] Failed to lock the state directory: /opt/rocksdb/pulse10/0_18

2017-05-08 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5070?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5070 started by Matthias J. Sax. -- > org.apache.kafka.streams.errors.LockException: task [0_18] Failed to lock

[VOTE] KIP-146: Isolation of dependencies and classes in Kafka Connect (restarted voting thread)

2017-05-08 Thread Konstantine Karantasis
** Restarting the voting thread here, with a different title to avoid collapsing this thread's messages with the discussion thread's messages in mail clients. Apologies for the inconvenience. ** Hi all, Given that the comments during the discussion seem to have been addressed, I'm pleased to

[jira] [Commented] (KAFKA-5152) Kafka Streams keeps restoring state after shutdown is initiated during startup

2017-05-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5152?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001300#comment-16001300 ] Guozhang Wang commented on KAFKA-5152: -- Thanks [~xvrl] for reporting this issue. I think it is

Re: [VOTE] KIP-146: Classloading Isolation in Connect

2017-05-08 Thread Guozhang Wang
+1 On Mon, May 8, 2017 at 11:04 AM, Konstantine Karantasis < konstant...@confluent.io> wrote: > Hi all, > > Given that the comments during the discussion seem to have been addressed, > I'm pleased to bring > > KIP-146: Classloading Isolation in Connect >

Re: [DISCUSS] KIP-156 Add option "dry run" to Streams application reset tool

2017-05-08 Thread Steven Schlansker
> On May 8, 2017, at 11:14 AM, BigData dev wrote: > > Hi All, > I want to start a discussion on this simple KIP for Kafka Streams reset > tool (kafka-streams-application-reset.sh). > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69410150 I've not

[DISCUSS] KIP-156 Add option "dry run" to Streams application reset tool

2017-05-08 Thread BigData dev
Hi All, I want to start a discussion on this simple KIP for Kafka Streams reset tool (kafka-streams-application-reset.sh). https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=69410150 Thank you, Matthias J Sax for providing me Jira and info to work on. Thanks, Bharat

Re: [VOTE] KIP-144: Exponential backoff for broker reconnect attempts

2017-05-08 Thread Colin McCabe
+1 (non-binding) On Sat, May 6, 2017, at 11:13, Dana Powers wrote: > +1 ! > > On May 6, 2017 4:49 AM, "Edoardo Comar" wrote: > > > +1 (non binding) > > thanks > > -- > > Edoardo Comar > > IBM MessageHub > > eco...@uk.ibm.com

[jira] [Commented] (KAFKA-5070) org.apache.kafka.streams.errors.LockException: task [0_18] Failed to lock the state directory: /opt/rocksdb/pulse10/0_18

2017-05-08 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001219#comment-16001219 ] Matthias J. Sax commented on KAFKA-5070: [~guozhang] I agree about the logging... We did some

[VOTE] KIP-146: Classloading Isolation in Connect

2017-05-08 Thread Konstantine Karantasis
Hi all, Given that the comments during the discussion seem to have been addressed, I'm pleased to bring KIP-146: Classloading Isolation in Connect https://cwiki.apache.org/confluence/display/KAFKA/KIP-146+-+Classloading+Isolation+in+Connect up for voting. Again, this KIP aims to bring the

Re: [DISCUSS] KIP-146: Classloading Isolation in Connect

2017-05-08 Thread Konstantine Karantasis
In order to keep the voting thread on KIP-146 open for at least 72 hours, I'm amending my previous message and I'm starting the voting process on KIP-146 now. The discussion on KIP-146 may continue here for as long as the voting process is open. -Konstantine On Mon, May 8, 2017 at 9:47 AM,

Re: [DISCUSS] KIP-155 Add range scan for windowed state stores

2017-05-08 Thread Xavier Léauté
That sounds reasonable, Michal. Given the underlying implementation uses the same segmented bytes store I don't think it should be too difficult to add. On Mon, May 8, 2017 at 2:04 AM Michal Borowiecki < michal.borowie...@openbet.com> wrote: > Hi Xavier, > > I like your KIP. Do you think the

Re: [VOTE] KIP-140: Add administrative RPCs for adding, deleting, and listing ACLs

2017-05-08 Thread Colin McCabe
Quick update: I renamed ListAcls to DescribeAcls, for consistency with DescribeTopics, DescribeGroups, DescribeConfigs, etc. cheers, Colin On Fri, May 5, 2017, at 11:54, Colin McCabe wrote: > Hi all, > > Thanks for all the discussion. I made some changes to the KIP. > > * After some

Re: [VOTE] KIP-133: List and Alter Configs Admin APIs (second attempt)

2017-05-08 Thread Colin McCabe
+1 (non-binding) best, Colin On Mon, May 8, 2017, at 08:52, Ismael Juma wrote: > Quick update, I renamed ListConfigs to DescribeConfigs (and related > classes > and methods) as that is more consistent with other protocols (like > ListGroups and DescribeGroups). So the new link is: > >

Re: [DISCUSS] KIP-133: List and Alter Configs Admin APIs

2017-05-08 Thread Colin McCabe
On Sun, May 7, 2017, at 20:18, Ismael Juma wrote: > Thanks for the feedback Colin. Comments inline. > > On Sun, May 7, 2017 at 9:29 PM, Colin McCabe wrote: > > > > Hmm. What's the behavior if I try to list the configuration for a topic > > that doesn't exist? It seems like

[GitHub] kafka pull request #2997: MINOR: Rename InitPidRequest/InitPidResponse renam...

2017-05-08 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2997 MINOR: Rename InitPidRequest/InitPidResponse renamed to InitProducerIdRequest/InitProducerIdResponse You can merge this pull request into a Git repository by running: $ git pull

[jira] [Commented] (KAFKA-5070) org.apache.kafka.streams.errors.LockException: task [0_18] Failed to lock the state directory: /opt/rocksdb/pulse10/0_18

2017-05-08 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001178#comment-16001178 ] Guozhang Wang commented on KAFKA-5070: -- [~dnagarajan] [~shoyebpathan] The WARN entry in your stack

[GitHub] kafka pull request #2995: KAFKA-3763: Remove deprecated APIs for 0.11.0.0

2017-05-08 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2995 --- 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-3763) Remove deprecated APIs for 0.11.0.0

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

[jira] [Resolved] (KAFKA-3763) Remove deprecated APIs for 0.11.0.0

2017-05-08 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3763?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-3763. Resolution: Fixed Issue resolved by pull request 2995

[jira] [Commented] (KAFKA-5167) streams task gets stuck after re-balance due to LockException

2017-05-08 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5167?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16001150#comment-16001150 ] Matthias J. Sax commented on KAFKA-5167: Btw: "I have some instance filed which I am closing in

Re: [DISCUSS] KIP 141 - ProducerRecordBuilder Interface

2017-05-08 Thread Colin McCabe
Hadoop had a very similar issue in two places: in the constructor of MiniDFSCluster, and with the FileSystem#create API. In both cases, people kept adding more and more function overloads until the APIs got very ugly and hard to understand. This is especially the case when some of the parameters

[jira] [Created] (KAFKA-5196) LogCleaner should be transaction-aware

2017-05-08 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5196: -- Summary: LogCleaner should be transaction-aware Key: KAFKA-5196 URL: https://issues.apache.org/jira/browse/KAFKA-5196 Project: Kafka Issue Type:

[jira] [Work started] (KAFKA-5196) LogCleaner should be transaction-aware

2017-05-08 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5196?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5196 started by Jason Gustafson. -- > LogCleaner should be transaction-aware >

  1   2   >