Re: How many partition can one single machine handle in Kafka?

2014-10-21 Thread Todd Palino
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'm new to kafka, I'm wondering

Re: How many partition can one single machine handle in Kafka?

2014-10-22 Thread Todd Palino
M, Neil Harkins wrote: > On Tue, Oct 21, 2014 at 2:10 PM, Todd Palino wrote: > > 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 stabilit

Re: How many partition can one single machine handle in Kafka?

2014-10-22 Thread Todd Palino
request, how did Kafka make the write > operation on the disk to be sequential (which is emphasized in the design > document of Kafka) and make sure the disk access is effective? > > Thank you for your reply. > > xiaobinshe > > > > 2014-10-22 5:10 GMT+08:00 Todd Palino :

Re: How many partition can one single machine handle in Kafka?

2014-10-22 Thread Todd Palino
n though replication provides partition redundancy, RAID 10 on > >> each broker is usually a good tradeoff to prevent the typical most > common > >> cause of broker server failure (e.g. disk failure) as well, and overall > >> smoother operation. > >>> >

Re: How many partition can one single machine handle in Kafka?

2014-10-23 Thread Todd Palino
the write operation to thest 4000 open > files be sequential to the disk? > > As far as I know, write operation to different files on the same disk will > cause random write, which is not good for performance. > > xiaobinshe > > > > > 2014-10-23 1:00 GMT+08:00 Todd P

Re: How many partition can one single machine handle in Kafka?

2014-10-23 Thread Todd Palino
00 open > > files be sequential to the disk? > > > > As far as I know, write operation to different files on the same disk > will > > cause random write, which is not good for performance. > > > > xiaobinshe > > > > > > > > > >

Re: How many partition can one single machine handle in Kafka?

2014-10-24 Thread Todd Palino
city, one will have to use > multiple spindles with sufficiently fast disk speed, but will it be more > effective for the disk with fewer files? Or does the num of files is not an > important factor for the entire performance of Kafka? > > Thanks again. > > xiaobinshe > &

Re: How many partition can one single machine handle in Kafka?

2014-10-24 Thread Todd Palino
y SSD drives. -Todd On Fri, Oct 24, 2014 at 10:44 AM, Gwen Shapira wrote: > Todd, > > Did you load-test using SSDs? > Got numbers to share? > > On Fri, Oct 24, 2014 at 10:40 AM, Todd Palino wrote: > > Hmm, I haven't read the design doc lately, but I'm surprised

Re: Tuning replication

2014-11-04 Thread Todd Palino
I think your answers are pretty spot-on, Joel. Under Replicated Count is the metric that we monitor to make sure the cluster is healthy. It lets us know when a broker is down (because all the numbers except one broker are elevated), or when a broker is struggling (low counts fluctuating across a fe

Re: Tuning replication

2014-11-04 Thread Todd Palino
various tools to do mbean queries to 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, > >

Re: Using OffsetRequest to get the Head and Tail of a partition in a single request.

2014-11-18 Thread Todd Palino
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 client, and I have a

Re: Increased CPU usage with 0.8.2-beta

2015-02-13 Thread Todd Palino
I'm checking into this on our side. The version we're working on jumping to right now is not the 0.8.2 release version, but it is significantly ahead of 0.8.1.1. We've got it deployed on one cluster and I'm making sure it's balanced right now before I take a look at all the metrics. I'll fill in mo

Re: consumer lag metric

2015-02-16 Thread Todd Palino
The reason for this is the mechanic by which each of the lags are calculated. MaxLag (and the FetcherLagMetric) are calculated by the consumer itself using the difference between the offset it knows it is at, and the offset that the broker has as the end of the partition. The offset checker, howeve

Re: consumer lag metric

2015-02-17 Thread Todd Palino
tailed explanation. Currently I export all > metrics to graphite using the reporter configuration. is there a way I can > do similar thing with offset checker? > > On Mon, Feb 16, 2015 at 4:21 PM, Todd Palino wrote: > > > The reason for this is the mechanic by which each of the l

Re: Need to understand consumer groups.

2015-02-17 Thread Todd Palino
I'm assuming from your description here that all of these topics are being consumed by a single consumer (i.e. a single process that does something different with each topic it sees). In general, you're going to get more efficiency out of a single consumer instance that consumes multiple topics tha

Re: cross-colo writing/reading?

2015-03-02 Thread Todd Palino
Latencies like this are one of the big reasons that we run our Kafka clusters local to the producers and consumers. Another is network partition. As Jeff noted, mirror maker is the way to connect them together. Our architecture uses a local cluster in each datacenter, and then an aggregate cluster

Re: cross-colo writing/reading?

2015-03-03 Thread Todd Palino
uld be much nicer. personally I think the mirror > > maker is more of a patch work than a systematic design. > > > > On Mon, Mar 2, 2015 at 7:53 AM, Todd Palino wrote: > > > > > Latencies like this are one of the big reasons that we run our Kafka > > >

Re: mapping between disk and partition

2015-03-07 Thread Todd Palino
This is one of the major issues that we have noted with using JBOD disk layouts, that there is no tool like partition reassignment to move partitions between disks. Another is that the partition balance algorithm would need to be improved, allowing for better selection of a mount point than rou

Re: consumer groups in python

2015-03-17 Thread Todd Palino
Yeah, this is exactly correct. The python client does not implement the Zookeeper logic that would be needed to do a balanced consumer. While it's certainly possible to do it (for example, Joe implemented it in Go), the logic is non-trivial and nobody has bothered to this point. I don't think anyon

Post on running Kafka at LinkedIn

2015-03-20 Thread Todd Palino
For those who are interested in detail on how we've got Kafka set up at LinkedIn, I have just published a new posted to our Engineering blog titled "Running Kafka at Scale" https://engineering.linkedin.com/kafka/running-kafka-scale It's a general overview of our current Kafka install, tiered

Re: Post on running Kafka at LinkedIn

2015-03-23 Thread Todd Palino
s? Wowzers. > > -James > > On Mar 20, 2015, at 11:30 AM, James Cheng wrote: > > For those who missed it: > > The Kafka Audit tool was also presented at the 1/27 Kafka meetup: > http://www.meetup.com/http-kafka-apache-org/events/219626780/ > > Recorded video is here, sta

Re: kafka audit

2015-03-23 Thread Todd Palino
We've talked about it a little bit, but part of the problem is that it is pretty well integrated into our infrastructure, and as such it's hard to pull it out. I illustrated this a little differently than Jon did in my latest blog post (http://engineering.linkedin.com/kafka/running-kafka-scale), ho

Re: Kafka - deployment size and topologies

2015-04-06 Thread Todd Palino
Luckily, I was just reviewing a lot of this information for my ApacheCon talk next week. Those slides, and the video (I hope) will be published as soon as the talk is done. I'll give you the information I have from LinkedIn's point of view, but out of order :) Our Kafka brokers are all the same mo

Re: New broker ignoring retention

2015-04-06 Thread Todd Palino
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 broker to another, the mtime of the log segments on the new broker will be now. That means the retention clock starts over again. This means that

Re: Increasing replication factor of existing topics

2015-04-07 Thread Todd Palino
The partition reassignment is started by writing a zookeeper node in the admin tree. While it's possible to kick off the partition reassignment by writing the zookeeper node that controls it directly, you have to be very careful about doing this, making sure that the format is perfect and you perfo

Re: Number of Partitions and Performance

2015-04-07 Thread Todd Palino
Going to stand with Jay here :) I just posted an email yesterday about how we size clusters and topics. Basically, have at least as many partitions as you have consumers in your consumer group (preferably a multiple). If you want to balance it across the cluster, also have it be a multiple of the

Re: Kafka - deployment size and topologies

2015-04-08 Thread Todd Palino
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

Re: Upper-bound on number of consumers

2015-04-09 Thread Todd Palino
1000s of partitions should not be a problem at all. Our largest clusters have over 30k partitions in them without a problem (running on 40 brokers). We've run into some issues when you have more than 4000 partitions (either leader or replica) on a single broker, but that was on older code so there

Re: Post on running Kafka at LinkedIn

2015-04-08 Thread Todd Palino
licting) . > * How do you manage partitioning and balancing (and rebalancing when a > topic/partition start growing very quickly)? > * Have you/how have you enabled your users/customers to monitor their > data flow, or do they just trust you to let them know if there are > issues? >

Re: Replication tools to move topics/partitions gradually

2015-05-24 Thread Todd Palino
We've built tools on top of it that both build the list based on less information (like "clone this broker to that one") and break it down into a configurable number of discrete moves so it doesn't tank the cluster. And yes, I've finally started the process of departing them from the LinkedIn-s

Re: Replication tools to move topics/partitions gradually

2015-05-24 Thread Todd Palino
Cai wrote: > > Todd, > > This is very promising. Do you know when will we be able to see your tools > released to public? > >> On Sun, May 24, 2015 at 7:54 PM, Todd Palino wrote: >> >> We've built tools on top of it that both build the list based on

Re: Kafka JMS metrics meaning

2015-06-02 Thread Todd Palino
Under replicated is a must. Offline partitions is also good to monitor. We also use the active controller metric (it's 1 or 0) in aggregate for a cluster to know that the controller is running somewhere. For more general metrics, all topics bytes in and bytes out is good. We also watch the lea

[ANNOUNCE] Burrow - Consumer Lag Monitoring as a Service

2015-06-04 Thread Todd Palino
I am very happy to introduce Burrow, an application to provide Kafka consumer status as a service. Burrow is different than just a "lag checker": * Multiple Kafka cluster support - Burrow supports any number of Kafka clusters in a single instance. You can also run multiple copies of Burrow in para

Re: Consumer lag lies - orphaned offsets?

2015-06-04 Thread Todd Palino
I just sent out a separate email about the project that I've been working on, Burrow, to change the way we're monitoring consumer status. Like Joel said, the mbeans can be a little tricky to work with. Internally at LinkedIn, we've always depended more on using a tool like the Consumer Offset Chec

Re: [ANNOUNCE] Burrow - Consumer Lag Monitoring as a Service

2015-06-09 Thread Todd Palino
ow in > production? > > Jason > > On Thu, Jun 4, 2015 at 9:43 PM, Todd Palino wrote: > > > I am very happy to introduce Burrow, an application to provide Kafka > > consumer status as a service. Burrow is different than just a "lag > > checker": > >

Re: How to manage the consumer group id?

2015-06-10 Thread Todd Palino
For us, group ID is a configuration parameter of the application. So we store it in configuration files (generally on disk) and maintain it there through our configuration and deployment infrastructure. As you pointed out, hard coding the group ID into the application is not usually a good pattern.

Re: How to manage the consumer group id?

2015-06-11 Thread Todd Palino
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

Re: [ANNOUNCE] Burrow - Consumer Lag Monitoring as a Service

2015-06-12 Thread Todd Palino
per-path=/ > > offsets-topic=__consumer_offsets > > > [tickers] > > broker-offsets=60 > > > [lagcheck] > > intervals=10 > > expire-group=604800 > > > [httpserver] > > server=on > > port=7000 > > On Tue, Jun 9, 2015 at 12:34 PM,

Re: [ANNOUNCE] Burrow - Consumer Lag Monitoring as a Service

2015-06-12 Thread Todd Palino
un 12, 2015 at 9:49 AM, Todd Palino wrote: > > > 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).

Re: [ANNOUNCE] Burrow - Consumer Lag Monitoring as a Service

2015-06-14 Thread Todd Palino
43 PM, Todd Palino wrote: > I am very happy to introduce Burrow, an application to provide Kafka > consumer status as a service. Burrow is different than just a "lag checker": > > * Multiple Kafka cluster support - Burrow supports any number of Kafka > clusters in a sin

Re: Re: Closing socket connection to /192.115.190.61. (kafka.network.Processor)

2015-06-19 Thread Todd Palino
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 using kafka_2.10-0.8.2

Re: data loss - replicas

2015-06-22 Thread Todd Palino
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, > > I not

Re: Is trunk safe for production?

2015-06-23 Thread Todd Palino
Yes and no. We're running a version about a month behind trunk at any given time here at LinkedIn. That's generally the amount of time we spend testing and going through our release process internally (less if there are no problems). So it can be done. That said, we also have several Kafka contrib

Re: data loss - replicas

2015-06-23 Thread Todd Palino
ee this config change at almost (but not exactly) the > same time, but not sure if you are using that feature. > > You may want to use the DumpLogSegments tool to actually compare the > offsets present in both log files. > > On Mon, Jun 22, 2015 at 08:55:40AM -0700, Todd Palino wrot

Re: Leap Second Troubles

2015-07-09 Thread Todd Palino
Did you hit the problems in the Kafka brokers and consumers during the Zookeeper problem, or after you had already cleared it? For us, we decided to skip the leap second problem (even though we're supposedly on a version that doesn't have that bug) by shutting down ntpd everywhere and then allowin

Re: Leap Second Troubles

2015-07-10 Thread Todd Palino
21 AM, Christofer Hedbrandh < christo...@knewton.com> wrote: > Todd, the Kafka problems started when one of three ZooKeeper nodes was > restarted. > > On Thu, Jul 9, 2015 at 12:10 PM, Todd Palino wrote: > > > Did you hit the problems in the Kafka brokers and consumers duri

Re: stunning error - Request of length 1550939497 is not valid, it is larger than the maximum size of 104857600 bytes

2015-07-14 Thread Todd Palino
This is interesting. We have seen something similar internally at LinkedIn with one particular topic (and Avro schema), and only once in a while. We've seen it happen 2 or 3 times so far. We had chalked it up to bad content in the message, figuring that the sender was doing something like sending a

Re: stunning error - Request of length 1550939497 is not valid, it is larger than the maximum size of 104857600 bytes

2015-07-14 Thread Todd Palino
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 might be > running int

Re: Custom Zookeeper install with kafka

2015-07-22 Thread Todd Palino
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.6 for some time and it works > flawle

Re: Specify leader when doing partition reassignment

2015-08-05 Thread Todd Palino
To make sure you have a complete answer here, the order of the replica list that you specify in the partition reassignment will affect the leader selection, but if the current leader is in the new replica list, it will not change the leadership to change. That is, if your current replica list is [

Re: Documentation typo for offsets.topic.replication.factor ?

2015-08-05 Thread Todd Palino
That's exactly right. We've been talking about this internally at LinkedIn, and how to solve it. I think the best option would be to have the broker throw an error on offset commits until there are enough brokers to fulfill the configured RF. We've seen this several times now when bootstrapping

Re: Recovery of Kafka cluster takes very long time

2015-08-10 Thread Todd Palino
It looks like you did an unclean shutdown of the cluster, in which case each open log segment in each partition needs to be checked upon startup. It doesn't really have anything to do with RF=3 specifically, but it does mean that each of your brokers has 6000 partitions to check. What is the setti

Re: Raid vs individual disks

2015-08-21 Thread Todd Palino
At LinkedIn, we are using a RAID-10 of 14 disks. This is using software RAID. I recently did some performance testing with RAID 0, 5, and 6. I found that 5 and 6 underperformed significantly, possibly due to the parity calculations. RAID 0 had a sizable performance gain over 10, and I would expect

Re: How to monitor lag when "kafka" is used as offset.storage?

2015-09-03 Thread Todd Palino
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

Re: API to query cluster metadata on-demand

2015-09-03 Thread Todd Palino
What Gwen said :) We developed a python web service internally called Menagerie that provides this functionality for both Kafka and Zookeeper. We use it to drive a web dashboard for stats, our (old style) lag checking, and some other CLI tools. Unfortunately it ties into too much internal LinkedIn

Re: Amount of partitions

2015-09-04 Thread Todd Palino
Jun's post is a good start, but I find it's easier to talk in terms of more concrete reasons and guidance for having fewer or more partitions per topic. Start with the number of brokers in the cluster. This is a good baseline for the minimum number of partitions in a topic, as it will assure balan

Re: Amount of partitions

2015-09-08 Thread Todd Palino
t; testing for weeks, compared configuration and setups, without finding the > main cause. > Can this be a Kernel (version/configuration) or Java(7) issue? > > Cheers > Jörg > > > > On 04.09.2015 20:24, Todd Palino wrote: > >> Jun's post is a good start, but I fi

Re: Question on Kafka Replication

2015-09-15 Thread Todd Palino
I put an answer to this on Stack Overflow. Basically, that's not how RF works for Kafka. It's not a guarantee, it's just how the partitions are created, and how it is reported when something is down (under replicated partitions). While there is an option to do auto leader rebalancing, there's no eq

Re: Delay in Zookeeper Offset updates

2015-09-17 Thread Todd Palino
Consumer offsets in Zookeeper are not handled by the Kafka brokers at all - the consumer writes those directly to Zookeeper. Most likely, what you are seeing is the interval over which the consumer is committing offsets. Assuming that you are using the auto.commit.enable setting (it defaults to tru

Re: Closing connection messages

2015-09-17 Thread Todd Palino
This message is regarding a normal connection close. You see it in the logs for any connection - consumers, producers, replica fetchers. It can be particularly noisy because metadata requests often happen on their own connection. The log message has been moved to debug level in recent commits (it

Re: Log Cleaner Thread Stops

2015-09-18 Thread Todd Palino
Yes, this is a known concern, and it should be fixed with recent commits. In the meantime, you'll have to do a little manual cleanup. The problem you're running into is a corrupt message in the offsets topic. We've seen this a lot. What you need to do is set the topic configuration to remove the c

Re: Log Cleaner Thread Stops

2015-09-18 Thread Todd Palino
move the cleaner-offset-checkpoint file from the data directory and > restart the servers. The log indicates all is well. > > Do you know what version the fix to this will be in? I'm not looking > forward to dealing with this on a reoccurring basis. > > -John > > On

Re: log.retention.hours not working?

2015-09-21 Thread Todd Palino
Retention is going to be based on a combination of both the retention and segment size settings (as a side note, it's recommended to use log.retention.ms and log.segment.ms, not the hours config. That's there for legacy reasons, but the ms configs are more consistent). As messages are received by K

Re: Log Cleaner Thread Stops

2015-09-24 Thread Todd Palino
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

Re: Log Cleaner Thread Stops

2015-09-24 Thread Todd Palino
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

Re: Frequent Consumer and Producer Disconnects

2015-09-25 Thread Todd Palino
I don't see the logs attached, but what does the GC look like in your applications? A lot of times this is caused (at least on the consumer side) by the Zookeeper session expiring due to excessive GC activity, which causes the consumers to go into a rebalance and change up their connections. -Todd

Re: Frequent Consumer and Producer Disconnects

2015-09-25 Thread Todd Palino
> > Could it be the way we are bring up multiple consumers at the same time is > hitting some sort of endless rebalance cycle? And/or the resulting > thrashing is causing them to time out, rebalance, etc.? > > I've tried attaching the logs again. Thanks! > > On Fri, Sep 25, 20

Re: Frequent Consumer and Producer Disconnects

2015-09-26 Thread Todd Palino
We also still have developers creating topics semi-regularly, which it > seems like can cause the high level consumer to disconnect? > > > On Fri, Sep 25, 2015 at 6:16 PM Todd Palino > wrote: > >> That rebalance cycle doesn't look endless. I see that you started 23

Re: Log Cleaner Thread Stops

2015-09-28 Thread Todd Palino
> On Fri, Sep 25, 2015 at 12:15 AM, Todd Palino wrote: > > > For now, that's the way it is. Historically, we've only monitored the lag > > for our infrastructure applications. Other users are responsible for > their > > own checking, typically using the maxlag

Re: number of topics given many consumers and groups within the data

2015-09-30 Thread Todd Palino
So I disagree with the idea to use custom partitioning, depending on your requirements. Having a consumer consume from a single partition is not (currently) that easy. If you don't care which consumer gets which partition (group), then it's not that bad. You have 20 partitions, you have 20 consumer

Re: Kafka Consumers getting overlapped data

2015-09-30 Thread Todd Palino
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 have the sa

Re: mapping events to topics

2015-10-08 Thread Todd Palino
Multiple topics is the model I would recommend for what you have described. LinkedIn has an environment where we have a wide mix, in a lot of different clusters. We have some topics that have one producer and one consumer (queuing). We have some topics that are multi-producer (tracking and metrics,

Re: Kafka availability guarantee

2015-10-11 Thread Todd Palino
To answer the question, yes, it is incorrect. There are a few things you can do to minimize problems. One is to disable unclean leader election, use acks=-1 on the producers, have an RF of 3 or greater, and set the min ISR to 2. This means that the topic will only be available if there are at least

Re: Does Kafka recover all data if node is reimaged

2015-10-12 Thread Todd Palino
Yes. As long as you have not reassigned the partitions to other brokers, the wiped broker will rebuild from replication. Keep in mind, however, that if you are using the retention by time configuration, you will have 2x retention on that broker for the length of retention. This means that if your r

Re: G1 tuning

2015-10-14 Thread Todd Palino
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:18 PM, Scott Clasen

Re: Where is replication factor stored?

2015-10-16 Thread Todd Palino
Actually, be very careful with this. There are two different things stored in Zookeeper, and depending on what you're interested in you want to make sure you're looking at the right one. If you want to know the replica assignment - that is, what brokers a given partition is assigned to - you need

Re: Where is replication factor stored?

2015-10-16 Thread Todd Palino
(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

Re: Load balancer for Kafka brokers

2015-11-03 Thread Todd Palino
We use loadbalancers for our producer configurations, but what you need to keep in mind is that that connection is only used for metadata requests. The producer queries the loadbalancer IP for metadata for the topic, then disconnects and reconnects directly to the Kafka brokers for producing messag

Re: Change kafka broker ids dynamically

2015-11-06 Thread Todd Palino
I’m not quite sure why you would need to do this - the broker IDs are not significant outside of the internal metadata. But this is what you would have to do for each move (assuming you are running with at least replication factor 2): 1) Shut down the broker 2) Clear its partition data 3) Reconfig

Re: Help on understanding kafka-topics.sh output

2015-11-22 Thread Todd Palino
Replicas and Isr are both a comma separated list of broker IDs. So in this output, I am seeing that you have two Kafka brokers with IDs 1 and 2. You have a topic, capture, with 16 partitions at replication factor 1 (1 replica per partition). The broker with ID 2 is not online, which is why it shows

Re: Help on understanding kafka-topics.sh output

2015-11-22 Thread Todd Palino
mmon.serialization.StringSerializer" > > > > > kafkaProducer.send(new ProducerRecord[String,String](topic, key, data),new > Callback { > def onCompletion(recordMetadata: RecordMetadata, e: Exception):Unit = { >if(e != null) { > logger.error(s"Could

Re: Number of partitions and disks in a topic

2015-12-01 Thread Todd Palino
al to have a topic with 1000 partitions? I was thinking about about > two/four partitions per node. is it wrong my thought? > > As I'm going to process data with Spark, I could have numberPartitions > equals numberExecutors in Spark as max, always thinking in the future and >

Re: how to reset kafka offset in zookeeper

2015-12-18 Thread Todd Palino
offset.reset to "smallest", > but it does not reset the offset in zookeeper and that's why flume will not > read messages from first offset. > > Is there any way to reset kafka offset in zookeeper? > > Thanks, > Akhilesh > -- *—-* *Todd Palino* Staff S

Re: how to reset kafka offset in zookeeper

2015-12-18 Thread Todd Palino
t; > > > > > > -- > > Jens Rantil > > Backend engineer > > Tink AB > > > > Email: jens.ran...@tink.se > > Phone: +46 708 84 18 32 > > Web: www.tink.se > > > > Facebook <https://www.facebook.com/#!/tink.se> Linkedin > > < > > > http://www.linkedin.com/company/2735919?trk=vsrp_companies_res_photo&trkInfo=VSRPsearchId%3A1057023381369207406670%2CVSRPtargetId%3A2735919%2CVSRPcmpt%3Aprimary > > > > > Twitter <https://twitter.com/tink> > > > -- *—-* *Todd Palino* Staff Site Reliability Engineer Data Infrastructure Streaming linkedin.com/in/toddpalino

Re: how to reset kafka offset in zookeeper

2015-12-18 Thread Todd Palino
lt;http://sematext.com/about/contact.html> > > On Fri, Dec 18, 2015 at 6:29 PM, Todd Palino wrote: > > > That works if you want to set to an arbitrary offset, Marko. However in > the > > case the OP described, wanting to reset to smallest, it is better to just > >

Re: how to reset kafka offset in zookeeper

2015-12-19 Thread Todd Palino
. -Todd On Saturday, December 19, 2015, Akhilesh Pathodia < pathodia.akhil...@gmail.com> wrote: > What is the process for deleting the consumer group from zookeeper? Should > I export offset, delete and then import? > > Thanks, > Akhilesh > > On Fri, Dec 18, 2015 at 11:

Re: how to reset kafka offset in zookeeper

2015-12-19 Thread Todd Palino
manager where I can delete the group? > > Thanks > > On Sat, Dec 19, 2015 at 11:47 PM, Todd Palino wrote: > > > If what you want to do is reset to smallest, all you need to do is stop > the > > consumer, delete the group from Zookeeper, and restart the consumer. It

Re: Kafka + ZooKeeper on the same hardware?

2016-01-14 Thread Todd Palino
be interested > > in any opinions on this anyway. > > > > Thanks! > > \EF > > > -- *—-* *Todd Palino* Staff Site Reliability Engineer Data Infrastructure Streaming linkedin.com/in/toddpalino

Re: Apache Kafka Case Studies

2016-02-03 Thread Todd Palino
jens.ran...@tink.se > Phone: +46 708 84 18 32 > Web: www.tink.se > > Facebook <https://www.facebook.com/#!/tink.se> Linkedin > < > http://www.linkedin.com/company/2735919?trk=vsrp_companies_res_photo&trkInfo=VSRPsearchId%3A1057023381369207406670%2CVSRPtargetId%3A2735919%2CVSRPcmpt%3Aprimary > > > Twitter <https://twitter.com/tink> > -- *—-* *Todd Palino* Staff Site Reliability Engineer Data Infrastructure Streaming linkedin.com/in/toddpalino

Re: Kafka broker decommission steps

2016-03-04 Thread Todd Palino
ode to hold no > topic/group leadership (acting as passive copy) so that it can be > decommissioned with minimal effect to Kafka clients? > > Thank you, > > -- > Muqtafi Akhmad > Software Engineer > Traveloka > -- *—-* *Todd Palino* Staff Site Reliability Engineer Data Infrastructure Streaming linkedin.com/in/toddpalino

Re: [kafka] decreasing Lag

2016-03-07 Thread Todd Palino
decrease > partitions? What can be done to increase the amount of logs being read > from the cluster and ingested into Elastisearch? > > Like I said, very new to kafka. > > Thanks for the help > Tim > -- *—-* *Todd Palino* Staff Site Reliability Engineer Data Infrastructure Streaming linkedin.com/in/toddpalino

Re: Re-Balancing Kafka topics - Best practices

2017-06-13 Thread Todd Palino
eassign-partition.sh)* > *One of the blogs mentioned that - it is preferable to Re-balance Kafka > topics manually, since setting * > > *auto.leader.rebalance.enable = true causes issues.* > > Pls let me know. > Any other best practices wrt. Re-balancing kafka topics ? > > t

Re: Kafka Monitoring

2017-06-20 Thread Todd Palino
> subsidiaries that is proprietary, privileged, confidential and/or > subject > > > to copyright. Any review, retransmission, dissemination or other use > of, > > or > > > taking of any action in reliance upon, this information by persons or > > > entities other than the intended recipient(s) is prohibited and may be > > > unlawful. If you received this in error, please contact the sender > > > immediately and delete and destroy the communication and all of the > > > attachments you have received and all copies thereof. > > > > > > > > > > > > -- *Todd Palino* Senior Staff Engineer, Site Reliability Data Infrastructure Streaming linkedin.com/in/toddpalino

Re: Tuning up mirror maker for high thruput

2017-07-23 Thread Todd Palino
c from EU to US. Only one of them > > is high throughput. We also have a message handler to strip off some > > sensitive information from EU to US but it only works on a low thru put > > topic; the message handler still try to process the other topics but let > it > > pass thru. > > > > Thanks, > > Sunil Parmar > -- *Todd Palino* Senior Staff Engineer, Site Reliability Data Infrastructure Streaming linkedin.com/in/toddpalino

Re: Tuning up mirror maker for high thruput

2017-07-24 Thread Todd Palino
well. How does the OS/broker tuning > affect those consumers that are close to the source datacenter? Will they > continue to function well? > > -James > > > On Jul 23, 2017, at 7:16 AM, Todd Palino wrote: > > > > One of the best pieces of advice I can offer is th

Re: Avoid jvm swapping

2017-08-07 Thread Todd Palino
To avoid swap you should set swappiness to 1, not 0. 1 is a request (don't swap if avoidable) whereas 0 is a demand (processes will be killed as OOM instead of swapping. However, I'm wondering why you are running such large heaps. Most of the ZK heap is used for storage of the data in memory, and

Re: Avoid jvm swapping

2017-08-07 Thread Todd Palino
arch will index its. > > Is it a bad practice to have all these JVMs on the same virtual machine ? > What do you recommend (number of machines, quantity of GB, CPU...) ? For > the moment, each node has 4 vcpu. > > Gabriel. > > 2017-08-07 15:45 GMT+02:00 Todd Palino : > > &

Re: Kafka MirrorMaker - target or source datacenter deployment

2017-09-14 Thread Todd Palino
n the source data center (near the source Kafka > cluster), especially if I can't aggregate the metrics from the 2 data > centers. Is there anything else that would influence me to deploy in > MirrorMaker in either data center? > > Thanks > > Vu > -- *Todd

Re: Change replication factor for a topic in the runtime

2017-09-19 Thread Todd Palino
ble to change the replication factor in runtime? We're using > 10.x version. > > Thanks, > Devendar > -- *Todd Palino* Senior Staff Engineer, Site Reliability Data Infrastructure Streaming linkedin.com/in/toddpalino

New release of Burrow!

2017-12-03 Thread Todd Palino
the Github issues, or PRs for contributing! -Todd -- *Todd Palino* Senior Staff Engineer, Site Reliability Data Infrastructure Streaming linkedin.com/in/toddpalino

  1   2   >