Review Request 29379: Patch for KAFKA-1788
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29379/ --- Review request for kafka. Bugs: KAFKA-1788 https://issues.apache.org/jira/browse/KAFKA-1788 Repository: kafka Description --- KAFKA-1788: Adding configuration for batch.expiration.ms to ensure batches for partitions that has no leader do not stay in accumulator forever. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java f61efb35db7e0de590556e6a94a7b5cb850cdae9 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java a893d88c2f4e21509b6c70d6817b4b2cdd0fd657 clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java c15485d1af304ef53691d478f113f332fe67af77 clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java dd0af8aee98abed5d4a0dc50989e37888bb353fe clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java 2c9932401d573549c40f16fda8c4e3e11309cb85 clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java ef2ca65cabe97b909f17b62027a1bb06827e88fe Diff: https://reviews.apache.org/r/29379/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 29379: Patch for KAFKA-1788
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29379/ --- (Updated Dec. 23, 2014, 8:44 p.m.) Review request for kafka. Changes --- KAFKA-1788: Adding configuration for batch.expiration.ms to ensure batches for partitions that has no leader do not stay in accumulator forever. Bugs: KAFKA-1788 https://issues.apache.org/jira/browse/KAFKA-1788 Repository: kafka Description --- KAFKA-1788: Adding configuration for batch.expiration.ms to ensure batches for partitions that has no leader do not stay in accumulator forever. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java f61efb35db7e0de590556e6a94a7b5cb850cdae9 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java a893d88c2f4e21509b6c70d6817b4b2cdd0fd657 clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java c15485d1af304ef53691d478f113f332fe67af77 clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java dd0af8aee98abed5d4a0dc50989e37888bb353fe clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java 2c9932401d573549c40f16fda8c4e3e11309cb85 clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java ef2ca65cabe97b909f17b62027a1bb06827e88fe Diff: https://reviews.apache.org/r/29379/diff/ Testing (updated) --- Unit test added. Thanks, Parth Brahmbhatt
Review Request 29467: Patch for KAFKA-1660
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/ --- Review request for kafka. Bugs: KAFKA-1660 https://issues.apache.org/jira/browse/KAFKA-1660 Repository: kafka Description --- KAFKA-1660: Adding tryClose(timeoutMillis) to producer. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java f61efb35db7e0de590556e6a94a7b5cb850cdae9 clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 34624c3b7a1f28735ab6c63cc9e18a410e87e63c clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 5baa6062bd9ba8a7d38058856ed2d831fae491f0 Diff: https://reviews.apache.org/r/29467/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 29467: Patch for KAFKA-1660
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/ --- (Updated Dec. 29, 2014, 10:52 p.m.) Review request for kafka. Changes --- Adding tryClose(timeoutMillis) to producer. Bugs: KAFKA-1660 https://issues.apache.org/jira/browse/KAFKA-1660 Repository: kafka Description --- KAFKA-1660: Adding tryClose(timeoutMillis) to producer. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java f61efb35db7e0de590556e6a94a7b5cb850cdae9 clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 34624c3b7a1f28735ab6c63cc9e18a410e87e63c clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 5baa6062bd9ba8a7d38058856ed2d831fae491f0 Diff: https://reviews.apache.org/r/29467/diff/ Testing (updated) --- existing unit tests passed. Thanks, Parth Brahmbhatt
Review Request 29468: Patch for KAFKA-1805
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- Review request for kafka. Bugs: KAFKA-1805 https://issues.apache.org/jira/browse/KAFKA-1805 Repository: kafka Description --- KAFKA-1805: adding equals and hashcode methods to ProducerRecord. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java 065d4e6c6a4966ac216e98696782e2714044df29 core/src/main/scala/kafka/consumer/ConsumerConnector.scala 62c0686e816d2888772d5a911becf625eedee397 core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala e991d2187d03241f639eeaf6769fb59c8c99664c core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala 9baad34a9793e5067d11289ece2154ba87b388af core/src/main/scala/kafka/tools/MirrorMaker.scala 53cb16c2949e0ac36a0e943564fc9fc9b4c84caa core/src/main/scala/kafka/utils/DoublyLinkedList.scala PRE-CREATION core/src/test/scala/unit/kafka/utils/UtilsTest.scala 066553cad290c3d3821537a964c7d713c122d9fc Diff: https://reviews.apache.org/r/29468/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 29468: Patch for KAFKA-1805
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Dec. 30, 2014, 12:37 a.m.) Review request for kafka. Bugs: KAFKA-1805, KAFKA-1905 and KAFKA-42 https://issues.apache.org/jira/browse/KAFKA-1805 https://issues.apache.org/jira/browse/KAFKA-1905 https://issues.apache.org/jira/browse/KAFKA-42 Repository: kafka Description --- KAFKA-1805: adding equals and hashcode methods to ProducerRecord. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java 065d4e6c6a4966ac216e98696782e2714044df29 Diff: https://reviews.apache.org/r/29468/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 29468: Patch for KAFKA-1805
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:34 p.m.) Review request for kafka. Bugs: KAFKA-1805, KAFKA-1905 and KAFKA-42 https://issues.apache.org/jira/browse/KAFKA-1805 https://issues.apache.org/jira/browse/KAFKA-1905 https://issues.apache.org/jira/browse/KAFKA-42 Repository: kafka Description (updated) --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1805 Handling the case where al the fields in ProducerRecord can be null. Adding toString back. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java 065d4e6c6a4966ac216e98696782e2714044df29 Diff: https://reviews.apache.org/r/29468/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 29468: Patch for KAFKA-1805
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:30 p.m.) Review request for kafka. Bugs: KAFKA-1805, KAFKA-1905 and KAFKA-42 https://issues.apache.org/jira/browse/KAFKA-1805 https://issues.apache.org/jira/browse/KAFKA-1905 https://issues.apache.org/jira/browse/KAFKA-42 Repository: kafka Description (updated) --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1805 Handling the case where al the fields in ProducerRecord can be null. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java 065d4e6c6a4966ac216e98696782e2714044df29 Diff: https://reviews.apache.org/r/29468/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 29468: Patch for KAFKA-1805
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:37 p.m.) Review request for kafka. Bugs: KAFKA-1805, KAFKA-1905 and KAFKA-42 https://issues.apache.org/jira/browse/KAFKA-1805 https://issues.apache.org/jira/browse/KAFKA-1905 https://issues.apache.org/jira/browse/KAFKA-42 Repository: kafka Description --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1805 Handling the case where al the fields in ProducerRecord can be null. Adding toString back. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java 065d4e6c6a4966ac216e98696782e2714044df29 Diff: https://reviews.apache.org/r/29468/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 29468: Patch for KAFKA-1805
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/#review72031 --- clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java https://reviews.apache.org/r/29468/#comment117965 not intentional, removed. clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java https://reviews.apache.org/r/29468/#comment117966 handling null for all fields now. clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java https://reviews.apache.org/r/29468/#comment117967 nulls are handled now. - Parth Brahmbhatt On Feb. 11, 2015, 10:53 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:53 p.m.) Review request for kafka. Bugs: KAFKA-1805, KAFKA-1905 and KAFKA-42 https://issues.apache.org/jira/browse/KAFKA-1805 https://issues.apache.org/jira/browse/KAFKA-1905 https://issues.apache.org/jira/browse/KAFKA-42 Repository: kafka Description --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1805 Handling the case where al the fields in ProducerRecord can be null. Adding toString back. Added unit test for eqauls and hashcode. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java 065d4e6c6a4966ac216e98696782e2714044df29 clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java PRE-CREATION Diff: https://reviews.apache.org/r/29468/diff/ Testing --- Unit tests added. Thanks, Parth Brahmbhatt
Re: Review Request 29468: Patch for KAFKA-1805
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:53 p.m.) Review request for kafka. Bugs: KAFKA-1805, KAFKA-1905 and KAFKA-42 https://issues.apache.org/jira/browse/KAFKA-1805 https://issues.apache.org/jira/browse/KAFKA-1905 https://issues.apache.org/jira/browse/KAFKA-42 Repository: kafka Description --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1805 Handling the case where al the fields in ProducerRecord can be null. Adding toString back. Added unit test for eqauls and hashcode. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java 065d4e6c6a4966ac216e98696782e2714044df29 clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java PRE-CREATION Diff: https://reviews.apache.org/r/29468/diff/ Testing (updated) --- Unit tests added. Thanks, Parth Brahmbhatt
Re: Review Request 29468: Patch for KAFKA-1805
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:49 p.m.) Review request for kafka. Bugs: KAFKA-1805, KAFKA-1905 and KAFKA-42 https://issues.apache.org/jira/browse/KAFKA-1805 https://issues.apache.org/jira/browse/KAFKA-1905 https://issues.apache.org/jira/browse/KAFKA-42 Repository: kafka Description (updated) --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1805 Handling the case where al the fields in ProducerRecord can be null. Adding toString back. Added unit test for eqauls and hashcode. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java 065d4e6c6a4966ac216e98696782e2714044df29 clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java PRE-CREATION Diff: https://reviews.apache.org/r/29468/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 29468: Patch for KAFKA-1805
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:37 p.m.) Review request for kafka. Bugs: KAFKA-1805, KAFKA-1905 and KAFKA-42 https://issues.apache.org/jira/browse/KAFKA-1805 https://issues.apache.org/jira/browse/KAFKA-1905 https://issues.apache.org/jira/browse/KAFKA-42 Repository: kafka Description --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1805 Handling the case where al the fields in ProducerRecord can be null. Adding toString back. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java 065d4e6c6a4966ac216e98696782e2714044df29 Diff: https://reviews.apache.org/r/29468/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 29468: Patch for KAFKA-1805
On Feb. 11, 2015, 10:35 p.m., Gwen Shapira wrote: Can you add unit tests? Added uni tests. - Parth --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/#review72032 --- On Feb. 11, 2015, 10:49 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29468/ --- (Updated Feb. 11, 2015, 10:49 p.m.) Review request for kafka. Bugs: KAFKA-1805, KAFKA-1905 and KAFKA-42 https://issues.apache.org/jira/browse/KAFKA-1805 https://issues.apache.org/jira/browse/KAFKA-1905 https://issues.apache.org/jira/browse/KAFKA-42 Repository: kafka Description --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1805 Handling the case where al the fields in ProducerRecord can be null. Adding toString back. Added unit test for eqauls and hashcode. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java 065d4e6c6a4966ac216e98696782e2714044df29 clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java PRE-CREATION Diff: https://reviews.apache.org/r/29468/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 29467: Patch for KAFKA-1660
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/ --- (Updated Feb. 18, 2015, 12:41 a.m.) Review request for kafka. Bugs: KAFKA-1660 https://issues.apache.org/jira/browse/KAFKA-1660 Repository: kafka Description --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1660 Conflicts: clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 1fd6917c8a5131254c740abad7f7228a47e3628c clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 84530f2b948f9abd74203db48707e490dd9c81a5 clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 17fe541588d462c68c33f6209717cc4015e9b62f clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java ed9c63a6679e3aaf83d19fde19268553a4c107c2 Diff: https://reviews.apache.org/r/29467/diff/ Testing --- existing unit tests passed. Thanks, Parth Brahmbhatt
Re: Review Request 29467: Patch for KAFKA-1660
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/ --- (Updated Feb. 18, 2015, 12:36 a.m.) Review request for kafka. Bugs: KAFKA-1660 https://issues.apache.org/jira/browse/KAFKA-1660 Repository: kafka Description (updated) --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1660 Conflicts: clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 1fd6917c8a5131254c740abad7f7228a47e3628c clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 84530f2b948f9abd74203db48707e490dd9c81a5 clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 17fe541588d462c68c33f6209717cc4015e9b62f clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java ed9c63a6679e3aaf83d19fde19268553a4c107c2 Diff: https://reviews.apache.org/r/29467/diff/ Testing --- existing unit tests passed. Thanks, Parth Brahmbhatt
Re: Review Request 29379: Patch for KAFKA-1788
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29379/ --- (Updated Jan. 6, 2015, 6:44 p.m.) Review request for kafka. Bugs: KAFKA-1788 https://issues.apache.org/jira/browse/KAFKA-1788 Repository: kafka Description --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1788 KAFKA-1788: addressed Ewen's comments. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java f61efb35db7e0de590556e6a94a7b5cb850cdae9 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java a893d88c2f4e21509b6c70d6817b4b2cdd0fd657 clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java c15485d1af304ef53691d478f113f332fe67af77 clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java dd0af8aee98abed5d4a0dc50989e37888bb353fe clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java 2c9932401d573549c40f16fda8c4e3e11309cb85 clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java ef2ca65cabe97b909f17b62027a1bb06827e88fe Diff: https://reviews.apache.org/r/29379/diff/ Testing --- Unit test added. Thanks, Parth Brahmbhatt
Re: Review Request 29379: Patch for KAFKA-1788
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29379/#review66879 --- clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java https://reviews.apache.org/r/29379/#comment110566 pulled up the synchronized dequeue block. clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java https://reviews.apache.org/r/29379/#comment110567 yes.replaced with batchExpirationMillis. clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java https://reviews.apache.org/r/29379/#comment110569 sender.completeBatch() is only called as part of produce response handling or disconnect. Both of which will never be invoked when there is no broker. I could add sender as a member of record accumulator or pass it as the callback arg as part of the ready() method. All of which is too hecky. Let me know if you see some other alternative. clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java https://reviews.apache.org/r/29379/#comment110570 done. - Parth Brahmbhatt On Jan. 6, 2015, 6:42 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29379/ --- (Updated Jan. 6, 2015, 6:42 p.m.) Review request for kafka. Bugs: KAFKA-1788 https://issues.apache.org/jira/browse/KAFKA-1788 Repository: kafka Description --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1788 KAFKA-1788: addressed Ewen's comments. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java f61efb35db7e0de590556e6a94a7b5cb850cdae9 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java a893d88c2f4e21509b6c70d6817b4b2cdd0fd657 clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java c15485d1af304ef53691d478f113f332fe67af77 clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java dd0af8aee98abed5d4a0dc50989e37888bb353fe clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java 2c9932401d573549c40f16fda8c4e3e11309cb85 clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java ef2ca65cabe97b909f17b62027a1bb06827e88fe Diff: https://reviews.apache.org/r/29379/diff/ Testing --- Unit test added. Thanks, Parth Brahmbhatt
Re: Review Request 29379: Patch for KAFKA-1788
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29379/ --- (Updated Jan. 6, 2015, 6:42 p.m.) Review request for kafka. Bugs: KAFKA-1788 https://issues.apache.org/jira/browse/KAFKA-1788 Repository: kafka Description (updated) --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1788 KAFKA-1788: addressed Ewen's comments. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java f61efb35db7e0de590556e6a94a7b5cb850cdae9 clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java a893d88c2f4e21509b6c70d6817b4b2cdd0fd657 clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java c15485d1af304ef53691d478f113f332fe67af77 clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java dd0af8aee98abed5d4a0dc50989e37888bb353fe clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java 2c9932401d573549c40f16fda8c4e3e11309cb85 clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java ef2ca65cabe97b909f17b62027a1bb06827e88fe Diff: https://reviews.apache.org/r/29379/diff/ Testing --- Unit test added. Thanks, Parth Brahmbhatt
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Hi Michael, Thanks for taking the time to review. Currently I did not plan on adding “Deny” but I guess it can’t hurt except for adding more constructs would probably make the acls more complex. When a topic is created with no acls provided , I was planning to add a default ACL which would allow access to everyone from all hosts. I am assuming you are referring to principal in Acl and acls were supposed to be provided in property files, stored in zk so I thought it is better to just refer to a string. We will always be using session.principal.getName to get the actual principal name. Thanks Parth On 3/18/15, 2:20 PM, Michael Herstine mherst...@linkedin.com.INVALID wrote: Hi Parth, Thanks! A few questions: 1. Do you want to permit rules in your ACLs that DENY access as well as ALLOW? This can be handy setting up rules that have exceptions. E.g. “Allow principal P to READ resource R from all hosts” with “Deny principal P READ access to resource R from host H1” in combination would allow P to READ R from all hosts *except* H1. 2. When a topic is newly created, will there be an ACL created for it? If not, would that not deny subsequent access to it? (nit) Maybe use Principal instead of String to represent principals? On 3/9/15, 11:48 AM, Don Bosco Durai bo...@apache.org wrote: Parth Overall it is looking good. Couple of questionsŠ - Can you give an example how the policies will look like in the default implementation? - In the operations, can we support ³CONNECT² also? This can be used during Session connection - Regarding access control for ³Topic Creation², since we can¹t do it on the server side, can we de-scope it for? And plan it as a future feature request? Thanks Bosco On 3/6/15, 8:10 AM, Harsha ka...@harsha.io wrote: Hi Parth, Thanks for putting this together. Overall it looks good to me. Although AdminUtils is a concern KIP-4 can probably fix that part. Thanks, Harsha On Thu, Mar 5, 2015, at 10:39 AM, Parth Brahmbhatt wrote: Forgot to add links to wiki and jira. Link to wiki: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorizatio n + Interface Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 Thanks Parth From: Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com Date: Thursday, March 5, 2015 at 10:33 AM To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: [DISCUSS] KIP-11- Authorization design for kafka security Hi, KIP-11 is open for discussion , I have updated the wiki with the design and open questions. Thanks Parth
Review Request 32251: Patch for KAFKA-2032
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32251/ --- Review request for kafka. Bugs: KAFKA-2032 https://issues.apache.org/jira/browse/KAFKA-2032 Repository: kafka Description --- KAFKA-2032: Add validation for partition.assignment.strategy. Diffs - clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 42c72198a0325e234cf1d428b687663099de884e Diff: https://reviews.apache.org/r/32251/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 32251: Patch for KAFKA-2032
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32251/ --- (Updated March 19, 2015, 6:47 p.m.) Review request for kafka. Bugs: KAFKA-2032 https://issues.apache.org/jira/browse/KAFKA-2032 Repository: kafka Description --- KAFKA-2032: Add validation for partition.assignment.strategy. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 42c72198a0325e234cf1d428b687663099de884e core/src/main/scala/kafka/consumer/ConsumerConfig.scala 9ebbee6c16dc83767297c729d2d74ebbd063a993 Diff: https://reviews.apache.org/r/32251/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 32251: Patch for KAFKA-2032
On March 19, 2015, 6:52 p.m., Jason Rosenberg wrote: core/src/main/scala/kafka/consumer/ConsumerConfig.scala, line 90 https://reviews.apache.org/r/32251/diff/4/?file=900301#file900301line90 Should this be called 'validatePartitionAssignmentStrategy'? At any rate 'Stretegy' is mispelled there. Fixed. - Parth --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32251/#review77081 --- On March 19, 2015, 7:19 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32251/ --- (Updated March 19, 2015, 7:19 p.m.) Review request for kafka. Bugs: KAFKA-2032 https://issues.apache.org/jira/browse/KAFKA-2032 Repository: kafka Description --- KAFKA-2032: Add validation for partition.assignment.strategy. Diffs - clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 42c72198a0325e234cf1d428b687663099de884e core/src/main/scala/kafka/consumer/ConsumerConfig.scala 9ebbee6c16dc83767297c729d2d74ebbd063a993 Diff: https://reviews.apache.org/r/32251/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 32251: Patch for KAFKA-2032
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32251/ --- (Updated March 19, 2015, 7:19 p.m.) Review request for kafka. Bugs: KAFKA-2032 https://issues.apache.org/jira/browse/KAFKA-2032 Repository: kafka Description --- KAFKA-2032: Add validation for partition.assignment.strategy. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 42c72198a0325e234cf1d428b687663099de884e core/src/main/scala/kafka/consumer/ConsumerConfig.scala 9ebbee6c16dc83767297c729d2d74ebbd063a993 Diff: https://reviews.apache.org/r/32251/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 32251: Patch for KAFKA-2032
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32251/ --- (Updated March 19, 2015, 6:42 p.m.) Review request for kafka. Bugs: KAFKA-2032 https://issues.apache.org/jira/browse/KAFKA-2032 Repository: kafka Description --- KAFKA-2032: Add validation for partition.assignment.strategy. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 42c72198a0325e234cf1d428b687663099de884e core/src/main/scala/kafka/consumer/ConsumerConfig.scala 9ebbee6c16dc83767297c729d2d74ebbd063a993 Diff: https://reviews.apache.org/r/32251/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 32251: Patch for KAFKA-2032
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32251/ --- (Updated March 19, 2015, 6:44 p.m.) Review request for kafka. Bugs: KAFKA-2032 https://issues.apache.org/jira/browse/KAFKA-2032 Repository: kafka Description --- KAFKA-2032: Add validation for partition.assignment.strategy. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java 42c72198a0325e234cf1d428b687663099de884e core/src/main/scala/kafka/consumer/ConsumerConfig.scala 9ebbee6c16dc83767297c729d2d74ebbd063a993 Diff: https://reviews.apache.org/r/32251/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: [VOTE] KIP-7 Security - IP Filtering
I can confirm that KAFKA-1688 will cover this use case. Please go over https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+In terface and let me know if you think there is a different use case being covered by KIP-7. Thanks Parth On 3/20/15, 9:26 AM, Jun Rao j...@confluent.io wrote: Yes, we can discuss the implementation separately. As for the proposal itself, have you looked at KAFKA-1688? Could this just be a special case for authorization and be included there? Thanks, Jun On Wed, Mar 18, 2015 at 6:26 PM, Jeff Holoman jholo...@cloudera.com wrote: One other thought. Does the timing of the implementation (or lack thereof) affect the proposal? It seems like the question you are asking is an implementation detail in terms of when the work would be done. If there isn't really support for the KIP that's ok, just wanting to make sure we are segmenting the vote for the KIP from concerns about implementation timing. Thanks! Jeff On Wed, Mar 18, 2015 at 9:22 PM, Jeff Holoman jholo...@cloudera.com wrote: Hey Jun thanks for the comment. Is the plan to re-factor the SocketServer implementation significantly? The current check is just in the acceptor. Does this change with the refactor? Thanks Jeff On Wed, Mar 18, 2015 at 7:25 PM, Jun Rao j...@confluent.io wrote: The proposal sounds reasonable. Timing wise, since we plan to refactor the network layer code in the broker, perhaps this can wait until KAFKA-1928 is done? Thanks, Jun On Tue, Mar 17, 2015 at 6:56 AM, Jeff Holoman jholo...@cloudera.com wrote: bump On Tue, Mar 3, 2015 at 8:12 PM, Jeff Holoman jholo...@cloudera.com wrote: Guozhang, The way the patch is implemented, the check is done in the acceptor thread accept() method of the Socket Server, just before connectionQuotas. Thanks Jeff On Tue, Mar 3, 2015 at 7:59 PM, Guozhang Wang wangg...@gmail.com wrote: Jeff, I am wondering if the IP filtering rule can be enforced at the socket server level instead of the Kafka API level? Guozhang On Tue, Mar 3, 2015 at 2:24 PM, Jiangjie Qin j...@linkedin.com.invalid wrote: +1 (non-binding) On 3/3/15, 1:17 PM, Gwen Shapira gshap...@cloudera.com wrote: +1 (non-binding) On Tue, Mar 3, 2015 at 12:44 PM, Jeff Holoman jholo...@cloudera.com wrote: Details in the wiki. https://cwiki.apache.org/confluence/display/KAFKA/KIP-7+-+Security+-+IP+F iltering -- Jeff Holoman Systems Engineer -- -- Guozhang -- Jeff Holoman Systems Engineer -- Jeff Holoman Systems Engineer -- Jeff Holoman Systems Engineer -- Jeff Holoman Systems Engineer
Re: [VOTE] KIP-7 Security - IP Filtering
I am not entirely sure what you mean by integrating KIP-7 work with KAFKA-1688. Wouldn¹t the work done as part of KIP-7 become obsolete once KAFKA-1688 is done? Multiple ways of controlling these authorization just seems extra configuration that will confuse admins/users. If timing is the only issue don¹t you think its better to focus our energy on getting 1688 done faster which seem to be the longer term goal anyways? Thanks Parth On 3/20/15, 10:28 AM, Jeff Holoman jholo...@cloudera.com wrote: Hey Jun, The intent was for the same functionality to be utilized when 1688 is done, as mentioned in the KIP: The broader security initiative http://kafka-1682/ will add more robust controls for these types of environments, and this proposal could be integrated with that work at the appropriate time. This is also the specific request of a large financial services company. I don't think including the functionality now (as it's relatively simple) would preclude integration into 1688. At that point the implementation of the check might change, but as it's a broker config, there shouldn't be concerns about backward compatibility. Hope that helps Thanks Jeff On Fri, Mar 20, 2015 at 12:26 PM, Jun Rao j...@confluent.io wrote: Yes, we can discuss the implementation separately. As for the proposal itself, have you looked at KAFKA-1688? Could this just be a special case for authorization and be included there? Thanks, Jun On Wed, Mar 18, 2015 at 6:26 PM, Jeff Holoman jholo...@cloudera.com wrote: One other thought. Does the timing of the implementation (or lack thereof) affect the proposal? It seems like the question you are asking is an implementation detail in terms of when the work would be done. If there isn't really support for the KIP that's ok, just wanting to make sure we are segmenting the vote for the KIP from concerns about implementation timing. Thanks! Jeff On Wed, Mar 18, 2015 at 9:22 PM, Jeff Holoman jholo...@cloudera.com wrote: Hey Jun thanks for the comment. Is the plan to re-factor the SocketServer implementation significantly? The current check is just in the acceptor. Does this change with the refactor? Thanks Jeff On Wed, Mar 18, 2015 at 7:25 PM, Jun Rao j...@confluent.io wrote: The proposal sounds reasonable. Timing wise, since we plan to refactor the network layer code in the broker, perhaps this can wait until KAFKA-1928 is done? Thanks, Jun On Tue, Mar 17, 2015 at 6:56 AM, Jeff Holoman jholo...@cloudera.com wrote: bump On Tue, Mar 3, 2015 at 8:12 PM, Jeff Holoman jholo...@cloudera.com wrote: Guozhang, The way the patch is implemented, the check is done in the acceptor thread accept() method of the Socket Server, just before connectionQuotas. Thanks Jeff On Tue, Mar 3, 2015 at 7:59 PM, Guozhang Wang wangg...@gmail.com wrote: Jeff, I am wondering if the IP filtering rule can be enforced at the socket server level instead of the Kafka API level? Guozhang On Tue, Mar 3, 2015 at 2:24 PM, Jiangjie Qin j...@linkedin.com.invalid wrote: +1 (non-binding) On 3/3/15, 1:17 PM, Gwen Shapira gshap...@cloudera.com wrote: +1 (non-binding) On Tue, Mar 3, 2015 at 12:44 PM, Jeff Holoman jholo...@cloudera.com wrote: Details in the wiki. https://cwiki.apache.org/confluence/display/KAFKA/KIP-7+-+Security+-+IP+F iltering -- Jeff Holoman Systems Engineer -- -- Guozhang -- Jeff Holoman Systems Engineer -- Jeff Holoman Systems Engineer -- Jeff Holoman Systems Engineer -- Jeff Holoman Systems Engineer -- Jeff Holoman Systems Engineer
Re: [VOTE] KIP-7 Security - IP Filtering
I am guessing in your last reply you meant KIP-11. And yes, I think KIP-11 subsumed KIP-7 so if we can finish KIP-11 we should not need KIP=7 but I will let Jeff confirm that, Thanks Parth On 3/20/15, 2:32 PM, Jun Rao j...@confluent.io wrote: Right, if this KIP is subsumed by KIP-7, perhaps we just need to wait until KIP-7 is done? If we add the small change now, we will have to worry about migrating existing users and deprecating some configs when KIP-7 is done. Thanks, Jun On Fri, Mar 20, 2015 at 10:36 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: I am not entirely sure what you mean by integrating KIP-7 work with KAFKA-1688. Wouldn¹t the work done as part of KIP-7 become obsolete once KAFKA-1688 is done? Multiple ways of controlling these authorization just seems extra configuration that will confuse admins/users. If timing is the only issue don¹t you think its better to focus our energy on getting 1688 done faster which seem to be the longer term goal anyways? Thanks Parth On 3/20/15, 10:28 AM, Jeff Holoman jholo...@cloudera.com wrote: Hey Jun, The intent was for the same functionality to be utilized when 1688 is done, as mentioned in the KIP: The broader security initiative http://kafka-1682/ will add more robust controls for these types of environments, and this proposal could be integrated with that work at the appropriate time. This is also the specific request of a large financial services company. I don't think including the functionality now (as it's relatively simple) would preclude integration into 1688. At that point the implementation of the check might change, but as it's a broker config, there shouldn't be concerns about backward compatibility. Hope that helps Thanks Jeff On Fri, Mar 20, 2015 at 12:26 PM, Jun Rao j...@confluent.io wrote: Yes, we can discuss the implementation separately. As for the proposal itself, have you looked at KAFKA-1688? Could this just be a special case for authorization and be included there? Thanks, Jun On Wed, Mar 18, 2015 at 6:26 PM, Jeff Holoman jholo...@cloudera.com wrote: One other thought. Does the timing of the implementation (or lack thereof) affect the proposal? It seems like the question you are asking is an implementation detail in terms of when the work would be done. If there isn't really support for the KIP that's ok, just wanting to make sure we are segmenting the vote for the KIP from concerns about implementation timing. Thanks! Jeff On Wed, Mar 18, 2015 at 9:22 PM, Jeff Holoman jholo...@cloudera.com wrote: Hey Jun thanks for the comment. Is the plan to re-factor the SocketServer implementation significantly? The current check is just in the acceptor. Does this change with the refactor? Thanks Jeff On Wed, Mar 18, 2015 at 7:25 PM, Jun Rao j...@confluent.io wrote: The proposal sounds reasonable. Timing wise, since we plan to refactor the network layer code in the broker, perhaps this can wait until KAFKA-1928 is done? Thanks, Jun On Tue, Mar 17, 2015 at 6:56 AM, Jeff Holoman jholo...@cloudera.com wrote: bump On Tue, Mar 3, 2015 at 8:12 PM, Jeff Holoman jholo...@cloudera.com wrote: Guozhang, The way the patch is implemented, the check is done in the acceptor thread accept() method of the Socket Server, just before connectionQuotas. Thanks Jeff On Tue, Mar 3, 2015 at 7:59 PM, Guozhang Wang wangg...@gmail.com wrote: Jeff, I am wondering if the IP filtering rule can be enforced at the socket server level instead of the Kafka API level? Guozhang On Tue, Mar 3, 2015 at 2:24 PM, Jiangjie Qin j...@linkedin.com.invalid wrote: +1 (non-binding) On 3/3/15, 1:17 PM, Gwen Shapira gshap...@cloudera.com wrote: +1 (non-binding) On Tue, Mar 3, 2015 at 12:44 PM, Jeff Holoman jholo...@cloudera.com wrote: Details in the wiki. https://cwiki.apache.org/confluence/display/KAFKA/KIP-7+-+Security+-+IP+F iltering -- Jeff Holoman Systems Engineer -- -- Guozhang -- Jeff Holoman Systems Engineer -- Jeff Holoman Systems Engineer -- Jeff Holoman Systems Engineer -- Jeff Holoman Systems Engineer -- Jeff Holoman Systems Engineer
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Hi Bosco, Thanks for taking the time to review. I will update the doc with a policy example and I will add CONNECT as an operation. For Admin APIs, I agree the best thing to do right now is just wait for KIP-4 to be submitted. I will update the doc to reflect the same. Thanks Parth On 3/9/15, 11:48 AM, Don Bosco Durai bo...@apache.org wrote: Parth Overall it is looking good. Couple of questionsŠ - Can you give an example how the policies will look like in the default implementation? - In the operations, can we support ³CONNECT² also? This can be used during Session connection - Regarding access control for ³Topic Creation², since we can¹t do it on the server side, can we de-scope it for? And plan it as a future feature request? Thanks Bosco On 3/6/15, 8:10 AM, Harsha ka...@harsha.io wrote: Hi Parth, Thanks for putting this together. Overall it looks good to me. Although AdminUtils is a concern KIP-4 can probably fix that part. Thanks, Harsha On Thu, Mar 5, 2015, at 10:39 AM, Parth Brahmbhatt wrote: Forgot to add links to wiki and jira. Link to wiki: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization + Interface Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 Thanks Parth From: Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com Date: Thursday, March 5, 2015 at 10:33 AM To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: [DISCUSS] KIP-11- Authorization design for kafka security Hi, KIP-11 is open for discussion , I have updated the wiki with the design and open questions. Thanks Parth
Review Request 32460: Patch for KAFKA-2032
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32460/ --- Review request for kafka. Bugs: KAFKA-2032 https://issues.apache.org/jira/browse/KAFKA-2032 Repository: kafka Description --- KAFKA:2032 added topic config cache. Diffs - core/src/main/scala/kafka/server/KafkaServer.scala dddef938fabae157ed8644536eb1a2f329fb42b7 core/src/main/scala/kafka/server/TopicConfigCache.scala PRE-CREATION core/src/main/scala/kafka/server/TopicConfigManager.scala 47295d40131492aaac786273819b7bc6e22e5486 core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala 93182aeb342729d420d2e7d59a1035994164b7db core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala PRE-CREATION Diff: https://reviews.apache.org/r/32460/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Hi all, I have modified the KIP to reflect the recent change request from the reviewers. I have been working on the code and I have the server side code for authorization ready. I am now modifying the command line utilities. I would really appreciate if some of the committers can spend sometime to review the KIP so we can make progress on this. Thanks Parth On 3/18/15, 2:20 PM, Michael Herstine mherst...@linkedin.com.INVALID wrote: Hi Parth, Thanks! A few questions: 1. Do you want to permit rules in your ACLs that DENY access as well as ALLOW? This can be handy setting up rules that have exceptions. E.g. “Allow principal P to READ resource R from all hosts” with “Deny principal P READ access to resource R from host H1” in combination would allow P to READ R from all hosts *except* H1. 2. When a topic is newly created, will there be an ACL created for it? If not, would that not deny subsequent access to it? (nit) Maybe use Principal instead of String to represent principals? On 3/9/15, 11:48 AM, Don Bosco Durai bo...@apache.org wrote: Parth Overall it is looking good. Couple of questionsŠ - Can you give an example how the policies will look like in the default implementation? - In the operations, can we support ³CONNECT² also? This can be used during Session connection - Regarding access control for ³Topic Creation², since we can¹t do it on the server side, can we de-scope it for? And plan it as a future feature request? Thanks Bosco On 3/6/15, 8:10 AM, Harsha ka...@harsha.io wrote: Hi Parth, Thanks for putting this together. Overall it looks good to me. Although AdminUtils is a concern KIP-4 can probably fix that part. Thanks, Harsha On Thu, Mar 5, 2015, at 10:39 AM, Parth Brahmbhatt wrote: Forgot to add links to wiki and jira. Link to wiki: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorizatio n + Interface Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 Thanks Parth From: Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com Date: Thursday, March 5, 2015 at 10:33 AM To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: [DISCUSS] KIP-11- Authorization design for kafka security Hi, KIP-11 is open for discussion , I have updated the wiki with the design and open questions. Thanks Parth
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Thanks for reviewing, comments inline: On 3/31/15, 9:21 AM, Jun Rao j...@confluent.iomailto:j...@confluent.io wrote: Thanks for the writeup. A few more comments. 20. I agree that it would be better to do this after KIP-4 (admin commands) is done. With KIP-4, all admin operations will be sent as requests to the brokers instead of accessing ZK directly. This will make authorization easier. 21. Operation: What about other types of requests not covered in the list, such as committing and fetching offsets, list topics, fetching consumer metadata, heartbeat, join group, etc? * I was actually considering any kind of write (like commit offset) as WRITE operation, and kind of read (fetching offset, get consumer metadata) as READ and any kind of list(list topics) as DESCRIBE. We can either create a one to one mapping between API and operation or classify each API as one of the operation. I was going with the classification but if you think one to one mapping will be easier to understand I am open to that. 22. TopicConfigCache: We will need such a cache in KIP-4 as well. It would be useful to make sure that the implementation can be reused. * I already opened a separate jirahttps://issues.apache.org/jira/browse/KAFKA-2035 for this and posted a reviewhttps://reviews.apache.org/r/32460/diff/#. I plan to add Acl and owner as instance variables of TopicConfig class as part of authZ patch. 23. Authorizer: 23.1 Do cluster level operations go through authorize() too? If so, what will be the resource? * Yes and I was considering to use a constant string like “Kafka-Cluster” for cluster operations. 23.2 I assume that the authorize() check will be called on every request. So, we will have to make sure that the check is cheap. * Yes , that is why by design we will trade off for speed and cache all acls, which means if you update acls it may take a few minutes before the changes take effect. 24. The acl json string in the config: Should we version this so that we can evolve it in the future (e.g., adding group support)? * I am looking into this right now but this seemed like implementation details so I did not capture it in design. I will update the json format once I have settled on a solution. What are your thoughts on using some existing libraries that support json parsing with versioning? The current json encoding/decoding used by kafka is already failing for me when I try to parse a map that has an already json encoded string as value for some key. Jun On Sun, Mar 29, 2015 at 3:56 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com wrote: Hi Gwen, Thanks a lot for taking the time to review this. I have tried to address all your questions below. Thanks Parth On 3/28/15, 8:08 PM, Gwen Shapira gshap...@cloudera.commailto:gshap...@cloudera.commailto: gshap...@cloudera.commailto:gshap...@cloudera.com wrote: Preparing for Tuesday meeting, I went over the KIP :) First, Parth did an amazing job, the KIP is fantastic - detailed and readable. Thank you! Second, I have a lng list of questions :) No objections, just some things I'm unclear on and random minor comments. In general, I like the design, I just feel I'm missing parts of the picture. 1. Yes, Create topic will have an optional acls, the output of describe will display owner and acls and alter topic will allow to modify the acls.” - will be nice to see what the CLI will look like. * I will modify the KIP but I was going to add “—acl acl-file.json” to create-topic and alter-topic. 2. I like the addition of Topic owner. We made the mistake of forgetting about it when adding authorization to Sqoop2. We probably want to add “chown” command to the topic commands. * Again we can add “—owner user-name” to alter topic. 3. Kafka server will read authorizer.class” config value at startup time, create an instance of the specified class and call initialize method. We’ll need to validate that users specify only one of those. * The config type will be string so type validation should take care of it. 4. One added assumption is that on non-secure connections the session will have principal set to an object whose name() method will return Anonymous”. Can we keep DrWho? :) * Sure, its up to you actually as you are the owner of the jira that introduces session concept. 5. For cluster actions that do not apply to a specific topic like CREATE we have 2 options. We can either add a broker config called broker.acls which will point to a json file. This file will be available on all broker hosts and authorizer will read the acls on initialization and keep refreshing it every X minutes. Any changes will require re-distribution of the acl json file. Alternatively we can add a zookeeper path /brokers/acls and store the acl json as data. Authorizer can refresh the acl from json every X minutes. In absence of broker acls the authorizer will fail open, in other words it will allow
Re: [DISCUSS] KIP-11- Authorization design for kafka security
From the design doc , one of the added config: * kafka.superusers: list of users that will be given superuser access. These users will have access to everything. Users should set this to the user kafka broker processes are running as to avoid duplicate configuration for every single topic like ALLOW REPLICATION to BROKER_USER for TOPIC from ALL hosts. Thanks Parth On 3/31/15, 10:20 AM, Gwen Shapira gshap...@cloudera.com wrote: Related interesting question: Since a broker is a consumer (of lead replicas), how do we handle the broker level of permissions? Do we hardcode a broker-principal name and automatically authorize brokers to do anything? Or is there a cleaner way? On Tue, Mar 31, 2015 at 10:17 AM, Don Bosco Durai bo...@apache.org wrote: 21. Operation: What about other types of requests not covered in the list, such as committing and fetching offsets, list topics, fetching consumer metadata, heartbeat, join group, etc? Would “CONFIGURE”, “DESCRIBE”, etc take care of this? Or should we add high level grouping like “ADMIN”, “OPERATIONS/MANAGEMENT” to cover related permissions? Bosco On 3/31/15, 9:21 AM, Jun Rao j...@confluent.io wrote: Thanks for the writeup. A few more comments. 20. I agree that it would be better to do this after KIP-4 (admin commands) is done. With KIP-4, all admin operations will be sent as requests to the brokers instead of accessing ZK directly. This will make authorization easier. 21. Operation: What about other types of requests not covered in the list, such as committing and fetching offsets, list topics, fetching consumer metadata, heartbeat, join group, etc? 22. TopicConfigCache: We will need such a cache in KIP-4 as well. It would be useful to make sure that the implementation can be reused. 23. Authorizer: 23.1 Do cluster level operations go through authorize() too? If so, what will be the resource? 23.2 I assume that the authorize() check will be called on every request. So, we will have to make sure that the check is cheap. 24. The acl json string in the config: Should we version this so that we can evolve it in the future (e.g., adding group support)? Jun On Sun, Mar 29, 2015 at 3:56 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi Gwen, Thanks a lot for taking the time to review this. I have tried to address all your questions below. Thanks Parth On 3/28/15, 8:08 PM, Gwen Shapira gshap...@cloudera.commailto: gshap...@cloudera.com wrote: Preparing for Tuesday meeting, I went over the KIP :) First, Parth did an amazing job, the KIP is fantastic - detailed and readable. Thank you! Second, I have a lng list of questions :) No objections, just some things I'm unclear on and random minor comments. In general, I like the design, I just feel I'm missing parts of the picture. 1. Yes, Create topic will have an optional acls, the output of describe will display owner and acls and alter topic will allow to modify the acls.” - will be nice to see what the CLI will look like. * I will modify the KIP but I was going to add “—acl acl-file.json” to create-topic and alter-topic. 2. I like the addition of Topic owner. We made the mistake of forgetting about it when adding authorization to Sqoop2. We probably want to add “chown” command to the topic commands. * Again we can add “—owner user-name” to alter topic. 3. Kafka server will read authorizer.class” config value at startup time, create an instance of the specified class and call initialize method. We’ll need to validate that users specify only one of those. * The config type will be string so type validation should take care of it. 4. One added assumption is that on non-secure connections the session will have principal set to an object whose name() method will return Anonymous”. Can we keep DrWho? :) * Sure, its up to you actually as you are the owner of the jira that introduces session concept. 5. For cluster actions that do not apply to a specific topic like CREATE we have 2 options. We can either add a broker config called broker.acls which will point to a json file. This file will be available on all broker hosts and authorizer will read the acls on initialization and keep refreshing it every X minutes. Any changes will require re-distribution of the acl json file. Alternatively we can add a zookeeper path /brokers/acls and store the acl json as data. Authorizer can refresh the acl from json every X minutes. In absence of broker acls the authorizer will fail open, in other words it will allow all users from all hosts to perform all cluster actions” I prefer a file to ZK - since thats where we store all use-defined configurations for now. Everyone knows how to secure a file system :) * I will let everyone vote, file system is fine by me. 6. When an Acl is missing , this implementation will always fail open for backward compatibility. “ - agree, but we need to document that this makes the default
Re: Review Request 32460: Patch for KAFKA-2035
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32460/ --- (Updated March 31, 2015, 5:52 p.m.) Review request for kafka. Summary (updated) - Patch for KAFKA-2035 Bugs: KAFKA-2032 and KAFKA-2035 https://issues.apache.org/jira/browse/KAFKA-2032 https://issues.apache.org/jira/browse/KAFKA-2035 Repository: kafka Description (updated) --- Merge remote-tracking branch 'origin/trunk' into 2032 KAFKA-2035: Added TopicConfigCache. Merge remote-tracking branch 'origin/trunk' into 2032 Diffs (updated) - core/src/main/scala/kafka/server/KafkaServer.scala 4db3384545be8c237d6fc9646716ab67d5193ec5 core/src/main/scala/kafka/server/TopicConfigCache.scala PRE-CREATION core/src/main/scala/kafka/server/TopicConfigManager.scala 47295d40131492aaac786273819b7bc6e22e5486 core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala 93182aeb342729d420d2e7d59a1035994164b7db core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala PRE-CREATION Diff: https://reviews.apache.org/r/32460/diff/ Testing (updated) --- unit tests added. Thanks, Parth Brahmbhatt
Re: [DISCUSS] KIP-11- Authorization design for kafka security
to Kerberos authentication AFAIK.) * Yes, again I think this can be a separate issue for now and can only be worked on after KIP-4 is delivered. 12. Do we want to support group acls as part of this authorizer? Do we want to support principal to local user mapping? If yes we need to add plugins for UserToGroupMapper and PrincipalToUserMapper.” - Sentry uses Groups for authorizing, so we need to support that. I figured that as long as the API specifies Principal, it typically contains both user and group, so nothing else is needed. Did I miss anything? * Once we support group acls we will need someway to indicate if a principal is of type group or user(as part of acl) or we can have group acls and user acls stored separately with topic config. We will also need a way to map an authenticated user to list of groups the user belongs to. 13. It looks like the Authorizer stores the ACLs and not Kafka. So we need an API for Kafka to notify Authorizer when a topic is added and when ACLs are modified, right? I didn’t see that. * ACLs will be stored under /topic/config at time of topic creation in json format. Authorizer will get these acls using newly introduced TopicConfigCache which gets updated anytime topic config changes. 14. Are we going to have any API for Kafka to give out the ACLs on a topic? Or we leave this to the Authorizer? * I think it is better to leave this out side of Kafka. Mainly because most 3rd party authorizers will have their own ACL stores and configuration Uis like (Ranger-Argus not sure what are they calling it now). On Wed, Mar 25, 2015 at 9:26 PM, Neha Narkhede n...@confluent.iomailto:n...@confluent.io wrote: Parth, We can make some 15 mins or so to discuss this at the next KIP hangout. Thanks, Neha On Wed, Mar 25, 2015 at 1:07 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com wrote: Hi all, I have modified the KIP to reflect the recent change request from the reviewers. I have been working on the code and I have the server side code for authorization ready. I am now modifying the command line utilities. I would really appreciate if some of the committers can spend sometime to review the KIP so we can make progress on this. Thanks Parth On 3/18/15, 2:20 PM, Michael Herstine mherst...@linkedin.com.INVALIDmailto:mherst...@linkedin.com.INVALID wrote: Hi Parth, Thanks! A few questions: 1. Do you want to permit rules in your ACLs that DENY access as well as ALLOW? This can be handy setting up rules that have exceptions. E.g. “Allow principal P to READ resource R from all hosts” with “Deny principal P READ access to resource R from host H1” in combination would allow P to READ R from all hosts *except* H1. 2. When a topic is newly created, will there be an ACL created for it? If not, would that not deny subsequent access to it? (nit) Maybe use Principal instead of String to represent principals? On 3/9/15, 11:48 AM, Don Bosco Durai bo...@apache.orgmailto:bo...@apache.org wrote: Parth Overall it is looking good. Couple of questionsŠ - Can you give an example how the policies will look like in the default implementation? - In the operations, can we support ³CONNECT² also? This can be used during Session connection - Regarding access control for ³Topic Creation², since we can¹t do it on the server side, can we de-scope it for? And plan it as a future feature request? Thanks Bosco On 3/6/15, 8:10 AM, Harsha ka...@harsha.iomailto:ka...@harsha.io wrote: Hi Parth, Thanks for putting this together. Overall it looks good to me. Although AdminUtils is a concern KIP-4 can probably fix that part. Thanks, Harsha On Thu, Mar 5, 2015, at 10:39 AM, Parth Brahmbhatt wrote: Forgot to add links to wiki and jira. Link to wiki: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorizatio n + Interface Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 Thanks Parth From: Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com Date: Thursday, March 5, 2015 at 10:33 AM To: dev@kafka.apache.orgmailto:dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: [DISCUSS] KIP-11- Authorization design for kafka security Hi, KIP-11 is open for discussion , I have updated the wiki with the design and open questions. Thanks Parth -- Thanks, Neha
Re: Review Request 29467: Patch for KAFKA-1660
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/#review74777 --- clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java https://reviews.apache.org/r/29467/#comment121523 Changed log level as suggested. clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java https://reviews.apache.org/r/29467/#comment121524 included. clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java https://reviews.apache.org/r/29467/#comment121525 changed log level to suggested value. - Parth Brahmbhatt On March 2, 2015, 6:41 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/ --- (Updated March 2, 2015, 6:41 p.m.) Review request for kafka. Bugs: KAFKA-1660 https://issues.apache.org/jira/browse/KAFKA-1660 Repository: kafka Description --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1660 Conflicts: clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java Merge remote-tracking branch 'origin/trunk' into KAFKA-1660 Changing log levels as suggested. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 7397e565fd865214529ffccadd4222d835ac8110 clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 6913090af03a455452b0b5c3df78f266126b3854 clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 5b3e75ed83a940bc922f9eca10d4008d67aa37c9 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java ed9c63a6679e3aaf83d19fde19268553a4c107c2 Diff: https://reviews.apache.org/r/29467/diff/ Testing --- existing unit tests passed. Thanks, Parth Brahmbhatt
Re: Review Request 29467: Patch for KAFKA-1660
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/ --- (Updated March 2, 2015, 6:41 p.m.) Review request for kafka. Bugs: KAFKA-1660 https://issues.apache.org/jira/browse/KAFKA-1660 Repository: kafka Description (updated) --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1660 Conflicts: clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java Merge remote-tracking branch 'origin/trunk' into KAFKA-1660 Changing log levels as suggested. Diffs (updated) - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 7397e565fd865214529ffccadd4222d835ac8110 clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 6913090af03a455452b0b5c3df78f266126b3854 clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 5b3e75ed83a940bc922f9eca10d4008d67aa37c9 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java ed9c63a6679e3aaf83d19fde19268553a4c107c2 Diff: https://reviews.apache.org/r/29467/diff/ Testing --- existing unit tests passed. Thanks, Parth Brahmbhatt
Re: Review Request 29467: Patch for KAFKA-1660
On March 3, 2015, 4:10 a.m., Jay Kreps wrote: clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java, line 560 https://reviews.apache.org/r/29467/diff/4/?file=882247#file882247line560 This seems to call initiateClose() twice, once in initiateClose and then again from forceClose. This seems like it depends on all the things getting closed being idempotent to repeated calls (e.g. record accumulator etc). Would it make more sense to have forceClose() just set the force flag? The issue with that is someone can just call sender.forceClose and it will never call accumulator.close which is part of initiate close. Also shouldn't the calls be idempotent given this can be called from multiple threds multiple times? On March 3, 2015, 4:10 a.m., Jay Kreps wrote: clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java, line 554 https://reviews.apache.org/r/29467/diff/4/?file=882247#file882247line554 It's probably worth adding an if(timeout 0) on this. Added. On March 3, 2015, 4:10 a.m., Parth Brahmbhatt wrote: Two minor changes I noted, but otherwise looks good to me. Needs some unit tests, as you mentioned. Jay Kreps wrote: Actually one probably I didn't think of is that forceClose() leaves the in-flight requests forever incomplete. A better approach would be to fail them all with TimeoutException. To do this correctly I will need to get the imcomplete and unsent RecordBatches from RecordAccumulator. I can add methods to get these with default scope. The sender will need these to emit correct metrics and failing the bathces. For unit testing I need someway to mock RecordAccumulator as the Seneder's run method where the force close logic lives is a while(true) loop which dependes on the values of record accumulator. RecordAccumulator is a final class right now, is it ok to change that so I can create a MockRecordAccumulator? - Parth --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/#review74884 --- On March 2, 2015, 6:41 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/ --- (Updated March 2, 2015, 6:41 p.m.) Review request for kafka. Bugs: KAFKA-1660 https://issues.apache.org/jira/browse/KAFKA-1660 Repository: kafka Description --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1660 Conflicts: clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java Merge remote-tracking branch 'origin/trunk' into KAFKA-1660 Changing log levels as suggested. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 7397e565fd865214529ffccadd4222d835ac8110 clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 6913090af03a455452b0b5c3df78f266126b3854 clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 5b3e75ed83a940bc922f9eca10d4008d67aa37c9 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java ed9c63a6679e3aaf83d19fde19268553a4c107c2 Diff: https://reviews.apache.org/r/29467/diff/ Testing --- existing unit tests passed. Thanks, Parth Brahmbhatt
Re: Review Request 29467: Patch for KAFKA-1660
On March 3, 2015, 5:37 a.m., Jay Kreps wrote: clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java, line 533 https://reviews.apache.org/r/29467/diff/4/?file=882247#file882247line533 Now there is a bit of duplicate code between the two close methods. Maybe this would be cleaner if we just made public void close() { close(Long.MAX_VALUE, TimeUnit.MILLISECONDS); } Fixed. - Parth --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/#review74897 --- On March 2, 2015, 6:41 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29467/ --- (Updated March 2, 2015, 6:41 p.m.) Review request for kafka. Bugs: KAFKA-1660 https://issues.apache.org/jira/browse/KAFKA-1660 Repository: kafka Description --- Merge remote-tracking branch 'origin/trunk' into KAFKA-1660 Conflicts: clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java Merge remote-tracking branch 'origin/trunk' into KAFKA-1660 Changing log levels as suggested. Diffs - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 7397e565fd865214529ffccadd4222d835ac8110 clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java 6913090af03a455452b0b5c3df78f266126b3854 clients/src/main/java/org/apache/kafka/clients/producer/Producer.java 5b3e75ed83a940bc922f9eca10d4008d67aa37c9 clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java ed9c63a6679e3aaf83d19fde19268553a4c107c2 Diff: https://reviews.apache.org/r/29467/diff/ Testing --- existing unit tests passed. Thanks, Parth Brahmbhatt
[DISCUSS] KIP-11- Authorization design for kafka security
Hi, KIP-11 is open for discussion , I have updated the wiki with the design and open questions. Thanks Parth
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Forgot to add links to wiki and jira. Link to wiki: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 Thanks Parth From: Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com Date: Thursday, March 5, 2015 at 10:33 AM To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: [DISCUSS] KIP-11- Authorization design for kafka security Hi, KIP-11 is open for discussion , I have updated the wiki with the design and open questions. Thanks Parth
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Here is a pseudo code that explains my current approach: acls = authorizer.getAcl(resource) if(acls == null || acls.isEmpty) { allow all requests for backward compatibility. (any topics that were created prior to security support will not have acls) This is debatable , generally we should block everyone which is what I would prefer but that means anyone moving to authorizer must go to all of his existing topics and add acl to allow all. If we are fine with imposing this requirement I can start returning deny when no acls are found. } else { //So the user has set some acls explicitly, this means they have knowingly enabled authorizer. Let’t first check if they have set an Acl to deny this user/host/operation combination. if some acl denies this request for this principal/host/operation combination , return deny //this principal/host/operation does not have any explicit deny acl, check if there is some explicit acl that allows the operation if at least one acl allows this request for this principal/host/operation combination , return allow // no acl was found for this principal/host/operation combination to allow this operation, so we will deny the request return deny } Thanks Parth On 4/20/15, 2:21 PM, Jun Rao j...@confluent.io wrote: Hmm, I thought the semantics is that if you only have rule deny user2, it means that everyone except user2 has access? Thanks, Jun On Mon, Apr 20, 2015 at 3:25 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: user3 does not have access and removing the deny rule does not grant him or user2 access. user2 even without the deny rule will not have access. Thanks Parth On 4/20/15, 12:03 PM, Jun Rao j...@confluent.io wrote: Just a followup question. Suppose there are two rules. Rule1 allows user1 and rule2 denies user2. Does user3 have access? If not, does removing rule1 enable user3 access? Thanks, Jun On Mon, Apr 20, 2015 at 1:34 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi Joel, Thanks for the review and I plan to update the KIP today with all the updated info. My comments in line below. Thanks Parth On 4/20/15, 10:07 AM, Joel Koshy jjkosh...@gmail.commailto: jjkosh...@gmail.com wrote: Hi Parth, Nice work on this KIP. I did another read through and had a few more comments (with edits after I went through the thread). Many of these comments were brought up by others as well, so it appears that the KIP would benefit from an update at this point to incorporate comments from the thread and last hangout. - The operation enum is mostly self-explanatory, but it would help (for the sake of clarity and completeness if nothing else) to document exactly what each of the enums are. E.g., I think this came up in our hangout - SEND_CONTROL_MESSAGE is unclear and I don't remember what was said about it. Edit: After going through the thread it seems the conclusion was to categorize operations. E.g., WRITE could apply to multiple requests. Again, this is unclear, so if it would be great if you could update the KIP to clarify what you intend. Will add to document. SEND_CONTROL_MESSAGE Probably a very bad name but these are intra borker API calls like controller notifying other brokers to update metadata or heartbeats. Any better naming suggestions? - When you update the KIP to categorize the requests it would also help to have a column for what the resource is for each. Will add to the KIP. - FWIW I prefer a 1-1 mapping between requests and operations. I think categorizing requests into these can be confusing because: - The resource being protected for different requests will be different. We are mostly thinking about topics (read/write) but there are requests for which topic is not the right resource. E.g., for topic creation, the resource as you suggested would be something global/common such as “cluster”. For OffsetCommit/FetchRequest, the resource may be the consumer group, or maybe a tuple of consumer group, topic. So this can be confusing - i.e., different resources and request types in the same category. It may be simpler and clearer to just have a 1-1 mapping between the operation enum and requests. I only see 2 resource categories right now cluster and topic. I don’t really care one way or another so we can probably make a quick decision in tomorrow’s meeting to either to 1-1 mapping or have categorization? - Some requests that are intuitively READ have WRITE side-effects. E.g., (currently) TopicMetadataRequest with auto-create, although that will eventually go away. ConsumerMetadataRequest still auto-creates the offsets topic. Likewise, ADMIN-type requests may be interpreted as having side-effects (depending on who you ask). Yes and what I am doing right now
Re: [DISCUSS] KIP-11- Authorization design for kafka security
The iptables on unix supports the DENY operator, not that it should matter. The deny operator can also be used to specify ³allow user1 to READ from topic1 from all hosts but host1,host2². Again we could add a host group semantic and extra complexity around that, not sure if its worth it. In addition with DENY operator you are now not forced to create a special group just to support the authorization use case. I am not convinced that the operator it self is really all that confusing. There are 3 practical use cases: - Resource with no acl what so ever - allow access to everyone ( just for backward compatibility, I would much rather fail close and force users to explicitly grant acls that allows access to all users.) - Resource with some acl attached - only users that have a matching allow acl are allowed (i.e. ³allow READ access to topic1 to user1 from all hosts², only user1 has READ access and no other user has access of any kind) - Resource with some allow and some deny acl attached - users are allowed to perform operation only when they satisfy allow acl and do not have conflicting deny acl. Users that have no acl(allow or deny) will still not have any access. (i.e. ³allow READ access to topic1 to user1 from all hosts except host1 and host², only user1 has access but not from host1 an host2) I think we need to make a decision on deny primarily because with introduction of acl management API, Acl is now a public class that will be used by Ranger/Santry and other authroization providers. In Current design the acl has a permissionType enum field with possible values of Allow and Deny. If we chose to remove deny we can assume all acls to be of allow type and remove the permissionType field completely. Thanks Parth On 4/20/15, 6:12 PM, Gwen Shapira gshap...@cloudera.com wrote: I think thats how its done in pretty much any system I can think of.
Re: [DISCUSS] KIP-11- Authorization design for kafka security
user3 does not have access and removing the deny rule does not grant him or user2 access. user2 even without the deny rule will not have access. Thanks Parth On 4/20/15, 12:03 PM, Jun Rao j...@confluent.io wrote: Just a followup question. Suppose there are two rules. Rule1 allows user1 and rule2 denies user2. Does user3 have access? If not, does removing rule1 enable user3 access? Thanks, Jun On Mon, Apr 20, 2015 at 1:34 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi Joel, Thanks for the review and I plan to update the KIP today with all the updated info. My comments in line below. Thanks Parth On 4/20/15, 10:07 AM, Joel Koshy jjkosh...@gmail.commailto: jjkosh...@gmail.com wrote: Hi Parth, Nice work on this KIP. I did another read through and had a few more comments (with edits after I went through the thread). Many of these comments were brought up by others as well, so it appears that the KIP would benefit from an update at this point to incorporate comments from the thread and last hangout. - The operation enum is mostly self-explanatory, but it would help (for the sake of clarity and completeness if nothing else) to document exactly what each of the enums are. E.g., I think this came up in our hangout - SEND_CONTROL_MESSAGE is unclear and I don't remember what was said about it. Edit: After going through the thread it seems the conclusion was to categorize operations. E.g., WRITE could apply to multiple requests. Again, this is unclear, so if it would be great if you could update the KIP to clarify what you intend. Will add to document. SEND_CONTROL_MESSAGE Probably a very bad name but these are intra borker API calls like controller notifying other brokers to update metadata or heartbeats. Any better naming suggestions? - When you update the KIP to categorize the requests it would also help to have a column for what the resource is for each. Will add to the KIP. - FWIW I prefer a 1-1 mapping between requests and operations. I think categorizing requests into these can be confusing because: - The resource being protected for different requests will be different. We are mostly thinking about topics (read/write) but there are requests for which topic is not the right resource. E.g., for topic creation, the resource as you suggested would be something global/common such as “cluster”. For OffsetCommit/FetchRequest, the resource may be the consumer group, or maybe a tuple of consumer group, topic. So this can be confusing - i.e., different resources and request types in the same category. It may be simpler and clearer to just have a 1-1 mapping between the operation enum and requests. I only see 2 resource categories right now cluster and topic. I don’t really care one way or another so we can probably make a quick decision in tomorrow’s meeting to either to 1-1 mapping or have categorization? - Some requests that are intuitively READ have WRITE side-effects. E.g., (currently) TopicMetadataRequest with auto-create, although that will eventually go away. ConsumerMetadataRequest still auto-creates the offsets topic. Likewise, ADMIN-type requests may be interpreted as having side-effects (depending on who you ask). Yes and what I am doing right now is checking authorization for all possible actions i.e. for auto-create it checks if the config has it enabled and if yes, check for read + create authorization. Its not very meaningful right now as there is no CREATE authorization but I think this is implementation detail, we need to ensure we call authorize with all possible operations from KafkaAPI. - quoteWhen an ACL is missing - fail open/quote. What does missing mean? i.e., no explicit ACL for a principal? I'm confused by this especially in relation to the precedence of DENY over ALLOW. So per the description: - If no ACLs exist for topic A then ALLOW all operations on it by anyone. - If I now add an ACL for a certain principal P to ALLOW (say) WRITE to the topic then either: - This has the effect of DENYing WRITE to all other principals - Or, this ACL serves no purpose - If the effect is to DENY WRITE to all other principals, what about READ. Do all principals (including P) have READ permissions to topic A? - In other words, it seems for a specific ACL to be meaningful then fail close is necessary for an absent ACL. - editAfter through the thread: it appears that the DENY override only applies to the given principal. i.e., in the above case it appears that the other principals will in fact be granted access. Then this makes the ACL that was added pointless right? The rule I was going with is - If there is no ACL I.e. This might be a topic that was created in non secure mode or was created before we supported ACLs. We assume you do not want authorization and let all
Re: [DISCUSS] KIP-11- Authorization design for kafka security
FYI, I have modified the KIP to include group as resource. In order to access “joinGroup” and “commitOFfset” APIs the user will need a read permission on topic and WRITE permission on group. I plan to open a VOTE thread by noon if there are no more concerns. Thanks Parth On 4/22/15, 9:03 AM, Tom Graves tgraves...@yahoo.com.INVALID wrote: Hey everyone, Sorry to jump in on the conversation so late. I'm new to Kafka. I'll apologize in advance if you have already covered some of my questions. I read through the wiki and had some comments and questions. 1) public enum Operation needs EDIT changed to ALTER Done. 2) Does the Authorizer class need a setAcls? Rather then just add to be able to set to explicit list and overwrite what was there? I see the kafka-acl.sh lists a removeall so I guess you could do removeall and then add. I also don't see a removeall in the Authorizer class, is it going to loop through them all to remove each one? There is an overloaded version of removeAcls in the interface that takes in resource as the only input and as described in the javadoc all the acls attached to that resource will be deleted. To cover the setAcl use case the caller can first call remove and then add. 3) Can someone tell me what the use case to do acls based on the hosts? I can see some possibilities just wondering if we can concrete ones where one user is allowed from one host but not another. I am not sure if I understand the question given the use case you described in your question is what we are trying to cover with use of hosts in Acl. There are some additional use cases like “allow access to any user from host1,host2” but I think primarily it gives the admins the ability to define acls at a more granular level. 4) I'm a bit unclear how the resource works in the Authorizer class. From what I see we have 2 resources - topics and cluster. If I want to add an acl to allow joe to CREATE for the cluster then I call addAcls with Acl(user: joe, ALLOW, Set(*), Set(CREATE)) and cluster? What if I want to call addAcls for DESCRIBE on a topic? Is the resource then topic or is it the topic name? We now have 3 resources(added group), please see the updated doc. The CREATE acl that you described is correct. For any topic operation you should use topic name as the resource name and for group the user will provide groupId as resource name. 5) reassigning partitions is a CLUSTER_ACTION or superuser? Its not totally clear to me the differences between these. what about increasing # of partitions? I see this as an alter topic operation so it is at topic level and the user must have alter permissions on topic. 6) groups are mentioned, are we supporting right away or is that a follow on item? (is there going to be a kafka.supergroups) I think it can be a separate jira just for braking down the code review in smaller chunk. We will support it in first version but I think if we can not do it for any reason that should not block a release with all the other authZ work. We made deliberate design choices (like introducing a principalType in KafkaPrinciapl) to allow supporting groups as an incremental change. 7) Are there config options for setting acls when I create my topic? Or do I have to create my topic and then run the kafka-acl.sh script to set them? Although its very small, there would be possible race there that someone could start producing to topic before acls are set. We discussed this yesterday and we agreed to go with kafka-acl.sh. Yes there is a very very small window of vulnerability but I think that really does not warrant to change the decision in this case. 8) are there configs for cluster level acl defaults? Or does it default to superusers on bringing up new cluster and you have to modify with cli. thanks,Tom No defaults, the default is superusers will have full access. I don’t think making assumptions about ones security requirement should be our burden. On Tuesday, April 21, 2015 7:10 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: I have added the notes to KIP-11 Open question sections. Thanks Parth On 4/21/15, 4:49 PM, Gwen Shapira gshap...@cloudera.com wrote: Adding my notes from today's call to the thread: ** Deny or Allow all by default? We will add a configuration to control this. The configuration will default to “allow” for backward compatibility. Security admins can set it to deny ** Storing ACLs for default authorizers: We'll store them in ZK. We'll support pointing the authorizer to any ZK. The use of ZK will be internal to the default authorizer. Authorizer reads ACLs from cache every hour. We proposed having mechanism (possibly via new ZK node) to tell broker to refresh the cache immediately. ** Support deny as permission type - we agreed to keep this. ** Mapping operations to API: We may need to add Group as a resource, with JoinGroup and OffsetCommit require privilege
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Changed Edit to Alter. I did not think about it that way but Sriharsha raised the same point in a private conversation. I did not think about it that way but I agree it makes sense. If no one objects I think in default implementation we can infer that if user have READ or WRITE access he gets DESCRIBE for free. Thanks Parth On 4/21/15, 2:04 PM, Jay Kreps jay.kr...@gmail.com wrote: Also, I think I may have missed this but does READ imply you also have DESCRIBE? A reader will need access to both read offsets (to determine their own initial position) as well as commit offsets. Currently, though fetching offsets is under DESCRIBE only and commit offsets is under READ. If READ=DESCRIBE are there any other implied permissions like that? -Jay On Tue, Apr 21, 2015 at 1:59 PM, Jay Kreps jay.kr...@gmail.com wrote: Hey Parth, Great write-up! One super minor thing: could we change the EDIT permission to be called ALTER? The request name in KIP-4 is Alter and the command line tool has always been alter (or we could go the other way and change those to EDIT). Not sure that one is any better than the other but consistency is always nice. -Jay On Tue, Apr 21, 2015 at 9:06 AM, Jun Rao j...@confluent.io wrote: Harsha, Parth, Thanks for the clarification. This makes sense. Perhaps we can clarify the meaning of those rules in the wiki. Related to this, it seems that we need to support wildcard in cli/request protocol for topics? Jun On Mon, Apr 20, 2015 at 9:07 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: The iptables on unix supports the DENY operator, not that it should matter. The deny operator can also be used to specify ³allow user1 to READ from topic1 from all hosts but host1,host2². Again we could add a host group semantic and extra complexity around that, not sure if its worth it. In addition with DENY operator you are now not forced to create a special group just to support the authorization use case. I am not convinced that the operator it self is really all that confusing. There are 3 practical use cases: - Resource with no acl what so ever - allow access to everyone ( just for backward compatibility, I would much rather fail close and force users to explicitly grant acls that allows access to all users.) - Resource with some acl attached - only users that have a matching allow acl are allowed (i.e. ³allow READ access to topic1 to user1 from all hosts², only user1 has READ access and no other user has access of any kind) - Resource with some allow and some deny acl attached - users are allowed to perform operation only when they satisfy allow acl and do not have conflicting deny acl. Users that have no acl(allow or deny) will still not have any access. (i.e. ³allow READ access to topic1 to user1 from all hosts except host1 and host², only user1 has access but not from host1 an host2) I think we need to make a decision on deny primarily because with introduction of acl management API, Acl is now a public class that will be used by Ranger/Santry and other authroization providers. In Current design the acl has a permissionType enum field with possible values of Allow and Deny. If we chose to remove deny we can assume all acls to be of allow type and remove the permissionType field completely. Thanks Parth On 4/20/15, 6:12 PM, Gwen Shapira gshap...@cloudera.com wrote: I think thats how its done in pretty much any system I can think of.
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Hey Jun, Yes and we support wild cards for all acl entities principal, hosts and operation. Thanks Parth On 4/21/15, 9:06 AM, Jun Rao j...@confluent.io wrote: Harsha, Parth, Thanks for the clarification. This makes sense. Perhaps we can clarify the meaning of those rules in the wiki. Related to this, it seems that we need to support wildcard in cli/request protocol for topics? Jun On Mon, Apr 20, 2015 at 9:07 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: The iptables on unix supports the DENY operator, not that it should matter. The deny operator can also be used to specify ³allow user1 to READ from topic1 from all hosts but host1,host2². Again we could add a host group semantic and extra complexity around that, not sure if its worth it. In addition with DENY operator you are now not forced to create a special group just to support the authorization use case. I am not convinced that the operator it self is really all that confusing. There are 3 practical use cases: - Resource with no acl what so ever - allow access to everyone ( just for backward compatibility, I would much rather fail close and force users to explicitly grant acls that allows access to all users.) - Resource with some acl attached - only users that have a matching allow acl are allowed (i.e. ³allow READ access to topic1 to user1 from all hosts², only user1 has READ access and no other user has access of any kind) - Resource with some allow and some deny acl attached - users are allowed to perform operation only when they satisfy allow acl and do not have conflicting deny acl. Users that have no acl(allow or deny) will still not have any access. (i.e. ³allow READ access to topic1 to user1 from all hosts except host1 and host², only user1 has access but not from host1 an host2) I think we need to make a decision on deny primarily because with introduction of acl management API, Acl is now a public class that will be used by Ranger/Santry and other authroization providers. In Current design the acl has a permissionType enum field with possible values of Allow and Deny. If we chose to remove deny we can assume all acls to be of allow type and remove the permissionType field completely. Thanks Parth On 4/20/15, 6:12 PM, Gwen Shapira gshap...@cloudera.com wrote: I think thats how its done in pretty much any system I can think of.
Review Request 33431: Patch for KAFKA-1688
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33431/ --- Review request for kafka. Bugs: KAFKA-1688 https://issues.apache.org/jira/browse/KAFKA-1688 Repository: kafka Description --- KAFKA-1688: initial check in. Merge remote-tracking branch 'origin/trunk' into trunk Merge remote-tracking branch 'origin/trunk' into 2032 Merge branch '2032' into trunk KAFKA-1688: Add authorization. Merge remote-tracking branch 'origin/trunk' into trunk Conflicts: core/src/main/scala/kafka/admin/AdminUtils.scala core/src/main/scala/kafka/admin/TopicCommand.scala core/src/main/scala/kafka/network/RequestChannel.scala core/src/main/scala/kafka/server/KafkaApis.scala core/src/main/scala/kafka/server/KafkaServer.scala core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala Fixing some merge errors. Merge remote-tracking branch 'origin/trunk' into trunk Conflicts: core/src/main/scala/kafka/common/ErrorMapping.scala Converted some code to idomatic scala. Merge branch 'trunk' of http://git-wip-us.apache.org/repos/asf/kafka into trunk Removing some unintended changes. Reverting the topic config related changes public classes and interfaces to support pluggable authorizer implementation for kafka Merge branch 'trunk' of http://git-wip-us.apache.org/repos/asf/kafka into trunk all public entities for pluggable authorizer support in kafka. Diffs - core/src/main/scala/kafka/common/AuthorizationException.scala PRE-CREATION core/src/main/scala/kafka/common/ErrorMapping.scala c75c68589681b2c9d6eba2b440ce5e58cddf6370 core/src/main/scala/kafka/network/RequestChannel.scala 1d0024c8f0c2ab0efa6d8cfca6455877a6ed8026 core/src/main/scala/kafka/security/auth/Acl.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Authorizer.scala PRE-CREATION core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Operation.java PRE-CREATION core/src/main/scala/kafka/security/auth/PermissionType.java PRE-CREATION core/src/main/scala/kafka/server/KafkaApis.scala b4004aa3a1456d337199aa1245fb0ae61f6add46 core/src/main/scala/kafka/server/KafkaConfig.scala cfbbd2be550947dd2b3c8c2cab981fa08fb6d859 core/src/main/scala/kafka/server/KafkaServer.scala c63f4ba9d622817ea8636d4e6135fba917ce085a core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala 62d183248e3be4c83d2c768e762f61f92448c6a6 Diff: https://reviews.apache.org/r/33431/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: [DISCUSS] KIP-11- Authorization design for kafka security
I have added the notes to KIP-11 Open question sections. Thanks Parth On 4/21/15, 4:49 PM, Gwen Shapira gshap...@cloudera.com wrote: Adding my notes from today's call to the thread: ** Deny or Allow all by default? We will add a configuration to control this. The configuration will default to “allow” for backward compatibility. Security admins can set it to deny ** Storing ACLs for default authorizers: We'll store them in ZK. We'll support pointing the authorizer to any ZK. The use of ZK will be internal to the default authorizer. Authorizer reads ACLs from cache every hour. We proposed having mechanism (possibly via new ZK node) to tell broker to refresh the cache immediately. ** Support deny as permission type - we agreed to keep this. ** Mapping operations to API: We may need to add Group as a resource, with JoinGroup and OffsetCommit require privilege on the consumer group. This can be something we pass now and authorizers can support in future. - Jay will write specifics to the mailing list discussion. On Tue, Apr 21, 2015 at 4:32 PM, Jay Kreps jay.kr...@gmail.com wrote: Following up on the KIP discussion. Two options for authorizing consumers to read topic t as part of group g: 1. READ permission on resource /topic/t 2. READ permission on resource /topic/t AND WRITE permission on /group/g The advantage of (1) is that it is simpler. The disadvantage is that any member of any group that reads from t can commit offsets as any other member of a different group. This doesn't effect data security (who can access what) but it is a bit of a management issue--a malicious person can cause data loss or duplicates for another consumer by committing offset. I think I favor (2) but it's worth it to think it through. -Jay On Tue, Apr 21, 2015 at 2:43 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hey Jun, Yes and we support wild cards for all acl entities principal, hosts and operation. Thanks Parth On 4/21/15, 9:06 AM, Jun Rao j...@confluent.io wrote: Harsha, Parth, Thanks for the clarification. This makes sense. Perhaps we can clarify the meaning of those rules in the wiki. Related to this, it seems that we need to support wildcard in cli/request protocol for topics? Jun On Mon, Apr 20, 2015 at 9:07 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: The iptables on unix supports the DENY operator, not that it should matter. The deny operator can also be used to specify ³allow user1 to READ from topic1 from all hosts but host1,host2². Again we could add a host group semantic and extra complexity around that, not sure if its worth it. In addition with DENY operator you are now not forced to create a special group just to support the authorization use case. I am not convinced that the operator it self is really all that confusing. There are 3 practical use cases: - Resource with no acl what so ever - allow access to everyone ( just for backward compatibility, I would much rather fail close and force users to explicitly grant acls that allows access to all users.) - Resource with some acl attached - only users that have a matching allow acl are allowed (i.e. ³allow READ access to topic1 to user1 from all hosts², only user1 has READ access and no other user has access of any kind) - Resource with some allow and some deny acl attached - users are allowed to perform operation only when they satisfy allow acl and do not have conflicting deny acl. Users that have no acl(allow or deny) will still not have any access. (i.e. ³allow READ access to topic1 to user1 from all hosts except host1 and host², only user1 has access but not from host1 an host2) I think we need to make a decision on deny primarily because with introduction of acl management API, Acl is now a public class that will be used by Ranger/Santry and other authroization providers. In Current design the acl has a permissionType enum field with possible values of Allow and Deny. If we chose to remove deny we can assume all acls to be of allow type and remove the permissionType field completely. Thanks Parth On 4/20/15, 6:12 PM, Gwen Shapira gshap...@cloudera.com wrote: I think thats how its done in pretty much any system I can think of.
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Please see all the available options here https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface#KIP-11-AuthorizationInterface-AclManagement(CLI) . I think it covers both hosts and operations and allows to specify a list for both. Thanks Parth From: Tom Graves tgraves...@yahoo.commailto:tgraves...@yahoo.com Reply-To: Tom Graves tgraves...@yahoo.commailto:tgraves...@yahoo.com Date: Wednesday, April 22, 2015 at 11:02 AM To: Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com, dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: Re: [DISCUSS] KIP-11- Authorization design for kafka security Thanks for the explanations Parth. On the configs questions, the way I see it is its more likely to accidentally give everyone access, especially since you have to run a separate command to change the acls. If there was some config for defaults, a cluster admin could change that to be nobody or certain set of users, then grant others permissions. This would also remove the race between commands. This is something you can always add later though if people request it. So in kafka-acl.sh how do I actually tell it what the operation is? kafka-acl.sh --topic testtopic --add --grandprincipal user:joe,user:kate where does READ, WRITE, etc go? Can specify as a list so I don't have to run this a bunch of times for each. Do you want to have a --host option for --list so that admins could see what acls apply to specific host(s)? Tom On Wednesday, April 22, 2015 11:38 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com wrote: FYI, I have modified the KIP to include group as resource. In order to access “joinGroup” and “commitOFfset” APIs the user will need a read permission on topic and WRITE permission on group. I plan to open a VOTE thread by noon if there are no more concerns. Thanks Parth On 4/22/15, 9:03 AM, Tom Graves tgraves...@yahoo.com.INVALIDmailto:tgraves...@yahoo.com.INVALID wrote: Hey everyone, Sorry to jump in on the conversation so late. I'm new to Kafka. I'll apologize in advance if you have already covered some of my questions. I read through the wiki and had some comments and questions. 1) public enum Operation needs EDIT changed to ALTER Done. 2) Does the Authorizer class need a setAcls? Rather then just add to be able to set to explicit list and overwrite what was there? I see the kafka-acl.sh lists a removeall so I guess you could do removeall and then add. I also don't see a removeall in the Authorizer class, is it going to loop through them all to remove each one? There is an overloaded version of removeAcls in the interface that takes in resource as the only input and as described in the javadoc all the acls attached to that resource will be deleted. To cover the setAcl use case the caller can first call remove and then add. 3) Can someone tell me what the use case to do acls based on the hosts? I can see some possibilities just wondering if we can concrete ones where one user is allowed from one host but not another. I am not sure if I understand the question given the use case you described in your question is what we are trying to cover with use of hosts in Acl. There are some additional use cases like “allow access to any user from host1,host2” but I think primarily it gives the admins the ability to define acls at a more granular level. 4) I'm a bit unclear how the resource works in the Authorizer class. From what I see we have 2 resources - topics and cluster. If I want to add an acl to allow joe to CREATE for the cluster then I call addAcls with Acl(user: joe, ALLOW, Set(*), Set(CREATE)) and cluster? What if I want to call addAcls for DESCRIBE on a topic? Is the resource then topic or is it the topic name? We now have 3 resources(added group), please see the updated doc. The CREATE acl that you described is correct. For any topic operation you should use topic name as the resource name and for group the user will provide groupId as resource name. 5) reassigning partitions is a CLUSTER_ACTION or superuser? Its not totally clear to me the differences between these. what about increasing # of partitions? I see this as an alter topic operation so it is at topic level and the user must have alter permissions on topic. 6) groups are mentioned, are we supporting right away or is that a follow on item? (is there going to be a kafka.supergroups) I think it can be a separate jira just for braking down the code review in smaller chunk. We will support it in first version but I think if we can not do it for any reason that should not block a release with all the other authZ work. We made deliberate design choices (like introducing a principalType in KafkaPrinciapl) to allow supporting groups as an incremental change. 7) Are there config options for setting acls when I create my topic? Or do I have
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Sorry I missed your last questions. I am +0 on adding ―host option for ―list, we could add it for symmetry. Again if this is only a CLI change it can be added later if you mean adding this in authorizer interface then we should make a decision now. Given a choice I would like to actually keep only one option which is resource based get (remove even the get based on principal). I see those (getAcl for principal or host) as special filtering case which can easily be achieved by a third party tool by doing list all topics and calling getAcls for each topic and applying filtering logic on that. I really don’t see the need to make those first class citizens of the authorizer interface given these kind of queries will be issued outside of broker JVM so they will not benefit from the caching and because the storage will be indexed on resource both these options even as a first class API will just scan all topic acls and apply filtering logic. Thanks Parth On 4/22/15, 11:08 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Please see all the available options here https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+I nterface#KIP-11-AuthorizationInterface-AclManagement(CLI) . I think it covers both hosts and operations and allows to specify a list for both. Thanks Parth From: Tom Graves tgraves...@yahoo.commailto:tgraves...@yahoo.com Reply-To: Tom Graves tgraves...@yahoo.commailto:tgraves...@yahoo.com Date: Wednesday, April 22, 2015 at 11:02 AM To: Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com, dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: Re: [DISCUSS] KIP-11- Authorization design for kafka security Thanks for the explanations Parth. On the configs questions, the way I see it is its more likely to accidentally give everyone access, especially since you have to run a separate command to change the acls. If there was some config for defaults, a cluster admin could change that to be nobody or certain set of users, then grant others permissions. This would also remove the race between commands. This is something you can always add later though if people request it. So in kafka-acl.sh how do I actually tell it what the operation is? kafka-acl.sh --topic testtopic --add --grandprincipal user:joe,user:kate where does READ, WRITE, etc go? Can specify as a list so I don't have to run this a bunch of times for each. Do you want to have a --host option for --list so that admins could see what acls apply to specific host(s)? Tom On Wednesday, April 22, 2015 11:38 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com wrote: FYI, I have modified the KIP to include group as resource. In order to access “joinGroup” and “commitOFfset” APIs the user will need a read permission on topic and WRITE permission on group. I plan to open a VOTE thread by noon if there are no more concerns. Thanks Parth On 4/22/15, 9:03 AM, Tom Graves tgraves...@yahoo.com.INVALIDmailto:tgraves...@yahoo.com.INVALID wrote: Hey everyone, Sorry to jump in on the conversation so late. I'm new to Kafka. I'll apologize in advance if you have already covered some of my questions. I read through the wiki and had some comments and questions. 1) public enum Operation needs EDIT changed to ALTER Done. 2) Does the Authorizer class need a setAcls? Rather then just add to be able to set to explicit list and overwrite what was there? I see the kafka-acl.sh lists a removeall so I guess you could do removeall and then add. I also don't see a removeall in the Authorizer class, is it going to loop through them all to remove each one? There is an overloaded version of removeAcls in the interface that takes in resource as the only input and as described in the javadoc all the acls attached to that resource will be deleted. To cover the setAcl use case the caller can first call remove and then add. 3) Can someone tell me what the use case to do acls based on the hosts? I can see some possibilities just wondering if we can concrete ones where one user is allowed from one host but not another. I am not sure if I understand the question given the use case you described in your question is what we are trying to cover with use of hosts in Acl. There are some additional use cases like “allow access to any user from host1,host2” but I think primarily it gives the admins the ability to define acls at a more granular level. 4) I'm a bit unclear how the resource works in the Authorizer class. From what I see we have 2 resources - topics and cluster. If I want to add an acl to allow joe to CREATE for the cluster then I call addAcls with Acl(user: joe, ALLOW, Set(*), Set(CREATE)) and cluster? What if I want to call addAcls for DESCRIBE on a topic? Is the resource then topic or is it the topic name? We now have 3 resources(added group), please see the updated doc. The CREATE acl that you
Re: [DISCUSS] KIP-11- Authorization design for kafka security
You are right , I forgot to mention the ―operation option in CLI , I just added it. Sorry for about the confusion. Thanks Parth On 4/22/15, 11:22 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Sorry I missed your last questions. I am +0 on adding ―host option for ―list, we could add it for symmetry. Again if this is only a CLI change it can be added later if you mean adding this in authorizer interface then we should make a decision now. Given a choice I would like to actually keep only one option which is resource based get (remove even the get based on principal). I see those (getAcl for principal or host) as special filtering case which can easily be achieved by a third party tool by doing list all topics and calling getAcls for each topic and applying filtering logic on that. I really don’t see the need to make those first class citizens of the authorizer interface given these kind of queries will be issued outside of broker JVM so they will not benefit from the caching and because the storage will be indexed on resource both these options even as a first class API will just scan all topic acls and apply filtering logic. Thanks Parth On 4/22/15, 11:08 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Please see all the available options here https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+ I nterface#KIP-11-AuthorizationInterface-AclManagement(CLI) . I think it covers both hosts and operations and allows to specify a list for both. Thanks Parth From: Tom Graves tgraves...@yahoo.commailto:tgraves...@yahoo.com Reply-To: Tom Graves tgraves...@yahoo.commailto:tgraves...@yahoo.com Date: Wednesday, April 22, 2015 at 11:02 AM To: Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com, dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: Re: [DISCUSS] KIP-11- Authorization design for kafka security Thanks for the explanations Parth. On the configs questions, the way I see it is its more likely to accidentally give everyone access, especially since you have to run a separate command to change the acls. If there was some config for defaults, a cluster admin could change that to be nobody or certain set of users, then grant others permissions. This would also remove the race between commands. This is something you can always add later though if people request it. So in kafka-acl.sh how do I actually tell it what the operation is? kafka-acl.sh --topic testtopic --add --grandprincipal user:joe,user:kate where does READ, WRITE, etc go? Can specify as a list so I don't have to run this a bunch of times for each. Do you want to have a --host option for --list so that admins could see what acls apply to specific host(s)? Tom On Wednesday, April 22, 2015 11:38 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com wrote: FYI, I have modified the KIP to include group as resource. In order to access “joinGroup” and “commitOFfset” APIs the user will need a read permission on topic and WRITE permission on group. I plan to open a VOTE thread by noon if there are no more concerns. Thanks Parth On 4/22/15, 9:03 AM, Tom Graves tgraves...@yahoo.com.INVALIDmailto:tgraves...@yahoo.com.INVALID wrote: Hey everyone, Sorry to jump in on the conversation so late. I'm new to Kafka. I'll apologize in advance if you have already covered some of my questions. I read through the wiki and had some comments and questions. 1) public enum Operation needs EDIT changed to ALTER Done. 2) Does the Authorizer class need a setAcls? Rather then just add to be able to set to explicit list and overwrite what was there? I see the kafka-acl.sh lists a removeall so I guess you could do removeall and then add. I also don't see a removeall in the Authorizer class, is it going to loop through them all to remove each one? There is an overloaded version of removeAcls in the interface that takes in resource as the only input and as described in the javadoc all the acls attached to that resource will be deleted. To cover the setAcl use case the caller can first call remove and then add. 3) Can someone tell me what the use case to do acls based on the hosts? I can see some possibilities just wondering if we can concrete ones where one user is allowed from one host but not another. I am not sure if I understand the question given the use case you described in your question is what we are trying to cover with use of hosts in Acl. There are some additional use cases like “allow access to any user from host1,host2” but I think primarily it gives the admins the ability to define acls at a more granular level. 4) I'm a bit unclear how the resource works in the Authorizer class. From what I see we have 2 resources - topics and cluster. If I want to add an acl to allow joe to CREATE for the cluster then I call addAcls with Acl(user: joe, ALLOW, Set(*), Set(CREATE)) and cluster
Re: [VOTE] KIP-11- Authorization design for kafka security
I see Groups as something we can add incrementally in the current model. The acls take principalType: name so groups can be represented as group: groupName. We are not managing group memberships anywhere in kafka and I don’t see the need to do so. So for a topic1 using the CLI an admin can add an acl to grant access to group:kafka-test-users. The authorizer implementation can have a plugin to map authenticated user to groups ( This is how hadoop and storm works). The plugin could be mapping user to linux/ldap/active directory groups but that is again upto the implementation. What we are offering is an interface that is extensible so these features can be added incrementally. I can add support for this in the first release but don’t necessarily see why this would be absolute necessity. Thanks Parth On 4/24/15, 11:00 AM, Gwen Shapira gshap...@cloudera.com wrote: Thanks. One more thing I'm missing in the KIP is details on the Group resource (I think we discussed this and it was just not fully updated): * Does everyone have the privilege to create a new Group and use it to consume from Topics he's already privileged on? * Will the CLI tool be used to manage group membership too? * Groups are kind of ephemeral, right? If all consumers in the group disconnect the group is gone, AFAIK. Do we preserve the ACLs? Or do we treat the new group as completely new resource? Can we create ACLs before the group exists, in anticipation of it getting created? Its all small details, but it will be difficult to implement KIP-11 without knowing the answers :) Gwen On Fri, Apr 24, 2015 at 9:58 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: You are right, moved it to the default implementation section. Thanks Parth On 4/24/15, 9:52 AM, Gwen Shapira gshap...@cloudera.com wrote: Sample ACL JSON and Zookeeper is in public API, but I thought it is part of DefaultAuthorizer (Since Sentry and Argus won't be using Zookeeper). Am I wrong? Or is it the KIP? On Fri, Apr 24, 2015 at 9:49 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Thanks for clarifying Gwen, KIP updated. I tried to make the distinction by creating a section for all public APIs https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorizatio n+ In terface#KIP-11-AuthorizationInterface-PublicInterfacesandclasses Let me know if you think there is a better way to reflect this. Thanks Parth On 4/24/15, 9:37 AM, Gwen Shapira gshap...@cloudera.com wrote: +1 (non-binding) Two nitpicks for the wiki: * Heartbeat is probably a READ and not CLUSTER operation. I'm pretty sure new consumers need it to be part of a consumer group. * Can you clearly separate which parts are the API (common to every Authorizer) and which parts are DefaultAuthorizer implementation? It will make reviews and Authorizer implementations a bit easier to know exactly which is which. Gwen On Fri, Apr 24, 2015 at 9:28 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi, I would like to open KIP-11 for voting. Thanks Parth On 4/22/15, 1:56 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi Jeff, Thanks a lot for the review. I think you have a valid point about acls being duplicated and the simplest solution would be to modify acls class so they hold a set of principals instead of single principal. i.e user_a,user_b has READ,WRITE,DESCRIBE Permissions on Topic1 from Host1, Host2, Host3. I think the evaluation order only matters for the permissionType which is Deny acls should be evaluated before allow acls. To give you an example suppose we have following acls acl1 - user1 is allowed to READ from all hosts. acl2 - host1 is allowed to READ regardless of who is the user. acl3 - host2 is allowed to READ regardless of who is the user. acl4 - user1 is denied to READ from host1. As stated in the KIP we first evaluate DENY so if user1 tries to access from host1 he will be denied(acl4), even though both user1 and host1 has acl’s for allow with wildcards (acl1, acl2). If user1 tried to READ from host2 , the action will be allowed and it does not matter if we match acl3 or acl1 so I don’t think the evaluation order matters here. “Will people actually use hosts with users?” I really don’t know but given ACl’s are part of our Public APIs I thought it is better to try and cover more use cases. If others think this extra complexity is not worth the value its adding please raise your concerns so we can discuss if it should be removed from the acl structure. Note that even in absence of hosts from ACL users will still be able to whitelist/blacklist host as long as we start supporting principalType = “host”, easy to add and can be an incremental improvement. They will however loose the ability to restrict access to users just from a set of hosts. We agreed to offer a CLI to overcome the JSON acl config https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authoriza ti on +I n terface#KIP-11-AuthorizationInterface-AclManagement(CLI
Re: [VOTE] KIP-11- Authorization design for kafka security
would then modify the ACL to look like: {version:1, {acls:[ { principal_types:[KafkaUserPrincipal,KafkaGroupPrincipal], principals:[alice,kafka-devs 3) The advantage of all of this is that it now provides more flexibility for custom modules for both authentication and authorization moving forward. On Fri, Apr 24, 2015 at 12:37 PM, Gwen Shapira gshap...@cloudera.com wrote: +1 (non-binding) Two nitpicks for the wiki: * Heartbeat is probably a READ and not CLUSTER operation. I'm pretty sure new consumers need it to be part of a consumer group. * Can you clearly separate which parts are the API (common to every Authorizer) and which parts are DefaultAuthorizer implementation? It will make reviews and Authorizer implementations a bit easier to know exactly which is which. Gwen On Fri, Apr 24, 2015 at 9:28 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi, I would like to open KIP-11 for voting. Thanks Parth On 4/22/15, 1:56 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi Jeff, Thanks a lot for the review. I think you have a valid point about acls being duplicated and the simplest solution would be to modify acls class so they hold a set of principals instead of single principal. i.e user_a,user_b has READ,WRITE,DESCRIBE Permissions on Topic1 from Host1, Host2, Host3. I think the evaluation order only matters for the permissionType which is Deny acls should be evaluated before allow acls. To give you an example suppose we have following acls acl1 - user1 is allowed to READ from all hosts. acl2 - host1 is allowed to READ regardless of who is the user. acl3 - host2 is allowed to READ regardless of who is the user. acl4 - user1 is denied to READ from host1. As stated in the KIP we first evaluate DENY so if user1 tries to access from host1 he will be denied(acl4), even though both user1 and host1 has acl’s for allow with wildcards (acl1, acl2). If user1 tried to READ from host2 , the action will be allowed and it does not matter if we match acl3 or acl1 so I don’t think the evaluation order matters here. “Will people actually use hosts with users?” I really don’t know but given ACl’s are part of our Public APIs I thought it is better to try and cover more use cases. If others think this extra complexity is not worth the value its adding please raise your concerns so we can discuss if it should be removed from the acl structure. Note that even in absence of hosts from ACL users will still be able to whitelist/blacklist host as long as we start supporting principalType = “host”, easy to add and can be an incremental improvement. They will however loose the ability to restrict access to users just from a set of hosts. We agreed to offer a CLI to overcome the JSON acl config https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization +I n terface#KIP-11-AuthorizationInterface-AclManagement(CLI). I still like Jsons but that probably has something to do with me being a developer :-). Thanks Parth On 4/22/15, 11:38 AM, Jeff Holoman jholo...@cloudera.com wrote: Parth, This is a long thread, so trying to keep up here, sorry if this has been covered before. First, great job on the KIP proposal and work so far. Are we sure that we want to tie host level access to a given user? My understanding is that the ACL will be (omitting some fields) user_a, host1, host2, host3 user_b, host1, host2, host3 So there would potentially be a lot of redundancy in the configs. Does it make sense to have hosts be at the same level as principal in the hierarchy? This way you could just blanket the allowed / denied hosts and only have to worry about the users. So if you follow this, then we can wildcard the user so we can have a separate list of just host-based access. What's the order that the perms would be evaluated if a there was more than one match on a principal ? Is the thought that there wouldn't usually be much overlap on hosts? I guess I can imagine a scenario where I want to offline/online access to a particular hosts or set of hosts and if there was overlap, I'm doing a bunch of alter commands for just a single host. Maybe this is too contrived an example? I agree that having this level of granularity gives flexibility but I wonder if people will actually use it and not just * the hosts for a given user and create separate global list as i mentioned above? The only other system I know of that ties users with hosts for access is MySql and I don't love that model. Companies usually standardize on group authorization anyway, are we complicating that issue with the inclusion of hosts attached to users? Additionally I worry about the debt of big JSON configs in the first place, most non-developers find them non-intuitive already, so anything to ease this I think would be beneficial. Thanks Jeff On Wed, Apr 22
Re: [VOTE] KIP-11- Authorization design for kafka security
You are right, moved it to the default implementation section. Thanks Parth On 4/24/15, 9:52 AM, Gwen Shapira gshap...@cloudera.com wrote: Sample ACL JSON and Zookeeper is in public API, but I thought it is part of DefaultAuthorizer (Since Sentry and Argus won't be using Zookeeper). Am I wrong? Or is it the KIP? On Fri, Apr 24, 2015 at 9:49 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Thanks for clarifying Gwen, KIP updated. I tried to make the distinction by creating a section for all public APIs https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+ In terface#KIP-11-AuthorizationInterface-PublicInterfacesandclasses Let me know if you think there is a better way to reflect this. Thanks Parth On 4/24/15, 9:37 AM, Gwen Shapira gshap...@cloudera.com wrote: +1 (non-binding) Two nitpicks for the wiki: * Heartbeat is probably a READ and not CLUSTER operation. I'm pretty sure new consumers need it to be part of a consumer group. * Can you clearly separate which parts are the API (common to every Authorizer) and which parts are DefaultAuthorizer implementation? It will make reviews and Authorizer implementations a bit easier to know exactly which is which. Gwen On Fri, Apr 24, 2015 at 9:28 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi, I would like to open KIP-11 for voting. Thanks Parth On 4/22/15, 1:56 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi Jeff, Thanks a lot for the review. I think you have a valid point about acls being duplicated and the simplest solution would be to modify acls class so they hold a set of principals instead of single principal. i.e user_a,user_b has READ,WRITE,DESCRIBE Permissions on Topic1 from Host1, Host2, Host3. I think the evaluation order only matters for the permissionType which is Deny acls should be evaluated before allow acls. To give you an example suppose we have following acls acl1 - user1 is allowed to READ from all hosts. acl2 - host1 is allowed to READ regardless of who is the user. acl3 - host2 is allowed to READ regardless of who is the user. acl4 - user1 is denied to READ from host1. As stated in the KIP we first evaluate DENY so if user1 tries to access from host1 he will be denied(acl4), even though both user1 and host1 has acl’s for allow with wildcards (acl1, acl2). If user1 tried to READ from host2 , the action will be allowed and it does not matter if we match acl3 or acl1 so I don’t think the evaluation order matters here. “Will people actually use hosts with users?” I really don’t know but given ACl’s are part of our Public APIs I thought it is better to try and cover more use cases. If others think this extra complexity is not worth the value its adding please raise your concerns so we can discuss if it should be removed from the acl structure. Note that even in absence of hosts from ACL users will still be able to whitelist/blacklist host as long as we start supporting principalType = “host”, easy to add and can be an incremental improvement. They will however loose the ability to restrict access to users just from a set of hosts. We agreed to offer a CLI to overcome the JSON acl config https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorizati on +I n terface#KIP-11-AuthorizationInterface-AclManagement(CLI). I still like Jsons but that probably has something to do with me being a developer :-). Thanks Parth On 4/22/15, 11:38 AM, Jeff Holoman jholo...@cloudera.com wrote: Parth, This is a long thread, so trying to keep up here, sorry if this has been covered before. First, great job on the KIP proposal and work so far. Are we sure that we want to tie host level access to a given user? My understanding is that the ACL will be (omitting some fields) user_a, host1, host2, host3 user_b, host1, host2, host3 So there would potentially be a lot of redundancy in the configs. Does it make sense to have hosts be at the same level as principal in the hierarchy? This way you could just blanket the allowed / denied hosts and only have to worry about the users. So if you follow this, then we can wildcard the user so we can have a separate list of just host-based access. What's the order that the perms would be evaluated if a there was more than one match on a principal ? Is the thought that there wouldn't usually be much overlap on hosts? I guess I can imagine a scenario where I want to offline/online access to a particular hosts or set of hosts and if there was overlap, I'm doing a bunch of alter commands for just a single host. Maybe this is too contrived an example? I agree that having this level of granularity gives flexibility but I wonder if people will actually use it and not just * the hosts for a given user and create separate global list as i mentioned above? The only other system I know of that ties users with hosts for access is MySql and I don't love that model. Companies usually standardize on group authorization anyway, are we
Re: [VOTE] KIP-11- Authorization design for kafka security
Hi, I would like to open KIP-11 for voting. Thanks Parth On 4/22/15, 1:56 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi Jeff, Thanks a lot for the review. I think you have a valid point about acls being duplicated and the simplest solution would be to modify acls class so they hold a set of principals instead of single principal. i.e user_a,user_b has READ,WRITE,DESCRIBE Permissions on Topic1 from Host1, Host2, Host3. I think the evaluation order only matters for the permissionType which is Deny acls should be evaluated before allow acls. To give you an example suppose we have following acls acl1 - user1 is allowed to READ from all hosts. acl2 - host1 is allowed to READ regardless of who is the user. acl3 - host2 is allowed to READ regardless of who is the user. acl4 - user1 is denied to READ from host1. As stated in the KIP we first evaluate DENY so if user1 tries to access from host1 he will be denied(acl4), even though both user1 and host1 has acl’s for allow with wildcards (acl1, acl2). If user1 tried to READ from host2 , the action will be allowed and it does not matter if we match acl3 or acl1 so I don’t think the evaluation order matters here. “Will people actually use hosts with users?” I really don’t know but given ACl’s are part of our Public APIs I thought it is better to try and cover more use cases. If others think this extra complexity is not worth the value its adding please raise your concerns so we can discuss if it should be removed from the acl structure. Note that even in absence of hosts from ACL users will still be able to whitelist/blacklist host as long as we start supporting principalType = “host”, easy to add and can be an incremental improvement. They will however loose the ability to restrict access to users just from a set of hosts. We agreed to offer a CLI to overcome the JSON acl config https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+I n terface#KIP-11-AuthorizationInterface-AclManagement(CLI). I still like Jsons but that probably has something to do with me being a developer :-). Thanks Parth On 4/22/15, 11:38 AM, Jeff Holoman jholo...@cloudera.com wrote: Parth, This is a long thread, so trying to keep up here, sorry if this has been covered before. First, great job on the KIP proposal and work so far. Are we sure that we want to tie host level access to a given user? My understanding is that the ACL will be (omitting some fields) user_a, host1, host2, host3 user_b, host1, host2, host3 So there would potentially be a lot of redundancy in the configs. Does it make sense to have hosts be at the same level as principal in the hierarchy? This way you could just blanket the allowed / denied hosts and only have to worry about the users. So if you follow this, then we can wildcard the user so we can have a separate list of just host-based access. What's the order that the perms would be evaluated if a there was more than one match on a principal ? Is the thought that there wouldn't usually be much overlap on hosts? I guess I can imagine a scenario where I want to offline/online access to a particular hosts or set of hosts and if there was overlap, I'm doing a bunch of alter commands for just a single host. Maybe this is too contrived an example? I agree that having this level of granularity gives flexibility but I wonder if people will actually use it and not just * the hosts for a given user and create separate global list as i mentioned above? The only other system I know of that ties users with hosts for access is MySql and I don't love that model. Companies usually standardize on group authorization anyway, are we complicating that issue with the inclusion of hosts attached to users? Additionally I worry about the debt of big JSON configs in the first place, most non-developers find them non-intuitive already, so anything to ease this I think would be beneficial. Thanks Jeff On Wed, Apr 22, 2015 at 2:22 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Sorry I missed your last questions. I am +0 on adding ―host option for ―list, we could add it for symmetry. Again if this is only a CLI change it can be added later if you mean adding this in authorizer interface then we should make a decision now. Given a choice I would like to actually keep only one option which is resource based get (remove even the get based on principal). I see those (getAcl for principal or host) as special filtering case which can easily be achieved by a third party tool by doing list all topics and calling getAcls for each topic and applying filtering logic on that. I really don’t see the need to make those first class citizens of the authorizer interface given these kind of queries will be issued outside of broker JVM so they will not benefit from the caching and because the storage will be indexed on resource both these options even as a first class API will just scan all topic acls and apply filtering logic
Re: [VOTE] KIP-11- Authorization design for kafka security
Thanks for clarifying Gwen, KIP updated. I tried to make the distinction by creating a section for all public APIs https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+In terface#KIP-11-AuthorizationInterface-PublicInterfacesandclasses Let me know if you think there is a better way to reflect this. Thanks Parth On 4/24/15, 9:37 AM, Gwen Shapira gshap...@cloudera.com wrote: +1 (non-binding) Two nitpicks for the wiki: * Heartbeat is probably a READ and not CLUSTER operation. I'm pretty sure new consumers need it to be part of a consumer group. * Can you clearly separate which parts are the API (common to every Authorizer) and which parts are DefaultAuthorizer implementation? It will make reviews and Authorizer implementations a bit easier to know exactly which is which. Gwen On Fri, Apr 24, 2015 at 9:28 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi, I would like to open KIP-11 for voting. Thanks Parth On 4/22/15, 1:56 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi Jeff, Thanks a lot for the review. I think you have a valid point about acls being duplicated and the simplest solution would be to modify acls class so they hold a set of principals instead of single principal. i.e user_a,user_b has READ,WRITE,DESCRIBE Permissions on Topic1 from Host1, Host2, Host3. I think the evaluation order only matters for the permissionType which is Deny acls should be evaluated before allow acls. To give you an example suppose we have following acls acl1 - user1 is allowed to READ from all hosts. acl2 - host1 is allowed to READ regardless of who is the user. acl3 - host2 is allowed to READ regardless of who is the user. acl4 - user1 is denied to READ from host1. As stated in the KIP we first evaluate DENY so if user1 tries to access from host1 he will be denied(acl4), even though both user1 and host1 has acl’s for allow with wildcards (acl1, acl2). If user1 tried to READ from host2 , the action will be allowed and it does not matter if we match acl3 or acl1 so I don’t think the evaluation order matters here. “Will people actually use hosts with users?” I really don’t know but given ACl’s are part of our Public APIs I thought it is better to try and cover more use cases. If others think this extra complexity is not worth the value its adding please raise your concerns so we can discuss if it should be removed from the acl structure. Note that even in absence of hosts from ACL users will still be able to whitelist/blacklist host as long as we start supporting principalType = “host”, easy to add and can be an incremental improvement. They will however loose the ability to restrict access to users just from a set of hosts. We agreed to offer a CLI to overcome the JSON acl config https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization +I n terface#KIP-11-AuthorizationInterface-AclManagement(CLI). I still like Jsons but that probably has something to do with me being a developer :-). Thanks Parth On 4/22/15, 11:38 AM, Jeff Holoman jholo...@cloudera.com wrote: Parth, This is a long thread, so trying to keep up here, sorry if this has been covered before. First, great job on the KIP proposal and work so far. Are we sure that we want to tie host level access to a given user? My understanding is that the ACL will be (omitting some fields) user_a, host1, host2, host3 user_b, host1, host2, host3 So there would potentially be a lot of redundancy in the configs. Does it make sense to have hosts be at the same level as principal in the hierarchy? This way you could just blanket the allowed / denied hosts and only have to worry about the users. So if you follow this, then we can wildcard the user so we can have a separate list of just host-based access. What's the order that the perms would be evaluated if a there was more than one match on a principal ? Is the thought that there wouldn't usually be much overlap on hosts? I guess I can imagine a scenario where I want to offline/online access to a particular hosts or set of hosts and if there was overlap, I'm doing a bunch of alter commands for just a single host. Maybe this is too contrived an example? I agree that having this level of granularity gives flexibility but I wonder if people will actually use it and not just * the hosts for a given user and create separate global list as i mentioned above? The only other system I know of that ties users with hosts for access is MySql and I don't love that model. Companies usually standardize on group authorization anyway, are we complicating that issue with the inclusion of hosts attached to users? Additionally I worry about the debt of big JSON configs in the first place, most non-developers find them non-intuitive already, so anything to ease this I think would be beneficial. Thanks Jeff On Wed, Apr 22, 2015 at 2:22 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Sorry I missed your last questions. I am +0 on adding ―host option
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Hi Jeff, Thanks a lot for the review. I think you have a valid point about acls being duplicated and the simplest solution would be to modify acls class so they hold a set of principals instead of single principal. i.e user_a,user_b has READ,WRITE,DESCRIBE Permissions on Topic1 from Host1, Host2, Host3. I think the evaluation order only matters for the permissionType which is Deny acls should be evaluated before allow acls. To give you an example suppose we have following acls acl1 - user1 is allowed to READ from all hosts. acl2 - host1 is allowed to READ regardless of who is the user. acl3 - host2 is allowed to READ regardless of who is the user. acl4 - user1 is denied to READ from host1. As stated in the KIP we first evaluate DENY so if user1 tries to access from host1 he will be denied(acl4), even though both user1 and host1 has acl’s for allow with wildcards (acl1, acl2). If user1 tried to READ from host2 , the action will be allowed and it does not matter if we match acl3 or acl1 so I don’t think the evaluation order matters here. “Will people actually use hosts with users?” I really don’t know but given ACl’s are part of our Public APIs I thought it is better to try and cover more use cases. If others think this extra complexity is not worth the value its adding please raise your concerns so we can discuss if it should be removed from the acl structure. Note that even in absence of hosts from ACL users will still be able to whitelist/blacklist host as long as we start supporting principalType = “host”, easy to add and can be an incremental improvement. They will however loose the ability to restrict access to users just from a set of hosts. We agreed to offer a CLI to overcome the JSON acl config https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+In terface#KIP-11-AuthorizationInterface-AclManagement(CLI). I still like Jsons but that probably has something to do with me being a developer :-). Thanks Parth On 4/22/15, 11:38 AM, Jeff Holoman jholo...@cloudera.com wrote: Parth, This is a long thread, so trying to keep up here, sorry if this has been covered before. First, great job on the KIP proposal and work so far. Are we sure that we want to tie host level access to a given user? My understanding is that the ACL will be (omitting some fields) user_a, host1, host2, host3 user_b, host1, host2, host3 So there would potentially be a lot of redundancy in the configs. Does it make sense to have hosts be at the same level as principal in the hierarchy? This way you could just blanket the allowed / denied hosts and only have to worry about the users. So if you follow this, then we can wildcard the user so we can have a separate list of just host-based access. What's the order that the perms would be evaluated if a there was more than one match on a principal ? Is the thought that there wouldn't usually be much overlap on hosts? I guess I can imagine a scenario where I want to offline/online access to a particular hosts or set of hosts and if there was overlap, I'm doing a bunch of alter commands for just a single host. Maybe this is too contrived an example? I agree that having this level of granularity gives flexibility but I wonder if people will actually use it and not just * the hosts for a given user and create separate global list as i mentioned above? The only other system I know of that ties users with hosts for access is MySql and I don't love that model. Companies usually standardize on group authorization anyway, are we complicating that issue with the inclusion of hosts attached to users? Additionally I worry about the debt of big JSON configs in the first place, most non-developers find them non-intuitive already, so anything to ease this I think would be beneficial. Thanks Jeff On Wed, Apr 22, 2015 at 2:22 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Sorry I missed your last questions. I am +0 on adding ―host option for ―list, we could add it for symmetry. Again if this is only a CLI change it can be added later if you mean adding this in authorizer interface then we should make a decision now. Given a choice I would like to actually keep only one option which is resource based get (remove even the get based on principal). I see those (getAcl for principal or host) as special filtering case which can easily be achieved by a third party tool by doing list all topics and calling getAcls for each topic and applying filtering logic on that. I really don’t see the need to make those first class citizens of the authorizer interface given these kind of queries will be issued outside of broker JVM so they will not benefit from the caching and because the storage will be indexed on resource both these options even as a first class API will just scan all topic acls and apply filtering logic. Thanks Parth On 4/22/15, 11:08 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Please see all the available options here
Re: [DISCUSS] KIP-11- Authorization design for kafka security
hosts and no other user has any access(except for superusers who have all the access). - Deny ACLS are suppose to be used to restrict access authorized by some allow ACL, they are not suppose to be required. Implicitly anyone who does not have an allow acl, gets denied. The Deny ACLs are only added to give more control to administrators who wants more granular control with lesser config. The scenario described in mailing list was “Allow user X access from all hosts but Host1,Host2”. in absence of DENY operator you will have to exhaustively list all possible hosts in your ACL which is what we are trying to avoid. - On ZK ACLs: I think ZK will be closed to everyone except Kafka brokers. This is a dependency on KIP-4 though. i.e., eventually all clients should talk to brokers only via RPC. Yes. - Topic owner: list vs single entry - both have issues off the bat (although list is more intuitive at least to me), but perhaps you could write up some example workflows to clarify the current proposal. I was thinking that anyone in the owner list should be considered a super-user of the topic and can grant/revoke permissions. They should also be allowed to add other principals as owners. Even with this it is unclear who should be allowed to remove owners. As you pointed out in the last KIP meeting owners/creators have use out side of security context (plain simple auditing). I don’t think the authorizer work depends on this, it was my bad to even mention it in first place. I think we can have this discussion outside of authorizer/security context and once we have a way to get topic owners the default Authorizer can start using it. It makes sense to treat all owners as super users and I think it is safe to assume superusers can also modify ownership but I think this should not be treated as blocking work for authorization. - What is the effect of deleting a topic - should all associated ACLs be deleted as well? They should be and with acls being stored as part of TopicConfig this was taken care of automatically. With the new ACL management API the users will have to call remove ACLs explicitly to perform the cleanup. If everyone thinks this should be automated , with the new APIs we will need a hook(or poll) to be notified when a topic is deleted to perform cleanup. - TopicConfigCache to store topic-ACLs. As mentioned above, not all requests will be tied to topics. We may want to have an entirely separate ZK directory for ACLs. We have a similar issue with quotas. This ties in with dynamic config management. We can certainly leverage the dynamic config management part of topic configs but I think we need to have a story for non-topic resources. In the first proposal I was going with a topic-Acl and cluster-Acl where cluster-Acls were json acl local files on all brokers. With the new ACL management APIs we are planning to have /kafka-acl node under which all acls will be stored in /kakfa-acls/resource-name - {acl json data}. Cluster acls will just have resource name kafka-cluster. Thanks, Joel On Thu, Apr 16, 2015 at 12:15:37AM +, Parth Brahmbhatt wrote: Kafka currently stores logConfig overrides specified during topic creation in zookeeper, its just an instance of java.util.Properties converted to json. I am proposing in addition to that we store acls and owner as well as part of same Properties map. There is some infrastructure around reading this config, converting it back to Properties map and most importantly propagating any changes efficiently which we will be able to leverage. As this infrastructure is common to the cluster the reading (not interpreting) of config happens outside of any authorization code. If the TopicConfigCache just kept the json representation and left it to authorizer to parse it, the authorizer will have to either parse the json for each request(not acceptable) or it will have to keep one more layer of parsed ACL instance cache. Assuming authorizer will keep an additional caching layer we will now have to implement some way to invalidate the cache which means the TopicConfigCache will have to be an observable which the Authorizer observes and invalidates its cache entries when topicConfigCache gets updated. Seemed like unnecessary complexity with not lot to gain so I went with TopicConfigCache interpreting the json and caching a higher level modeled object. In summary, the interpretation is done for both optimization and simplicity. If you think it is important to allow custom ACL format support we can add one more pluggable config(acl.parser) and interface(AclParser) or it could just be another method in Authorizer. One thing to note the current ACL json is versioned so it is easy to make changes to it however it won’t be possible to support custom ACL formats with the current design. Thanks Parth On 4/15/15, 4:29 PM, Michael Herstine mherst...@linkedin.com.INVALIDmailto:mherst...@linkedin.com.INVALID wrote: Hi Parth, I’m
Re: [VOTE] KIP-11- Authorization design for kafka security
Sorry Gwen, completely misunderstood the question :-). * Does everyone have the privilege to create a new Group and use it to consume from Topics he's already privileged on? Yes in current proposal. I did not see an API to create group but if you have a READ permission on a TOPIC and WRITE permission on that Group you are free to join and consume. * Will the CLI tool be used to manage group membership too? Yes and I think that means I need to add ―group. Updating the KIP. Thanks for pointing this out. * Groups are kind of ephemeral, right? If all consumers in the group disconnect the group is gone, AFAIK. Do we preserve the ACLs? Or do we treat the new group as completely new resource? Can we create ACLs before the group exists, in anticipation of it getting created? I have considered any auto delete and auto create as out of scope for the first release. So Right now I was going with preserving the acls. Do you see any issues with this? Auto deleting would mean authorizer will now have to get into implementation details of kafka which I was trying to avoid. Thanks Parth On 4/24/15, 11:33 AM, Gwen Shapira gshap...@cloudera.com wrote: We are not talking about same Groups :) I meant, Groups of consumers (which KIP-11 lists as a separate resource in the Privilege table) On Fri, Apr 24, 2015 at 11:31 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: I see Groups as something we can add incrementally in the current model. The acls take principalType: name so groups can be represented as group: groupName. We are not managing group memberships anywhere in kafka and I don’t see the need to do so. So for a topic1 using the CLI an admin can add an acl to grant access to group:kafka-test-users. The authorizer implementation can have a plugin to map authenticated user to groups ( This is how hadoop and storm works). The plugin could be mapping user to linux/ldap/active directory groups but that is again upto the implementation. What we are offering is an interface that is extensible so these features can be added incrementally. I can add support for this in the first release but don’t necessarily see why this would be absolute necessity. Thanks Parth On 4/24/15, 11:00 AM, Gwen Shapira gshap...@cloudera.com wrote: Thanks. One more thing I'm missing in the KIP is details on the Group resource (I think we discussed this and it was just not fully updated): * Does everyone have the privilege to create a new Group and use it to consume from Topics he's already privileged on? * Will the CLI tool be used to manage group membership too? * Groups are kind of ephemeral, right? If all consumers in the group disconnect the group is gone, AFAIK. Do we preserve the ACLs? Or do we treat the new group as completely new resource? Can we create ACLs before the group exists, in anticipation of it getting created? Its all small details, but it will be difficult to implement KIP-11 without knowing the answers :) Gwen On Fri, Apr 24, 2015 at 9:58 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: You are right, moved it to the default implementation section. Thanks Parth On 4/24/15, 9:52 AM, Gwen Shapira gshap...@cloudera.com wrote: Sample ACL JSON and Zookeeper is in public API, but I thought it is part of DefaultAuthorizer (Since Sentry and Argus won't be using Zookeeper). Am I wrong? Or is it the KIP? On Fri, Apr 24, 2015 at 9:49 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Thanks for clarifying Gwen, KIP updated. I tried to make the distinction by creating a section for all public APIs https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorizat io n+ In terface#KIP-11-AuthorizationInterface-PublicInterfacesandclasses Let me know if you think there is a better way to reflect this. Thanks Parth On 4/24/15, 9:37 AM, Gwen Shapira gshap...@cloudera.com wrote: +1 (non-binding) Two nitpicks for the wiki: * Heartbeat is probably a READ and not CLUSTER operation. I'm pretty sure new consumers need it to be part of a consumer group. * Can you clearly separate which parts are the API (common to every Authorizer) and which parts are DefaultAuthorizer implementation? It will make reviews and Authorizer implementations a bit easier to know exactly which is which. Gwen On Fri, Apr 24, 2015 at 9:28 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi, I would like to open KIP-11 for voting. Thanks Parth On 4/22/15, 1:56 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi Jeff, Thanks a lot for the review. I think you have a valid point about acls being duplicated and the simplest solution would be to modify acls class so they hold a set of principals instead of single principal. i.e user_a,user_b has READ,WRITE,DESCRIBE Permissions on Topic1 from Host1, Host2, Host3. I think the evaluation order only matters for the permissionType which is Deny acls should be evaluated before allow acls. To give you
Re: [VOTE] KIP-11- Authorization design for kafka security
* We are not supporting regex matching to any of the strings (host,resource,principal) yet but this can be added. We have a special wild card (*) to refer to ALL but there is no other regex matching going on right now. We can associate CREATE with topics as you are proposing once KIP-4 is merged I am just not sure if admins currently try to figure out/control what topic names different tenents can have. * With current API they will have to do exactly what you said. Call list for each resource (cluster, topic and group) and reissue the same acls by calling add in the mirrored cluster. Thanks Parth On 4/27/15, 2:17 PM, Jun Rao j...@confluent.io wrote: Parth, I was thinking that in a multi-tenant environment, an admin may want to carve out some topic space to a user. For example, allow user X to create any topic of X_*. Not sure how critical it is though. Also, with the current api, what would the admin do to replicate the acls from one cluster to another? Will she just list all acls from cli and reissue them to another cluster periodically? Thanks, Jun On Mon, Apr 27, 2015 at 10:56 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Thanks for your comments Jun. * Renamed the resource to consumer-group in wiki. * I don’t see a use case where admins/users would want to reserve topic names in advance. Can you describe why this would be needed. Thanks Parth On 4/26/15, 2:01 PM, Jun Rao j...@confluent.io wrote: A few more minor comments. 100. To make it clear, perhaps we should rename the resource group to consumer-group. We can probably make the same change in CLI as well so that it's not confused with user group. 101. Currently, create is only at the cluster level. Should it also be at topic level? For example, perhaps it's useful to allow only user X to create topic X. Thanks, Jun On Sun, Apr 26, 2015 at 12:36 AM, Gwen Shapira gshap...@cloudera.com wrote: Thanks for clarifying, Parth. I think you are taking the right approach here. On Fri, Apr 24, 2015 at 11:46 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Sorry Gwen, completely misunderstood the question :-). * Does everyone have the privilege to create a new Group and use it to consume from Topics he's already privileged on? Yes in current proposal. I did not see an API to create group but if you have a READ permission on a TOPIC and WRITE permission on that Group you are free to join and consume. * Will the CLI tool be used to manage group membership too? Yes and I think that means I need to add ―group. Updating the KIP. Thanks for pointing this out. * Groups are kind of ephemeral, right? If all consumers in the group disconnect the group is gone, AFAIK. Do we preserve the ACLs? Or do we treat the new group as completely new resource? Can we create ACLs before the group exists, in anticipation of it getting created? I have considered any auto delete and auto create as out of scope for the first release. So Right now I was going with preserving the acls. Do you see any issues with this? Auto deleting would mean authorizer will now have to get into implementation details of kafka which I was trying to avoid. Thanks Parth On 4/24/15, 11:33 AM, Gwen Shapira gshap...@cloudera.com wrote: We are not talking about same Groups :) I meant, Groups of consumers (which KIP-11 lists as a separate resource in the Privilege table) On Fri, Apr 24, 2015 at 11:31 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: I see Groups as something we can add incrementally in the current model. The acls take principalType: name so groups can be represented as group: groupName. We are not managing group memberships anywhere in kafka and I don’t see the need to do so. So for a topic1 using the CLI an admin can add an acl to grant access to group:kafka-test-users. The authorizer implementation can have a plugin to map authenticated user to groups ( This is how hadoop and storm works). The plugin could be mapping user to linux/ldap/active directory groups but that is again upto the implementation. What we are offering is an interface that is extensible so these features can be added incrementally. I can add support for this in the first release but don’t necessarily see why this would be absolute necessity. Thanks Parth On 4/24/15, 11:00 AM, Gwen Shapira gshap...@cloudera.com wrote: Thanks. One more thing I'm missing in the KIP is details on the Group resource (I think we discussed this and it was just not fully updated): * Does everyone have the privilege to create a new Group and use it to consume from Topics he's already privileged on? * Will the CLI tool be used to manage group membership too? * Groups are kind of ephemeral, right? If all consumers in the group disconnect
Re: [VOTE] KIP-11- Authorization design for kafka security
Thanks for your comments Jun. * Renamed the resource to consumer-group in wiki. * I don’t see a use case where admins/users would want to reserve topic names in advance. Can you describe why this would be needed. Thanks Parth On 4/26/15, 2:01 PM, Jun Rao j...@confluent.io wrote: A few more minor comments. 100. To make it clear, perhaps we should rename the resource group to consumer-group. We can probably make the same change in CLI as well so that it's not confused with user group. 101. Currently, create is only at the cluster level. Should it also be at topic level? For example, perhaps it's useful to allow only user X to create topic X. Thanks, Jun On Sun, Apr 26, 2015 at 12:36 AM, Gwen Shapira gshap...@cloudera.com wrote: Thanks for clarifying, Parth. I think you are taking the right approach here. On Fri, Apr 24, 2015 at 11:46 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Sorry Gwen, completely misunderstood the question :-). * Does everyone have the privilege to create a new Group and use it to consume from Topics he's already privileged on? Yes in current proposal. I did not see an API to create group but if you have a READ permission on a TOPIC and WRITE permission on that Group you are free to join and consume. * Will the CLI tool be used to manage group membership too? Yes and I think that means I need to add ―group. Updating the KIP. Thanks for pointing this out. * Groups are kind of ephemeral, right? If all consumers in the group disconnect the group is gone, AFAIK. Do we preserve the ACLs? Or do we treat the new group as completely new resource? Can we create ACLs before the group exists, in anticipation of it getting created? I have considered any auto delete and auto create as out of scope for the first release. So Right now I was going with preserving the acls. Do you see any issues with this? Auto deleting would mean authorizer will now have to get into implementation details of kafka which I was trying to avoid. Thanks Parth On 4/24/15, 11:33 AM, Gwen Shapira gshap...@cloudera.com wrote: We are not talking about same Groups :) I meant, Groups of consumers (which KIP-11 lists as a separate resource in the Privilege table) On Fri, Apr 24, 2015 at 11:31 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: I see Groups as something we can add incrementally in the current model. The acls take principalType: name so groups can be represented as group: groupName. We are not managing group memberships anywhere in kafka and I don’t see the need to do so. So for a topic1 using the CLI an admin can add an acl to grant access to group:kafka-test-users. The authorizer implementation can have a plugin to map authenticated user to groups ( This is how hadoop and storm works). The plugin could be mapping user to linux/ldap/active directory groups but that is again upto the implementation. What we are offering is an interface that is extensible so these features can be added incrementally. I can add support for this in the first release but don’t necessarily see why this would be absolute necessity. Thanks Parth On 4/24/15, 11:00 AM, Gwen Shapira gshap...@cloudera.com wrote: Thanks. One more thing I'm missing in the KIP is details on the Group resource (I think we discussed this and it was just not fully updated): * Does everyone have the privilege to create a new Group and use it to consume from Topics he's already privileged on? * Will the CLI tool be used to manage group membership too? * Groups are kind of ephemeral, right? If all consumers in the group disconnect the group is gone, AFAIK. Do we preserve the ACLs? Or do we treat the new group as completely new resource? Can we create ACLs before the group exists, in anticipation of it getting created? Its all small details, but it will be difficult to implement KIP-11 without knowing the answers :) Gwen On Fri, Apr 24, 2015 at 9:58 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: You are right, moved it to the default implementation section. Thanks Parth On 4/24/15, 9:52 AM, Gwen Shapira gshap...@cloudera.com wrote: Sample ACL JSON and Zookeeper is in public API, but I thought it is part of DefaultAuthorizer (Since Sentry and Argus won't be using Zookeeper). Am I wrong? Or is it the KIP? On Fri, Apr 24, 2015 at 9:49 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Thanks for clarifying Gwen, KIP updated. I tried to make the distinction by creating a section for all public APIs https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorizat io n+ In terface#KIP-11-AuthorizationInterface-PublicInterfacesandclasses Let me know if you think there is a better way to reflect this. Thanks Parth On 4/24/15, 9:37 AM, Gwen Shapira gshap...@cloudera.com wrote: +1 (non-binding) Two
Re: [VOTE] KIP-11- Authorization design for kafka security
Hi Sun, thanks for the comments, my answers are below: * I think the wiki already describes the precedence order as Deny taking precedence over allow when conflicting acls are found https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+In terface#KIP-11-AuthorizationInterface-PermissionType * In the first version that I am currently writing there is no group support. Even when we add it I don’t see the need to add a precedence for evaluation. it does not matter which principal matches as long as we have a match. * Acl storage is indexed by resource right now because that is the primary lookup id for all authorize operations. Given acls are cached I don’t see the need to optimized the storage layer any further for lookup. * The reason why we have acl with multi everything is to reduce redundancy in acl storage. I am not sure how will we be able to reduce redundancy if we divide it by using one principal,one host, one operation. Thanks Parth On 4/26/15, 8:06 PM, Sun, Dapeng dapeng@intel.com wrote: Hi Parth The design looks good, a few minor comments below. Since I just started looking into the discussion and many previous discussions I may missed, I'm sorry if these comments had be discussed. 1. About SimpleAclAuthorizer (SimpleAuthorizer): a. As my understanding, I think there should only one type privilege(allow/deny) of a topic on a principle, or we make it deny allow. For example, acl_1 host1 - group1- user1 - read-allow and acl_2 host1- group1 - user1 -read-deny, if the two acls are for a same topic, it may be hard to understand, do you think it's necessary to add some details about this to wiki. b. And when we do authorize a user on a topic, we may should check user's user level acl first, then check user's group level acl, finally we check the host level and default level acl. do you think it's necessary we add some contents like these to wiki. For example, host1 - group1- user1host1 - group1host1 2.About SimpleAclAuthorizer (Acl Json will be stored in zookeeper) a. It may be better to make acl json stored hierarchily. It may be easy to search and do authorize. For example, when we authorize a user, we only need user related acls. b. I found one acl may contains multi-principles, multi-operations and multi-hosts, I'm strongly agreed with we provide api like these, but the acls stored in zookeeper or memory we may better to separate to one-principle, one-operation and one host. So we could make sure there are not many acls with same meaning and make acl management easily. Regards Dapeng -Original Message- From: Jun Rao [mailto:j...@confluent.io] Sent: Monday, April 27, 2015 5:02 AM To: dev@kafka.apache.org Subject: Re: [VOTE] KIP-11- Authorization design for kafka security A few more minor comments. 100. To make it clear, perhaps we should rename the resource group to consumer-group. We can probably make the same change in CLI as well so that it's not confused with user group. 101. Currently, create is only at the cluster level. Should it also be at topic level? For example, perhaps it's useful to allow only user X to create topic X. Thanks, Jun On Sun, Apr 26, 2015 at 12:36 AM, Gwen Shapira gshap...@cloudera.com wrote: Thanks for clarifying, Parth. I think you are taking the right approach here. On Fri, Apr 24, 2015 at 11:46 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Sorry Gwen, completely misunderstood the question :-). * Does everyone have the privilege to create a new Group and use it to consume from Topics he's already privileged on? Yes in current proposal. I did not see an API to create group but if you have a READ permission on a TOPIC and WRITE permission on that Group you are free to join and consume. * Will the CLI tool be used to manage group membership too? Yes and I think that means I need to add ―group. Updating the KIP. Thanks for pointing this out. * Groups are kind of ephemeral, right? If all consumers in the group disconnect the group is gone, AFAIK. Do we preserve the ACLs? Or do we treat the new group as completely new resource? Can we create ACLs before the group exists, in anticipation of it getting created? I have considered any auto delete and auto create as out of scope for the first release. So Right now I was going with preserving the acls. Do you see any issues with this? Auto deleting would mean authorizer will now have to get into implementation details of kafka which I was trying to avoid. Thanks Parth On 4/24/15, 11:33 AM, Gwen Shapira gshap...@cloudera.com wrote: We are not talking about same Groups :) I meant, Groups of consumers (which KIP-11 lists as a separate resource in the Privilege table) On Fri, Apr 24, 2015 at 11:31 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: I see Groups as something we can add incrementally in the current model. The acls take principalType: name so
Re: [DISCUSS] Using GitHub Pull Requests for contributions and code review
+1. Thanks Parth On 5/1/15, 12:38 AM, Ewen Cheslack-Postava e...@confluent.io wrote: Also +1. There are some drawbacks to using Github for reviews, e.g. lots of emails for each review because they don't let you publish your entire review in one go like RB does, but it drastically lowers the barrier to contributing for most developers. Also, if you haven't tried it, hub https://hub.github.com/ makes it really easy to checkout and test PRs. One thing I noticed is that when you try to generate a PR it defaults to the 0.8.2 branch. Can we fix that up to be trunk by default? That's the most common use case; version branches are really only useful when a release is being prepared. Do changes to the Github repo require tickets to the Apache Infra team or is this something committers have control over? On a related note, which perhaps should be discussed on another thread: The CI setup is a related issue that we might want to rethink. Apache also supports Travis CI, and now pays for dedicated build slaves: https://blogs.apache.org/infra/entry/apache_gains_additional_travis_ci I think the setup should be pretty easy since building + running tests is just a gradle command. Having tests run automatically on PRs (and promptly!) makes it a lot easier to confidently commit a change, especially if the build merges with trunk first. I started looking at this when I was trying to sort out Confluent's build test infrastructure. See https://travis-ci.org/ewencp/kafka/builds/60802386 for an example, the patch is about 10 lines in a .travis.yml file and the failure in that example seems to be unrelated to the Travis confg. I think the basic config I created is all we'd need. Unfortunately, I couldn't easily tell what the delay on builds is, i.e. would it be an improvement over the delays with the current Jenkins setup. But having them run on PR creation/update means the results will usually be ready by the time a reviewer gets to looking at the PR and would be reported in the PR so the state is easy to evaluate. (I'm also having trouble telling exactly how the two ASF Jenkins builders differ since they both seem to poll trunk, so I can't be certain whether Travis would be able to completely replace the current setup. That said, it should be telling that I have never paid attention to Jenkins output at all since it seems so far removed from any of my actions as a contributor.) As another alternative, Confluent would also be happy to provide build/test infrastructure to help out. AMPLab does this for Spark, so it seems to be acceptable to ASF. We already have PR builders and trunk/master builders set up for other projects, so it wouldn't be hard to setup the same for Kafka with the right access to the repos. Based on the current frequency of builds (https://builds.apache.org/view/All/job/Kafka-trunk/ and https://builds.apache.org/view/All/job/KafkaPreCommit/) I think it'll be easy for even our current infrastructure to keep up. On Thu, Apr 30, 2015 at 9:41 PM, Jaikiran Pai jai.forums2...@gmail.com wrote: I think this will help a lot in contributions. Some of my local changes that I want to contribute back have been pending because I sometimes switch machines and I then have to go through setting up the Ruby/python and other stuff for the current review process. Using just github is going to help in quickly submitting the changes. -Jaikiran On Thursday 30 April 2015 06:42 PM, Ismael Juma wrote: Hi all, Kafka currently uses a combination of Review Board and JIRA for contributions and code review. In my opinion, this makes contribution and code review a bit harder than it has to be. I think the approach used by Spark would improve the current situation: Generally, Spark uses JIRA to track logical issues, including bugs and improvements, and uses Github pull requests to manage the review and merge of specific code changes. That is, JIRAs are used to describe what should be fixed or changed, and high-level approaches, and pull requests describe how to implement that change in the project's source code. For example, major design decisions are discussed in JIRA.[1] It's worth reading the wiki page for all the details, but I will summarise the suggested workflow for code changes: 1. Fork the Github repository at http://github.com/apache/kafka (if you haven't already) 2. git checkout -b kafka-XXX 3. Make one or more commits (smaller commits can be easier to review and reviewboard makes that hard) 4. git push origin kafka-XXX 5. Create PR against upstream/trunk (this will update JIRA automatically[2] and it will send an email to the dev mailing list too) 6. A CI build will be triggered[3] 7. Review process happens on GitHub (it's quite handy to be able to comment on both commit or PR-level, unlike Review Board) 8. Once all feedback has been addressed and the build is green, a variant of the `merge_spark_pr.py`[4] script is used
[GitHub] kafka pull request: KAFKA-2169: Moving to zkClient 0.5 release.
GitHub user Parth-Brahmbhatt opened a pull request: https://github.com/apache/kafka/pull/61 KAFKA-2169: Moving to zkClient 0.5 release. You can merge this pull request into a Git repository by running: $ git pull https://github.com/Parth-Brahmbhatt/kafka KAFKA-2169 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/61.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #61 commit e5eb373dcec7562292cec32f3962e42dda5cea24 Author: Parth Brahmbhatt brahmbhatt.pa...@gmail.com Date: 2015-05-07T20:15:55Z KAFKA-2169: Moving to zkClient 0.5 release. --- 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 enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
Re: Review Request 34050: Patch for KAFKA-2169
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34050/#review83286 --- core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala https://reviews.apache.org/r/34050/#comment134190 Actually it does not throw any exception now that we are just using System.exit. I have removed the @throws annotation. core/src/main/scala/kafka/controller/KafkaController.scala https://reviews.apache.org/r/34050/#comment134200 Why would we want to do this? If the listeners are invoked twice as long as both of them exit whichever one gets invoked first will just kill the process and the other one will not be invoked. Why would we care which System.exit kills the process? - Parth Brahmbhatt On May 11, 2015, 8:53 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34050/ --- (Updated May 11, 2015, 8:53 p.m.) Review request for kafka. Bugs: KAFKA-2169 https://issues.apache.org/jira/browse/KAFKA-2169 Repository: kafka Description --- System.exit instead of throwing RuntimeException when zokeeper session establishment fails. Removing the unnecessary @throws. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala aa8d9404a3e78a365df06404b79d0d8f694b4bd6 core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala 38f4ec0bd1b388cc8fc04b38bbb2e7aaa1c3f43b core/src/main/scala/kafka/controller/KafkaController.scala a6351163f5b6f080d6fa50bcc3533d445fcbc067 core/src/main/scala/kafka/server/KafkaHealthcheck.scala 861b7f644941f88ce04a4e95f6b28d18bf1db16d Diff: https://reviews.apache.org/r/34050/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 34050: Patch for KAFKA-2169
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34050/ --- (Updated May 11, 2015, 8:53 p.m.) Review request for kafka. Bugs: KAFKA-2169 https://issues.apache.org/jira/browse/KAFKA-2169 Repository: kafka Description (updated) --- System.exit instead of throwing RuntimeException when zokeeper session establishment fails. Removing the unnecessary @throws. Diffs (updated) - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala aa8d9404a3e78a365df06404b79d0d8f694b4bd6 core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala 38f4ec0bd1b388cc8fc04b38bbb2e7aaa1c3f43b core/src/main/scala/kafka/controller/KafkaController.scala a6351163f5b6f080d6fa50bcc3533d445fcbc067 core/src/main/scala/kafka/server/KafkaHealthcheck.scala 861b7f644941f88ce04a4e95f6b28d18bf1db16d Diff: https://reviews.apache.org/r/34050/diff/ Testing --- Thanks, Parth Brahmbhatt
Review Request 34047: Patch for KAFKA-2169
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34047/ --- Review request for kafka. Bugs: KAFKA-2169 https://issues.apache.org/jira/browse/KAFKA-2169 Repository: kafka Description --- KAFKA-2169: Moving to zkClient 0.5 release. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala aa8d9404a3e78a365df06404b79d0d8f694b4bd6 core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala 38f4ec0bd1b388cc8fc04b38bbb2e7aaa1c3f43b core/src/main/scala/kafka/controller/KafkaController.scala a6351163f5b6f080d6fa50bcc3533d445fcbc067 core/src/main/scala/kafka/server/KafkaHealthcheck.scala 861b7f644941f88ce04a4e95f6b28d18bf1db16d Diff: https://reviews.apache.org/r/34047/diff/ Testing --- Thanks, Parth Brahmbhatt
Review Request 34050: Patch for KAFKA-2169
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34050/ --- Review request for kafka. Bugs: KAFKA-2169 https://issues.apache.org/jira/browse/KAFKA-2169 Repository: kafka Description --- System.exit instead of throwing RuntimeException when zokeeper session establishment fails. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala aa8d9404a3e78a365df06404b79d0d8f694b4bd6 core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala 38f4ec0bd1b388cc8fc04b38bbb2e7aaa1c3f43b core/src/main/scala/kafka/controller/KafkaController.scala a6351163f5b6f080d6fa50bcc3533d445fcbc067 core/src/main/scala/kafka/server/KafkaHealthcheck.scala 861b7f644941f88ce04a4e95f6b28d18bf1db16d Diff: https://reviews.apache.org/r/34050/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 34050: Patch for KAFKA-2169
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34050/#review83427 --- core/src/main/scala/kafka/controller/KafkaController.scala https://reviews.apache.org/r/34050/#comment134408 I don't understand why this needs to be done which is why I haven't addressed it. Can you elloborate why would it matter which one of the 2 calls exits the process? - Parth Brahmbhatt On May 11, 2015, 8:53 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34050/ --- (Updated May 11, 2015, 8:53 p.m.) Review request for kafka. Bugs: KAFKA-2169 https://issues.apache.org/jira/browse/KAFKA-2169 Repository: kafka Description --- System.exit instead of throwing RuntimeException when zokeeper session establishment fails. Removing the unnecessary @throws. Diffs - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala aa8d9404a3e78a365df06404b79d0d8f694b4bd6 core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala 38f4ec0bd1b388cc8fc04b38bbb2e7aaa1c3f43b core/src/main/scala/kafka/controller/KafkaController.scala a6351163f5b6f080d6fa50bcc3533d445fcbc067 core/src/main/scala/kafka/server/KafkaHealthcheck.scala 861b7f644941f88ce04a4e95f6b28d18bf1db16d Diff: https://reviews.apache.org/r/34050/diff/ Testing --- Thanks, Parth Brahmbhatt
[Vote] KIP-11 Authorization design for kafka security
Hi, Opening the voting thread for KIP-11. Link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 Thanks Parth
Re: Review Request 34050: Patch for KAFKA-2169
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34050/ --- (Updated May 15, 2015, 5:19 p.m.) Review request for kafka. Bugs: KAFKA-2169 https://issues.apache.org/jira/browse/KAFKA-2169 Repository: kafka Description (updated) --- System.exit instead of throwing RuntimeException when zokeeper session establishment fails. Removing the unnecessary @throws. Consumer will only log when zk session can not be established. Diffs (updated) - build.gradle fef515b3b2276b1f861e7cc2e33e74c3ce5e405b core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala aa8d9404a3e78a365df06404b79d0d8f694b4bd6 core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala 38f4ec0bd1b388cc8fc04b38bbb2e7aaa1c3f43b core/src/main/scala/kafka/controller/KafkaController.scala a6351163f5b6f080d6fa50bcc3533d445fcbc067 core/src/main/scala/kafka/server/KafkaHealthcheck.scala 861b7f644941f88ce04a4e95f6b28d18bf1db16d Diff: https://reviews.apache.org/r/34050/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Currently the authorizer does not perform any dns lookups and uses the hostname it receives as part of request.session as is. So in a way we are allowing only ip addresses. The only match is equality based so no ip ranges yet but that is easy to add. However, I think it is ok to allow for both ip addresses and hostnames and we should allow both. I am not sure why would I want to secure dns lookups and the host lookups extending to dns server are only necessary when the dns cache does not have the entry or the cache entry expires. This can be controlled by setting networkaddress.cache.ttl setting in jvm. Thanks Parth On 4/14/15, 10:56 PM, Don Bosco Durai bo...@apache.org wrote: I also feel, having just IP would be more appropriate. Host lookup will unnecessary slow things down and would be insecure as you pointed out. With IP, it will be also able to setup policies (in future if needed) with ranges or netmasks and it would be more scalable. Bosco On 4/14/15, 1:40 PM, Michael Herstine mherst...@linkedin.com.INVALID wrote: Hi Parth, Sorry to chime in so late, but I’ve got a minor question on the KIP. Several methods take a parameter named “host” of type String. Is that intended to be a hostname, or an IP address? If the former, I’m curious as to how that’s found (in my experience, when accepting an incoming socket connection, you only know the IP address, and there isn’t a way to map that to a hostname without a round trip to a DNS server, which is insecure anyway). On 3/25/15, 1:07 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi all, I have modified the KIP to reflect the recent change request from the reviewers. I have been working on the code and I have the server side code for authorization ready. I am now modifying the command line utilities. I would really appreciate if some of the committers can spend sometime to review the KIP so we can make progress on this. Thanks Parth On 3/18/15, 2:20 PM, Michael Herstine mherst...@linkedin.com.INVALID wrote: Hi Parth, Thanks! A few questions: 1. Do you want to permit rules in your ACLs that DENY access as well as ALLOW? This can be handy setting up rules that have exceptions. E.g. “Allow principal P to READ resource R from all hosts” with “Deny principal P READ access to resource R from host H1” in combination would allow P to READ R from all hosts *except* H1. 2. When a topic is newly created, will there be an ACL created for it? If not, would that not deny subsequent access to it? (nit) Maybe use Principal instead of String to represent principals? On 3/9/15, 11:48 AM, Don Bosco Durai bo...@apache.org wrote: Parth Overall it is looking good. Couple of questionsŠ - Can you give an example how the policies will look like in the default implementation? - In the operations, can we support ³CONNECT² also? This can be used during Session connection - Regarding access control for ³Topic Creation², since we can¹t do it on the server side, can we de-scope it for? And plan it as a future feature request? Thanks Bosco On 3/6/15, 8:10 AM, Harsha ka...@harsha.io wrote: Hi Parth, Thanks for putting this together. Overall it looks good to me. Although AdminUtils is a concern KIP-4 can probably fix that part. Thanks, Harsha On Thu, Mar 5, 2015, at 10:39 AM, Parth Brahmbhatt wrote: Forgot to add links to wiki and jira. Link to wiki: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authoriza t i o n + Interface Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 Thanks Parth From: Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com Date: Thursday, March 5, 2015 at 10:33 AM To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: [DISCUSS] KIP-11- Authorization design for kafka security Hi, KIP-11 is open for discussion , I have updated the wiki with the design and open questions. Thanks Parth
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Hi Michael, There is code in kafka codebase that reads and interprets the topic config JSON which has acls, owner and logconfig properties. There are 3 use cases that we are supporting with current proposal: * You use out of box simpleAcl authorizer which is tied to the acl stored in topic config and the format is locked down. * You have a custom authorizer and a custom ACL store. Ranger/Argus falls under this as they have their own acl store and ui that users use to configure acls on the cluster and cluster resources like topic. It is upto the custom authorizer to leverage the kafka acl configs or completely ignore them as they have set a user expectation that only acls configured via their ui/system will be effective. * You have a custom authorizer but no custom Acl store. You are completely tied to Acl structure that we have provided in out of box implementation. Thanks Parth On 4/15/15, 10:31 AM, Michael Herstine mherst...@linkedin.com.INVALIDmailto:mherst...@linkedin.com.INVALID wrote: Hi Parth, One question that occurred to me at the end of today’s hangout: how tied are we to a particular ACL representation under your proposal? I know that TopicConfigCache will just contain JSON— if a particular site decides they want to represent their ACLs differently, and swap out the authorizer implementation, will that work? I guess what I’m asking is whether there’s any code in the Kafka codebase that will interpret that JSON, or does that logic live exclusively in the authorizer? On 4/14/15, 10:56 PM, Don Bosco Durai bo...@apache.orgmailto:bo...@apache.org wrote: I also feel, having just IP would be more appropriate. Host lookup will unnecessary slow things down and would be insecure as you pointed out. With IP, it will be also able to setup policies (in future if needed) with ranges or netmasks and it would be more scalable. Bosco On 4/14/15, 1:40 PM, Michael Herstine mherst...@linkedin.com.INVALIDmailto:mherst...@linkedin.com.INVALID wrote: Hi Parth, Sorry to chime in so late, but I’ve got a minor question on the KIP. Several methods take a parameter named “host” of type String. Is that intended to be a hostname, or an IP address? If the former, I’m curious as to how that’s found (in my experience, when accepting an incoming socket connection, you only know the IP address, and there isn’t a way to map that to a hostname without a round trip to a DNS server, which is insecure anyway). On 3/25/15, 1:07 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com wrote: Hi all, I have modified the KIP to reflect the recent change request from the reviewers. I have been working on the code and I have the server side code for authorization ready. I am now modifying the command line utilities. I would really appreciate if some of the committers can spend sometime to review the KIP so we can make progress on this. Thanks Parth On 3/18/15, 2:20 PM, Michael Herstine mherst...@linkedin.com.INVALIDmailto:mherst...@linkedin.com.INVALID wrote: Hi Parth, Thanks! A few questions: 1. Do you want to permit rules in your ACLs that DENY access as well as ALLOW? This can be handy setting up rules that have exceptions. E.g. “Allow principal P to READ resource R from all hosts” with “Deny principal P READ access to resource R from host H1” in combination would allow P to READ R from all hosts *except* H1. 2. When a topic is newly created, will there be an ACL created for it? If not, would that not deny subsequent access to it? (nit) Maybe use Principal instead of String to represent principals? On 3/9/15, 11:48 AM, Don Bosco Durai bo...@apache.orgmailto:bo...@apache.org wrote: Parth Overall it is looking good. Couple of questionsŠ - Can you give an example how the policies will look like in the default implementation? - In the operations, can we support ³CONNECT² also? This can be used during Session connection - Regarding access control for ³Topic Creation², since we can¹t do it on the server side, can we de-scope it for? And plan it as a future feature request? Thanks Bosco On 3/6/15, 8:10 AM, Harsha ka...@harsha.iomailto:ka...@harsha.io wrote: Hi Parth, Thanks for putting this together. Overall it looks good to me. Although AdminUtils is a concern KIP-4 can probably fix that part. Thanks, Harsha On Thu, Mar 5, 2015, at 10:39 AM, Parth Brahmbhatt wrote: Forgot to add links to wiki and jira. Link to wiki: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authoriza t i o n + Interface Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 Thanks Parth From: Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com Date: Thursday, March 5, 2015 at 10:33 AM To: dev@kafka.apache.orgmailto:dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: [DISCUSS] KIP-11
Re: [DISCUSS] KIP-11- Authorization design for kafka security
Kafka currently stores logConfig overrides specified during topic creation in zookeeper, its just an instance of java.util.Properties converted to json. I am proposing in addition to that we store acls and owner as well as part of same Properties map. There is some infrastructure around reading this config, converting it back to Properties map and most importantly propagating any changes efficiently which we will be able to leverage. As this infrastructure is common to the cluster the reading (not interpreting) of config happens outside of any authorization code. If the TopicConfigCache just kept the json representation and left it to authorizer to parse it, the authorizer will have to either parse the json for each request(not acceptable) or it will have to keep one more layer of parsed ACL instance cache. Assuming authorizer will keep an additional caching layer we will now have to implement some way to invalidate the cache which means the TopicConfigCache will have to be an observable which the Authorizer observes and invalidates its cache entries when topicConfigCache gets updated. Seemed like unnecessary complexity with not lot to gain so I went with TopicConfigCache interpreting the json and caching a higher level modeled object. In summary, the interpretation is done for both optimization and simplicity. If you think it is important to allow custom ACL format support we can add one more pluggable config(acl.parser) and interface(AclParser) or it could just be another method in Authorizer. One thing to note the current ACL json is versioned so it is easy to make changes to it however it won’t be possible to support custom ACL formats with the current design. Thanks Parth On 4/15/15, 4:29 PM, Michael Herstine mherst...@linkedin.com.INVALID wrote: Hi Parth, I’m a little confused: why would Kafka need to interpret the JSON? IIRC KIP-11 even says that the TopicConfigData will just store the JSON. I’m not really making a design recommendation here, just trying to understand what you’re proposing. On 4/15/15, 11:20 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi Michael, There is code in kafka codebase that reads and interprets the topic config JSON which has acls, owner and logconfig properties. There are 3 use cases that we are supporting with current proposal: * You use out of box simpleAcl authorizer which is tied to the acl stored in topic config and the format is locked down. * You have a custom authorizer and a custom ACL store. Ranger/Argus falls under this as they have their own acl store and ui that users use to configure acls on the cluster and cluster resources like topic. It is upto the custom authorizer to leverage the kafka acl configs or completely ignore them as they have set a user expectation that only acls configured via their ui/system will be effective. * You have a custom authorizer but no custom Acl store. You are completely tied to Acl structure that we have provided in out of box implementation. Thanks Parth On 4/15/15, 10:31 AM, Michael Herstine mherst...@linkedin.com.INVALIDmailto:mherst...@linkedin.com.INVALID wrote: Hi Parth, One question that occurred to me at the end of today’s hangout: how tied are we to a particular ACL representation under your proposal? I know that TopicConfigCache will just contain JSON— if a particular site decides they want to represent their ACLs differently, and swap out the authorizer implementation, will that work? I guess what I’m asking is whether there’s any code in the Kafka codebase that will interpret that JSON, or does that logic live exclusively in the authorizer? On 4/14/15, 10:56 PM, Don Bosco Durai bo...@apache.orgmailto:bo...@apache.org wrote: I also feel, having just IP would be more appropriate. Host lookup will unnecessary slow things down and would be insecure as you pointed out. With IP, it will be also able to setup policies (in future if needed) with ranges or netmasks and it would be more scalable. Bosco On 4/14/15, 1:40 PM, Michael Herstine mherst...@linkedin.com.INVALIDmailto:mherst...@linkedin.com.INVALID wrote: Hi Parth, Sorry to chime in so late, but I’ve got a minor question on the KIP. Several methods take a parameter named “host” of type String. Is that intended to be a hostname, or an IP address? If the former, I’m curious as to how that’s found (in my experience, when accepting an incoming socket connection, you only know the IP address, and there isn’t a way to map that to a hostname without a round trip to a DNS server, which is insecure anyway). On 3/25/15, 1:07 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com wrote: Hi all, I have modified the KIP to reflect the recent change request from the reviewers. I have been working on the code and I have the server side code for authorization ready. I am now modifying the command line utilities. I would really appreciate if some of the committers can spend sometime to review the KIP so
Re: [DISCUSS] KIP-11- Authorization design for kafka security
I have added the following to list of open questions based on the hangout discussion: * The owner field of a topic in current proposal is set to the user who created the topic and this user has all access to the topic. There was suggestion on making this a list of users who can share ownership. alternatively we can keep the user as a single entity but the user creating the topic will have to ensure that the topic acls are configured to allow admin access to all the other users that wants to assume co-ownership. It will be great if we can at least agree on the following things: * The newly proposed broker configs, their types and names * The Authorizer interface and the Acl structure * The command line options being added, their name and types * The new structure of topic config which is being stored in zookeeper Thanks Parth On 4/15/15, 12:53 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Yes, it can be turned off completely. We are proposing to add authorizer.class.name as a broker config. The value of this config can be null/unspecified (which is the default) in which case no authorization will be performed. It can be set to any FQCN of any class that implements Authorizer so you can plugin custom authorizers. Authentication is a different beast and you should look at other security related KIPs, here is the top level jira https://issues.apache.org/jira/browse/KAFKA-1682 and top level document https://cwiki.apache.org/confluence/display/KAFKA/Security Thanks Parth On 4/15/15, 11:56 AM, Tong Li liton...@us.ibm.com wrote: Parth, If one wants to use his or her own access control including authentication system, with this design what will be needed to be done? Can one completely turn this off so that the system behaves exactly same as it is today? Thanks. Tong Sent from my iPhone On Apr 15, 2015, at 1:51 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: Hi Michael, There is code in kafka codebase that reads and interprets the topic config JSON which has acls, owner and logconfig properties. There are 3 use cases that we are supporting with current proposal: * You use out of box simpleAcl authorizer which is tied to the acl stored in topic config and the format is locked down. * You have a custom authorizer and a custom ACL store. Ranger/Argus falls under this as they have their own acl store and ui that users use to configure acls on the cluster and cluster resources like topic. It is upto the custom authorizer to leverage the kafka acl configs or completely ignore them as they have set a user expectation that only acls configured via their ui/system will be effective. * You have a custom authorizer but no custom Acl store. You are completely tied to Acl structure that we have provided in out of box implementation. Thanks Parth On 4/15/15, 10:31 AM, Michael Herstine mherst...@linkedin.com.INVALIDmailto:mherst...@linkedin.com.INVALID wrote: Hi Parth, One question that occurred to me at the end of today’s hangout: how tied are we to a particular ACL representation under your proposal? I know that TopicConfigCache will just contain JSON— if a particular site decides they want to represent their ACLs differently, and swap out the authorizer implementation, will that work? I guess what I’m asking is whether there’s any code in the Kafka codebase that will interpret that JSON, or does that logic live exclusively in the authorizer? On 4/14/15, 10:56 PM, Don Bosco Durai bo...@apache.orgmailto:bo...@apache.org wrote: I also feel, having just IP would be more appropriate. Host lookup will unnecessary slow things down and would be insecure as you pointed out. With IP, it will be also able to setup policies (in future if needed) with ranges or netmasks and it would be more scalable. Bosco On 4/14/15, 1:40 PM, Michael Herstine mherst...@linkedin.com.INVALIDmailto:mherst...@linkedin.com.INVALID wrote: Hi Parth, Sorry to chime in so late, but I’ve got a minor question on the KIP. Several methods take a parameter named “host” of type String. Is that intended to be a hostname, or an IP address? If the former, I’m curious as to how that’s found (in my experience, when accepting an incoming socket connection, you only know the IP address, and there isn’t a way to map that to a hostname without a round trip to a DNS server, which is insecure anyway). On 3/25/15, 1:07 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.commailto:pbrahmbh...@hortonworks.com wrote: Hi all, I have modified the KIP to reflect the recent change request from the reviewers. I have been working on the code and I have the server side code for authorization ready. I am now modifying the command line utilities. I would really appreciate if some of the committers can spend sometime to review the KIP so we can make progress on this. Thanks Parth On 3/18/15, 2:20 PM, Michael Herstine mherst...@linkedin.com.INVALIDmailto:mherst
Re: [DISCUSSION] KIP-11: ACL Management
We could do this but I think its too simplistic plus now we are adding authorization related options in CLI which I thought everyone wants to avoid. When I say its too simplistic I mean there are missing options like —hosts, what happens when we start supporting group now we will probably end up adding —grant —groups”. I think we will just endup polluting kafka create CLI with all the different acl options or we will have 2 CLIs one for the basic stuff and for anything advance you will have to use a different tool. It might be better to just have a single separate ACL management CLI. Thanks Parth On 4/17/15, 10:42 AM, Gwen Shapira gshap...@cloudera.com wrote: I've probably been a DBA for too long, but I imagined something like: kafka-topic --topic t1 --grant user --action action kafka-topic --topic t1 --revoke user --action action (i.e. the commandline equivalent of grant select on table1 to gwenshap and revoke select on table2 from gwenshap) When you need gazillion of them, you generate a script with gazillion of those and execute. Maybe it just looks reasonable to me because I'm used to it, though :) Or maybe including the json parsing code in TopicCommand is not so bad? On Fri, Apr 17, 2015 at 10:30 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: * Yes, Acl pretty much captures everything. Originally I had resource as part of Acls, we can go back to that. * The describe can call getAcl and I plan to do so. addAcl is tricky because the user will have to specify the acls through command lines, which will probably be a location to some file. Basically the CLI won¹t know how to parse user input and convert it to a principal/acl that the plugin understands. We could add an API in authorizer that can take a file as input if we want ‹acl as an option during create. * Yes also getAcls(Principal principal). Thanks Parth On 4/17/15, 10:05 AM, Gwen Shapira gshap...@cloudera.com wrote: On Fri, Apr 17, 2015 at 9:31 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: I was following the storm model but I think this is a reasonable change. I recommend changing the API names to addAcls, removeAcls and getAcls. And they probably just need to get ListAcl instead of everything I specified? Looks like Acl encapsulates everything we need. Couple of points to ensure we are on same page: * With this approach the kafka command line will not provide a way to add/edit acls during topic creation, neither it will provide a way to modify the acls. It will be up to the authorizer to either define a command line utility or to allow other means to add acls(CLI/UI/REST). For the default implementation we can provide CLI. You looked into this deeper than I did - is there a reason TopicCommand can't invoke addACL and getACL? * We probably want to add ListAcl getAcls(Resource resource) so users can list all acls on a topic. Also getAcls(Principal princ)? I haven¹t looked at how consumer offsets are currently stored so I will have to take a look but I think that is implementation detail. Gwen,Jun and other interested parties, do you have time to jump on a quick hangout so we can go over some of the lower level details? Thanks Parth From: Tong Li liton...@us.ibm.commailto:liton...@us.ibm.com Reply-To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Date: Friday, April 17, 2015 at 7:34 AM To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: Re: [DISCUSSION] KIP-11: ACL Management Gwen, There is one product called ElasticSearch which has been quite successful. They recently added security, what they actually did is quite nice. They really separated Authentication and Authorization which many people get really confused about and often mix them up. I looked through what they did and quite impressed by it, I think there are many things we can borrow from. Here is a link to it. http://www.elastic.co/guide/en/shield/current/architecture.html. The product name is called shield which is implemented as an ElasticSearch plugin. The promise here is that you can have a running ElasticSearch, then you install this plugin, configure it, then your ElasticSearch service is secured. The goal should be really the same for Kafka, you have a Kafka service running, you install a new plugin (in this case security plugin), configure it, then your Kafka service is secured. I think that the key here is that we should introduce a true pluggable framework in Kafka which allows security, quota, encryption, compression, serialization/deserialization all being developed as plugins which can be all easily added and configured onto a running Kafka service, then the functions/features provided by the plugins will start working. Once we have this framework in, how a security plugin works internally becomes the really the concern of that plugin, for example, how a new user gets registered, permission
Re: [DISCUSSION] KIP-11: ACL Management
* Yes, Acl pretty much captures everything. Originally I had resource as part of Acls, we can go back to that. * The describe can call getAcl and I plan to do so. addAcl is tricky because the user will have to specify the acls through command lines, which will probably be a location to some file. Basically the CLI won¹t know how to parse user input and convert it to a principal/acl that the plugin understands. We could add an API in authorizer that can take a file as input if we want ‹acl as an option during create. * Yes also getAcls(Principal principal). Thanks Parth On 4/17/15, 10:05 AM, Gwen Shapira gshap...@cloudera.com wrote: On Fri, Apr 17, 2015 at 9:31 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: I was following the storm model but I think this is a reasonable change. I recommend changing the API names to addAcls, removeAcls and getAcls. And they probably just need to get ListAcl instead of everything I specified? Looks like Acl encapsulates everything we need. Couple of points to ensure we are on same page: * With this approach the kafka command line will not provide a way to add/edit acls during topic creation, neither it will provide a way to modify the acls. It will be up to the authorizer to either define a command line utility or to allow other means to add acls(CLI/UI/REST). For the default implementation we can provide CLI. You looked into this deeper than I did - is there a reason TopicCommand can't invoke addACL and getACL? * We probably want to add ListAcl getAcls(Resource resource) so users can list all acls on a topic. Also getAcls(Principal princ)? I haven¹t looked at how consumer offsets are currently stored so I will have to take a look but I think that is implementation detail. Gwen,Jun and other interested parties, do you have time to jump on a quick hangout so we can go over some of the lower level details? Thanks Parth From: Tong Li liton...@us.ibm.commailto:liton...@us.ibm.com Reply-To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Date: Friday, April 17, 2015 at 7:34 AM To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: Re: [DISCUSSION] KIP-11: ACL Management Gwen, There is one product called ElasticSearch which has been quite successful. They recently added security, what they actually did is quite nice. They really separated Authentication and Authorization which many people get really confused about and often mix them up. I looked through what they did and quite impressed by it, I think there are many things we can borrow from. Here is a link to it. http://www.elastic.co/guide/en/shield/current/architecture.html. The product name is called shield which is implemented as an ElasticSearch plugin. The promise here is that you can have a running ElasticSearch, then you install this plugin, configure it, then your ElasticSearch service is secured. The goal should be really the same for Kafka, you have a Kafka service running, you install a new plugin (in this case security plugin), configure it, then your Kafka service is secured. I think that the key here is that we should introduce a true pluggable framework in Kafka which allows security, quota, encryption, compression, serialization/deserialization all being developed as plugins which can be all easily added and configured onto a running Kafka service, then the functions/features provided by the plugins will start working. Once we have this framework in, how a security plugin works internally becomes the really the concern of that plugin, for example, how a new user gets registered, permission granted, revoked, all these will be the concern of that plugin, rest of the Kafka components should not really be concerned about them. This way we are really following the design principal (Separation of concerns). With all that, what I am proposing is a true pluggable framework introduction into Kafka which I have also talked about in a previous email. For security we can implement a simple file based security plugin, other plugins such as LDAP, AD for authentication can come later, plugin for authorization such as RBAC can also come later if people care so much about using them. Thanks. Tong Li OpenStack Kafka Community Development Building 501/B205 liton...@us.ibm.commailto:liton...@us.ibm.com [Inactive hide details for Gwen Shapira ---04/16/2015 12:44:54 PM---Hi Kafka Authorization Fans, I'm starting a new thread on a]Gwen Shapira ---04/16/2015 12:44:54 PM---Hi Kafka Authorization Fans, I'm starting a new thread on a specific sub-topic of KIP-11, since From: Gwen Shapira gshap...@cloudera.commailto:gshap...@cloudera.com To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Date: 04/16/2015 12:44 PM Subject: [DISCUSSION] KIP-11: ACL Management Hi Kafka Authorization Fans
Re: [DISCUSSION] KIP-11: ACL Management
I have copied Thejas from hive team in cc list. Here is what I learnt from him * Hive calls the authorizer plugin if you execute “grant/revoke Operation to User on Table. They use this as hive provides the SQL layer and SQL has standards for grant/revoke which they follow. * If the plugin provides more entities then what can be expressed by the above statement (like unix/ldap groups or host level control) you have to go to the plugin’s CLI/UI to create this acl. So as mentioned below you will have 2 tools. One for the very basic grant/revoke access and for anything complex you have a secondary interface provided by Authorizer plugin. Thanks Parth On 4/17/15, 12:01 PM, Jun Rao j...@confluent.io wrote: Hi, Parth, How does this work in Hive? I thought authorization in Hive always goes through it's SQL cli for any authorization plugin. When integrating with Ranger(Argus), does Hive do authorization through a separate CLI? Thanks, Jun On Fri, Apr 17, 2015 at 11:01 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: We could do this but I think its too simplistic plus now we are adding authorization related options in CLI which I thought everyone wants to avoid. When I say its too simplistic I mean there are missing options like —hosts, what happens when we start supporting group now we will probably end up adding —grant —groups”. I think we will just endup polluting kafka create CLI with all the different acl options or we will have 2 CLIs one for the basic stuff and for anything advance you will have to use a different tool. It might be better to just have a single separate ACL management CLI. Thanks Parth On 4/17/15, 10:42 AM, Gwen Shapira gshap...@cloudera.com wrote: I've probably been a DBA for too long, but I imagined something like: kafka-topic --topic t1 --grant user --action action kafka-topic --topic t1 --revoke user --action action (i.e. the commandline equivalent of grant select on table1 to gwenshap and revoke select on table2 from gwenshap) When you need gazillion of them, you generate a script with gazillion of those and execute. Maybe it just looks reasonable to me because I'm used to it, though :) Or maybe including the json parsing code in TopicCommand is not so bad? On Fri, Apr 17, 2015 at 10:30 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: * Yes, Acl pretty much captures everything. Originally I had resource as part of Acls, we can go back to that. * The describe can call getAcl and I plan to do so. addAcl is tricky because the user will have to specify the acls through command lines, which will probably be a location to some file. Basically the CLI won¹t know how to parse user input and convert it to a principal/acl that the plugin understands. We could add an API in authorizer that can take a file as input if we want ‹acl as an option during create. * Yes also getAcls(Principal principal). Thanks Parth On 4/17/15, 10:05 AM, Gwen Shapira gshap...@cloudera.com wrote: On Fri, Apr 17, 2015 at 9:31 AM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: I was following the storm model but I think this is a reasonable change. I recommend changing the API names to addAcls, removeAcls and getAcls. And they probably just need to get ListAcl instead of everything I specified? Looks like Acl encapsulates everything we need. Couple of points to ensure we are on same page: * With this approach the kafka command line will not provide a way to add/edit acls during topic creation, neither it will provide a way to modify the acls. It will be up to the authorizer to either define a command line utility or to allow other means to add acls(CLI/UI/REST). For the default implementation we can provide CLI. You looked into this deeper than I did - is there a reason TopicCommand can't invoke addACL and getACL? * We probably want to add ListAcl getAcls(Resource resource) so users can list all acls on a topic. Also getAcls(Principal princ)? I haven¹t looked at how consumer offsets are currently stored so I will have to take a look but I think that is implementation detail. Gwen,Jun and other interested parties, do you have time to jump on a quick hangout so we can go over some of the lower level details? Thanks Parth From: Tong Li liton...@us.ibm.commailto:liton...@us.ibm.com Reply-To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Date: Friday, April 17, 2015 at 7:34 AM To: dev@kafka.apache.orgmailto:dev@kafka.apache.org dev@kafka.apache.orgmailto:dev@kafka.apache.org Subject: Re: [DISCUSSION] KIP-11: ACL Management Gwen, There is one product called ElasticSearch which has been quite successful. They recently added security, what they actually did is quite nice. They really separated Authentication and Authorization which many people get really confused about and often mix them up. I
Re: [DISCUSSION] KIP-11: ACL Management
I looked into the consumer offset storage and it seems like for acl storage we should not need something as complex. Consumer offset has different throughput requirements which is why I think it made sense to move away from zookeeper. Acls on the other hand seldom change and because of the caching layer will require very low read rate. Zookeeper seems like a perfect storage solution for small metadata like this. As far as mirror maker goes, given we are adding the storage APIs for acls (add/remove/get) it should be easy for mirror maker to use these APIs to read acls and add it back in the new cluster. Thanks Parth On 4/16/15, 6:13 PM, Jun Rao j...@confluent.io wrote: Hi, Gwen, What you suggested seems reasonable. I guess we will need the Principal, Privilege pair and the Resource in grant() and revoke()? Is the Hive authorization api the following? It's weird that it takes user in checkPermissions(), but not in authorize(). http://hive.apache.org/javadocs/r0.11.0/api/org/apache/hadoop/hive/ql/secu rity/authorization/StorageBasedAuthorizationProvider.html I was imagining that a default implementation could be similar to how we store offsets in Kafka. Basically, store all acls in a special topic with compact retention. Then, every broker will build an in-memory cache off that topic. Another thing that we haven't discussed so far is how to manage ACLs across different mirrored Kafka clusters. Let's say you use mirror maker to mirror all topics from cluster A to cluster B. You probably want to have exactly the same ACL on both A and B. It would be good if the ACL can be set up just once. If we use the above default implementation, since the ACL topic is mirrored too, the ACL will be propagated automatically. Thanks, Jun On Thu, Apr 16, 2015 at 9:44 AM, Gwen Shapira gshap...@cloudera.com wrote: Hi Kafka Authorization Fans, I'm starting a new thread on a specific sub-topic of KIP-11, since this is a bit long :) Currently KIP-11, as I understand it, proposes: * Authorizers are pluggable, with Kafka providing DefaultAuthorizer. * Kafka tools allow adding / managing ACLs. * Those ACLs are stored in ZK and cached in a new TopicCache * Authorizers can either use the ACLs defined and stored in Kafka, or define and use their own. I am concerned of two possible issues with this design: 1. Separation of concerns - only authorizers should worry about ACLs, and therefore the less code for ACLs that exist in Kafka core, the better. 2. User confusion - It sounded like we can define ACLs in Kafka itself but authorizers can also define their own, so kafka-topics --describe may show an ACL different than the one in use. This can be super confusing for admins. My alternative suggestion: * Authorizer API will include: grantPrivilege(ListPrincipals, ListPrivilege) revokePrivilege(ListPrincipals, ListPrivilege), getPrivilegesByPrincipal(Principal, Resource) (The exact API can be discussed in detail, but you get the idea) * Kafka tools will simply invoke these APIs when topics are added / modified / described. * Each authorizer (including the default one) will be responsible for storing, caching and using those ACLs. This way, we keep almost all ACL code with the Authorizer, where it belongs and users get a nice unified interface that reflects what is actually getting used in the system. This is pretty much how Sqoop and Hive implement their authorization APIs. What do you think? Gwen
Review Request 32942: Patch for KAFKA-1688
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/32942/ --- Review request for kafka. Bugs: KAFKA-1688 https://issues.apache.org/jira/browse/KAFKA-1688 Repository: kafka Description --- This is still a draft version and I am working on making the code more scala less java. There are a few TODOs some of which are dependent on other jiras some are notes to my self. I am posting the review to get early feedback on clear red-flags. KAFKA-1688: Add authorization. Diffs - core/src/main/scala/kafka/admin/AdminUtils.scala eee80f9c2c12da8e4879e96785f3b75a8ff7d1cd core/src/main/scala/kafka/admin/TopicCommand.scala 60f0228e6735a492315842afe28bcfa317478968 core/src/main/scala/kafka/api/TopicMetadata.scala 5e39f453b429cb42ab87f695b6b30576f9ce98ce core/src/main/scala/kafka/common/AuthorizationException.scala PRE-CREATION core/src/main/scala/kafka/common/ErrorMapping.scala eb1eb4a703098253d0aae79577084569177768d1 core/src/main/scala/kafka/log/LogConfig.scala 558c703f26da22b1a938bbbf8a6c4409a8e107fb core/src/main/scala/kafka/network/RequestChannel.scala 1d9c57b0b5a0ad31e4f3d7562f0266af83cc9024 core/src/main/scala/kafka/security/auth/Acl.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Authorizer.scala PRE-CREATION core/src/main/scala/kafka/security/auth/ClusterAclCache.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Operation.java PRE-CREATION core/src/main/scala/kafka/security/auth/PermissionType.java PRE-CREATION core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala PRE-CREATION core/src/main/scala/kafka/server/KafkaApis.scala b4004aa3a1456d337199aa1245fb0ae61f6add46 core/src/main/scala/kafka/server/KafkaConfig.scala 69b772c1941865fbe15b34bb2784c511f8ce519a core/src/main/scala/kafka/server/KafkaServer.scala c63f4ba9d622817ea8636d4e6135fba917ce085a core/src/main/scala/kafka/server/MetadataCache.scala 9a9205f78ff208a8103650dc8ea958ce59b7b507 core/src/main/scala/kafka/server/TopicConfig.scala PRE-CREATION core/src/main/scala/kafka/server/TopicConfigCache.scala PRE-CREATION core/src/main/scala/kafka/server/TopicConfigManager.scala 47295d40131492aaac786273819b7bc6e22e5486 core/src/main/scala/kafka/utils/Json.scala d1102844748f2e88f79932281fe95583a57d2d16 core/src/test/resources/acl.json PRE-CREATION core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala ab5d16c1d22191708410b15332882ba40eda5f28 core/src/test/scala/unit/kafka/admin/AdminTest.scala cfe38df577e3f179ebecad3f45429a15aa69e7b4 core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala 61cc6028dd7c9a2eec2d9cbe6947764655801eee core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala c7136f20972614ac47aa57ab13e3c94ef775a4b7 core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala a1300894258c0ee77dffc96df24a2f7369eb68da core/src/test/scala/unit/kafka/security/auth/AclTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/ClusterAclTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala PRE-CREATION core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala 7877f6ca1845c2edbf96d4a9783a07a552db8f07 core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala 62d183248e3be4c83d2c768e762f61f92448c6a6 core/src/test/scala/unit/kafka/server/TopicConfigCacheTest.scala PRE-CREATION core/src/test/scala/unit/kafka/utils/TestUtils.scala 5a9e84d44f6567c3a01a4e068c751edb07ee9634 Diff: https://reviews.apache.org/r/32942/diff/ Testing --- Unit tests added for the SimpleACLAuthorizer, ACl and ClusterAclCache. Modified existing unit tests for TopicCommands and TopicConfigCache. Thanks, Parth Brahmbhatt
Review Request 34494: Patch for KAFKA-2212
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34494/ --- Review request for kafka. Bugs: KAFKA-2212 https://issues.apache.org/jira/browse/KAFKA-2212 Repository: kafka Description --- KAFKA-2212: Add CLI for acl management of authorizer. Diffs - core/src/main/scala/kafka/admin/AclCommand.scala PRE-CREATION core/src/test/scala/unit/kafka/admin/AclCommandTest.scala PRE-CREATION Diff: https://reviews.apache.org/r/34494/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: [Vote] KIP-11 Authorization design for kafka security
This vote is now Closed with 4 binding +1s and 4 non binding +1s. Thanks Parth On 5/20/15, 12:04 PM, Joel Koshy jjkosh...@gmail.com wrote: +1 On Fri, May 15, 2015 at 04:18:49PM +, Parth Brahmbhatt wrote: Hi, Opening the voting thread for KIP-11. Link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+ Interface Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 Thanks Parth
Review Request 34492: Patch for KAFKA-2210
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- Review request for kafka. Bugs: KAFKA-2210 https://issues.apache.org/jira/browse/KAFKA-2210 Repository: kafka Description --- KAFKA-2210: Kafka authorizer public entities and changes to KafkaAPI and KafkaServer to allow custom authorizer implementation. Diffs - core/src/main/scala/kafka/common/AuthorizationException.scala PRE-CREATION core/src/main/scala/kafka/common/ErrorMapping.scala c75c68589681b2c9d6eba2b440ce5e58cddf6370 core/src/main/scala/kafka/security/auth/Acl.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Authorizer.scala PRE-CREATION core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Operation.java PRE-CREATION core/src/main/scala/kafka/security/auth/PermissionType.java PRE-CREATION core/src/main/scala/kafka/security/auth/Resource.scala PRE-CREATION core/src/main/scala/kafka/security/auth/ResourceType.java PRE-CREATION core/src/main/scala/kafka/server/KafkaApis.scala 387e387998fc3a6c9cb585dab02b5f77b0381fbf core/src/main/scala/kafka/server/KafkaConfig.scala 9efa15ca5567b295ab412ee9eea7c03eb4cdc18b core/src/main/scala/kafka/server/KafkaServer.scala ea6d165d8e5c3146d2c65e8ad1a513308334bf6f core/src/test/resources/acl.json PRE-CREATION core/src/test/scala/unit/kafka/security/auth/AclTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/KafkaPrincipalTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/ResourceTest.scala PRE-CREATION core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala 8014a5a6c362785539f24eb03d77278434614fe6 Diff: https://reviews.apache.org/r/34492/diff/ Testing --- Thanks, Parth Brahmbhatt
Review Request 34493: Patch for KAFKA-2211
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34493/ --- Review request for kafka. Bugs: KAFKA-2211 https://issues.apache.org/jira/browse/KAFKA-2211 Repository: kafka Description --- KAFKA-2211: Out of box implementation for authorizer interface. Diffs - core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala PRE-CREATION core/src/test/resources/authorizer-config.properties PRE-CREATION core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala PRE-CREATION Diff: https://reviews.apache.org/r/34493/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 34492: Patch for KAFKA-2210
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated June 4, 2015, 11:07 p.m.) Review request for kafka. Bugs: KAFKA-2210 https://issues.apache.org/jira/browse/KAFKA-2210 Repository: kafka Description (updated) --- Addressing review comments from Jun. Adding CREATE check for offset topic only if the topic does not exist already. Diffs (updated) - core/src/main/scala/kafka/api/OffsetRequest.scala 3d483bc7518ad76f9548772522751afb4d046b78 core/src/main/scala/kafka/common/AuthorizationException.scala PRE-CREATION core/src/main/scala/kafka/common/ErrorMapping.scala c75c68589681b2c9d6eba2b440ce5e58cddf6370 core/src/main/scala/kafka/security/auth/Acl.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Authorizer.scala PRE-CREATION core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Operation.java PRE-CREATION core/src/main/scala/kafka/security/auth/PermissionType.java PRE-CREATION core/src/main/scala/kafka/security/auth/Resource.scala PRE-CREATION core/src/main/scala/kafka/security/auth/ResourceType.java PRE-CREATION core/src/main/scala/kafka/server/KafkaApis.scala 387e387998fc3a6c9cb585dab02b5f77b0381fbf core/src/main/scala/kafka/server/KafkaConfig.scala 6f25afd0e5df98258640252661dee271b1795111 core/src/main/scala/kafka/server/KafkaServer.scala e66710d2368334ece66f70d55f57b3f888262620 core/src/test/resources/acl.json PRE-CREATION core/src/test/scala/unit/kafka/security/auth/AclTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/KafkaPrincipalTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/ResourceTest.scala PRE-CREATION core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala 71f48c07723e334e6489efab500a43fa93a52d0c Diff: https://reviews.apache.org/r/34492/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 34492: Patch for KAFKA-2210
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated June 3, 2015, 11:36 p.m.) Review request for kafka. Bugs: KAFKA-2210 https://issues.apache.org/jira/browse/KAFKA-2210 Repository: kafka Description --- Addressing review comments from Jun. Diffs (updated) - core/src/main/scala/kafka/api/OffsetRequest.scala 3d483bc7518ad76f9548772522751afb4d046b78 core/src/main/scala/kafka/common/AuthorizationException.scala PRE-CREATION core/src/main/scala/kafka/common/ErrorMapping.scala c75c68589681b2c9d6eba2b440ce5e58cddf6370 core/src/main/scala/kafka/security/auth/Acl.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Authorizer.scala PRE-CREATION core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Operation.java PRE-CREATION core/src/main/scala/kafka/security/auth/PermissionType.java PRE-CREATION core/src/main/scala/kafka/security/auth/Resource.scala PRE-CREATION core/src/main/scala/kafka/security/auth/ResourceType.java PRE-CREATION core/src/main/scala/kafka/server/KafkaApis.scala 387e387998fc3a6c9cb585dab02b5f77b0381fbf core/src/main/scala/kafka/server/KafkaConfig.scala 6f25afd0e5df98258640252661dee271b1795111 core/src/main/scala/kafka/server/KafkaServer.scala e66710d2368334ece66f70d55f57b3f888262620 core/src/test/resources/acl.json PRE-CREATION core/src/test/scala/unit/kafka/security/auth/AclTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/KafkaPrincipalTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/ResourceTest.scala PRE-CREATION core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala 71f48c07723e334e6489efab500a43fa93a52d0c Diff: https://reviews.apache.org/r/34492/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 34492: Patch for KAFKA-2210
/server/KafkaApis.scala https://reviews.apache.org/r/34492/#comment138257 Fixed. core/src/main/scala/kafka/server/KafkaApis.scala https://reviews.apache.org/r/34492/#comment138259 Fixed. core/src/main/scala/kafka/server/KafkaApis.scala https://reviews.apache.org/r/34492/#comment138260 Yes, Fixed. core/src/main/scala/kafka/server/KafkaApis.scala https://reviews.apache.org/r/34492/#comment138277 This TODO is misplaced , I have added the create check in handleTopicMetadataRequest. The create here only creates OffsetManager.OffsetsTopicName aka __consumer_offsets. I am assuming __consumer_offsets should be created by brokers when they issue updateMetaDataRequest so adding a create check here as well which means any consumer that tries to consumer before __consumer_offsets is created will get authorization exception unless an explicit CREATE is granted to them on __consumer_offsets topic. we should probably disucss if its ok to allow anyone to create this special topic in absence of which consumers can fail. core/src/main/scala/kafka/server/KafkaConfig.scala https://reviews.apache.org/r/34492/#comment138278 Not suppose to be part of this PR. Removed. core/src/test/scala/unit/kafka/security/auth/AclTest.scala https://reviews.apache.org/r/34492/#comment138526 can you elloborate why do you think that is a better approach? core/src/test/scala/unit/kafka/security/auth/AclTest.scala https://reviews.apache.org/r/34492/#comment138528 Yes, lets decide if we want to get rid of case insensitive checks for hosts and I can remove this test by making acl a case class. core/src/test/scala/unit/kafka/security/auth/ResourceTest.scala https://reviews.apache.org/r/34492/#comment138523 Same rationale as mentioned few times before for case senstivity. - Parth Brahmbhatt On June 3, 2015, 11:36 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated June 3, 2015, 11:36 p.m.) Review request for kafka. Bugs: KAFKA-2210 https://issues.apache.org/jira/browse/KAFKA-2210 Repository: kafka Description --- Addressing review comments from Jun. Diffs - core/src/main/scala/kafka/api/OffsetRequest.scala 3d483bc7518ad76f9548772522751afb4d046b78 core/src/main/scala/kafka/common/AuthorizationException.scala PRE-CREATION core/src/main/scala/kafka/common/ErrorMapping.scala c75c68589681b2c9d6eba2b440ce5e58cddf6370 core/src/main/scala/kafka/security/auth/Acl.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Authorizer.scala PRE-CREATION core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Operation.java PRE-CREATION core/src/main/scala/kafka/security/auth/PermissionType.java PRE-CREATION core/src/main/scala/kafka/security/auth/Resource.scala PRE-CREATION core/src/main/scala/kafka/security/auth/ResourceType.java PRE-CREATION core/src/main/scala/kafka/server/KafkaApis.scala 387e387998fc3a6c9cb585dab02b5f77b0381fbf core/src/main/scala/kafka/server/KafkaConfig.scala 6f25afd0e5df98258640252661dee271b1795111 core/src/main/scala/kafka/server/KafkaServer.scala e66710d2368334ece66f70d55f57b3f888262620 core/src/test/resources/acl.json PRE-CREATION core/src/test/scala/unit/kafka/security/auth/AclTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/KafkaPrincipalTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/ResourceTest.scala PRE-CREATION core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala 71f48c07723e334e6489efab500a43fa93a52d0c Diff: https://reviews.apache.org/r/34492/diff/ Testing --- Thanks, Parth Brahmbhatt
[Bump] Code review for KIP-11
Hi, Can someone please review the following CRs: Public entities and interfaces with changes to KafkaAPI and KafkaServer: https://reviews.apache.org/r/34492/diff/ Actual Implementation: https://reviews.apache.org/r/34493/diff/ CLI: https://reviews.apache.org/r/34494/diff/ Thanks Parth
Re: [Vote] KIP-11 Authorization design for kafka security
I am sorry to be ignorant about this but what is the new state? Adopted seems too early given we are still in code review process. Should I just make it ³Code review²? Thanks Parth On 5/21/15, 8:43 AM, Jun Rao j...@confluent.io wrote: Parth, Thanks for driving this. Could you update the status of the KIP in the wiki? Thanks, Jun On Wed, May 20, 2015 at 2:37 PM, Parth Brahmbhatt pbrahmbh...@hortonworks.com wrote: This vote is now Closed with 4 binding +1s and 4 non binding +1s. Thanks Parth On 5/20/15, 12:04 PM, Joel Koshy jjkosh...@gmail.com wrote: +1 On Fri, May 15, 2015 at 04:18:49PM +, Parth Brahmbhatt wrote: Hi, Opening the voting thread for KIP-11. Link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+ Interface Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 Thanks Parth
Re: Review Request 34493: Patch for KAFKA-2211
On Aug. 21, 2015, 2:31 p.m., Ismael Juma wrote: Thanks for this Parth. I did an initial pass where I left a number comments (many of them style-related, see http://kafka.apache.org/coding-guide.html for reference). I know, we should have a tool that checks some of these things automatically. That is my main priority after we get the security stuff in shape. I think it would be useful if you had a look and made the changes (if you agree) to the cases I pointed out and similar ones. I noticed that KAFKA-2212 has some similar issues too, it may be worth taking a pass there too. I will look at these two patches again early next week. Hey , thanks for reviewing this. This patch needs to be updated with all the changes that we have made in 2210. As 2210 was moving slow and was kind of a moving target I did not update this patch. I think I have gained little more understanding around scala styles from the 2210 review so I will incorporate all those in this patch. - Parth --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34493/#review96042 --- On May 20, 2015, 8:03 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34493/ --- (Updated May 20, 2015, 8:03 p.m.) Review request for kafka. Bugs: KAFKA-2211 https://issues.apache.org/jira/browse/KAFKA-2211 Repository: kafka Description --- KAFKA-2211: Out of box implementation for authorizer interface. Diffs - core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala PRE-CREATION core/src/test/resources/authorizer-config.properties PRE-CREATION core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala PRE-CREATION Diff: https://reviews.apache.org/r/34493/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 34492: Patch for KAFKA-2210
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated Aug. 20, 2015, 6:27 p.m.) Review request for kafka. Bugs: KAFKA-2210 https://issues.apache.org/jira/browse/KAFKA-2210 Repository: kafka Description (updated) --- Addressing review comments from Jun. Adding CREATE check for offset topic only if the topic does not exist already. Addressing some more comments. Removing acl.json file Moving PermissionType to trait instead of enum. Following the convention for defining constants. Adding authorizer.config.path back. Addressing more comments from Jun. Addressing more comments. Now addressing Ismael's comments. Case sensitive checks. Addressing Jun's comments. Merge remote-tracking branch 'origin/trunk' into az Conflicts: core/src/main/scala/kafka/server/KafkaApis.scala core/src/main/scala/kafka/server/KafkaServer.scala Deleting KafkaConfigDefTest Addressing comments from Ismael. Diffs (updated) - core/src/main/scala/kafka/api/OffsetRequest.scala f418868046f7c99aefdccd9956541a0cb72b1500 core/src/main/scala/kafka/common/AuthorizationException.scala PRE-CREATION core/src/main/scala/kafka/common/ErrorMapping.scala c75c68589681b2c9d6eba2b440ce5e58cddf6370 core/src/main/scala/kafka/security/auth/Acl.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Authorizer.scala PRE-CREATION core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Operation.scala PRE-CREATION core/src/main/scala/kafka/security/auth/PermissionType.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Resource.scala PRE-CREATION core/src/main/scala/kafka/security/auth/ResourceType.scala PRE-CREATION core/src/main/scala/kafka/server/KafkaApis.scala 67f0cad802f901f255825aa2158545d7f5e7cc3d core/src/main/scala/kafka/server/KafkaConfig.scala d547a01cf7098f216a3775e1e1901c5794e1b24c core/src/main/scala/kafka/server/KafkaServer.scala 0e7ba3ede78dbc995c404e0387a6be687703836a core/src/test/scala/unit/kafka/security/auth/AclTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/KafkaPrincipalTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/OperationTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/PermissionTypeTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/ResourceTypeTest.scala PRE-CREATION core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala 3da666f73227fc7ef7093e3790546344065f6825 Diff: https://reviews.apache.org/r/34492/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 34492: Patch for KAFKA-2210
On Aug. 20, 2015, 10:29 a.m., Ismael Juma wrote: core/src/main/scala/kafka/network/RequestChannel.scala, line 48 https://reviews.apache.org/r/34492/diff/10/?file=1037026#file1037026line48 Normally one would use `Option[Session]` here. Are we using `null` due to efficiency concerns? Sorry if I am missing some context. My bad, This class is not suppose to be part of this PR but the dependent jira. Removed this class. On Aug. 20, 2015, 10:29 a.m., Ismael Juma wrote: core/src/main/scala/kafka/security/auth/Operation.scala, line 42 https://reviews.apache.org/r/34492/diff/10/?file=1037030#file1037030line42 Generally a good idea to set the result type for public methods. This makes it possible to change the underlying implementation without affecting binary compatibility. For example, here we may set the result type as `Seq[Operation]`, which would give us the option of changing the underlying implementation to `Vector` if that turned out to be better. In `Scala`, `List` is a concrete type unlike `Java`. Not sure what is the usual policy for Kafka though, would be useful to have some input from Jun. If we decide to change it, there are other places where the same comment would apply. Jun Rao wrote: We don't have a policy on that yet. I think explicitly defining return types in this case makes sense. Fixed in Operation, PermissionType and ResourceType. On Aug. 20, 2015, 10:29 a.m., Ismael Juma wrote: core/src/main/scala/kafka/security/auth/Resource.scala, line 24 https://reviews.apache.org/r/34492/diff/10/?file=1037032#file1037032line24 Nitpick: space before `:` should be removed. Fixed. On Aug. 20, 2015, 10:29 a.m., Ismael Juma wrote: core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala, line 25 https://reviews.apache.org/r/34492/diff/10/?file=1037029#file1037029line25 Nitpick: space before `:` should be removed. Fixed. On Aug. 20, 2015, 10:29 a.m., Ismael Juma wrote: core/src/main/scala/kafka/security/auth/Acl.scala, line 116 https://reviews.apache.org/r/34492/diff/10/?file=1037027#file1037027line116 Nitpick: no need for `()` and space before `:` should be removed. Fixed. On Aug. 20, 2015, 10:29 a.m., Ismael Juma wrote: core/src/main/scala/kafka/server/KafkaApis.scala, line 104 https://reviews.apache.org/r/34492/diff/10/?file=1037034#file1037034line104 This code exists in 4 places, how about we introduce an method like: ``` def authorizeClusterAction(authorizer, request): Unit = { if (authorizer.map(_.authorizer(request.session, ClusterAction, Resource.ClusterResource)).getOrElse(false)) throw new AuthorizationException(sRequest $request is not authorized.) } ``` And then callers can just do (as an example): `authorizeClusterAction(authorizer, leaderAndIsrRequest)` Am I missing something? Fixed. On Aug. 20, 2015, 10:29 a.m., Ismael Juma wrote: core/src/main/scala/kafka/server/KafkaApis.scala, lines 189-192 https://reviews.apache.org/r/34492/diff/10/?file=1037034#file1037034line189 Nitpick: space after `case`. There are a number of other cases like this. Fixed. On Aug. 20, 2015, 10:29 a.m., Ismael Juma wrote: core/src/main/scala/kafka/server/KafkaApis.scala, line 549 https://reviews.apache.org/r/34492/diff/10/?file=1037034#file1037034line549 Nitpick: val instead of var. I am actually changing these values later so they need to be vars. On Aug. 20, 2015, 10:29 a.m., Ismael Juma wrote: core/src/test/scala/unit/kafka/security/auth/AclTest.scala, line 24 https://reviews.apache.org/r/34492/diff/10/?file=1037037#file1037037line24 We don't use `JUnit3Suite` anymore. Either use `JUnitSuite` or don't inherit from anything (we have both examples in the codebase now). This applies to all the tests. I noticed that Jun already mentioned this in another test. Fixed. On Aug. 20, 2015, 10:29 a.m., Ismael Juma wrote: core/src/test/scala/unit/kafka/security/auth/AclTest.scala, line 30 https://reviews.apache.org/r/34492/diff/10/?file=1037037#file1037037line30 @Test annotation is needed after you remove `JUnit3Suite`. This applies to all the tests. Fixed. - Parth --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review95934 --- On Aug. 20, 2015, 6:27 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated Aug. 20, 2015, 6:27 p.m.) Review request for kafka. Bugs: KAFKA-2210 https://issues.apache.org/jira/browse/KAFKA-2210
Re: Review Request 34492: Patch for KAFKA-2210
On Aug. 20, 2015, 1:07 a.m., Jun Rao wrote: core/src/main/scala/kafka/security/auth/Acl.scala, lines 84-87 https://reviews.apache.org/r/34492/diff/10/?file=1037027#file1037027line84 Do we need the case match here since acls is always a Set? Fixed. On Aug. 20, 2015, 1:07 a.m., Jun Rao wrote: core/src/main/scala/kafka/security/auth/ResourceType.scala, line 21 https://reviews.apache.org/r/34492/diff/10/?file=1037033#file1037033line21 Not needed. Fixed. On Aug. 20, 2015, 1:07 a.m., Jun Rao wrote: core/src/main/scala/kafka/server/KafkaApis.scala, line 104 https://reviews.apache.org/r/34492/diff/10/?file=1037034#file1037034line104 If authorizer is not specified, getOrElse() should return true, right? There are a few other places like that. Fixed. On Aug. 20, 2015, 1:07 a.m., Jun Rao wrote: core/src/main/scala/kafka/server/KafkaApis.scala, lines 189-192 https://reviews.apache.org/r/34492/diff/10/?file=1037034#file1037034line189 For coding style, to be consistent with most existing code, it seems it's better to do authorizer.map{ ... }.getOrElse(). Done. On Aug. 20, 2015, 1:07 a.m., Jun Rao wrote: core/src/main/scala/kafka/server/KafkaApis.scala, lines 641-644 https://reviews.apache.org/r/34492/diff/10/?file=1037034#file1037034line641 The current logic requires that we grant CREATE on CLUSTER to the consumer, which is a bit weird. Perhaps we should just always allow the consumer to create the offset topic as long as it has the permission to read the topic and the consumer group. That way, we don't have to grant CREATE permssion to the consumer. Done. On Aug. 20, 2015, 1:07 a.m., Jun Rao wrote: core/src/main/scala/kafka/server/KafkaApis.scala, line 677 https://reviews.apache.org/r/34492/diff/10/?file=1037034#file1037034line677 It seems the test on errorCode == ErrorMapping.NoError is unnecessary. The code tries to priortize non authoirzation errors above authorization error. We can only send one error code and IMO if we have an error other than Authorization error we should propogate that to the user. On Aug. 20, 2015, 1:07 a.m., Jun Rao wrote: core/src/main/scala/kafka/server/KafkaApis.scala, line 553 https://reviews.apache.org/r/34492/diff/10/?file=1037034#file1037034line553 Currently, metadataCache.getTopicMetadata() will return all the metadata of all topics if the input topic is empty. This causes a couple of issues here. (1) If authorizedTopics is empty, we end up returning more topics than needed. (2) If the original request has an empty topic list, we will return the metadata of all topics whether the client has the DESCRIBE permission or not. Fixed. On Aug. 20, 2015, 1:07 a.m., Jun Rao wrote: core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala, line 30 https://reviews.apache.org/r/34492/diff/10/?file=1037042#file1037042line30 We removed this class in KAFKA-2288 since it's no longer necessary. Removed the class. - Parth --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review95801 --- On Aug. 20, 2015, 6:27 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated Aug. 20, 2015, 6:27 p.m.) Review request for kafka. Bugs: KAFKA-2210 https://issues.apache.org/jira/browse/KAFKA-2210 Repository: kafka Description --- Addressing review comments from Jun. Adding CREATE check for offset topic only if the topic does not exist already. Addressing some more comments. Removing acl.json file Moving PermissionType to trait instead of enum. Following the convention for defining constants. Adding authorizer.config.path back. Addressing more comments from Jun. Addressing more comments. Now addressing Ismael's comments. Case sensitive checks. Addressing Jun's comments. Merge remote-tracking branch 'origin/trunk' into az Conflicts: core/src/main/scala/kafka/server/KafkaApis.scala core/src/main/scala/kafka/server/KafkaServer.scala Deleting KafkaConfigDefTest Addressing comments from Ismael. Diffs - core/src/main/scala/kafka/api/OffsetRequest.scala f418868046f7c99aefdccd9956541a0cb72b1500 core/src/main/scala/kafka/common/AuthorizationException.scala PRE-CREATION core/src/main/scala/kafka/common/ErrorMapping.scala c75c68589681b2c9d6eba2b440ce5e58cddf6370 core/src/main/scala/kafka/security/auth/Acl.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Authorizer.scala PRE-CREATION core/src/main/scala/kafka/security/auth
Re: Review Request 34492: Patch for KAFKA-2210
On Aug. 20, 2015, 10:30 a.m., Ismael Juma wrote: One more thing: it may be a good idea to rebase against trunk since the large SSL/TLS patch has now been merged (not sure if there are any conflicts). Done. - Parth --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review95942 --- On Aug. 20, 2015, 6:27 p.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated Aug. 20, 2015, 6:27 p.m.) Review request for kafka. Bugs: KAFKA-2210 https://issues.apache.org/jira/browse/KAFKA-2210 Repository: kafka Description --- Addressing review comments from Jun. Adding CREATE check for offset topic only if the topic does not exist already. Addressing some more comments. Removing acl.json file Moving PermissionType to trait instead of enum. Following the convention for defining constants. Adding authorizer.config.path back. Addressing more comments from Jun. Addressing more comments. Now addressing Ismael's comments. Case sensitive checks. Addressing Jun's comments. Merge remote-tracking branch 'origin/trunk' into az Conflicts: core/src/main/scala/kafka/server/KafkaApis.scala core/src/main/scala/kafka/server/KafkaServer.scala Deleting KafkaConfigDefTest Addressing comments from Ismael. Diffs - core/src/main/scala/kafka/api/OffsetRequest.scala f418868046f7c99aefdccd9956541a0cb72b1500 core/src/main/scala/kafka/common/AuthorizationException.scala PRE-CREATION core/src/main/scala/kafka/common/ErrorMapping.scala c75c68589681b2c9d6eba2b440ce5e58cddf6370 core/src/main/scala/kafka/security/auth/Acl.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Authorizer.scala PRE-CREATION core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Operation.scala PRE-CREATION core/src/main/scala/kafka/security/auth/PermissionType.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Resource.scala PRE-CREATION core/src/main/scala/kafka/security/auth/ResourceType.scala PRE-CREATION core/src/main/scala/kafka/server/KafkaApis.scala 67f0cad802f901f255825aa2158545d7f5e7cc3d core/src/main/scala/kafka/server/KafkaConfig.scala d547a01cf7098f216a3775e1e1901c5794e1b24c core/src/main/scala/kafka/server/KafkaServer.scala 0e7ba3ede78dbc995c404e0387a6be687703836a core/src/test/scala/unit/kafka/security/auth/AclTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/KafkaPrincipalTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/OperationTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/PermissionTypeTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/ResourceTypeTest.scala PRE-CREATION core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala 3da666f73227fc7ef7093e3790546344065f6825 Diff: https://reviews.apache.org/r/34492/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: [DISCUSS] Reviewers in commit message
+1 on Gwen¹s suggestion. Consider this as my thank you for all the reviews everyone has done in past and are going to do in future. Don¹t make me say thanks on every single commit. Introducing another process when the project has 50 PR open pretty much all the time is not really going to help. Thanks Parth On 7/29/15, 10:40 AM, Gwen Shapira gshap...@cloudera.com wrote: My two cents: The jira comment is a way for the committer to say thank you to people who were involved in the review process. It doesn't have any formal implications - the responsibility for committing good code is on the committer (thats the whole point). It doesn't even have informal implications - no one ever went after a reviewer if a code turned out buggy. I suggest: Leave it up to the committer best judgement and not introduce process where there's really no need for one. Gwen On Wed, Jul 29, 2015 at 6:18 AM, Ismael Juma ism...@juma.me.uk wrote: Hi all, As a general rule, we credit reviewers in the commit message. This is good. However, it is not clear to me if there are guidelines on who should be included as a reviewer (please correct me if I am wrong). I can think of a few options: 1. Anyone that commented on the patch (in the pull request or Review Board) 2. The ones that have reviewed and approved the patch (+1, LGTM, Ship it, etc.) 3. A more sophisticated system that differentiates between someone who reviews and approves a patch versus someone who simply comments on aspects of the patch [1] On the surface, `1` seems appealing because it 's simple and credits people who do partial reviews. The issue, however, is that people (including myself) may not want to be tagged as a reviewer if they left a comment or two, but didn't review the change properly. Option `2` is still simple and it avoids this issue. As such, I lean towards option `2`, although `3` would work for me too (the additional complexity is the main downside). Thoughts? Best, Ismael [1] I don't think we should go this far, but the Linux Kernel is an extreme example of this with `Signed-off-by`, `Acked-by`, `Cc`, `Reviewed-by`, `Tested-by`, `Suggested-by`, `Reported-by`, `Fixes`, etc. More details in their documentation: https://www.kernel.org/doc/Documentation/SubmittingPatches
Re: Review Request 34492: Patch for KAFKA-2210
(...) throw new ... ``` `filter`/`filterNot` could also be used before `foreach` instead of the `if`, but it doesn't give much in this case. done. - Parth --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review93302 --- On Aug. 11, 2015, 1:32 a.m., Parth Brahmbhatt wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated Aug. 11, 2015, 1:32 a.m.) Review request for kafka. Bugs: KAFKA-2210 https://issues.apache.org/jira/browse/KAFKA-2210 Repository: kafka Description --- Addressing review comments from Jun. Adding CREATE check for offset topic only if the topic does not exist already. Addressing some more comments. Removing acl.json file Moving PermissionType to trait instead of enum. Following the convention for defining constants. Adding authorizer.config.path back. Addressing more comments from Jun. Addressing more comments. Now addressing Ismael's comments. Case sensitive checks. Diffs - core/src/main/scala/kafka/api/OffsetRequest.scala f418868046f7c99aefdccd9956541a0cb72b1500 core/src/main/scala/kafka/common/AuthorizationException.scala PRE-CREATION core/src/main/scala/kafka/common/ErrorMapping.scala c75c68589681b2c9d6eba2b440ce5e58cddf6370 core/src/main/scala/kafka/network/RequestChannel.scala 20741281dcaa76374ea6f86a2185dad27b515339 core/src/main/scala/kafka/security/auth/Acl.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Authorizer.scala PRE-CREATION core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Operation.scala PRE-CREATION core/src/main/scala/kafka/security/auth/PermissionType.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Resource.scala PRE-CREATION core/src/main/scala/kafka/security/auth/ResourceType.scala PRE-CREATION core/src/main/scala/kafka/server/KafkaApis.scala 7ea509c2c41acc00430c74e025e069a833aac4e7 core/src/main/scala/kafka/server/KafkaConfig.scala dbe170f87331f43e2dc30165080d2cb7dfe5fdbf core/src/main/scala/kafka/server/KafkaServer.scala 84d4730ac634f9a5bf12a656e422fea03ad72da8 core/src/test/scala/unit/kafka/security/auth/AclTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/KafkaPrincipalTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/OperationTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/PermissionTypeTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/ResourceTypeTest.scala PRE-CREATION core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala PRE-CREATION Diff: https://reviews.apache.org/r/34492/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: Review Request 34492: Patch for KAFKA-2210
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated Aug. 11, 2015, 1:32 a.m.) Review request for kafka. Bugs: KAFKA-2210 https://issues.apache.org/jira/browse/KAFKA-2210 Repository: kafka Description (updated) --- Addressing review comments from Jun. Adding CREATE check for offset topic only if the topic does not exist already. Addressing some more comments. Removing acl.json file Moving PermissionType to trait instead of enum. Following the convention for defining constants. Adding authorizer.config.path back. Addressing more comments from Jun. Addressing more comments. Now addressing Ismael's comments. Case sensitive checks. Diffs (updated) - core/src/main/scala/kafka/api/OffsetRequest.scala f418868046f7c99aefdccd9956541a0cb72b1500 core/src/main/scala/kafka/common/AuthorizationException.scala PRE-CREATION core/src/main/scala/kafka/common/ErrorMapping.scala c75c68589681b2c9d6eba2b440ce5e58cddf6370 core/src/main/scala/kafka/network/RequestChannel.scala 20741281dcaa76374ea6f86a2185dad27b515339 core/src/main/scala/kafka/security/auth/Acl.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Authorizer.scala PRE-CREATION core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Operation.scala PRE-CREATION core/src/main/scala/kafka/security/auth/PermissionType.scala PRE-CREATION core/src/main/scala/kafka/security/auth/Resource.scala PRE-CREATION core/src/main/scala/kafka/security/auth/ResourceType.scala PRE-CREATION core/src/main/scala/kafka/server/KafkaApis.scala 7ea509c2c41acc00430c74e025e069a833aac4e7 core/src/main/scala/kafka/server/KafkaConfig.scala dbe170f87331f43e2dc30165080d2cb7dfe5fdbf core/src/main/scala/kafka/server/KafkaServer.scala 84d4730ac634f9a5bf12a656e422fea03ad72da8 core/src/test/scala/unit/kafka/security/auth/AclTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/KafkaPrincipalTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/OperationTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/PermissionTypeTest.scala PRE-CREATION core/src/test/scala/unit/kafka/security/auth/ResourceTypeTest.scala PRE-CREATION core/src/test/scala/unit/kafka/server/KafkaConfigConfigDefTest.scala PRE-CREATION Diff: https://reviews.apache.org/r/34492/diff/ Testing --- Thanks, Parth Brahmbhatt
Re: [VOTE] Switch to GitHub pull requests for new contributions
+1 (non-binding) Thanks Parth On 7/21/15, 10:24 AM, Gwen Shapira gshap...@cloudera.com wrote: +1 (binding) on using PRs. It sounds like we need additional discussion on how the transition will happen. Maybe move that to a separate thread, to keep the vote easy to follow. On Tue, Jul 21, 2015 at 4:28 AM, Ismael Juma ism...@juma.me.uk wrote: Hi all, I would like to start a vote on switching to GitHub pull requests for new contributions. To be precise, the vote is on whether we should: * Update the documentation to tell users to use pull requests instead of patches and Review Board (i.e. merge KAFKA-2321 and KAFKA-2349) * Use pull requests for new contributions In a previous discussion[1], everyone that participated was in favour. It's also worth reading the Contributing Code Changes wiki page[2] (if you haven't already) to understand the flow. A number of pull requests have been merged in the last few weeks to test this flow and I believe it's working well enough. As usual, there is always room for improvement and I expect is to tweak things as time goes on. The main downside of using GitHub pull requests is that we don't have write access to https://github.com/apache/kafka. That means that we rely on commit hooks to close integrated pull requests (the merge script takes care of formatting the message so that this happens) and the PR creator or Apache Infra to close pull requests that are not integrated. Regarding existing contributions, I think it's up to the contributor to decide whether they want to resubmit it as a pull request or not. I expect that there will be a transition period where the old and new way will co-exist. But that can be discussed separately. The vote will run for 72 hours. +1 (non-binding) from me. Best, Ismael [1] http://search-hadoop.com/m/uyzND1N6CDH1DUc82 [2] https://cwiki.apache.org/confluence/display/KAFKA/Contributing+Code+Chang es