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

2016-04-11 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: special handling first ever triggered punctuate -- [...truncated 5264 lines...] org.apache.kafka.streams.processor.internals.StreamThreadTest > testMaybeCommit PA

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-11 Thread Magnus Edenhill
2016-04-11 3:01 GMT+02:00 Jun Rao : > Thinking about ApiVersionRequest a bit more. There are quite a few things > special about it. In the ideal case, (1) its version should never change; > The only thing we know of the future is that we dont know anything, we can't think of every possible future

Re: [VOTE] KIP-33 - Add a time based log index

2016-04-11 Thread Becket Qin
Hi Guozhang and Jun, Thanks for the comments. Please see the responses below. Regarding to Guozhang's question #1 and Jun's question #12. I was inserting the time index and offset index entry together mostly for simplicity as Guozhang mentioned. The purpose of using index interval bytes for time

[jira] [Commented] (KAFKA-3160) Kafka LZ4 framing code miscalculates header checksum

2016-04-11 Thread Magnus Edenhill (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15234610#comment-15234610 ] Magnus Edenhill commented on KAFKA-3160: [~ijuma] It needs to go through slow-path

[jira] [Comment Edited] (KAFKA-3160) Kafka LZ4 framing code miscalculates header checksum

2016-04-11 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15234578#comment-15234578 ] Ismael Juma edited comment on KAFKA-3160 at 4/11/16 7:24 AM: -

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

2016-04-11 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-11 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15234770#comment-15234770 ] Michael Noll commented on KAFKA-3511: - Hmm. I'd prefer an API that allows me to write

[jira] [Assigned] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-11 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reassigned KAFKA-3511: --- Assignee: Eno Thereska > Provide built-in aggregators sum() and avg() in Kafka Streams DSL >

[jira] [Comment Edited] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-11 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15234770#comment-15234770 ] Michael Noll edited comment on KAFKA-3511 at 4/11/16 9:05 AM: --

[jira] [Comment Edited] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-11 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15234770#comment-15234770 ] Michael Noll edited comment on KAFKA-3511 at 4/11/16 9:07 AM: --

[jira] [Comment Edited] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-11 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15234770#comment-15234770 ] Michael Noll edited comment on KAFKA-3511 at 4/11/16 9:08 AM: --

[jira] [Assigned] (KAFKA-3143) inconsistent state in ZK when all replicas are dead

2016-04-11 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3143?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reassigned KAFKA-3143: -- Assignee: Ismael Juma > inconsistent state in ZK when all replicas are dead > -

[jira] [Updated] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-11 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-3511: Assignee: (was: Eno Thereska) > Provide built-in aggregators sum() and avg() in Kafka Streams DS

Re: KStream Close Processor

2016-04-11 Thread Matthias J. Sax
What about extending the API with a method beforeClose() that enables the user to flush buffered data? Maybe we can also rename close() to afterClose(), to make the difference clear. At least, we should document when close() is called -- from a user point of view, I would expect that close() allow

[jira] [Created] (KAFKA-3535) Add metrics ability for streams serde components

2016-04-11 Thread Michael Coon (JIRA)
Michael Coon created KAFKA-3535: --- Summary: Add metrics ability for streams serde components Key: KAFKA-3535 URL: https://issues.apache.org/jira/browse/KAFKA-3535 Project: Kafka Issue Type: New

[jira] [Commented] (KAFKA-725) Broker Exception: Attempt to read with a maximum offset less than start offset

2016-04-11 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-725?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15235016#comment-15235016 ] Stig Rohde Døssing commented on KAFKA-725: -- [~guozhang] Okay, returning the error

[jira] [Created] (KAFKA-3536) ReplicaFetcherThread should not log errors when leadership changes

2016-04-11 Thread JIRA
Stig Rohde Døssing created KAFKA-3536: - Summary: ReplicaFetcherThread should not log errors when leadership changes Key: KAFKA-3536 URL: https://issues.apache.org/jira/browse/KAFKA-3536 Project: K

[jira] [Assigned] (KAFKA-3430) Allow users to set key in KTable.toStream() and KStream

2016-04-11 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3430?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck reassigned KAFKA-3430: -- Assignee: Bill Bejeck > Allow users to set key in KTable.toStream() and KStream > -

[jira] [Assigned] (KAFKA-3439) Document possible exception thrown in public APIs

2016-04-11 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3439?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska reassigned KAFKA-3439: --- Assignee: Eno Thereska > Document possible exception thrown in public APIs >

[jira] [Commented] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-11 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15235113#comment-15235113 ] Michael Noll commented on KAFKA-3511: - PS: As [~guozhang] hinted at, we should also ke

[jira] [Comment Edited] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-11 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15235113#comment-15235113 ] Michael Noll edited comment on KAFKA-3511 at 4/11/16 1:54 PM: --

[GitHub] kafka pull request: KAFKA-3439: Added exceptions thrown

2016-04-11 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/1213 KAFKA-3439: Added exceptions thrown You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka KAFKA-3439-throws Alternatively you c

[jira] [Work started] (KAFKA-3439) Document possible exception thrown in public APIs

2016-04-11 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3439?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3439 started by Eno Thereska. --- > Document possible exception thrown in public APIs > -

[jira] [Commented] (KAFKA-3439) Document possible exception thrown in public APIs

2016-04-11 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3439?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15235137#comment-15235137 ] ASF GitHub Bot commented on KAFKA-3439: --- GitHub user enothereska opened a pull reque

Re: [VOTE] KIP-4 Metadata Schema

2016-04-11 Thread Grant Henke
Jun, That makes sense. It is important that we have an accurate view of the metadata in the cluster. Especially if it will be used for the admin tools. I will make the changes to support this and be backwards compatible with v0 requests and see if it can get into this initial patch. I will post a

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-11 Thread Jun Rao
Hi, Magnus, Let me understand your proposal in more details just from the client's perspective. My understanding of your proposal is the following. On plaintext port, the client will send the following bytes in order. ApiVersionRequest, SaslHandshakeRequest, SASL tokens (if SASL is enabled),

[GitHub] kafka pull request: MINOR: Make VerifiableProducer in system tests...

2016-04-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1207 --- 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 enab

Re: [VOTE] KIP-4 Metadata Schema (Round 2)

2016-04-11 Thread Grant Henke
Based on the discussion in the previous vote thread I also would like to include a behavior change to the MetadataResponse. I have update the wiki

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-04-11 Thread Magnus Edenhill
Hey Jun, see inline 2016-04-11 19:19 GMT+02:00 Jun Rao : > Hi, Magnus, > > Let me understand your proposal in more details just from the client's > perspective. My understanding of your proposal is the following. > > On plaintext port, the client will send the following bytes in order. > ApiV

[jira] [Created] (KAFKA-3537) Provide access to low-level Metrics in ProcessorContext

2016-04-11 Thread Michael Coon (JIRA)
Michael Coon created KAFKA-3537: --- Summary: Provide access to low-level Metrics in ProcessorContext Key: KAFKA-3537 URL: https://issues.apache.org/jira/browse/KAFKA-3537 Project: Kafka Issue Typ

[jira] [Commented] (KAFKA-3511) Provide built-in aggregators sum() and avg() in Kafka Streams DSL

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15235723#comment-15235723 ] Guozhang Wang commented on KAFKA-3511: -- One thing about providing built-in operators,

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

2016-04-11 Thread Apache Jenkins Server
See Changes: [cshapi] MINOR: Make VerifiableProducer in system tests lookup security -- [...truncated 4370 lines...] kafka.admin.AdminRackAwareTest > testAssignmentWith2ReplicasRackAwareWith6Pa

[GitHub] kafka pull request: MINOR: fix incorrect exception message

2016-04-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1210 --- 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 enab

[jira] [Updated] (KAFKA-3537) Provide access to low-level Metrics in ProcessorContext

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3537?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3537: - Labels: semantics (was: ) > Provide access to low-level Metrics in ProcessorContext > ---

[jira] [Updated] (KAFKA-3537) Provide access to low-level Metrics in ProcessorContext

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3537?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3537: - Fix Version/s: 0.10.1.0 > Provide access to low-level Metrics in ProcessorContext > --

[GitHub] kafka pull request: KAFKA-3519: Refactor Transformer's transform /...

2016-04-11 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1204 --- 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 enab

[jira] [Commented] (KAFKA-3519) Refactor Transformer templates to return the same strong-typed value.

2016-04-11 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3519?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15235801#comment-15235801 ] ASF GitHub Bot commented on KAFKA-3519: --- Github user asfgit closed the pull request

[jira] [Commented] (KAFKA-3499) byte[] should not be used as Map key nor Set member

2016-04-11 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3499?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15235799#comment-15235799 ] Michael Noll commented on KAFKA-3499: - FWIW, we ran into the same problem when handlin

[jira] [Resolved] (KAFKA-3519) Refactor Transformer templates to return the same strong-typed value.

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3519?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3519. -- Resolution: Fixed Issue resolved by pull request 1204 [https://github.com/apache/kafka/pull/1204

[jira] [Comment Edited] (KAFKA-3499) byte[] should not be used as Map key nor Set member

2016-04-11 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3499?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15235799#comment-15235799 ] Michael Noll edited comment on KAFKA-3499 at 4/11/16 7:35 PM: --

[jira] [Commented] (KAFKA-3522) Consider adding version information into rocksDB storage format

2016-04-11 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3522?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15235814#comment-15235814 ] Michael Noll commented on KAFKA-3522: - [~jkreps] wrote: {quote} I think you could pot

[jira] [Comment Edited] (KAFKA-3522) Consider adding version information into rocksDB storage format

2016-04-11 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3522?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15235814#comment-15235814 ] Michael Noll edited comment on KAFKA-3522 at 4/11/16 7:41 PM: --

[jira] [Created] (KAFKA-3538) Abstract the creation/retrieval of Producer for stream sinks for unit testing

2016-04-11 Thread Michael Coon (JIRA)
Michael Coon created KAFKA-3538: --- Summary: Abstract the creation/retrieval of Producer for stream sinks for unit testing Key: KAFKA-3538 URL: https://issues.apache.org/jira/browse/KAFKA-3538 Project: Ka

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

2016-04-11 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: fix incorrect exception message in KafkaProducer -- [...truncated 4372 lines...] kafka.admin.AdminRackAwareTest > testRackAwareExpansion PASSED kafka.admin.AdminR

[jira] [Created] (KAFKA-3539) KafkaProducer.send() may block even though it returns the Future

2016-04-11 Thread Oleg Zhurakousky (JIRA)
Oleg Zhurakousky created KAFKA-3539: --- Summary: KafkaProducer.send() may block even though it returns the Future Key: KAFKA-3539 URL: https://issues.apache.org/jira/browse/KAFKA-3539 Project: Kafka

[jira] [Comment Edited] (KAFKA-3499) byte[] should not be used as Map key nor Set member

2016-04-11 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3499?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15235799#comment-15235799 ] Michael Noll edited comment on KAFKA-3499 at 4/11/16 8:05 PM: --

[jira] [Created] (KAFKA-3540) KafkaConsumer.close() may block indefinitely

2016-04-11 Thread Oleg Zhurakousky (JIRA)
Oleg Zhurakousky created KAFKA-3540: --- Summary: KafkaConsumer.close() may block indefinitely Key: KAFKA-3540 URL: https://issues.apache.org/jira/browse/KAFKA-3540 Project: Kafka Issue Type:

[jira] [Updated] (KAFKA-3540) KafkaConsumer.close() may block indefinitely

2016-04-11 Thread Oleg Zhurakousky (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3540?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Oleg Zhurakousky updated KAFKA-3540: Description: KafkaConsumer API doc states {code} Close the consumer, waiting indefinitely f

[jira] [Created] (KAFKA-3541) OffsetIndex-Memory Mapped files gets corrupted on a shared drive or encrypted drive

2016-04-11 Thread Sriharsha Chintalapani (JIRA)
Sriharsha Chintalapani created KAFKA-3541: - Summary: OffsetIndex-Memory Mapped files gets corrupted on a shared drive or encrypted drive Key: KAFKA-3541 URL: https://issues.apache.org/jira/browse/KAFKA-354

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

2016-04-11 Thread Apache Jenkins Server
See

[jira] [Updated] (KAFKA-3538) Abstract the creation/retrieval of Producer for stream sinks for unit testing

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3538?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3538: - Labels: semantics (was: ) > Abstract the creation/retrieval of Producer for stream sinks for unit

[jira] [Updated] (KAFKA-3538) Abstract the creation/retrieval of Producer for stream sinks for unit testing

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3538?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3538: - Fix Version/s: 0.10.1.0 > Abstract the creation/retrieval of Producer for stream sinks for unit te

[jira] [Updated] (KAFKA-3337) Extract selector as a separate groupBy operator for KTable aggregations

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3337?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3337: - Fix Version/s: (was: 0.10.1.0) 0.10.0.0 > Extract selector as a separate gr

Re: [VOTE] KIP-4 Metadata Schema (Round 2)

2016-04-11 Thread Guozhang Wang
+1. On Mon, Apr 11, 2016 at 10:42 AM, Grant Henke wrote: > Based on the discussion in the previous vote thread > < > http://search-hadoop.com/m/uyzND1xlaiU10QlYX&subj=+VOTE+KIP+4+Metadata+Schema > > > I also would like to include a behavior change to the MetadataResponse. I > have update the wik

[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-11 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236055#comment-15236055 ] Flavio Junqueira commented on KAFKA-3042: - Thanks for the logs. I still don’t have

[jira] [Comment Edited] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-11 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236055#comment-15236055 ] Flavio Junqueira edited comment on KAFKA-3042 at 4/11/16 9:58 PM: --

[GitHub] kafka pull request: MINOR: Remove unused hadoop version

2016-04-11 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/1214 MINOR: Remove unused hadoop version All dependencies on hadoop were removed with MiniKDC. This removes the left over version entry. You can merge this pull request into a Git repository by runni

[jira] [Created] (KAFKA-3542) Add "repartition (+ join)" operations to streams

2016-04-11 Thread Greg Fodor (JIRA)
Greg Fodor created KAFKA-3542: - Summary: Add "repartition (+ join)" operations to streams Key: KAFKA-3542 URL: https://issues.apache.org/jira/browse/KAFKA-3542 Project: Kafka Issue Type: Improvem

[jira] [Updated] (KAFKA-3543) Allow a variant of transform() which can emit multiple values

2016-04-11 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3543?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Fodor updated KAFKA-3543: -- Summary: Allow a variant of transform() which can emit multiple values (was: Allow a variant of transfo

[jira] [Created] (KAFKA-3543) Allow a variant of transform() which allows emitting multiple values

2016-04-11 Thread Greg Fodor (JIRA)
Greg Fodor created KAFKA-3543: - Summary: Allow a variant of transform() which allows emitting multiple values Key: KAFKA-3543 URL: https://issues.apache.org/jira/browse/KAFKA-3543 Project: Kafka

[jira] [Comment Edited] (KAFKA-3543) Allow a variant of transform() which can emit multiple values

2016-04-11 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3543?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236121#comment-15236121 ] Greg Fodor edited comment on KAFKA-3543 at 4/11/16 10:40 PM: -

[jira] [Updated] (KAFKA-3543) Allow a variant of transform() which can emit multiple values

2016-04-11 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3543?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Greg Fodor updated KAFKA-3543: -- Description: Right now it seems that if you want to apply an arbitrary stateful transformation to a str

[jira] [Commented] (KAFKA-3543) Allow a variant of transform() which can emit multiple values

2016-04-11 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3543?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236121#comment-15236121 ] Greg Fodor commented on KAFKA-3543: --- Also note that in my case the iflatMap() is not suf

[jira] [Created] (KAFKA-3544) Missing topics on startup

2016-04-11 Thread Greg Fodor (JIRA)
Greg Fodor created KAFKA-3544: - Summary: Missing topics on startup Key: KAFKA-3544 URL: https://issues.apache.org/jira/browse/KAFKA-3544 Project: Kafka Issue Type: Bug Components: strea

[jira] [Created] (KAFKA-3545) Generalized Serdes for List/Map

2016-04-11 Thread Greg Fodor (JIRA)
Greg Fodor created KAFKA-3545: - Summary: Generalized Serdes for List/Map Key: KAFKA-3545 URL: https://issues.apache.org/jira/browse/KAFKA-3545 Project: Kafka Issue Type: Improvement Com

[GitHub] kafka pull request: KAFKA-3163: Add time based index to Kafka.

2016-04-11 Thread becketqin
GitHub user becketqin opened a pull request: https://github.com/apache/kafka/pull/1215 KAFKA-3163: Add time based index to Kafka. This is patch for KIP-33. https://cwiki.apache.org/confluence/display/KAFKA/KIP-33+-+Add+a+time+based+log+index You can merge this pull request into

Re: [VOTE] KIP-52: Kafka Connect Control APIs

2016-04-11 Thread Jason Gustafson
Thanks to all who voted. Looks like this KIP passes with +5 (binding), and +1 (non-binding). -Jason On Thu, Apr 7, 2016 at 2:44 AM, Ismael Juma wrote: > +1 (non-binding) > > Ismael > > On Thu, Apr 7, 2016 at 1:25 AM, Jason Gustafson > wrote: > > > Minor note: I've changed the restart API to be

[jira] [Commented] (KAFKA-3163) KIP-33 - Add a time based log index to Kafka

2016-04-11 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3163?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236160#comment-15236160 ] ASF GitHub Bot commented on KAFKA-3163: --- GitHub user becketqin opened a pull request

Merged trunk to 0.10.0

2016-04-11 Thread Gwen Shapira
Hi Kafka Devs, As you remember, I agreed to merge trunk to 0.10.0 branch on weekly basis until the next RC to avoid burdening committers with double commits. I just merged latest trunk into 0.10.0 and pushed to apache. Enjoy. Gwen Shapira

[jira] [Updated] (KAFKA-496) high level producer send should return a response

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-496?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-496: --- Fix Version/s: (was: 0.10.1.0) 0.9.0.0 > high level producer send should return

[jira] [Updated] (KAFKA-2930) Update references to ZooKeeper in the docs

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2930?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2930: Fix Version/s: 0.10.0.0 > Update references to ZooKeeper in the docs > -

[jira] [Updated] (KAFKA-2844) Use different keyTab for client and server in SASL tests

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2844?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2844: Fix Version/s: 0.10.0.0 > Use different keyTab for client and server in SASL tests > ---

[jira] [Updated] (KAFKA-2239) Allow coordinator to aggresively remove group when it is no longer owning the group

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2239?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2239: Fix Version/s: 0.9.0.0 > Allow coordinator to aggresively remove group when it is no longer owning t

[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-11 Thread Robert Christ (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236180#comment-15236180 ] Robert Christ commented on KAFKA-3042: -- The log on disk also stops and 00:39:46,246.

[jira] [Updated] (KAFKA-2672) SendFailedException when new consumer is run with SSL

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2672?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2672: Fix Version/s: 0.10.0.0 > SendFailedException when new consumer is run with SSL > --

[jira] [Updated] (KAFKA-3384) bin scripts may not be portable/POSIX compliant

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3384?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3384: Fix Version/s: 0.10.0.0 > bin scripts may not be portable/POSIX compliant >

[jira] [Updated] (KAFKA-3505) Set curRecord in punctuate() functions

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3505?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3505: Fix Version/s: 0.10.0.0 > Set curRecord in punctuate() functions > -

[jira] [Updated] (KAFKA-3464) Connect security system tests

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3464: Fix Version/s: 0.10.0.0 > Connect security system tests > - > >

[jira] [Updated] (KAFKA-3488) commitAsync() fails if metadata update creates new SASL/SSL connection

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3488?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3488: Fix Version/s: 0.10.0.0 > commitAsync() fails if metadata update creates new SASL/SSL connection > -

[jira] [Updated] (KAFKA-3407) ErrorLoggingCallback trims helpful diagnostic information.

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3407?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3407: Fix Version/s: 0.10.0.0 > ErrorLoggingCallback trims helpful diagnostic information. > -

[jira] [Updated] (KAFKA-3528) AbstractCoordinator wakeup during rebalance can cause spurious rebalances

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3528?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3528: Fix Version/s: 0.10.0.0 > AbstractCoordinator wakeup during rebalance can cause spurious rebalances

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

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3508?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3508: Fix Version/s: 0.10.0.0 > Transient failure in > kafka.security.auth.SimpleAclAuthorizerTest.testHi

[jira] [Updated] (KAFKA-3445) ConnectorConfig should validate TASKS_MAX_CONFIG's lower bound limit

2016-04-11 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3445?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-3445: Fix Version/s: 0.10.0.0 > ConnectorConfig should validate TASKS_MAX_CONFIG's lower bound limit > --

[jira] [Assigned] (KAFKA-3499) byte[] should not be used as Map key nor Set member

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3499?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-3499: Assignee: Guozhang Wang > byte[] should not be used as Map key nor Set member > ---

[jira] [Updated] (KAFKA-3523) Capture org.apache.kafka.clients.consumer.CommitFailedException in UncaughtExceptionHandler

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3523?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3523: - Description: When the sync commit failed due to an ongoing rebalance, it is thrown all the way up

[jira] [Commented] (KAFKA-3523) Capture org.apache.kafka.clients.consumer.CommitFailedException in UncaughtExceptionHandler

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3523?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236290#comment-15236290 ] Guozhang Wang commented on KAFKA-3523: -- Looked at the source code, the `CommitFailedE

Re: KStream Close Processor

2016-04-11 Thread Guozhang Wang
Yeah we can definitely do better in documentation. While regarding the API changes I would prefer to hold and think through if such use cases are common in pattern, and that if we can even re-order the closing process to get around the issue I mentioned above if it is required. Guozhang On Mon, A

[jira] [Commented] (KAFKA-3542) Add "repartition (+ join)" operations to streams

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3542?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236304#comment-15236304 ] Guozhang Wang commented on KAFKA-3542: -- [~gfodor] Kafka Streams already such kind of

[jira] [Commented] (KAFKA-3543) Allow a variant of transform() which can emit multiple values

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3543?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236313#comment-15236313 ] Guozhang Wang commented on KAFKA-3543: -- [~gfodor] We have seen such use cases in othe

Jenkins build is back to normal : kafka-0.10.0-jdk7 #34

2016-04-11 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-3544) Missing topics on startup

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3544?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236324#comment-15236324 ] Guozhang Wang commented on KAFKA-3544: -- Thanks for reporting this. Sounds like a bug

[jira] [Updated] (KAFKA-3544) Missing topics on startup

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3544?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3544: - Labels: semantics (was: ) > Missing topics on startup > - > >

[jira] [Updated] (KAFKA-3545) Generalized Serdes for List/Map

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3545?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3545: - Fix Version/s: 0.10.1.0 > Generalized Serdes for List/Map > --- > >

[jira] [Updated] (KAFKA-3545) Generalized Serdes for List/Map

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3545?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3545: - Labels: api (was: ) > Generalized Serdes for List/Map > --- > >

[jira] [Commented] (KAFKA-3542) Add "repartition (+ join)" operations to streams

2016-04-11 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3542?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236331#comment-15236331 ] Greg Fodor commented on KAFKA-3542: --- I might just not be understanding, but a simple exa

[jira] [Commented] (KAFKA-3545) Generalized Serdes for List/Map

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3545?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236334#comment-15236334 ] Guozhang Wang commented on KAFKA-3545: -- Thanks for reporting. Having generalized serd

[jira] [Work started] (KAFKA-3430) Allow users to set key in KTable.toStream() and KStream

2016-04-11 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3430?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3430 started by Bill Bejeck. -- > Allow users to set key in KTable.toStream() and KStream > -

[jira] [Commented] (KAFKA-3543) Allow a variant of transform() which can emit multiple values

2016-04-11 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3543?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236339#comment-15236339 ] Greg Fodor commented on KAFKA-3543: --- Ah interesting, my assumption was that the items em

[jira] [Commented] (KAFKA-3542) Add "repartition (+ join)" operations to streams

2016-04-11 Thread Greg Fodor (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3542?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236343#comment-15236343 ] Greg Fodor commented on KAFKA-3542: --- Ah, I may understand what you're getting at here --

[jira] [Commented] (KAFKA-3542) Add "repartition (+ join)" operations to streams

2016-04-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3542?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15236347#comment-15236347 ] Guozhang Wang commented on KAFKA-3542: -- [~gfodor] You can use "map" to set the key fo

  1   2   >