Re: reduce replication factor

2014-05-21 Thread David Birdsong
On Wed, May 21, 2014 at 9:06 PM, Jun Rao  wrote:

> Expansion can be done by following
> http://kafka.apache.org/documentation.html#basic_ops_cluster_expansion
>
> If you just want to free up a server, you can stop the broker there and
> start a broker using the same broker id on a new server. Data should be
> automatically replicated to the new server.
>
>
The thing is, I don't want to fully free up the server, I just want it to
stop hosting all the topic,partition sets it originally did.

Here's the flow:

broker_1 hosts topic,{1,2,3,4}
over time load for the partitions overwhelms broker_1

spin up broker_2
migrate topic,{3,4} to broker_2 successfully

broker_1 is only useful if partitions topic,{3,4} are dropped by broker_1

how does one get broker_1 to disassociate, drop, forget, evict...(not sure
the verb) topic,{3,4} and let broker_2 own it.

Thanks,
>
> Jun
>
>
> On Wed, May 21, 2014 at 3:23 PM, David Birdsong  >wrote:
>
> > Any suggestions? I'm kind of in a bind in that I don't understand how to
> > grow the cluster when more capacity is needed--which happens to be right
> > now for me.
> >
> > The only thing I can think that might work is to create new brokers,
> > cherry-pick topic/partition pairs and move them, then turn off the old
> ones
> > and forever retire their IDs freeing up my old hardware to come back
> online
> > as a new kafka broker ID.
> >
> >
> > On Wed, May 21, 2014 at 9:16 AM, David Birdsong <
> david.birds...@gmail.com
> > >wrote:
> >
> > >
> > >
> > >
> > > On Wed, May 21, 2014 at 9:11 AM, David Birdsong <
> > david.birds...@gmail.com>wrote:
> > >
> > >> Here's the reassignment json and current "describe" output:
> > >> https://gist.github.com/davidbirdsong/32cd0c4f49496a6a32e5
> > >>
> > >>
> > >> In my re-assignment json, I tried to re-assign to 2 when the repl is
> set
> > >> to 3. Once I noticed the the completely new node "133" had appeared in
> > the
> > >> ISR, I tried stopping 224, wiping kafka completely and then brought
> 224
> > >> back up as 224 again. It promptly replicated the topic, but never
> joined
> > >> the ISR.
> > >>
> > >
> > > to be clear, i don't want it to join the ISR. i'm curious how to make
> 224
> > > forget about the partitions for that topic since i have other plans for
> > it.
> > >
> > >
> > >>
> > >> How does one move a replica? This is exactly what I'm trying to do.
> > >>
> > >> My pattern is a common one. I started with a set of 3 kafka brokers.
> The
> > >> load and space is overwhelming them. I'm trying to add new brokers and
> > >> spread the partitions to new nodes while removing some of the
> > partitions on
> > >> the old nodes so as to make room. It's the latter that I don't get how
> > to
> > >> do.
> > >>
> > >> I've conflated two issues here mostly due to needing to get this
> cluster
> > >> stable again.
> > >> - reduce replication
> > >> - remove a partition from a broker, ie. remove the replica
> > >>
> > >> they're very distinct actions, but both would help me in the moment
> > >>
> > >>
> > >>
> > >>
> > >> On Wed, May 21, 2014 at 8:56 AM, Jun Rao  wrote:
> > >>
> > >>> During the re-assignment, did you move the replica off the old
> broker?
> > >>>
> > >>> Thanks,
> > >>>
> > >>> Jun
> > >>>
> > >>>
> > >>> On Wed, May 21, 2014 at 8:21 AM, David Birdsong <
> > >>> david.birds...@gmail.com>wrote:
> > >>>
> > >>> > I did that and so now the topic has 4 replicas for a repl count of
> 3,
> > >>> but
> > >>> > only the 'new' replicas exist in the ISR.
> > >>> >
> > >>> > The old broker that I want to clear disk space and generally free
> up
> > >>> > resources has fully synced a topic that I want to disassociate from
> > it.
> > >>> >
> > >>> > Is there a way to do this?
> > >>> >
> > >>> >
> > >>> > On Wed, May 21, 2014 at 7:48 AM, Jun Rao  wrote:
> > >>> >
> > >>> > > We don't have an exact tool for doing this. You may be able to do
> > >>> that
> > >>> > > through
> > >>> > >
> > >>> > >
> > >>> >
> > >>>
> >
> http://kafka.apache.org/documentation.html#basic_ops_increase_replication_factorby
> > >>> > > specifying fewer replicas.
> > >>> > >
> > >>> > > Thanks,
> > >>> > >
> > >>> > > Jun
> > >>> > >
> > >>> > >
> > >>> > > On Wed, May 21, 2014 at 1:23 AM, David Birdsong <
> > >>> > david.birds...@gmail.com
> > >>> > > >wrote:
> > >>> > >
> > >>> > > > Is there a way to reduce the replication count? I'm trying to
> > >>> spread
> > >>> > > > existing partitions across more brokers, but it's hard to
> decomm
> > a
> > >>> > > broker.
> > >>> > > > Reducing repl count would suffice for now.
> > >>> > > >
> > >>> > > > Any tips?
> > >>> > > >
> > >>> > > > I'm running a mix of 0.8.1.1 and 0.8.1 (I'm upgrading now.)
> > >>> > > >
> > >>> > >
> > >>> >
> > >>>
> > >>
> > >>
> > >
> >
>


Re: reduce replication factor

2014-05-21 Thread Jun Rao
Expansion can be done by following
http://kafka.apache.org/documentation.html#basic_ops_cluster_expansion

If you just want to free up a server, you can stop the broker there and
start a broker using the same broker id on a new server. Data should be
automatically replicated to the new server.

Thanks,

Jun


On Wed, May 21, 2014 at 3:23 PM, David Birdsong wrote:

> Any suggestions? I'm kind of in a bind in that I don't understand how to
> grow the cluster when more capacity is needed--which happens to be right
> now for me.
>
> The only thing I can think that might work is to create new brokers,
> cherry-pick topic/partition pairs and move them, then turn off the old ones
> and forever retire their IDs freeing up my old hardware to come back online
> as a new kafka broker ID.
>
>
> On Wed, May 21, 2014 at 9:16 AM, David Birdsong  >wrote:
>
> >
> >
> >
> > On Wed, May 21, 2014 at 9:11 AM, David Birdsong <
> david.birds...@gmail.com>wrote:
> >
> >> Here's the reassignment json and current "describe" output:
> >> https://gist.github.com/davidbirdsong/32cd0c4f49496a6a32e5
> >>
> >>
> >> In my re-assignment json, I tried to re-assign to 2 when the repl is set
> >> to 3. Once I noticed the the completely new node "133" had appeared in
> the
> >> ISR, I tried stopping 224, wiping kafka completely and then brought 224
> >> back up as 224 again. It promptly replicated the topic, but never joined
> >> the ISR.
> >>
> >
> > to be clear, i don't want it to join the ISR. i'm curious how to make 224
> > forget about the partitions for that topic since i have other plans for
> it.
> >
> >
> >>
> >> How does one move a replica? This is exactly what I'm trying to do.
> >>
> >> My pattern is a common one. I started with a set of 3 kafka brokers. The
> >> load and space is overwhelming them. I'm trying to add new brokers and
> >> spread the partitions to new nodes while removing some of the
> partitions on
> >> the old nodes so as to make room. It's the latter that I don't get how
> to
> >> do.
> >>
> >> I've conflated two issues here mostly due to needing to get this cluster
> >> stable again.
> >> - reduce replication
> >> - remove a partition from a broker, ie. remove the replica
> >>
> >> they're very distinct actions, but both would help me in the moment
> >>
> >>
> >>
> >>
> >> On Wed, May 21, 2014 at 8:56 AM, Jun Rao  wrote:
> >>
> >>> During the re-assignment, did you move the replica off the old broker?
> >>>
> >>> Thanks,
> >>>
> >>> Jun
> >>>
> >>>
> >>> On Wed, May 21, 2014 at 8:21 AM, David Birdsong <
> >>> david.birds...@gmail.com>wrote:
> >>>
> >>> > I did that and so now the topic has 4 replicas for a repl count of 3,
> >>> but
> >>> > only the 'new' replicas exist in the ISR.
> >>> >
> >>> > The old broker that I want to clear disk space and generally free up
> >>> > resources has fully synced a topic that I want to disassociate from
> it.
> >>> >
> >>> > Is there a way to do this?
> >>> >
> >>> >
> >>> > On Wed, May 21, 2014 at 7:48 AM, Jun Rao  wrote:
> >>> >
> >>> > > We don't have an exact tool for doing this. You may be able to do
> >>> that
> >>> > > through
> >>> > >
> >>> > >
> >>> >
> >>>
> http://kafka.apache.org/documentation.html#basic_ops_increase_replication_factorby
> >>> > > specifying fewer replicas.
> >>> > >
> >>> > > Thanks,
> >>> > >
> >>> > > Jun
> >>> > >
> >>> > >
> >>> > > On Wed, May 21, 2014 at 1:23 AM, David Birdsong <
> >>> > david.birds...@gmail.com
> >>> > > >wrote:
> >>> > >
> >>> > > > Is there a way to reduce the replication count? I'm trying to
> >>> spread
> >>> > > > existing partitions across more brokers, but it's hard to decomm
> a
> >>> > > broker.
> >>> > > > Reducing repl count would suffice for now.
> >>> > > >
> >>> > > > Any tips?
> >>> > > >
> >>> > > > I'm running a mix of 0.8.1.1 and 0.8.1 (I'm upgrading now.)
> >>> > > >
> >>> > >
> >>> >
> >>>
> >>
> >>
> >
>


Re: reduce replication factor

2014-05-21 Thread David Birdsong
Any suggestions? I'm kind of in a bind in that I don't understand how to
grow the cluster when more capacity is needed--which happens to be right
now for me.

The only thing I can think that might work is to create new brokers,
cherry-pick topic/partition pairs and move them, then turn off the old ones
and forever retire their IDs freeing up my old hardware to come back online
as a new kafka broker ID.


On Wed, May 21, 2014 at 9:16 AM, David Birdsong wrote:

>
>
>
> On Wed, May 21, 2014 at 9:11 AM, David Birdsong 
> wrote:
>
>> Here's the reassignment json and current "describe" output:
>> https://gist.github.com/davidbirdsong/32cd0c4f49496a6a32e5
>>
>>
>> In my re-assignment json, I tried to re-assign to 2 when the repl is set
>> to 3. Once I noticed the the completely new node "133" had appeared in the
>> ISR, I tried stopping 224, wiping kafka completely and then brought 224
>> back up as 224 again. It promptly replicated the topic, but never joined
>> the ISR.
>>
>
> to be clear, i don't want it to join the ISR. i'm curious how to make 224
> forget about the partitions for that topic since i have other plans for it.
>
>
>>
>> How does one move a replica? This is exactly what I'm trying to do.
>>
>> My pattern is a common one. I started with a set of 3 kafka brokers. The
>> load and space is overwhelming them. I'm trying to add new brokers and
>> spread the partitions to new nodes while removing some of the partitions on
>> the old nodes so as to make room. It's the latter that I don't get how to
>> do.
>>
>> I've conflated two issues here mostly due to needing to get this cluster
>> stable again.
>> - reduce replication
>> - remove a partition from a broker, ie. remove the replica
>>
>> they're very distinct actions, but both would help me in the moment
>>
>>
>>
>>
>> On Wed, May 21, 2014 at 8:56 AM, Jun Rao  wrote:
>>
>>> During the re-assignment, did you move the replica off the old broker?
>>>
>>> Thanks,
>>>
>>> Jun
>>>
>>>
>>> On Wed, May 21, 2014 at 8:21 AM, David Birdsong <
>>> david.birds...@gmail.com>wrote:
>>>
>>> > I did that and so now the topic has 4 replicas for a repl count of 3,
>>> but
>>> > only the 'new' replicas exist in the ISR.
>>> >
>>> > The old broker that I want to clear disk space and generally free up
>>> > resources has fully synced a topic that I want to disassociate from it.
>>> >
>>> > Is there a way to do this?
>>> >
>>> >
>>> > On Wed, May 21, 2014 at 7:48 AM, Jun Rao  wrote:
>>> >
>>> > > We don't have an exact tool for doing this. You may be able to do
>>> that
>>> > > through
>>> > >
>>> > >
>>> >
>>> http://kafka.apache.org/documentation.html#basic_ops_increase_replication_factorby
>>> > > specifying fewer replicas.
>>> > >
>>> > > Thanks,
>>> > >
>>> > > Jun
>>> > >
>>> > >
>>> > > On Wed, May 21, 2014 at 1:23 AM, David Birdsong <
>>> > david.birds...@gmail.com
>>> > > >wrote:
>>> > >
>>> > > > Is there a way to reduce the replication count? I'm trying to
>>> spread
>>> > > > existing partitions across more brokers, but it's hard to decomm a
>>> > > broker.
>>> > > > Reducing repl count would suffice for now.
>>> > > >
>>> > > > Any tips?
>>> > > >
>>> > > > I'm running a mix of 0.8.1.1 and 0.8.1 (I'm upgrading now.)
>>> > > >
>>> > >
>>> >
>>>
>>
>>
>


Re: Producer throughput question

2014-05-21 Thread Joel Koshy
> 
> What is the good way to measure the producer throughput? 

The producer exposes a number of mbeans that you can poke to monitor
throughput (e.g., bytes-per-sec, messages-per-sec, etc.) and it does
so on an aggregate as well as per-topic basis.

> We also find compression (such as GZip) does not add more time to the send. 
> Is the compression done in the send thread or in a separate thread for async 
> mode?
> 

In the current producer implementation it is done in a separate thread
(i.e., separate from the calling thread). The send also happens in
that separate thread.

In the new producer implementation (under kafka.clients) the
compression happens in the calling (client thread) and the send
happens in a separate thread.

Joel



Producer throughput question

2014-05-21 Thread Maung Than
Hi All, 

What is the good way to measure the producer throughput? 

We are currently accumulating time take by each individual send in the producer 
and then dividing it with the total volume to calculate the producer 
throughput. We seems to be getting 20% more throughput than our network can 
support that is 1000 mbps. 

We also find compression (such as GZip) does not add more time to the send. Is 
the compression done in the send thread or in a separate thread for async mode?

async with batch size 1000
No of Partitions = 5
Replication factor = 3
No of brokers = 5; 
No of producers = 2; 

Thanks,
Maung


Re: Kafka replication throttling

2014-05-21 Thread Marcos Juarez Lopez
Thanks for your response Jun.

JIRA has been filed (see link below).  Please let me know if I should add
more details/context:

https://issues.apache.org/jira/browse/KAFKA-1464

Thanks,

Marcos Juarez



On Wed, May 21, 2014 at 8:40 AM, Jun Rao  wrote:

> We don't have such throttling right now. Could you file a jira?
>
> Thanks,
>
> Jun
>
>
> On Tue, May 20, 2014 at 10:29 PM, Marcos Juarez Lopez  >wrote:
>
> > Hi,
> >
> > We have several Kafka clusters in production, and we've had to reassign
> > replication a few times now in production.  Some of our topic/partitions
> > are pretty large, up to 32 partitions per topic, and 16GB per partition,
> so
> > adding a new broker and/or repairing a broker that had been down for some
> > time turns out to be a major undertaking.
> >
> > Today, when we attempt to replicate a single partition, it pegs the disk
> > IO, and uses a significant chunk of the 10Gbps interface for a good ~5
> > minutes.  This is causing problems for our downstream consumers, which
> rely
> > on having a consistent stream of realtime data being sent to them.
> >
> > Is there a way to throttle Kafka replication between nodes, so that
> instead
> > of it going full blast, it will replicate at a fixed rate in megabytes or
> > activities/batches per second?  Or maybe is this planned for a future
> > release, maybe 0.9?
> >
> > Thanks,
> >
> > Marcos Juarez
> >
>


Re: reduce replication factor

2014-05-21 Thread David Birdsong
On Wed, May 21, 2014 at 9:11 AM, David Birdsong wrote:

> Here's the reassignment json and current "describe" output:
> https://gist.github.com/davidbirdsong/32cd0c4f49496a6a32e5
>
>
> In my re-assignment json, I tried to re-assign to 2 when the repl is set
> to 3. Once I noticed the the completely new node "133" had appeared in the
> ISR, I tried stopping 224, wiping kafka completely and then brought 224
> back up as 224 again. It promptly replicated the topic, but never joined
> the ISR.
>

to be clear, i don't want it to join the ISR. i'm curious how to make 224
forget about the partitions for that topic since i have other plans for it.


>
> How does one move a replica? This is exactly what I'm trying to do.
>
> My pattern is a common one. I started with a set of 3 kafka brokers. The
> load and space is overwhelming them. I'm trying to add new brokers and
> spread the partitions to new nodes while removing some of the partitions on
> the old nodes so as to make room. It's the latter that I don't get how to
> do.
>
> I've conflated two issues here mostly due to needing to get this cluster
> stable again.
> - reduce replication
> - remove a partition from a broker, ie. remove the replica
>
> they're very distinct actions, but both would help me in the moment
>
>
>
>
> On Wed, May 21, 2014 at 8:56 AM, Jun Rao  wrote:
>
>> During the re-assignment, did you move the replica off the old broker?
>>
>> Thanks,
>>
>> Jun
>>
>>
>> On Wed, May 21, 2014 at 8:21 AM, David Birdsong > >wrote:
>>
>> > I did that and so now the topic has 4 replicas for a repl count of 3,
>> but
>> > only the 'new' replicas exist in the ISR.
>> >
>> > The old broker that I want to clear disk space and generally free up
>> > resources has fully synced a topic that I want to disassociate from it.
>> >
>> > Is there a way to do this?
>> >
>> >
>> > On Wed, May 21, 2014 at 7:48 AM, Jun Rao  wrote:
>> >
>> > > We don't have an exact tool for doing this. You may be able to do that
>> > > through
>> > >
>> > >
>> >
>> http://kafka.apache.org/documentation.html#basic_ops_increase_replication_factorby
>> > > specifying fewer replicas.
>> > >
>> > > Thanks,
>> > >
>> > > Jun
>> > >
>> > >
>> > > On Wed, May 21, 2014 at 1:23 AM, David Birdsong <
>> > david.birds...@gmail.com
>> > > >wrote:
>> > >
>> > > > Is there a way to reduce the replication count? I'm trying to spread
>> > > > existing partitions across more brokers, but it's hard to decomm a
>> > > broker.
>> > > > Reducing repl count would suffice for now.
>> > > >
>> > > > Any tips?
>> > > >
>> > > > I'm running a mix of 0.8.1.1 and 0.8.1 (I'm upgrading now.)
>> > > >
>> > >
>> >
>>
>
>


Re: reduce replication factor

2014-05-21 Thread David Birdsong
Here's the reassignment json and current "describe" output:
https://gist.github.com/davidbirdsong/32cd0c4f49496a6a32e5


In my re-assignment json, I tried to re-assign to 2 when the repl is set to
3. Once I noticed the the completely new node "133" had appeared in the
ISR, I tried stopping 224, wiping kafka completely and then brought 224
back up as 224 again. It promptly replicated the topic, but never joined
the ISR.

How does one move a replica? This is exactly what I'm trying to do.

My pattern is a common one. I started with a set of 3 kafka brokers. The
load and space is overwhelming them. I'm trying to add new brokers and
spread the partitions to new nodes while removing some of the partitions on
the old nodes so as to make room. It's the latter that I don't get how to
do.

I've conflated two issues here mostly due to needing to get this cluster
stable again.
- reduce replication
- remove a partition from a broker, ie. remove the replica

they're very distinct actions, but both would help me in the moment




On Wed, May 21, 2014 at 8:56 AM, Jun Rao  wrote:

> During the re-assignment, did you move the replica off the old broker?
>
> Thanks,
>
> Jun
>
>
> On Wed, May 21, 2014 at 8:21 AM, David Birdsong  >wrote:
>
> > I did that and so now the topic has 4 replicas for a repl count of 3, but
> > only the 'new' replicas exist in the ISR.
> >
> > The old broker that I want to clear disk space and generally free up
> > resources has fully synced a topic that I want to disassociate from it.
> >
> > Is there a way to do this?
> >
> >
> > On Wed, May 21, 2014 at 7:48 AM, Jun Rao  wrote:
> >
> > > We don't have an exact tool for doing this. You may be able to do that
> > > through
> > >
> > >
> >
> http://kafka.apache.org/documentation.html#basic_ops_increase_replication_factorby
> > > specifying fewer replicas.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Wed, May 21, 2014 at 1:23 AM, David Birdsong <
> > david.birds...@gmail.com
> > > >wrote:
> > >
> > > > Is there a way to reduce the replication count? I'm trying to spread
> > > > existing partitions across more brokers, but it's hard to decomm a
> > > broker.
> > > > Reducing repl count would suffice for now.
> > > >
> > > > Any tips?
> > > >
> > > > I'm running a mix of 0.8.1.1 and 0.8.1 (I'm upgrading now.)
> > > >
> > >
> >
>


Re: reduce replication factor

2014-05-21 Thread Jun Rao
During the re-assignment, did you move the replica off the old broker?

Thanks,

Jun


On Wed, May 21, 2014 at 8:21 AM, David Birdsong wrote:

> I did that and so now the topic has 4 replicas for a repl count of 3, but
> only the 'new' replicas exist in the ISR.
>
> The old broker that I want to clear disk space and generally free up
> resources has fully synced a topic that I want to disassociate from it.
>
> Is there a way to do this?
>
>
> On Wed, May 21, 2014 at 7:48 AM, Jun Rao  wrote:
>
> > We don't have an exact tool for doing this. You may be able to do that
> > through
> >
> >
> http://kafka.apache.org/documentation.html#basic_ops_increase_replication_factorby
> > specifying fewer replicas.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Wed, May 21, 2014 at 1:23 AM, David Birdsong <
> david.birds...@gmail.com
> > >wrote:
> >
> > > Is there a way to reduce the replication count? I'm trying to spread
> > > existing partitions across more brokers, but it's hard to decomm a
> > broker.
> > > Reducing repl count would suffice for now.
> > >
> > > Any tips?
> > >
> > > I'm running a mix of 0.8.1.1 and 0.8.1 (I'm upgrading now.)
> > >
> >
>


Re: Producer not producing msgs

2014-05-21 Thread Guozhang Wang
Do you see any exceptions from the consumer logs?

Guozhang


On Wed, May 21, 2014 at 4:15 AM, Ranjith Venkatesan
wrote:

> Hi all,
>
>   We are using kafka - 0.8 in our production setup. We have 2 kafka
> servers and 2 zookeepers. There seem to be some sort of network problem in
> our DC. As a result of this, connection refused exception in replica which
> was thrown , is shown 
> here.
> Our producer didnt throw any exception. its log is normal as shown 
> here.
> But our consumers couldnt consume from kafka.
>
> When i restart kafka servers, it worked correctly. Is there any approach
> to overcome this in future ???
>
> Any help in resolving this issue will be appreaciated.
>
> PS : Our producers having configuration  "request.required.acks=1"
>
>
>
>
>
>
>


-- 
-- Guozhang


Re: reduce replication factor

2014-05-21 Thread Todd Palino
It is possible to do this using the kafka-reassign-partitions admin
command. You can explicitly set the replica list for a partition to be
less than the current replication factor, and that will effectively reduce
it. However, I will say that you should be really sure that you want to do
this. If, for example, you reduce the RF from 2 to 1, you will not be able
to do any maintenance on a live cluster without taking partitions offline.

-Todd

On 5/21/14, 1:23 AM, "David Birdsong"  wrote:

>Is there a way to reduce the replication count? I'm trying to spread
>existing partitions across more brokers, but it's hard to decomm a broker.
>Reducing repl count would suffice for now.
>
>Any tips?
>
>I'm running a mix of 0.8.1.1 and 0.8.1 (I'm upgrading now.)



Re: reduce replication factor

2014-05-21 Thread David Birdsong
I did that and so now the topic has 4 replicas for a repl count of 3, but
only the 'new' replicas exist in the ISR.

The old broker that I want to clear disk space and generally free up
resources has fully synced a topic that I want to disassociate from it.

Is there a way to do this?


On Wed, May 21, 2014 at 7:48 AM, Jun Rao  wrote:

> We don't have an exact tool for doing this. You may be able to do that
> through
>
> http://kafka.apache.org/documentation.html#basic_ops_increase_replication_factorby
> specifying fewer replicas.
>
> Thanks,
>
> Jun
>
>
> On Wed, May 21, 2014 at 1:23 AM, David Birdsong  >wrote:
>
> > Is there a way to reduce the replication count? I'm trying to spread
> > existing partitions across more brokers, but it's hard to decomm a
> broker.
> > Reducing repl count would suffice for now.
> >
> > Any tips?
> >
> > I'm running a mix of 0.8.1.1 and 0.8.1 (I'm upgrading now.)
> >
>


Re: Make kafka storage engine pluggable and provide a HDFS plugin?

2014-05-21 Thread Kam Kasravi
Hi Hangjun

I've explored deploying kafka on yarn and current YARN does not support long 
running services with locality constraints. Deploying kafka producers / 
consumers (not brokers) is supported in the apache incubator samza project. 
Background on YARN limitations can be found here: YARN-371, YARN-1040, 
YARN-1404, YARN-1412 and YARN-2027.  Support for long running services within 
YARN will likely change with the work that Carlo Curino and team are doing 
(rayon) which is described in YARN-1051. Background/technical details are 
described within that JIRA.

Thanks
Kam
On Tuesday, May 20, 2014 10:40 PM, Hangjun Ye  wrote:
 


Hi Steve,

Yes, what I want is that Kafka doesn't have to care about machines
physically (as an option).

Best,
Hangjun


2014-05-21 11:46 GMT+08:00 Steve Morin :

> Hangjun,
>   Does having Kafka in Yarn would be a big architectural change from where
> it is now?  From what I have seen on most typical setup you want machines
> optimized for Kafka, not just it on top of hdfs.
> -Steve
>
>
> On Tue, May 20, 2014 at 8:37 PM, Hangjun Ye  wrote:
>
> > Thanks Jun and Francois.
> >
> > We used Kafka 0.8.0 previously. We got some weird error when expanding
> > cluster and it couldn't be finished.
> > Now we use 0.8.1.1, I would have a try on cluster expansion sometime.
> >
> > I read the discussion on that jira issue and I agree with points raised
> > there.
> > HDFS was also improved a lot since then and many issues have been
> resolved
> > (e.g. SPOF).
> >
> > We have a team for building and providing storage/computing platform for
> > our company and we have already provided a Hadoop cluster.
> > If Kafka has an option to store data on HDFS, we just need to allocate
> some
> > space quota for it on our cluster (and increase it on demand) and it
> might
> > reduce our operational cost a lot.
> >
> > Another (and maybe more aggressive) thought is about the deployment. Jun
> > has a good point: "HDFS only provides data redundancy, but not
> > computational redundancy". If Kafka could be deployed on YARN, it could
> > offload some computational resource management to YARN and we don't have
> to
> > allocate machines physically. Kafka still needs to take care of load
> > balance and partition assignment among brokers by itself.
> > Many computational frameworks like spark/samza have such an option and
> it's
> > a big attractive point for us.
> >
> > Best,
> > Hangjun
> >
> >
> > 2014-05-20 21:00 GMT+08:00 François Langelier :
> >
> > > Take a look at Camus 
> > >
> > >
> > >
> > > François Langelier
> > > Étudiant en génie Logiciel - École de Technologie
> > > Supérieure
> > > Capitaine Club Capra 
> > > VP-Communication - CS Games  2014
> > > Jeux de Génie  2011 à 2014
> > > Argentier Fraternité du Piranha 
> > > 2012-2014
> > > Comité Organisateur Olympiades ÉTS 2012
> > > Compétition Québécoise d'Ingénierie 2012 - Compétition Senior
> > >
> > >
> > > On 19 May 2014 05:28, Hangjun Ye  wrote:
> > >
> > > > Hi there,
> > > >
> > > > I recently started to use Kafka for our data analysis pipeline and it
> > > works
> > > > very well.
> > > >
> > > > One problem to us so far is expanding our cluster when we need more
> > > storage
> > > > space.
> > > > Kafka provides some scripts for helping do this but the process
> wasn't
> > > > smooth.
> > > >
> > > > To make it work perfectly, seems Kafka needs to do some jobs that a
> > > > distributed file system has already done.
> > > > So just wondering if any thoughts to make Kafka work on top of HDFS?
> > > Maybe
> > > > make the Kafka storage engine pluggable and HDFS is one option?
> > > >
> > > > The pros might be that HDFS has already handled storage management
> > > > (replication, corrupted disk/machine, migration, load balance, etc.)
> > very
> > > > well and it frees Kafka and the users from the burden, and the cons
> > might
> > > > be performance degradation.
> > > > As Kafka does very well on performance, possibly even with some
> degree
> > of
> > > > degradation, it's still competitive for the most situations.
> > > >
> > > > Best,
> > > > --
> > > > Hangjun Ye
> > > >
> > >
> >
> >
> >
> > --
> > Hangjun Ye
> >
>



-- 
Hangjun Ye

Re: reduce replication factor

2014-05-21 Thread Jun Rao
We don't have an exact tool for doing this. You may be able to do that
through
http://kafka.apache.org/documentation.html#basic_ops_increase_replication_factorby
specifying fewer replicas.

Thanks,

Jun


On Wed, May 21, 2014 at 1:23 AM, David Birdsong wrote:

> Is there a way to reduce the replication count? I'm trying to spread
> existing partitions across more brokers, but it's hard to decomm a broker.
> Reducing repl count would suffice for now.
>
> Any tips?
>
> I'm running a mix of 0.8.1.1 and 0.8.1 (I'm upgrading now.)
>


Re: Kafka replication throttling

2014-05-21 Thread Jun Rao
We don't have such throttling right now. Could you file a jira?

Thanks,

Jun


On Tue, May 20, 2014 at 10:29 PM, Marcos Juarez Lopez wrote:

> Hi,
>
> We have several Kafka clusters in production, and we've had to reassign
> replication a few times now in production.  Some of our topic/partitions
> are pretty large, up to 32 partitions per topic, and 16GB per partition, so
> adding a new broker and/or repairing a broker that had been down for some
> time turns out to be a major undertaking.
>
> Today, when we attempt to replicate a single partition, it pegs the disk
> IO, and uses a significant chunk of the 10Gbps interface for a good ~5
> minutes.  This is causing problems for our downstream consumers, which rely
> on having a consistent stream of realtime data being sent to them.
>
> Is there a way to throttle Kafka replication between nodes, so that instead
> of it going full blast, it will replicate at a fixed rate in megabytes or
> activities/batches per second?  Or maybe is this planned for a future
> release, maybe 0.9?
>
> Thanks,
>
> Marcos Juarez
>


Re: ISR not updating

2014-05-21 Thread Paul Mackles
Restarting the partition leaders cleared things up. We were hesitant to do
that at first because it was unclear what would happen with the
availability of the partitions in question since there were no other
replicas in the ISR (at least according to ZK). From what we observed, the
partitions did remain available during the restart. In other words, the
replicas were in sync the whole time and it was really just a matter of
the ISRs in ZK being out-of-sync.

I am not sure if this is an issue in more recent versions.

Paul

On 5/17/14 9:16 PM, "Paul Mackles"  wrote:

>Today we did a rolling restart of ZK. We also restarted the kafka
>controller and ISRs are still not being updated in ZK. Again, the cluster
>seems fine and the replicas in question do appear to be getting updated. I
>am guessing there must be some bad state persisted in ZK.
>
>On 5/17/14 7:50 PM, "Shone Sadler"  wrote:
>
>>Hi Jun,
>>
>>I work with Paul and am monitoring the cluster as well.   The status has
>>not changed.
>>
>>When we execute kafka-list-topic we are seeing the following (showing one
>>of two partitions having the problem)
>>
>>topic: t1 partition: 33 leader: 1 replicas: 1,2,3 isr: 1
>>
>>when inspecting the logs of leader: I do see a spurt of ISR
>>shrinkage/expansion  around the time that the brokers were partitioned
>>from
>>ZK. But nothing past the last message "Cached zkVersion [17] not equal to
>>that in zookeeper." from  yesterday.  There are not constant changes to
>>the
>>ISR list.
>>
>>Is there a way to force the leader to update ZK with the latest ISR list?
>>
>>Thanks,
>>Shone
>>
>>Logs:
>>
>>cat server.log | grep "\[t1,33\]"
>>
>>[2014-04-18 10:16:32,814] INFO [ReplicaFetcherManager on broker 1]
>>Removing
>>fetcher for partition [t1,33] (kafka.server.ReplicaFetcherManager)
>>[2014-05-13 19:42:10,784] ERROR [KafkaApi-1] Error when processing fetch
>>request for partition [t1,33] offset 330118156 from consumer with
>>correlation id 0 (kafka.server.KafkaApis)
>>[2014-05-14 11:02:25,255] ERROR [KafkaApi-1] Error when processing fetch
>>request for partition [t1,33] offset 332896470 from consumer with
>>correlation id 0 (kafka.server.KafkaApis)
>>[2014-05-16 12:00:11,344] INFO Partition [t1,33] on broker 1: Shrinking
>>ISR
>>for partition [t1,33] from 3,1,2 to 1 (kafka.cluster.Partition)
>>[2014-05-16 12:00:18,009] INFO Partition [t1,33] on broker 1: Cached
>>zkVersion [17] not equal to that in zookeeper, skip updating ISR
>>(kafka.cluster.Partition)
>>[2014-05-16 13:33:11,344] INFO Partition [t1,33] on broker 1: Shrinking
>>ISR
>>for partition [t1,33] from 3,1,2 to 1 (kafka.cluster.Partition)
>>[2014-05-16 13:33:12,403] INFO Partition [t1,33] on broker 1: Cached
>>zkVersion [17] not equal to that in zookeeper, skip updating ISR
>>(kafka.cluster.Partition)
>>
>>
>>On Sat, May 17, 2014 at 11:44 AM, Jun Rao  wrote:
>>
>>> Do you see constant ISR shrinking/expansion of those two partitions in
>>>the
>>> leader broker's log ?
>>>
>>> Thanks,
>>>
>>> Jun
>>>
>>>
>>> On Fri, May 16, 2014 at 4:25 PM, Paul Mackles 
>>>wrote:
>>>
>>> > Hi - We are running kafka_2.8.0-0.8.0-beta1 (we are a little behind
>>>in
>>> > upgrading).
>>> >
>>> > From what I can tell, connectivity to ZK was lost for a brief period.
>>>The
>>> > cluster seemed to recover OK except that we now have 2 (out of 125)
>>> > partitions where the ISR appears to be out of date. In other words,
>>> > kafka-list-topic is showing only one replica in the ISR for the 2
>>> > partitions in question (there should be 3).
>>> >
>>> > What's odd is that in looking at the log segments for those
>>>partitions on
>>> > the file system, I can see that they are in fact getting updated and
>>>by
>>> all
>>> > measures look to be in sync. I can also see that the brokers where
>>>the
>>> > out-of-sync replicas reside are doing fine and leading other
>>>partitions
>>> > like nothing ever happened. Based on that, it seems like the ISR in
>>>ZK is
>>> > just out-of-date due to a botched recovery from the brief ZK outage.
>>> >
>>> > Has anyone seen anything like this before? I saw this ticket which
>>> sounded
>>> > similar:
>>> >
>>> > https://issues.apache.org/jira/browse/KAFKA-948
>>> >
>>> > Anyone have any suggestions for recovering from this state? I was
>>> thinking
>>> > of running the preferred-replica-election tool next to see if that
>>>gets
>>> the
>>> > ISRs in ZK back in sync.
>>> >
>>> > After that, I guess the next step would be to bounce the kafka
>>>servers in
>>> > question.
>>> >
>>> > Thanks,
>>> > Paul
>>> >
>>> >
>>>
>



Producer not producing msgs

2014-05-21 Thread Ranjith Venkatesan
Hi all,

  We are using kafka - 0.8 in our production setup. We have 2 kafka servers 
and 2 zookeepers. There seem to be some sort of network problem in our DC. As a 
result of this, connection refused exception in replica which was thrown , is 
shown here. Our producer didnt throw any exception. its log is normal as shown 
here. But our consumers couldnt consume from kafka. 


When i restart kafka servers, it worked correctly. Is there any approach to 
overcome this in future ??? 


Any help in resolving this issue will be appreaciated. 


PS : Our producers having configuration  "request.required.acks=1" 













Re: Kafka Migration Tool

2014-05-21 Thread Mo Firouz
Thanks, you are right, my consumer config was pointing to the 0.8! Cheers!


On 20 May 2014 05:36, Jun Rao  wrote:

> It seems that you may have set the zk connection string to one used by 0.8
> Kafka brokers.
>
> Thanks,
>
> Jun
>
>
> On Mon, May 19, 2014 at 8:34 AM, Mo Firouz 
> wrote:
>
> > Hi all,
> >
> > I'm trying to migrate from Kafka 0.7.2-2.9.2 (with Zookeeper 3.3.4 from
> > Cloudera) to Kafka 0.8.1.1-2.9.2 (with official Zookeeper 3.4.5 ) -
> However
> > hitting brick walls with a very mysterious problem:
> >
> > 6) at kafka.tools.KafkaMigrationTool.main(KafkaMigrationTool.java:217)
> > Caused by: java.lang.NumberFormatException: For input string:
> > ""1400511498394","host""
> >
> > I've attached logs, scripts and configs from everything that I'm trying
> to
> > run.
> >
> > FYI: We have three servers for Kafka Brokers and three servers for
> > Zookeeper. They are running on Staging (stag) as number s04, s05 and s06.
> > I've only given the properties for s04 as the other two are almost
> > identical.
> >
> > Thanks,
> > Cheers,
> > Mo.
> >
> >
> >
>


Re: Java API to list topics and partitions

2014-05-21 Thread Saurabh Agarwal (BLOOMBERG/ 731 LEX -)
Thanks. It works.  


- Original Message -
From: Jun Rao 
At: Tuesday, May 20, 2014 23:33

You can issue a TopicMetadataRequest. See 
https://cwiki.apache.org/confluence/display/KAFKA/0.8.0+SimpleConsumer+Example

Thanks,

Jun


On Tue, May 20, 2014 at 3:41 PM, Saurabh Agarwal (BLOOMBERG/ 731 LEX -) 
 wrote:

Hi,

Is there java API in kafka to list topics and partitions in the kafka broker?
Thanks,
Saurabh.




reduce replication factor

2014-05-21 Thread David Birdsong
Is there a way to reduce the replication count? I'm trying to spread
existing partitions across more brokers, but it's hard to decomm a broker.
Reducing repl count would suffice for now.

Any tips?

I'm running a mix of 0.8.1.1 and 0.8.1 (I'm upgrading now.)