[jira] [Commented] (KAFKA-1690) Add SSL support to Kafka Broker, Producer and Consumer
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14711376#comment-14711376 ] Rajasekar Elango commented on KAFKA-1690: - SSLv3 was disabled since jdk version JDK 7u75. So if this works in 1.7.0_51, but not in 1.7.0_79. It's likely that kafka SSL implementation is using SSLv3. [~NavjotBhardwaj] You can try commenting out property jdk.tls.disabledAlgorithms=SSLv3 on $JAVA_HOME/jre/lib/security/java.security/java.security to confirm if this is the case. Thanks, Raja. Add SSL support to Kafka Broker, Producer and Consumer -- Key: KAFKA-1690 URL: https://issues.apache.org/jira/browse/KAFKA-1690 Project: Kafka Issue Type: Sub-task Reporter: Joe Stein Assignee: Sriharsha Chintalapani Fix For: 0.8.3 Attachments: KAFKA-1690.patch, KAFKA-1690.patch, KAFKA-1690_2015-05-10_23:20:30.patch, KAFKA-1690_2015-05-10_23:31:42.patch, KAFKA-1690_2015-05-11_16:09:36.patch, KAFKA-1690_2015-05-12_16:20:08.patch, KAFKA-1690_2015-05-15_07:18:21.patch, KAFKA-1690_2015-05-20_14:54:35.patch, KAFKA-1690_2015-05-21_10:37:08.patch, KAFKA-1690_2015-06-03_18:52:29.patch, KAFKA-1690_2015-06-23_13:18:20.patch, KAFKA-1690_2015-07-20_06:10:42.patch, KAFKA-1690_2015-07-20_11:59:57.patch, KAFKA-1690_2015-07-25_12:10:55.patch, KAFKA-1690_2015-08-16_20:41:02.patch, KAFKA-1690_2015-08-17_08:12:50.patch, KAFKA-1690_2015-08-17_09:28:52.patch, KAFKA-1690_2015-08-17_12:20:53.patch, KAFKA-1690_2015-08-18_11:24:46.patch, KAFKA-1690_2015-08-18_17:24:48.patch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14654142#comment-14654142 ] Rajasekar Elango commented on KAFKA-1690: - [~singhashish] I did not get that exception, did you run console producer with all SSL properties as documented in setup https://github.com/harshach/kafka/blob/KAFKA-1690-V1/SSL-setup.md .? new java producer needs ssl support as a client --- Key: KAFKA-1690 URL: https://issues.apache.org/jira/browse/KAFKA-1690 Project: Kafka Issue Type: Sub-task Reporter: Joe Stein Assignee: Sriharsha Chintalapani Fix For: 0.8.3 Attachments: KAFKA-1690.patch, KAFKA-1690.patch, KAFKA-1690_2015-05-10_23:20:30.patch, KAFKA-1690_2015-05-10_23:31:42.patch, KAFKA-1690_2015-05-11_16:09:36.patch, KAFKA-1690_2015-05-12_16:20:08.patch, KAFKA-1690_2015-05-15_07:18:21.patch, KAFKA-1690_2015-05-20_14:54:35.patch, KAFKA-1690_2015-05-21_10:37:08.patch, KAFKA-1690_2015-06-03_18:52:29.patch, KAFKA-1690_2015-06-23_13:18:20.patch, KAFKA-1690_2015-07-20_06:10:42.patch, KAFKA-1690_2015-07-20_11:59:57.patch, KAFKA-1690_2015-07-25_12:10:55.patch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14652678#comment-14652678 ] Rajasekar Elango commented on KAFKA-1690: - [~harsha_ch] Thanks for documentation, I tried to run this locally and here are my observations. 1. On kafka server.properties, I had to remove quotes for SSL properties to make it work. For eg when I had ssl.keystore.type = JKS; I got org.apache.kafka.common.KafkaException: java.security.KeyStoreException: JKS not found, when I changed to ssl.keystore.type = JKS , it worked, I had to do this for all ssl properties. Not sure if its just me, can you confirm if it works with quotes? 2. Console producer worked in secure mode, but I need to specify keystore location and password in addition to truststore, I guess documentation need to be updated. 3. Console consumer works in plaintext mode, not sure how to force SSL, I added --property security.protocol=SSL, It seem to be ignored, can you provide an example? I would suggest moving all SSL related configurations to separate ssl config file for broker and producer/consumer. Reason I ask is ssl properties contains secret information like passwords that need to be stored in secure location. If it's part of kafka server.properties we can't keep it in source control and we need keep whole kafka server.properties in secure location. So it's better to accept ssl.config.location as property in server.properties and read all ssl properties from there. The same applies to producer/consumer, producer/consumer.properties can be in source control while security properties can be pulled from secure location. It will also simplify running console-producer/console-consumer easily with one ssl.config.location option instead of bunch of ssl properties. new java producer needs ssl support as a client --- Key: KAFKA-1690 URL: https://issues.apache.org/jira/browse/KAFKA-1690 Project: Kafka Issue Type: Sub-task Reporter: Joe Stein Assignee: Sriharsha Chintalapani Fix For: 0.8.3 Attachments: KAFKA-1690.patch, KAFKA-1690.patch, KAFKA-1690_2015-05-10_23:20:30.patch, KAFKA-1690_2015-05-10_23:31:42.patch, KAFKA-1690_2015-05-11_16:09:36.patch, KAFKA-1690_2015-05-12_16:20:08.patch, KAFKA-1690_2015-05-15_07:18:21.patch, KAFKA-1690_2015-05-20_14:54:35.patch, KAFKA-1690_2015-05-21_10:37:08.patch, KAFKA-1690_2015-06-03_18:52:29.patch, KAFKA-1690_2015-06-23_13:18:20.patch, KAFKA-1690_2015-07-20_06:10:42.patch, KAFKA-1690_2015-07-20_11:59:57.patch, KAFKA-1690_2015-07-25_12:10:55.patch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
ReplicaManager recover after losing a broker
We run 5 node kafka cluster in production with replication factor is 3. If we lose a broker for couple days and kafka-data is wiped off when it comes back online, we had to do rolling restart of all brokers to make them heathy. It recovers itself for most part, FailedFetchRequests and UnderReplicatedPartitions decreases slowly after failed broker comes online. But after some time UnderReplicatedPartitions is flat for 2 brokers and it never drops to zero. When I checked broker logs, I see this exception 2015-07-29 02:15:57,289 [kafka-request-handler-5] ERROR (kafka.server.ReplicaManager) - [Replica Manager on Broker 4]: Error when processing fetch request for partition [com.salesforce.mandm.ajna.Metric.puppet.system,7] offset 5627 from follower with correlation id 2425050. Possible cause: Request for offset 5627 but we only have log segments in the range 5808 to 5985. 2015-07-29 02:15:57,289 [kafka-network-thread-6667-3] ERROR (kafka.network.Processor) - Closing socket for kafka-broker-host1 because of error kafka.common.KafkaException: This operation cannot be completed on a complete request. kafka-broker-host1 is the failed broker that came online. Is this a bug or expected behavior..? Are we supposed to always do rolling restart if kafka-data dir in one broker is wiped off? BTW, we did not had any impact to producers or consumers, we only lost some replications until rolling restart was done. -- Thanks, Raja.
[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14642821#comment-14642821 ] Rajasekar Elango commented on KAFKA-1690: - [~harsha_ch] Is there any documentation on how to use SSL feature? E.g How to enable SSL on broker? how to make producer/consumer talk to broker via SSL? new java producer needs ssl support as a client --- Key: KAFKA-1690 URL: https://issues.apache.org/jira/browse/KAFKA-1690 Project: Kafka Issue Type: Sub-task Reporter: Joe Stein Assignee: Sriharsha Chintalapani Fix For: 0.8.3 Attachments: KAFKA-1690.patch, KAFKA-1690.patch, KAFKA-1690_2015-05-10_23:20:30.patch, KAFKA-1690_2015-05-10_23:31:42.patch, KAFKA-1690_2015-05-11_16:09:36.patch, KAFKA-1690_2015-05-12_16:20:08.patch, KAFKA-1690_2015-05-15_07:18:21.patch, KAFKA-1690_2015-05-20_14:54:35.patch, KAFKA-1690_2015-05-21_10:37:08.patch, KAFKA-1690_2015-06-03_18:52:29.patch, KAFKA-1690_2015-06-23_13:18:20.patch, KAFKA-1690_2015-07-20_06:10:42.patch, KAFKA-1690_2015-07-20_11:59:57.patch, KAFKA-1690_2015-07-25_12:10:55.patch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14608572#comment-14608572 ] Rajasekar Elango commented on KAFKA-1690: - [~harsha_ch] Thanks for working on this. Is this code available anywhere publically for me to try it ? new java producer needs ssl support as a client --- Key: KAFKA-1690 URL: https://issues.apache.org/jira/browse/KAFKA-1690 Project: Kafka Issue Type: Sub-task Reporter: Joe Stein Assignee: Sriharsha Chintalapani Fix For: 0.8.3 Attachments: KAFKA-1690.patch, KAFKA-1690.patch, KAFKA-1690_2015-05-10_23:20:30.patch, KAFKA-1690_2015-05-10_23:31:42.patch, KAFKA-1690_2015-05-11_16:09:36.patch, KAFKA-1690_2015-05-12_16:20:08.patch, KAFKA-1690_2015-05-15_07:18:21.patch, KAFKA-1690_2015-05-20_14:54:35.patch, KAFKA-1690_2015-05-21_10:37:08.patch, KAFKA-1690_2015-06-03_18:52:29.patch, KAFKA-1690_2015-06-23_13:18:20.patch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: [ANNOUNCEMENT] Apache Kafka 0.8.2.0 Released
YaY!. Thanks to Jun and everybody who contributed to this release. We have been waiting for this release for a while. Thanks, Rajasekar Elango (Salesforce.com). On Tue, Feb 3, 2015 at 8:37 PM, Jun Rao j...@confluent.io wrote: The Apache Kafka community is pleased to announce the release for Apache Kafka 0.8.2.0. The 0.8.2.0 release introduces many new features, improvements and fixes including: - A new Java producer for ease of implementation and enhanced performance. - A Kafka-based offset storage. - Delete topic support. - Per topic configuration of preference for consistency over availability. - Scala 2.11 support and dropping support for Scala 2.8. - LZ4 Compression. All of the changes in this release can be found: https://archive.apache.org/dist/kafka/0.8.2.0/RELEASE_NOTES.html Apache Kafka is high-throughput, publish-subscribe messaging system rethought of as a distributed commit log. ** Fast = A single Kafka broker can handle hundreds of megabytes of reads and writes per second from thousands of clients. ** Scalable = Kafka is designed to allow a single cluster to serve as the central data backbone for a large organization. It can be elastically and transparently expanded without downtime. Data streams are partitioned and spread over a cluster of machines to allow data streams larger than the capability of any single machine and to allow clusters of co-ordinated consumers. ** Durable = Messages are persisted on disk and replicated within the cluster to prevent data loss. Each broker can handle terabytes of messages without performance impact. ** Distributed by Design = Kafka has a modern cluster-centric design that offers strong durability and fault-tolerance guarantees. You can download the release from: http://kafka.apache.org/downloads.html We welcome your help and feedback. For more information on how to report problems, and to get involved, visit the project website at http://kafka.apache.org/ Thanks, Jun -- Thanks, Raja.
Re: Review Request 22131: Patch for KAFKA-1477
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/#review54412 --- core/src/main/scala/kafka/utils/ZkUtils.scala https://reviews.apache.org/r/22131/#comment94544 Can we please register secure value of broker in zookeeper as a boolean true/false instead of number 1|0? Two reasons for this request 1. It's really a boolean, not an number. 2. It's breaks backward compactibility with our (salesforce's) original implementation. Our older secure client can no longer work with this newer version of secure broker. - Rajasekar Elango On Sept. 23, 2014, 11:14 a.m., Ivan Lyutov wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/ --- (Updated Sept. 23, 2014, 11:14 a.m.) Review request for kafka. Bugs: KAFKA-1477 https://issues.apache.org/jira/browse/KAFKA-1477 Repository: kafka Description --- Minor fixes, cleanup Refactoring Fixed tests compilation error. Updated according to requested changes: refactoring, minor edits. Added basic functionality for new producer. bug fixes after rebase bug fix after rebase updated tests after rebase changed default security to false Diffs - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java f58b8508d3f813a51015abed772c704390887d7e clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java f9de4af426449cceca12a8de9a9f54a6241d28d8 clients/src/main/java/org/apache/kafka/clients/producer/internals/SSLSocketChannel.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/errors/UnknownKeyStoreException.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/Selector.java 4dd2cdf773f7eb01a93d7f994383088960303dfc clients/src/main/java/org/apache/kafka/common/network/security/AuthConfig.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/security/KeyStores.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/security/SecureAuth.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/security/StoreInitializer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/security/store/JKSInitializer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java 5c5e3d40819e41cab7b52a0eeaee5f2e7317b7b3 config/client.keystore PRE-CREATION config/client.public-key PRE-CREATION config/client.security.properties PRE-CREATION config/consumer.properties 83847de30d10b6e78bb8de28e0bb925d7c0e6ca2 config/producer.properties 39d65d7c6c21f4fccd7af89be6ca12a088d5dd98 config/server.keystore PRE-CREATION config/server.properties 5c0905a572b1f0d8b07bfca967a09cb856a6b09f config/server.public-key PRE-CREATION config/server.security.properties PRE-CREATION core/src/main/scala/kafka/api/FetchRequest.scala 59c09155dd25fad7bed07d3d00039e3dc66db95c core/src/main/scala/kafka/client/ClientUtils.scala ebba87f0566684c796c26cb76c64b4640a5ccfde core/src/main/scala/kafka/cluster/Broker.scala 0060add008bb3bc4b0092f2173c469fce0120be6 core/src/main/scala/kafka/common/UnknownKeyStoreException.scala PRE-CREATION core/src/main/scala/kafka/consumer/ConsumerConfig.scala 9ebbee6c16dc83767297c729d2d74ebbd063a993 core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b9e2bea7b442a19bcebd1b350d39541a8c9dd068 core/src/main/scala/kafka/consumer/SimpleConsumer.scala d349a3000feb9ccd57d1f3cb163548d5bf432186 core/src/main/scala/kafka/controller/ControllerChannelManager.scala ecbfa0f328ba6a652a758ab20cacef324a8b2fb8 core/src/main/scala/kafka/network/BlockingChannel.scala eb7bb14d94cb3648c06d4de36a3b34aacbde4556 core/src/main/scala/kafka/network/SocketServer.scala 3a6f8d121e822e7b6ec32c9147829e91f40e9038 core/src/main/scala/kafka/network/security/AuthConfig.scala PRE-CREATION core/src/main/scala/kafka/network/security/KeyStores.scala PRE-CREATION core/src/main/scala/kafka/network/security/SSLSocketChannel.scala PRE-CREATION core/src/main/scala/kafka/network/security/SecureAuth.scala PRE-CREATION core/src/main/scala/kafka/network/security/store/JKSInitializer.scala PRE-CREATION core/src/main/scala/kafka/producer/ProducerConfig.scala 3cdf23dce3407f1770b9c6543e3a8ae8ab3ff255 core/src/main/scala/kafka/producer/ProducerPool.scala 43df70bb461dd3e385e6b20396adef3c4016a3fc core/src/main/scala/kafka/producer/SyncProducer.scala 42c950375098b51f45c79c6a4a99a36f387bf02b core/src/main/scala/kafka/producer/SyncProducerConfig.scala
[jira] [Commented] (KAFKA-1477) add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication
[ https://issues.apache.org/jira/browse/KAFKA-1477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14146395#comment-14146395 ] Rajasekar Elango commented on KAFKA-1477: - [~gwenshap] config/server.security.properties is already in review. You should able to set need.client.auth=false to disable mutual authentication (client will only authenticate server, server won't authenticate server). [~edgefox] FYI. I left one more comment to ZkUtils.java. add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication -- Key: KAFKA-1477 URL: https://issues.apache.org/jira/browse/KAFKA-1477 Project: Kafka Issue Type: New Feature Reporter: Joe Stein Assignee: Ivan Lyutov Fix For: 0.9.0 Attachments: KAFKA-1477-binary.patch, KAFKA-1477.patch, KAFKA-1477_2014-06-02_16:59:40.patch, KAFKA-1477_2014-06-02_17:24:26.patch, KAFKA-1477_2014-06-03_13:46:17.patch, KAFKA-1477_trunk.patch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1477) add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication
[ https://issues.apache.org/jira/browse/KAFKA-1477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14145089#comment-14145089 ] Rajasekar Elango commented on KAFKA-1477: - This is great [~edgefox] . I did a quick test and able to get it working in both secure and non-secure mode. But I noticed that we original used format broker:port:secureflag to specify brokerlist for producer, now looks like new secure property is added to producer config and specified separately. Do you have usage documented anywhere? I also noticed a change to security.config.file option in console producer and left a comment in review. add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication -- Key: KAFKA-1477 URL: https://issues.apache.org/jira/browse/KAFKA-1477 Project: Kafka Issue Type: New Feature Reporter: Joe Stein Assignee: Ivan Lyutov Fix For: 0.9.0 Attachments: KAFKA-1477-binary.patch, KAFKA-1477.patch, KAFKA-1477_2014-06-02_16:59:40.patch, KAFKA-1477_2014-06-02_17:24:26.patch, KAFKA-1477_2014-06-03_13:46:17.patch, KAFKA-1477_trunk.patch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 22131: Patch for KAFKA-1477
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/#review54191 --- config/server.properties https://reviews.apache.org/r/22131/#comment94187 Can we set secure property to ***false*** to make kafka run in non-secure mode by default so it won't impact existing users - Rajasekar Elango On Sept. 18, 2014, 12:43 p.m., Ivan Lyutov wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/ --- (Updated Sept. 18, 2014, 12:43 p.m.) Review request for kafka. Bugs: KAFKA-1477 https://issues.apache.org/jira/browse/KAFKA-1477 Repository: kafka Description --- Minor fixes, cleanup Refactoring Fixed tests compilation error. Updated according to requested changes: refactoring, minor edits. Added basic functionality for new producer. bug fixes after rebase bug fix after rebase Diffs - clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java f58b8508d3f813a51015abed772c704390887d7e clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java f9de4af426449cceca12a8de9a9f54a6241d28d8 clients/src/main/java/org/apache/kafka/clients/producer/internals/SSLSocketChannel.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/errors/UnknownKeyStoreException.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/Selector.java 4dd2cdf773f7eb01a93d7f994383088960303dfc clients/src/main/java/org/apache/kafka/common/network/security/AuthConfig.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/security/KeyStores.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/security/SecureAuth.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/security/StoreInitializer.java PRE-CREATION clients/src/main/java/org/apache/kafka/common/network/security/store/JKSInitializer.java PRE-CREATION clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java 5c5e3d40819e41cab7b52a0eeaee5f2e7317b7b3 config/client.keystore PRE-CREATION config/client.public-key PRE-CREATION config/client.security.properties PRE-CREATION config/consumer.properties 83847de30d10b6e78bb8de28e0bb925d7c0e6ca2 config/producer.properties 39d65d7c6c21f4fccd7af89be6ca12a088d5dd98 config/server.keystore PRE-CREATION config/server.properties 5c0905a572b1f0d8b07bfca967a09cb856a6b09f config/server.public-key PRE-CREATION config/server.security.properties PRE-CREATION core/src/main/scala/kafka/api/FetchRequest.scala 51cdccf7f90eb530cc62b094ed822b8469d50b12 core/src/main/scala/kafka/client/ClientUtils.scala ce7ede3f6d60e756e252257bd8c6fedc21f21e1c core/src/main/scala/kafka/cluster/Broker.scala 9407ed21fbbd57edeecd888edc32bea6a05d95b3 core/src/main/scala/kafka/common/UnknownKeyStoreException.scala PRE-CREATION core/src/main/scala/kafka/consumer/ConsumerConfig.scala 9ebbee6c16dc83767297c729d2d74ebbd063a993 core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b9e2bea7b442a19bcebd1b350d39541a8c9dd068 core/src/main/scala/kafka/consumer/SimpleConsumer.scala 8db9203d164a4a54f94d8d289e070a0f61e03ff9 core/src/main/scala/kafka/controller/ControllerChannelManager.scala ecbfa0f328ba6a652a758ab20cacef324a8b2fb8 core/src/main/scala/kafka/network/BlockingChannel.scala eb7bb14d94cb3648c06d4de36a3b34aacbde4556 core/src/main/scala/kafka/network/SocketServer.scala d67899080c21e0b6db84657d6845c7ef23b59b0e core/src/main/scala/kafka/network/security/AuthConfig.scala PRE-CREATION core/src/main/scala/kafka/network/security/KeyStores.scala PRE-CREATION core/src/main/scala/kafka/network/security/SSLSocketChannel.scala PRE-CREATION core/src/main/scala/kafka/network/security/SecureAuth.scala PRE-CREATION core/src/main/scala/kafka/network/security/store/JKSInitializer.scala PRE-CREATION core/src/main/scala/kafka/producer/ProducerConfig.scala 3cdf23dce3407f1770b9c6543e3a8ae8ab3ff255 core/src/main/scala/kafka/producer/ProducerPool.scala 43df70bb461dd3e385e6b20396adef3c4016a3fc core/src/main/scala/kafka/producer/SyncProducer.scala 489f0077512d9a69be81649c490274964290fa40 core/src/main/scala/kafka/producer/SyncProducerConfig.scala 69b2d0c11bb1412ce76d566f285333c806be301a core/src/main/scala/kafka/server/AbstractFetcherThread.scala 2e9532e820b5b5c63dfd55f5454b32866d084a37 core/src/main/scala/kafka/server/KafkaConfig.scala dce48db175d6ea379f848a7768de0b1c8e4b929f core/src/main/scala/kafka/server/KafkaHealthcheck.scala
[jira] [Commented] (KAFKA-1477) add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication
[ https://issues.apache.org/jira/browse/KAFKA-1477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14143826#comment-14143826 ] Rajasekar Elango commented on KAFKA-1477: - [~lagivan] Thanks for picking this up. The review looks good, I just left one comment to set default value of secure flag to false. Also do you have this code available on a git repo which I can try to build and test..? add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication -- Key: KAFKA-1477 URL: https://issues.apache.org/jira/browse/KAFKA-1477 Project: Kafka Issue Type: New Feature Reporter: Joe Stein Assignee: Ivan Lyutov Fix For: 0.9.0 Attachments: KAFKA-1477-binary.patch, KAFKA-1477.patch, KAFKA-1477_2014-06-02_16:59:40.patch, KAFKA-1477_2014-06-02_17:24:26.patch, KAFKA-1477_2014-06-03_13:46:17.patch, KAFKA-1477_trunk.patch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1477) add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication
[ https://issues.apache.org/jira/browse/KAFKA-1477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14140751#comment-14140751 ] Rajasekar Elango commented on KAFKA-1477: - Thanks [~edgefox] , could you upload patch to review so that it is easier to review and add comments ? add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication -- Key: KAFKA-1477 URL: https://issues.apache.org/jira/browse/KAFKA-1477 Project: Kafka Issue Type: New Feature Reporter: Joe Stein Assignee: Ivan Lyutov Fix For: 0.9.0 Attachments: KAFKA-1477-binary.patch, KAFKA-1477.patch, KAFKA-1477_2014-06-02_16:59:40.patch, KAFKA-1477_2014-06-02_17:24:26.patch, KAFKA-1477_2014-06-03_13:46:17.patch, KAFKA-1477_trunk.patch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-1477) add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication
[ https://issues.apache.org/jira/browse/KAFKA-1477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14078452#comment-14078452 ] Rajasekar Elango commented on KAFKA-1477: - Very Good Idea [~joestein] . We (@ salesforce) like secure features to be rebased with latest release so that we can get benefits of using latest version. if required we can help with merging/testing etc. In parallel, we should also plan to merge this to trunk (after design spec review) to avoid cost of merging testing for each rebase. For the port, our use case is to run it in secure mode or in non-secure mode. We thought about supporting both secure and non-secure at different ports, but it looked more complicated to implement, so we went with simple flag to turn on secure mode. Thanks, Raja. add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication -- Key: KAFKA-1477 URL: https://issues.apache.org/jira/browse/KAFKA-1477 Project: Kafka Issue Type: New Feature Reporter: Joe Stein Assignee: Ivan Lyutov Fix For: 0.8.2 Attachments: KAFKA-1477-binary.patch, KAFKA-1477.patch, KAFKA-1477_2014-06-02_16:59:40.patch, KAFKA-1477_2014-06-02_17:24:26.patch, KAFKA-1477_2014-06-03_13:46:17.patch -- This message was sent by Atlassian JIRA (v6.2#6252)
Re: [DISCUSS] Kafka Security Specific Features
--topic secure.test [2014-07-17 15:37:46,889] WARN Property topic is not valid (kafka.utils.VerifiableProperties) Hello Secure Kafka *[2014-07-17 15:38:14,186] ERROR OOME with size 352518400 (kafka.network.BoundedByteBufferReceive)* *java.lang.OutOfMemoryError: Java heap space* at java.nio.HeapByteBuffer.init(HeapByteBuffer.java:57) at java.nio.ByteBuffer.allocate(ByteBuffer.java:331) at kafka.network.BoundedByteBufferReceive.byteBufferAllocate(BoundedByteBufferReceive.scala:80) at kafka.network.BoundedByteBufferReceive.readFrom(BoundedByteBufferReceive.scala:63) at kafka.network.Receive$class.readCompletely(Transmission.scala:56) at kafka.network.BoundedByteBufferReceive.readCompletely(BoundedByteBufferReceive.scala:29) at kafka.network.BlockingChannel.receive(BlockingChannel.scala:102) at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:79) at kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:76) at kafka.producer.SyncProducer.send(SyncProducer.scala:117) at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:58) at kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82) at kafka.producer.async.DefaultEventHandler$$anonfun$handle$1.apply$mcV$sp(DefaultEventHandler.scala:67) at kafka.utils.Utils$.swallow(Utils.scala:172) at kafka.utils.Logging$class.swallowError(Logging.scala:106) at kafka.utils.Utils$.swallowError(Utils.scala:45) at kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:67) at kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104) at kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87) at kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67) at scala.collection.immutable.Stream.foreach(Stream.scala:526) at kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66) at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44) On Wed, Jul 16, 2014 at 6:07 PM, Rajasekar Elango rela...@salesforce.com wrote: Pramod, I presented secure kafka configuration and usage at last meet up. So hope this video recording http://www.ustream.tv/recorded/48396701would help. You can skip to about 59 min to jump to security talk. Thanks, Raja. On Wed, Jul 16, 2014 at 5:57 PM, Pramod Deshmukh dpram...@gmail.com wrote: Hello Joe, Is there a configuration or example to test Kafka security piece? Thanks, Pramod On Wed, Jul 16, 2014 at 5:20 PM, Pramod Deshmukh dpram...@gmail.com wrote: Thanks Joe, This branch works. I was able to proceed. I still had to set scala version to 2.9.2 in kafka-run-class.sh. On Wed, Jul 16, 2014 at 3:57 PM, Joe Stein joe.st...@stealth.ly wrote: That is a very old branch. Here is a more up to date one https://github.com/stealthly/kafka/tree/v0.8.2_KAFKA-1477 (needs to be updated to latest trunk might have a chance to-do that next week). You should be using gradle now as per the README. /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://www.stealth.ly Twitter: @allthingshadoop http://www.twitter.com/allthingshadoop / On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh dpram...@gmail.com wrote: Thanks Joe for this, I cloned this branch and tried to run zookeeper but I get Error: Could not find or load main class org.apache.zookeeper.server.quorum.QuorumPeerMain I see scala version is still set to 2.8.0 if [ -z $SCALA_VERSION ]; then SCALA_VERSION=2.8.0 fi Then I installed sbt and scala and followed your instructions for different scala versions. I was able to bring zookeeper up but brokers fail to start with error Error: Could not find or load main class kafka.Kafka I think I am doing something wrong. Can you please help me? Our current production setup is with 2.8.0 and want to stick to it. Thanks, Pramod On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein joe.st...@stealth.ly wrote: Hi,I wanted to re-ignite the discussion around Apache Kafka Security. This is a huge bottleneck (non-starter in some cases) for a lot of organizations (due to regulatory, compliance and other requirements). Below are my suggestions for specific changes in Kafka to accommodate security requirements. This comes from what folks are doing in the wild to workaround and implement security with Kafka as it is today and also what I
Re: [DISCUSS] Kafka Security Specific Features
Pramod, I presented secure kafka configuration and usage at last meet up. So hope this video recording http://www.ustream.tv/recorded/48396701would help. You can skip to about 59 min to jump to security talk. Thanks, Raja. On Wed, Jul 16, 2014 at 5:57 PM, Pramod Deshmukh dpram...@gmail.com wrote: Hello Joe, Is there a configuration or example to test Kafka security piece? Thanks, Pramod On Wed, Jul 16, 2014 at 5:20 PM, Pramod Deshmukh dpram...@gmail.com wrote: Thanks Joe, This branch works. I was able to proceed. I still had to set scala version to 2.9.2 in kafka-run-class.sh. On Wed, Jul 16, 2014 at 3:57 PM, Joe Stein joe.st...@stealth.ly wrote: That is a very old branch. Here is a more up to date one https://github.com/stealthly/kafka/tree/v0.8.2_KAFKA-1477 (needs to be updated to latest trunk might have a chance to-do that next week). You should be using gradle now as per the README. /*** Joe Stein Founder, Principal Consultant Big Data Open Source Security LLC http://www.stealth.ly Twitter: @allthingshadoop http://www.twitter.com/allthingshadoop / On Wed, Jul 16, 2014 at 3:49 PM, Pramod Deshmukh dpram...@gmail.com wrote: Thanks Joe for this, I cloned this branch and tried to run zookeeper but I get Error: Could not find or load main class org.apache.zookeeper.server.quorum.QuorumPeerMain I see scala version is still set to 2.8.0 if [ -z $SCALA_VERSION ]; then SCALA_VERSION=2.8.0 fi Then I installed sbt and scala and followed your instructions for different scala versions. I was able to bring zookeeper up but brokers fail to start with error Error: Could not find or load main class kafka.Kafka I think I am doing something wrong. Can you please help me? Our current production setup is with 2.8.0 and want to stick to it. Thanks, Pramod On Tue, Jun 3, 2014 at 3:57 PM, Joe Stein joe.st...@stealth.ly wrote: Hi,I wanted to re-ignite the discussion around Apache Kafka Security. This is a huge bottleneck (non-starter in some cases) for a lot of organizations (due to regulatory, compliance and other requirements). Below are my suggestions for specific changes in Kafka to accommodate security requirements. This comes from what folks are doing in the wild to workaround and implement security with Kafka as it is today and also what I have discovered from organizations about their blockers. It also picks up from the wiki (which I should have time to update later in the week based on the below and feedback from the thread). 1) Transport Layer Security (i.e. SSL) This also includes client authentication in addition to in-transit security layer. This work has been picked up here https://issues.apache.org/jira/browse/KAFKA-1477 and do appreciate any thoughts, comments, feedback, tomatoes, whatever for this patch. It is a pickup from the fork of the work first done here https://github.com/relango/kafka/tree/kafka_security. 2) Data encryption at rest. This is very important and something that can be facilitated within the wire protocol. It requires an additional map data structure for the encrypted [data encryption key]. With this map (either in your object or in the wire protocol) you can store the dynamically generated symmetric key (for each message) and then encrypt the data using that dynamically generated key. You then encrypt the encryption key using each public key for whom is expected to be able to decrypt the encryption key to then decrypt the message. For each public key encrypted symmetric key (which is now the encrypted [data encryption key] along with which public key it was encrypted with for (so a map of [publicKey] = encryptedDataEncryptionKey) as a chain. Other patterns can be implemented but this is a pretty standard digital enveloping [0] pattern with only 1 field added. Other patterns should be able to use that field to-do their implementation too. 3) Non-repudiation and long term non-repudiation. Non-repudiation is proving data hasn't changed. This is often (if not always) done with x509 public certificates (chained to a certificate authority). Long term non-repudiation is what happens when the certificates of the certificate authority are expired (or revoked) and everything ever signed (ever) with that certificate's public key then becomes no longer provable as ever being authentic. That is where RFC3126 [1] and RFC3161 [2] come in (or worm drives [hardware], etc). For either (or both) of these it is an operation of the encryptor to sign/hash the data
[jira] [Commented] (KAFKA-1477) add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication
[ https://issues.apache.org/jira/browse/KAFKA-1477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14042234#comment-14042234 ] Rajasekar Elango commented on KAFKA-1477: - [~lindong] It doesn't look like review https://reviews.apache.org/r/22905/diff/ commented on this jira is not related to this functionality. Can you double check? add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication -- Key: KAFKA-1477 URL: https://issues.apache.org/jira/browse/KAFKA-1477 Project: Kafka Issue Type: New Feature Reporter: Joe Stein Assignee: Ivan Lyutov Fix For: 0.8.2 Attachments: KAFKA-1477-binary.patch, KAFKA-1477.patch, KAFKA-1477.patch, KAFKA-1477.patch, KAFKA-1477_2014-06-02_16:59:40.patch, KAFKA-1477_2014-06-02_17:24:26.patch, KAFKA-1477_2014-06-03_13:46:17.patch -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (KAFKA-1477) add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication
[ https://issues.apache.org/jira/browse/KAFKA-1477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027817#comment-14027817 ] Rajasekar Elango commented on KAFKA-1477: - Ivan, Looks like the patch you attached to Jira yesterday is uploaded to review. Could you upload..? Can you also provide short summary of changes in last patch? add authentication layer and initial JKS x509 implementation for brokers, producers and consumer for network communication -- Key: KAFKA-1477 URL: https://issues.apache.org/jira/browse/KAFKA-1477 Project: Kafka Issue Type: New Feature Reporter: Joe Stein Assignee: Ivan Lyutov Fix For: 0.8.2 Attachments: KAFKA-1477-binary.patch, KAFKA-1477.patch, KAFKA-1477_2014-06-02_16:59:40.patch, KAFKA-1477_2014-06-02_17:24:26.patch, KAFKA-1477_2014-06-03_13:46:17.patch -- This message was sent by Atlassian JIRA (v6.2#6252)
Re: Review Request 22131: Patch for KAFKA-1477
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/#review44914 --- Ship it! Looks like the changes we did on this commit https://github.com/relango/kafka/commit/0ec255e94973df995c43818bb09d1246440aded9 is not included in patch. We made those changes to fix BadVersion Error thrown by zookeeper. Hopefully they don't happen anymore with latest code in trunk. If it comes back we can create another patch since it is not related to security. So ok with not including it. - Rajasekar Elango On June 3, 2014, 10:53 a.m., Ivan Lyutov wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/ --- (Updated June 3, 2014, 10:53 a.m.) Review request for kafka. Bugs: KAFKA-1477 https://issues.apache.org/jira/browse/KAFKA-1477 Repository: kafka Description --- Updated according to requested changes: refactoring, minor edits. Reverted gradle version Added SSL for Kafka Minor fixes, cleanup Refactoring Fixed tests compilation error. Diffs - config/client.keystore PRE-CREATION config/client.public-key PRE-CREATION config/client.security.properties PRE-CREATION config/consumer.properties 7343cbc28cf8b8de3f096d09c2be955bea73164f config/producer.properties 39d65d7c6c21f4fccd7af89be6ca12a088d5dd98 config/server.keystore PRE-CREATION config/server.properties c9e923aed8551e0797b1ea6f69628b277faf8f48 config/server.public-key PRE-CREATION config/server.security.properties PRE-CREATION core/src/main/scala/kafka/api/FetchRequest.scala a8b73acd1a813284744359e8434cb52d22063c99 core/src/main/scala/kafka/client/ClientUtils.scala ba5fbdcd9e60f953575e529325caf4c41e22f22d core/src/main/scala/kafka/cluster/Broker.scala 9407ed21fbbd57edeecd888edc32bea6a05d95b3 core/src/main/scala/kafka/common/UnknownKeyStoreException.scala PRE-CREATION core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 1a16c691683dda0c53f316e3c4797ea38e776574 core/src/main/scala/kafka/consumer/ConsumerConfig.scala 1cf2f62ba02e4aa66bfa7575865e5d57baf82212 core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b9e2bea7b442a19bcebd1b350d39541a8c9dd068 core/src/main/scala/kafka/consumer/SimpleConsumer.scala 0e64632210385ef63c2ad3445b55ac4f37a63df2 core/src/main/scala/kafka/controller/ControllerChannelManager.scala 8763968fbff697e4c5c98ab1274627c192a4d26a core/src/main/scala/kafka/network/BlockingChannel.scala eb7bb14d94cb3648c06d4de36a3b34aacbde4556 core/src/main/scala/kafka/network/SocketServer.scala 4976d9c3a66bc965f5870a0736e21c7b32650bab core/src/main/scala/kafka/network/security/AuthConfig.scala PRE-CREATION core/src/main/scala/kafka/network/security/KeyStores.scala PRE-CREATION core/src/main/scala/kafka/network/security/SSLSocketChannel.scala PRE-CREATION core/src/main/scala/kafka/network/security/SecureAuth.scala PRE-CREATION core/src/main/scala/kafka/network/security/store/JKSInitializer.scala PRE-CREATION core/src/main/scala/kafka/producer/ConsoleProducer.scala a2af988d99a94a20291d6a2dc9bec73197f1b756 core/src/main/scala/kafka/producer/ProducerConfig.scala 3cdf23dce3407f1770b9c6543e3a8ae8ab3ff255 core/src/main/scala/kafka/producer/ProducerPool.scala 43df70bb461dd3e385e6b20396adef3c4016a3fc core/src/main/scala/kafka/producer/SyncProducer.scala 489f0077512d9a69be81649c490274964290fa40 core/src/main/scala/kafka/producer/SyncProducerConfig.scala 69b2d0c11bb1412ce76d566f285333c806be301a core/src/main/scala/kafka/server/AbstractFetcherThread.scala 3b15254f32252cf824d7a292889ac7662d73ada1 core/src/main/scala/kafka/server/KafkaConfig.scala ef75b67b67676ae5b8931902cbc8c0c2cc72c0d3 core/src/main/scala/kafka/server/KafkaHealthcheck.scala 4acdd70fe9c1ee78d6510741006c2ece65450671 core/src/main/scala/kafka/server/KafkaServer.scala c22e51e0412843ec993721ad3230824c0aadd2ba core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 19df757d75fdbb3ff0b434b6cb10338ff5cc32da core/src/main/scala/kafka/tools/GetOffsetShell.scala fba652e3716a67b04431fc46790ad255201b639f core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 91f072816418040a396a0cad26bc889f539dadd6 core/src/main/scala/kafka/tools/SimpleConsumerShell.scala 747e07280cce72d621acbc771337b909a9b2487e core/src/main/scala/kafka/utils/ZkUtils.scala fcbe269b6057b45793ea95f357890d5d6922e8d4 core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala fcd5eee09fc1831e7fac4c3f1151e9708dc6f5f1 core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala 35dc071b1056e775326981573c9618d8046e601d
Re: Review Request 22131: Patch for KAFKA-1477
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/#review44916 --- Ship it! Looks like the changes we did on this commit https://github.com/relango/kafka/commit/0ec255e94973df995c43818bb09d1246440aded9 is not included in patch. We made those changes to fix BadVersion Error thrown by zookeeper. Hopefully they don't happen anymore with latest code in trunk. If it comes back we can create another patch since it is not related to security. So ok with not including it. - Rajasekar Elango On June 3, 2014, 10:53 a.m., Ivan Lyutov wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/ --- (Updated June 3, 2014, 10:53 a.m.) Review request for kafka. Bugs: KAFKA-1477 https://issues.apache.org/jira/browse/KAFKA-1477 Repository: kafka Description --- Updated according to requested changes: refactoring, minor edits. Reverted gradle version Added SSL for Kafka Minor fixes, cleanup Refactoring Fixed tests compilation error. Diffs - config/client.keystore PRE-CREATION config/client.public-key PRE-CREATION config/client.security.properties PRE-CREATION config/consumer.properties 7343cbc28cf8b8de3f096d09c2be955bea73164f config/producer.properties 39d65d7c6c21f4fccd7af89be6ca12a088d5dd98 config/server.keystore PRE-CREATION config/server.properties c9e923aed8551e0797b1ea6f69628b277faf8f48 config/server.public-key PRE-CREATION config/server.security.properties PRE-CREATION core/src/main/scala/kafka/api/FetchRequest.scala a8b73acd1a813284744359e8434cb52d22063c99 core/src/main/scala/kafka/client/ClientUtils.scala ba5fbdcd9e60f953575e529325caf4c41e22f22d core/src/main/scala/kafka/cluster/Broker.scala 9407ed21fbbd57edeecd888edc32bea6a05d95b3 core/src/main/scala/kafka/common/UnknownKeyStoreException.scala PRE-CREATION core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 1a16c691683dda0c53f316e3c4797ea38e776574 core/src/main/scala/kafka/consumer/ConsumerConfig.scala 1cf2f62ba02e4aa66bfa7575865e5d57baf82212 core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b9e2bea7b442a19bcebd1b350d39541a8c9dd068 core/src/main/scala/kafka/consumer/SimpleConsumer.scala 0e64632210385ef63c2ad3445b55ac4f37a63df2 core/src/main/scala/kafka/controller/ControllerChannelManager.scala 8763968fbff697e4c5c98ab1274627c192a4d26a core/src/main/scala/kafka/network/BlockingChannel.scala eb7bb14d94cb3648c06d4de36a3b34aacbde4556 core/src/main/scala/kafka/network/SocketServer.scala 4976d9c3a66bc965f5870a0736e21c7b32650bab core/src/main/scala/kafka/network/security/AuthConfig.scala PRE-CREATION core/src/main/scala/kafka/network/security/KeyStores.scala PRE-CREATION core/src/main/scala/kafka/network/security/SSLSocketChannel.scala PRE-CREATION core/src/main/scala/kafka/network/security/SecureAuth.scala PRE-CREATION core/src/main/scala/kafka/network/security/store/JKSInitializer.scala PRE-CREATION core/src/main/scala/kafka/producer/ConsoleProducer.scala a2af988d99a94a20291d6a2dc9bec73197f1b756 core/src/main/scala/kafka/producer/ProducerConfig.scala 3cdf23dce3407f1770b9c6543e3a8ae8ab3ff255 core/src/main/scala/kafka/producer/ProducerPool.scala 43df70bb461dd3e385e6b20396adef3c4016a3fc core/src/main/scala/kafka/producer/SyncProducer.scala 489f0077512d9a69be81649c490274964290fa40 core/src/main/scala/kafka/producer/SyncProducerConfig.scala 69b2d0c11bb1412ce76d566f285333c806be301a core/src/main/scala/kafka/server/AbstractFetcherThread.scala 3b15254f32252cf824d7a292889ac7662d73ada1 core/src/main/scala/kafka/server/KafkaConfig.scala ef75b67b67676ae5b8931902cbc8c0c2cc72c0d3 core/src/main/scala/kafka/server/KafkaHealthcheck.scala 4acdd70fe9c1ee78d6510741006c2ece65450671 core/src/main/scala/kafka/server/KafkaServer.scala c22e51e0412843ec993721ad3230824c0aadd2ba core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 19df757d75fdbb3ff0b434b6cb10338ff5cc32da core/src/main/scala/kafka/tools/GetOffsetShell.scala fba652e3716a67b04431fc46790ad255201b639f core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 91f072816418040a396a0cad26bc889f539dadd6 core/src/main/scala/kafka/tools/SimpleConsumerShell.scala 747e07280cce72d621acbc771337b909a9b2487e core/src/main/scala/kafka/utils/ZkUtils.scala fcbe269b6057b45793ea95f357890d5d6922e8d4 core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala fcd5eee09fc1831e7fac4c3f1151e9708dc6f5f1 core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala 35dc071b1056e775326981573c9618d8046e601d
Re: Review Request 22131: Patch for KAFKA-1477
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/#review44927 --- config/server.properties https://reviews.apache.org/r/22131/#comment79513 Should we secure property to false to make kafka run in non-secure mode by default so it won't impact existing users. - Rajasekar Elango On June 3, 2014, 10:53 a.m., Ivan Lyutov wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/ --- (Updated June 3, 2014, 10:53 a.m.) Review request for kafka. Bugs: KAFKA-1477 https://issues.apache.org/jira/browse/KAFKA-1477 Repository: kafka Description --- Updated according to requested changes: refactoring, minor edits. Reverted gradle version Added SSL for Kafka Minor fixes, cleanup Refactoring Fixed tests compilation error. Diffs - config/client.keystore PRE-CREATION config/client.public-key PRE-CREATION config/client.security.properties PRE-CREATION config/consumer.properties 7343cbc28cf8b8de3f096d09c2be955bea73164f config/producer.properties 39d65d7c6c21f4fccd7af89be6ca12a088d5dd98 config/server.keystore PRE-CREATION config/server.properties c9e923aed8551e0797b1ea6f69628b277faf8f48 config/server.public-key PRE-CREATION config/server.security.properties PRE-CREATION core/src/main/scala/kafka/api/FetchRequest.scala a8b73acd1a813284744359e8434cb52d22063c99 core/src/main/scala/kafka/client/ClientUtils.scala ba5fbdcd9e60f953575e529325caf4c41e22f22d core/src/main/scala/kafka/cluster/Broker.scala 9407ed21fbbd57edeecd888edc32bea6a05d95b3 core/src/main/scala/kafka/common/UnknownKeyStoreException.scala PRE-CREATION core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 1a16c691683dda0c53f316e3c4797ea38e776574 core/src/main/scala/kafka/consumer/ConsumerConfig.scala 1cf2f62ba02e4aa66bfa7575865e5d57baf82212 core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b9e2bea7b442a19bcebd1b350d39541a8c9dd068 core/src/main/scala/kafka/consumer/SimpleConsumer.scala 0e64632210385ef63c2ad3445b55ac4f37a63df2 core/src/main/scala/kafka/controller/ControllerChannelManager.scala 8763968fbff697e4c5c98ab1274627c192a4d26a core/src/main/scala/kafka/network/BlockingChannel.scala eb7bb14d94cb3648c06d4de36a3b34aacbde4556 core/src/main/scala/kafka/network/SocketServer.scala 4976d9c3a66bc965f5870a0736e21c7b32650bab core/src/main/scala/kafka/network/security/AuthConfig.scala PRE-CREATION core/src/main/scala/kafka/network/security/KeyStores.scala PRE-CREATION core/src/main/scala/kafka/network/security/SSLSocketChannel.scala PRE-CREATION core/src/main/scala/kafka/network/security/SecureAuth.scala PRE-CREATION core/src/main/scala/kafka/network/security/store/JKSInitializer.scala PRE-CREATION core/src/main/scala/kafka/producer/ConsoleProducer.scala a2af988d99a94a20291d6a2dc9bec73197f1b756 core/src/main/scala/kafka/producer/ProducerConfig.scala 3cdf23dce3407f1770b9c6543e3a8ae8ab3ff255 core/src/main/scala/kafka/producer/ProducerPool.scala 43df70bb461dd3e385e6b20396adef3c4016a3fc core/src/main/scala/kafka/producer/SyncProducer.scala 489f0077512d9a69be81649c490274964290fa40 core/src/main/scala/kafka/producer/SyncProducerConfig.scala 69b2d0c11bb1412ce76d566f285333c806be301a core/src/main/scala/kafka/server/AbstractFetcherThread.scala 3b15254f32252cf824d7a292889ac7662d73ada1 core/src/main/scala/kafka/server/KafkaConfig.scala ef75b67b67676ae5b8931902cbc8c0c2cc72c0d3 core/src/main/scala/kafka/server/KafkaHealthcheck.scala 4acdd70fe9c1ee78d6510741006c2ece65450671 core/src/main/scala/kafka/server/KafkaServer.scala c22e51e0412843ec993721ad3230824c0aadd2ba core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 19df757d75fdbb3ff0b434b6cb10338ff5cc32da core/src/main/scala/kafka/tools/GetOffsetShell.scala fba652e3716a67b04431fc46790ad255201b639f core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 91f072816418040a396a0cad26bc889f539dadd6 core/src/main/scala/kafka/tools/SimpleConsumerShell.scala 747e07280cce72d621acbc771337b909a9b2487e core/src/main/scala/kafka/utils/ZkUtils.scala fcbe269b6057b45793ea95f357890d5d6922e8d4 core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala fcd5eee09fc1831e7fac4c3f1151e9708dc6f5f1 core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala 35dc071b1056e775326981573c9618d8046e601d core/src/test/scala/unit/kafka/network/SocketServerTest.scala 62fb02cf02d3876b9804d756c4bf8514554cc836 core/src/test/scala/unit/kafka/utils/TestUtils.scala 4da0f2c245f75ff0dcab4ecf0af085ab9f8da1bb
Re: Review Request 22131: Patch for KAFKA-1477
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/#review44855 --- Looks like change to UpdateOffsetsInZK.scala (https://github.com/relango/kafka/compare/0.8-snapshot...kafka_security#diff-43) is not included in the patch (= - Rajasekar Elango On June 3, 2014, 10:53 a.m., Ivan Lyutov wrote: --- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/22131/ --- (Updated June 3, 2014, 10:53 a.m.) Review request for kafka. Bugs: KAFKA-1477 https://issues.apache.org/jira/browse/KAFKA-1477 Repository: kafka Description --- Updated according to requested changes: refactoring, minor edits. Reverted gradle version Added SSL for Kafka Minor fixes, cleanup Refactoring Fixed tests compilation error. Diffs - config/client.keystore PRE-CREATION config/client.public-key PRE-CREATION config/client.security.properties PRE-CREATION config/consumer.properties 7343cbc28cf8b8de3f096d09c2be955bea73164f config/producer.properties 39d65d7c6c21f4fccd7af89be6ca12a088d5dd98 config/server.keystore PRE-CREATION config/server.properties c9e923aed8551e0797b1ea6f69628b277faf8f48 config/server.public-key PRE-CREATION config/server.security.properties PRE-CREATION core/src/main/scala/kafka/api/FetchRequest.scala a8b73acd1a813284744359e8434cb52d22063c99 core/src/main/scala/kafka/client/ClientUtils.scala ba5fbdcd9e60f953575e529325caf4c41e22f22d core/src/main/scala/kafka/cluster/Broker.scala 9407ed21fbbd57edeecd888edc32bea6a05d95b3 core/src/main/scala/kafka/common/UnknownKeyStoreException.scala PRE-CREATION core/src/main/scala/kafka/consumer/ConsoleConsumer.scala 1a16c691683dda0c53f316e3c4797ea38e776574 core/src/main/scala/kafka/consumer/ConsumerConfig.scala 1cf2f62ba02e4aa66bfa7575865e5d57baf82212 core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b9e2bea7b442a19bcebd1b350d39541a8c9dd068 core/src/main/scala/kafka/consumer/SimpleConsumer.scala 0e64632210385ef63c2ad3445b55ac4f37a63df2 core/src/main/scala/kafka/controller/ControllerChannelManager.scala 8763968fbff697e4c5c98ab1274627c192a4d26a core/src/main/scala/kafka/network/BlockingChannel.scala eb7bb14d94cb3648c06d4de36a3b34aacbde4556 core/src/main/scala/kafka/network/SocketServer.scala 4976d9c3a66bc965f5870a0736e21c7b32650bab core/src/main/scala/kafka/network/security/AuthConfig.scala PRE-CREATION core/src/main/scala/kafka/network/security/KeyStores.scala PRE-CREATION core/src/main/scala/kafka/network/security/SSLSocketChannel.scala PRE-CREATION core/src/main/scala/kafka/network/security/SecureAuth.scala PRE-CREATION core/src/main/scala/kafka/network/security/store/JKSInitializer.scala PRE-CREATION core/src/main/scala/kafka/producer/ConsoleProducer.scala a2af988d99a94a20291d6a2dc9bec73197f1b756 core/src/main/scala/kafka/producer/ProducerConfig.scala 3cdf23dce3407f1770b9c6543e3a8ae8ab3ff255 core/src/main/scala/kafka/producer/ProducerPool.scala 43df70bb461dd3e385e6b20396adef3c4016a3fc core/src/main/scala/kafka/producer/SyncProducer.scala 489f0077512d9a69be81649c490274964290fa40 core/src/main/scala/kafka/producer/SyncProducerConfig.scala 69b2d0c11bb1412ce76d566f285333c806be301a core/src/main/scala/kafka/server/AbstractFetcherThread.scala 3b15254f32252cf824d7a292889ac7662d73ada1 core/src/main/scala/kafka/server/KafkaConfig.scala ef75b67b67676ae5b8931902cbc8c0c2cc72c0d3 core/src/main/scala/kafka/server/KafkaHealthcheck.scala 4acdd70fe9c1ee78d6510741006c2ece65450671 core/src/main/scala/kafka/server/KafkaServer.scala c22e51e0412843ec993721ad3230824c0aadd2ba core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala 19df757d75fdbb3ff0b434b6cb10338ff5cc32da core/src/main/scala/kafka/tools/GetOffsetShell.scala fba652e3716a67b04431fc46790ad255201b639f core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala 91f072816418040a396a0cad26bc889f539dadd6 core/src/main/scala/kafka/tools/SimpleConsumerShell.scala 747e07280cce72d621acbc771337b909a9b2487e core/src/main/scala/kafka/utils/ZkUtils.scala fcbe269b6057b45793ea95f357890d5d6922e8d4 core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala fcd5eee09fc1831e7fac4c3f1151e9708dc6f5f1 core/src/test/scala/unit/kafka/integration/TopicMetadataTest.scala 35dc071b1056e775326981573c9618d8046e601d core/src/test/scala/unit/kafka/network/SocketServerTest.scala 62fb02cf02d3876b9804d756c4bf8514554cc836 core/src/test/scala/unit/kafka/utils/TestUtils.scala 4da0f2c245f75ff0dcab4ecf0af085ab9f8da1bb Diff: https://reviews.apache.org/r/22131
[jira] [Created] (KAFKA-1041) Number of file handles increases indefinitely in producer if broker host is unresolvable
Rajasekar Elango created KAFKA-1041: --- Summary: Number of file handles increases indefinitely in producer if broker host is unresolvable Key: KAFKA-1041 URL: https://issues.apache.org/jira/browse/KAFKA-1041 Project: Kafka Issue Type: Bug Components: producer Affects Versions: 0.8 Environment: *unix* Reporter: Rajasekar Elango Assignee: Jun Rao We found a issue that if broker host is un resolvable, the number of file handle keep increasing for every message we produce and eventually it uses up all available files handles in operating system. If broker itself is not running and broker host name is resolvable, open file handles count stays flat. lsof output shows number of these open file handles continue to grow for every message we produce. java 19631relango 81u sock0,6 0t0 196966526 can't identify protocol I can easily reproduce this with console producer, If I run console producer with right hostname and if broker is not running, the console producer will exit after three tries. But If I run console producer with unresolvable broker, it throws below exception and continues to wait for user input, every time I enter new message, it opens socket and file handle count keeps increasing.. Here is Exception in producer ERROR fetching topic metadata for topics [Set(test-1378245487417)] from broker [ArrayBuffer(id:0,host:localhost1,port:6667)] failed (kafka.utils.Utils$) kafka.common.KafkaException: fetching topic metadata for topics [Set(test-1378245487417)] from broker [ArrayBuffer(id:0,host:localhost1,port:6667)] failed at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:51) at kafka.producer.BrokerPartitionInfo.updateInfo(BrokerPartitionInfo.scala:82) at kafka.producer.async.DefaultEventHandler$$anonfun$handle$2.apply$mcV$sp(DefaultEventHandler.scala:79) at kafka.utils.Utils$.swallow(Utils.scala:186) at kafka.utils.Logging$class.swallowError(Logging.scala:105) at kafka.utils.Utils$.swallowError(Utils.scala:45) at kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:79) at kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104) at kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87) at kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67) at scala.collection.immutable.Stream.foreach(Stream.scala:526) at kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66) at kafka.producer.async.ProducerSendThread.run(ProducerSendThread.scala:44) Caused by: java.nio.channels.UnresolvedAddressException at sun.nio.ch.Net.checkAddress(Net.java:30) at sun.nio.ch.SocketChannelImpl.connect(SocketChannelImpl.java:487) at kafka.network.BlockingChannel.connect(BlockingChannel.scala:59) at kafka.producer.SyncProducer.connect(SyncProducer.scala:151) at kafka.producer.SyncProducer.getOrMakeConnection(SyncProducer.scala:166) at kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:73) at kafka.producer.SyncProducer.send(SyncProducer.scala:117) at kafka.client.ClientUtils$.fetchTopicMetadata(ClientUtils.scala:37) ... 12 more -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (KAFKA-1035) Add message-send-max-retries and retry-backoff-ms options to console producer
[ https://issues.apache.org/jira/browse/KAFKA-1035?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13755561#comment-13755561 ] Rajasekar Elango commented on KAFKA-1035: - Thanks, So could you apply patch and commit changes to repo. Raja. Add message-send-max-retries and retry-backoff-ms options to console producer - Key: KAFKA-1035 URL: https://issues.apache.org/jira/browse/KAFKA-1035 Project: Kafka Issue Type: Bug Components: producer Affects Versions: 0.7, 0.8 Reporter: Rajasekar Elango Assignee: Jun Rao Labels: patch Fix For: 0.8 Attachments: console_producer_add_options.patch, console_producer_add_options-v2.patch It's possible for console producer to give up too soon if it can't find a leader of a topic. Increasing message-send-max-retries would resolve this but. Console producer doesn't provide options to set message-send-max-retries and retry-backoff-ms. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (KAFKA-1035) Add message-send-max-retries and retry-backoff-ms options to console producer
[ https://issues.apache.org/jira/browse/KAFKA-1035?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13754918#comment-13754918 ] Rajasekar Elango commented on KAFKA-1035: - Sure. do you want me to change description as Brokers can fail receiving the message for multiple reasons, and being unavailable transiently is just one of them.This properties specifies the number of retires before the producer give up and drop this message. Add message-send-max-retries and retry-backoff-ms options to console producer - Key: KAFKA-1035 URL: https://issues.apache.org/jira/browse/KAFKA-1035 Project: Kafka Issue Type: Bug Components: producer Affects Versions: 0.7, 0.8 Reporter: Rajasekar Elango Assignee: Jun Rao Labels: patch Fix For: 0.8 Attachments: console_producer_add_options.patch It's possible for console producer to give up too soon if it can't find a leader of a topic. Increasing message-send-max-retries would resolve this but. Console producer doesn't provide options to set message-send-max-retries and retry-backoff-ms. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (KAFKA-1035) Add message-send-max-retries and retry-backoff-ms options to console producer
[ https://issues.apache.org/jira/browse/KAFKA-1035?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13755130#comment-13755130 ] Rajasekar Elango commented on KAFKA-1035: - Attached patch with modified description. Please review. Add message-send-max-retries and retry-backoff-ms options to console producer - Key: KAFKA-1035 URL: https://issues.apache.org/jira/browse/KAFKA-1035 Project: Kafka Issue Type: Bug Components: producer Affects Versions: 0.7, 0.8 Reporter: Rajasekar Elango Assignee: Jun Rao Labels: patch Fix For: 0.8 Attachments: console_producer_add_options.patch, console_producer_add_options-v2.patch It's possible for console producer to give up too soon if it can't find a leader of a topic. Increasing message-send-max-retries would resolve this but. Console producer doesn't provide options to set message-send-max-retries and retry-backoff-ms. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (KAFKA-1035) Add message-send-max-retries and retry-backoff-ms options to console producer
[ https://issues.apache.org/jira/browse/KAFKA-1035?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajasekar Elango updated KAFKA-1035: Attachment: console_producer_add_options-v2.patch Add message-send-max-retries and retry-backoff-ms options to console producer - Key: KAFKA-1035 URL: https://issues.apache.org/jira/browse/KAFKA-1035 Project: Kafka Issue Type: Bug Components: producer Affects Versions: 0.7, 0.8 Reporter: Rajasekar Elango Assignee: Jun Rao Labels: patch Fix For: 0.8 Attachments: console_producer_add_options.patch, console_producer_add_options-v2.patch It's possible for console producer to give up too soon if it can't find a leader of a topic. Increasing message-send-max-retries would resolve this but. Console producer doesn't provide options to set message-send-max-retries and retry-backoff-ms. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Created] (KAFKA-1035) Add message-send-max-retries and retry-backoff-ms options to console producer
Rajasekar Elango created KAFKA-1035: --- Summary: Add message-send-max-retries and retry-backoff-ms options to console producer Key: KAFKA-1035 URL: https://issues.apache.org/jira/browse/KAFKA-1035 Project: Kafka Issue Type: Bug Components: producer Affects Versions: 0.7, 0.8 Reporter: Rajasekar Elango Assignee: Jun Rao Fix For: 0.8 It's possible for console producer to give up too soon if it can't find a leader of a topic. Increasing message-send-max-retries would resolve this but. Console producer doesn't provide options to set message-send-max-retries and retry-backoff-ms. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (KAFKA-1035) Add message-send-max-retries and retry-backoff-ms options to console producer
[ https://issues.apache.org/jira/browse/KAFKA-1035?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajasekar Elango updated KAFKA-1035: Attachment: console_producer_add_options.patch Patch for fixing this. Add message-send-max-retries and retry-backoff-ms options to console producer - Key: KAFKA-1035 URL: https://issues.apache.org/jira/browse/KAFKA-1035 Project: Kafka Issue Type: Bug Components: producer Affects Versions: 0.7, 0.8 Reporter: Rajasekar Elango Assignee: Jun Rao Labels: patch Fix For: 0.8 Attachments: console_producer_add_options.patch It's possible for console producer to give up too soon if it can't find a leader of a topic. Increasing message-send-max-retries would resolve this but. Console producer doesn't provide options to set message-send-max-retries and retry-backoff-ms. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Updated] (KAFKA-1035) Add message-send-max-retries and retry-backoff-ms options to console producer
[ https://issues.apache.org/jira/browse/KAFKA-1035?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajasekar Elango updated KAFKA-1035: Status: Patch Available (was: Open) See attachment for patch https://issues.apache.org/jira/secure/attachment/12600604/console_producer_add_options.patch Add message-send-max-retries and retry-backoff-ms options to console producer - Key: KAFKA-1035 URL: https://issues.apache.org/jira/browse/KAFKA-1035 Project: Kafka Issue Type: Bug Components: producer Affects Versions: 0.7, 0.8 Reporter: Rajasekar Elango Assignee: Jun Rao Labels: patch Fix For: 0.8 Attachments: console_producer_add_options.patch It's possible for console producer to give up too soon if it can't find a leader of a topic. Increasing message-send-max-retries would resolve this but. Console producer doesn't provide options to set message-send-max-retries and retry-backoff-ms. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Commented] (KAFKA-133) Publish kafka jar to a public maven repository
[ https://issues.apache.org/jira/browse/KAFKA-133?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13560026#comment-13560026 ] Rajasekar Elango commented on KAFKA-133: [~charmalloc] Are we expecting 0.7.3 release with feature to publish to maven repository..? If so do you know the time frame.. ? Also publish-local publish it to local ivy repository.. Is there a way to change it to publish to local maven repository..? Raja. Publish kafka jar to a public maven repository -- Key: KAFKA-133 URL: https://issues.apache.org/jira/browse/KAFKA-133 Project: Kafka Issue Type: Improvement Affects Versions: 0.6, 0.8 Reporter: Neha Narkhede Labels: patch Fix For: 0.8 Attachments: KAFKA-133.patch, pom.xml The released kafka jar must be download manually and then deploy to a private repository before they can be used by a developer using maven2. Similar to other Apache projects, it will be nice to have a way to publish Kafka releases to a public maven repo. In the past, we gave it a try using sbt publish to Sonatype Nexus maven repo, but ran into some authentication problems. It will be good to revisit this and get it resolved. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira
[jira] [Comment Edited] (KAFKA-133) Publish kafka jar to a public maven repository
[ https://issues.apache.org/jira/browse/KAFKA-133?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13560026#comment-13560026 ] Rajasekar Elango edited comment on KAFKA-133 at 1/22/13 9:13 PM: - Are we expecting 0.7.3 release with feature to publish to maven repository..? If so do you know the time frame.. ? Also publish-local publish it to local ivy repository.. Is there a way to change it to publish to local maven repository..? Raja. was (Author: erajasekar): [~charmalloc] Are we expecting 0.7.3 release with feature to publish to maven repository..? If so do you know the time frame.. ? Also publish-local publish it to local ivy repository.. Is there a way to change it to publish to local maven repository..? Raja. Publish kafka jar to a public maven repository -- Key: KAFKA-133 URL: https://issues.apache.org/jira/browse/KAFKA-133 Project: Kafka Issue Type: Improvement Affects Versions: 0.6, 0.8 Reporter: Neha Narkhede Labels: patch Fix For: 0.8 Attachments: KAFKA-133.patch, pom.xml The released kafka jar must be download manually and then deploy to a private repository before they can be used by a developer using maven2. Similar to other Apache projects, it will be nice to have a way to publish Kafka releases to a public maven repo. In the past, we gave it a try using sbt publish to Sonatype Nexus maven repo, but ran into some authentication problems. It will be good to revisit this and get it resolved. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira