[jira] [Commented] (KAFKA-4514) Add Codec for ZStandard Compression
[ 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.apache.org/confluence/display/KAFKA/KIP-110%3A+Add+Codec+for+ZStandard+Compression > Add Codec for ZStandard Compression > --- > > Key: KAFKA-4514 > URL: https://issues.apache.org/jira/browse/KAFKA-4514 > Project: Kafka > Issue Type: Improvement > Components: compression >Reporter: Thomas Graves >Assignee: Lee Dongjin > > ZStandard: https://github.com/facebook/zstd and > http://facebook.github.io/zstd/ has been in use for a while now. v1.0 was > recently released. Hadoop > (https://issues.apache.org/jira/browse/HADOOP-13578) and others are adopting > it. > We have done some initial trials and seen good results. Zstd seems to give > great results => Gzip level Compression with Lz4 level CPU. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-4514) Add Codec for ZStandard Compression
[ https://issues.apache.org/jira/browse/KAFKA-4514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15741991#comment-15741991 ] Thomas Graves commented on KAFKA-4514: -- [~dongjin] That is fine, go ahead. > Add Codec for ZStandard Compression > --- > > Key: KAFKA-4514 > URL: https://issues.apache.org/jira/browse/KAFKA-4514 > Project: Kafka > Issue Type: Improvement > Components: compression >Reporter: Thomas Graves > > ZStandard: https://github.com/facebook/zstd and > http://facebook.github.io/zstd/ has been in use for a while now. v1.0 was > recently released. Hadoop > (https://issues.apache.org/jira/browse/HADOOP-13578) and others are adopting > it. > We have done some initial trials and seen good results. Zstd seems to give > great results => Gzip level Compression with Lz4 level CPU. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-4514) Add Codec for ZStandard Compression
[ https://issues.apache.org/jira/browse/KAFKA-4514?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15733627#comment-15733627 ] Thomas Graves commented on KAFKA-4514: -- Ok thanks. I wanted to get it filed to get peoples input on interest level and see if anyone else had perhaps started it. I won't get time for this immediately but plan to in Q1. > Add Codec for ZStandard Compression > --- > > Key: KAFKA-4514 > URL: https://issues.apache.org/jira/browse/KAFKA-4514 > Project: Kafka > Issue Type: Improvement > Components: compression >Reporter: Thomas Graves > > ZStandard: https://github.com/facebook/zstd and > http://facebook.github.io/zstd/ has been in use for a while now. v1.0 was > recently released. Hadoop > (https://issues.apache.org/jira/browse/HADOOP-13578) and others are adopting > it. > We have done some initial trials and seen good results. Zstd seems to give > great results => Gzip level Compression with Lz4 level CPU. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-4514) Add Codec for ZStandard Compression
Thomas Graves created KAFKA-4514: Summary: Add Codec for ZStandard Compression Key: KAFKA-4514 URL: https://issues.apache.org/jira/browse/KAFKA-4514 Project: Kafka Issue Type: Improvement Components: compression Reporter: Thomas Graves ZStandard: https://github.com/facebook/zstd and http://facebook.github.io/zstd/ has been in use for a while now. v1.0 was recently released. Hadoop (https://issues.apache.org/jira/browse/HADOOP-13578) and others are adopting it. We have done some initial trials and seen good results. Zstd seems to give great results => Gzip level Compression with Lz4 level CPU. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Thomas Graves resolved KAFKA-3236. -- Resolution: Won't Fix Ok, closing this for now then. > Honor Producer Configuration "block.on.buffer.full" > --- > > Key: KAFKA-3236 > URL: https://issues.apache.org/jira/browse/KAFKA-3236 > Project: Kafka > Issue Type: Improvement > Components: producer >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves >Assignee: Thomas Graves > > In Kafka-0.9, "max.block.ms" is used to control how long the following > methods will block. > KafkaProducer.send() when >* Buffer is full >* Metadata is unavailable > KafkaProducer.partitionsFor() when >* Metadata is unavailable > However when "block.on.buffer.full" is set to false, "max.block.ms" is in > effect whenever a buffer is requested/allocated from the Producer BufferPool. > Instead it should throw a BufferExhaustedException without waiting for > "max.block.ms" > This is particulary useful if a producer application does not wish to block > at all on KafkaProducer.send() . We avoid waiting on KafkaProducer.send() > when metadata is unavailable by invoking send() only if the producer instance > has fetched the metadata for the topic in a different thread using the same > producer instance. However "max.block.ms" is still required to specify a > timeout for bootstrapping the metadata fetch. > We should resolve this limitation by decoupling "max.block.ms" and > "block.on.buffer.full". >* "max.block.ms" will be used exclusively for fetching metadata when > "block.on.buffer.full" = false (in pure non-blocking mode ) >* "max.block.ms" will be applicable to both fetching metadata as well as > buffer allocation when "block.on.buffer.full = true -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15178094#comment-15178094 ] Thomas Graves commented on KAFKA-3236: -- So you are thinking you would rather not have this change unless it makes the send() guaranteed to be non-blocking? I agree with what you are saying that it might still block on the send() but this at least adds the option to not block on the full call and adds some flexibility. I would be fine with adding parameter to partitionsFor too, and I could see other possible use cases but that is also an API which is a bit more public then just the config. > Honor Producer Configuration "block.on.buffer.full" > --- > > Key: KAFKA-3236 > URL: https://issues.apache.org/jira/browse/KAFKA-3236 > Project: Kafka > Issue Type: Improvement > Components: producer >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves >Assignee: Thomas Graves > > In Kafka-0.9, "max.block.ms" is used to control how long the following > methods will block. > KafkaProducer.send() when >* Buffer is full >* Metadata is unavailable > KafkaProducer.partitionsFor() when >* Metadata is unavailable > However when "block.on.buffer.full" is set to false, "max.block.ms" is in > effect whenever a buffer is requested/allocated from the Producer BufferPool. > Instead it should throw a BufferExhaustedException without waiting for > "max.block.ms" > This is particulary useful if a producer application does not wish to block > at all on KafkaProducer.send() . We avoid waiting on KafkaProducer.send() > when metadata is unavailable by invoking send() only if the producer instance > has fetched the metadata for the topic in a different thread using the same > producer instance. However "max.block.ms" is still required to specify a > timeout for bootstrapping the metadata fetch. > We should resolve this limitation by decoupling "max.block.ms" and > "block.on.buffer.full". >* "max.block.ms" will be used exclusively for fetching metadata when > "block.on.buffer.full" = false (in pure non-blocking mode ) >* "max.block.ms" will be applicable to both fetching metadata as well as > buffer allocation when "block.on.buffer.full = true -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15176728#comment-15176728 ] Thomas Graves commented on KAFKA-3236: -- Yeah it would be possible to do it outside in our code its just not as nice. >From my understanding the metadata gather thread is ok to block. it needs the >metadata to do anything with that topic. Note this is one of our users I'm >representing here so I'm just going by what they are saying. I think its ok >to block here because the send() thread can still go ahead and send to the >topics it already has metadata on. Note that also send() only blocks waiting on the metadata if its not already cached, in waitOnMetadata: while(metadata.fetch().partitionsForTopic(topic) == null) { so by our metadata thread making sure it is there first I don't think send() will block waiting on metadata at all. Then I think the metadata gets updated when needed by internal kafka threads and doesn't block the sender. We do also expect the buffer full to be much more frequent give the metadata should be cached anyway. > Honor Producer Configuration "block.on.buffer.full" > --- > > Key: KAFKA-3236 > URL: https://issues.apache.org/jira/browse/KAFKA-3236 > Project: Kafka > Issue Type: Improvement > Components: producer >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves >Assignee: Thomas Graves > > In Kafka-0.9, "max.block.ms" is used to control how long the following > methods will block. > KafkaProducer.send() when >* Buffer is full >* Metadata is unavailable > KafkaProducer.partitionsFor() when >* Metadata is unavailable > However when "block.on.buffer.full" is set to false, "max.block.ms" is in > effect whenever a buffer is requested/allocated from the Producer BufferPool. > Instead it should throw a BufferExhaustedException without waiting for > "max.block.ms" > This is particulary useful if a producer application does not wish to block > at all on KafkaProducer.send() . We avoid waiting on KafkaProducer.send() > when metadata is unavailable by invoking send() only if the producer instance > has fetched the metadata for the topic in a different thread using the same > producer instance. However "max.block.ms" is still required to specify a > timeout for bootstrapping the metadata fetch. > We should resolve this limitation by decoupling "max.block.ms" and > "block.on.buffer.full". >* "max.block.ms" will be used exclusively for fetching metadata when > "block.on.buffer.full" = false (in pure non-blocking mode ) >* "max.block.ms" will be applicable to both fetching metadata as well as > buffer allocation when "block.on.buffer.full = true -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15176573#comment-15176573 ] Thomas Graves commented on KAFKA-3236: -- [~becket_qin] So if your concern that this change doesn't make it fully non-blocking since it could potentially still block on the metadata query? You are correct in what you say that block.on.buffer.full = false and max.block.ms > 0 its not purely non-blocking and that is actually what we want. We want to be able to control the metadata fetch block time separate from if the buffer is full. I expect most users to leave block.on.buffer.full=true. We have a use case where we have a process sending to multiple different Kafka clusters. It has one thread that is getting the metadata for all the topics from all the clusters and then another thread that is doing the send() on the produce. The second thread sends to multiple kafka clusters so we don't want it to block if at all possible. It using a single instance of a producer per topic. Since its a single instance we can't set the max.blocking.ms to 0 because then the thread getting the metadata wouldn't block which we want it to. Note the thread getting metadata is calling partitionsFor().With this patch and block.on.buffer.full = false and max.block.ms > 0 it is still possible that the send() blocks but I consider this a rare/special case. We don't want it to block if the buffer is full though. An alternative to this would be to add another interface for partitionsFor() that would take in the maxBlockTimeMs rather then using it from the config. Then the thread doing the send could set max.blocking.ms to 0 and all is well. Thoughts or other ideas? > Honor Producer Configuration "block.on.buffer.full" > --- > > Key: KAFKA-3236 > URL: https://issues.apache.org/jira/browse/KAFKA-3236 > Project: Kafka > Issue Type: Improvement > Components: producer >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves >Assignee: Thomas Graves > > In Kafka-0.9, "max.block.ms" is used to control how long the following > methods will block. > KafkaProducer.send() when >* Buffer is full >* Metadata is unavailable > KafkaProducer.partitionsFor() when >* Metadata is unavailable > However when "block.on.buffer.full" is set to false, "max.block.ms" is in > effect whenever a buffer is requested/allocated from the Producer BufferPool. > Instead it should throw a BufferExhaustedException without waiting for > "max.block.ms" > This is particulary useful if a producer application does not wish to block > at all on KafkaProducer.send() . We avoid waiting on KafkaProducer.send() > when metadata is unavailable by invoking send() only if the producer instance > has fetched the metadata for the topic in a different thread using the same > producer instance. However "max.block.ms" is still required to specify a > timeout for bootstrapping the metadata fetch. > We should resolve this limitation by decoupling "max.block.ms" and > "block.on.buffer.full". >* "max.block.ms" will be used exclusively for fetching metadata when > "block.on.buffer.full" = false (in pure non-blocking mode ) >* "max.block.ms" will be applicable to both fetching metadata as well as > buffer allocation when "block.on.buffer.full = true -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15173834#comment-15173834 ] Thomas Graves commented on KAFKA-3236: -- Thanks for the response. I saw that the config were deprecated in the code but we were hoping to bring them back for this use case. Currently (Kafka 0.9) if block.on.buffer.full = false it blocks up to max.block.ms if either the metadata is unavailable of the buffer is full. The behavior we want (and is implemented in pr) is for it to throw BufferExhaustedException immediately instead of waiting the max.block.ms when the buffer is full. We still use max.block.ms for the metadata unavailable. Do you think this is a reasonable request or is there a reason not to decouple these? > Honor Producer Configuration "block.on.buffer.full" > --- > > Key: KAFKA-3236 > URL: https://issues.apache.org/jira/browse/KAFKA-3236 > Project: Kafka > Issue Type: Improvement > Components: producer >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves >Assignee: Thomas Graves > > In Kafka-0.9, "max.block.ms" is used to control how long the following > methods will block. > KafkaProducer.send() when >* Buffer is full >* Metadata is unavailable > KafkaProducer.partitionsFor() when >* Metadata is unavailable > However when "block.on.buffer.full" is set to false, "max.block.ms" is in > effect whenever a buffer is requested/allocated from the Producer BufferPool. > Instead it should throw a BufferExhaustedException without waiting for > "max.block.ms" > This is particulary useful if a producer application does not wish to block > at all on KafkaProducer.send() . We avoid waiting on KafkaProducer.send() > when metadata is unavailable by invoking send() only if the producer instance > has fetched the metadata for the topic in a different thread using the same > producer instance. However "max.block.ms" is still required to specify a > timeout for bootstrapping the metadata fetch. > We should resolve this limitation by decoupling "max.block.ms" and > "block.on.buffer.full". >* "max.block.ms" will be used exclusively for fetching metadata when > "block.on.buffer.full" = false (in pure non-blocking mode ) >* "max.block.ms" will be applicable to both fetching metadata as well as > buffer allocation when "block.on.buffer.full = true -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Thomas Graves updated KAFKA-3236: - Issue Type: Improvement (was: Bug) > Honor Producer Configuration "block.on.buffer.full" > --- > > Key: KAFKA-3236 > URL: https://issues.apache.org/jira/browse/KAFKA-3236 > Project: Kafka > Issue Type: Improvement > Components: producer >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves >Assignee: Thomas Graves > > In Kafka-0.9, "max.block.ms" is used to control how long the following > methods will block. > KafkaProducer.send() when >* Buffer is full >* Metadata is unavailable > KafkaProducer.partitionsFor() when >* Metadata is unavailable > However when "block.on.buffer.full" is set to false, "max.block.ms" is in > effect whenever a buffer is requested/allocated from the Producer BufferPool. > Instead it should throw a BufferExhaustedException without waiting for > "max.block.ms" > This is particulary useful if a producer application does not wish to block > at all on KafkaProducer.send() . We avoid waiting on KafkaProducer.send() > when metadata is unavailable by invoking send() only if the producer instance > has fetched the metadata for the topic in a different thread using the same > producer instance. However "max.block.ms" is still required to specify a > timeout for bootstrapping the metadata fetch. > We should resolve this limitation by decoupling "max.block.ms" and > "block.on.buffer.full". >* "max.block.ms" will be used exclusively for fetching metadata when > "block.on.buffer.full" = false (in pure non-blocking mode ) >* "max.block.ms" will be applicable to both fetching metadata as well as > buffer allocation when "block.on.buffer.full = true -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15150802#comment-15150802 ] Thomas Graves commented on KAFKA-3236: -- The reason I filed as a defect is because I think the documentation is confusing. Looking at the code I think its a bug with documentation and as you say this is actual improvement so I'll change it. > Honor Producer Configuration "block.on.buffer.full" > --- > > Key: KAFKA-3236 > URL: https://issues.apache.org/jira/browse/KAFKA-3236 > Project: Kafka > Issue Type: Bug > Components: producer >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves >Assignee: Thomas Graves > > In Kafka-0.9, "max.block.ms" is used to control how long the following > methods will block. > KafkaProducer.send() when >* Buffer is full >* Metadata is unavailable > KafkaProducer.partitionsFor() when >* Metadata is unavailable > However when "block.on.buffer.full" is set to false, "max.block.ms" is in > effect whenever a buffer is requested/allocated from the Producer BufferPool. > Instead it should throw a BufferExhaustedException without waiting for > "max.block.ms" > This is particulary useful if a producer application does not wish to block > at all on KafkaProducer.send() . We avoid waiting on KafkaProducer.send() > when metadata is unavailable by invoking send() only if the producer instance > has fetched the metadata for the topic in a different thread using the same > producer instance. However "max.block.ms" is still required to specify a > timeout for bootstrapping the metadata fetch. > We should resolve this limitation by decoupling "max.block.ms" and > "block.on.buffer.full". >* "max.block.ms" will be used exclusively for fetching metadata when > "block.on.buffer.full" = false (in pure non-blocking mode ) >* "max.block.ms" will be applicable to both fetching metadata as well as > buffer allocation when "block.on.buffer.full = true -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
Thomas Graves created KAFKA-3236: Summary: Honor Producer Configuration "block.on.buffer.full" Key: KAFKA-3236 URL: https://issues.apache.org/jira/browse/KAFKA-3236 Project: Kafka Issue Type: Bug Components: producer Affects Versions: 0.9.0.0 Reporter: Thomas Graves Assignee: Thomas Graves In Kafka-0.9, "max.block.ms" is used to control how long the following methods will block. KafkaProducer.send() when * Buffer is full * Metadata is unavailable KafkaProducer.partitionsFor() when * Metadata is unavailable However when "block.on.buffer.full" is set to false, "max.block.ms" is in effect whenever a buffer is requested/allocated from the Producer BufferPool. Instead it should throw a BufferExhaustedException without waiting for "max.block.ms" This is particulary useful if a producer application does not wish to block at all on KafkaProducer.send() . We avoid waiting on KafkaProducer.send() when metadata is unavailable by invoking send() only if the producer instance has fetched the metadata for the topic in a different thread using the same producer instance. However "max.block.ms" is still required to specify a timeout for bootstrapping the metadata fetch. We should resolve this limitation by decoupling "max.block.ms" and "block.on.buffer.full". * "max.block.ms" will be used exclusively for fetching metadata when "block.on.buffer.full" = false (in pure non-blocking mode ) * "max.block.ms" will be applicable to both fetching metadata as well as buffer allocation when "block.on.buffer.full = true -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (KAFKA-3110) can't set cluster acl for a user to CREATE topics without first creating a topic
[ https://issues.apache.org/jira/browse/KAFKA-3110?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Thomas Graves resolved KAFKA-3110. -- Resolution: Invalid > can't set cluster acl for a user to CREATE topics without first creating a > topic > > > Key: KAFKA-3110 > URL: https://issues.apache.org/jira/browse/KAFKA-3110 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves > > I started a new kafka cluster with security. I tried to give a user cluster > CREATE permissions so they could create topics: > kafka-acls.sh --authorizer-properties zookeeper.connect=host.com:2181 > --cluster --add --operation CREATE --allow-principal User:myuser > This failed with the error below and the broker ended up shutting down and > wouldn't restart without removing the zookeeper data. > @40005699398806bd170c org.I0Itec.zkclient.exception.ZkNoNodeException: > org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = > NoNode for /kafka-acl/Topic > To work around this you can first create any topic which creates the > zookeeper node and then after that you can give the user create permissions. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-3110) can't set cluster acl for a user to CREATE topics without first creating a topic
[ https://issues.apache.org/jira/browse/KAFKA-3110?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15123845#comment-15123845 ] Thomas Graves commented on KAFKA-3110: -- Tried to reproduce this again but couldn't. I must have done something wrong initially. > can't set cluster acl for a user to CREATE topics without first creating a > topic > > > Key: KAFKA-3110 > URL: https://issues.apache.org/jira/browse/KAFKA-3110 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves > > I started a new kafka cluster with security. I tried to give a user cluster > CREATE permissions so they could create topics: > kafka-acls.sh --authorizer-properties zookeeper.connect=host.com:2181 > --cluster --add --operation CREATE --allow-principal User:myuser > This failed with the error below and the broker ended up shutting down and > wouldn't restart without removing the zookeeper data. > @40005699398806bd170c org.I0Itec.zkclient.exception.ZkNoNodeException: > org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = > NoNode for /kafka-acl/Topic > To work around this you can first create any topic which creates the > zookeeper node and then after that you can give the user create permissions. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-3110) can't set cluster acl for a user to CREATE topics without first creating a topic
[ https://issues.apache.org/jira/browse/KAFKA-3110?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15117246#comment-15117246 ] Thomas Graves commented on KAFKA-3110: -- I was using 0.9 release so I'll give it a try with trunk. > can't set cluster acl for a user to CREATE topics without first creating a > topic > > > Key: KAFKA-3110 > URL: https://issues.apache.org/jira/browse/KAFKA-3110 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves > > I started a new kafka cluster with security. I tried to give a user cluster > CREATE permissions so they could create topics: > kafka-acls.sh --authorizer-properties zookeeper.connect=host.com:2181 > --cluster --add --operation CREATE --allow-principal User:myuser > This failed with the error below and the broker ended up shutting down and > wouldn't restart without removing the zookeeper data. > @40005699398806bd170c org.I0Itec.zkclient.exception.ZkNoNodeException: > org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = > NoNode for /kafka-acl/Topic > To work around this you can first create any topic which creates the > zookeeper node and then after that you can give the user create permissions. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-3110) can't set cluster acl for a user to CREATE topics without first creating a topic
Thomas Graves created KAFKA-3110: Summary: can't set cluster acl for a user to CREATE topics without first creating a topic Key: KAFKA-3110 URL: https://issues.apache.org/jira/browse/KAFKA-3110 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.9.0.0 Reporter: Thomas Graves I started a new kafka cluster with security. I tried to give a user cluster CREATE permissions so they could create topics: kafka-acls.sh --authorizer-properties zookeeper.connect=host.com:2181 --cluster --add --operation CREATE --allow-principal User:myuser This failed with the error below and the broker ended up shutting down and wouldn't restart without removing the zookeeper data. @40005699398806bd170c org.I0Itec.zkclient.exception.ZkNoNodeException: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /kafka-acl/Topic To work around this you can first create any topic which creates the zookeeper node and then after that you can give the user create permissions. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-3095) No documentation on format of sasl.kerberos.principal.to.local.rules
[ https://issues.apache.org/jira/browse/KAFKA-3095?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15096616#comment-15096616 ] Thomas Graves commented on KAFKA-3095: -- rules that worked for me to get a normal user tgraves (non-service principal) to properly authenticate: sasl.kerberos.principal.to.local.rules=RULE:[1:$1@$0](.*@MYDOMAIN.COM)s/@.*//,DEFAULT > No documentation on format of sasl.kerberos.principal.to.local.rules > > > Key: KAFKA-3095 > URL: https://issues.apache.org/jira/browse/KAFKA-3095 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves >Assignee: Thomas Graves > > The documentation talked about the config > sasl.kerberos.principal.to.local.rules and the format of the default but it > doesn't say what format the rules should be specified in. A description and > perhaps an example would be very useful here. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-3097) Acls for PrincipalType User are case sensitive
Thomas Graves created KAFKA-3097: Summary: Acls for PrincipalType User are case sensitive Key: KAFKA-3097 URL: https://issues.apache.org/jira/browse/KAFKA-3097 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.9.0.0 Reporter: Thomas Graves I gave a user acls for READ/WRITE but when I went to actually write to the topic failed with auth exception. I figured out it was due to me specifying the user as: user:tgraves rather then User:tgraves. Seems like It should either fail on assign or be case insensitive. The principal type of User should also probably be documented. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-3095) No documentation on format of sasl.kerberos.principal.to.local.rules
[ https://issues.apache.org/jira/browse/KAFKA-3095?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15096395#comment-15096395 ] Thomas Graves commented on KAFKA-3095: -- thanks. that is what I was figuring (same as hadoop) and in the process of trying. I can put up a PR when I have it working. I don't have permission to assign this to myself. > No documentation on format of sasl.kerberos.principal.to.local.rules > > > Key: KAFKA-3095 > URL: https://issues.apache.org/jira/browse/KAFKA-3095 > Project: Kafka > Issue Type: Bug > Components: core >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves > > The documentation talked about the config > sasl.kerberos.principal.to.local.rules and the format of the default but it > doesn't say what format the rules should be specified in. A description and > perhaps an example would be very useful here. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-3095) No documentation on format of sasl.kerberos.principal.to.local.rules
Thomas Graves created KAFKA-3095: Summary: No documentation on format of sasl.kerberos.principal.to.local.rules Key: KAFKA-3095 URL: https://issues.apache.org/jira/browse/KAFKA-3095 Project: Kafka Issue Type: Bug Components: core Affects Versions: 0.9.0.0 Reporter: Thomas Graves The documentation talked about the config sasl.kerberos.principal.to.local.rules and the format of the default but it doesn't say what format the rules should be specified in. A description and perhaps an example would be very useful here. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2808) Support auto.create.topics.enable with automatic WRITE permissions for creator
[ https://issues.apache.org/jira/browse/KAFKA-2808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15000880#comment-15000880 ] Thomas Graves commented on KAFKA-2808: -- Sorry just realized perhaps what you were thinking. You were saying if you had topic owner and auto create was on it would automatically set the owner to the creator and thus would also have WRITE permissions. > Support auto.create.topics.enable with automatic WRITE permissions for > creator > --- > > Key: KAFKA-2808 > URL: https://issues.apache.org/jira/browse/KAFKA-2808 > Project: Kafka > Issue Type: Improvement > Components: core >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves > > we have a user that wants to use the topic auto create functionality and > automatically have it give WRITE permissions so that they don't have to > explicitly create and grant acls ahead of time or make explicit call. > it seems like if you have auto.create.topics.enable enabled and the user has > CREATE acls we could automatically just give WRITE acls to the user who > creates the topic. Without that the auto create topics with acls doesn't add > much benefit. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2808) Support auto.create.topics.enable with automatic WRITE permissions for creator
[ https://issues.apache.org/jira/browse/KAFKA-2808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15000876#comment-15000876 ] Thomas Graves commented on KAFKA-2808: -- I'm sure there are cases you wouldn't want this so it could be a config. > Support auto.create.topics.enable with automatic WRITE permissions for > creator > --- > > Key: KAFKA-2808 > URL: https://issues.apache.org/jira/browse/KAFKA-2808 > Project: Kafka > Issue Type: Improvement > Components: core >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves > > we have a user that wants to use the topic auto create functionality and > automatically have it give WRITE permissions so that they don't have to > explicitly create and grant acls ahead of time or make explicit call. > it seems like if you have auto.create.topics.enable enabled and the user has > CREATE acls we could automatically just give WRITE acls to the user who > creates the topic. Without that the auto create topics with acls doesn't add > much benefit. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (KAFKA-2808) Support auto.create.topics.enable with automatic WRITE permissions for creator
[ https://issues.apache.org/jira/browse/KAFKA-2808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15000874#comment-15000874 ] Thomas Graves edited comment on KAFKA-2808 at 11/11/15 6:44 PM: I haven't looked a the pull request but I'm not sure that really solves this problem because I again would have to explicitly set it. I guess perhaps namespaces can help solve this. Or if we could do acls on wildcard topics. In this particular use case at least the topics will all have the same prefix but the suffix part is unknown and can change. What do you see as the problem with automatically giving WRITE permissions to the creator? If you trust them enough to CREATE the topic wouldn't it generally mean they should be allowed to write also. was (Author: tgraves): I haven't looked a the pull request but I'm not sure that really solves this problem because I again would have to explicitly set it. I guess perhaps namespaces can help solve this. Or if we could do acls on wildcard topics. In this particular use case at least the topics will all have the same prefix but the suffix part is unknown and can change. What do you see as the problem with automatically giving WRITE permissions to the creator? If you trust them enough to CREATE the topic would it generally mean they should be allowed to write also. > Support auto.create.topics.enable with automatic WRITE permissions for > creator > --- > > Key: KAFKA-2808 > URL: https://issues.apache.org/jira/browse/KAFKA-2808 > Project: Kafka > Issue Type: Improvement > Components: core >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves > > we have a user that wants to use the topic auto create functionality and > automatically have it give WRITE permissions so that they don't have to > explicitly create and grant acls ahead of time or make explicit call. > it seems like if you have auto.create.topics.enable enabled and the user has > CREATE acls we could automatically just give WRITE acls to the user who > creates the topic. Without that the auto create topics with acls doesn't add > much benefit. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2808) Support auto.create.topics.enable with automatic WRITE permissions for creator
[ https://issues.apache.org/jira/browse/KAFKA-2808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15000874#comment-15000874 ] Thomas Graves commented on KAFKA-2808: -- I haven't looked a the pull request but I'm not sure that really solves this problem because I again would have to explicitly set it. I guess perhaps namespaces can help solve this. Or if we could do acls on wildcard topics. In this particular use case at least the topics will all have the same prefix but the suffix part is unknown and can change. What do you see as the problem with automatically giving WRITE permissions to the creator? If you trust them enough to CREATE the topic would it generally mean they should be allowed to write also. > Support auto.create.topics.enable with automatic WRITE permissions for > creator > --- > > Key: KAFKA-2808 > URL: https://issues.apache.org/jira/browse/KAFKA-2808 > Project: Kafka > Issue Type: Improvement > Components: core >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves > > we have a user that wants to use the topic auto create functionality and > automatically have it give WRITE permissions so that they don't have to > explicitly create and grant acls ahead of time or make explicit call. > it seems like if you have auto.create.topics.enable enabled and the user has > CREATE acls we could automatically just give WRITE acls to the user who > creates the topic. Without that the auto create topics with acls doesn't add > much benefit. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-2808) Support auto.create.topics.enable with automatic WRITE permissions for creator
[ https://issues.apache.org/jira/browse/KAFKA-2808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15000828#comment-15000828 ] Thomas Graves commented on KAFKA-2808: -- ccd [~parth.brahmbhatt] > Support auto.create.topics.enable with automatic WRITE permissions for > creator > --- > > Key: KAFKA-2808 > URL: https://issues.apache.org/jira/browse/KAFKA-2808 > Project: Kafka > Issue Type: Improvement > Components: core >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves > > we have a user that wants to use the topic auto create functionality and > automatically have it give WRITE permissions so that they don't have to > explicitly create and grant acls ahead of time or make explicit call. > it seems like if you have auto.create.topics.enable enabled and the user has > CREATE acls we could automatically just give WRITE acls to the user who > creates the topic. Without that the auto create topics with acls doesn't add > much benefit. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-2808) Support auto.create.topics.enable with automatic WRITE permissions for creator
Thomas Graves created KAFKA-2808: Summary: Support auto.create.topics.enable with automatic WRITE permissions for creator Key: KAFKA-2808 URL: https://issues.apache.org/jira/browse/KAFKA-2808 Project: Kafka Issue Type: Improvement Components: core Affects Versions: 0.9.0.0 Reporter: Thomas Graves we have a user that wants to use the topic auto create functionality and automatically have it give WRITE permissions so that they don't have to explicitly create and grant acls ahead of time or make explicit call. it seems like if you have auto.create.topics.enable enabled and the user has CREATE acls we could automatically just give WRITE acls to the user who creates the topic. Without that the auto create topics with acls doesn't add much benefit. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-2754) Support defining ACLs at topic creation time
[ https://issues.apache.org/jira/browse/KAFKA-2754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Thomas Graves updated KAFKA-2754: - Description: With a secured kafka cluster we want to be able have certain users create topics and set the acls at the same time. We have a use case where topics will be dynamically created and we don't know the names ahead of time so it would be really nice to create the topic and set the acls at the same time. We want to be able to do this programmatically. Ideally it would be nice to have a way to set the acls with the auto create topics enabled also but that might be a separate jira. was: With a secured kafka cluster we want to be able have certain users create topics and set the acls at the same time. We have a use case where topics will be dynamically created and we don't know the names ahead of time so it would be really nice to create the topic and set the acls at the same time. Ideally it would be nice to have a way to set the acls with the auto create topics enabled also but that might be a separate jira. > Support defining ACLs at topic creation time > > > Key: KAFKA-2754 > URL: https://issues.apache.org/jira/browse/KAFKA-2754 > Project: Kafka > Issue Type: Improvement >Affects Versions: 0.9.0.0 >Reporter: Thomas Graves > > With a secured kafka cluster we want to be able have certain users create > topics and set the acls at the same time. We have a use case where topics > will be dynamically created and we don't know the names ahead of time so it > would be really nice to create the topic and set the acls at the same time. > We want to be able to do this programmatically. > Ideally it would be nice to have a way to set the acls with the auto create > topics enabled also but that might be a separate jira. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-2754) Support defining ACLs at topic creation time
Thomas Graves created KAFKA-2754: Summary: Support defining ACLs at topic creation time Key: KAFKA-2754 URL: https://issues.apache.org/jira/browse/KAFKA-2754 Project: Kafka Issue Type: Improvement Affects Versions: 0.9.0.0 Reporter: Thomas Graves With a secured kafka cluster we want to be able have certain users create topics and set the acls at the same time. We have a use case where topics will be dynamically created and we don't know the names ahead of time so it would be really nice to create the topic and set the acls at the same time. Ideally it would be nice to have a way to set the acls with the auto create topics enabled also but that might be a separate jira. -- This message was sent by Atlassian JIRA (v6.3.4#6332)