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

2017-03-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3143?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3143: --- Labels: reliability (was: ) > inconsistent state in ZK when all replicas are dead >

Re: [DISCUSS] KIP-124: Request rate quotas

2017-03-09 Thread Jun Rao
Hi, Rajini, Thanks for the updated KIP. A few more comments. 30. Should we just account for the time in network threads in this KIP too? The issue with doing this later is that existing quotas may be too small and everyone will have to adjust them before upgrading, which is inconvenient. If we

[jira] [Commented] (KAFKA-2082) Kafka Replication ends up in a bad state

2017-03-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15903243#comment-15903243 ] Ismael Juma commented on KAFKA-2082: Has anyone experienced this with a recent version? Say 0.10.1.0

Re: [DISCUSS] KIP-124: Request rate quotas

2017-03-09 Thread Rajini Sivaram
I have updated the KIP to use "request.percentage" quotas where the percentage is out of a total of (num.io.threads * 100). I have added the other options considered so far under "Rejected Alternatives". To address Todd's concern about per-thread quotas: Even though the quotas are out of

[jira] [Commented] (KAFKA-4433) Kafka Controller Does not send a LeaderAndIsr to old leader of a topicPartition during reassignment, if the old leader is not a part of the new assigned replicas

2017-03-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4433?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15903289#comment-15903289 ] Ismael Juma commented on KAFKA-4433: [~mgharat], are you working on this? > Kafka Controller Does not

[jira] [Updated] (KAFKA-4433) Kafka Controller Does not send a LeaderAndIsr to old leader of a topicPartition during reassignment, if the old leader is not a part of the new assigned replicas

2017-03-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4433?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-4433: --- Priority: Critical (was: Major) > Kafka Controller Does not send a LeaderAndIsr to old leader of a

[jira] [Created] (KAFKA-4876) WindowStore.fetch(key, 0, Long.MaxValue) is very slow

2017-03-09 Thread JIRA
Xavier Léauté created KAFKA-4876: Summary: WindowStore.fetch(key, 0, Long.MaxValue) is very slow Key: KAFKA-4876 URL: https://issues.apache.org/jira/browse/KAFKA-4876 Project: Kafka Issue

[jira] [Updated] (KAFKA-3096) Leader is not set to -1 when it is shutdown if followers are down

2017-03-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3096?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3096: --- Labels: reliability (was: ) > Leader is not set to -1 when it is shutdown if followers are down >

[jira] [Resolved] (KAFKA-4305) Possible race condition in log segment truncation and Kafka Request Handler

2017-03-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4305?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4305. Resolution: Duplicate Thanks for the report, this is a duplicate of KAFKA-4205. > Possible race

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

2017-03-09 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4861; GroupMetadataManager record is rejected if broker configured -- [...truncated 157.10 KB...] kafka.security.auth.SimpleAclAuthorizerTest

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

2017-03-09 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4861; GroupMetadataManager record is rejected if broker configured -- [...truncated 322.23 KB...] kafka.server.KafkaConfigTest >

[GitHub] kafka pull request #2662: KAFKA-4863: Querying window store may return unwan...

2017-03-09 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2662 KAFKA-4863: Querying window store may return unwanted keys Make sure that the iterator returned from `WindowStore.fetch(..)` only returns matching keys, rather than all keys that are a prefix match.

[jira] [Commented] (KAFKA-4863) Querying window store may return unwanted keys

2017-03-09 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15903155#comment-15903155 ] ASF GitHub Bot commented on KAFKA-4863: --- GitHub user dguy opened a pull request:

[jira] [Updated] (KAFKA-1211) Hold the produce request with ack > 1 in purgatory until replicas' HW has larger than the produce offset (KIP-101)

2017-03-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1211?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-1211: --- Summary: Hold the produce request with ack > 1 in purgatory until replicas' HW has larger than the

[jira] [Work started] (KAFKA-4863) Querying window store may return unwanted keys

2017-03-09 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4863?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-4863 started by Damian Guy. - > Querying window store may return unwanted keys >

[jira] [Updated] (KAFKA-4863) Querying window store may return unwanted keys

2017-03-09 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4863?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy updated KAFKA-4863: -- Status: Patch Available (was: In Progress) > Querying window store may return unwanted keys >

Re: [DISCUSS] KIP-125: ZookeeperConsumerConnector to KafkaConsumer Migration and Rollback

2017-03-09 Thread Jason Gustafson
Hey Onur, Sorry for the late reply. Thanks for the well-written KIP! I think the proposal makes sense. The only thing I was wondering is whether the process is a bit complex for most users. You'd probably have no trouble at LI (especially given you're implementing it!), but I'm not so sure about

[GitHub] kafka pull request #2658: MINOR: Introduce NetworkClient.hasInFlightRequests

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

[GitHub] kafka pull request #2663: MINOR: increase RocksDb parallelism

2017-03-09 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/2663 MINOR: increase RocksDb parallelism You can merge this pull request into a Git repository by running: $ git pull https://github.com/enothereska/kafka minor-rocksdb-parallel Alternatively

Re: Becoming a contributor

2017-03-09 Thread Matthias J. Sax
Just create a JIRA account and share you JIRA account name here, so you can be added to the contributor list. -Matthias On 3/6/17 5:59 PM, Tyler Hale wrote: > Kafka devs, > > I'm looking to start contributing to kafka and have been reading over the > getting started and contributing guides.

[jira] [Resolved] (KAFKA-4876) WindowStore.fetch(key, 0, Long.MaxValue) is very slow

2017-03-09 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4876?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy resolved KAFKA-4876. --- Resolution: Duplicate Fix Version/s: 0.10.2.1 > WindowStore.fetch(key, 0, Long.MaxValue) is

[GitHub] kafka pull request #2665: MINOR: Let MirrorMaker pick up client.id if specif...

2017-03-09 Thread Xaelias
GitHub user Xaelias opened a pull request: https://github.com/apache/kafka/pull/2665 MINOR: Let MirrorMaker pick up client.id if specified in consumer.properties With this change, defining client.id in the consumer.properties used by mirror maker will no longer be ignored.

[jira] [Commented] (KAFKA-4689) OffsetValidationTest fails validation with "Current position greater than the total number of consumed records"

2017-03-09 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4689?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15903934#comment-15903934 ] Apurva Mehta commented on KAFKA-4689: - This happened again last night. Logs are at:

[jira] [Commented] (KAFKA-4877) Expose TimestampType in Producer RecordMetadata

2017-03-09 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4877?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15903932#comment-15903932 ] Jason Gustafson commented on KAFKA-4877: [~becket_qin] Perhaps there was a reason we didn't do

[jira] [Commented] (KAFKA-4858) Long topic names created using old kafka-topics.sh can prevent newer brokers from joining any ISRs

2017-03-09 Thread Jeff Widman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15903928#comment-15903928 ] Jeff Widman commented on KAFKA-4858: My understanding was that the general expectation was that you

[GitHub] kafka pull request #2664: MINOR: Added additional -start/-stop files for con...

2017-03-09 Thread nnordrum
GitHub user nnordrum opened a pull request: https://github.com/apache/kafka/pull/2664 MINOR: Added additional -start/-stop files for consistency I added *-start.sh and *-stop.sh for connect-* and kafka-mirror-maker to follow the pattern used by the other services. The purpose of

[jira] [Updated] (KAFKA-4872) Getting java.io.IOException in kafka 0.10.1.0 & 0.10.1.1

2017-03-09 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4872?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4872: - Component/s: (was: KafkaConnect) > Getting java.io.IOException in kafka

[jira] [Commented] (KAFKA-4858) Long topic names created using old kafka-topics.sh can prevent newer brokers from joining any ISRs

2017-03-09 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15903897#comment-15903897 ] Vahid Hashemian commented on KAFKA-4858: [~wushujames], so the issue is actually the same as what

[jira] [Created] (KAFKA-4877) Expose TimestampType in Producer RecordMetadata

2017-03-09 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4877: -- Summary: Expose TimestampType in Producer RecordMetadata Key: KAFKA-4877 URL: https://issues.apache.org/jira/browse/KAFKA-4877 Project: Kafka Issue

[jira] [Commented] (KAFKA-4858) Long topic names created using old kafka-topics.sh can prevent newer brokers from joining any ISRs

2017-03-09 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15903926#comment-15903926 ] James Cheng commented on KAFKA-4858: Yup, I agree that it's actually the same as KAFKA-3219. The

[jira] [Comment Edited] (KAFKA-4689) OffsetValidationTest fails validation with "Current position greater than the total number of consumed records"

2017-03-09 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4689?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15903934#comment-15903934 ] Apurva Mehta edited comment on KAFKA-4689 at 3/9/17 10:10 PM: -- This happened

[jira] [Commented] (KAFKA-4858) Long topic names created using old kafka-topics.sh can prevent newer brokers from joining any ISRs

2017-03-09 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15904026#comment-15904026 ] Vahid Hashemian commented on KAFKA-4858: [~wushujames] [~jeffwidman] From what I can tell it all

[jira] [Commented] (KAFKA-4433) Kafka Controller Does not send a LeaderAndIsr to old leader of a topicPartition during reassignment, if the old leader is not a part of the new assigned replicas

2017-03-09 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4433?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15903990#comment-15903990 ] Jun Rao commented on KAFKA-4433: [~mgharat], are you sure this is a real issue?

Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-03-09 Thread Colin McCabe
Hi all, We've been discussing this for a while (about a month) and I think people have made some great points that improved the proposal. In particular, adding async and batching was important. I've also been talking with some end-users who would like to make use of this API. Once this is in,

[jira] [Created] (KAFKA-4878) Kafka Connect does not log connector configuration errors

2017-03-09 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-4878: --- Summary: Kafka Connect does not log connector configuration errors Key: KAFKA-4878 URL: https://issues.apache.org/jira/browse/KAFKA-4878 Project: Kafka Issue

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

2017-03-09 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3143?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15904086#comment-15904086 ] Jun Rao commented on KAFKA-3143: The ISR is only set to empty if unclean leader election is allowed.

[jira] [Created] (KAFKA-4879) KafkaConsumer.position may hang forever when deleting a topic

2017-03-09 Thread Shixiong Zhu (JIRA)
Shixiong Zhu created KAFKA-4879: --- Summary: KafkaConsumer.position may hang forever when deleting a topic Key: KAFKA-4879 URL: https://issues.apache.org/jira/browse/KAFKA-4879 Project: Kafka

[jira] [Assigned] (KAFKA-4574) Transient failure in ZooKeeperSecurityUpgradeTest.test_zk_security_upgrade with security_protocol = SASL_PLAINTEXT, SSL

2017-03-09 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4574?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta reassigned KAFKA-4574: --- Assignee: Apurva Mehta > Transient failure in

[jira] [Commented] (KAFKA-4574) Transient failure in ZooKeeperSecurityUpgradeTest.test_zk_security_upgrade with security_protocol = SASL_PLAINTEXT, SSL

2017-03-09 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15904224#comment-15904224 ] Apurva Mehta commented on KAFKA-4574: - I don't think this is due to

[jira] [Commented] (KAFKA-4574) Transient failure in ZooKeeperSecurityUpgradeTest.test_zk_security_upgrade with security_protocol = SASL_PLAINTEXT, SSL

2017-03-09 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15904206#comment-15904206 ] Apurva Mehta commented on KAFKA-4574: - Another occurrence:

[GitHub] kafka pull request #2657: KAFKA-4861; GroupMetadataManager record is rejecte...

2017-03-09 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2657 --- 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-4861) log.message.timestamp.type=LogAppendTime breaks Kafka based consumers

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

[jira] [Updated] (KAFKA-4467) Run tests on travis-ci using docker

2017-03-09 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4467?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-4467: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved

[GitHub] kafka pull request #2376: KAFKA-4467: Run tests on travis-ci using docker

2017-03-09 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2376 --- 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-4467) Run tests on travis-ci using docker

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

[jira] [Commented] (KAFKA-4877) Expose TimestampType in Producer RecordMetadata

2017-03-09 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4877?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15904148#comment-15904148 ] Jiangjie Qin commented on KAFKA-4877: - [~hachikuji] The original idea was that the producer should

Re: [VOTE] KIP-81: Bound Fetch memory usage in the consumer

2017-03-09 Thread Jason Gustafson
Just a minor comment. The KIP suggests that coordinator responses are always allocated outside of the memory pool, but maybe we can reserve that capability for only when the pool does not have enough space? It seems a little nicer to use the pool if we can. If that seems reasonable, I'm +1 on the

[jira] [Commented] (KAFKA-4277) creating ephemeral node already exist

2017-03-09 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4277?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15904215#comment-15904215 ] Jun Rao commented on KAFKA-4277: [~Wrikken], [~steven.aerts], according to Zookeeper's guarantee, after a

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

2017-03-09 Thread Jason Gustafson
Re; Purge vs PurgeRecords: I think I'm with Ismael and Jeff that the increasing surface area of the request APIs calls for more explicit naming. PurgeRecords sounds reasonable to me. Using simple verbs like "fetch" and "produce" made sense when there were 6 or 7 APIs, but we'll soon be up to 30. I

[jira] [Commented] (KAFKA-4790) Kafka cannot recover after a disk full

2017-03-09 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4790?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15904121#comment-15904121 ] Jun Rao commented on KAFKA-4790: [~pengwei], thanks for reporting that. That's a good find. I think this

[jira] [Commented] (KAFKA-4464) Clean shutdown of broker fails due to controller error

2017-03-09 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15904151#comment-15904151 ] Jun Rao commented on KAFKA-4464: When the process hang, could you take a thread dump and see what the

[jira] [Closed] (KAFKA-4864) Kafka Secure Migrator tool doesn't secure all the nodes

2017-03-09 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4864?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar closed KAFKA-4864. > Kafka Secure Migrator tool doesn't secure all the nodes >

[jira] [Commented] (KAFKA-4574) Transient failure in ZooKeeperSecurityUpgradeTest.test_zk_security_upgrade with security_protocol = SASL_PLAINTEXT, SSL

2017-03-09 Thread Apurva Mehta (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15904270#comment-15904270 ] Apurva Mehta commented on KAFKA-4574: - So, some interesting findings. I dumped the log segments of

[jira] [Updated] (KAFKA-4861) log.message.timestamp.type=LogAppendTime breaks Kafka based consumers

2017-03-09 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4861?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-4861: --- Resolution: Fixed Fix Version/s: (was: 0.11.0.0) Status: Resolved (was:

[jira] [Commented] (KAFKA-4879) KafkaConsumer.position may hang forever when deleting a topic

2017-03-09 Thread huxi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4879?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15904322#comment-15904322 ] huxi commented on KAFKA-4879: - Could have KafkaConsumer#updateFetchPositions, Fetcher#updateFetchPositions,

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-09 Thread Dong Lin
Hey Jun, Thanks for your comments! I have updated the KIP to address your comments. Please see my reply inline. Can you let me know if the latest KIP has addressed your comments? On Wed, Mar 8, 2017 at 9:56 PM, Jun Rao wrote: > Hi, Dong, > > Thanks for the reply. > > 1.3 So

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-09 Thread Dong Lin
I just made one correction in the KIP. If broker receives ChangeReplicaDirRequest and the replica hasn't been created there, the broker will respond ReplicaNotAvailableException. The kafka-reassignemnt-partitions.sh will need to re-send ChangeReplicaDirRequest in this case in order to wait for

[jira] [Commented] (KAFKA-4790) Kafka cannot recover after a disk full

2017-03-09 Thread Pengwei (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4790?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15904333#comment-15904333 ] Pengwei commented on KAFKA-4790: [~junrao] If KIP-98 can fix this issue, we can wait for the KIP-98.

[jira] [Commented] (KAFKA-4867) zookeeper-security-migration.sh does not clear ACLs from all nodes

2017-03-09 Thread Stevo Slavic (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4867?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15902670#comment-15902670 ] Stevo Slavic commented on KAFKA-4867: - Same problem affects setting ACLs with {noformat}

[jira] [Commented] (KAFKA-4871) Kafka doesn't respect TTL on Zookeeper hostname - crash if zookeeper IP changes

2017-03-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15902678#comment-15902678 ] Ismael Juma commented on KAFKA-4871: Thanks for the report. I think this is the following ZooKeeper

[jira] [Updated] (KAFKA-4874) SASL driven connnections are not dropped when client ticket expires

2017-03-09 Thread Pawel Tomasik (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4874?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pawel Tomasik updated KAFKA-4874: - Description: The proposal is to improve SASL protocol logic In current implementation, Broker

[jira] [Created] (KAFKA-4874) SASL driven connnections are not dropped when client ticket expires

2017-03-09 Thread Pawel Tomasik (JIRA)
Pawel Tomasik created KAFKA-4874: Summary: SASL driven connnections are not dropped when client ticket expires Key: KAFKA-4874 URL: https://issues.apache.org/jira/browse/KAFKA-4874 Project: Kafka

Re: SASL security issue

2017-03-09 Thread Paweł Tomasik
Ismael Thank you for the response I've walked through changes for KAFKA-3866. I think it shall fix the case I mentioned. As for server side, I've added a jira wish issue: https://issues.apache.org/jira/browse/KAFKA-4874 I'm working on project with high security restrictions, so I need to find an

[jira] [Resolved] (KAFKA-4825) Likely Data Loss in ReassignPartitionsTest System Test

2017-03-09 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4825?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford resolved KAFKA-4825. - Resolution: Later Resolving this as it will be fixed by KIP-101 in 3.3 > Likely Data Loss in

[jira] [Created] (KAFKA-4873) Investigate issues uncovered by CORDS

2017-03-09 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-4873: -- Summary: Investigate issues uncovered by CORDS Key: KAFKA-4873 URL: https://issues.apache.org/jira/browse/KAFKA-4873 Project: Kafka Issue Type: Bug

Re: [DISCUSS] KIP-120: Cleanup Kafka Streams builder API

2017-03-09 Thread Michael Noll
Thanks for the update, Matthias. +1 to the points 1,2,3,4 you mentioned. Naming is always a tricky subject, but renaming KStreamBuilder to StreamsTopologyBuilder looks ok to me (I would have had a slight preference towards DslTopologyBuilder, but hey.) The most important aspect is, IMHO, what

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

2017-03-09 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-4825) Likely Data Loss in ReassignPartitionsTest System Test

2017-03-09 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4825?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15902877#comment-15902877 ] Ben Stopford commented on KAFKA-4825: - Thanks Jun. Makes sense. > Likely Data Loss in

[jira] [Commented] (KAFKA-4867) zookeeper-security-migration.sh does not clear ACLs from all nodes

2017-03-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4867?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15902675#comment-15902675 ] Ismael Juma commented on KAFKA-4867: Have you checked that

[jira] [Resolved] (KAFKA-4867) zookeeper-security-migration.sh does not clear ACLs from all nodes

2017-03-09 Thread Stevo Slavic (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4867?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stevo Slavic resolved KAFKA-4867. - Resolution: Duplicate > zookeeper-security-migration.sh does not clear ACLs from all nodes >

[jira] [Updated] (KAFKA-4871) Kafka doesn't respect TTL on Zookeeper hostname - crash if zookeeper IP changes

2017-03-09 Thread Stephane Maarek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4871?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stephane Maarek updated KAFKA-4871: --- Yeah definitely ! Seeing how this issue has been opened for two years I'm quite worried about the

[jira] [Commented] (KAFKA-4867) zookeeper-security-migration.sh does not clear ACLs from all nodes

2017-03-09 Thread Stevo Slavic (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4867?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15902683#comment-15902683 ] Stevo Slavic commented on KAFKA-4867: - Duplicate of KAFKA-4864. Sorry. >

Re: SASL security issue

2017-03-09 Thread Ismael Juma
Hi Pawel, It is by design that authentication is only performed during connection establishment in the broker. Kafka relies on long-lived connections, which means that another mechanism is needed to handle users who have been removed from the system. A typical approach is to remove all ACLs for

Re: SASL security issue

2017-03-09 Thread Ismael Juma
Thanks for filing the JIRA and checking KAFKA-3866. I'll try to add tests to the PR so that we can merge it. Ismael On Thu, Mar 9, 2017 at 10:44 AM, Paweł Tomasik wrote: > Ismael > > Thank you for the response > I've walked through changes for KAFKA-3866. > I think it shall

[jira] [Comment Edited] (KAFKA-4825) Likely Data Loss in ReassignPartitionsTest System Test

2017-03-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4825?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15902909#comment-15902909 ] Ismael Juma edited comment on KAFKA-4825 at 3/9/17 11:30 AM: - Resolving this

[jira] [Updated] (KAFKA-4875) Kafka Streams: topic groups and builder.stream API

2017-03-09 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4875?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Noll updated KAFKA-4875: Summary: Kafka Streams: topic groups and builder.stream API (was: Kafka streams topic groups and

[jira] [Commented] (KAFKA-4875) Kafka streams topic groups and builder.stream API

2017-03-09 Thread Michael Noll (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4875?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15902975#comment-15902975 ] Michael Noll commented on KAFKA-4875: - There two things at play here IMHO: 1. Whether or not there's

[jira] [Updated] (KAFKA-3866) KerberosLogin refresh time bug and other improvements

2017-03-09 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3866?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3866: --- Fix Version/s: 0.10.2.1 0.11.0.0 > KerberosLogin refresh time bug and other

[jira] [Created] (KAFKA-4875) Kafka streams topic groups and builder.stream API

2017-03-09 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-4875: --- Summary: Kafka streams topic groups and builder.stream API Key: KAFKA-4875 URL: https://issues.apache.org/jira/browse/KAFKA-4875 Project: Kafka Issue Type:

[jira] [Commented] (KAFKA-4669) KafkaProducer.flush hangs when NetworkClient.handleCompletedReceives throws exception

2017-03-09 Thread Paul Slater (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4669?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15902986#comment-15902986 ] Paul Slater commented on KAFKA-4669: +1 for fixing the client as well. Similar hang occurred for me