Hello Jun. Unfortunately I do not have logs from broker 6 to find out
reasons for it to be unresponsive, but yes it was not healthy. I found it
to be unresponsive as well.
How can I recover from all this failures with minimum data loss?
On Tue, Aug 27, 2013 at 8:51 PM, Jun Rao wrote:
> It see
You need to set the serializer to StringEncoder. See
https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+Producer+Example
Thanks,
Jun
On Tue, Aug 27, 2013 at 1:34 PM, Robert Heise wrote:
> Thanks Joe for the response.
>
> I've tried the beta1 binary distribution as well. Here is the steps
For the first question, yes.
For the second one, this is documented in
http://kafka.apache.org/documentation.html#brokerconfigs
"Note that all per topic configuration properties below have the format of
csv (e.g., "topic1:value1,topic2:value2")."
Thanks,
Jun
On Tue, Aug 27, 2013 at 11:52 AM, Y
It seems the replica fetch thread died because of socket timeout (defaults
to 30 secs). Was broker 6 healthy at that point?
Thanks,
Jun
On Tue, Aug 27, 2013 at 11:36 AM, Vadim Keylis wrote:
> We do not use controlled shutdown through JMX, its configured in the
> property file. I do not see con
Cool! You can follow the process of creating a JIRA here:
http://kafka.apache.org/contributing.html
And submit patch here:
https://cwiki.apache.org/confluence/display/KAFKA/Git+Workflow
It will be great if you can also add an entry for this issue in FAQ since I
think this is a common question:
ah, I thought you were trying to run the server and not code against the API
how are you compiling/building? through the IDE?
it seems that the libraries maybe are not on the CLASSPATH for the test
when running it in the IDE
seems like something with how your IDE is setup for your eclipse projec
Thanks Guozhang, Changing max retry to 5 worked. Since I am changing
console producer code, I can also submit patch adding both
message.send.max.retries
and retry.backoff.ms to console producer. Can you let me know process for
submitting patch?
Thanks,
Raja.
On Tue, Aug 27, 2013 at 4:03 PM, Guoz
Thanks Joe for the response.
I've tried the beta1 binary distribution as well. Here is the steps I am
taking:
* Downloaded kafka_2.8.0-0.8.0-beta1
* Imported jar files to eclipse project:
- kafka_2.8.0-0.8.0-beta1/kafka_2.8.0-0.8.0-beta1.jar
- added kafka_2.8.0-0.8.0-beta1/libs/*.* to build pat
Hello Rajasekar,
The remove fetcher log entry is normal under addition of partitions, since
they indicate that some leader changes have happened so brokers are closing
the fetchers to the old leaders.
I just realized that the console Producer does not have the
message.send.max.retries options yet
Thanks Neha & Guozhang,
When I ran StateChangeLogMerger, I am seeing this message repeated 16 times
for each partition:
[2013-08-27 12:30:02,535] INFO [ReplicaFetcherManager on broker 1] Removing
fetcher for partition [test-60,13] (kafka.server.ReplicaFetcherManager)
[2013-08-27 12:30:02,536] INF
You can use the binary distribution if you like
https://dist.apache.org/repos/dist/release/kafka/kafka_2.8.0-0.8.0-beta1.tgz
As far as what is causing the error will need more info can you maybe take
everything going on (commands/steps being done and the output) in your terminal
and gist it.
Hello,
I have to no avail. Im still getting the same exception:
java.lang.ClassNotFoundException:
at java.lang.Class.forName0(Native Method)
at java.lang.Class.forName(Class.java:171)
at kafka.utils.Utils$.createObject(Utils.scala:457)
at kafka.producer.Producer.(Producer.scala:59)
at kafka.jav
Hi Jun,
In a previous email thread
http://markmail.org/search/?q=kafka+log.retention.bytes#query:kafka%20log.retention.bytes+page:1+mid:qnt4pbq47goii2ui+state:results,
you said log.retention.bytes is for each partition. Could you clarify on that?
Say if I have a topic with three partitions. I wa
Hey Robert, the latest quick start is here
http://kafka.apache.org/documentation.html#quickstart
Can you give that a try please, thanks!
/***
Joe Stein
Founder, Principal Consultant
Big Data Open Source Security LLC
http://www.stealth.ly
Twitter: @allt
We do not use controlled shutdown through JMX, its configured in the
property file. I do not see control shutdown message at the time I
initiated the shutdown. However searching for the string produced the
following error messages which happened hours before I started shutting
down service.
[2013-
Hello,
I am getting started with Kafka and the various getting started guides are
getting me close, but Ive run into some basic issues.
Ive followed these guides:
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.8+Quick+Start
http://www.michael-noll.com/blog/2013/03/13/running-a-mult
When you said you tried to shutdown the broker, did you try controlled
shutdown? Do you see "Shutting down" in the logs of the broker that seemed
to hang?
Thanks,
Neha
On Tue, Aug 27, 2013 at 9:12 AM, Vadim Keylis wrote:
> No. They actually were stuck. Not responding to shutdown request. I had
As Guozhang said, your producer might give up sooner than the leader
election completes for the new topic. To confirm if your producer gave up
too soon, you can run the state change log merge tool for this topic and
see when the leader election finished for all partitions
./bin/kafka-run-class.sh
Hello Rajasekar,
In 0.8 producers keep a cache of the partition -> leader_broker_id map
which is used to determine to which brokers should the messages be sent.
After new partitions are added, the cache on the producer has not populated
yet hence it will throw this exception. The producer will the
No. They actually were stuck. Not responding to shutdown request. I had to kill
them with kill -9 command. I try to take heap dump which hang as well.
Sent from my iPhone
On Aug 27, 2013, at 8:14 AM, Jun Rao wrote:
> The errors you listed may not be serious, as long as they are transient.
> Wh
I can't answer the rest but the catchy name is from Gregor Samza. A
character from Kafka's novel called The Metamorphosis.
https://en.wikipedia.org/wiki/Gregor_Samsa#Gregor_Samsa
-Xavier
On Tue, Aug 27, 2013 at 6:51 AM, Jonathan Hodges wrote:
> First off, I want to say this is awesome! It h
Thanks, Jun. That is exactly what I want to know.
Regards,
Libo
-Original Message-
From: Jun Rao [mailto:jun...@gmail.com]
Sent: Tuesday, August 27, 2013 11:25 AM
To: users@kafka.apache.org
Subject: Re: questions about ISR
Look for jmx beans under kafka.server. You will see ???MaxLag
Hello everyone,
We recently increased number of partitions from 4 to 16 and after that
console producer mostly fails with LeaderNotAvailableException and exits
after 3 tries:
Here is last few lines of console producer log:
No partition metadata for topic test-41 due to
kafka.common.LeaderNotAvai
Your understanding is correct. Try running the tool again when both
replicas are in ISR. If it still doesn't work, see if there is any error in
the state-change log.
Thanks,
Jun
On Tue, Aug 27, 2013 at 8:00 AM, Yu, Libo wrote:
> Hi,
>
> We have three brokers in our kafka cluster. For all topi
Look for jmx beans under kafka.server. You will see ???MaxLag and
???MinFetchRate. In the normal case, when a broker fails, the controller
will drop the failed broker out of ISR during leader election. So, the
value of replica.lag.time.max.ms doesn't matter. This value only matters
when the control
The errors you listed may not be serious, as long as they are transient.
When you say 2 of the brokers are not responsive, are they issuing fetch
requests to the 3rd broker (look at the request log)? During a restart of
the whole cluster, brokers that are started later may not have any leader
and t
Hi,
We have three brokers in our kafka cluster. For all topics, the replica factor
is two.
Here is the distribution of leaders. After I ran the leader election tool,
nothing
happened. In this list, the first broker in ISR is the leader. I assume after
running
the tool, the first broker is repli
First off, I want to say this is awesome! It has been great to see all the
great YARN offerings being released lately. I noticed Hadoop 2.x was
recently voted beta so very exciting!
Like many we use Storm for near real-time processing our Kafka based
streams. In addition we send this data to Ha
Thanks, Jun. That is very helpful. However, I still have a couple of
questions. "We have a min fetch rate JMX in the broker". How to
find out how such min fetch rate is defined? And if
replica.lag.time.max.ms is too large, what is the consequence?
Regards,
Libo
-Original Message-
29 matches
Mail list logo