Since these errors are recoverable, the producer client can retry the
requests. kafka.javaapi.producer.Producer does the same.
Thanks,
Neha
On Fri, Aug 23, 2013 at 9:41 PM, Vadim Keylis wrote:
> Jun. In general how one will recover from NotLeaderForPartitionException?
> What are the steps?
>
>
Sorry Jun. Somehow logs from that particular event got wiped out.
On Thu, Aug 22, 2013 at 8:59 PM, Jun Rao wrote:
> Any error in the controller and state-change log?
>
> Thanks,
>
> Jun
>
>
> On Thu, Aug 22, 2013 at 10:01 AM, Vadim Keylis >wrote:
>
> > I ran command to create 300 topics sequen
Jun. In general how one will recover from NotLeaderForPartitionException?
What are the steps?
Thanks,
Vadim
On Fri, Aug 23, 2013 at 9:07 PM, Jun Rao wrote:
> For the most part, only SocketExceptions and NotLeaderForPartitionException
> are recoverable. MessageSizeTooLargeException may be recov
Search for *-ConsumerLag under kafka.server. There is a jmx bean per
partition in the follower.
Thanks,
Jun
On Fri, Aug 23, 2013 at 7:59 PM, Vadim Keylis wrote:
> Hi Neha. What are jmx properties to monitor for lag and progress of
> replica, until you release the tool?
>
> Sent from my iPhone
To recover, you have to manually delete log segment files without an index.
Thanks,
Jun
On Fri, Aug 23, 2013 at 7:53 PM, Vadim Keylis wrote:
> I was creating topics and ran out of open file handlers that caused index
> not be created.
>
> Are there plans to push it into 0.8 branch? If not, wh
For the most part, only SocketExceptions and NotLeaderForPartitionException
are recoverable. MessageSizeTooLargeException may be recoverable with a
smaller batch size.
Thanks,
Jun
On Fri, Aug 23, 2013 at 4:09 PM, Jason Rosenberg wrote:
> I'm using the kafka.javaapi.producer.Producer class fro
When a broker is restarted, it will automatically catch up from the leader
and will join ISR when it's caught up. Are you not seeing this happening?
Thanks,
Jun
On Fri, Aug 23, 2013 at 11:33 AM, Yu, Libo wrote:
> Hi,
>
> When a broker is not in a topic's ISR, will it try to catch up to go bac
Hi Neha. What are jmx properties to monitor for lag and progress of
replica, until you release the tool?
Sent from my iPhone
On Aug 23, 2013, at 8:32 AM, Neha Narkhede wrote:
> By that, I meant bringing down all brokers, deleting zookeeper data and
> kafka data and restarting the brokers. I sus
I was creating topics and ran out of open file handlers that caused index
not be created.
Are there plans to push it into 0.8 branch? If not, what are the steps to
recover?
Thanks,
Vadim
On Fri, Aug 23, 2013 at 3:45 PM, Neha Narkhede wrote:
> This is fixed on trunk. What I am curious about i
I see. Ideally, this error state is recoverable since we re-read the new
value from zookeeper before retrying the conditional write. Can you
describe how to reproduce this issue? Did you try deleting topics manually
that might've somehow led to this issue?
Thanks,
Neha
On Fri, Aug 23, 2013 at 10
Vadim,
We wrap kafka in our own java service container, which as a happy
coincidence, uses yammer metrics also. The yammer library has a
GraphiteReporter, which you can configure, which will run a background
thread and send all configured yammer metrics to graphite at regular
intervals, e.g. ever
I'm using the kafka.javaapi.producer.Producer class from a java client.
I'm wondering if it ever makes sense to refresh a producer by stopping it
and creating a new one, for example in response to a downstream IO error
(e.g. a broker got restarted, or a stale socket, etc.).
Or should it always be
This is fixed on trunk. What I am curious about is how you reached this
state ?
Thanks,
Neha
On Fri, Aug 23, 2013 at 10:13 AM, Vadim Keylis wrote:
> We got to this state somehow. Is there way to rebuild index or tell kafka
> recovery process skip topics that are incomplete
> [2013-08-13 17:20:0
I maintain a kafka wrapper shell script wrapper for our Kafka packaging. I've
noticed that 0.8 has several bin/*topic.sh scripts, while trunk has as single
bin/kafka-topics.sh, which uses the TopicCommand class.
If you are discussing support for delete-topics in 0.8, perhaps the
TopicCommand c
I mean I think the meta point Neha is making, is that clearly we need
tooling support for topic deletion. We just haven't done it yet. This is
kind of silly but works okay for us temporarily (we don't really delete
topics).
-Jay
On Fri, Aug 23, 2013 at 2:22 PM, Yu, Libo wrote:
> Here is the li
Here is the list of paths that should be deleted:
/brokers/topics/my_topic
/consumers/[groupId]/owners/my_topic
/consumers/[groupId]/offsets/my_topic (if the topic has been consumed)
Let me know if I miss anything. Thanks.
Regards,
Libo
-Original Message-
From: Neha Narkhede [mailto:neh
Great, thanks. I'll build against 0.8 for now then.
On Aug 23, 2013, at 4:33 PM, Joe Stein wrote:
> I need to delete the 0.8.0-beta1-candidate1 branch
>
> The 0.8.0-beta1 release is tagged 0.8.0-beta1 checkout that if you want to
> match the release
>
> 0.8 branch will be the next release
You can either try using rmr on zookeeper client 3.4.x (./zkCli.sh rmr
/node) or write your own script that does a recursive delete -
https://issues.apache.org/jira/browse/ZOOKEEPER-729
Thanks,
Neha
On Fri, Aug 23, 2013 at 10:52 AM, Yu, Libo wrote:
> I will give it a try. I know how to delete
I need to delete the 0.8.0-beta1-candidate1 branch
The 0.8.0-beta1 release is tagged 0.8.0-beta1 checkout that if you want to
match the release
0.8 branch will be the next release
trunk would be 0.8.1 and moving forward
/***
Joe Stein
Founder, Princip
0.8 has some critical fixes that we discovered on 0.8.0-beta1-candidate1.
But note that every revision of 0.8 might not be completely stable until we
have 0.8.0-final release. trunk is a somewhat big change compared to 0.8
and all system tests don't pass on trunk yet.
>From a pure stability standp
Hi all,
I'm building packages for testing 0.8. Which branch should I base my builds
off of?
- 0.8
- 0.8.0-beta1-candidate1
- trunk
?
Thanks!
-Andrew Otto
Hi,
When a broker is not in a topic's ISR, will it try to catch up to go back to
ISR itself?
Or do we have to restart it?
We can increase replica.lag.time.max.ms and replica.lag.max.messages
to let brokers stay longer in ISR. Is that good practice? Still this is
related to the first questions. W
I will give it a try. I know how to delete log files. But to delete the
zookeeper data,
do I only need to run the delete script?
Regards,
Libo
-Original Message-
From: Neha Narkhede [mailto:neha.narkh...@gmail.com]
Sent: Friday, August 23, 2013 1:43 PM
To: users@kafka.apache.org
Subje
Hi Neha,
This is not a logging issue. That broker' log is flooded by this exception.
The exception first appeared early this morning and kept flooding the log.
The broker is not in any topic's ISR so it will not be a leader.
Regards,
Libo
-Original Message-
From: Neha Narkhede [mailto
Thanks so much. Let me look into this.
On Fri, Aug 23, 2013 at 10:38 AM, Neha Narkhede wrote:
> Have you taken a look at this - https://github.com/criteo/kafka-ganglia.
> It
> is linked off of
> https://cwiki.apache.org/confluence/display/KAFKA/JMX+Reporters
>
> Thanks,
> Neha
>
>
> On Fri, Aug
Without the proper functionality of delete topic, I'm not sure if deleting
the zookeeper and kafka data just for that topic is enough or not. On the
surface, this approach seems sufficient, but I'm not sure of all the
consequences of doing that. Also, remember to bounce the entire cluster
once you'
I think we can certainly improve the logging, but this is normal when the
leader tries to shrink/expand the ISR when the controller is in the middle
of shrinking ISR or electing a new leader for the same partition.
Could you please file a JIRA to improve the quality of logging in this case?
Thank
Hi team,
During normal operation, all of a sudden, we found many exceptions in the log
like this:
It seems one thread' zookeeper's data is written unexpectedly by some other
thread.
Any expertise will be appreciated.
[2013-08-23 13:17:00,622] INFO Partition [our.own.topic
one.default,0] on bro
Have you taken a look at this - https://github.com/criteo/kafka-ganglia. It
is linked off of
https://cwiki.apache.org/confluence/display/KAFKA/JMX+Reporters
Thanks,
Neha
On Fri, Aug 23, 2013 at 10:27 AM, Vadim Keylis wrote:
> Good morning. i have not used yammer metrics until we deploy kafka. C
Good morning. i have not used yammer metrics until we deploy kafka. Csv
output is great, but sending data to graphite or similar straight from
kafka would be better. We can do our custom implementation that queries jmx
and sends data, but I would be curious how community does it.
is there module th
Hi Neha,
One more questions. Assume I want to delete a topic. When you say deleting
zookeeper data and kafka data,
do you mean deleting zookeeper data and kafka data for ALL the topics or only
for that particular topic?
Regards,
Libo
-Original Message-
From: Neha Narkhede [mailto:ne
We got to this state somehow. Is there way to rebuild index or tell kafka
recovery process skip topics that are incomplete
[2013-08-13 17:20:08,953] INFO [Log Manager on Broker 7] Loading log
'm3_registration-29' (kafka.log.LogManager)
[2013-08-13 17:20:08,992] FATAL Fatal error during KafkaServerS
We are currently working on the following JIRA to avoid decompress and
re-compress at MirrorMaker, when this is done, the size of the logs on
source and target clusters should be the same as long as the batch size of
the mirror maker producer is the same as the batch size of the source
producer:
h
Hey guys,
This may be relevant to people on this list. A few of us at LinkedIn have
been working on Samza, a stream processing framework built on YARN. We just
added this as an Apache Incubator project. We would love to get people's
feedback (and help!). Here are the docs:
http://samza.incubator.
Yeah I agree, this is a problem.
The issue is that a produce request which is either in the network buffer
or in the request processing queue on the broker may still be processed
after a disconnect. So there is a race condition between that processing
and the reconnect/retry logic. You could work
Hi LIbo,
Yes, this is still the case in 0.8.
Guozhang
On Fri, Aug 23, 2013 at 8:14 AM, Philip O'Toole wrote:
> Yes, the Kafka team has told me that this is how it works (at least for
> 0.72).
>
> Philip
>
> On Fri, Aug 23, 2013 at 7:53 AM, Yu, Libo wrote:
> > Hi team,
> >
> > Right now, from
By that, I meant bringing down all brokers, deleting zookeeper data and
kafka data and restarting the brokers. I suspect attempting a delete topic
might have caused something bad to happen on the ISR side. It will be great
if you can start clean and then see if the ISR is still an issue.
Also, I f
An auto-increment index can be assigned to a message as a key when it is being
published.
The consumer can monitor this index when receiving. If the expected message
does not
show up, buffer all received messages in a hashtable (use index as hash key)
until it is
received. Then handle all mes
Ross,
This is a general issue with resending. Since resending is typically done
on a new socket, essentially new messages are sent from a new instance of
producer. So, there is no easy way to ensure that the new messages are
ordered behind the ones sent by the old instance of the producer. So 0.8
Yes, the Kafka team has told me that this is how it works (at least for 0.72).
Philip
On Fri, Aug 23, 2013 at 7:53 AM, Yu, Libo wrote:
> Hi team,
>
> Right now, from a stream, an iterator can be obtained which has a blocking
> hasNext().
> So what is the implementation behind the iterator? I as
The bottleneck can be either CPU, network, or disk I/O. You just need to
monitor the load on each. For example, if you monitor the per thread level
CPU load, in MM you can figure out if there is single thread that's the
bottleneck. Then you can look at the I/O load on the target broker and see
if I
Hi team,
Right now, from a stream, an iterator can be obtained which has a blocking
hasNext().
So what is the implementation behind the iterator? I assume there must be queue
and
the iterator monitors the queue. And a separate thread fetches data and feeds
to the
queue when it is almost empty.
Ross -- thanks.
How much code are you writing to do all this, post-Kafka? Have you
considered Storm? I believe the Trident topologies can give you
guaranteed-once semantics, so you may be interested in checking that
out, if you have the time (I have not yet played with Trident stuff
myself, but St
That would be great. Thanks,
Jun
On Fri, Aug 23, 2013 at 7:04 AM, Andrew Otto wrote:
> Jun,
>
> Note that the puppet module README I liked to isn't a full jmxtrans
> example JSON query. It is a jmxtrans puppet module usage example. So,
> using that in puppet will generate a .json file contai
ZK paths for 0.8 are documented in
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+data+structures+in+Zookeeper
If a controller fails, any live broker can become the leader. This is
coordinated through an ephemeral path in ZK.
Thanks,
Jun
On Fri, Aug 23, 2013 at 3:37 AM, James Wu wrot
Ah, one thing to be aware of is that the effectiveness of compression is
directly related to the producer batch size--more batching, more
compression. So even if you use compression on both clusters the mirror may
be much smaller.
-jay
On Friday, August 23, 2013, Rajasekar Elango wrote:
> Thanks
Thanks Guazhang, Jun,
Yes we doing gzip compression and that should be reason for difference in
disk usage. I had a typo that the size is actually 91G in source cluster.So
25G/91G ratio makes sense for compression.
Thanks,
Raja.
On Thu, Aug 22, 2013 at 7:00 PM, Guozhang Wang wrote:
> When you
Thanks Jun,
What trouble shooting steps can we do to identify if bottleneck is with
consuming or producing..? Does changing anything in log4j configuration or
a jmx mbeans provide insight into it..? Does Metadata refresh interval
affect picking up new partitions for only existing topic or it affec
Jun,
Note that the puppet module README I liked to isn't a full jmxtrans example
JSON query. It is a jmxtrans puppet module usage example. So, using that in
puppet will generate a .json file containing the query.
We'll be moving to 0.8 in the next coming months, and I'll try to get some real
Hi Neha,
"Wipe out the cluster" Do you mean you uninstall the cluster and reinstall it?
Or you just delete all kafka data and zookeeper data for the cluster?
This is not a blocking issue for us. Our blocking issue is that some broker
will
fall out of ISR and never get back to it. We have observe
Can you tell us how to reproduce this behavior?
Thanks,
Neha
On Aug 23, 2013 4:44 AM, "Wu, Jiang2" wrote:
> We are using high-level API, not the SimpleConsumer.
>
> Thanks,
> Jiang
>
> -Original Message-
> From: Jun Rao [mailto:jun...@gmail.com]
> Sent: Friday, August 23, 2013 12:13 AM
>
We are using high-level API, not the SimpleConsumer.
Thanks,
Jiang
-Original Message-
From: Jun Rao [mailto:jun...@gmail.com]
Sent: Friday, August 23, 2013 12:13 AM
To: users@kafka.apache.org
Subject: Re: repeated for request offset out of range
Are you using the ZK-based consumer or Sim
Thank you Neha, it's very helpful information!
I also read this article
http://engineering.linkedin.com/kafka/intra-cluster-replication-apache-kafka
For the section "Handling Failures", I am wondering some questions:
1. "The leader and the ISR for each partition are also stored in Zookeeper
and a
Hi Phillip,
If I can assume that all messages within a single partition are ordered the
same as delivery order, the state management to eliminate duplicates is far
simpler.
I am using Kafka as the infrastructure for a streaming map/reduce style
solution, where throughput is critical.
Events are s
54 matches
Mail list logo