gt; Can I have 20k partition on a single kafka broker ?
>
--
*Todd Palino*
Senior Staff Engineer, Site Reliability
Data Infrastructure Streaming
linkedin.com/in/toddpalino
> ———-
> <http://www.trimble.com/>
> Srinivasa Balaji L
> Principal Architect Cloud & DevOPS - TPaaS
> 10368, Westmoor Drive, Westminster, CO 80021
> *M*: +1(303) 324-9822 <+919790804422>
> *Email*: lsbal...@trimble.com
>
--
*Todd Palino*
Senior Staff Engineer, Site Reliability
Data Infrastructure Streaming
linkedin.com/in/toddpalino
alancing option?
>
> Regards,
> Srikanth
>
--
*—-*
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
”. If you are performing a rolling bounce, this can
conflict seriously with our shutdown check which assures that the cluster
under replicated count is zero before performing a shutdown.
-Todd
On Tue, Mar 29, 2016 at 1:29 PM, James Cheng wrote:
>
> > On Mar 29, 2016, at 10:33 AM, Todd Pali
timise for this level of traffic
>
> To help your answer, we're looking at potentially 16GBit/sec inbound which
> concerns our network team.
>
> If you can please share pointers to existing materials or specific details
> of your deployment, that will be great.
>
>
> Reg
consumer,
but if you’re not doing that you have to maintain some mapping of consumers
to topics. And if you are using a wildcard consumer, you’re going to run
into issues with the number of topics any given group is consuming at some
point. Your system may work fine for 5 topics, but what a
gt; it´s possible to change an existing topic name?
>
> Thanks and best wishes
> Johannes
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
. I don't know if there
> is anything in log file format info or Zoo info attached to original topic
> which would prevent it to work.
>
> Petr
>
> -----Original Message-
> From: Todd Palino [mailto:tpal...@gmail.com]
> Sent: 1. června 2016 10:48
> To: users@kafka.ap
#design_loadbalancing is a
> > good
> > > start.
> > >
> > > Thanks
> > >
> > > Tom Crayford
> > > Heroku Kafka
> > >
> > > On Fri, Jun 3, 2016 at 1:15 PM, cs user wrote:
> > >
> > > > Hi All,
> > >
iatingHeapOccupancyPercent=35 -XX:+DisableExplicitGC
> > > > > > > -Djava.awt.headless=true"
> > > > > > > fi
> > > > > > >
> > > > > > >
> > > > > > > Is this the confluent doc you were r
g a few more.
> The
> > new broker machines would have a lot more storage available to them than
> > the existing brokers. Am I setting myself up for operational headaches by
> > deploying a heterogeneous (in terms of storage capacity) cluster?
> >
> > (Asked on I
mar wrote:
> >
> >> Hey,
> >>
> >> How can I delete particular messages from particular topic?Is that
> >> possible?
> >>
> >> Thanks,
> >> Mudit
> >>
> >>
>
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
ficient, way to do it?
>
> Cheers,
> -Kristoffer
>
> [1] https://gist.github.com/krisskross/a49e462bedb89505e372672cd81129ab
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
otstrap-server
> 10.10.1.61:9092 --describe —group consumer_group_name and parsing the
> response. Is it safe or advisable to do this? I like the fact that it
> tells me each partition lag, although it is also not available if no
> consumer from the group is currently consuming.
> 3. Is
my colleague Jon Bringhurst
profusely for helping to get the structure around the project and the
documentation cleaned up.
-Todd
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
--
*Todd Palino*
Staff Site Reliability Engineer
Data
in real time per partition? I
> cannot find any official documentation on it? How safe is it to use?
>
> Also we're planning on upgrading to 0.10.0.0 - is it still available?
>
> Cheers,
> -Kristoffer
>
> [1] kafka.log:type=Log,name=LogEndOffset,topic=xxx,partition=xxx
&g
consuming data over the internet. Is there a way to encrypt this data
> without turning SSL on. There would be some performance degradation and it
> is not required in other scenarios. Has anyone tried doing cross-regional
> replication in production?
>
> Thanks,
> Reenal
>
--
*
thing in them. I know this is not
> very like production, but on my local this I was only testing with one user
> so get just one partition filled.
>
> Tom
> > On 6 Jul 2016, at 18:08, Todd Palino >
> wrote:
> >
> > Yeah, I've written dissertations at thi
y-1_/10.100.0.113
> voidbridge-oneworks-dummy integration-oneworks-dummy 15
> 0 0 0
> integration-oneworks-dummy-voidbridge-oneworks-dummy-1_/10.100.0.113
>
> > On 8 Jul 2016, at 17:20, Tom Dearman wrote:
> >
> > When you say ‘for
pics? Is it safe to reassign the new
> layout for this internal topic, using kafka-reassign-partitions.sh?
>
>
> Thanks, Anderson
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
past trying to automate
> legitimate mail to GMail. Getting washed out with the spammers is a real
> hazard.
>
> Thanks!
>
>
>
> --
> information informs. analysis enlightens.
>
> Brian M. Dennis, PhD
> 43403 Coton Commons Dr, Leesburg, VA 20176
> e: b...@crossjam.net
the brokers (B-16) got completely messed up
> and is sent for repair.
>
> But I can still see some partitions including the B-16 in its replicas,
> thereby becoming under-replicated.
>
> Is there a proper way to take broker out of rotation?
>
> Praveen
>
--
*Todd Palino*
S
000 sensor messages in parallel but each sensor
> message should be in order.If I create 1 partition it doesn't give high
> throughput .Order is guaranteed only inside the partition. How can
> parallelize messages without changing the order pls help me to find the
> solution.
elp me to configure the topics and partitions.
>
> On Tue, Jan 8, 2019 at 9:19 PM Todd Palino wrote:
>
> > I think you’ll need to expand a little more here and explain what you
> mean
> > by processing them in parallel. Nearly by definition, parallelization and
> > strict o
Going to pile on here, and also say take that blog post with a grain of
salt :)
Look at your data size, and your desired degree of parallelism for
consumers, and guide to that. Yes, 1000 partitions takes a while to create.
But unless your data size is something like 1 TB per day or more, you
proba
I'm assuming the writers were perhaps intending to emphasise the Cruise
> Control or Confluents self-balancing-cluster / auto-balancing features were
> preferable, but in my very brief Google didn't see any advice to set
> auto.leader.rebalance.enabled to false to use those tool
n is: Since there are 2 volumes, is the second just a
> better update?, or should I read the first one too?.
>
> Thanks in advance,
> Miguel Suárez
--
*Todd Palino*
Senior Staff Engineer, Site Reliability
Capacity Engineering
linkedin.com/in/toddpalino
r hand-wavy ideas that came to mind might be:
> * handling topic creation in a MirrorMakerMessageHandler
> * handling topic creation in an interceptor
>
> Anyway, was hoping to get some thoughts from people who are already doing
> this.
>
> Thanks!
> -James
>
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
> > > >
> > > > After doing all that, the topic comes back, every time.
> > > >
> > > > What can we do to delete that topic?
> > > >
> > > > --
> > > >
> > > > In Christ,
> > > >
> > > > Timmy V.
> > > >
> > > > http://blog.twonegatives.com/
> > > > http://five.sentenc.es/ -- Spend less time on mail
> > > >
> > >
> >
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
recreating a topic that has been deleted as it issues a metadata request to
try and find out what happened after an offset request for the topic fails.
-Todd
On Fri, Dec 9, 2016 at 8:37 AM, Tim Visher wrote:
> On Fri, Dec 9, 2016 at 11:34 AM, Todd Palino wrote:
>
> > Given that
t; > -- Surendra Manchikanti
> >
> > On Sat, Dec 10, 2016 at 10:59 AM, Todd Palino wrote:
> >
> > > Are you running something else besides the consumers that would
> maintain
> > a
> > > memory of the topics and potentially recreate them by iss
the same problem.
> nvalid receive (size = 1347375956 larger than 104857600).
>
> When trying to increase the size, Java Out of Memory Exception.
> Did you find a work around for the same ??
>
> Thanks.
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructur
the servers. It’s worked for the last couple without a problem.
-Todd
On Tue, Dec 20, 2016 at 7:55 PM, Sanjeev T wrote:
> Hi,
>
> Can some of you share points on, the versions and handling leap second
> delay on Dec 31, 2016.
>
> Regards
> -Sanjeev
>
--
*Todd Palino
en I try to reassign with the config...
>
> {"version":1,"partitions":[{"topic":"foo","partition":2,"
> replicas":[1004,1001]}]}
>
> I see that it doesn't resolve.
>
> Status of partition reassignment:
> Reas
for our message size we are at the max. This would
> argue that we need to shrink the message size - so perhaps switching to
> avro is the next step?
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
ng kafka 0.9.0.1 and we use org.apache.kafka.common.
> serialization.ByteArrayDeserializer and
> GroupMetadataManager.OffsetsMessageFormatter
> to parse the message.
>
> Thanks,
> Jun
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
to? I have it at 65535 but I just read a
> doc that suggested > 100K is better
>
>
> On Tue, Feb 21, 2017 at 10:45 AM, Todd Palino wrote:
>
> > So I think the important thing to look at here is the IO wait on your
> > system. You’re hitting disk throughput issues, an
ill –9 processID” to stop the Kafka Mirror
> Maker. I am wondering whether there is a better way (e.g. a command) to do
> so? I don’t expect to stop the mirror maker frequently but I would like to
> have a script to automate the start and stop.
>
> Thanks a lot!
> Qian Zhu
>
-
ndering if it s the case if we don t use keys (pure queuing
> system with key=null).
>
> Cheers
> Nico
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
;
> > If it is true, I don t get why the message has to be decoded by Kafka. I
> > would assume that whether the message is encrypted or not, Kafka simply
> > receives it, appends it to the file, and when a consumer wants to read
> it,
> > it simply reads at the right offse
ub.com/ktls/af_ktls
> http://www.phoronix.com/scan.php?page=news_item&px=FreeBSD-Faster-Sendfile
>
> Ismael
>
> On Mon, Mar 6, 2017 at 4:18 PM, Todd Palino wrote:
>
> > So that’s not quite true, Hans. First, as far as the performance hit
> being
> > not a big impact (25%
any unauthorized
> action based on information contained herein is strictly prohibited.
> Unauthorized use of information contained herein may subject you to civil
> and criminal prosecution and penalties. If you are not the intended
> recipient, please immediately notify the sender by t
r.properties and acks=all on
> producer? min.insync.replicas only applies when acks=all.
>
> -James
>
> >
> > -Original Message-
> > From: Todd Palino [mailto:tpal...@gmail.com]
> > Sent: Monday, March 06, 2017 6:48 PM
> > To: users@kafka.apache.org
> > S
ms to suggest no, but doesn't
> address the point directly:
> http://events.linuxfoundation.org/sites/events/files/slides/
> Kafka%20At%20Scale.pdf
>
> --
> Jack Foy
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
notify the sender by reply e-mail immediately and
> destroy all copies of the e-mail and any attachments.
>
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
at the right offset...
> >
> > Also I m wondering if it s the case if we don t use keys (pure queuing
> > system with key=null).
> >
> > Cheers
> > Nico
> >
> >
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
he broker to
> failover it to other brokers. Is there a way to failover controller to a
> specific broker? Is there a way to failover it without restart the broker?
>
> Thanks,
> Jun
--
*Todd Palino*
Senior Staff Engineer, Site Reliability
Data Infrastructure Streaming
linkedin.com/in/toddpalino
pics now? Do we need to come up
> with a manual assignment plan for each of the partitions? Is there any
> quicker way to achieve this?
>
> Thanks!
>
--
*Todd Palino*
Senior Staff Engineer, Site Reliability
Data Infrastructure Streaming
linkedin.com/in/toddpalino
1
> > #zookeeper-path=/kafka-cluster/stormconsumers
> >
> > [tickers]
> > broker-offsets=20
> >
> > [lagcheck]
> > intervals=10
> > expire-group=604800
> >
> > [notify]
> > interval=10
> >
> > [httpserver]
> > server=on
> > port=8000
> > ; Alternatively, use listen (cannot be specified when port is)
> > ; listen=host:port
> > ; listen=host2:port2
> >
> > [smtp]
> > server=mailserver.example.com
> > port=25
> > from=burrow-nore...@example.com
> > template=config/default-email.tmpl
> >
> > [emailnotifier "b...@example.com"]
> > group=local,critical-consumer-group
> > group=local,other-consumer-group
> > interval=60
> >
> > [notify]
> > interval=10
> >
> > [httpnotifier]
> > url=http://notification.server.example.com:9000/v1/alert
> > interval=60
> > extra=app=burrow
> > extra=tier=STG
> > template-post=config/default-http-post.tmpl
> > template-delete=config/default-http-delete.tmpl
> > timeout=5
> > keepalive=30
> >
> > So Can you please let me know what I am missing and how to fix these
> > issues.Any help would be appreciated.
> >
> >
> >
> > Regards,
> > Abhimanyu
> >
>
--
*Todd Palino*
Senior Staff Engineer, Site Reliability
Data Infrastructure Streaming
linkedin.com/in/toddpalino
We're not at the moment, but I'd definitely be interested in hearing your
results if you do. We're going to be experimenting with the latest version
soon to evaluate it.
-Todd
On 2/14/14 4:32 PM, "Clark Breyman" wrote:
>Is anyone running 0.8 (or pre-0.8.1) with the latest Zookeeper? Any known
>
Both the producer and consumer clients will take care of rebalancing if
there is a broker failure. The consumer handles this entirely in Zookeeper.
For the producer side, you do need some sort of front end to handle the
metadata requests, or else you would constantly have to change your
producer c
, Atig wrote:
>
>> Hello
>>
>> Can you elaborate on what is your producer setup in terms of fetching
>>the
>> metadata (in conjunction with a load balancer) and some sort of code
>> skeleton? New to Kafka here.
>>
>> Also, if you don't mind sharing wha
Filehandles can definitely be a concern here, but you can mitigate it to
some extent by adding more brokers to the cluster. The number of open file
handles is going to be driven in large part by the number of log files on
disk. This, in turn, is governed by the number of partitions and how many
fil
The numbers are offsets, which means that the lag is computed as ³number
of messages², not bytes. I know the header ³logSize² can be a little
confusing with regards to that.
-Todd
On 4/4/14, 12:32 AM, "S L" wrote:
>Hi,
>
>Just wondering what the units for the results from the
>ConsumerOffsetCh
In 0.8, those metrics are in messages. Previously, they were in bytes.
-Todd
On 4/7/14, 1:19 PM, "Harshvardhan Chauhan" wrote:
>Hi,
>
>For kafka 0.8 what are the units of consumer offset,log size and lag
>reported by the ConsumerOffsetChecker tool ?
>
>Thanks,
>--
>*Harshvardhan Chauhan* |
We¹ve got a script that rebalances the partitions in a cluster based on
their size (to try and keep the data size across the brokers even), which
works very well for moving partitions onto new cluster members. The only
problem with it is that it¹s got a couple hooks into our internal
configuration
We typically run all of our Zookeeper instances separate, but we do have
one Kafka cluster that is colocated with the Zookeeper nodes. It works
just fine, probably in part because Zookeeper handles everything serially.
The caveat is that the cluster that we¹re doing this on is not designed
for perf
ka.apache.org/documentation.html#java are
>old.
>We meant to update the documentation with the new GC settings using the G7
>collector, but we haven't gotten around to doing that. Let me reach out to
>our engineer, Todd Palino, who worked on tuning GC for Kafka at LinkedIn
>
Well, if you have a cluster in each datacenter, all with the same topics,
you can¹t just mirror the messages between them, as you will create a
loop. The way we do it is to have a ³local² cluster and an ³aggregate²
cluster. The local cluster has the data for only that datacenter. Then we
run mirror
ducer) due to cross data center network issue, mirror maker need to
>be placed along with the target cluster so that this scenario is minimized
>?
>
>
>On Sat, May 10, 2014 at 11:39 PM, Todd Palino
>wrote:
>
>> Well, if you have a cluster in each datacenter, all wit
msgs
>to
>target cluster, it can be solved by disabling auto-commit. and only commit
>msgs that are actually persisted in target cluster.
>
>what do you think of this opposite approach?
>
>
>On Sun, May 11, 2014 at 8:48 PM, Todd Palino wrote:
>
>> Yes, on both
014 at 9:40 AM, Jay Kreps wrote:
>>
>> > Hey Todd,
>> >
>> > Doc patch? :-)
>> >
>> > svn co http://svn.apache.org/repos/asf/kafka/site/081/
>> >
>> > Don't stress about html or formatting, I'm happy to do that part. I
&
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,
I think that¹s one option. What I would offer here is that we need to
separate out the concepts of authorization and authentication.
Authentication should definitely be modular, so that we can plug in
appropriate schemes depending on the organization. For example, you may
want client certificates,
No, at-rest encryption is definitely important. When you start talking
about data that is used for financial reporting, restricting access to it
(both modification and visibility) is a critical component.
-Todd
On 6/5/14, 2:01 PM, "Jay Kreps" wrote:
>Hey Joe,
>
>I don't really understand the s
reason I am pushing on these things a bit is because I want to make
>sure we don't end up with a set of requirements so broad we can never
>really get them implemented...
>
>-Jay
>
>
>
>
>On Thu, Jun 5, 2014 at 2:05 PM, Todd Palino
>wrote:
>
>> No, at-rest e
Yes, I realized last night that I needed to be clearer in what I was
saying. Encryption should ABSOLUTELY not be handled server-side. I think
it¹s a good idea to enable use of it in the consumer/producer, but doing
it server side will not solve many use cases for needing encryption
because the serv
y management is a
>significant issue. This is best handled internally, with key
>management stored in either ZK or in a topic. Truly, perhaps
>annealing Hadoop/HBASE as a metadata store seems applicable.
>
>Thanks, another 2 cents,
>Rob
>
>On Jun 6, 2014, at 12:15 PM, Todd Pal
internal at-
>rest encryption being an unchanging OTP with ACL access to it as the
>upper layer. Are you saying it is possible to re-encrypt with new
>keys or that there is a chain of keys over time?
>
>Thanks,
>Rob
>
>On Jun 8, 2014, at 3:06 PM, Todd Palino wrote:
>
>> I
ption metadata for that msg. All good.
>>>
>>> I don't see your OTP example. Could you delve deeper for me, please?
>>> The model I envision is internal OTP, with access to decryption
>>> accessed by cert. A double layer of security, with the inte
at strikes me as an opportunity is to define a plug gable at-rest
>encryption module interface, that supports each/both of our security
>needs.
>
>Thanks,
>Rob
>
>> On Jun 10, 2014, at 4:01 PM, Todd Palino
>>wrote:
>>
>> The situation of production before
We¹re about to start testing in the next month. As soon as I have 1.8 u5
on the systems to work with. I don¹t anticipate too much of an issue,
since 7 -> 8 is supposed to be less trouble than 6 -> 7, and we didn¹t
have any problems moving to 7.
-Todd
On 6/17/14, 6:22 AM, "Chris Neal" wrote:
>H
You¹re out of date, Jun. We¹re up to 20 now :)
Our ops presentation on Kafka is a little more up to date on numbers:
http://www.slideshare.net/ToddPalino/enterprise-kafka-kafka-as-a-service
-Todd
On 7/7/14, 7:21 AM, "Jun Rao" wrote:
>LinkedIn's largest Kafka cluster has 16 nodes now. More det
d. Any thoughts of going JBOD? In
>our testing we saw significant performance improvements. This of course
>comes with trade off of manual steps if brokers fail.
>
>Bert
>
>On Monday, July 7, 2014, Todd Palino wrote:
>
>> You¹re out of date, Jun. We¹re up to 20 now
The doc is not up to date. What you are seeing in ZK is what is expected.
The broker information about the partition is found in:
/brokers/topics/[topic]/partitions/[partition_id]/state
The node data will tell you what ID is the current leader, as well as the
ISR list.
-Todd
On 7/14/14, 12:11
The MaxLag mbean is only valid for an active consumer. So while the
consumer is actively running, it should be accurate (as far as telling you
the lag on the partition that you are consuming that has the most lag).
However, if you look at the output from the ConsumerOffsetChecker, you see
that the
You need to consider your total partition count as you do this. After 30
days, assuming 1 partition per topic, you have 7200 partitions. Depending
on how many brokers you have, this can start to be a problem. We just
found an issue on one of our clusters that has over 70k partitions that
there¹s no
>our topic has 3 partitions, so its around 3*240*3 =2160 partitions. Since
>there is no api for deleting topic, i guess i could set up a cron job
>deleting the out dated topics(folders) from zookeeper..
>do you know when the delete topic api will be available in kafka?
>Chen
>
>
&
ood
>experience
>in using this many topics in your system?:-)
>Thanks,
>Chen
>
>
>On Mon, Aug 11, 2014 at 9:02 PM, Todd Palino
>
>wrote:
>
>> In order to delete topics, you need to shut down the entire cluster (all
>> brokers), delete the topics from Zookee
101 - 179 of 179 matches
Mail list logo