[jira] [Commented] (KAFKA-4557) ConcurrentModificationException in KafkaProducer event loop

2017-01-17 Thread Sergey Alaev (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15825654#comment-15825654 ] Sergey Alaev commented on KAFKA-4557: - No, I only use KafkaProducer.send, calling it c

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-17 Thread Eno Thereska
Thanks Damian, answers inline: > On 16 Jan 2017, at 17:17, Damian Guy wrote: > > Hi Eno, > > Thanks for the KIP. Some comments: > > 1. I'd probably rename materialized to materialize. Ok. > 2. I don't think the addition of the new Log compaction mechanism is > necessary for this KIP, i

[jira] [Created] (KAFKA-4640) Improve Streams unit test coverage

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4640: - Summary: Improve Streams unit test coverage Key: KAFKA-4640 URL: https://issues.apache.org/jira/browse/KAFKA-4640 Project: Kafka Issue Type: Bug Componen

[jira] [Created] (KAFKA-4641) Improve test coverage of StreamsThread

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4641: - Summary: Improve test coverage of StreamsThread Key: KAFKA-4641 URL: https://issues.apache.org/jira/browse/KAFKA-4641 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-4642) Improve test coverage of ProcessorStateManager

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4642: - Summary: Improve test coverage of ProcessorStateManager Key: KAFKA-4642 URL: https://issues.apache.org/jira/browse/KAFKA-4642 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-4643) Improve test coverage of StreamsKafkaClient

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4643: - Summary: Improve test coverage of StreamsKafkaClient Key: KAFKA-4643 URL: https://issues.apache.org/jira/browse/KAFKA-4643 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-4644) Improve test coverage of StreamsPartitionAssignor

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4644: - Summary: Improve test coverage of StreamsPartitionAssignor Key: KAFKA-4644 URL: https://issues.apache.org/jira/browse/KAFKA-4644 Project: Kafka Issue Type: Sub-tas

[jira] [Created] (KAFKA-4645) Improve test coverage of ProcessorTopology

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4645: - Summary: Improve test coverage of ProcessorTopology Key: KAFKA-4645 URL: https://issues.apache.org/jira/browse/KAFKA-4645 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-4646) Improve Streams unit test coverage

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4646: - Summary: Improve Streams unit test coverage Key: KAFKA-4646 URL: https://issues.apache.org/jira/browse/KAFKA-4646 Project: Kafka Issue Type: Sub-task R

[jira] [Resolved] (KAFKA-4646) Improve Streams unit test coverage

2017-01-17 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4646?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy resolved KAFKA-4646. --- Resolution: Duplicate > Improve Streams unit test coverage > -- > >

[jira] [Updated] (KAFKA-4646) Improve test coverage AbstractProcessorContext

2017-01-17 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4646?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4646: -- Summary: Improve test coverage AbstractProcessorContext (was: Improve Streams unit test coverage) > Im

[jira] [Reopened] (KAFKA-4646) Improve test coverage AbstractProcessorContext

2017-01-17 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4646?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy reopened KAFKA-4646: --- Exception paths in {{register}}, {{topic}}, {{partition}}, {{offset}}, {{timestamp}} > Improve test cove

[jira] [Updated] (KAFKA-4647) Improve test coverage of GlobalStreamThread

2017-01-17 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4647?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4647: -- Priority: Minor (was: Major) > Improve test coverage of GlobalStreamThread > --

[jira] [Created] (KAFKA-4647) Improve test coverage of GlobalStreamThread

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4647: - Summary: Improve test coverage of GlobalStreamThread Key: KAFKA-4647 URL: https://issues.apache.org/jira/browse/KAFKA-4647 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-4648) Improve test coverage StreamTask

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4648: - Summary: Improve test coverage StreamTask Key: KAFKA-4648 URL: https://issues.apache.org/jira/browse/KAFKA-4648 Project: Kafka Issue Type: Sub-task Rep

[jira] [Created] (KAFKA-4649) Improve test coverage GlobalStateManagerImpl

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4649: - Summary: Improve test coverage GlobalStateManagerImpl Key: KAFKA-4649 URL: https://issues.apache.org/jira/browse/KAFKA-4649 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-4650) Improve test coverage org.apache.kafka.streams.kstream.internals

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4650: - Summary: Improve test coverage org.apache.kafka.streams.kstream.internals Key: KAFKA-4650 URL: https://issues.apache.org/jira/browse/KAFKA-4650 Project: Kafka Iss

[jira] [Created] (KAFKA-4651) Improve test coverage of Stores

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4651: - Summary: Improve test coverage of Stores Key: KAFKA-4651 URL: https://issues.apache.org/jira/browse/KAFKA-4651 Project: Kafka Issue Type: Sub-task Repo

[jira] [Created] (KAFKA-4652) Improve test coverage KStreamBuilder

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4652: - Summary: Improve test coverage KStreamBuilder Key: KAFKA-4652 URL: https://issues.apache.org/jira/browse/KAFKA-4652 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-4653) Improve test coverage of RocksDBStore

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4653: - Summary: Improve test coverage of RocksDBStore Key: KAFKA-4653 URL: https://issues.apache.org/jira/browse/KAFKA-4653 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-4654) Improve test coverage MemoryLRUCache

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4654: - Summary: Improve test coverage MemoryLRUCache Key: KAFKA-4654 URL: https://issues.apache.org/jira/browse/KAFKA-4654 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-4655) Improve test coverage of CompositeReadOnlySessionStore

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4655: - Summary: Improve test coverage of CompositeReadOnlySessionStore Key: KAFKA-4655 URL: https://issues.apache.org/jira/browse/KAFKA-4655 Project: Kafka Issue Type: Su

[jira] [Created] (KAFKA-4656) Improve test coverage of CompositeReadOnlyKeyValueStore

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4656: - Summary: Improve test coverage of CompositeReadOnlyKeyValueStore Key: KAFKA-4656 URL: https://issues.apache.org/jira/browse/KAFKA-4656 Project: Kafka Issue Type: S

[jira] [Created] (KAFKA-4657) Improve test coverage of CompositeReadOnlyWindowStore

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4657: - Summary: Improve test coverage of CompositeReadOnlyWindowStore Key: KAFKA-4657 URL: https://issues.apache.org/jira/browse/KAFKA-4657 Project: Kafka Issue Type: Sub

[jira] [Created] (KAFKA-4658) Improve test coverage InMemoryKeyValueLoggedStore

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4658: - Summary: Improve test coverage InMemoryKeyValueLoggedStore Key: KAFKA-4658 URL: https://issues.apache.org/jira/browse/KAFKA-4658 Project: Kafka Issue Type: Sub-tas

[jira] [Created] (KAFKA-4659) Improve test coverage of CachingKeyValueStore

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4659: - Summary: Improve test coverage of CachingKeyValueStore Key: KAFKA-4659 URL: https://issues.apache.org/jira/browse/KAFKA-4659 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-4660) Improve test coverage KafkaStreams

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4660: - Summary: Improve test coverage KafkaStreams Key: KAFKA-4660 URL: https://issues.apache.org/jira/browse/KAFKA-4660 Project: Kafka Issue Type: Sub-task R

[jira] [Created] (KAFKA-4661) Improve test coverage UsePreviousTimeOnInvalidTimestamp

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4661: - Summary: Improve test coverage UsePreviousTimeOnInvalidTimestamp Key: KAFKA-4661 URL: https://issues.apache.org/jira/browse/KAFKA-4661 Project: Kafka Issue Type: S

[jira] [Created] (KAFKA-4662) Improve test coverage TopologyBuilder

2017-01-17 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-4662: - Summary: Improve test coverage TopologyBuilder Key: KAFKA-4662 URL: https://issues.apache.org/jira/browse/KAFKA-4662 Project: Kafka Issue Type: Sub-task

[jira] [Updated] (KAFKA-4662) Improve test coverage TopologyBuilder

2017-01-17 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4662?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4662: -- Description: overloaded {{addSource}} methods with {{AutoOffsetReset}} param not tested. Also some excep

[GitHub] kafka pull request #2316: KAFKA-4363: Documentation for sasl.jaas.config pro...

2017-01-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2316 --- 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-4363) Add document for dynamic JAAS configuration property sasl.jaas.config

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

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-17 Thread Michael Noll
I think section "Compatibility, Deprecation, and Migration Plan" needs updating. It currently reads "No impact on existing users" but the KIP proposes to remove existing API methods (such as `KTable#foreach()`). -Michael On Tue, Jan 17, 2017 at 10:55 AM, Eno Thereska wrote: > Thanks Damian, a

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-17 Thread Michael Noll
Thanks for the KIP, Eno. In section "Rejected Alternatives" the KIP says: > It is not clear that collapsing 2 abstractions helps. In particular, a KTable models a changelog. > That itself is a useful abstraction. A state store is a materialized view. That’s a distinct abstraction > with parallels

[jira] [Resolved] (KAFKA-4363) Add document for dynamic JAAS configuration property sasl.jaas.config

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4363?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4363. Resolution: Fixed Issue resolved by pull request 2316 [https://github.com/apache/kafka/pull/2316] >

[GitHub] kafka pull request #2355: KAFKA-4590: SASL/SCRAM system tests

2017-01-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2355 --- 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-4590) Add system test for SASL/SCRAM

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4590?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4590. Resolution: Fixed Issue resolved by pull request 2355 [https://github.com/apache/kafka/pull/2355] >

[jira] [Commented] (KAFKA-4590) Add system test for SASL/SCRAM

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

[jira] [Updated] (KAFKA-4457) Add a command to list the broker version information

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4457?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4457: --- Reviewer: Ismael Juma > Add a command to list the broker version information > ---

[jira] [Updated] (KAFKA-4547) Consumer.position returns incorrect results for Kafka 0.10.1.0 client

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4547?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4547: --- Reviewer: Jason Gustafson > Consumer.position returns incorrect results for Kafka 0.10.1.0 client > --

[jira] [Updated] (KAFKA-4589) Add documentation for SASL/SCRAM

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4589?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4589: --- Reviewer: Jun Rao > Add documentation for SASL/SCRAM > > >

[jira] [Updated] (KAFKA-4630) Implement RecordTooLargeException when communicating with pre-KIP-74 brokers

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4630?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4630: --- Reviewer: Ismael Juma > Implement RecordTooLargeException when communicating with pre-KIP-74 brokers >

[jira] [Updated] (KAFKA-4580) Change one SASL system test to use new JAAS config property

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4580?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4580: --- Reviewer: Ismael Juma > Change one SASL system test to use new JAAS config property >

[jira] [Updated] (KAFKA-4595) Controller send thread can't stop when broker change listener event trigger for dead brokers

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4595?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4595: --- Reviewer: Jun Rao > Controller send thread can't stop when broker change listener event trigger > for

[jira] [Updated] (KAFKA-4576) Log segments close to max size break on fetch

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4576?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4576: --- Reviewer: Ismael Juma > Log segments close to max size break on fetch > --

[jira] [Commented] (KAFKA-1352) Reduce logging on the server

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1352?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826019#comment-15826019 ] Ismael Juma commented on KAFKA-1352: Is this still relevant? If so and if anyone is in

[jira] [Updated] (KAFKA-1352) Reduce logging on the server

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1352?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-1352: --- Fix Version/s: (was: 0.10.2.0) > Reduce logging on the server > > >

[jira] [Commented] (KAFKA-4617) gradle-generated core eclipse project has incorrect source folder structure

2017-01-17 Thread Dhwani Katagade (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4617?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826017#comment-15826017 ] Dhwani Katagade commented on KAFKA-4617: [~ecomar] thanks for your comments. I tri

[jira] [Updated] (KAFKA-1300) Added WaitForReplaction admin tool.

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1300?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-1300: --- Fix Version/s: (was: 0.10.2.0) > Added WaitForReplaction admin tool. > ---

[jira] [Commented] (KAFKA-1206) allow Kafka to start from a resource negotiator system

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1206?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826025#comment-15826025 ] Ismael Juma commented on KAFKA-1206: No activity in a long time, removing fix version.

[jira] [Commented] (KAFKA-1300) Added WaitForReplaction admin tool.

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1300?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826023#comment-15826023 ] Ismael Juma commented on KAFKA-1300: No conclusion to the discussion, so removing the

[jira] [Commented] (KAFKA-1207) Launch Kafka from within Apache Mesos

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1207?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826024#comment-15826024 ] Ismael Juma commented on KAFKA-1207: No activity in a long time, removing fix version.

[jira] [Updated] (KAFKA-1207) Launch Kafka from within Apache Mesos

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1207?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-1207: --- Fix Version/s: (was: 0.10.2.0) > Launch Kafka from within Apache Mesos > -

[jira] [Updated] (KAFKA-1206) allow Kafka to start from a resource negotiator system

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-1206: --- Fix Version/s: (was: 0.10.2.0) > allow Kafka to start from a resource negotiator system >

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

2017-01-17 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4363; Documentation for sasl.jaas.config property -- [...truncated 4073 lines...] kafka.log.LogTest > testReadWithTooSmallMaxLength STARTED kafka.log.LogTest > test

[jira] [Assigned] (KAFKA-4662) Improve test coverage TopologyBuilder

2017-01-17 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4662?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bill Bejeck reassigned KAFKA-4662: -- Assignee: Bill Bejeck > Improve test coverage TopologyBuilder > ---

[jira] [Commented] (KAFKA-4662) Improve test coverage TopologyBuilder

2017-01-17 Thread Bill Bejeck (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4662?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826055#comment-15826055 ] Bill Bejeck commented on KAFKA-4662: I missed these with my PR, picking this one up.

[jira] [Commented] (KAFKA-3705) Support non-key joining in KTable

2017-01-17 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3705?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826064#comment-15826064 ] Michael Noll commented on KAFKA-3705: - [~jfilipiak]: Now that support for global KTabl

[jira] [Comment Edited] (KAFKA-3705) Support non-key joining in KTable

2017-01-17 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3705?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826064#comment-15826064 ] Michael Noll edited comment on KAFKA-3705 at 1/17/17 1:48 PM: --

Re: [DISCUSS] KIP-114: KTable materialization and improved semantics

2017-01-17 Thread Michael Noll
> Rename toStream() to toKStream() for consistency. Not sure whether that is really required. We also use `KStreamBuilder#stream()` and `KStreamBuilder#table()`, for example, and don't care about the "K" prefix. On Tue, Jan 17, 2017 at 10:55 AM, Eno Thereska wrote: > Thanks Damian, answers in

[jira] [Commented] (KAFKA-4514) Add Codec for ZStandard Compression

2017-01-17 Thread Thomas Graves (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826105#comment-15826105 ] Thomas Graves commented on KAFKA-4514: -- just for reference, the KIP: https://cwiki.a

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

2017-01-17 Thread Apache Jenkins Server
See

[jira] [Created] (KAFKA-4663) Update documentation for Authorizer

2017-01-17 Thread Joseph Aliase (JIRA)
Joseph Aliase created KAFKA-4663: Summary: Update documentation for Authorizer Key: KAFKA-4663 URL: https://issues.apache.org/jira/browse/KAFKA-4663 Project: Kafka Issue Type: Improvement

[jira] [Commented] (KAFKA-4663) Update documentation for Authorizer

2017-01-17 Thread Joseph Aliase (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4663?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826314#comment-15826314 ] Joseph Aliase commented on KAFKA-4663: -- ijuma Kindly take a look on this issue > Upd

[jira] [Comment Edited] (KAFKA-4663) Update documentation for Authorizer

2017-01-17 Thread Joseph Aliase (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4663?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826314#comment-15826314 ] Joseph Aliase edited comment on KAFKA-4663 at 1/17/17 4:11 PM: -

[jira] [Commented] (KAFKA-3705) Support non-key joining in KTable

2017-01-17 Thread Jan Filipiak (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3705?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826331#comment-15826331 ] Jan Filipiak commented on KAFKA-3705: - Hi, with regard of what I am trying here, the G

[jira] [Commented] (KAFKA-3705) Support non-key joining in KTable

2017-01-17 Thread Jan Filipiak (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3705?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826332#comment-15826332 ] Jan Filipiak commented on KAFKA-3705: - Hi, with regard of what I am trying here, the G

[jira] [Issue Comment Deleted] (KAFKA-3705) Support non-key joining in KTable

2017-01-17 Thread Jan Filipiak (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3705?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jan Filipiak updated KAFKA-3705: Comment: was deleted (was: Hi, with regard of what I am trying here, the GlobalKTable is not usefull

[jira] [Commented] (KAFKA-4663) Update documentation for Authorizer

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4663?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826335#comment-15826335 ] Ismael Juma commented on KAFKA-4663: It does need to be thread-safe. Would you like to

[jira] [Commented] (KAFKA-3705) Support non-key joining in KTable

2017-01-17 Thread Jan Filipiak (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3705?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826346#comment-15826346 ] Jan Filipiak commented on KAFKA-3705: - AFAICS the table will be also filled by a diffe

[jira] [Commented] (KAFKA-1207) Launch Kafka from within Apache Mesos

2017-01-17 Thread postmas...@inn.ru (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1207?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826416#comment-15826416 ] postmas...@inn.ru commented on KAFKA-1207: -- Delivery is delayed to these recipien

[jira] [Commented] (KAFKA-1207) Launch Kafka from within Apache Mesos

2017-01-17 Thread postmas...@inn.ru (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1207?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826415#comment-15826415 ] postmas...@inn.ru commented on KAFKA-1207: -- Delivery is delayed to these recipien

Re: [VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-01-17 Thread Dong Lin
Hey Jun, Do you have time to review the KIP again or vote for it? Hey Ewen, Can you also review the KIP again or vote for it? I have discussed with Radai and Becket regarding your concern. We still think putting it in Admin Client seems more intuitive because there is use-case where application

Re: [VOTE] KIP-107 Add purgeDataBefore() API in AdminClient

2017-01-17 Thread Becket Qin
+1. Thanks for the KIP. On Thu, Jan 12, 2017 at 10:33 AM, Joel Koshy wrote: > +1 > > (for the record, I favor the rejected alternative of not awaiting low > watermarks to go past the purge offset. I realize it offers a weaker > guarantee but it is still very useful, easier to implement, slightly

[jira] [Commented] (KAFKA-3838) Bump zkclient and Zookeeper versions

2017-01-17 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3838?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826573#comment-15826573 ] James Cheng commented on KAFKA-3838: This JIRA moves ZKClient from 0.7 to 0.9. Can ZKC

[GitHub] kafka pull request #2383: MINOR: Some cleanups and additional testing for KI...

2017-01-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2383 --- 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 #2323: KAFKA-4580: Use sasl.jaas.config for some system t...

2017-01-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2323 --- 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-4580) Change one SASL system test to use new JAAS config property

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

[jira] [Resolved] (KAFKA-4580) Change one SASL system test to use new JAAS config property

2017-01-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4580?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4580. Resolution: Fixed Issue resolved by pull request 2323 [https://github.com/apache/kafka/pull/2323] >

[GitHub] kafka pull request #2386: Upgrade Kafka and apply windows fix

2017-01-17 Thread silpamittapalli
GitHub user silpamittapalli opened a pull request: https://github.com/apache/kafka/pull/2386 Upgrade Kafka and apply windows fix Based on https://github.com/fluetm/kafka/tree/kafka-upgrade. 1. Cherry-picked windows fix for https://issues.apache.org/jira/browse/KAFKA-1194 fr

[jira] [Commented] (KAFKA-1194) The kafka broker cannot delete the old log files after the configured time

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

[jira] [Created] (KAFKA-4664) Update docs/protocol.html with KIP-97 information

2017-01-17 Thread Colin P. McCabe (JIRA)
Colin P. McCabe created KAFKA-4664: -- Summary: Update docs/protocol.html with KIP-97 information Key: KAFKA-4664 URL: https://issues.apache.org/jira/browse/KAFKA-4664 Project: Kafka Issue Typ

[jira] [Resolved] (KAFKA-4456) Offsets of deleted topics are not removed from consumer groups

2017-01-17 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4456?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-4456. Resolution: Duplicate This looks like a duplicate of KAFKA-2000, which has a patch available

[GitHub] kafka pull request #2387: KAFKA-4664: Update docs/protocol.html with KIP-97 ...

2017-01-17 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/2387 KAFKA-4664: Update docs/protocol.html with KIP-97 information You can merge this pull request into a Git repository by running: $ git pull https://github.com/cmccabe/kafka KAFKA-4664 Alternat

[jira] [Commented] (KAFKA-4664) Update docs/protocol.html with KIP-97 information

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

[jira] [Commented] (KAFKA-4664) Update docs/protocol.html with KIP-97 information

2017-01-17 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4664?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826636#comment-15826636 ] Colin P. McCabe commented on KAFKA-4664: https://github.com/apache/kafka/pull/2387

[jira] [Work started] (KAFKA-4664) Update docs/protocol.html with KIP-97 information

2017-01-17 Thread Colin P. McCabe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4664?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4664 started by Colin P. McCabe. -- > Update docs/protocol.html with KIP-97 information > ---

[GitHub] kafka pull request #2386: [WIP] Upgrade Kafka and apply windows fix

2017-01-17 Thread silpamittapalli
Github user silpamittapalli closed the pull request at: https://github.com/apache/kafka/pull/2386 --- 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 featur

[jira] [Commented] (KAFKA-1194) The kafka broker cannot delete the old log files after the configured time

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

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

2017-01-17 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4665: -- Summary: Inconsistent handling of non-existing topics in offset fetch handling Key: KAFKA-4665 URL: https://issues.apache.org/jira/browse/KAFKA-4665 Project: Kafk

[GitHub] kafka pull request #1157: KAFKA-3177: log warning when topic/partition doesn...

2017-01-17 Thread hachikuji
Github user hachikuji closed the pull request at: https://github.com/apache/kafka/pull/1157 --- 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-3177) Kafka consumer can hang when position() is called on a non-existing partition.

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

[jira] [Updated] (KAFKA-3177) Kafka consumer can hang when position() is called on a non-existing partition.

2017-01-17 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3177?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-3177: --- Fix Version/s: (was: 0.10.2.0) 0.10.3.0 > Kafka consumer can hang when

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

2017-01-17 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4665?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian reassigned KAFKA-4665: -- Assignee: Vahid Hashemian > Inconsistent handling of non-existing topics in offset fetc

[jira] [Created] (KAFKA-4666) Failure test for Kafka configured for consistency vs availability

2017-01-17 Thread Emanuele Cesena (JIRA)
Emanuele Cesena created KAFKA-4666: -- Summary: Failure test for Kafka configured for consistency vs availability Key: KAFKA-4666 URL: https://issues.apache.org/jira/browse/KAFKA-4666 Project: Kafka

Build failed in Jenkins: kafka-0.10.2-jdk7 #4

2017-01-17 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4580; Use sasl.jaas.config for some system tests [jason] MINOR: Some cleanups and additional testing for KIP-88 -- [...truncated 18962 lines...] org.apache.kafka.strea

[jira] [Updated] (KAFKA-3896) Unstable test KStreamRepartitionJoinTest.shouldCorrectlyRepartitionOnJoinOperations

2017-01-17 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3896?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-3896: - Assignee: Guozhang Wang (was: Damian Guy) > Unstable test > KStreamRepartitionJoinTest.shouldCor

[jira] [Commented] (KAFKA-3896) Unstable test KStreamRepartitionJoinTest.shouldCorrectlyRepartitionOnJoinOperations

2017-01-17 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15826752#comment-15826752 ] Guozhang Wang commented on KAFKA-3896: -- Another instance: https://builds.apache.org/

[GitHub] kafka pull request #2371: KAFKA-4588: Wait for topics to be created in Query...

2017-01-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2371 --- 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-4588) QueryableStateIntegrationTest.shouldNotMakeStoreAvailableUntilAllStoresAvailable is occasionally failing on jenkins

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

  1   2   >