Dropping support for Scala 2.9.x

2015-03-27 Thread Ismael Juma
Hi all, The Kafka build currently includes support for Scala 2.9, which means that it cannot take advantage of features introduced in Scala 2.10 or depend on libraries that require it. This restricts the solutions available while trying to solve existing issues. I was browsing JIRA looking for ar

Re: Dropping support for Scala 2.9.x

2015-07-08 Thread Ismael Juma
inues to work or do we want to focus our efforts on 2.10 and 2.11? Please share your opinion. Best, Ismael [1] https://issues.apache.org/jira/browse/KAFKA-2325 On Fri, Mar 27, 2015 at 2:20 PM, Ismael Juma wrote: > Hi all, > > The Kafka build currently includes support for Scala 2.9, wh

Re: [DISCUSSION] Kafka 0.8.2.2 release?

2015-08-14 Thread Ismael Juma
I think this is a good idea as the change is minimal on our side and it has been tested in production for some time by the reporter. Best, Ismael On Fri, Aug 14, 2015 at 1:15 PM, Jun Rao wrote: > Hi, Everyone, > > Since the release of Kafka 0.8.2.1, a number of people have reported an > issue w

Re: Patch for SSL setup for Kafka

2015-08-20 Thread Ismael Juma
Hi LCassa, The SSL/TLS patch was recently merged to trunk, so it should be easier to test. Best, Ismael On Wed, Aug 19, 2015 at 12:54 AM, Cassa L wrote: > Thank You! I will look into it. > > On Tue, Aug 18, 2015 at 10:34 AM, Sriharsha Chintalapani > wrote: > > > Hi , > > you can use the l

Re: 0.9.0.0 remaining jiras

2015-09-18 Thread Ismael Juma
Do we want to deprecate anything before 0.9.0 is released? Maybe the old producer? Ismael On Thu, Sep 17, 2015 at 8:07 PM, Dong Lin wrote: > Hey Jun, > > Should we also include https://issues.apache.org/jira/browse/KAFKA-2390 in > 0.9.0? Becket told me that this is one of those patches ( > http

Re: [ANNOUCE] Apache Kafka 0.8.2.2 Released

2015-10-06 Thread Ismael Juma
On Sat, Oct 3, 2015 at 4:36 PM, Jun Rao wrote: > > We will update the download link in our website shortly. > The download page has been updated: http://kafka.apache.org/downloads.html Ismael

Re: secure communication between producer/consumer to brokers

2015-10-30 Thread Ismael Juma
SSL will be supported in 0.9.0.0, so please use trunk of you'd like to test it in the meantime. Ismael On 28 Oct 2015 12:09, "Pratapi Hemant Patel" wrote: > Hi all, > I followed the wiki > https://cwiki.apache.org/confluence/display/KAFKA/Deploying+SSL+for+Kafka > for secure communication betwee

Re: [VOTE] 0.9.0.0 Candiate 3

2015-11-19 Thread Ismael Juma
+1 (non-binding). Verified source and binary artifacts, ran ./gradlew testAll with JDK 7u80, quick start on source artifact and Scala 2.11 binary artifact. Ismael On Wed, Nov 18, 2015 at 5:57 AM, Jun Rao wrote: > This is the third candidate for release of Apache Kafka 0.9.0.0. This a > major r

Re: 0.9.0.0 RC4

2015-11-23 Thread Ismael Juma
+1 (non-binding). Verified source and binary artifacts, ran ./gradlew testAll with JDK 7u80, quick start on source artifact and Scala 2.11 binary artifact. On Sat, Nov 21, 2015 at 1:21 AM, Jun Rao wrote: > This is the fourth candidate for release of Apache Kafka 0.9.0.0. This a > major release

Re: 0.9.0.0 RC4

2015-11-23 Thread Ismael Juma
On Mon, Nov 23, 2015 at 4:15 PM, hsy...@gmail.com wrote: > In http://kafka.apache.org/090/documentation.html#newconsumerconfigs > partition.assignment.strategy should string, not a list of string? > List is correct, I believe, see how it's used in the code: List assignors = config.getConfigured

Re: Mirrormaker issue with Kafka 0.9 (confluent platform 2.0)

2015-12-11 Thread Ismael Juma
On Fri, Dec 11, 2015 at 3:31 AM, Guozhang Wang wrote: > As Ewen said in KAFKA-1650 we removed the support of multiple consumer > configs in MM, so that you cannot get one MM instance to pull from multiple > origin clusters, but instead you need to use one MM instance for each > origin. > Seems l

Re: SSL - kafka producer cannot publish to topic

2015-12-11 Thread Ismael Juma
Hi Shrikant, On Thu, Dec 10, 2015 at 9:03 PM, Shrikant Patel wrote: > Figured it out. > > I was adding the ssl properties to producer.properties. We need to add > this to separate file and provide that file as input to procuder bat\sh > script --producer.config client-ssl.properties. > > It seem

Re: compression-rate-avg returns 0 with compression enabled

2015-12-15 Thread Ismael Juma
Yes, please. Ismael On 15 Dec 2015 11:52, "tao xiao" wrote: > Hi team, > > I found that the producer metric compression-rate-avg always returns 0 even > with compression.type set to snappy. I drilled down the code and discovered > that the position of bytebuffer in > org.apache.kafka.common.reco

Re: Low-latency, high message size variance

2015-12-16 Thread Ismael Juma
Jason, maybe useful to KAFKA-2986 with this information if we ever decide to do this? Ismael On 16 Dec 2015 04:42, "Jason Gustafson" wrote: > I was talking with Jay this afternoon about this use case. The tricky thing > about adding a ping() or heartbeat() API is that you have to deal with the >

Re: Measuring Kafka Producer request latency when it is less than 1ms

2015-12-21 Thread Ismael Juma
Hi Alexey, Could you please report a bug in JIRA for the NaN result? We should handle that better. Thanks, Ismael On Mon, Dec 21, 2015 at 9:12 AM, Alexey Pirogov wrote: > I'm looking for help with a question regarding measuring of producer > request latency. > I expected that "producer.request

Re: Consumer - Failed to find leader

2015-12-28 Thread Ismael Juma
Hi Prabhu, kafka-console-consumer.sh uses the old consumer by default, but only the new consumer supports security. Use --new-consumer to change this. Hope this helps. Ismael On 28 Dec 2015 05:48, "prabhu v" wrote: > Hi Experts, > > I am getting the below error when running the consumer > "kaf

Re: Gradle build error

2015-12-28 Thread Ismael Juma
Hi Oliver, You need gradle 2.x to build Kafka. Ismael On 27 Dec 2015 18:10, "Oliver Pačut" wrote: I have Gradle 1.4, Groovy 1.8.6, Ant 1.9.3. I followed the included Readme. I cd'd to the kafka directory and just wrote "gradle". I also tried running "./bin/kafka-server-start.sh ./config/serve

Re: Consumer - Failed to find leader

2016-01-04 Thread Ismael Juma
Prabhu, were you able to get this to work in the end? Ismael

Re: Topic Deletion Issues

2016-01-05 Thread Ismael Juma
By the way, https://issues.apache.org/jira/browse/KAFKA-2937 was filed recently and it includes a PR. Ismael On Tue, Jan 5, 2016 at 7:14 PM, Gwen Shapira wrote: > If you have any way of capturing logs at trace level from the point of the > failed attempt until 10 minutes after, it will be super

Re: mirror maker against 0.8.2 source cluster and 0.9.0 destination cluster

2016-01-06 Thread Ismael Juma
Hi Stephen, Newer brokers support older clients, but not the other way around. You could try 0.8.2 MirrorMaker against 0.8.2 source and 0.9.0 target clusters perhaps? Ismael On 6 Jan 2016 11:18, "Stephen Powis" wrote: > Hey! > > So I'm trying to get mirror maker going between two different clus

Re: Upgraded from 0.8.2 to 0.9.0 and FetchFollower request time has increased

2016-01-08 Thread Ismael Juma
Hi Stephen, Have things settled down since? Thanks, Ismael On Fri, Jan 8, 2016 at 7:39 AM, Stephen Powis wrote: > Hey! > > So I upgraded our production kafka cluster from 0.8.2 to 0.9.0 this > morning, and it seems like everything went smoothly. > > As the last step when I changed the inter.br

Re: Upgraded from 0.8.2 to 0.9.0 and FetchFollower request time has increased

2016-01-08 Thread Ismael Juma
Great! On 8 Jan 2016 13:53, "Stephen Powis" wrote: > Hey Ismael, > > It did, just took a few hours :) It seems like its just hit where it was > before the upgrade. > > > > > Thanks! > Stephen > > On Fri, Jan 8, 2016 at 8:03 PM, Ismael Juma wrote:

Re: Problems using ssl

2016-01-08 Thread Ismael Juma
For reference, here's the PR with the fixes: https://github.com/apache/kafka/pull/745 Ismael On Thu, Jan 7, 2016 at 7:51 PM, Ismael Juma wrote: > Hi Mats, > > There's a bug in the documentation where quotes are used in the properties > file even though they should not. I

Re: Kafka 0.9 client producer compatibility with Kafka 0.8.2 broker

2016-01-13 Thread Ismael Juma
Hi Rajiv, This is expected. Newer broker versions support older client versions, but newer client versions don't support older broker versions. Ismael On Wed, Jan 13, 2016 at 10:54 PM, Rajiv Kurian wrote: > We just upgraded one of our Kafka client producers from 0.8.2 to 0.9. Our > broker is s

Re: trouble upgrading from 0.8.2.1 to 0.9.0.0: invalid message

2016-01-15 Thread Ismael Juma
Hi Dave, On Fri, Jan 15, 2016 at 2:04 AM, Dave Peterson wrote: > I was trying to upgrade an 0.8.2.1 broker cluster to 0.9.0.0 by following > the instructions here: > > http://kafka.apache.org/documentation.html#upgrade > > After upgrading one broker, with inter.broker.protocol.version=0.8.2.

Re: Possible Error Code for Kafka API Calls

2016-01-18 Thread Ismael Juma
In addition to this, it would be great to move the protocol docs to the docs folder of the git repo: https://github.com/apache/kafka/tree/trunk/docs This way, we can ensure that the protocol docs are updated at the same time as the protocol code. Ismael On Mon, Jan 18, 2016 at 3:36 PM, Dana Pow

Re: Possible Error Code for Kafka API Calls

2016-01-18 Thread Ismael Juma
think you need special wiki-edit privileges. If you don't see the "edit" bottom, I'd ask in "dev" list for that privilege (sorry, looks like I'm not a wiki admin, so I can't add you). On Mon, Jan 18, 2016 at 7:46 AM, Ismael Juma wrote: > In addition to

Re: Memory records is not writable in MirrorMaker

2016-01-19 Thread Ismael Juma
Can you please file an issue in JIRA for this? Ismael On Tue, Jan 12, 2016 at 2:40 PM, Meghana Narasimhan < mnarasim...@bandwidth.com> wrote: > Hi, > Came across a similar issue. We are running a 3 node cluster (kafka version > 0.9) and Node 0 also has a few mirror makers running. > When we do a

Re: trouble upgrading from 0.8.2.1 to 0.9.0.0: invalid message

2016-01-19 Thread Ismael Juma
pache.kafka.clients.producer.internals.Sender.run(Sender.java:216) > at > org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:128) > at java.lang.Thread.run(Thread.java:745) > > > On Fri, Jan 15, 2016 at 1:06 AM, Ismael Juma wrote: > > > Hi Dave,

Re: Support customized security protocol

2016-01-19 Thread Ismael Juma
Hi Tao, As you say, security protocols are not currently pluggable. `ChannelBuilder` is already an interface, but `SecurityProtocol` is an enum, which makes it hard for users to add additional security protocols. Changing this would probably require a KIP: https://cwiki.apache.org/confluence/disp

Re: kafka version compatibility

2016-01-19 Thread Ismael Juma
Hi Craig, On 19 Jan 2016 20:20, "craig w" wrote: > > is it possible to use the kafka 0.8 java client with kafka brokers running > 0.9? Yes. > is it possible to use the kafka 0.9 client with kafka brokers 0.8.2? No. Ismael

Re: Support customized security protocol

2016-01-19 Thread Ismael Juma
t 09:40 tao xiao wrote: > > > Hi Ismael, > > > > Thank you for your reply. I am happy to have a writeup on this. > > > > Can you think of any other ways to make security protocol pluggable > > instead of extending ChannelBuilder? > > > > On Wed, 20 J

Re: Support customized security protocol

2016-01-20 Thread Ismael Juma
Hi Tao, On Wed, Jan 20, 2016 at 2:25 AM, tao xiao wrote: > The PR provides a new SASL mech but it doesn't provide a pluggable way to > implement user's own logic to do authentication. So I don't think the PR > will meet my need. > Yes, as I said, that would not be enough for your use-case. But

Re: Support customized security protocol

2016-01-20 Thread Ismael Juma
Hi Connie, On Wed, Jan 20, 2016 at 3:56 AM, Connie Yang wrote: > @Ismael, what's the status of the SASL/PLAIN PR, > https://github.com/apache/kafka/pull/341? Rajini said he would create a KIP for it: https://issues.apache.org/jira/browse/KAFKA-2658?focusedCommentId=14987903&page=com.atlassian

Re: trouble upgrading from 0.8.2.1 to 0.9.0.0: invalid message

2016-01-21 Thread Ismael Juma
> > at > > kafka.server.ReplicaManager.appendToLocalLog(ReplicaManager.scala:386) > > at > > kafka.server.ReplicaManager.appendMessages(ReplicaManager.scala:322) > > at > > kafka.server.KafkaApis.handleProducerRequest(KafkaApis.sca

Re: Stuck consumer with new consumer API in 0.9

2016-01-23 Thread Ismael Juma
Hi, Can you please file an issue in JIRA so that we make sure this is investigated? Ismael On Fri, Jan 22, 2016 at 3:13 PM, Han JU wrote: > Hi, > > I'm prototyping with the new consumer API of kafka 0.9 and I'm particularly > interested in the `ConsumerRebalanceListener`. > > My test setup is

Re: End to end compression?

2016-01-23 Thread Ismael Juma
Hi Elias, Please see KIP-31: https://cwiki.apache.org/confluence/display/KAFKA/KIP-31+-+Move+to+relative+offsets+in+compressed+message+sets With regards to broker compression performance, the following is also relevant: http://www.confluent.io/blog/compression-in-apache-kafka-is-now-34-percent-

Re: Kafka 0.9 -> consumer.poll() occasionally returns 0 elements

2016-01-24 Thread Ismael Juma
Hi, On Sun, Jan 24, 2016 at 7:17 PM, Krzysztof Ciesielski < krzysztof.ciesiel...@softwaremill.com> wrote: > Yes, it's exactly 5 seconds on every machine. Sure, I'll open a JIRA > Jason already did so: https://issues.apache.org/jira/browse/KAFKA-3135 Feel free to add relevant information there.

Re: Stuck consumer with new consumer API in 0.9

2016-01-25 Thread Ismael Juma
veloper > >>> > >>> Novazone, Edingsesteenweg 302, B-1755 Gooik, Belgium > >>> T: +32(0)54/26.02.03 - M:+32(0)477/39.01.15 > >>> bruno.rassae...@novazone.be -www.novazone.be > >>> > >>> > On 23 Jan 2016, at 17:52, Ismael Juma

Re: Memory records is not writable in MirrorMaker

2016-01-25 Thread Ismael Juma
Thanks. Ismael On Mon, Jan 25, 2016 at 8:28 PM, Meghana Narasimhan < mnarasim...@bandwidth.com> wrote: > I have created a JIRA, https://issues.apache.org/jira/browse/KAFKA-3147 > > Thanks, > Meghana > > On Tue, Jan 19, 2016 at 5:07 AM, Ismael Juma wrote: > > >

Re: [DISCUSS] KIP-45 Standardize all client sequence interaction on j.u.Collection.

2016-01-26 Thread Ismael Juma
Thanks Pierre. Including the dev mailing list. A few comments: 1. It's worth mentioning that the KafkaConsumer has the @InterfaceStability.Unstable annotation. 2. It would be good to show the existing signatures of the methods being changed before we show the changed signatures. 3. The proposed c

Re: Broker Exception: Attempt to read with a maximum offset less than start offset

2016-01-27 Thread Ismael Juma
Hi Manu and Robert, It would help to know if this still happens in trunk or the 0.9.0 branch. Ismael On 27 Jan 2016 13:05, "Robert Metzger" wrote: > Hi Manu, > > in the streaming-benchmark, are seeing the issue only when reading with > Gearpump, or is it triggered by a different processing fram

Re: [DISCUSS] KIP-45 Standardize all client sequence interaction on j.u.Collection.

2016-01-27 Thread Ismael Juma
Hi Pierre and Jason, A comment below. On Wed, Jan 27, 2016 at 9:01 PM, Jason Gustafson wrote: > Hi Pierre, > > Thanks for your persistence on this issue. I've gone back and forth on this > a few times. The current API can definitely be annoying in some cases, but > breaking compatibility still

Re: Kafka SSL Configuration Problems

2016-02-01 Thread Ismael Juma
Please use advertised.listeners instead of advertised.host.name. See this comment: https://github.com/apache/kafka/pull/793#issuecomment-174287124 Ismael On Mon, Feb 1, 2016 at 4:44 PM, Nazario Parsacala wrote: > Hi, > > We were using kafka for a while now. We have been using the binary releas

Re: Kafka SSL Configuration Problems

2016-02-01 Thread Ismael Juma
On Mon, Feb 1, 2016 at 7:15 PM, Nazario Parsacala wrote: > So it looks like you need both listeners and advertised.listeners ..? > No, you always need to set `listeners` (`advertised.listeners` defaults to `listeners`). If you want `advertised.listeners` to be different than `listeners`, then yo

Re: Kafka SSL Configuration Problems

2016-02-01 Thread Ismael Juma
INFO Registered broker 0 at path /brokers/ids/0 > with addresses: PLAINTEXT -> EndPoint(servername,9092,PLAINTEXT) > (kafka.utils.ZkUtils) > > > The only way I have been able to do this, is setting both. > > > > > On Feb 1, 2016, at 2:54 PM, Ismael Juma wrote: >

Re: [DISCUSS] KIP-45 Standardize all client sequence interaction on j.u.Collection.

2016-02-03 Thread Ismael Juma
Hi Becket, On Wed, Jan 27, 2016 at 10:51 PM, Becket Qin wrote: > 2. For seek(), pause(), resume(), it depends on how easily user can use > them. > If we take current interface, and user have a list of partitions to > pause(), what they can do is something like: > pause(patitionList.toArr

Re: FW: 0.9 consumer log spam - Marking the coordinator dead

2016-02-05 Thread Ismael Juma
Hi Simon, It may be worth trying the 0.9.0 branch as it includes a number of important fixes to the new consumer. Ismael On Fri, Feb 5, 2016 at 12:33 PM, Simon Cooper < simon.coo...@featurespace.co.uk> wrote: > Actually, this is incorrect - it looks like the consumer does not receive > any mess

Re: Kafka protocol fetch request max wait.

2016-02-05 Thread Ismael Juma
Thanks for getting to the bottom of this Rajiv. Ismael On Fri, Feb 5, 2016 at 5:50 PM, Rajiv Kurian wrote: > I've updated Kafka-3159 with my findings. > > Thanks, > Rajiv > > On Thu, Feb 4, 2016 at 10:25 PM, Rajiv Kurian wrote: > > > I think I found out when the problem happens. When a broker

Re: FW: 0.9 consumer log spam - Marking the coordinator dead

2016-02-05 Thread Ismael Juma
t; > Thanks, > Rajiv > > On Fri, Feb 5, 2016 at 5:15 AM, Ismael Juma wrote: > > > Hi Simon, > > > > It may be worth trying the 0.9.0 branch as it includes a number of > > important fixes to the new consumer. > > > > Ismael > > > >

Re: Kafka 0.9.0.1 plan

2016-02-05 Thread Ismael Juma
Hi Becket, On Fri, Feb 5, 2016 at 9:15 PM, Becket Qin wrote: > I am taking KAFKA-3177 off the list because the correct fix might involve > some refactoring of exception hierarchy in new consumer. That may take some > time and 0.9.0.1 probably does not need to block on it. > Sounds good to me.

Re: Kafka 0.9.0.1 plan

2016-02-05 Thread Ismael Juma
gt; > > On Fri, Feb 5, 2016 at 1:19 PM, Ismael Juma wrote: > > > Hi Becket, > > > > On Fri, Feb 5, 2016 at 9:15 PM, Becket Qin wrote: > > > > > I am taking KAFKA-3177 off the list because the correct fix might > involve > > > some refactoring of

Re: Kafka 0.9.0.1 plan

2016-02-08 Thread Ismael Juma
t's still early in 0.9.0.0's life, if KAFKA-3006 has a chance of > making the cut (provided a resolution is attained on the KIP-45) it would > be great to avoid leaving too much time for code relying on Arrays to > become common place. > > On Sat, Feb 6, 2016 at 12:05 AM,

Re: Building the 0.9.0 branch

2016-02-09 Thread Ismael Juma
Hi John, The core jar can be built with either Scala 2.10 or Scala 2.11 and the Scala version is included as part of the artifact. The read me page explains this in more detail. Search for "all scala versions" to find out how to run commands for all Scala versions or "particular version of Scala"

Re: SSL Mirror Maker

2016-02-14 Thread Ismael Juma
Hi Lee, Is the CA used to sign the client certificates in the server truststore and the CA used to sign the server certificates in the client truststore? See the following blog post for a working example (including a Vagrant setup): http://www.confluent.io/blog/apache-kafka-security-authorization

Re: Consumption: poll one message at a time

2016-02-15 Thread Ismael Juma
Hi Andre, Please see KIP-41: https://cwiki.apache.org/confluence/display/KAFKA/KIP-41%3A+KafkaConsumer+Max+Records The aim is to include this in the next release of Kafka. Ismael On Mon, Feb 15, 2016 at 12:21 PM, André wrote: > Hi there > > I've just started evaluating Kafka as an additional

Re: 0.9.0.1 RC1

2016-02-15 Thread Ismael Juma
+1 (non-binding). Verified source and binary artifacts, ran ./gradlew testAll with JDK 7u80, quick start on source artifact and Scala 2.11 binary artifact. Ismael On Fri, Feb 12, 2016 at 2:55 AM, Jun Rao wrote: > This is the first candidate for release of Apache Kafka 0.9.0.1. This a bug > fix

Re: new consumer still classified as beta in 0.9.0.1?

2016-02-19 Thread Ismael Juma
Even though we did not remove the beta label, all significant bugs we are aware of have been fixed (thanks Jason!). I'd say you should try it out. :) Ismael On Fri, Feb 19, 2016 at 9:06 PM, allen chan wrote: > My company is waiting for the new consumer to move out of "beta" mode > before using

Re: Kafka 0.9.0.0 on Solaris 5.10 not running

2016-02-22 Thread Ismael Juma
Hi Marcus, One thing that changed in 0.9.0.0 is that we now use `NetworkClient` for inter-broker communication. `NetworkClient` is also used by the new Java producer and consumer. Does the Java producer in 0.8.2 work fine for you? Ismael On Mon, Feb 22, 2016 at 3:06 PM, Marcus Gründler wrote:

Re: Kafka Security quality level in 0.9.0.1

2016-02-26 Thread Ismael Juma
Hi Anatoliy, We labelled 0.9.0.0 as beta as it's a lot of new code and we want to: 1. Give our users a chance to test it and give us feedback 2. Do additional testing ourselves 0.9.0.1 has fixes for all the security issues we became aware of after the 0.9.0.0 release, but we haven't removed the

Re: [sdc-user] Re: Having trouble to connect StreamSets to Kafka with Kerberos authentication

2016-03-03 Thread Ismael Juma
Hi Harikiran, One comment: `advertised.host.name` is not used if `advertised.listeners` is set and similarly `host.name` is not used if `listeners` is set. In general, the use of those properties is now discouraged in favour of listeners. There is a PR to make the documentation clearer: https://g

Re: Kafka Security

2016-03-04 Thread Ismael Juma
Hi Martin, I suggest reading http://www.confluent.io/blog/apache-kafka-security-authorization-authentication-encryption for an end to end example of how to secure Kafka. Ismael On Fri, Mar 4, 2016 at 12:38 PM, Martin Gainty wrote: > Although authors suggest using existing Cloud security produc

Re: [DISCUSS] KIP-45 Standardize all client sequence interaction on j.u.Collection.

2016-03-07 Thread Ismael Juma
Coming back to this, see below. On Wed, Jan 27, 2016 at 9:01 PM, Jason Gustafson wrote: > > 1. For subscribe() and assign(), change the parameter type to collection as > planned in the KIP. This is at least source-compatible, so as long as users > compile against the updated release, there shoul

Re: Connect bug in 0.9.0.1 client

2016-03-09 Thread Ismael Juma
Well spotted Larkin. Please file an issue as we definitely want to fix this before the next release. Ismael On Wed, Mar 9, 2016 at 10:46 PM, Christian Posta wrote: > Open a JIRA here: https://issues.apache.org/jira/browse/KAFKA > and open a github.com pull request here: https://github.com/apach

Re: Kafka 0.9.0.1 broker 0.9 consumer location of consumer group data

2016-03-14 Thread Ismael Juma
Please upgrade indeed, 0.9.0.1 includes a number of important fixes. Ismael On 14 Mar 2016 18:36, "Rajiv Kurian" wrote: > No I haven't. It's still running the 0.9.0 client. I'll try upgrading if it > sounds like an old bug. > > On Mon, Mar 14, 2016 at 11:24 AM, Jason Gustafson > wrote: > > > He

[VOTE] 0.11.0.0 RC0

2017-06-08 Thread Ismael Juma
Hello Kafka users, developers and client-developers, This is the first candidate for release of Apache Kafka 0.11.0.0. It's worth noting that there are a small number of unresolved issues (including documentation and system tests) related to the new AdminClient and Exactly-once functionality[1] th

Re: Kafka 0.10.1 cluster using 100% disk usage (reads)

2017-06-12 Thread Ismael Juma
For what is worth, 0.9.0.0 and above don't accept messages with no key for compacted topics. Ismael On Mon, Jun 12, 2017 at 10:17 PM, ext-gfenol...@eramet-sln.nc < ext-gfenol...@eramet-sln.nc> wrote: > Hello Milind, and thank you for your answer. > > We just found the solution to our problem. >

[VOTE] 0.11.0.0 RC1

2017-06-18 Thread Ismael Juma
Hello Kafka users, developers and client-developers, This is the second candidate for release of Apache Kafka 0.11.0.0. This is a major version release of Apache Kafka. It includes 32 new KIPs. See the release notes and release plan (https://cwiki.apache.org/conf luence/display/KAFKA/Release+Plan

Re: [VOTE] 0.11.0.0 RC1

2017-06-19 Thread Ismael Juma
> > > > > > +1 (non-binding) > > > > > > Passes librdkafka integration tests (v0.9.5 and master) > > > > > > > > > 2017-06-19 0:32 GMT+02:00 Ismael Juma : > > > > > >> Hello Kafka users, developers and client-develo

Re: [VOTE] 0.11.0.0 RC0

2017-06-19 Thread Ismael Juma
ested the exactly once features, because various core > committers said that they didn't expect this feature to be perfect in this > release. We expect to test this this week though. > > Given that the blockers fixed between RC0 and RC1 haven't changed much in > the areas we t

Re: [VOTE] 0.11.0.0 RC1

2017-06-22 Thread Ismael Juma
> > [ant:checkstyle] [ERROR] >> > > C:\Users\User\Downloads\kafka-0.11.0.0-src\clients\src\main\ >> > > java\org\apache\kafka\common\protocol\Errors.java:89:1: >> > > Class Fan-Out Complexity is 60 (max allowed is 40). >> > > [ClassFan

[VOTE] 0.11.0.0 RC2

2017-06-22 Thread Ismael Juma
Hello Kafka users, developers and client-developers, This is the third candidate for release of Apache Kafka 0.11.0.0. This is a major version release of Apache Kafka. It includes 32 new KIPs. See the release notes and release plan ( https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+

Re: [VOTE] 0.11.0.0 RC2

2017-06-22 Thread Ismael Juma
: https://github.com/apache/kafka/commit/186e3d5efc79ed803f0915d472ace77cbec88694 Full diff: https://github.com/apache/kafka/compare/5b351216621f52a471c21826d0dec3ce3187e697...0.11.0.0-rc2 Ismael On Fri, Jun 23, 2017 at 2:16 AM, Ismael Juma wrote: > Hello Kafka users, developers and cli

Re: [VOTE] 0.11.0.0 RC2

2017-06-26 Thread Ismael Juma
checkstyle] [ERROR] C:\Users\User\Downloads\kafka- > 0.11.0.0-src\clients\src\main\java\org\apache\kafka\common\ > requests\AbstractResponse.java:26:1: Class Fan-Out Complexity is 42 (max > allowed is 40). [ClassFanOutComplexity] > :clients:checkstyleMain FAILED > > FAILURE: Build f

Re: Broker-side Compression and Migration to lz4

2017-06-26 Thread Ismael Juma
Hi Sven, If you change the topic config, any new data received by that broker will be in the new compression type. However, followers don't uncompress data, so they will store the data as it was in the leader. An easier way to test what you are trying to test is to use MirrorMaker to mirror the da

Re: [VOTE] 0.11.0.0 RC2

2017-06-26 Thread Ismael Juma
:54976):, Test worker-SendThread(127.0.0.1:54976), Fin > alizer, metrics-meter-tick-thread-1) > > I tested on a VM and a physical machine, and both give me a lot of errors > like this. > > Thanks. > --Vahid > > > > > From: Ismael Juma > To: Vahid S H

Re: [VOTE] 0.11.0.0 RC2

2017-06-26 Thread Ismael Juma
gCommandTest > testQuotaDescribeEntities PASSED > > kafka.admin.AdminTest > testGetBrokerMetadatas PASSED > > kafka.admin.ConfigCommandTest > shouldParseArgumentsForClientsEntityType > PASSED > > kafka.admin.AclCommandTest > testAclCli PASSED > > kafka.admin.ReassignPartit

Re: [VOTE] 0.11.0.0 RC2

2017-06-27 Thread Ismael Juma
ead, Test > > worker-SendThread(127.0.0.1:59720), /127.0.0.1:5494 > > 2 to /127.0.0.1:54926 workers Thread 3, > > ZkClient-EventThread-22-127.0.0.1:54976, ProcessThread(sid:0 > > cport:54976):, Test worker-SendThread(127.0.0.1:54976), Fin > > alizer, metrics-meter-tick-thread

Re: [VOTE] 0.11.0.0 RC2

2017-06-27 Thread Ismael Juma
CommandTest > > > shouldNotUpdateBrokerConfigIfMalformedConfig > > > PASSED > > > > > > kafka.admin.DeleteConsumerGroupTest > testGroupTopicWideDeleteInZKDo > > > esNothingForActiveGroupConsumingMultipleTopics PASSED > > > > > >

Re: [VOTE] 0.11.0.0 RC2

2017-06-27 Thread Ismael Juma
h the release process and the release announcement will follow in the next few days. Ismael P.S. Guozhang's vote was in a separate thread due to some delivery issues when replying to this thread. On Fri, Jun 23, 2017 at 2:16 AM, Ismael Juma wrote: > Hello Kafka users, develope

Re: 0.11.0.0 RC2 Vote

2017-06-28 Thread Ismael Juma
> +1 > > > > Verified 0110 web docs and java docs; verified quick start with 2.11 / > > 2.12 scala versions. > > > > One minor observation: on the web docs we show the cmd for 2.11 scala > > version; we'd better make it templated with the version num

[ANNOUNCE] Apache Kafka 0.11.0.0 Released

2017-06-28 Thread Ismael Juma
lack-Postava, gosubpl, Grant Henke, Guozhang Wang, Gwen Shapira, Hamidreza Afzali, Hao Chen, hejiefang, Hojjat Jafarpour, huxi, Ismael Juma, Ivan A. Melnikov, Jaikiran Pai, James Cheng, James Chien, Jan Lukavsky, Jason Gustafson, Jean-Philippe Daigle, Jeff Chao, Jeff Widman, Jeyhun Karimov, Jiangjie Qin,

Re: 0.10.1 memory and garbage collection issues

2017-07-09 Thread Ismael Juma
Hi John, We would need more details to be able to help. What is the version of your producers and consumers, is compression being used (and the compression type if it is) and what is the broker/topic message format version? Ismael On Sun, Jul 9, 2017 at 1:13 PM, John Yost wrote: > Hey Everyone

Re: 0.9.x to 0.10.x upgrade--any default settings gotchas?

2017-07-09 Thread Ismael Juma
Hi John, Please read the upgrade documentation for the relevant versions: http://kafka.apache.org/documentation.html#upgrade Also, let's try to keep the discussion in one thread. I asked some questions in the related "0.10.1 memory and garbage collection issues" thread that you started. Ismael

Re: 0.9.x to 0.10.x upgrade--any default settings gotchas?

2017-07-10 Thread Ismael Juma
o update the > log.message.format.version to 0.9.0 until we upgrade the clients. > > --John > > --John > > On Sun, Jul 9, 2017 at 10:46 AM, Ismael Juma wrote: > > > Hi John, > > > > Please read the upgrade documentation for the relevant versions: > > >

Re: Kafka AdminClient

2017-07-17 Thread Ismael Juma
Hi Nitin, The AdminClient doesn't support partition reassignment yet. It is something we would like to support in the future. Ismael On Mon, Jul 17, 2017 at 12:16 AM, nitin sharma wrote: > Hi All, > > My requirement is to automate "Partition Rebalance" through java APIs, > rather than .sh scri

Re: Kafka AdminClient

2017-07-17 Thread Ismael Juma
> kindly highlight options it supports. > > Regards, > Nitin Kumar Sharma. > > > On Mon, Jul 17, 2017 at 4:02 PM, Ismael Juma wrote: > > > Hi Nitin, > > > > The AdminClient doesn't support partition reassignment yet. It is > something > > we would

Re: Kafka compatibility matrix needed

2017-07-18 Thread Ismael Juma
Hi all, 0.8.x clients should work with 0.9.x, 0.10.x and 0.11.x brokers. We have system tests for all the relevant combinations. One thing to be careful about is that Scala consumers and Java consumers store offsets and group management information differently and the Java consumer was only introd

Re: kafka cluster crashs periodically

2017-07-18 Thread Ismael Juma
Hi, This is not really a crash, it just means that a connection to the leader was disconnected. The follower will try to reconnect periodically. If the leader is really down, the Controller will elect a new leader and the following will stop trying to reconnect to the old leader. Hope this helps.

Re: Kafka compatibility matrix needed

2017-07-19 Thread Ismael Juma
possible. Ismael On Wed, Jul 19, 2017 at 12:39 AM, Anish Mashankar wrote: > After upgrading Kafka 0.10.0 to 0.11.0, and changing the Message protocol > to 0.11 on brokers, the consumers with version 0.8.2.1 started reporting > Invalid message error logs. > > On Tue, Jul 18, 2017 at 6:37

Re: Kafka compatibility matrix needed

2017-07-19 Thread Ismael Juma
Hi Sachin, Yes, that should work as the broker will down convert the messages for the older consumer. The one thing to avoid is decreasing the message format version for an existing topic. That is not supported. As usual, you should test this in a non production environment first. Also, one addit

Re: Kafka compatibility matrix needed

2017-07-19 Thread Ismael Juma
Also, make On Wed, Jul 19, 2017 at 5:10 AM, Ismael Juma wrote: > Hi Anish, > > What should happen in that case is that the broker down converts the > messages to a format supported by the older clients. We have tests for this > scenario so it would be good to understand what

Re: [DISCUSS] KIP-177 Consumer perf tool should count rebalance time

2017-07-19 Thread Ismael Juma
I think this is a good chance although it's unfortunate that it's likely to break code that is parsing the output of the performance tool. Would it make sense to only enable this if an option is provided? Ismael On Mon, Jul 17, 2017 at 3:41 PM, Jason Gustafson wrote: > +Users > > Thanks for the

Re: [DISCUSS] KIP-177 Consumer perf tool should count rebalance time

2017-07-20 Thread Ismael Juma
ourselves compatible changes > given the output format that we have chosen for a tool. > > -Jason > > On Wed, Jul 19, 2017 at 7:54 AM, Ismael Juma wrote: > > > I think this is a good chance although it's unfortunate that it's likely > to > > break code that is

Re: Consumer throughput drop

2017-07-21 Thread Ismael Juma
Thanks for reporting the results. Maybe you could submit a PR that updates the ops section? https://github.com/apache/kafka/blob/trunk/docs/ops.html Ismael On Fri, Jul 21, 2017 at 2:49 PM, Ovidiu-Cristian MARCU < ovidiu-cristian.ma...@inria.fr> wrote: > After some tuning, I got better results.

Re: Without data loss migration | Kafka 10 to Kafka 11

2017-07-26 Thread Ismael Juma
Hi Sameer, Yes, the upgrade should be seamless. Can you please share the log entries with the errors? Ismael On Wed, Jul 26, 2017 at 1:35 PM, Sameer Kumar wrote: > Hi , > > I wanted to understand the process for production upgrade of Kafka. As > documented in the https://kafka.apache.org/docum

Re: Confluent Kafka 3.2.2 - rebalancing not happenning

2017-08-02 Thread Ismael Juma
Hi Karan, I noticed that you posted this to the Confluent Google Group as well. Let's discuss it over there. This mailing list is only for Apache Kafka. Thanks, Ismael On Wed, Aug 2, 2017 at 7:11 AM, karan alang wrote: > Hello, here is the update .. > > when i ran script - kafka-preferred-repl

Re: Kafka 0.11.0 problem with transactions.

2017-08-03 Thread Ismael Juma
Hi Marcin, The console producer hasn't been updated to invoke the appropriate methods if transactions are enabled. It also requires a bit of thinking on how it should work. Would there be a way to start and commit the transaction via the console or would the console producer do it periodically? Wh

Re: Random consumer offset rewinds/resets

2017-08-07 Thread Ismael Juma
Hi Christiane, Thanks for the email. That looks like https://issues.apache.org/jira/browse/KAFKA-5600 Ismael On Mon, Aug 7, 2017 at 7:04 PM, Christiane Lemke wrote: > Hi all, > > we are fighting with offset rewinds of seemingly random size and hitting > seemingly random partitions on restartin

Re: Random consumer offset rewinds/resets

2017-08-10 Thread Ismael Juma
oduce > included a consumer group restart, which we didn't do, so that we thought > we our problem is different. > > However, since patching with the fixing commit, everything works fine. > > Thanks again! Christiane > > On 7 August 2017 at 22:07, Ismael Juma wrote: > >

  1   2   3   4   5   >