HI Jun,
Can we also include KAFKA-2448 in 0.9 as well? We see this issue a few
times before and that cause replica fetcher threads not startup.
Thanks,
Jiangjie (Becket) Qin
On Sat, Sep 12, 2015 at 9:40 AM, Jun Rao wrote:
> The following is a candidate list of jiras that we want to complete i
Actually Kafka only support two mutually exclusive log cleanup policy: 1)
delete logs after retention period passed. 2) compact the log to only keep
the last updated value of a key.
log.retention.hours is only used by (1). For log compaction, currently it
is not compacting the logs by its age, but
You are looking for a K-V store here, so the general answer is no. But
Kafka does have an internal K-V store but only for consumer offsets. So
there are some tricks we can play:
If your processing node is consuming from Kafka and the offset of a
snapshot is the offset of a Kafka partition, you can
That's not true Shaninder. Assuming you are using the high level consumer,
if all the consumers have different group id, each of them will get the all
the messages.
Also, another way to think about this a little bit is that maybe you can
produce the same data to many different topics each have one
Yes. That is correct. New consumer will only store offsets in Kafka.
On Tue, Jul 21, 2015 at 8:57 AM, Stevo Slavić wrote:
> Hello Apache Kafka community,
>
> It seems new high level consumer coming in 0.8.3 will support only offset
> storage in Kafka topic.
> Can somebody please confirm/comment?
Since Kafka's performance largely depends on the operating system page
cache. So usually people want to use dedicated machines for Kafka. But that
depends on your performance requirement as well. If you only have 5 million
messages per month, I think letting Kafka share the machines with other
syst
This looks an issue to be fixed. I created KAFKA-2385 for this.
Thanks,
Jiangjie (Becket) Qin
On Wed, Jul 29, 2015 at 10:33 AM, Chris Barlock wrote:
> I'm a user of Kafka/ZooKeeper not one of its developers, so I can't give
> you a technical explanation. I do agree that Kafka should ship the
Mirror Maker does not have specific restrictions on cluster size.
The error you saw was because consumer was not able to talk to the broker.
Can you try to use kafka-console-consumer to consume some data from your
source cluster and see if it works? It should be under KAFKA_HOME/bin/
Jiangjie (Be
I think the rough calculation of max memory footprint for each high level
consumer would be:
(Number Of Partitions For All Topics) * fetch.message.max.bytes *
queued.max.message.chunks + (some decompression memory cost for a message)
In your case, it would be 10 times above.
Thanks,
Jiangjie (B
Hey Stefan,
I think if you follow the one-commit-per-message approach, you will be
able to achieve exact once semantic. However this would be very expensive
and also everything has to be synchronous in order to make it work.
In that sense, de-duplication on client side seems a more favorable opti
AhŠ It seems you are more focusing on producer side workload balanceŠ If
that is the case, please ignore my previous comments.
Jiangjie (Becket) Qin
On 7/15/15, 6:01 PM, "Jiangjie Qin" wrote:
>If you have pretty balanced traffic on each partition and have set
>auto.leader.reba
If you have pretty balanced traffic on each partition and have set
auto.leader.rebalance.enabled to true or false, you might not need to do
further workload balance.
However, in most cases you probably still need to do some sort of load
balancing based on the traffic and disk utilization of each b
If that is the case, I guess that might still be some value to try to run
broker and clients locally and see if the issue still exist.
Thanks,
Jiangjie (Becket) Qin
On 7/15/15, 1:23 PM, "Vadim Bobrov" wrote:
>it is pretty random
>
>On Wed, Jul 15, 2015 at 4:22 PM, J
at since gone forever
>
>On Wed, Jul 15, 2015 at 3:43 PM, Jiangjie Qin
>wrote:
>
>> Is there anything on the broker log?
>> Is it possible that your client and broker are not running on the same
>> version?
>>
>> Jiangjie (Becket) Qin
>>
>> On
committed and report back
>>
>> On Wed, Jul 15, 2015 at 1:33 PM, Joel Koshy wrote:
>>
>>> - You can also change the log4j level dynamically via the
>>> kafka.Log4jController mbean.
>>> - You can also look at offset commit request metrics (mbeans) on the
>&
eeze out any logging from kafka. Logging
>for
>all other libs (like zookeeper e.g.) work perfectly. Am I doing something
>wrong?
>
>
>On Tue, Jul 14, 2015 at 6:55 PM, Jiangjie Qin
>wrote:
>
>> Hi Vadim,
>>
>> Can you turn on trace level logging on your co
It looks kafka.admin.ConsumerGroupCommand class is what you need.
Jiangjie (Becket) Qin
On 7/14/15, 8:23 PM, "Swati Suman" wrote:
>Hi Team,
>
>Currently, I am able to fetch the Topic,Partition,Leader,Log Size through
>TopicMetadataRequest API available in Kafka.
>
>Is there any java api that gi
Hi Vadim,
Can you turn on trace level logging on your consumer and search for
"offset commit response² in the log?
Also maybe take a look at the log to see if there is any exception thrown.
Thanks,
Jiangjie (Becket) Qin
On 7/14/15, 11:06 AM, "Vadim Bobrov" wrote:
>just caught this error agai
Hi Tao,
We see this error from time to time but did not think of this as a big
issue. Any reason it bothers you much?
I¹m not sure if throwing exception to user on this exception is a good
handling or not. What are user supposed to do in that case other than
retry?
Thanks,
Jiangjie (Becket) Qin
Yes, we are going to use new consumer after it is ready.
Jiangjie (Becket) Qin
On 7/12/15, 8:21 PM, "tao xiao" wrote:
>Hi team,
>
>The trunk code of mirror maker now uses the old consumer API, Is there any
>plan to use new Java consumer api in mirror maker?
Hi Stevo,
Kafka does not support customized topic metadata. What we are doing now is
having topic metadata store in a separate zookeeper structure. But I agree
this is probably a good candidate feature to consider.
Thanks,
Jiangjie (Becket) Qin
On 7/12/15, 4:16 PM, "Stevo Slavić" wrote:
>Hell
You can take a look at Burrow. We use it in LinkedIn.
Thanks,
Jiangjie (Becket) Qin
On 7/9/15, 8:53 PM, "Anandh Kumar" wrote:
>Hi
>
>We have any monitor tool which monitor kafka offset?
>
>I need some opensource admin tool for kafka.
>
>Please guide me.
>
>Regards,
>-Anandh Kumar
The replica lag definition now is time based, so as long as a replica can
catch up with leader in replica.lag.time.max.ms, it is in ISR, no matter
how many messages it is behind.
And yes, your understanding is correct - ACK is sent back either when all
replica in ISR got the message or the request
:), Maybe we should put this in to Kafka FAQ?
On 7/6/15, 7:25 PM, "Gwen Shapira" wrote:
>Nice :)
>
>I always thought its a reference to the Kafkaesque process of building
>data pipelines in a large organization :)
>
>On Mon, Jul 6, 2015 at 6:52 PM, luo.fucong wrote:
>> I just found the answer
I guess it is called SimpleConsumer for a reason. Simple consumer is
really simple and does not support any failure recovery. You might need to
implement you own logic, it is probably not trivial though. As a
reference, high level consumer uses Zookeeper ephemeral path to monitor
the liveliness of
From: "Michael J. Kitchin" mailto:mcoyote...@gmail.com>>
Reply-To: "mcoy...@mcoyote.com<mailto:mcoy...@mcoyote.com>"
mailto:mcoy...@mcoyote.com>>
Date: Friday, July 3, 2015 at 3:01 PM
To: Jiangjie Qin mailto:j...@linkedin.com>>
Cc: "users@kafka.apache.
Hi Michael,
For the consumer side question. Yes, turning off auto offset commit is
what you want. But you should also commit offset manually after you have
written the processed data to somewhere else. Also, the offset is only
associated with a particular consumer group. So if you restart your
con
+1
On 7/1/15, 1:00 PM, "Gwen Shapira" wrote:
>Huge +1.
>
>I don't think there is any other project that still supports 1.6.
>
>On Wed, Jul 1, 2015 at 8:05 AM, Harsha wrote:
>> Hi,
>> During our SSL Patch KAFKA-1690. Some of the reviewers/users
>> asked for support this config
>>
Yes, we have plan to add better support for this.
Thanks,
Jiangjie (Becket) Qin
On 6/30/15, 4:23 PM, "Zaiming Shi" wrote:
>Hi Jiangjie !
>Does the word 'Currently' imply any plan in introducing timestamp in log
>entries?
>
>Regards
>/Zaiming
>On Jun
Currently Kafka only have a very coarse solution to find offset by time,
which is based on the segment last modified time.
This interface is only available in simple consumer. You may issue an
OffsetRequest to specify a timestamp. The offset returned will be the
first offset of segments whose last
27;s
>rebalancing?
>
>The new KafkaConsumer code doesn't appear to do a commit in the same way
>during rebalance, when autocommit is enabled. So if current users of the
>high level consumer switch to the new consumer, they might get more
>duplicates on rebalance, right?
>
.
>
>On Mon, Jun 22, 2015 at 5:26 AM Jiangjie Qin
>wrote:
>
>> Yes, we can expose a user callback in MM, just like we did for rebalance
>> listener.
>> I still think ErrorLoggingCallback needs some change, though. Can we
>>only
>> store the value bytes when
nsumer thread: loop on consume -> process -> commit offset every N
messages. So we can make sure there is no weird race condition.
Thanks,
Jiangjie (Becket) Qin
On 6/21/15, 6:23 AM, "noah" wrote:
>On Sun, Jun 21, 2015 at 1:10 AM Jiangjie Qin
>wrote:
>
>> Hey Noah,
>
t;>
>> On Wed, Jun 17, 2015 at 1:06 AM, tao xiao wrote:
>>
>> > Thank you for the reply.
>> >
>> > Patch submitted https://issues.apache.org/jira/browse/KAFKA-2281
>> >
>> > On Mon, 15 Jun 2015 at 02:16 Jiangjie Qin
>> > wrote:
>&
Hey Noah,
Carl is right about the offset. The offset to be commit should be the
largest-consumed-offset + 1. But this should not break the at least once
guarantee.
>From what I can see, your consumer should not skip messages. Just to make
sure I understand your test correctly,
1. There is a consum
Hi Carl,
Generally, you approach works to guarantee at least once consumption -
basically people have to commit offset only after they have processed the
message.
The only problem is that in old high level consumer, during consumer
rebalance consumer will (and should) commit offsets. To guarantee
It seems that your log.index.size.max.bytes was 1K and probably was too
small. This will cause your index file to reach its upper limit before
fully index the log segment.
Jiangjie (Becket) Qin
On 6/18/15, 4:52 PM, "Zakee" wrote:
>Any ideas on why one of the brokers which was down for a day, fa
Hi Olof,
I am just wondering what is the benefit of rebalancing with minimal number
of reassignments here?
I am asking this because in new consumer, the rebalance actually is quite
cheap on the consumer side - just updating a topic partition set. That
means the actually rebalance time on consumer
Hi Tao,
Yes, the issue that ErrorLoggingCallback keeps value as local variable is
known for a while and we probably should fix it as the value is not used
except logging the its size. Can you open a ticket and maybe also submit a
patch?
For unreachable objects I don¹t think it is memory leak. As
Which version of MM are you running?
On 6/9/15, 4:49 AM, "tao xiao" wrote:
>Hi,
>
>I have two mirror makers A and B both subscripting to the same whitelist.
>During topic rebalancing one of the mirror maker A encountered
>ZkNoNodeException and then stopped all connections. but mirror maker B
>di
PM, Jay Kreps
mailto:jay.kr...@gmail.com>> wrote:
Could it also be that the log cleaner is running? This will definitely use
some CPU while the cleaning is occurring (it would attempt to use one cpu
per log cleaner thread you configure).
-Jay
On Mon, Jun 8, 2015 at 1:07 PM, Jiangjie Qin
r
Seems to be LogCleaner and FetchRequest threads.
Attached is the profiling screenshot
On Fri, Jun 5, 2015 at 3:06 PM, Jiangjie Qin
mailto:j...@linkedin.com.invalid>> wrote:
Has this to do with KAFKA-1461?
Can you see which thread is taking a lot of cpu? Some jconsole plugin ca
eans while sending messages ??*
>
>On Mon, Jun 8, 2015 at 10:46 PM, Jiangjie Qin
>wrote:
>
>> What replication factor are you using?
>>
>> Currently if a partition is offline, the message in producer will not be
>> sent but sit in accumulator until the partition
KIP-19 should address this issue.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-19+-+Add+a+request+ti
meout+to+NetworkClient
Thanks,
Jiangjie (Becket) Qin
On 6/8/15, 10:44 AM, "Christofer Hedbrandh" wrote:
>I think this question might relate to the very recently posted "callback
>hand
That is offset out of range error. So you are fetching from an offset
either greater than latest offset or smaller than earliest offset
available on broker.
Jiangjie (Becket) Qin
On 6/8/15, 2:21 AM, "sunil kalva" wrote:
>Hi
>For few topics i always get FetchResponse.error code as "1", i am usin
What replication factor are you using?
Currently if a partition is offline, the message in producer will not be
sent but sit in accumulator until the partition comes back online. Do you
mean you want to use the message send callback to detect broker failure?
Jiangjie (Becket) Qin
On 6/8/15, 12:2
Has this to do with KAFKA-1461?
Can you see which thread is taking a lot of cpu? Some jconsole plugin can
get that information.
Jiangjie (Becket) Qin
On 6/5/15, 2:57 PM, "pundlik.anuja" wrote:
>Hi Jay,
>
>Good to hear from you. I met you at the kafka meetup at linkedin.
>
>- No, I am running ka
ucer.ProducerConfig
>val prodConfig : ProducerConfig = new ProducerConfig(properties)
>val producer : Producer[Integer,String] = new
>Producer[Integer,String](prodConfig)
>
>How can I know which producer I am using? And what is the behavior of the
>new producer?
>
>Thanks,
>Sébast
Are you using new producer or old producer?
The old producer has 10 min sticky partition behavior while the new
producer does not.
Thanks,
Jiangjie (Becket) Qin
On 6/2/15, 11:58 PM, "Sebastien Falquier"
wrote:
>Hi Jason,
>
>The default partitioner does not make the job since my producers hav
;>
Date: Thursday, May 28, 2015 at 1:09 PM
To: Jiangjie Qin mailto:j...@linkedin.com>>
Cc: "users@kafka.apache.org<mailto:users@kafka.apache.org>"
mailto:users@kafka.apache.org>>
Subject: Re: Kafka Not Commiting Messages
Hi Jiangjie,
Thanks for you message. Unf
Auto.offset.reset only comes into place when
1. the consumer fetches message from offset out of range - available
offsets on broker side
2. The consumer group has no offset committed.
So in your case, I guess high level consumer has already committed its
offset before and that offset is not out of
It should work, but usually we prefer the server version to be not lower
than client version.
On 5/27/15, 3:12 PM, "Zhuo Liu" wrote:
>Dear all,
>
>
>In 0.8.2.1 Kafka, there is new Producer API (KafkaProducer etc.).
>
>My question is: will 0.8.2.1 new Producer API
>
>be able to successfully talk
Can you turn on TRACE level logging for kafka-request.log and see if
broker received the producer request or not?
You can go to KAKFA_FOLDER/config/log4j.properties and change
log4j.logger.kafka.network.RequestChannels to TRACE.
Jiangjie (Becket) Qin
On 5/27/15, 12:12 PM, "Charlie Mason" wrote:
This should be just a message fetch failure. The socket was disconnected
when broker was writing to it. There should not be data loss.
Jiangjie (Becket) Qin
On 5/27/15, 11:00 AM, "Andrey Yegorov" wrote:
>I've noticed a few exceptions in the logs like the one below, does it
>indicate data loss?
Hmm, which Kafka version are you running? From the code it looks we should
have already ignored the deletion of a topic if it does not exist.
Jiangjie (Becket) Qin
On 5/18/15, 10:15 AM, "Dillian Murphey" wrote:
>If a broker doesn't have the topic, and I run delete topic, that topic
>will
>be in
The default range partition assignment algorithm will assign partition on
per topic basis. If you have more consumer threads than number of
partitions in a topic, some threads will not be assigned any partition.
If you are consuming from multiple topics, You might want to set the
partition.assignme
node that had the
>NotLeaderForPartitionException
>is the leader of the topic
>
>On Thu, May 14, 2015 at 6:12 AM, Jiangjie Qin
>wrote:
>
>> Does this topic exist in Zookeeper?
>>
>> On 5/12/15, 11:35 PM, "tao xiao" wrote:
>>
>> >Hi,
>&
Yes, in old producer we don¹t control the compressed message size. In new
producer, we estimate the compressed size heuristically and decide whether
to close the batch or not. It is not perfect but at least better than the
old one.
Jiangjie (Becket) Qin
On 5/13/15, 4:00 PM, "Jamie X" wrote:
>Ji
Thanks for sharing this, Garry. I actually did similar tests before but
unfortunately lost the test data because my laptop rebooted and I forgot
to save the dataŠ
Anyway, several things to verify:
1. Remember KafkaProducer holds lock per partition. So if you have only
one partition in the target
Automatic preferred leader election hasn¹t been turned on in 0.8.1.1. It¹s
been turned on in latest trunk though.
The config name is ³auto.leader.rebalance.enable".
Jiangjie (Becket) Qin
On 5/13/15, 10:50 AM, "Stephen Armstrong"
wrote:
>Does anyone have any insight into this? Am I correct that
If you are sending in sync mode, producer will just group by partition the
list of messages you provided as argument of send() and send them out. You
don¹t need to worry about batch.num.messages.
There is a potential that compressed message is even bigger than
uncompressed message, though. I¹m not
e very rare.
>> > Say, the application is rebooted and the Kafka brokers registered in
>>the
>> > producer are not reachable ( could be due to network issues or those
>> > brokers are actually down ). Since, no metadata is available the send
>> will
>> >
Does this topic exist in Zookeeper?
On 5/12/15, 11:35 PM, "tao xiao" wrote:
>Hi,
>
>Any updates on this issue? I keep seeing this issue happening over and
>over
>again
>
>On Thu, May 7, 2015 at 7:28 PM, tao xiao wrote:
>
>> Hi team,
>>
>> I have a 12 nodes cluster that has 800 topics and each o
Mayuresh, this is about the old producer instead of the new Java producer.
Jamie,
In the old producer, if you use sync mode, the list of message will be
sent as a batch. On the other hand, if you are using async mode, the
messages are just put into the queue and batched with other messages.
Notice
a producer has less meaning.
>
>Sorry if my understanding is incorrect.
>
>Rendy
>On May 13, 2015 6:59 AM, "Jiangjie Qin" wrote:
>
>> Send() will only block if the metadata is *not available* for the topic.
>> It won’t block if metadata there is stale. The meta
If you are using new Java producer, reorder could happen if
max.inflight.requests.per.connection is set to > 1 and retries are enabled
- which are both default settings.
Can you set max.in.flight.requests.per.connection to 1 and see if this
solve the issue?
Jiangjie (Becket) Qin
On 5/12/15, 12:5
e way it works I suppose is that, the producer will do
>>fetchMetadata,
>> if
>> > the last fetched metadata is stale (the refresh interval has expired)
>>or
>> if
>> > it is not able to send data to a particular broker in its current
>> meta
That¹s right. Send() will first try to get metadata of a topic, that is a
blocking operation.
On 5/12/15, 2:48 AM, "Rendy Bambang Junior"
wrote:
>Hi, sorry if my understanding is incorrect.
>
>I am integrating kafka producer with application, when i try to shutdown
>all kafka broker (preparing f
Dan,
Just to make sure I understand it correctly. What do you mean by different
ip -> broker mapping? Do you mean you changed your broker ip? We have
different mechanism in and producer to get the cluster information.
Consumer get all the information from Zookeeper while producer has to talk
to br
Hi Dillian,
Yeah, ELB + ASG will is pretty popular however might need further tricks
to use them for Kafka brokers.
As I replied to Chandrashekhar in another email. You can use ELB as
bootstrap.servers/metadata.broker.list to serve as client bootstrap use
case. But all the producing/consuming tr
Yes.
On 5/3/15, 10:57 AM, "Chandrashekhar Kotekar"
wrote:
>Thanks a lot for the information. So basically each broker needs to have
>his own public IP address?
>
>
>Regards,
>Chandrash3khar Kotekar
>Mobile - +91 8600011455
>
>On Sun, May 3, 2015 at 11:09 P
What do you mean by cluster mode with 3 Zookeeper and 3 Kafka brokers? Do
you mean 1 Zookeeper and 3 brokers?
On 5/2/15, 11:01 PM, "Kamal C" wrote:
>Any comments on this issue?
>
>On Sat, May 2, 2015 at 9:16 AM, Kamal C wrote:
>
>> Hi,
>> We are using Kafka_2.10-0.8.2.0, new Kafka producer and
You can use ELB address as metadata broker list. But you still need to
allow direct traffic from clients to a brokers because clients need to
talk to a specific broker who has the interested partition.
Jiangjie (Becket) Qin
On 5/2/15, 11:10 AM, "Chandrashekhar Kotekar"
wrote:
>Hi,
>
>I am new t
Which mirror maker version did you look at? The MirrorMaker in trunk
should not have data loss if you just use the default setting.
On 4/30/15, 7:53 PM, "Joong Lee" wrote:
>Hi,
>We are exploring Kafka to keep two data centers (primary and DR) running
>hosts of elastic search nodes in sync. One k
Roshan,
If I understand correctly, you just want to make sure a number of messages
has been sent successfully. Using callback might be easier to do so.
Public class MyCallback implements Callback {
public Set failedSend;
@Override
Public void onCompletion(RecordMetadata me
OK, so you turned off auto.offset.commit, and set the auto.offset.reset to
largest.
That means when you consume,
1. If you did not commit offsets manually, no offsets will be committed to
Kafka.
2. If you do not have an offset stored in Kafka, you will start from the
log end and ignore the existin
Yes, if you set the offset storage to Kafka, high level consumer will be
using Kafka for all offset related operations.
Jiangjie (Becket) Qin
On 4/27/15, 7:03 PM, "Gomathivinayagam Muthuvinayagam"
wrote:
>I am trying to commit offset request in a background thread. I am able to
>commit it so f
Hi Jay,
Does o.a.k.clients.tools.ProducerPerformance provide multi-thread test? I
did not find it.
I tweaked the test a little bit to make it multi-threaded and what I found
is that in a single thread case, with each message of 10 bytes, single
caller thread has ~2M messages/second throughput. 2
Hi Tao,
KAFKA-2150 has been filed.
Jiangjie
On 4/24/15, 12:38 PM, "tao xiao" wrote:
>Hi team,
>
>I observed java.lang.IllegalMonitorStateException thrown
>from AbstractFetcherThread in mirror maker when it is trying to build the
>fetchrequst. Below is the error
>
>[2015-04-23 16:16:02,049] ERR
lier this morning. Do the other
>brokers have to be bounced after the controller situation is fixed? I did
>not do that for all brokers.
>
>Wes
> On Apr 21, 2015 8:25 PM, "Jiangjie Qin"
>wrote:
>
>> Yes, should be broker 25 thread 0 from the log.
>> This ne
n 116's replica set is: [4, 7,
25]. /brokers/topics/click_engage/partitions/116/state says the leader is 28
and the ISR is [28, 15]. Does this need to be resolved, and if so how?
Thanks,
Wes
[cid:part1.03010908.07060808@chartbeat.com]
Jiangjie Qin<mailto:j...@linkedin.com.INVALID>
your consumer offsets in kafka itself instead of Zk which
>>further
>> brings down write load on ZKs.
>>
>> Regards,
>> Pushkar
>>
>> On Tue, Apr 21, 2015 at 1:13 PM, Jiangjie Qin
>>
>> wrote:
>>
>> > 2 partitions should be OK.
Apis.handleFetchRequest(KafkaApis.scala:424)
at kafka.server.KafkaApis.handle(KafkaApis.scala:186)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:42)
at java.lang.Thread.run(Thread.java:745)
What does this mean?
Thanks!
Wes
[cid:part1.08040305.06010608@chartbeat.com]
Jiangjie Qin&l
Those .index files are for different partitions and
they should be generated if new replicas is assigned to the broker.
We might want to know what caused the UnknownException. Did you see any
error log on broker 28?
Jiangjie (Becket) Qin
On 4/21/15, 9:16 AM, "Wes Chow" wrote
2 partitions should be OK.
On 4/21/15, 12:33 AM, "Achanta Vamsi Subhash"
wrote:
>We are planning to have ~2 partitions. Will it be a bottleneck?
>
>On Mon, Apr 20, 2015 at 10:48 PM, Jiangjie Qin
>wrote:
>
>> Producers usually do not query zookeeper at
Producers usually do not query zookeeper at all.
Consumers usually query zookeeper at beginning or rebalance. It is
supposed to be in frequent if you don¹t have consumers come and go all the
time. One exception is that if you are using zookeeper based consumer
offset commit, it will commit offset t
It seems there are many different symptoms you see...
Maybe we can start from leader flapping issue. Any findings in controller
log?
Jiangjie (Becket) Qin
On 4/16/15, 12:09 PM, "Kyle Banker" wrote:
>Hi,
>
>I've run into a pretty serious production issue with Kafka 0.8.2, and I'm
>wondering w
Hey Evan,
Is this issue only observed when mirror maker is consuming? It looks that
for Cluster A you have some other consumers.
Do you mean if you stop mirror maker the problem goes away?
Jiangjie (Becket) Qin
On 4/14/15, 6:55 AM, "Evan Huus" wrote:
>Any ideas on this? It's still occurring...
cting to 192.168.1.201:2181
>Welcome to ZooKeeper!
>JLine support is disabled
>
>WATCHER::
>
>WatchedEvent state:SyncConnected type:None path:null
>ls /config/topics
>[my-control]
>ls /consumers
>[]
>quit
>
>
>Thanks.
>Mayank.
>
>On Fri, Mar 20, 2015 at
A quick reference.
http://www.slideshare.net/junrao/kafka-replication-apachecon2013
On 4/12/15, 11:36 PM, "Bill Hastings" wrote:
>Hi Guys
>
>How do topics get assigned to brokers? I mean if I were to create a topic
>X
>and publish to it how does Kafka assign the topic and the message to a
>part
og to Kafka's producer?
>
>Thanks
>Liang
>
>-----Original Message-
>From: Jiangjie Qin [mailto:j...@linkedin.com.INVALID]
>Sent: Monday, April 06, 2015 11:46 AM
>To: users@kafka.apache.org
>Subject: Re: question about Kafka
>
>Hey Liang,
>
>
Also if you are using Kafka from the latest trunk, KafkaProducer has a
flush() interface that you may call. This will ensure all the message
previously sent from send() methods are sent to Kafka server.
On 4/3/15, 3:38 PM, "Sun, Joey" wrote:
>Hello, group
>
>I am a newbie to Kafka. I am research
Hey Liang,
Have you looked at the quick start here:
https://kafka.apache.org/documentation.html#quickstart
In Kafka, on the producer side, there is no concept of ³commit². If you
are producing using KafkaProducer, you can do a send.get(), this is a
synchronized send so if no exception was thrown,
This sounds a very serious issueŠ Could you provide the controller log and
the log for the first broker on which you tried controlled shutdown and
upgrade?
On 4/3/15, 8:57 AM, "Jason Rosenberg" wrote:
>I'm preparing a longer post here, but we recently ran into a similar
>scenario. Not sure yet
Another thing is that the active log segment would not be deleted, so if
there is only one log segment, it will not honor the retention policy. You
may config log.roll.ms to make sure you have a rolled over log segment.
On 4/3/15, 5:20 AM, "Madhukar Bharti" wrote:
>Hi Gaurav,
>
>What is your "lo
Yes, KafkaConsumer in 0.8.2 is still in development. You probably still
want to use ZookeeperConsumerConnector for now.
On 4/1/15, 9:28 AM, "Mark Zang" wrote:
>I found the 0.8.2.0 and 0.8.2.1 has a KafkaConsumer. But this class seems
>not completed and not functional. Lots of method returns null
The Mirror maker in trunk now supports mirroring with preserved partition.
You can wire in a message handler to assign partitions for each producer
record before handing them to producer.
Jiangjie (Becket) Qin
On 3/31/15, 3:41 AM, "Ivan Balashov" wrote:
>Alex,
>
>Just wondering, did you have an
The async send() put the message into a message queue then returns. When
the messages are pulled out of the queue by the sender thread, it still
uses SyncProducer to send ProducerRequests to brokers.
Jiangjie (Becket) Qin
On 3/30/15, 10:44 PM, "Madhukar Bharti" wrote:
>Hi All,
>
>I am using *as
Hi,
KafkaConsumer is still under development and not ready for wide use case.
Currently, it can be used to replace SimpleConsumer (low level consumer),
but can not replace ZookeeperConsumerConnector(high level consumer). So if
you need to use simple consumer, I would suggest using KafkaConsumer
in
If you want to do a seamless migration. I think a better way is to build a
cross datacenter Kafka cluster temporarily. So the process is:
1. Add several new Kafka brokers in your new datacenter and add them to
the old cluster.
2. Use replica assignment tool to reassign all the partitions to brokers
1 - 100 of 172 matches
Mail list logo