Re: [DISCUSS] KIP-213 Support non-key joining in KTable

2017-12-12 Thread Jan Filipiak
Hi, I updated the KIP I would be open for this: We mark the "less intrusive" and the "back and forth mapper" approach as rejected alternatives. and implement the two remaining methods. any thoughts? Best jan On 07.12.2017 12:58, Jan Filipiak wrote: On 05.12.2017 00:42, Matthias J. Sax

[jira] [Created] (KAFKA-6355) transient failure in org.apache.kafka.streams.integration.EosIntegrationTest.shouldBeAbleToRunWithTwoSubtopologies

2017-12-12 Thread huxihx (JIRA)
huxihx created KAFKA-6355: - Summary: transient failure in org.apache.kafka.streams.integration.EosIntegrationTest.shouldBeAbleToRunWithTwoSubtopologies Key: KAFKA-6355 URL:

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

2017-12-12 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-6349: Fix concurrent modification exception in -- [...truncated 148.34 KB...] kafka.server.DynamicConfigChangeTest >

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

2017-12-12 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP 145 - Expose Record Headers in Kafka Connect

2017-12-12 Thread Michael André Pearce
*concert = convert Sent from my iPhone > On 13 Dec 2017, at 05:35, Michael André Pearce > wrote: > > Hi Randall > > What’s the main difference between this and my earlier alternative option PR > https://github.com/apache/kafka/pull/2942/files > > If none then

Re: [DISCUSS] KIP 145 - Expose Record Headers in Kafka Connect

2017-12-12 Thread Michael André Pearce
Hi Randall What’s the main difference between this and my earlier alternative option PR https://github.com/apache/kafka/pull/2942/files If none then +1. From what I can tell the only difference I make is the headers you support being able to cross convert primitive types eg if value after

[GitHub] kafka pull request #4317: KAFKA-6349: fix concurrent modification exception ...

2017-12-12 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4317 ---

[GitHub] kafka pull request #4318: MINOR: fix warn logging in RecordCollectorImpl

2017-12-12 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4318 ---

Re: [DISCUSS] KIP-234: add support for getting topic defaults from AdminClient

2017-12-12 Thread Ewen Cheslack-Postava
re: API versions, I actually wasn't sure if we needed it or not. I'm fine if people would prefer just bumping it, but I was actually curious if we could get away without bumping it. I don't know the behavior of the broker code paths for this well enough to know what types of errors those non-null

Re: [VOTE] KIP-210: Provide for custom error handling when Kafka Streams fails to produce

2017-12-12 Thread Matt Farmer
Thank you, Gwen! =) I think that puts us over the finish line. Unless I hear any objections in the next 12(ish) hours I’ll move the KIP to accepted on the wiki. On December 12, 2017 at 7:50:23 PM, Gwen Shapira (g...@confluent.io) wrote: +1 (binding) - looks awesome. On Tue, Dec 12, 2017 at

Re: [DISCUSS] KIP-237: More Controller Health Metrics

2017-12-12 Thread Dong Lin
Hey Ismael, Thanks for your comments. Yeah the set of metrics in the KIP includes the two metrics discussed in the future work in KIP-143. This KIP was also named after KIP-143. I think it may be better to use to use ControllerEventManager instead of ControllerStats. The metrics 1 and 2 are

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

2017-12-12 Thread Hu Xi
Hi all, KIP-223 passed with 3 binding votes and 4 non-binding votes: Binding +1: Jun Rao, Guozhang, Becket Non-binding +1: Manikumar, Ted Yu, Dong Lin, Charly Molter Thanks to everyone who shared feedback and voted on this KIP. huxihx 发件人: Guozhang Wang

Re: [DISCUSS] KIP 145 - Expose Record Headers in Kafka Connect

2017-12-12 Thread Randall Hauch
Trying to revive this after several months of inactivity I've spent quite a bit of time evaluating the current KIP-145 proposal and several of the suggested PRs. The original KIP-145 proposal is relatively minimalist (which is very nice), and it adopts Kafka's approach to headers where header

[GitHub] kafka pull request #4319: [WIP] KAFKA-5142: Add Connect support for message ...

2017-12-12 Thread rhauch
GitHub user rhauch opened a pull request: https://github.com/apache/kafka/pull/4319 [WIP] KAFKA-5142: Add Connect support for message headers (KIP-145) *NEW PROPOSAL FOR KIP-145... DO NOT MERGE* Changed the Connect API and runtime to support message headers as described in

Re: [VOTE] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-12-12 Thread Guozhang Wang
+1 On Mon, Dec 11, 2017 at 7:07 PM, Colin McCabe wrote: > +1 (non-binding) > > P.S. Suggest to use whenComplete instead of making addWaiter public. > > (The differences is very slight : addWaiter returns void, but > whenComplete returns a future which gets completed with

[VOTE] KIP-232: Detect outdated metadata using per-partition leaderEpoch field

2017-12-12 Thread Dong Lin
Hi all, Since there are no more outstanding comments, I would like to start voting thread for KIP-232: https://cwiki.apache.org/confluence/display/KAFKA/KIP-232%3A+Detect+outdated+metadata+using+per-partition+leaderEpoch+field The KIP will fix an issue which can cause consumer to either lose

Re: [DISCUSS] KIP-232: Detect outdated metadata by adding ControllerMetadataEpoch field

2017-12-12 Thread Dong Lin
Hey Jun, I see. Sounds good. Yeah it is probably simpler to leave this to another KIP in the future. Thanks for all the comments. Since there is no further comment in the community, I will open the voting thread. Thanks, Dong On Mon, Dec 11, 2017 at 5:37 PM, Jun Rao wrote:

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

2017-12-12 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-5764; Add toLowerCase support to sasl.kerberos.principal.to.local -- [...truncated 404.80 KB...] kafka.utils.CoreUtilsTest > testReadInt

Re: [VOTE] KIP-210: Provide for custom error handling when Kafka Streams fails to produce

2017-12-12 Thread Gwen Shapira
+1 (binding) - looks awesome. On Tue, Dec 12, 2017 at 10:42 AM Matt Farmer wrote: > Current tally here is 2 binding +1s, 4 non-binding +1s. > > The remaining remarks on the PR seem to mostly be nits, so I feel like > we’ve converged a bit. If a committer could take a look and

[GitHub] kafka pull request #3800: KAFKA-5764: Add toLowerCase support to sasl.kerber...

2017-12-12 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3800 ---

Re: [DISCUSS] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2017-12-12 Thread dan
Jorge, Would you consider also adding `listGroupOffsets` to this KIP? thanks dan On Tue, Dec 12, 2017 at 9:39 AM, Jason Gustafson wrote: > Hi Colin, > > They do share the same namespace. We have a "protocol type" field in the > JoinGroup request to make sure that all

Re: [DISCUSS] KIP-229: DeleteGroups API

2017-12-12 Thread Vahid S Hashemian
The KIP is updated based on the feedback so far. Apologies to Dong for the typo in his name in my earlier message! --Vahid From: "Vahid S Hashemian" To: dev@kafka.apache.org Date: 12/12/2017 02:37 PM Subject:Re: [DISCUSS] KIP-229: DeleteGroups API

Re: [DISCUSS] KIP-229: DeleteGroups API

2017-12-12 Thread Vahid S Hashemian
Thanks for the clarification Jason. Your suggestions make sense. I'll leave individual offset deletion and exposing the functionality in AdminClient out of this KIP. --Vahid From: Jason Gustafson To: dev@kafka.apache.org Date: 12/12/2017 01:46 PM Subject:

Re: [DISCUSS] KIP-234: add support for getting topic defaults from AdminClient

2017-12-12 Thread dan
Colin/Ewen, i will add changes to bump the API version. any preferences on the return type for the new method? tbh it seems like returning a NewTopic could make sense because the ConfigResource for a TOPIC type does not let me encode `numPartitions` thanks dan On Mon, Dec 11, 2017 at 7:22 PM,

[jira] [Created] (KAFKA-6354) Update KStream JavaDoc with regard to KIP-182

2017-12-12 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-6354: -- Summary: Update KStream JavaDoc with regard to KIP-182 Key: KAFKA-6354 URL: https://issues.apache.org/jira/browse/KAFKA-6354 Project: Kafka Issue Type:

[jira] [Created] (KAFKA-6353) Connector status shows FAILED but actually task is in RUNNING status

2017-12-12 Thread Chen He (JIRA)
Chen He created KAFKA-6353: -- Summary: Connector status shows FAILED but actually task is in RUNNING status Key: KAFKA-6353 URL: https://issues.apache.org/jira/browse/KAFKA-6353 Project: Kafka

Re: [DISCUSS] KIP-229: DeleteGroups API

2017-12-12 Thread Jason Gustafson
> > That makes sense. I'll add that functionality in the KIP. This was in response to my suggestion about deleting offsets. After thinking about it some more, it might be better to leave it as potential future work. I wouldn't want to complicate the DeleteGroups API unnecessarily, and the

[jira] [Created] (KAFKA-6352) Delay message down-conversion until response is ready to be sent

2017-12-12 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-6352: -- Summary: Delay message down-conversion until response is ready to be sent Key: KAFKA-6352 URL: https://issues.apache.org/jira/browse/KAFKA-6352 Project: Kafka

Re: [DISCUSS] KIP-229: DeleteGroups API

2017-12-12 Thread Vahid S Hashemian
Hi Jason, Thanks for reviewing the KIP. Please see my responses inline. I'll make the necessary updates to the KIP and send another note once that's done. --Vahid From: Jason Gustafson To: dev@kafka.apache.org Date: 12/11/2017 03:43 PM Subject:Re:

Re: [VOTE] Allowing write access to GitHub repositories (aka GitBox)

2017-12-12 Thread Sriram Subramanian
+1 On Tue, Dec 12, 2017 at 8:22 AM, Manikumar wrote: > +1 > > On Tue, Dec 12, 2017 at 9:49 PM, Rajini Sivaram > wrote: > > > +1 > > > > Thanks, Ismael! > > > > On Tue, Dec 12, 2017 at 4:18 PM, Damian Guy > wrote: > > >

[jira] [Resolved] (KAFKA-5156) Options for handling exceptions in streams

2017-12-12 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5156?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-5156. -- Resolution: Fixed > Options for handling exceptions in streams >

[jira] [Resolved] (KAFKA-5397) streams are not recovering from LockException during rebalancing

2017-12-12 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5397?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-5397. -- Resolution: Fixed > streams are not recovering from LockException during rebalancing >

[GitHub] kafka-site issue #112: Migrate Streams Dev Guide content to AK

2017-12-12 Thread joel-hamill
Github user joel-hamill commented on the issue: https://github.com/apache/kafka-site/pull/112 ping @guozhangwang ---

[GitHub] kafka-site pull request #112: Migrate Streams Dev Guide content to AK

2017-12-12 Thread joel-hamill
GitHub user joel-hamill opened a pull request: https://github.com/apache/kafka-site/pull/112 Migrate Streams Dev Guide content to AK See related https://github.com/apache/kafka/pull/4252 You can merge this pull request into a Git repository by running: $ git pull

Re: [DISCUSS] KIP-227: Introduce Incremental FetchRequests to Increase Partition Scalability

2017-12-12 Thread Becket Qin
Hi Colin, I am not completely sure, but I am hoping that when we do FileChannel.transferTo() the OS will just use a fixed buffer to transfer the data to the socket channel without polluting the page cache. But this might not be true if we are using SSL. The point I want to make is that avoiding

[jira] [Resolved] (KAFKA-2394) Use RollingFileAppender by default in log4j.properties

2017-12-12 Thread Dustin Cote (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2394?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dustin Cote resolved KAFKA-2394. Resolution: Won't Do > Use RollingFileAppender by default in log4j.properties >

[GitHub] kafka pull request #1652: KAFKA-2394: move to RollingFileAppender by default...

2017-12-12 Thread cotedm
Github user cotedm closed the pull request at: https://github.com/apache/kafka/pull/1652 ---

Re: [DISCUSS] KIP-228 Negative record timestamp support

2017-12-12 Thread Dong Lin
Hey Konstantin, Thanks for updating the KIP. If we were to support negative timestamp in the message, we probably also want to support negative timestamp in ListOffsetRequest. Currently in ListOffsetRequest, timestamp value -2 is used to indicate earliest timestamp and timestamp value -1 is used

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

2017-12-12 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-210: Provide for custom error handling when Kafka Streams fails to produce

2017-12-12 Thread Matt Farmer
Current tally here is 2 binding +1s, 4 non-binding +1s. The remaining remarks on the PR seem to mostly be nits, so I feel like we’ve converged a bit. If a committer could take a look and either leave me some feedback on the discussion thread or give me a +1, I’d really appreciate it. :) Thanks!

Re: [DISCUSS] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2017-12-12 Thread Jason Gustafson
Hi Colin, They do share the same namespace. We have a "protocol type" field in the JoinGroup request to make sure that all members are of the same kind. Very roughly what I was thinking is something like this. First we introduce an interface for deserialization: interface

Re: [VOTE] Allowing write access to GitHub repositories (aka GitBox)

2017-12-12 Thread Guozhang Wang
+1 On Tue, Dec 12, 2017 at 9:24 AM, Gwen Shapira wrote: > +1 > > On Tue, Dec 12, 2017 at 9:23 AM Jason Gustafson > wrote: > > > +1 > > > > On Tue, Dec 12, 2017 at 8:22 AM, Manikumar > > wrote: > > > > > +1 > > > > > > On Tue,

Re: [VOTE] Allowing write access to GitHub repositories (aka GitBox)

2017-12-12 Thread Gwen Shapira
+1 On Tue, Dec 12, 2017 at 9:23 AM Jason Gustafson wrote: > +1 > > On Tue, Dec 12, 2017 at 8:22 AM, Manikumar > wrote: > > > +1 > > > > On Tue, Dec 12, 2017 at 9:49 PM, Rajini Sivaram > > > wrote: > > > > > +1 > > > > >

Re: [VOTE] KIP-226 - Dynamic Broker Configuration

2017-12-12 Thread Gwen Shapira
+1 (binding). Thank you for leading this, Rajini. On Tue, Dec 12, 2017 at 8:35 AM Tom Bentley wrote: > +1 (nonbinding) > > On 12 December 2017 at 15:34, Ted Yu wrote: > > > +1 > > > > On Tue, Dec 12, 2017 at 5:44 AM, Rajini Sivaram

Re: [VOTE] Allowing write access to GitHub repositories (aka GitBox)

2017-12-12 Thread Jason Gustafson
+1 On Tue, Dec 12, 2017 at 8:22 AM, Manikumar wrote: > +1 > > On Tue, Dec 12, 2017 at 9:49 PM, Rajini Sivaram > wrote: > > > +1 > > > > Thanks, Ismael! > > > > On Tue, Dec 12, 2017 at 4:18 PM, Damian Guy > wrote: > > >

[jira] [Resolved] (KAFKA-3508) Transient failure in kafka.security.auth.SimpleAclAuthorizerTest.testHighConcurrencyModificationOfResourceAcls

2017-12-12 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3508?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-3508. Resolution: Fixed Marking this back to resolved since the new error is a little different

[DISCUSS] KIP-239 Add queryableStoreName() to GlobalKTable

2017-12-12 Thread Ted Yu
Hi, Please kindly provide feedback on the following KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-239+Add+queryableStoreName%28%29+to+GlobalKTable Thanks

[jira] [Resolved] (KAFKA-5631) Use Jackson for serialising to JSON

2017-12-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5631?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5631. Resolution: Fixed > Use Jackson for serialising to JSON > --- > >

[GitHub] kafka pull request #4259: KAFKA-5631 : Use Jackson for serialising to JSON

2017-12-12 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4259 ---

Re: [VOTE] KIP-226 - Dynamic Broker Configuration

2017-12-12 Thread Tom Bentley
+1 (nonbinding) On 12 December 2017 at 15:34, Ted Yu wrote: > +1 > > On Tue, Dec 12, 2017 at 5:44 AM, Rajini Sivaram > wrote: > > > Since there are no more outstanding comments, I would like to start vote > > for KIP-226: > > > >

Re: [DISCUSS] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-12-12 Thread Xavier Léauté
I'm fine with the whenComplete solution as well. On Tue, Dec 12, 2017 at 03:57 Tom Bentley wrote: > Hi Steven, > > I am happy with adding whenComplete() instead of addWaiter(), > > Cheers, > > Tom > > On 12 December 2017 at 11:11, Steven Aerts

Re: [VOTE] Allowing write access to GitHub repositories (aka GitBox)

2017-12-12 Thread Manikumar
+1 On Tue, Dec 12, 2017 at 9:49 PM, Rajini Sivaram wrote: > +1 > > Thanks, Ismael! > > On Tue, Dec 12, 2017 at 4:18 PM, Damian Guy wrote: > > > +1 > > > > On Tue, 12 Dec 2017 at 15:47 Ismael Juma wrote: > > > > > Hi all, > > >

Re: [VOTE] Allowing write access to GitHub repositories (aka GitBox)

2017-12-12 Thread Rajini Sivaram
+1 Thanks, Ismael! On Tue, Dec 12, 2017 at 4:18 PM, Damian Guy wrote: > +1 > > On Tue, 12 Dec 2017 at 15:47 Ismael Juma wrote: > > > Hi all, > > > > The Apache Infra team has started a new project earlier this year called > > GitBox that supports

Re: [VOTE] Allowing write access to GitHub repositories (aka GitBox)

2017-12-12 Thread Damian Guy
+1 On Tue, 12 Dec 2017 at 15:47 Ismael Juma wrote: > Hi all, > > The Apache Infra team has started a new project earlier this year called > GitBox that supports two-way synchronization between GitHub and > git-wip-us.apache.org and, most importantly, provides GitHub write

[VOTE] Allowing write access to GitHub repositories (aka GitBox)

2017-12-12 Thread Ismael Juma
Hi all, The Apache Infra team has started a new project earlier this year called GitBox that supports two-way synchronization between GitHub and git-wip-us.apache.org and, most importantly, provides GitHub write access to committers. GitBox is not generally available yet, but individual projects

Re: [VOTE] KIP-226 - Dynamic Broker Configuration

2017-12-12 Thread Ted Yu
+1 On Tue, Dec 12, 2017 at 5:44 AM, Rajini Sivaram wrote: > Since there are no more outstanding comments, I would like to start vote > for KIP-226: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 226+-+Dynamic+Broker+Configuration > > > The KIP enables

Re: [DISCUSS] Allowing write access to GitHub repositories

2017-12-12 Thread Ismael Juma
Thanks. Since this seems like a clear win I'll start the vote thread. Any other thoughts are welcome, of course. Ismael On Tue, Dec 12, 2017 at 12:40 PM, Damian Guy wrote: > +1 > > On Tue, 12 Dec 2017 at 10:20 Rajini Sivaram > wrote: > > > Hi

[GitHub] kafka pull request #4318: MINOR: fix warn logging in RecordCollectorImpl

2017-12-12 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/4318 MINOR: fix warn logging in RecordCollectorImpl Fix warn log message in RecordCollectorImpl so it prints the exception message rather than `{}` ### Committer Checklist (excluded from commit

[GitHub] kafka pull request #4317: KAFKA-6349: fix concurrent modification exception ...

2017-12-12 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/4317 KAFKA-6349: fix concurrent modification exception in AbstractStateManager during restore Fixes a `ConcurrentModificationException` in`AbstractStateManager` that is triggered when a `StateStore` is

[jira] [Created] (KAFKA-6351) libs directory has duplicate javassist jars

2017-12-12 Thread pre sto (JIRA)
pre sto created KAFKA-6351: -- Summary: libs directory has duplicate javassist jars Key: KAFKA-6351 URL: https://issues.apache.org/jira/browse/KAFKA-6351 Project: Kafka Issue Type: Bug

[VOTE] KIP-226 - Dynamic Broker Configuration

2017-12-12 Thread Rajini Sivaram
Since there are no more outstanding comments, I would like to start vote for KIP-226: https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration The KIP enables dynamic update of commonly updated broker configuration options to avoid expensive restarts. Thank you,

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

2017-12-12 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-6319: Quote strings stored in JSON configs -- [...truncated 1.91 MB...] org.apache.kafka.connect.runtime.rest.entities.ConnectorTypeTest >

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

2017-12-12 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-6350) File descriptors leak with persistent KeyValueStore

2017-12-12 Thread Alin Gheorghe (JIRA)
Alin Gheorghe created KAFKA-6350: Summary: File descriptors leak with persistent KeyValueStore Key: KAFKA-6350 URL: https://issues.apache.org/jira/browse/KAFKA-6350 Project: Kafka Issue

Re: [DISCUSS] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-12-12 Thread Tom Bentley
Hi Steven, I am happy with adding whenComplete() instead of addWaiter(), Cheers, Tom On 12 December 2017 at 11:11, Steven Aerts wrote: > Xavier, Colin and Tom > > can you line up on this? > I don't really mind which solution is chosen, but I think it needs to be >

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

2017-12-12 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-6319: Quote strings stored in JSON configs -- [...truncated 373.26 KB...] kafka.log.BrokerCompressionTest > testBrokerSideCompression[17] PASSED

Re: [DISCUSS] KIP-218: Make KafkaFuture.Function java 8 lambda compatible

2017-12-12 Thread Steven Aerts
Xavier, Colin and Tom can you line up on this? I don't really mind which solution is chosen, but I think it needs to be done be before I can close the vote. I want to help you with the implementation after a decision has been made. Just let me know. Thanks, Steven Op di 12 dec. 2017 om

[jira] [Created] (KAFKA-6349) ConcurrentModificationException during streams state restoration

2017-12-12 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-6349: - Summary: ConcurrentModificationException during streams state restoration Key: KAFKA-6349 URL: https://issues.apache.org/jira/browse/KAFKA-6349 Project: Kafka

Re: [DISCUSS] Allowing write access to GitHub repositories

2017-12-12 Thread Damian Guy
+1 On Tue, 12 Dec 2017 at 10:20 Rajini Sivaram wrote: > Hi Ismael, > > Thank you for looking into this. Sounds like a good improvement. > > Regards, > > Rajini > > On Mon, Dec 11, 2017 at 6:35 AM, Guozhang Wang wrote: > > > Sounds good to me. > > >

[jira] [Resolved] (KAFKA-6319) kafka-acls regression for comma characters (and maybe other characters as well)

2017-12-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6319?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-6319. Resolution: Fixed > kafka-acls regression for comma characters (and maybe other characters as >

[GitHub] kafka pull request #4303: KAFKA-6319: Quote strings stored in JSON configs

2017-12-12 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4303 ---

Re: [DISCUSS] Allowing write access to GitHub repositories

2017-12-12 Thread Rajini Sivaram
Hi Ismael, Thank you for looking into this. Sounds like a good improvement. Regards, Rajini On Mon, Dec 11, 2017 at 6:35 AM, Guozhang Wang wrote: > Sounds good to me. > > On Sun, Dec 10, 2017 at 10:14 PM, Ismael Juma wrote: > > > Thanks Guozhang. > > >

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

2017-12-12 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-237: More Controller Health Metrics

2017-12-12 Thread Ismael Juma
Thanks for the KIP, Dong. The general idea is good. In fact, two of the three metrics had been listed under future work for KIP-143: "KAFKA-5028 introduced a queue for Controller events. It would be useful to have a gauge for the queue size and a histogram for how long an event waits in the queue

[jira] [Resolved] (KAFKA-6194) Server crash while deleting segments

2017-12-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6194?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-6194. Resolution: Fixed Assignee: Ismael Juma Fix Version/s: 1.1.0 I believe this was

[jira] [Resolved] (KAFKA-6075) Kafka cannot recover after an unclean shutdown on Windows

2017-12-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-6075. Resolution: Fixed Assignee: Ismael Juma Fix Version/s: 1.1.0 I believe this was

[jira] [Resolved] (KAFKA-6322) Error deleting log for topic, all log dirs failed.

2017-12-12 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-6322. Resolution: Fixed Assignee: Ismael Juma I believe this was fixed by KAFKA-6324, please reopen

[GitHub] kafka pull request #4040: KAFKA-6324: Change LogSegment.delete to deleteIfEx...

2017-12-12 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/4040 ---