[jira] [Commented] (KAFKA-4666) Failure test for Kafka configured for consistency vs availability
[ https://issues.apache.org/jira/browse/KAFKA-4666?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15836828#comment-15836828 ] Emanuele Cesena commented on KAFKA-4666: Yes, you're correct, there's nothing unexpected here from Kafka perspective. There can be some confusion, imo, from the user perspective. Say you config Kafka for consistency and forget acks=all. Your producer sends some messages, and receives acks (=1) from the master. You think all is good, but in the situation described by the test you end up actually loosing these messages, which is confusing (they were acked... though not by all, this is the user's mistake). So my proposal was just to clarify a bit in the docs, because the acks=all is kind of hidden. Furthermore, as I wrote the test for myself to be sure I understood the issue correctly, I was wondering if you'd like to merge the test for future ref, but this is kind of a minor/conveniency thing. > Failure test for Kafka configured for consistency vs availability > - > > Key: KAFKA-4666 > URL: https://issues.apache.org/jira/browse/KAFKA-4666 > Project: Kafka > Issue Type: Improvement >Reporter: Emanuele Cesena > Attachments: consistency_test.py > > > We recently had an issue with our Kafka setup because of a misconfiguration. > In short, we thought we have configured Kafka for durability, but we didn't > set the producers to acks=all. During a full outage, we had situations where > some partitions were "partitioned", meaning that the followers started > without properly waiting for the right leader, and thus we lost data. Again, > this is not an issue with Kafka, but a misconfiguration on our side. > I think we reproduced the issue, and we built a docker test that proves that, > if the producer isn't set with acks=all, then data can be lost during an > almost full outage. The test is attached. > I was thinking to send a PR, but wanted to run this through you first, as > it's not necessarily proving that a feature works as expected. > In addition, I think the documentation could be slightly improved, for > instance in the section: > http://kafka.apache.org/documentation/#design_ha > by clearly stating that there are 3 steps one should do for configuring kafka > for consistency, the third being that producers should be set with acks=all > (which is now part of the 2nd point). > Please let me know what do you think, and I can send a PR if you agree. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-4667) Connect (and Stream?) should create internal topics with availability or consistency properly set
[ https://issues.apache.org/jira/browse/KAFKA-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15836817#comment-15836817 ] Emanuele Cesena commented on KAFKA-4667: I see - I think I was confused because we do have automatic topic creation enabled, thank you for clarifying. I agree with you, feel free to create the new ticket and close this one, or rename this if you prefer. I guess this can go into Kafka docs as a note under configuring Kafka for consistency. If you have auto topics creation enabled and Kafka configured for consistency, make sure your producers have acks=all, or you risk to end up in corner case situations (as per the other ticket that I mentioned). > Connect (and Stream?) should create internal topics with availability or > consistency properly set > - > > Key: KAFKA-4667 > URL: https://issues.apache.org/jira/browse/KAFKA-4667 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Reporter: Emanuele Cesena > > I'm reporting this as an issue but in fact it requires more investigation > (which unfortunately I'm not able to perform at this time). > Repro steps: > - configure Kafka for consistency, for example: > default.replication.factor=3 > min.insync.replicas=2 > unclean.leader.election.enable=false > - run Connect for the first time, which should create its internal topics > I believe these topics are created with the broker's default, in particular: > min.insync.replicas=2 > unclean.leader.election.enable=false > but connect doesn't produce with acks=all, which in turn may cause the > cluster to go in a bad state (see, e.g., > https://issues.apache.org/jira/browse/KAFKA-4666). > Solution would be to force availability mode, i.e. force: > unclean.leader.election.enable=true > when creating the connect topics, or viceversa detect availability vs > consistency mode and turn acks=all if needed. > I assume the same happens with other kafka-based services such as streams. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-4667) Connect (and Stream?) should create internal topics with availability or consistency properly set
Emanuele Cesena created KAFKA-4667: -- Summary: Connect (and Stream?) should create internal topics with availability or consistency properly set Key: KAFKA-4667 URL: https://issues.apache.org/jira/browse/KAFKA-4667 Project: Kafka Issue Type: Bug Reporter: Emanuele Cesena I'm reporting this as an issue but in fact it requires more investigation (which unfortunately I'm not able to perform at this time). Repro steps: - configure Kafka for consistency, for example: default.replication.factor=3 min.insync.replicas=2 unclean.leader.election.enable=false - run Connect for the first time, which should create its internal topics I believe these topics are created with the broker's default, in particular: min.insync.replicas=2 unclean.leader.election.enable=false but connect doesn't produce with acks=all, which in turn may cause the cluster to go in a bad state (see, e.g., https://issues.apache.org/jira/browse/KAFKA-4666). Solution would be to force availability mode, i.e. force: unclean.leader.election.enable=true when creating the connect topics, or viceversa detect availability vs consistency mode and turn acks=all if needed. I assume the same happens with other kafka-based services such as streams. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-4666) Failure test for Kafka configured for consistency vs availability
Emanuele Cesena created KAFKA-4666: -- Summary: Failure test for Kafka configured for consistency vs availability Key: KAFKA-4666 URL: https://issues.apache.org/jira/browse/KAFKA-4666 Project: Kafka Issue Type: Improvement Reporter: Emanuele Cesena Attachments: consistency_test.py We recently had an issue with our Kafka setup because of a misconfiguration. In short, we thought we have configured Kafka for durability, but we didn't set the producers to acks=all. During a full outage, we had situations where some partitions were "partitioned", meaning that the followers started without properly waiting for the right leader, and thus we lost data. Again, this is not an issue with Kafka, but a misconfiguration on our side. I think we reproduced the issue, and we built a docker test that proves that, if the producer isn't set with acks=all, then data can be lost during an almost full outage. The test is attached. I was thinking to send a PR, but wanted to run this through you first, as it's not necessarily proving that a feature works as expected. In addition, I think the documentation could be slightly improved, for instance in the section: http://kafka.apache.org/documentation/#design_ha by clearly stating that there are 3 steps one should do for configuring kafka for consistency, the third being that producers should be set with acks=all (which is now part of the 2nd point). Please let me know what do you think, and I can send a PR if you agree. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (KAFKA-4350) Can't mirror from Kafka 0.9 to Kafka 0.10.1
[ https://issues.apache.org/jira/browse/KAFKA-4350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15621249#comment-15621249 ] Emanuele Cesena commented on KAFKA-4350: [~hachikuji] Sorry, I'm a bit confused. I need to replicate from Kafka 0.9 to 0.10. I don't think MM 0.9 works for writing to Kafka 0.10, or at least I've tried it in my settings (i.e., K 0.9 -> MM 0.9 -> K 0.10), it didn't work, then I read around and always saw recommendations to use the new MM against the old broker, not viceversa. So I went for MM 0.10: K 0.9 -> MM 0.10 -> K 0.10. In this case, I can use either the new client or the old client (setting bootstrap.servers vs zookeeper.connect). I understand that the new client may not work for reading from K 0.9. But supposedly, the old client (in MM 0.10) should work, shouldn't it? This is the core of the issue I'm reporting, I'm getting errors even with the old client. > Can't mirror from Kafka 0.9 to Kafka 0.10.1 > --- > > Key: KAFKA-4350 > URL: https://issues.apache.org/jira/browse/KAFKA-4350 > Project: Kafka > Issue Type: Bug >Reporter: Emanuele Cesena > > I'm running 2 clusters: K9 with Kafka 0.9 and K10 with Kafka 0.10.1. > In K10, I've set up mirror maker to clone a topic from K9 to K10. > Mirror maker immediately fails while starting, any suggestion? Following > error message and configs. > Error message: > {code:java} > [2016-10-26 23:54:01,663] FATAL [mirrormaker-thread-0] Mirror maker thread > failure due to (kafka.tools.MirrorMaker$MirrorMakerThread) > org.apache.kafka.common.protocol.types.SchemaException: Error reading field > 'cluster_id': Error reading string of length 418, only 43 bytes available > at org.apache.kafka.common.protocol.types.Schema.read(Schema.java:73) > at > org.apache.kafka.clients.NetworkClient.parseResponse(NetworkClient.java:380) > at > org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:449) > at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:269) > at > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:232) > at > org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:180) > at > org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureCoordinatorReady(AbstractCoordinator.java:193) > at > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:248) > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1013) > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:979) > at > kafka.tools.MirrorMaker$MirrorMakerNewConsumer.receive(MirrorMaker.scala:582) > at > kafka.tools.MirrorMaker$MirrorMakerThread.run(MirrorMaker.scala:431) > [2016-10-26 23:54:01,679] FATAL [mirrormaker-thread-0] Mirror maker thread > exited abnormally, stopping the whole mirror maker. > (kafka.tools.MirrorMaker$MirrorMakerThread) > {code} > Consumer: > {code:} > group.id=mirrormaker001 > client.id=mirrormaker001 > bootstrap.servers=...K9... > security.protocol=PLAINTEXT > auto.offset.reset=earliest > {code} > (note that I first run without client.id, then tried adding a client.id > because -- same error in both cases) > Producer: > {code:} > bootstrap.servers=...K10... > security.protocol=PLAINTEXT > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (KAFKA-4350) Can't mirror from Kafka 0.9 to Kafka 0.10.1
[ https://issues.apache.org/jira/browse/KAFKA-4350?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Emanuele Cesena updated KAFKA-4350: --- Description: I'm running 2 clusters: K9 with Kafka 0.9 and K10 with Kafka 0.10.1. In K10, I've set up mirror maker to clone a topic from K9 to K10. Mirror maker immediately fails while starting, any suggestion? Following error message and configs. Error message: {code:java} [2016-10-26 23:54:01,663] FATAL [mirrormaker-thread-0] Mirror maker thread failure due to (kafka.tools.MirrorMaker$MirrorMakerThread) org.apache.kafka.common.protocol.types.SchemaException: Error reading field 'cluster_id': Error reading string of length 418, only 43 bytes available at org.apache.kafka.common.protocol.types.Schema.read(Schema.java:73) at org.apache.kafka.clients.NetworkClient.parseResponse(NetworkClient.java:380) at org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:449) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:269) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:232) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:180) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureCoordinatorReady(AbstractCoordinator.java:193) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:248) at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1013) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:979) at kafka.tools.MirrorMaker$MirrorMakerNewConsumer.receive(MirrorMaker.scala:582) at kafka.tools.MirrorMaker$MirrorMakerThread.run(MirrorMaker.scala:431) [2016-10-26 23:54:01,679] FATAL [mirrormaker-thread-0] Mirror maker thread exited abnormally, stopping the whole mirror maker. (kafka.tools.MirrorMaker$MirrorMakerThread) {code} Consumer: {code:} group.id=mirrormaker001 client.id=mirrormaker001 bootstrap.servers=...K9... security.protocol=PLAINTEXT auto.offset.reset=earliest {code} (note that I first run without client.id, then tried adding a client.id because -- same error in both cases) Producer: {code:} bootstrap.servers=...K10... security.protocol=PLAINTEXT {code} was: I'm running 2 clusters: K9 with Kafka 0.9 and K10 with Kafka 0.10.1. In K10, I've set up mirror maker to clone a topic from K9 to K10. Mirror maker immediately fails while starting, any suggestion? Following error message and configs. Error message: {{ [2016-10-26 23:54:01,663] FATAL [mirrormaker-thread-0] Mirror maker thread failure due to (kafka.tools.MirrorMaker$MirrorMakerThread) org.apache.kafka.common.protocol.types.SchemaException: Error reading field 'cluster_id': Error reading string of length 418, only 43 bytes available at org.apache.kafka.common.protocol.types.Schema.read(Schema.java:73) at org.apache.kafka.clients.NetworkClient.parseResponse(NetworkClient.java:380) at org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:449) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:269) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:232) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:180) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureCoordinatorReady(AbstractCoordinator.java:193) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:248) at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1013) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:979) at kafka.tools.MirrorMaker$MirrorMakerNewConsumer.receive(MirrorMaker.scala:582) at kafka.tools.MirrorMaker$MirrorMakerThread.run(MirrorMaker.scala:431) [2016-10-26 23:54:01,679] FATAL [mirrormaker-thread-0] Mirror maker thread exited abnormally, stopping the whole mirror maker. (kafka.tools.MirrorMaker$MirrorMakerThread) }} Consumer: {{ group.id=mirrormaker001 client.id=mirrormaker001 bootstrap.servers=...K9... security.protocol=PLAINTEXT auto.offset.reset=earliest }} (note that I first run without client.id, then tried adding a client.id because -- same error in both cases) Producer: {{ bootstrap.servers=...K10... security.protocol=PLAINTEXT }} > Can't mirror from Kafka 0.9 to Kafka 0.10.1 > --- > > Key: KAFKA-4350 > URL: https://issues.apache.org/jira/browse/KAFKA-4350 > Project: Kafka > Issue Type: Bug >Reporter: Emanuele Cesena > > I
[jira] [Commented] (KAFKA-4350) Can't mirror from Kafka 0.9 to Kafka 0.10.1
[ https://issues.apache.org/jira/browse/KAFKA-4350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15612592#comment-15612592 ] Emanuele Cesena commented on KAFKA-4350: I did a bit more investigation. I can reproduce similar errors (not exactly the same, but I think very much related) just using the kafka-console-consumer from kafka 0.10.1 distribution against kafka 0.9 broker(s). If I use the new consumer (default), I get a similar error to the mirror maker above. {code:java} org.apache.kafka.common.protocol.types.SchemaException: Error reading field 'brokers': Error reading field 'host': Error reading string of length 27233, only 515 bytes available at org.apache.kafka.common.protocol.types.Schema.read(Schema.java:73) at org.apache.kafka.clients.NetworkClient.parseResponse(NetworkClient.java:380) at org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:449) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:269) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:232) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:180) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureCoordinatorReady(AbstractCoordinator.java:193) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:248) at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1013) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:979) at kafka.consumer.NewShinyConsumer.(BaseConsumer.scala:67) at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:69) at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:50) at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) {code} Interestingly, even the old consumer fails (and similarly, mirror maker with the old consumer also fails). The setup seems to be fine, with debug logs I can see multiple "Got ping response" messages, but suddenly I receive a SocketTimeoutException: {code:java} [2016-10-27 17:25:32,549] DEBUG Got ping response for sessionid: 0x557fcb463450fb0 after 1ms (org.apache.zookeeper.ClientCnxn) ... [2016-10-27 17:25:34,691] INFO Reconnect due to error: (kafka.consumer.SimpleConsumer) java.net.SocketTimeoutException at sun.nio.ch.SocketAdaptor$SocketInputStream.read(SocketAdaptor.java:211) at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103) at java.nio.channels.Channels$ReadableByteChannelImpl.read(Channels.java:385) at org.apache.kafka.common.network.NetworkReceive.readFromReadableChannel(NetworkReceive.java:81) at kafka.network.BlockingChannel.readCompletely(BlockingChannel.scala:129) at kafka.network.BlockingChannel.receive(BlockingChannel.scala:120) at kafka.consumer.SimpleConsumer.liftedTree1$1(SimpleConsumer.scala:86) at kafka.consumer.SimpleConsumer.kafka$consumer$SimpleConsumer$$sendRequest(SimpleConsumer.scala:83) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(SimpleConsumer.scala:132) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1$$anonfun$apply$mcV$sp$1.apply(SimpleConsumer.scala:132) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1$$anonfun$apply$mcV$sp$1.apply(SimpleConsumer.scala:132) at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply$mcV$sp(SimpleConsumer.scala:131) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply(SimpleConsumer.scala:131) at kafka.consumer.SimpleConsumer$$anonfun$fetch$1.apply(SimpleConsumer.scala:131) at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33) at kafka.consumer.SimpleConsumer.fetch(SimpleConsumer.scala:130) at kafka.consumer.ConsumerFetcherThread.fetch(ConsumerFetcherThread.scala:109) at kafka.consumer.ConsumerFetcherThread.fetch(ConsumerFetcherThread.scala:29) at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:118) at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:103) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63) {code} In the same system (but well isolated) I also have kafka 0.9, and the consumer works fine with the server. I mean, if I run: {code:java} /path/to/kafka0.9/kafka-console-consumer {code} works, and with the same params: {code:java} /path/to/kafka0.10.1/kafka-console-consumer {code} fails as above. > Can't mirror from Kafka 0.9 to Kafka 0.10.1 > --- > > Key: KAFKA-4350 > URL: https://issu
[jira] [Created] (KAFKA-4350) Can't mirror from Kafka 0.9 to Kafka 0.10.1
Emanuele Cesena created KAFKA-4350: -- Summary: Can't mirror from Kafka 0.9 to Kafka 0.10.1 Key: KAFKA-4350 URL: https://issues.apache.org/jira/browse/KAFKA-4350 Project: Kafka Issue Type: Bug Reporter: Emanuele Cesena I'm running 2 clusters: K9 with Kafka 0.9 and K10 with Kafka 0.10.1. In K10, I've set up mirror maker to clone a topic from K9 to K10. Mirror maker immediately fails while starting, any suggestion? Following error message and configs. Error message: {{ [2016-10-26 23:54:01,663] FATAL [mirrormaker-thread-0] Mirror maker thread failure due to (kafka.tools.MirrorMaker$MirrorMakerThread) org.apache.kafka.common.protocol.types.SchemaException: Error reading field 'cluster_id': Error reading string of length 418, only 43 bytes available at org.apache.kafka.common.protocol.types.Schema.read(Schema.java:73) at org.apache.kafka.clients.NetworkClient.parseResponse(NetworkClient.java:380) at org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:449) at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:269) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:232) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:180) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureCoordinatorReady(AbstractCoordinator.java:193) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:248) at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1013) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:979) at kafka.tools.MirrorMaker$MirrorMakerNewConsumer.receive(MirrorMaker.scala:582) at kafka.tools.MirrorMaker$MirrorMakerThread.run(MirrorMaker.scala:431) [2016-10-26 23:54:01,679] FATAL [mirrormaker-thread-0] Mirror maker thread exited abnormally, stopping the whole mirror maker. (kafka.tools.MirrorMaker$MirrorMakerThread) }} Consumer: {{ group.id=mirrormaker001 client.id=mirrormaker001 bootstrap.servers=...K9... security.protocol=PLAINTEXT auto.offset.reset=earliest }} (note that I first run without client.id, then tried adding a client.id because -- same error in both cases) Producer: {{ bootstrap.servers=...K10... security.protocol=PLAINTEXT }} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (KAFKA-4338) Release Kafka 0.10.1.0 on Maven Central
[ https://issues.apache.org/jira/browse/KAFKA-4338?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Emanuele Cesena resolved KAFKA-4338. Resolution: Invalid Omg, you're right! Sorry about that. > Release Kafka 0.10.1.0 on Maven Central > --- > > Key: KAFKA-4338 > URL: https://issues.apache.org/jira/browse/KAFKA-4338 > Project: Kafka > Issue Type: Task >Reporter: Emanuele Cesena > > Unless I'm missing something, Kafka 0.10.1.0 doesn't seem to be on maven > central yet: > https://mvnrepository.com/artifact/org.apache.kafka/kafka_2.10 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (KAFKA-4338) Release Kafka 0.10.1.0 on Maven Central
Emanuele Cesena created KAFKA-4338: -- Summary: Release Kafka 0.10.1.0 on Maven Central Key: KAFKA-4338 URL: https://issues.apache.org/jira/browse/KAFKA-4338 Project: Kafka Issue Type: Task Reporter: Emanuele Cesena Unless I'm missing something, Kafka 0.10.1.0 doesn't seem to be on maven central yet: https://mvnrepository.com/artifact/org.apache.kafka/kafka_2.10 -- This message was sent by Atlassian JIRA (v6.3.4#6332)