We have seen cases with 0.8.1 when, under load, replication threads would hang
up and not transfer data any longer. Restarting clears this.
I haven't found a way to monitor for this in a nice way, other than seeing
partitions stay under-replicated for long periods of time.
Sent from my BlackBe
I am still trying to find a way to detect how far behind a replica is, nicely,
so I can differentiate between "10 offsets" and "1 offsets" behind.
This would help with problems like this one, as we often have replicas that are
just slightly behind, due to Bursty traffic, but the ones
Hi gwen,
Can you share how you do these end to end latency tests? I am more sysadmin
than coder and have wanted to get something like that going for my kafka
clusters. I'd love more details about how you do it, and how you monitor the
results.
Thanks!
Sent from my BlackBerry 10 smartphone on
When we ran in to this problem we ended up going in to zookeeper and changing
the leader to point to one of the replicas, then did a force leader election.
This got the partition back online.
Original Message
From: Virendra Pratap Singh
Sent: Wednesday, March 4, 2015 2:00 AM
To: Gwen Shapir
You can enable unclean leader election, which would allow the lagging partition
to still become leader. There would be some data loss (offsets between the
leggy partition and the old leader) but the partition would stay online and
available.
Sent from my BlackBerry 10 smartphone on the TELUS
Rsyslog (8.15+) now supports producing to Kafka, and doesn't require java
(that can be a bonus). Rsyslog can use a disk buffer, then when it can
connect to Kafka, it will start streaming logs until the connection drops.
That's a pretty simple config, and there are lots of examples online.
T
As far as the number of partitions a single broker can handle, we've set
our cap at 4000 partitions (including replicas). Above that we've seen some
performance and stability issues.
-Todd
On Tue, Oct 21, 2014 at 12:15 AM, Xiaobin She wrote:
> hello, everyone
>
> I&
each
broker to keep that list (to serve metadata requests). The number of
partitions on a single broker is governed by that broker's ability to
handle the messages and files on disk. That's a much more limiting factor
than what the controller can do.
-Todd
On Tue, Oct 21, 2014 at 2:52 P
In fact there are many more than 4000 open files. Many of our brokers run
with 28,000+ open files (regular file handles, not network connections). In
our case, we're beefing up the disk performance as much as we can by
running in a RAID-10 configuration with 14 disks.
-Todd
On Tue, Oct 21,
obably going to test out RAID 5 and 6 to start with and see how much we
lose from the parity calculations.
-Todd
On Wed, Oct 22, 2014 at 3:59 PM, Jonathan Weeks
wrote:
> Neha,
>
> Do you mean RAID 10 or RAID 5 or 6? With RAID 5 or 6, recovery is
> definitely very painful, but l
data. What you have to do is use
multiple spindles, with sufficiently fast disk speeds, to increase your
overall IO capacity. You can also tune to get a little more. For example,
we use a 120 second commit on that mount point to reduce the frequency of
flushing to disk.
-Todd
On Wed, Oct 22, 20
sy to implement within the broker.
-Todd
On Wed, Oct 22, 2014 at 11:53 PM, Neil Harkins wrote:
> I've been thinking about this recently.
> If kafka provided cmdline hooks to be executed on segment rotation,
> similar to postgres' wal 'archive_command', configurations could
go above 4000 partitions, and
that's probably a combination of what the software can handle and the
number of open files.
-Todd
On Thu, Oct 23, 2014 at 11:19 PM, Xiaobin She wrote:
> Todd,
>
> Thank you very much for your reply. My understanding of RAID 10 is wrong.
>
> I unders
why we're
going to reevaluate our storage soon.
That said, we are using SSDs for the transaction log volume on our
Zookeeper nodes, with great success. We detailed some of that in the
presentation that Jonathan linked (no latency or outstanding requests). It
helps that we use very high qualit
s up, so we can't take any load
off, lather, rinse, repeat.
Ultimately, we need to add even more hardware to the busy clusters, but
that times some time, so I'm hoping we can get some ideas about what we can
tune and improve.
Thanks,
Todd.
few hosts).
As far as lots of small partitions vs. a few large partitions, we prefer
the former. It means we can spread the load out over brokers more evenly.
-Todd
On Tue, Nov 4, 2014 at 10:07 AM, Joel Koshy wrote:
> Ops-experts can share more details but here are some comments:
> >
&
figure out which
> partition is lagging behind. Another thing you can look at is the ISR
> shrink/expand rate. If you see a lot of churn you may need to tune the
> settings that affect ISR maintenance (replica.lag.time.max.ms,
> replica.lag.max.messages).
and Todd Palino said:
> Unde
mal operation unless we have a problem like a down
broker. If replicas are falling behind due to network congestion (or other
resource exhaustion), we balance things out, expand the cluster, or find
our problem producer or consumer and fix them.
-Todd
On Tue, Nov 4, 2014 at 12:13 PM, Todd S
It is not possible, due to how the results for the offset request are stored
within the broker and API (as a map). You will have to do 2 requests to get
both offsets.
-Todd
> On Nov 18, 2014, at 8:52 PM, Thunder Stumpges wrote:
>
> Hey all,
>
> We are working on a .net cli
Log compaction though allows it to work as a data store quite well for some use
cases . It's exactly why I started looking hard at Kafka lately.
"The general idea is quite simple. Rather than maintaining only recent
log entries in the log and throwing away old log segments we maintain
the most
Hi all,
I'm trying to find an existing metric, or method to monitor for data
being sent to a topic that doesn't exist. Is there a JMX stat I
should look at now, or some other way you catch this sort of thing?
We'd like to get ahead of our users noticing a missing topic, or
sending data to a typo
,#,,
Z. Z
Sent from the wilds on my BlackBerry smartphone.
Original Message
From: Gary Ogden
Sent: Thursday, February 12, 2015 8:23 AM
To: users@kafka.apache.org
Reply To: users@kafka.apache.org
Subject: Re: understanding partition key
Thanks David. Whether Kafka is the right choic
the metrics. I'll fill in
more detail once I have it.
-Todd
On Thu, Feb 12, 2015 at 9:51 PM, Jay Kreps wrote:
> This is a serious issue, we'll take a look.
>
> -Jay
>
> On Thu, Feb 12, 2015 at 3:19 PM, Solon Gordon wrote:
>
> > I saw a very similar jump in CPU
itted consumer offsets. This allows us to catch a broken consumer, as
well as an active consumer that is just falling behind.
-Todd
On Fri, Feb 13, 2015 at 9:34 PM, tao xiao wrote:
> Thanks Joel. But I discover that both MaxLag and FetcherLagMetrics are
> always
> much smaller than th
p the metrics
and sends them to our internal system.
It's not ideal. We're working on a script to calculate lag efficiently for
all consumers who commit offsets to Kafka, rather than a select set.
-Todd
On Mon, Feb 16, 2015 at 12:27 AM, tao xiao wrote:
> Thank you Todd for your de
27;ll find that it doesn't matter much either way. In
that case I would probably go with individual groups for isolation.
-Todd
On Mon, Feb 16, 2015 at 3:30 PM, Scott Chapman wrote:
> We have several dozen topics, each with only one topic (replication factor
> or 2).
>
>
(either in those datacenters or in another) where we
mirror all the traffic from the local clusters together into one place.
This way clients can pick a local or a global view of the data, and we can
have everyone still stay in their local Kafka clusters.
-Todd
On Fri, Feb 27, 2015 at 4:21 PM
tting it to 0). If you
do care, you just have to tolerate the latency. The speed of light isn't
just a good idea, it's the law.
-Todd
On Tue, Mar 3, 2015 at 3:34 PM, Jeff Schroeder
wrote:
> Mirror maker is about separating latency and failure domains. I think it is
> a very elegant
round robin allows. This
could be handled with pluggable partition assignment schemes (see my comment on
the KIP), but I think it also speaks to the need for compound reassignment
schemes. For example, rack aware across brokers, and by mount point usage
within the broker.
-Todd
> On Mar 7, 2
ch is easier to work with. Another option that
some consumers use is to pipe messages in from the kafka-console-consumer.
This works well, but if you're not careful with stopping it you can easily
lose messages.
-Todd
On Tue, Mar 17, 2015 at 6:47 AM, Sloot, Hans-Peter <
hans-peter.sl..
tiered Kafka architectures. If you're in Austin for that, please come by
and check it out.
-Todd
particular can lead
to a significant increase in produce latency as it increases even over
10-15% utilization.
-Todd
On Fri, Mar 20, 2015 at 3:41 PM, Emmanuel wrote:
> This is why I'm confused because I'm tryign to benchmark and I see numbers
> that seem pretty low to me...80
a, it gets woven in pretty tightly.
Confluent has made a start on this by releasing a stack with schemas
integrated in. This is probably a good place to start as far as building an
open source audit service.
-Todd
On Mon, Mar 23, 2015 at 12:47 AM, Navneet Gupta (Tech - BLR) <
navneet.gu...
rchitectures) and at the meet up that Jun has
set up at ApacheCon. If you have any questions, just ask!
-Todd
On Mon, Apr 6, 2015 at 9:35 AM, Rama Ramani wrote:
> Hello,
> I am trying to understand some of the common Kafka deployment
> sizes ("small", "medium&qu
files manually after a move (we have a
separate time-series database of offsets for every partition, so we can
tell what the mtime of the file "should" be within 60 seconds), but we
haven't done any experimentation with this as to whether or not it would
actually work without problems.
-
d you
perform all the same checks that the command line tool does. If you're
asking if you can just change the /brokers/topics/* znodes directly, the
answer is no.
-Todd
On Tue, Apr 7, 2015 at 6:44 AM, Navneet Gupta (Tech - BLR) <
navneet.gu...@flipkart.com> wrote:
> Hi,
>
> I
number of brokers you have.
We tend to ignore the second one on most clusters, but we will expand a
topic (as long as it is not keyed) if the retention on disk exceeds 50 GB.
That's just a guideline we have so it's easier to balance the traffic and
move partitions around when needed.
FWIW, we've had good luck changing the mtime. No problems found.
On Mon, Apr 6, 2015 at 4:37 PM, Todd Palino wrote:
> I answered this in IRC, but the issue is that retention depends on the
> modification time of the log segments on disk. When you copy a partition
> from one bro
if there are
issues?
Thanks very much, sorry for the question dump!
On Mon, Mar 23, 2015 at 9:42 AM, Todd Palino wrote:
> Emmanuel, if it helps, here's a little more detail on the hardware spec we
> are using at the moment:
>
> 12 CPU (HT enabled)
> 64 GB RAM
> 16 x 1TB SAS dr
topics.
-Todd
On Wed, Apr 8, 2015 at 9:29 AM, Akshat Aranya wrote:
> Thanks for the info, Todd. This is very useful. Please see my question
> inline:
>
> On Mon, Apr 6, 2015 at 10:24 AM, Todd Palino wrote:
>
> >
> > - Partition count (leader and follower combin
rtitions. That way you
could have the customers just consume the bucket they are interested in. It
would require more up front work to come up with the custom partitioner,
but it would be very efficient as you move forwards.
-Todd
On Wed, Apr 8, 2015 at 7:35 PM, Ralph Caraveo wrote:
> Hello Kafka
ted inside an
application, they are automatically put into the monitoring system for the
customers.
-Todd
On Wed, Apr 8, 2015 at 10:50 AM, Todd S wrote:
> Sorry go back this far in time, I just noticed that the list had
> replied accusing this email being spam, so I'll try again w
We've been running Kafka 0.7.0 in production for several months and have been
quite happy. Our use case to date has been to pull from the head of our topics,
so we're normally consuming within seconds of message production using the high
level consumer which is working great as far as I can tell
Kafka producer/consumer
library which is used by both Klogger and KaBoom.
The project is still active internally, with frequent releases.
Please have a look if you're interested. We welcome any feedback or
contributions.
Cheers,
Todd
Todd Snyder
Senior Infrastructure Specialist, Big Dat
g them from the
LinkedIn-specific tooling so we can release them to everyone else :)
-Todd
> On May 24, 2015, at 7:45 PM, Henry Cai wrote:
>
> We have a kafka cluster with 10 brokers and we are using the kafka
> replication tool (kafka-reassign-partitions.sh) when we need to add mor
have written that need to be incorporated which do partition
balancing by count and by size.
Let me figure out where we are going to host this, but we can probably put out
the script we have in the next couple weeks. Then we can just iterate on it.
-Todd
> On May 24, 2015, at 8:46 PM, Henry
doing performance testing or have a specific
concern.
-Todd
> On Jun 2, 2015, at 1:01 PM, Aditya Auradkar
> wrote:
>
> Number of underreplicated partitions, total request time are some good bets.
>
> Aditya
>
>
> From:
cense, and hosted on
GitHub at:
https://github.com/linkedin/Burrow
Documentation is on the GitHub wiki at:
https://github.com/linkedin/Burrow/wiki
-Todd
on system).
It's up on GitHub at https://github.com/linkedin/Burrow if you would like
to take a look and give it a try.
-Todd
On Thu, Jun 4, 2015 at 5:52 PM, Joel Koshy wrote:
> Hi Otis,
>
> Yes this is a limitation in the old consumer. i.e., a number of
> per-topic/partition m
gs on SSD.
And yeah, this is specific for kafka-committed offsets. I'm looking at some
options for handling Zookeeper as well, but since our goal with this was to
monitor our own infrastructure applications and move forwards, it hasn't
gotten a lot of my attention yet.
-Todd
On Tue,
to roll back to a specific point, for example), and restart.
-Todd
On Wed, Jun 10, 2015 at 1:20 PM, James Cheng wrote:
> Hi,
>
> How are people specifying/persisting/resetting the consumer group
> identifier ("group.id") when using the high-level consumer?
>
>
work before it is
finalized.
-Todd
On Wed, Jun 10, 2015 at 4:48 PM, James Cheng wrote:
>
> > On Jun 10, 2015, at 1:26 PM, Todd Palino wrote:
> >
> > For us, group ID is a configuration parameter of the application. So we
> > store it in configuration files (generally
The invalid ACL error is an error that is passed back from Zookeeper. What
version of Zookeeper are you using, and have you set up ACLs within it? I'm
not able to see this on our ZK (3.4.6 with no ACLs).
-Todd
On Fri, Jun 12, 2015 at 9:34 AM, Roger Hoover
wrote:
> Hi,
>
> I was
Can you open an issue on the github page please, and we can investigate
further there?
-Todd
On Fri, Jun 12, 2015 at 10:22 AM, Roger Hoover
wrote:
> Thanks, Todd. I'm also using ZK 3.4.6 with no ACLs. I'm surprised and
> currently stumped by this error.
>
> On Fri, J
It took a little while for this to post (aligning with company PR and all),
but there's a detailed description of what Burrow is on the LinkedIn
Engineering Blog now:
http://engineering.linkedin.com/apache-kafka/burrow-kafka-consumer-monitoring-reinvented
-Todd
On Thu, Jun 4, 2015 at 6:
I don't think this got changed until after 0.8.2. I believe the change is
still in trunk and not a released version. We haven't even picked it up
internally at LinkedIn yet.
-Todd
On Fri, Jun 19, 2015 at 12:03 AM, bit1...@163.com wrote:
> Thank you for the replay.
> I am
I assume that you are considering the data loss to be the difference in
size between the two directories? This is generally not a good guideline,
as the batching and compression will be different between the two replicas.
-Todd
On Mon, Jun 22, 2015 at 7:26 AM, Nirmal ram
wrote:
> Hi,
&g
tick with the release versions, unless you run in a
development environment where you can tolerate failures and performance
regressions.
-Todd
On Tue, Jun 23, 2015 at 2:59 AM, Achanta Vamsi Subhash <
achanta.va...@flipkart.com> wrote:
> I am planning to use for the producer part. How stab
Thanks, Joel. I know I remember a case where we had a difference like this
between two brokers, and it was not due to retention settings or some other
problem, but I can't remember exactly what we determined it was.
-Todd
On Mon, Jun 22, 2015 at 4:22 PM, Joel Koshy wrote:
> The replica
e and then allowing it to slowly adjust the time afterwards
without sending the leap second.
-Todd
On Thu, Jul 9, 2015 at 7:58 AM, Christofer Hedbrandh wrote:
> Hi Kafka users,
>
> ZooKeeper in our staging environment was running on a very old ubuntu
> version, that was exposed to t
OK, in that case then I'm thinking that you ran into issues that were a
natural result of the Zookeeper ensemble having very high CPU usage.
Unfortunate, but this would not be an unexpected situation when your ZK
ensemble is having significant problems.
-Todd
On Fri, Jul 10, 2015 at 10:
You can also checkout Klogger (https://github.com/blackberry/Klogger), which
will take input from a TCP port or a file.
Todd.
-Original Message-
From: Jason Gustafson [mailto:ja...@confluent.io]
Sent: Monday, July 13, 2015 20:09
To: users@kafka.apache.org
Subject: Re: Kafka producer
nding a long stream of a single character, in error, which was creating a
highly compressible message. Given these cases, I'm no longer certain
that's the case.
Becket, you had been taking a look at this internally. Do you have any
thoughts on this?
-Todd
On Tue, Jul 14, 2015 at 11:18
It could be a client error, but we're seeing it show up in Mirror Maker.
-Todd
On Tue, Jul 14, 2015 at 1:27 PM, JIEFU GONG wrote:
> Got it, looks like I didn't understand the request process and am failing
> to use AB properly. Thanks for the help everyone! I suspect you mi
Yes, we use ZK 3.4.6 exclusively at LinkedIn and there's no problem.
-Todd
> On Jul 22, 2015, at 9:49 AM, Adam Dubiel wrote:
>
> Hi,
>
> I don't think it matters much which version of ZK will you use (meaning
> minor/patch versions). We have been using 3.4.
Flume is one of the options to do this, yes. There are a (growing) number of
tools you could use to get logs in to Kafka, depending on your environment,
needs, level of skill, deployment size, applications, log volume:
(in no particular order)
Kafkacat : https://github.com/edenhill/ka
next preferred replica election, broker 2 will be selected as
the leader.
-Todd
On Wed, Aug 5, 2015 at 3:37 AM, tao xiao wrote:
> I think I figured it out. As pointed out in
>
> https://cwiki.apache.org/confluence/display/KAFKA/Replication+tools#Replicationtools-6.ReassignPartitionsTo
times now when bootstrapping test and development
clusters. You can work around it by making sure all consumers are shut down
until you have all the brokers started. But that's not a good long term
solution for this problem.
-Todd
> On Aug 5, 2015, at 6:28 PM, James Cheng wrote:
>
> Hi
larly helpful for recovering from unclean shutdown.
We generally set it to the number of CPUs in the system, because we want a
fast recovery.
-Todd
On Mon, Aug 10, 2015 at 8:57 AM, Alexey Sverdelov <
alexey.sverde...@googlemail.com> wrote:
> Hi all,
>
> I have a 3 node Kafka cluster.
Hi Andrew,
Are you using Snappy Compression by chance? When we tested the 0.8.2.1 upgrade
initially we saw similar results and tracked it down to a problem with Snappy
version 1.1.1.6 (https://issues.apache.org/jira/browse/KAFKA-2189). We’re
running with Snappy 1.1.1.7 now and the performan
single disks to perform similarly. I didn't test it because it lacks
some balancing ability that We would need.
-Todd
On Friday, August 21, 2015, Prabhjot Bharaj wrote:
> Hi,
>
> I've gone through the details mentioned about Raid and individual disks in
> the ops sectio
You can use the emailer config in Burrow to send alerts directly (it will
monitor specific groups and send emails out when there is a problem). If
you need something more complex than that, I think the best practice is
always to send the output into an general alert/notification system.
-Todd
On
partition
counts, and broker offsets. You can also get information for consumers (as
long as they are committing offsets to Kafka and not ZK).
If it looks useful and there's some bit of info you'd like it to provide,
you can submit a github issue and I'll take a look at it.
-Todd
count for most clusters is 8, and we've got topics up to 512
partitions in some places just taking into account the produce rate alone
(not counting those 720-partition topics that aren't that busy). Many of
our brokers run with over 10k open file handles for regular files alone,
and over 50k o
lly if you have problems
on the broker side that is reducing the efficiency of your system.
I don't think you're looking at anything specific to the kernel version or
the JDK at this point, I think you may just be underpowered (depending on
the answers to the questions buried in there).
ing,
there's no equivalent option for auto partition migration. It's a fairly
resource intensive task, moving all that data around. If you want to move
replicas around when something is down, you have to do it manually.
That said, it may be interesting to consider.
-Todd
On Tue, Sep 1
nment like ours, where there is a
large number of consumers for a given cluster, we actually had to increase
the interval to 10 minutes for quite some time because the load on
Zookeeper was just too high. Once we moved the ZK transaction logs to SSDs,
we were able to drop that back to 60 seconds.
actually was changed a while back but snuck back in with one of the larger
sets of changes). In general, I'd say you should just ignore it for now.
-Todd
On Wed, Sep 16, 2015 at 9:55 PM, Rajiv Kurian wrote:
> My broker logs are full of messages of the following type of log message:
ly. But if you have
an offline consumer, you'll lose the committed offsets by doing this.
-Todd
On Fri, Sep 18, 2015 at 5:31 AM, John Holland <
john.holl...@objectpartners.com> wrote:
> I've been experiencing this issue across several of our environments ever
> since we enab
ould
only be if you were using other log compacted topics), you're likely in the
clear now.
-Todd
On Fri, Sep 18, 2015 at 9:54 AM, John Holland <
john.holl...@objectpartners.com> wrote:
> Thanks!
>
> I did what you suggested and it worked except it was necessary for me to
> re
ou've only
written in 1k of messages, you have a long way to go before that segment
gets rotated. This is why the retention is referred to as a minimum time.
You can easily retain much more than you're expecting for slow topics.
-Todd
On Mon, Sep 21, 2015 at 7:28 PM, allen chan
wrote:
are considered infrastructure
applications for Kafka), but we're not encouraging other internal users to
switch over just yet.
-Todd
On Wed, Sep 23, 2015 at 3:21 PM, James Cheng wrote:
>
> On Sep 18, 2015, at 10:25 AM, Todd Palino wrote:
>
> > I think the last major
or so consumers moved
over to Kafka committed offsets at this point.
Of course, just those apps do cover well over a hundred consumer groups :)
-Todd
On Thursday, September 24, 2015, James Cheng wrote:
>
> > On Sep 24, 2015, at 8:11 PM, Todd Palino > wrote:
> >
> > W
tions.
-Todd
On Fri, Sep 25, 2015 at 1:25 PM, Gwen Shapira wrote:
> How busy are the clients?
>
> The brokers occasionally close idle connections, this is normal and
> typically not something to worry about.
> However, this shouldn't happen to consumers that are actively reading d
be idle because they do not own partitions.
-Todd
On Fri, Sep 25, 2015 at 3:27 PM, noah wrote:
> We're seeing this the most on developer machines that are starting up
> multiple high level consumers on the same topic+group as part of service
> startup. The consumers do not se
trying to reduce it.
-Todd
On Saturday, September 26, 2015, noah wrote:
> Thanks, that gives us some more to look at.
>
> That is unfortunately a small section of the log file. When we hit this
> problem (which is not every time,) it will continue like that for hours.
>
>
sufficient.
-Todd
On Mon, Sep 28, 2015 at 9:53 AM, Jason Rosenberg wrote:
> Just to clarify too, if the only use case for log-compaction we use is for
> the __consumer_offsets, we should be ok, correct? I assume compression is
> not used by default for consumer offsets?
>
> Jason
>
for Cassandra or Redis or anything else).
Then your real consumers can all consume their separate topics. Reading and
writing the data one extra time is much better than rereading all of it 400
times and throwing most of it away.
-Todd
On Wed, Sep 30, 2015 at 9:06 AM, Ben Stopford wrote:
> Hi
What Python library are you using?
In addition, there's no real guarantee that any two libraries will
implement consumer balancing using the same algorithm (if they do it at
all).
-Todd
On Wednesday, September 30, 2015, Rahul R wrote:
> I have 2 kafka consumers. Both the consumers
and if it can be handled separately inside of the
application.
-Todd
On Thu, Oct 8, 2015 at 8:50 AM, Mark Drago wrote:
> Gwen,
>
> Thanks for your reply. I understand all of the points you've made. I
> think the challenge for us is that we have some consumers that are
> intereste
can lose messages).
Basically, you have to trade availability for correctness here. You get to
pick one.
-Todd
On Sun, Oct 11, 2015 at 5:10 PM, wrote:
> You can enable unclean leader election, which would allow the lagging
> partition to still become leader. There would be some dat
retention is 1 week, replication will copy over the last week's worth
of data. That data will not be expired for 1 week, as the expiration is
based on the file modification time.
There is work ongoing that will resolve this extra retention problem.
-Todd
On Monday, October 12, 2015, Raja
We've had no problems with G1 in all of our clusters with varying load
levels. I think we've seen an occasional long GC here and there, but
nothing recurring at this point.
What's the full command line that you're using with all the options?
-Todd
On Wed, Oct 14, 2015 at 2
t are currently considered to be in sync. The important
distinction here is that this list can be shorter than the actual assigned
replica list (from the znode above) if not all of the replicas are in sync.
The state znode also has a 'leader' key which holds the broker ID of the
replica that is
(you can use partition reassignment to
change it). But if they are not all the same, some of the tooling will
break (such as altering the partition count for the topic).
-Todd
On Fri, Oct 16, 2015 at 5:39 PM, Todd Palino wrote:
> Actually, be very careful with this. There are two differ
authentication, and supports
very high throughout.
It's still actively being developed, with a new release coming soon with
enhanced configuration through a new rest api (kontroller).
Cheers
Todd.
Sent from my BlackBerry 10 smartphone on the TELUS network.
Original Message
From: Guozhang
themselves. Your producers would all send their messages through
the GSLB to that endpoint, rather than talking to Kafka directly.
-Todd
On Tue, Nov 3, 2015 at 10:15 AM, Cassa L wrote:
> Hi,
> Has anyone used load balancers between publishers and Kafka brokers? I
> want to do activ
replica information. That assumes you can shut the whole cluster down for a
while.
-Todd
On Fri, Nov 6, 2015 at 1:23 PM, Arathi Maddula
wrote:
> Hi,
> Is it possible to change the broker.ids property for a node belonging to a
> Kafka cluster? For example, currently if I have brokers
will
perform an unclean leader election and select broker 2 (the only replica
available) as the leader for those partitions.
-Todd
On Sun, Nov 22, 2015 at 11:39 AM, Jan Algermissen <
algermissen1...@icloud.com> wrote:
> Hi,
>
> I have a topic with 16 partitions that shows the follo
producer will
attempt a failed request
-Todd
On Sun, Nov 22, 2015 at 12:31 PM, Jan Algermissen <
algermissen1...@icloud.com> wrote:
> Hi Todd,
>
> yes, correct - thanks.
>
> However, what I am not getting is that the KafkaProducer (see my other
> mail from today) silently
1 - 100 of 211 matches
Mail list logo