Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-01 Thread Matthias J. Sax
First a meta comment. KIP discussion should take place on the dev list -- if user list is cc'ed please make sure to reply to both lists. Thanks. Thanks for making the scope of the KIP clear. Makes a lot of sense to focus on deserialization exceptions for now. With regard to corrupted state stores

[jira] [Updated] (KAFKA-5283) Update clients and server code to make sure that epoch and sequence numbers wrap around

2017-06-01 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5283?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-5283: --- Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull requ

[jira] [Commented] (KAFKA-5283) Update clients and server code to make sure that epoch and sequence numbers wrap around

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

[GitHub] kafka pull request #3183: KAFKA-5283: Handle producer epoch/sequence overflo...

2017-06-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3183 --- 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] [Assigned] (KAFKA-4370) CorruptRecordException when ProducerRecord constructed without key nor partition and send

2017-06-01 Thread Mariam John (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4370?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mariam John reassigned KAFKA-4370: -- Assignee: Mariam John > CorruptRecordException when ProducerRecord constructed without key nor

Re: [DISCUSS] KIP-147: Add missing type parameters to StateStoreSupplier factories and KGroupedStream/Table methods

2017-06-01 Thread Matthias J. Sax
Thanks for the update Michal. I did skip over the PR. Looks good to me, as far as I can tell. Maybe Damian, Xavier, or Ismael can comment on this. Would be good to get confirmation that the change is backward compatible. -Matthias On 5/27/17 11:11 AM, Michal Borowiecki wrote: > Hi all, > > I'

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

2017-06-01 Thread Apache Jenkins Server
See

[jira] [Updated] (KAFKA-5322) Resolve AddPartitions response error code inconsistency

2017-06-01 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta updated KAFKA-5322: Status: Patch Available (was: In Progress) > Resolve AddPartitions response error code inconsistenc

[jira] [Commented] (KAFKA-5322) Resolve AddPartitions response error code inconsistency

2017-06-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5322?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16034177#comment-16034177 ] ASF GitHub Bot commented on KAFKA-5322: --- GitHub user apurvam opened a pull request:

[GitHub] kafka pull request #3204: KAFKA-5322: Add an `OPERATION_NOT_ATTEMPTED` error...

2017-06-01 Thread apurvam
GitHub user apurvam opened a pull request: https://github.com/apache/kafka/pull/3204 KAFKA-5322: Add an `OPERATION_NOT_ATTEMPTED` error code In the `AddPartitionsToTxn` request handling, if even one partition fails authorization checks, the entire request is essentially failed. How

Re: [DISCUSS] KIP-165: Extend Interactive Queries for return latest update timestamp per key

2017-06-01 Thread Matthias J. Sax
Thanks for the KIP Jeyhun. Some comments: - why do you only consider get() and not range() and all() ? - we cannot have a second get() (this would be ambiguous) but need another name like getWithTs() (or something better) - what use case do you have in mind for getKeyTs() ? Would a single new m

Re: KIP-160 - Augment KStream.print() to allow users pass in extra parameters in the printed string

2017-06-01 Thread Matthias J. Sax
Thanks for the KIP. Two comments: - I think we should include #writeAsText() - I am not sure if we should use > "[" + this.streamName + "]: " + mapper.apply(keyToPrint, valueToPrint) in case a mapper is provided. This still dictates a fixed prefix a user might not want to have (what contradict

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-01 Thread Matthias J. Sax
I agree with Jeyhun. As already mention, the overall API improvement ideas are overlapping and/or contradicting each other. For this reason, not all ideas can be accomplished and some Jira might just be closed as "won't fix". For this reason, we try to do those KIP discussion with are large scope

Re: [DISCUSS] KIP-167: Add a restoreAll method to StateRestoreCallback

2017-06-01 Thread Matthias J. Sax
Thanks for the KIP Bill. With regard to backward compatibility, we should not change the current interface, but add a new interface that extends the current one. If we are going to add "begin" and "after", we might also consider to add some intermediate call backs. This would allow an application

[jira] [Updated] (KAFKA-5363) Add restoreAll functionality to StateRestoreCallback

2017-06-01 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-5363: --- Description: Add a new method {{restoreAll(List> records)}} to the {{StateRestoreCallback}} t

[jira] [Updated] (KAFKA-5363) Add restoreAll functionality to StateRestoreCallback

2017-06-01 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-5363: --- Labels: kip (was: ) > Add restoreAll functionality to StateRestoreCallback >

Re: Reg: [VOTE] KIP 157 - Add consumer config options to streams reset tool

2017-06-01 Thread Matthias J. Sax
+1 Thanks for updating the KIP! -Matthias On 6/1/17 6:18 PM, Bill Bejeck wrote: > +1 > > Thanks, > Bill > > On Thu, Jun 1, 2017 at 7:45 PM, Guozhang Wang wrote: > >> +1 again. Thanks. >> >> On Tue, May 30, 2017 at 1:46 PM, BigData dev >> wrote: >> >>> Hi All, >>> Updated the KIP, as the con

[jira] [Assigned] (KAFKA-5359) Exceptions from RequestFuture lack parts of the stack trace

2017-06-01 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5359?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian reassigned KAFKA-5359: -- Assignee: Vahid Hashemian > Exceptions from RequestFuture lack parts of the stack trace

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

2017-06-01 Thread Apache Jenkins Server
See

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

2017-06-01 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Add upgrade notes for KAFKA-2358 -- [...truncated 4.06 MB...] kafka.integration.PrimitiveApiTest > testMultiProduce PASSED kafka.integratio

[jira] [Commented] (KAFKA-2289) KafkaProducer logs erroneous warning on startup

2017-06-01 Thread Bharat Viswanadham (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16034055#comment-16034055 ] Bharat Viswanadham commented on KAFKA-2289: --- [~hgschmie] Would you like to conti

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

2017-06-01 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Add upgrade notes for KAFKA-2358 -- [...truncated 2.32 MB...] org.apache.kafka.streams.integration.QueryableStateIntegrationTest > shouldN

[jira] [Commented] (KAFKA-1595) Remove deprecated and slower scala JSON parser from kafka.consumer.TopicCount

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1595?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16034042#comment-16034042 ] Ismael Juma commented on KAFKA-1595: [~onurkaraman], I had trouble getting people to r

[GitHub] kafka pull request #3185: MINOR: Logging/debugging improvements for transact...

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

[GitHub] kafka pull request #83: KAFKA-1595; Remove deprecated and slower scala JSON ...

2017-06-01 Thread ijuma
GitHub user ijuma reopened a pull request: https://github.com/apache/kafka/pull/83 KAFKA-1595; Remove deprecated and slower scala JSON parser Tested that we only use Jackson methods introduced in 2.0 in the main codebase by compiling it with the older version locally. We use a const

[jira] [Commented] (KAFKA-1595) Remove deprecated and slower scala JSON parser from kafka.consumer.TopicCount

2017-06-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1595?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16034038#comment-16034038 ] ASF GitHub Bot commented on KAFKA-1595: --- GitHub user ijuma reopened a pull request:

[jira] [Commented] (KAFKA-5319) Add a tool to make cluster replica and leader balance

2017-06-01 Thread Ma Tianchi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5319?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16034037#comment-16034037 ] Ma Tianchi commented on KAFKA-5319: --- I create a KIP about it. Comment welcome.(https://

[jira] [Commented] (KAFKA-5332) When resize the index file, maybe caused the content disappear

2017-06-01 Thread xuzq (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5332?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16034005#comment-16034005 ] xuzq commented on KAFKA-5332: - [~junrao] when you are idle, please check this bug, maybe it wi

[jira] [Comment Edited] (KAFKA-5332) When resize the index file, maybe caused the content disappear

2017-06-01 Thread xuzq (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5332?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16034005#comment-16034005 ] xuzq edited comment on KAFKA-5332 at 6/2/17 1:36 AM: - [~junrao] when y

[jira] [Comment Edited] (KAFKA-5332) When resize the index file, maybe caused the content disappear

2017-06-01 Thread xuzq (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5332?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16034005#comment-16034005 ] xuzq edited comment on KAFKA-5332 at 6/2/17 1:34 AM: - [~junrao] when y

[jira] [Resolved] (KAFKA-2358) Cluster collection returning methods should never return null

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2358?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-2358. Resolution: Fixed > Cluster collection returning methods should never return null >

[jira] [Commented] (KAFKA-2358) Cluster collection returning methods should never return null

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

[GitHub] kafka pull request #3200: MINOR: Add upgrade notes for KAFKA-2358

2017-06-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3200 --- 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: Reg: [VOTE] KIP 157 - Add consumer config options to streams reset tool

2017-06-01 Thread Bill Bejeck
+1 Thanks, Bill On Thu, Jun 1, 2017 at 7:45 PM, Guozhang Wang wrote: > +1 again. Thanks. > > On Tue, May 30, 2017 at 1:46 PM, BigData dev > wrote: > > > Hi All, > > Updated the KIP, as the consumer configurations are required for both > Admin > > Client and Consumer in Stream reset tool. Updat

Re: [DISCUSS] KIP-167: Add a restoreAll method to StateRestoreCallback

2017-06-01 Thread Bill Bejeck
Sure thing. I'll update the KIP. Thanks, Bill On Thu, Jun 1, 2017 at 6:20 PM, Guozhang Wang wrote: > There are also some request to add "begin" and "after" callbacks in the > restoration func: > > https://issues.apache.org/jira/browse/KAFKA-4322 > > Could we piggy back them into the same KIP? >

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-01 Thread Jeyhun Karimov
Hi Guozhang, It subsumes partially. Initially the idea was to support RichFunctions as a separate interface. Throughout the discussion, however, we considered maybe overloading the related methods (with RecodContext param) is better approach than providing a separate RichFunction interface. Cheer

[GitHub] kafka pull request #3203: KAFKA-5365 [WIP]: Fix regression in compressed mes...

2017-06-01 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/3203 KAFKA-5365 [WIP]: Fix regression in compressed message iteration affecting magic v0 and v1 You can merge this pull request into a Git repository by running: $ git pull https://github.com/ha

[jira] [Commented] (KAFKA-5365) Fix regression in compressed message iteration affecting magic v0 and v1

2017-06-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5365?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16033989#comment-16033989 ] ASF GitHub Bot commented on KAFKA-5365: --- GitHub user hachikuji opened a pull request

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

2017-06-01 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: update Streams docs for KIP-123 -- [...truncated 907.87 KB...] kafka.server.CreateTopicsRequestWithPolicyTest > testValidCreateTopicsRequest

[jira] [Work started] (KAFKA-5322) Resolve AddPartitions response error code inconsistency

2017-06-01 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5322 started by Apurva Mehta. --- > Resolve AddPartitions response error code inconsistency > ---

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-01 Thread Guozhang Wang
Does this KIP subsume this ticket as well? https://issues.apache.org/jira/browse/KAFKA-4125 On Sat, May 20, 2017 at 9:05 AM, Jeyhun Karimov wrote: > Dear community, > > As we discussed in KIP-149 [DISCUSS] thread [1], I would like to initiate > KIP for rich functions (interfaces) [2]. > I would

[jira] [Updated] (KAFKA-5364) Producer attempts to send transactional messages before adding partitions to transaction

2017-06-01 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5364?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta updated KAFKA-5364: Status: Patch Available (was: In Progress) > Producer attempts to send transactional messages befor

[jira] [Commented] (KAFKA-5364) Producer attempts to send transactional messages before adding partitions to transaction

2017-06-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5364?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16033956#comment-16033956 ] ASF GitHub Bot commented on KAFKA-5364: --- GitHub user apurvam opened a pull request:

[GitHub] kafka pull request #3202: KAFKA-5364: Don't fail producer if drained partiti...

2017-06-01 Thread apurvam
GitHub user apurvam opened a pull request: https://github.com/apache/kafka/pull/3202 KAFKA-5364: Don't fail producer if drained partition is not yet in transaction Due to the async nature of the producer, it is possible to attempt to drain a messages whose partition hasn't been add

[jira] [Work started] (KAFKA-5364) Producer attempts to send transactional messages before adding partitions to transaction

2017-06-01 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5364?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-5364 started by Apurva Mehta. --- > Producer attempts to send transactional messages before adding partitions to > t

Re: Reg: [VOTE] KIP 157 - Add consumer config options to streams reset tool

2017-06-01 Thread Guozhang Wang
+1 again. Thanks. On Tue, May 30, 2017 at 1:46 PM, BigData dev wrote: > Hi All, > Updated the KIP, as the consumer configurations are required for both Admin > Client and Consumer in Stream reset tool. Updated the KIP to use > command-config option, similar to other tools like kafka-consumer-gro

[jira] [Created] (KAFKA-5365) Fix regression in compressed message iteration affecting magic v0 and v1

2017-06-01 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5365: -- Summary: Fix regression in compressed message iteration affecting magic v0 and v1 Key: KAFKA-5365 URL: https://issues.apache.org/jira/browse/KAFKA-5365 Project: K

[jira] [Commented] (KAFKA-5362) Add EOS system tests for Streams API

2017-06-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5362?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16033909#comment-16033909 ] ASF GitHub Bot commented on KAFKA-5362: --- GitHub user mjsax opened a pull request:

[GitHub] kafka pull request #3201: KAFKA-5362: Add EOS system tests for Streams API

2017-06-01 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/3201 KAFKA-5362: Add EOS system tests for Streams API You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka kafka-5362-add-eos-system-tests-for-

[GitHub] kafka-site pull request #48: Filled in og meta tags

2017-06-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka-site/pull/48 --- 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 e

[GitHub] kafka-site issue #48: Filled in og meta tags

2017-06-01 Thread guozhangwang
Github user guozhangwang commented on the issue: https://github.com/apache/kafka-site/pull/48 LGTM. Merged to asf-site. --- 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 an

[GitHub] kafka-site issue #59: Add auto syntax highlighter

2017-06-01 Thread guozhangwang
Github user guozhangwang commented on the issue: https://github.com/apache/kafka-site/pull/59 LGTM. Merged to asf-site. --- 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 an

[GitHub] kafka-site pull request #59: Add auto syntax highlighter

2017-06-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka-site/pull/59 --- 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 e

[jira] [Commented] (KAFKA-2358) Cluster collection returning methods should never return null

2017-06-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2358?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16033836#comment-16033836 ] ASF GitHub Bot commented on KAFKA-2358: --- GitHub user guozhangwang opened a pull requ

[GitHub] kafka pull request #3200: MINOR: Add upgrade notes for KAFKA-2358

2017-06-01 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/3200 MINOR: Add upgrade notes for KAFKA-2358 You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka KMinor-KAFKA-2358-upgrade-notes

[GitHub] kafka pull request #3194: MINOR: update Streams docs for KIP-123

2017-06-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3194 --- 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] [Created] (KAFKA-5364) Producer attempts to send transactional messages before adding partitions to transaction

2017-06-01 Thread Apurva Mehta (JIRA)
Apurva Mehta created KAFKA-5364: --- Summary: Producer attempts to send transactional messages before adding partitions to transaction Key: KAFKA-5364 URL: https://issues.apache.org/jira/browse/KAFKA-5364

Re: [DISCUSS] KIP-167: Add a restoreAll method to StateRestoreCallback

2017-06-01 Thread Guozhang Wang
There are also some request to add "begin" and "after" callbacks in the restoration func: https://issues.apache.org/jira/browse/KAFKA-4322 Could we piggy back them into the same KIP? Guozhang On Thu, Jun 1, 2017 at 2:04 PM, Bill Bejeck wrote: > All, > > I'd like to start the discussion for a

[jira] [Commented] (KAFKA-5311) Support ExtendedDeserializer in Kafka Streams

2017-06-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16033793#comment-16033793 ] ASF GitHub Bot commented on KAFKA-5311: --- GitHub user subnova opened a pull request:

[GitHub] kafka pull request #3199: KAFKA-5311: Support ExtendedDeserializer in Kafka ...

2017-06-01 Thread subnova
GitHub user subnova opened a pull request: https://github.com/apache/kafka/pull/3199 KAFKA-5311: Support ExtendedDeserializer in Kafka Streams. The contribution is my original work and I license the work to the project under the project's open source license. This is a clon

[DISCUSS] KIP-167: Add a restoreAll method to StateRestoreCallback

2017-06-01 Thread Bill Bejeck
All, I'd like to start the discussion for adding bulk add functionality when restoring a state store. The KIP can be found here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-167%3A+Add+a+restoreAll+method+to+StateRestoreCallback Thanks, Bill

[jira] [Updated] (KAFKA-5164) SetSchemaMetadata does not replace the schemas in structs correctly

2017-06-01 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5164?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch updated KAFKA-5164: - Status: Patch Available (was: Open) > SetSchemaMetadata does not replace the schemas in structs c

[jira] [Commented] (KAFKA-5164) SetSchemaMetadata does not replace the schemas in structs correctly

2017-06-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5164?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16033687#comment-16033687 ] ASF GitHub Bot commented on KAFKA-5164: --- GitHub user rhauch opened a pull request:

[jira] [Assigned] (KAFKA-5164) SetSchemaMetadata does not replace the schemas in structs correctly

2017-06-01 Thread Randall Hauch (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5164?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Randall Hauch reassigned KAFKA-5164: Assignee: Randall Hauch > SetSchemaMetadata does not replace the schemas in structs correct

[GitHub] kafka pull request #3198: KAFKA-5164 Ensure SetSchemaMetadata updates key or...

2017-06-01 Thread rhauch
GitHub user rhauch opened a pull request: https://github.com/apache/kafka/pull/3198 KAFKA-5164 Ensure SetSchemaMetadata updates key or value when Schema changes When the `SetSchemaMetadata` SMT is used to change the name and/or version of the key or value’s schema, any references

[jira] [Commented] (KAFKA-5345) Some socket connections not closed after restart of Kafka Streams

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

[jira] [Resolved] (KAFKA-5345) Some socket connections not closed after restart of Kafka Streams

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5345. Resolution: Fixed Fix Version/s: 0.11.0.0 Issue resolved by pull request 3195 [https://github

[GitHub] kafka pull request #3195: KAFKA-5345: Close KafkaClient when streams client ...

2017-06-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3195 --- 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-5355) Broker returns messages beyond "latest stable offset" to transactional consumer in read_committed mode

2017-06-01 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5355?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax updated KAFKA-5355: --- Description: This issue is exposed by the new Streams EOS integration test. Streams has two t

[jira] [Commented] (KAFKA-5355) Broker returns messages beyond "latest stable offset" to transactional consumer in read_committed mode

2017-06-01 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16033666#comment-16033666 ] Matthias J. Sax commented on KAFKA-5355: The Streams test revealing this issue is

[jira] [Commented] (KAFKA-5229) Reflections logs excessive warnings when scanning classpaths

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

[GitHub] kafka pull request #3072: KAFKA-5229:Reflections logs excessive warnings whe...

2017-06-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3072 --- 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-5229) Reflections logs excessive warnings when scanning classpaths

2017-06-01 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5229?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-5229: - Resolution: Fixed Fix Version/s: 0.11.1.0 0.11.0.0

[GitHub] kafka pull request #3197: MINOR: Set log level for org.reflections to ERROR.

2017-06-01 Thread kkonstantine
Github user kkonstantine closed the pull request at: https://github.com/apache/kafka/pull/3197 --- 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 i

[GitHub] kafka pull request #3189: MINOR: add upgrade not for group.initial.rebalance...

2017-06-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3189 --- 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-5363) Add restoreAll functionality to StateRestoreCallback

2017-06-01 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck updated KAFKA-5363: --- Description: Add a new method {{restoreAll(List> records)}} to the {{StateRestoreCallback}} to enable

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

2017-06-01 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5154?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16033636#comment-16033636 ] Guozhang Wang commented on KAFKA-5154: -- [~Lukas Gemela] We have found the root cause,

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

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

[GitHub] kafka pull request #3197: MINOR: Set log level for org.reflections to ERROR.

2017-06-01 Thread kkonstantine
GitHub user kkonstantine opened a pull request: https://github.com/apache/kafka/pull/3197 MINOR: Set log level for org.reflections to ERROR. You can merge this pull request into a Git repository by running: $ git pull https://github.com/kkonstantine/kafka MINOR-Set-log-level-

[GitHub] kafka pull request #3181: KAFKA-5154: Consumer fetches from revoked partitio...

2017-06-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3181 --- 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-5363) Add restoreAll functionality to StateRestoreCallback

2017-06-01 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck updated KAFKA-5363: --- Description: Add a new method {{restoreAll(KeyValue records)}} to the {{StateRestoreCallback}} to enab

[jira] [Commented] (KAFKA-5293) Do not apply exponential backoff if users have overridden reconnect.backoff.ms

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

[GitHub] kafka pull request #3174: KAFKA-5293. Do not apply exponential backoff if us...

2017-06-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3174 --- 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-5293) Do not apply exponential backoff if users have overridden reconnect.backoff.ms

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5293?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-5293: --- Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request 3174

[jira] [Commented] (KAFKA-5361) Add EOS integration tests for Streams API

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

[jira] [Resolved] (KAFKA-5361) Add EOS integration tests for Streams API

2017-06-01 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5361?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-5361. -- Resolution: Fixed Issue resolved by pull request 3193 [https://github.com/apache/kafka/pull/3193

[GitHub] kafka pull request #3193: KAFKA-5361: Add EOS integration tests for Streams ...

2017-06-01 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3193 --- 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-5363) Add restoreAll functionality to StateRestoreCallback

2017-06-01 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck updated KAFKA-5363: --- Description: Add a new method {{restoreAll(KeyValue records)}} to the {{StateRestoreCallback}} to enab

[jira] [Created] (KAFKA-5363) Add restoreAll functionality to StateRestoreCallback

2017-06-01 Thread Bill Bejeck (JIRA)
Bill Bejeck created KAFKA-5363: -- Summary: Add restoreAll functionality to StateRestoreCallback Key: KAFKA-5363 URL: https://issues.apache.org/jira/browse/KAFKA-5363 Project: Kafka Issue Type: Im

[jira] [Commented] (KAFKA-5330) Use per-task converters in Connect

2017-06-01 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5330?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16033578#comment-16033578 ] ASF GitHub Bot commented on KAFKA-5330: --- GitHub user tbcdns opened a pull request:

[GitHub] kafka pull request #3196: KAFKA-5330: Use per-task converters

2017-06-01 Thread tbcdns
GitHub user tbcdns opened a pull request: https://github.com/apache/kafka/pull/3196 KAFKA-5330: Use per-task converters Instead of sharing the same converter instance within the worker, use a converter per task. More details: - https://github.com/confluentinc/schema-reg

[jira] [Updated] (KAFKA-5204) Connect needs to validate Connector type during instantiation

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5204?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-5204: --- Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Connect needs to validate Connector t

[jira] [Comment Edited] (KAFKA-5054) ChangeLoggingKeyValueByteStore delete and putIfAbsent should be synchronized

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5054?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16033577#comment-16033577 ] Ismael Juma edited comment on KAFKA-5054 at 6/1/17 8:00 PM: Th

[jira] [Updated] (KAFKA-5054) ChangeLoggingKeyValueByteStore delete and putIfAbsent should be synchronized

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5054?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-5054: --- Priority: Critical (was: Major) > ChangeLoggingKeyValueByteStore delete and putIfAbsent should be syn

[jira] [Updated] (KAFKA-4928) Add integration test for DumpLogSegments

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4928: --- Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Add integration test for DumpLogSegme

[jira] [Updated] (KAFKA-5018) LogCleaner tests to verify behaviour of message format v2

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5018?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-5018: --- Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > LogCleaner tests to verify behaviour

[jira] [Updated] (KAFKA-4850) RocksDb cannot use Bloom Filters

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4850: --- Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > RocksDb cannot use Bloom Filters > --

[jira] [Commented] (KAFKA-5054) ChangeLoggingKeyValueByteStore delete and putIfAbsent should be synchronized

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5054?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16033577#comment-16033577 ] Ismael Juma commented on KAFKA-5054: This seems important and simple, @guozhangwang, c

[jira] [Updated] (KAFKA-4785) Records from internal repartitioning topics should always use RecordMetadataTimestampExtractor

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4785?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4785: --- Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Records from internal repartitioning

[jira] [Updated] (KAFKA-4665) Inconsistent handling of non-existing topics in offset fetch handling

2017-06-01 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4665?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4665: --- Fix Version/s: (was: 0.11.0.0) 0.11.1.0 > Inconsistent handling of non-existing

  1   2   >