[GitHub] kafka pull request #1986: HOTFIX: move restoreConsumer.assign() to shutdownT...

2016-10-07 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/1986 HOTFIX: move restoreConsumer.assign() to shutdownTasksAndState restoreConsumer.assign(..) in removeStandbyTasks was logging an (ignorable) exception due to the restoreConsumer being closed. Moved the

[jira] [Commented] (KAFKA-4263) QueryableStateIntegrationTest.concurrentAccess is failing occasionally in jenkins builds

2016-10-07 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4263?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15554500#comment-15554500 ] Damian Guy commented on KAFKA-4263: --- I've had this running in a loop and it has run 74 t

[jira] [Created] (KAFKA-4265) Intermittent test failure ReplicationQuotasTest.shouldBootstrapTwoBrokersWithFollowerThrottle

2016-10-07 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-4265: - Summary: Intermittent test failure ReplicationQuotasTest.shouldBootstrapTwoBrokersWithFollowerThrottle Key: KAFKA-4265 URL: https://issues.apache.org/jira/browse/KAFKA-4265

[jira] [Created] (KAFKA-4266) Replication Quota Tests: Ensure ZK updated before tests start

2016-10-07 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-4266: --- Summary: Replication Quota Tests: Ensure ZK updated before tests start Key: KAFKA-4266 URL: https://issues.apache.org/jira/browse/KAFKA-4266 Project: Kafka Is

[jira] [Assigned] (KAFKA-4266) Replication Quota Tests: Ensure ZK updated before tests start

2016-10-07 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4266?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-4266: --- Assignee: Ben Stopford > Replication Quota Tests: Ensure ZK updated before tests start >

[GitHub] kafka pull request #1987: MINOR: Test quota initialization when change confi...

2016-10-07 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/1987 MINOR: Test quota initialization when change config znodes are removed You can merge this pull request into a Git repository by running: $ git pull https://github.com/rajinisivaram/kafka

[jira] [Created] (KAFKA-4267) Quota initialization for uses incorrect ZK path

2016-10-07 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-4267: - Summary: Quota initialization for uses incorrect ZK path Key: KAFKA-4267 URL: https://issues.apache.org/jira/browse/KAFKA-4267 Project: Kafka Issue Type:

[jira] [Resolved] (KAFKA-4267) Quota initialization for uses incorrect ZK path

2016-10-07 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4267?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4267. Resolution: Fixed Fix Version/s: 0.10.1.1 Issue resolved by pull request 1987 [https://github

[GitHub] kafka pull request #1987: KAFKA-4267: Fix and test quota config path used fo...

2016-10-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1987 --- 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-4267) Quota initialization for uses incorrect ZK path

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

[jira] [Created] (KAFKA-4268) huge server.logs during the error frequently happen "Message format version for partition 200 not found"

2016-10-07 Thread Peyton Peng (JIRA)
Peyton Peng created KAFKA-4268: -- Summary: huge server.logs during the error frequently happen "Message format version for partition 200 not found" Key: KAFKA-4268 URL: https://issues.apache.org/jira/browse/KAFKA-426

[jira] [Created] (KAFKA-4269) Multiple KStream instances with at least one Regex source causes NPE when using multiple consumers

2016-10-07 Thread Bill Bejeck (JIRA)
Bill Bejeck created KAFKA-4269: -- Summary: Multiple KStream instances with at least one Regex source causes NPE when using multiple consumers Key: KAFKA-4269 URL: https://issues.apache.org/jira/browse/KAFKA-4269

[jira] [Assigned] (KAFKA-4269) Multiple KStream instances with at least one Regex source causes NPE when using multiple consumers

2016-10-07 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck reassigned KAFKA-4269: -- Assignee: Bill Bejeck (was: Guozhang Wang) > Multiple KStream instances with at least one Rege

[jira] [Work started] (KAFKA-4269) Multiple KStream instances with at least one Regex source causes NPE when using multiple consumers

2016-10-07 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4269 started by Bill Bejeck. -- > Multiple KStream instances with at least one Regex source causes NPE when > using

[jira] [Updated] (KAFKA-4269) Multiple KStream instances with at least one Regex source causes NPE when using multiple consumers

2016-10-07 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck updated KAFKA-4269: --- Description: I discovered this issue while doing testing for for KAFKA-4114. KAFKA-4131 fixed the is

[jira] [Created] (KAFKA-4270) ClassCast for Agregation

2016-10-07 Thread Mykola Polonskyi (JIRA)
Mykola Polonskyi created KAFKA-4270: --- Summary: ClassCast for Agregation Key: KAFKA-4270 URL: https://issues.apache.org/jira/browse/KAFKA-4270 Project: Kafka Issue Type: Bug Repo

[jira] [Updated] (KAFKA-4270) ClassCast for Agregation

2016-10-07 Thread Mykola Polonskyi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4270?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mykola Polonskyi updated KAFKA-4270: Component/s: streams > ClassCast for Agregation > > >

[GitHub] kafka pull request #1982: KAFKA-4262: Increase data volume in replication te...

2016-10-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1982 --- 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] [Resolved] (KAFKA-4262) Intermittent unit test failure ReassignPartitionsClusterTest.shouldExecuteThrottledReassignment

2016-10-07 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4262?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4262. Resolution: Fixed Fix Version/s: 0.10.1.1 Issue resolved by pull request 1982 [https://github

[jira] [Commented] (KAFKA-4262) Intermittent unit test failure ReassignPartitionsClusterTest.shouldExecuteThrottledReassignment

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

[GitHub] kafka pull request #1988: Commits the Gradle wrapper jar/properties file

2016-10-07 Thread valdisrigdon
GitHub user valdisrigdon opened a pull request: https://github.com/apache/kafka/pull/1988 Commits the Gradle wrapper jar/properties file Per the Gradle user guide, https://docs.gradle.org/current/userguide/gradle_wrapper.html, it's recommended that these are committed into versi

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

2016-10-07 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4267; Quota initialization for uses incorrect ZK -- [...truncated 14058 lines...] org.apache.kafka.streams.kstream.internals.KeyValuePrinterProcessorTest > testPrint

Jenkins build is back to normal : kafka-0.10.1-jdk7 #51

2016-10-07 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #1989: Kafka 4180 - Shared authentification with multiple...

2016-10-07 Thread edoardocomar
GitHub user edoardocomar opened a pull request: https://github.com/apache/kafka/pull/1989 Kafka 4180 - Shared authentification with multiple actives Kafka producers/consumers This PR builds on top of @rajinisivaram https://github.com/apache/kafka/pull/1979 codeveloped with @mim

[jira] [Commented] (KAFKA-4180) Shared authentification with multiple actives Kafka producers/consumers

2016-10-07 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1389#comment-1389 ] ASF GitHub Bot commented on KAFKA-4180: --- GitHub user edoardocomar opened a pull requ

Re: [DISCUSS] KIP-83 - Allow multiple SASL PLAIN authenticated Java clients in a single JVM process

2016-10-07 Thread Edoardo Comar
The JIRA associated with KIP-83 (though I think I could close that as a KIP) is addressed by https://github.com/apache/kafka/pull/1989 that builds on top of Rajini's PR for KIP-85 -- Edoardo Comar IBM MessageHub eco...@uk.ibm.com IBM UK Ltd, Hurs

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-10-07 Thread Jay Kreps
Hey guys, This discussion has come up a number of times and we've always passed. One of things that has helped keep Kafka simple is not adding in new abstractions and concepts except when the proposal is really elegant and makes things simpler. Consider three use cases for headers: 1. Kafka-

Re: [VOTE] 0.10.1.0 RC0

2016-10-07 Thread Jason Gustafson
@Vahid Thanks, do you want to submit a patch for the quickstart fixes? We won't need another RC if it's just doc changes. The exception is a little more troubling. Perhaps open a JIRA and we can begin investigation? It's especially strange that you say it's specific to the new consumer. @Henry Act

Re: [VOTE] 0.10.1.0 RC0

2016-10-07 Thread Ismael Juma
On Fri, Oct 7, 2016 at 4:56 PM, Jason Gustafson wrote: > @Vahid Thanks, do you want to submit a patch for the quickstart fixes? We > won't need another RC if it's just doc changes. The exception is a little > more troubling. Perhaps open a JIRA and we can begin investigation? It's > especially st

Re: [VOTE] 0.10.1.0 RC0

2016-10-07 Thread Jason Gustafson
> > I suggest not having a "Fix version" set for issues that don't fix anything > (it's not part of any release really). Yeah, good call. On Fri, Oct 7, 2016 at 8:59 AM, Ismael Juma wrote: > On Fri, Oct 7, 2016 at 4:56 PM, Jason Gustafson > wrote: > > > @Vahid Thanks, do you want to submit a

[jira] [Updated] (KAFKA-264) Change the consumer side load balancing and distributed co-ordination to use a consumer co-ordinator

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-264?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-264: -- Fix Version/s: (was: 0.10.1.0) > Change the consumer side load balancing and distributed co-o

[GitHub] kafka pull request #1890: MINOR: Use `hiResClockMs` in `testRequestExpiry` t...

2016-10-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1890 --- 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

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

2016-10-07 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4262; Increase data volume in replication test -- [...truncated 3832 lines...] kafka.coordinator.GroupMetadataManagerTest > testExpireGroup STARTED kafka.coordinator

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-10-07 Thread Joel Koshy
Hi Jay, Couple of comments inline: One of things that has helped keep Kafka simple is not adding in new > abstractions and concepts except when the proposal is really elegant and > makes things simpler. > I don't quite see how this impacts simplicity because (per your taxonomy) the scope is "com

Re: [VOTE] 0.10.1.0 RC0

2016-10-07 Thread Vahid S Hashemian
Jason, Sure, I'll submit a patch for the trivial changes in the quick start. Do you recommend adding Windows version of commands along with the current commands? I'll also open a JIRA for the new consumer issue. --Vahid From: Jason Gustafson To: dev@kafka.apache.org Cc: Kafka User

[jira] [Updated] (KAFKA-1040) ConsumerConfig and ProducerConfig do "work" in the Constructor

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1040?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-1040: --- Fix Version/s: (was: 0.10.1.0) > ConsumerConfig and ProducerConfig do "work" in the Constr

[jira] [Updated] (KAFKA-789) Producer-side persistence for delivery guarantee

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-789?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-789: -- Fix Version/s: (was: 0.10.1.0) > Producer-side persistence for delivery guarantee > -

[jira] [Updated] (KAFKA-1613) Improve system test documentation

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1613?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-1613: --- Fix Version/s: (was: 0.10.1.0) > Improve system test documentation > -

[jira] [Updated] (KAFKA-3801) Provide static serialize() and deserialize() for use as method references

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3801?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-3801: --- Fix Version/s: (was: 0.10.1.0) > Provide static serialize() and deserialize() for use as m

[jira] [Updated] (KAFKA-1324) Debian packaging

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1324?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-1324: --- Fix Version/s: (was: 0.10.1.0) > Debian packaging > > > K

[jira] [Updated] (KAFKA-4074) Deleting a topic can make it unavailable even if delete.topic.enable is false

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4074?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4074: --- Fix Version/s: (was: 0.10.1.0) > Deleting a topic can make it unavailable even if delete.t

[jira] [Updated] (KAFKA-3913) Old consumer's metrics error when using IPv6

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3913?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-3913: --- Fix Version/s: (was: 0.10.1.0) > Old consumer's metrics error when using IPv6 > -

[jira] [Updated] (KAFKA-3183) Add metrics for persistent store caching layer

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-3183: --- Fix Version/s: (was: 0.10.1.0) > Add metrics for persistent store caching layer >

[jira] [Updated] (KAFKA-3101) Optimize Aggregation Outputs

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3101?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-3101: --- Fix Version/s: (was: 0.10.1.0) > Optimize Aggregation Outputs > --

[jira] [Updated] (KAFKA-2901) Extend ListGroups and DescribeGroup APIs to cover offsets

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2901?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2901: --- Fix Version/s: (was: 0.10.1.0) > Extend ListGroups and DescribeGroup APIs to cover offsets

[jira] [Updated] (KAFKA-3759) Incorrect JDBC credentials cause Connect worker to permanently fail

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3759?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-3759: --- Fix Version/s: (was: 0.10.1.0) > Incorrect JDBC credentials cause Connect worker to perman

[jira] [Updated] (KAFKA-3637) Add method that checks if streams are initialised

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-3637: --- Fix Version/s: (was: 0.10.1.0) > Add method that checks if streams are initialised > -

[jira] [Updated] (KAFKA-3873) Gradle Test Executor non-zero exit code when running streams tests

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3873?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-3873: --- Fix Version/s: (was: 0.10.1.0) > Gradle Test Executor non-zero exit code when running stre

[jira] [Updated] (KAFKA-289) reuse topicdata when sending producerrequest

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-289?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-289: -- Fix Version/s: (was: 0.10.1.0) > reuse topicdata when sending producerrequest > -

[jira] [Updated] (KAFKA-478) Move start_consumer & start_producer inside "start_entity_in_background"

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-478?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-478: -- Fix Version/s: (was: 0.10.1.0) > Move start_consumer & start_producer inside "start_entity_in

[jira] [Updated] (KAFKA-564) Wildcard-based topic consumption should assign partitions to threads uniformly

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-564?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-564: -- Fix Version/s: (was: 0.10.1.0) > Wildcard-based topic consumption should assign partitions to

[jira] [Updated] (KAFKA-288) java impacted changes from new producer and consumer request format

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-288?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-288: -- Fix Version/s: (was: 0.10.1.0) > java impacted changes from new producer and consumer request

[jira] [Updated] (KAFKA-558) KafkaETLContext should use getTopicMetadata before sending offset requests

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-558?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-558: -- Fix Version/s: (was: 0.10.1.0) > KafkaETLContext should use getTopicMetadata before sending o

[jira] [Updated] (KAFKA-1589) Strengthen System Tests

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1589?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-1589: --- Fix Version/s: (was: 0.10.1.0) > Strengthen System Tests > --- > >

[jira] [Updated] (KAFKA-3511) Add common aggregation functions like Sum and Avg as build-ins in Kafka Streams DSL

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3511?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-3511: --- Fix Version/s: (was: 0.10.1.0) > Add common aggregation functions like Sum and Avg as buil

[jira] [Updated] (KAFKA-777) Add system tests for important tools

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-777?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-777: -- Fix Version/s: (was: 0.10.1.0) > Add system tests for important tools > -

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

2016-10-07 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4267; Quota initialization for uses incorrect ZK -- [...truncated 12106 lines...] org.apache.kafka.clients.producer.internals.SenderTest > testSendInOrder PASSED o

[jira] [Updated] (KAFKA-2658) Implement SASL/PLAIN

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2658?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2658: --- Fix Version/s: (was: 0.10.0.0) (was: 0.10.1.0) > Implement SASL/PLA

[jira] [Updated] (KAFKA-1554) Corrupt index found on clean startup

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1554?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-1554: --- Fix Version/s: (was: 0.10.0.0) (was: 0.10.1.0) > Corrupt index foun

[jira] [Updated] (KAFKA-1592) Some INFO level logging needs to be DEBUG

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-1592: --- Fix Version/s: (was: 0.10.1.0) > Some INFO level logging needs to be DEBUG > -

[jira] [Updated] (KAFKA-1255) Offset in RecordMetadata is Incorrect with New Producer Ack = -1

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1255?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-1255: --- Fix Version/s: (was: 0.10.1.0) > Offset in RecordMetadata is Incorrect with New Producer A

Build failed in Jenkins: kafka-0.10.1-jdk7 #52

2016-10-07 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4262; Increase data volume in replication test -- [...truncated 3832 lines...] kafka.coordinator.GroupCoordinatorResponseTest > testSyncGroupLeaderAfterFollower STAR

[jira] [Updated] (KAFKA-1183) DefaultEventHandler causes unbalanced distribution of messages across partitions

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1183?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-1183: --- Fix Version/s: (was: 0.10.1.0) (was: 0.8.1) > DefaultEventHandler c

[jira] [Updated] (KAFKA-1629) Replica fetcher thread need to back off upon getting errors on partitions

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1629?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-1629: --- Fix Version/s: (was: 0.10.1.0) > Replica fetcher thread need to back off upon getting erro

[jira] [Commented] (KAFKA-4217) KStream.transform equivalent of flatMap

2016-10-07 Thread Elias Levy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1654#comment-1654 ] Elias Levy commented on KAFKA-4217: --- It would seem to be the same request: allow a trans

[jira] [Commented] (KAFKA-4257) Inconsistencies in 0.10.1 upgrade docs

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4257?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1692#comment-1692 ] Jason Gustafson commented on KAFKA-4257: [~jeff.klu...@gmail.com] Can we resolve t

[GitHub] kafka pull request #1990: MINOR: Update Quickstart in documentation to accou...

2016-10-07 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/1990 MINOR: Update Quickstart in documentation to account for Windows platforms You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian

Build failed in Jenkins: kafka-0.10.1-jdk7 #53

2016-10-07 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Use `hiResClockMs` in `testRequestExpiry` to fix transient test -- [...truncated 1120 lines...] kafka.utils.timer.TimerTest > testTaskExpiration PASSED kafka.utils.ti

[GitHub] kafka pull request #1991: KAFKA-4265: Run replication quotas test with produ...

2016-10-07 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/1991 KAFKA-4265: Run replication quotas test with producer acks=1 Test expects all records to be published successfully, which cannot be guaranteed with acks=0 since failures are not retried. You

[jira] [Commented] (KAFKA-4265) Intermittent test failure ReplicationQuotasTest.shouldBootstrapTwoBrokersWithFollowerThrottle

2016-10-07 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4265?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1779#comment-1779 ] ASF GitHub Bot commented on KAFKA-4265: --- GitHub user rajinisivaram opened a pull req

[jira] [Updated] (KAFKA-4265) Intermittent test failure ReplicationQuotasTest.shouldBootstrapTwoBrokersWithFollowerThrottle

2016-10-07 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4265?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-4265: -- Description: https://builds.apache.org/job/kafka-trunk-git-pr-jdk7/6085/testReport/junit/kafka.s

[jira] [Resolved] (KAFKA-4257) Inconsistencies in 0.10.1 upgrade docs

2016-10-07 Thread Jeff Klukas (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4257?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jeff Klukas resolved KAFKA-4257. Resolution: Fixed Ismael's PR addresses these questions. > Inconsistencies in 0.10.1 upgrade docs

[jira] [Created] (KAFKA-4271) The console consumer fails on Windows with new consumer is used

2016-10-07 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-4271: -- Summary: The console consumer fails on Windows with new consumer is used Key: KAFKA-4271 URL: https://issues.apache.org/jira/browse/KAFKA-4271 Project: Kafka

[GitHub] kafka pull request #1986: HOTFIX: move restoreConsumer.assign() to shutdownT...

2016-10-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1986 --- 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-4272) Kafka Connect batch scripts are missing under `bin/windows/`

2016-10-07 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-4272: -- Summary: Kafka Connect batch scripts are missing under `bin/windows/` Key: KAFKA-4272 URL: https://issues.apache.org/jira/browse/KAFKA-4272 Project: Kafka

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

2016-10-07 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-80: Kafka REST Server

2016-10-07 Thread Harsha Ch
Ofir, " personally think it would be quite wasteful to re-implement the REST gateway just because that an actively-maintained piece of Apache-licensed software is not governed directly by the Apache Kafka community... While kafka-rest repo is owned by Confluent, the contributors including the m

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

2016-10-07 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on jenkins-test-39e (Ubuntu ubuntu jenkins-cloud-8GB jenkins-cloud-4GB cloud-slave) in workspace

Build failed in Jenkins: kafka-0.10.1-jdk7 #54

2016-10-07 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on jenkins-test-39e (Ubuntu ubuntu jenkins-cloud-8GB jenkins-cloud-4GB cloud-slave) in workspace

Re: [DISCUSS] KIP-80: Kafka REST Server

2016-10-07 Thread Harsha Chintalapani
Ofir, … " personally think it would be quite wasteful to re-implement the REST gateway just because that an actively-maintained piece of Apache-licensed software is not governed directly by the Apache Kafka community... While kafka-rest repo is owned by Confluent, the contributors including the

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

2016-10-07 Thread Apache Jenkins Server
See -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on jenkins-test-39e (Ubuntu ubuntu jenkins-cloud-8GB jenkins-cloud-4GB cloud-slave) in workspace

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

2016-10-07 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Use `hiResClockMs` in `testRequestExpiry` to fix transient test -- [...truncated 14017 lines...] org.apache.kafka.streams.state.internals.CompositeReadOnlyKeyValueSto

Re: [DISCUSS] KIP-80: Kafka REST Server

2016-10-07 Thread Neha Narkhede
Harsha/Mani, I completely agree that adding admin API support and security are important features for the Kafka REST proxy. Luckily the roadmap items that you mentioned as being important for a Kafka REST proxy server are exactly the ones the community working on this REST proxy want to add to it

[jira] [Created] (KAFKA-4273) Streams DSL - Add TTL / retention period support for intermediate topics and state stores

2016-10-07 Thread Davor Poldrugo (JIRA)
Davor Poldrugo created KAFKA-4273: - Summary: Streams DSL - Add TTL / retention period support for intermediate topics and state stores Key: KAFKA-4273 URL: https://issues.apache.org/jira/browse/KAFKA-4273

[jira] [Updated] (KAFKA-4273) Streams DSL - Add TTL / retention period support for intermediate topics and state stores

2016-10-07 Thread Davor Poldrugo (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4273?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Davor Poldrugo updated KAFKA-4273: -- Description: Hi! I'm using Streams DSL (0.10.0.1), which can only use RocksDB for local state as

[jira] [Updated] (KAFKA-4273) Streams DSL - Add TTL / retention period support for intermediate topics and state stores

2016-10-07 Thread Davor Poldrugo (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4273?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Davor Poldrugo updated KAFKA-4273: -- Description: Hi! I'm using Streams DSL (0.10.0.1), which can only use RocksDB for local state as

[GitHub] kafka pull request #1940: HOTFIX: recreate state.dir after cleanup

2016-10-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1940 --- 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: [DISCUSS] KIP-82 - Add Record Headers

2016-10-07 Thread Michael Pearce
Hi Jay, Thanks for the comments and feedback. I think its quite clear that if a problem keeps arising then it is clear that it needs resolving, and addressing properly. Fair enough at linkedIn, and historically for the very first use cases addressing this maybe not have been a big priority. B

[jira] [Created] (KAFKA-4274) KafkaConsumer.offsetsForTimes() hangs and times out on an empty map

2016-10-07 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-4274: -- Summary: KafkaConsumer.offsetsForTimes() hangs and times out on an empty map Key: KAFKA-4274 URL: https://issues.apache.org/jira/browse/KAFKA-4274 Project: Kafka Issue

[jira] [Commented] (KAFKA-4274) KafkaConsumer.offsetsForTimes() hangs and times out on an empty map

2016-10-07 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4274?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15556217#comment-15556217 ] Jun Rao commented on KAFKA-4274: [~becket_qin], do you want to fix this? Thanks. > KafkaC

[jira] [Commented] (KAFKA-4271) The console consumer fails on Windows with new consumer is used

2016-10-07 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4271?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15556219#comment-15556219 ] Vahid Hashemian commented on KAFKA-4271: The command {{bin\windows\kafka-run-class

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

2016-10-07 Thread Apache Jenkins Server
See

[jira] [Updated] (KAFKA-4274) KafkaConsumer.offsetsForTimes() hangs and times out on an empty map

2016-10-07 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4274?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4274: --- Fix Version/s: 0.10.1.0 > KafkaConsumer.offsetsForTimes() hangs and times out on an empty map

[jira] [Created] (KAFKA-4275) Check of State-Store-assignment to Processor-Nodes is not enabled

2016-10-07 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-4275: -- Summary: Check of State-Store-assignment to Processor-Nodes is not enabled Key: KAFKA-4275 URL: https://issues.apache.org/jira/browse/KAFKA-4275 Project: Kafka

[jira] [Work started] (KAFKA-4275) Check of State-Store-assignment to Processor-Nodes is not enabled

2016-10-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4275?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4275 started by Matthias J. Sax. -- > Check of State-Store-assignment to Processor-Nodes is not enabled > ---

Re: [DISCUSS] KIP-80: Kafka REST Server

2016-10-07 Thread Suresh Srinivas
ASF already gives us a clear framework and governance model for community development. This is already understood by the people contributing to Apache Kafka project, and they are the same people who want to contribute to the REST server capability as well. Everyone is in agreement on the need for c

Jenkins build is back to normal : kafka-0.10.1-jdk7 #55

2016-10-07 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-4217) KStream.transform equivalent of flatMap

2016-10-07 Thread Matthias J. Sax (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15556349#comment-15556349 ] Matthias J. Sax commented on KAFKA-4217: Would it not be simpler to change `proces

[jira] [Updated] (KAFKA-4117) Cleanup StreamPartitionAssignor behavior

2016-10-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4117?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4117: - Description: I went through the whole assignment logic once again and I feel the logic has now be

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

2016-10-07 Thread Apache Jenkins Server
See Changes: [wangguoz] HOTFIX: move restoreConsumer.assign() to shutdownTasksAndState [wangguoz] HOTFIX: recreate state.dir after cleanup -- [...truncated 5086 lines...] kafka.utils.Replicatio

  1   2   >