Re: [DISCUSS] KIP-70: Revise Partition Assignment Semantics on New Consumer's Subscription Change

2016-07-21 Thread Ewen Cheslack-Postava
Agreed w/ Jason re: compatibility. It seems like such an edge case to actually rely on this and I'd consider the current behavior essentially a bug given how surprising it is. While normally a stickler for compatibility, I think this is a case where its fine to make the change. -Ewen On Wed, Jul

Re: [DISCUSS] KIP-66 Kafka Connect Transformers for messages

2016-07-21 Thread Ewen Cheslack-Postava
Jun, The problem with it not being 1-1 is that Connect relies heavily on offsets, so we'd need to be able to track offsets at this finer granularity. Filtering is ok, but flatMap isn't. If you convert one message to many, what are the offsets for the new messages? One possibility would be to

[jira] [Updated] (KAFKA-3984) Broker doesn't retry reconnecting to an expired Zookeeper connection

2016-07-21 Thread Braedon Vickers (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3984?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Braedon Vickers updated KAFKA-3984: --- Description: We've been having issues with the network connectivity of our Kafka cluster,

[jira] [Created] (KAFKA-3984) Broker doesn't retry reconnecting to an expired Zookeeper connection

2016-07-21 Thread Braedon Vickers (JIRA)
Braedon Vickers created KAFKA-3984: -- Summary: Broker doesn't retry reconnecting to an expired Zookeeper connection Key: KAFKA-3984 URL: https://issues.apache.org/jira/browse/KAFKA-3984 Project:

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

2016-07-21 Thread Apache Jenkins Server
See Changes: [me] KAFKA-3782: Fix transient failure in connect distributed bounce test -- [...truncated 10686 lines...] org.apache.kafka.common.serialization.SerializationTest >

[GitHub] kafka pull request #1644: MINOR: Update to Gradle 2.14.1

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

[jira] [Resolved] (KAFKA-3782) Transient failure with kafkatest.tests.connect.connect_distributed_test.ConnectDistributedTest.test_bounce.clean=True

2016-07-21 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3782?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-3782. -- Resolution: Fixed Fix Version/s: 0.10.0.1 Issue resolved by pull request

[GitHub] kafka pull request #1650: KAFKA-3782: Fix transient failure in connect distr...

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

[jira] [Commented] (KAFKA-3782) Transient failure with kafkatest.tests.connect.connect_distributed_test.ConnectDistributedTest.test_bounce.clean=True

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

Re: [DISCUSS] KIP-4 ACL Admin Schema

2016-07-21 Thread Ismael Juma
Hi Grant, Thanks for the KIP. A few questions and comments: 1. My main concern is that we are skipping the discussion on the desired model for controlling ACL access and updates. I understand the desire to reduce the scope, but this seems to be a fundamental aspect of the design that we need to

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

2016-07-21 Thread Apache Jenkins Server
See

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

2016-07-21 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Upgrade RocksDB to 4.8.0 [ismael] HOTFIX: Adding init file so streams benchmark is autodiscovered -- [...truncated 13085 lines...]

[GitHub] kafka pull request #1650: KAFKA-3782: Fix transient failure in connect distr...

2016-07-21 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1650 KAFKA-3782: Fix transient failure in connect distributed bounce test You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka

[jira] [Commented] (KAFKA-3782) Transient failure with kafkatest.tests.connect.connect_distributed_test.ConnectDistributedTest.test_bounce.clean=True

2016-07-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3782?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388692#comment-15388692 ] ASF GitHub Bot commented on KAFKA-3782: --- GitHub user hachikuji opened a pull request:

[jira] [Commented] (KAFKA-3887) StreamBounceTest.test_bounce and StreamSmokeTest.test_streams failing

2016-07-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3887?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388630#comment-15388630 ] Guozhang Wang commented on KAFKA-3887: -- This seems to be a regression that was introduced in

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

2016-07-21 Thread Apache Jenkins Server
See Changes: [wangguoz] MINOR: Upgrade RocksDB to 4.8.0 -- [...truncated 5234 lines...] kafka.log.LogCleanerIntegrationTest > cleanerTest[2] PASSED kafka.log.LogCleanerIntegrationTest >

[GitHub] kafka pull request #1645: HOTFIX: Adding init file so streams benchmark is a...

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

[jira] [Commented] (KAFKA-3929) Add prefix for underlying clients configs in StreamConfig

2016-07-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3929?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388593#comment-15388593 ] ASF GitHub Bot commented on KAFKA-3929: --- GitHub user dguy opened a pull request:

[GitHub] kafka pull request #1649: KAFKA-3929: Add prefix for underlying clients conf...

2016-07-21 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/1649 KAFKA-3929: Add prefix for underlying clients configs in StreamConfig Add prefixes for consumer and producer configs to StreamsConfig, but be backward compatible. You can merge this pull request into

[GitHub] kafka pull request #1647: MINOR: Upgrade RocksDB to 4.8.0

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

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

2016-07-21 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3911: KTable source materialization -- [...truncated 4655 lines...] kafka.server.ServerShutdownTest > testCleanShutdownAfterFailedStartup STARTED

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

2016-07-21 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-3983) It would be helpful if SocketServer's Acceptors logged both the SocketChannel port and the processor ID upon registra

2016-07-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388519#comment-15388519 ] ASF GitHub Bot commented on KAFKA-3983: --- GitHub user rnpridgeon opened a pull request:

[GitHub] kafka pull request #1648: KAFKA-3983 - Add additional information to debug

2016-07-21 Thread rnpridgeon
GitHub user rnpridgeon opened a pull request: https://github.com/apache/kafka/pull/1648 KAFKA-3983 - Add additional information to debug Add additional information to Acceptor debug message upon connection acceptance You can merge this pull request into a Git repository by

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

2016-07-21 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3858: Add functions to print stream topologies -- [...truncated 4641 lines...] kafka.server.LogRecoveryTest > testHWCheckpointNoFailuresSingleLogSegment PASSED

[jira] [Updated] (KAFKA-3983) It would be helpful if SocketServer's Acceptors logged both the SocketChannel port and the processor ID upon registra

2016-07-21 Thread Ryan P (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3983?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ryan P updated KAFKA-3983: -- Description: Currently Acceptors log the following message prior to registering passing the accepted channel

[jira] [Updated] (KAFKA-3983) It would be helpful if SocketServer's Acceptors logged both the SocketChannel port and the processor ID upon registra

2016-07-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3983?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3983: --- Fix Version/s: 0.10.0.1 > It would be helpful if SocketServer's Acceptors logged both the

[jira] [Created] (KAFKA-3983) It would be helpful if SocketServer's Acceptors logged both the SocketChannel port and the processor ID upon registra

2016-07-21 Thread Ryan P (JIRA)
Ryan P created KAFKA-3983: - Summary: It would be helpful if SocketServer's Acceptors logged both the SocketChannel port and the processor ID upon registra Key: KAFKA-3983 URL:

[jira] [Commented] (KAFKA-3911) Enforce KTable materialization

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

[GitHub] kafka pull request #1638: KAFKA-3911: KTable source materialization

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

[jira] [Resolved] (KAFKA-3911) Enforce KTable materialization

2016-07-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3911?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3911. -- Resolution: Fixed Issue resolved by pull request 1638

[jira] [Comment Edited] (KAFKA-3980) JmxReporter uses excessive memory causing OutOfMemoryException

2016-07-21 Thread Andrew Jorgensen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388382#comment-15388382 ] Andrew Jorgensen edited comment on KAFKA-3980 at 7/21/16 9:37 PM: -- I was

[jira] [Commented] (KAFKA-3980) JmxReporter uses excessive memory causing OutOfMemoryException

2016-07-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388471#comment-15388471 ] Ismael Juma commented on KAFKA-3980: [~omkreddy], we do expire client quota metrics actually. We don't

[jira] [Assigned] (KAFKA-3929) Add prefix for underlying clients configs in StreamConfig

2016-07-21 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3929?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy reassigned KAFKA-3929: - Assignee: Damian Guy (was: Ishita Mandhan) > Add prefix for underlying clients configs in

[GitHub] kafka pull request #1647: MINOR: Upgrade RocksDB to 4.8.0

2016-07-21 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/1647 MINOR: Upgrade RocksDB to 4.8.0 You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka kafka-rocksdb-4.8 Alternatively you can review and

[jira] [Commented] (KAFKA-3929) Add prefix for underlying clients configs in StreamConfig

2016-07-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3929?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388440#comment-15388440 ] Guozhang Wang commented on KAFKA-3929: -- Thanks! I will let Damian to pick it up then. > Add prefix

[jira] [Commented] (KAFKA-2561) Optionally support OpenSSL for SSL/TLS

2016-07-21 Thread Hendrik Saly (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2561?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388432#comment-15388432 ] Hendrik Saly commented on KAFKA-2561: - Here is a working draft:

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

2016-07-21 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3740: Enable configuration of RocksDBStores -- [...truncated 4426 lines...] kafka.utils.ReplicationUtilsTest > testGetLeaderIsrAndEpochForPartition PASSED

[jira] [Commented] (KAFKA-1044) change log4j to slf4j

2016-07-21 Thread Shannon Carey (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1044?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388393#comment-15388393 ] Shannon Carey commented on KAFKA-1044: -- In addition, can we change the dependency on slf4j-log4j12

[jira] [Commented] (KAFKA-3980) JmxReporter uses excessive memory causing OutOfMemoryException

2016-07-21 Thread Andrew Jorgensen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388382#comment-15388382 ] Andrew Jorgensen commented on KAFKA-3980: - I was able to extract a list of all the ids from the

[jira] [Updated] (KAFKA-3982) Issue with processing order of consumer properties in console consumer

2016-07-21 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-3982: --- Description: With the recent introduction of {{consumer.property}} argument in console

[jira] [Updated] (KAFKA-3982) Issue with processing order of consumer properties in console consumer

2016-07-21 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-3982: --- Description: With the recent introduction of {{consumer.property}} argument in console

[jira] [Commented] (KAFKA-3858) Add functions to print stream topologies

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

[GitHub] kafka pull request #1619: KAFKA-3858: Add functions to print stream topologi...

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

[jira] [Resolved] (KAFKA-3858) Add functions to print stream topologies

2016-07-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3858?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3858. -- Resolution: Fixed Fix Version/s: 0.10.1.0 Issue resolved by pull request 1619

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

2016-07-21 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3740: Enable configuration of RocksDBStores -- [...truncated 1349 lines...] kafka.server.AbstractFetcherThreadTest > testMetricsRemovedOnShutdown STARTED

[jira] [Updated] (KAFKA-3982) Issue with processing order of consumer properties in console consumer

2016-07-21 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3982?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vahid Hashemian updated KAFKA-3982: --- Description: With the recent introduction of {{consumer.property}} argument in console

[jira] [Commented] (KAFKA-3929) Add prefix for underlying clients configs in StreamConfig

2016-07-21 Thread Ishita Mandhan (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3929?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388216#comment-15388216 ] Ishita Mandhan commented on KAFKA-3929: --- I've started but haven't made too much progress into it so

Proposal for an End-to-End encryption in Kafka

2016-07-21 Thread Hendrik Dev
Proposal for an End-to-End encryption in Kafka: https://github.com/salyh/kafka/commit/151a0340645f4165cb120f79df544326e5a0db6f Introduce EncryptingSerializer and DecryptingDeserializer. They act as wrappers for inner (de)serializer and provide transparent AES and RSA based end-to-end encryption.

Re: [DISCUSS] KIP-4 ACL Admin Schema

2016-07-21 Thread Jason Gustafson
Hi Grant, This looks good to me. One minor comment. You mention that "delete" actions will get processed before "add" actions, which makes sense to me. An alternative to avoid the confusion in the first place would be to replace the AlterAcls APIs with separate AddAcls and DeleteAcls APIs. Was

[jira] [Created] (KAFKA-3982) Issue with processing order of consumer properties in console consumer

2016-07-21 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-3982: -- Summary: Issue with processing order of consumer properties in console consumer Key: KAFKA-3982 URL: https://issues.apache.org/jira/browse/KAFKA-3982 Project:

[jira] [Commented] (KAFKA-3929) Add prefix for underlying clients configs in StreamConfig

2016-07-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3929?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15388137#comment-15388137 ] Guozhang Wang commented on KAFKA-3929: -- [~imandhan] Are you already working on this ticket? > Add

[jira] [Commented] (KAFKA-3740) Enable configuration of RocksDBStores

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

[jira] [Resolved] (KAFKA-3740) Enable configuration of RocksDBStores

2016-07-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3740?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-3740. -- Resolution: Fixed Fix Version/s: 0.10.1.0 Issue resolved by pull request 1640

[GitHub] kafka pull request #1640: KAFKA-3740: Enable configuration of RocksDBStores

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

[jira] [Reopened] (KAFKA-3887) StreamBounceTest.test_bounce and StreamSmokeTest.test_streams failing

2016-07-21 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3887?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson reopened KAFKA-3887: Still seeing this failure in nightly runs:

Re: [DISCUSS] Optimise memory used by replication process by using adaptive fetch message size

2016-07-21 Thread Jay Kreps
I think the memory usage for consumers can be improved a lot, but I think there may be a better way then what you are proposing. The problem is exactly what you describe: the bound the user sets is per-partition, but the number of partitions may be quite high. The consumer could provide a bound

Re: [DISCUSS] KIP-4 ACL Admin Schema

2016-07-21 Thread Grant Henke
Anyone else have any feedback on this protocol and implementation? I plan to start a vote soon. Thank you, Grant On Fri, Jul 15, 2016 at 1:04 PM, Gwen Shapira wrote: > > My goal in the protocol design was to keep the request simple and be able > > to answer what I think are

Re: [DISCUSS] Optimise memory used by replication process by using adaptive fetch message size

2016-07-21 Thread Andrey L. Neporada
Hi! Sorry for not being clear enough. The problem is on follower side, not on leader side. It is the follower who is allocating replica.fetch.max.bytes-sized buffers for fetch responses somewhere in Java client code. Andrey. > On 21 Jul 2016, at 16:28, Tom Crayford

Re: [DISCUSS] Optimise memory used by replication process by using adaptive fetch message size

2016-07-21 Thread Tom Crayford
Hi there, >From my understanding of the protocol (and from digging in the source code a bunch) I can't see anywhere where Kafka overallocates memory based on the fetch request's max bytes, but maybe I have missed something. If there is such a place, then I'd recommend fixing that issue instead -

[DISCUSS] Optimise memory used by replication process by using adaptive fetch message size

2016-07-21 Thread Andrey L. Neporada
Hi all! We noticed that our Kafka cluster uses a lot of memory for replication. Our Kafka usage pattern is following: 1. Most messages are small (tens or hundreds kilobytes at most), but some (rare) messages can be several megabytes.So, we have to set replica.fetch.max.bytes =

[GitHub] kafka pull request #1646: add an annotation

2016-07-21 Thread wangzzu
GitHub user wangzzu opened a pull request: https://github.com/apache/kafka/pull/1646 add an annotation You can merge this pull request into a Git repository by running: $ git pull https://github.com/wangzzu/kafka wangzzu Alternatively you can review and apply these changes

[jira] [Comment Edited] (KAFKA-3980) JmxReporter uses excessive memory causing OutOfMemoryException

2016-07-21 Thread Andrew Jorgensen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15387185#comment-15387185 ] Andrew Jorgensen edited comment on KAFKA-3980 at 7/21/16 5:59 AM: -- As far