You'll need a ZK quorum established before brokers boot, for sure.
On Thu, Jul 20, 2017 at 12:53 PM, M. Manna wrote:
> Hello,
>
> This might be too obvious for some people, but just thinking out loud here.
>
> So we need a recommended 3 node cluster to achieve the 1 point
All broker versions support all older client versions
On Tue, Jul 18, 2017 at 10:15 AM, Sachin Mittal wrote:
> Hi,
> This gives me some information but still not the complete picture.
>
> It says:
> 0.10.2, Java clients have acquired the ability to communicate with older
>
Hi,
Which folder are you storing kafka's data in? By default that's /tmp, which
might be getting wiped by your OS.
Thanks
Tom Crayford
Heroku Kafka
On Fri, Jul 14, 2017 at 11:50 AM, mosto...@gmail.com <mosto...@gmail.com>
wrote:
> anyone?
>
>
>
> On 13/07/17 17:09, mos
, this is shaping up to be a great release. We're going to
continue some further testing, but right now are heading towards a +1.
Thanks
Tom Crayford
Heroku Kafka
On Fri, Jun 23, 2017 at 2:36 AM, Ismael Juma <ism...@juma.me.uk> wrote:
> A quick note on notable changes since r
and shouldn't delay a release documented
somewhere?
Thanks
Tom Crayford
Heroku Kafka
On Thu, Jun 22, 2017 at 4:45 PM, Ismael Juma <isma...@gmail.com> wrote:
> Hi Tom,
>
> We are going to do another RC to include Apurva's significant performance
> improvement when transactions are
e though, since it
> doesn't really affect the usability of the feature any way.
>
> Thanks,
> Apurva
>
> On Wed, Jun 21, 2017 at 11:12 AM, Tom Crayford <tcrayf...@heroku.com>
> wrote:
>
> > Hi there,
> >
> > I'm -1 (non-binding) on shipping this RC.
> >
/apache/kafka/pull/3398 (the change is very small). Happy
to make a jira as well, if that makes sense.
Thanks
Tom Crayford
Heroku Kafka
On Tue, Jun 20, 2017 at 8:32 PM, Vahid S Hashemian <
vahidhashem...@us.ibm.com> wrote:
> Hi Ismael,
>
> Thanks for running the release.
&g
ature to be perfect in this
release. We expect to test this this week though.
Given that the blockers fixed between RC0 and RC1 haven't changed much in
the areas we tested, I think the positive results here still apply.
Thanks
Tom Crayford
Heroku Kafka
On Thu, Jun 8, 2017 at 2:55 PM, Ismael Juma
Is there a summary of which blockers were fixed in RC0 somewhere?
On Mon, Jun 19, 2017 at 1:41 PM, Eno Thereska
wrote:
> +1 (non-binding) passes Kafka Streams tests.
>
> Thanks,
> Eno
> > On 19 Jun 2017, at 06:49, Magnus Edenhill wrote:
> >
> > +1
are starting out.
Thanks,
Tom Crayford
Heroku Kafka
On Sat, Jun 3, 2017 at 8:20 AM, Michal Borowiecki <
michal.borowie...@openbet.com> wrote:
> I'm not an expert but I prefer keeping zookeepers on the same hosts as
> kafka brokers and mimic each-others topology. The reason is to minimize
Indeed I will not have more than 10 or 20 customer per cluster, so that's
> also the maximum number of partitions possible per topic.
> Still a bad idea?
>
> 2017-05-23 16:48 GMT+02:00 Tom Crayford <tcrayf...@heroku.com>:
>
> > Hi there,
> >
> > I don't know abo
ver expect to have
*very* small customer numbers (hundreds at most, ever). Instead, use a hash
function and a key, as recommended to land customers on the same partition.
Thanks
Tom Crayford
Heroku Kafka
On Tue, May 23, 2017 at 9:46 AM, David Espinosa <espi...@gmail.com> wrote:
> Hi,
&
ver expect to have
*very* small customer numbers (hundreds at most, ever). Instead, use a hash
function and a key, as recommended to land customers on the same partition.
Thanks
Tom Crayford
Heroku Kafka
On Tue, May 23, 2017 at 9:46 AM, David Espinosa <espi...@gmail.com> wrote:
> Hi,
&
Fathima,
In 0.11 there will be such a mechanism (see KIP-98), but in current
versions, you have to eat the duplicates if you want to not lose messages.
On Wed, May 17, 2017 at 5:31 AM, Fathima Amara wrote:
> Hi Mathieu,
>
> Thanks for replying. I've already tried by setting
Hi,
You should upgrade Kafka versions, this was a bug fixed in KAFKA-3894:
https://issues.apache.org/jira/browse/KAFKA-3894
Generally it's a very good idea to keep on top of Kafka version upgrades,
there are numerous bugs fixed with every release, and it's stability goes
up each time.
On Tue,
+1 (non-binding)
I didn't explicitly state my voting status in my previous comment, sorry.
On Thu, Feb 16, 2017 at 1:59 PM, Rajini Sivaram
wrote:
> +1 (non-binding)
>
> Ran quick start and some security tests on binary, checked source build and
> tests.
>
> Thank you,
Heroku tested this with our usual round of performance benchmarks, and
there seem to be no notable regressions in this RC that we can see (for a
sample on earlier regressions we found using these benchmarks during the
0.10.0.0 release,
+1
On Wed, Jan 4, 2017 at 5:28 PM, Gwen Shapira wrote:
> +1 - thanks for tackling those old and painful bugs!
>
> On Wed, Jan 4, 2017 at 9:24 AM, Ben Stopford wrote:
> > Hi All
> >
> > We’re having some problems with this thread being subsumed by the
>
This is confluent documentation, not Apache documentation. I'd recommend
talking to Confluent about that.
On Mon, Dec 12, 2016 at 4:57 AM, Sven Ludwig wrote:
> Hi,
>
> in JMX each Kafka broker has a value 1 or 0 for ActiveControllerCount. As
> I understood from this thread, the
it and run e.g. health checks
and such before restarting the next node.
Thanks
Tom Crayford
Heroku Kafka
>
> On Mon, Oct 3, 2016 at 9:27 PM, Tom Crayford <tcrayf...@heroku.com> wrote:
>
> > Yes, offset topic compaction is just the normal compaction.
> >
> > Th
Kafka doesn't support time delays at all, no.
On Thu, Oct 6, 2016 at 12:14 AM, Akshay Joglekar <
akshay.jogle...@danalinc.com> wrote:
> Hi,
>
> I have a use case where I need to process certain messages only after a
> certain amount time has elapsed. Does Kafka have any support for time
>
Yes, offset topic compaction is just the normal compaction.
Thanks
Tom Crayford
Heroku Kafka
On Monday, 3 October 2016, Tobias Adamson <tob...@stargazer.com.sg> wrote:
> Hi
> We are using Kafka 0.10.1 with offsets commits being stored inside of Kafka
> After a while thes
You'll need to do a rolling restart of your kafka nodes after changing the
zookeeper ensemble. There's no real way around that right now.
On Sun, Sep 25, 2016 at 6:41 PM, Ali Akhtar wrote:
> Perhaps if you add 1 node, take down existing node, etc?
>
> On Sun, Sep 25, 2016
0.10).
I'd recommend tracking reads somewhere else, but it may be somewhat
difficult. You could also potentially use consumer offsets for this - if
your consumer is storing offsets in Kafka anyway.
Thanks
Tom Crayford
Heroku Kafka
>
> Does this sound like a saner way?
>
> Cheers,
> Fran
is really
designed to operate with replication.
Thanks
Tom Crayford
Heroku Kafka
On Tue, Sep 6, 2016 at 5:30 AM, Harald Kirsch <harald.kir...@raytion.com>
wrote:
> Hi all,
>
> there are so many timeouts to tweak mentioned in the documentation that I
> wonder what the correct configur
This sounds like Kafka not being entirely robust to disk corruption, which
seems entirely possible and normal. I'd simply delete that log file and let
a replica replay catch it up at broker bootup.
Trying to guard against all possible disk corruption bugs sounds very
difficult to me, it seems
if you're running that at a bash or similar shell, you need to quote the
"*" so that bash doesn't expand it as a glob:
./kafka-acls.sh --authorizer-properties zookeeper.connect=
--add --allow-principal User:"user01" --topic 'com.domain.xyz.*' --group
group01 --operation read
It may be
inline
On Mon, Sep 5, 2016 at 12:00 AM, F21 wrote:
> Hi all,
>
> I am currently looking at using Kafka as a "message bus" for an event
> store. I plan to have all my events written into HBase for permanent
> storage and then have a reader/writer that reads from HBase to
It is not applied. An upcoming release will have the ability to combine the
two, but right now they are mutually exclusive.
On Thu, Sep 1, 2016 at 6:18 PM, David Yu wrote:
> Hi,
>
> Does Kafka "log.retention.bytes" or "log.retention.ms" apply to compaction
> enabled topic?
Are you running with unclean leader election on? Are you setting min in
sync replicas at all?
Can you attach controller and any other logs from the brokers you have?
They would be crucial in debugging this kind of issue.
Thanks
Tom Crayford
Heroku Kafka
On Thursday, 11 August 2016, Mazhar
as well.
Thanks
Tom Crayford
Heroku Kafka
On Wednesday, 10 August 2016, Christiane Lemke <christiane.le...@gmail.com>
wrote:
> Hi all,
>
> I am trying to set up a minimal example understanding log compaction
> behaviour using kafka-clients-0.10.0.0.jar. I got the compaction b
Heroku has tested this using the same performance testing setup we used to
evaluate the impact of 0.9 -> 0.10 (see https://engineering.
heroku.com/blogs/2016-05-27-apache-kafka-010-evaluating-
performance-in-distributed-systems/).
We see no issues at all with them, so +1 (non-binding) from here.
or plaintext. If they're going over plaintext you'll need to
authenticate those hosts. If they're going over TLS, you'll need to ensure
they're using the right client certs.
Thanks
Tom Crayford
Heroku Kafka
On Friday, 5 August 2016, Wannes De Smet <wannes...@gmail.com> wrote:
> Hi a
sample producer configs and code, maybe this would help in
diagnosing the issue.
Thanks
Tom Crayford
Heroku Kafka
On Friday, 5 August 2016, David Yu <guans...@gmail.com> wrote:
> We are using Avro as our format when writing to a Kafka topic. After we
> enabled snappy compression on
Kafka can't by itself do aggregation (nor does it really make sense for it
to). You can build such a feature on top of log compaction relatively
easily (by sending the new count as a message under an individual key), or
you can use the KTable and aggregation features of Kafka Streams.
Thanks
Tom
You have to run kafka-reassign-partitions.sh script to move partitions to a
new replica id.
On Wed, Aug 3, 2016 at 3:14 AM, Digumarthi, Prabhakar Venkata Surya <
prabhakarvenkatasurya.digumar...@capitalone.com> wrote:
> Hi ,
>
>
> I am right now using kafka version 0.9.1.0
>
> If I choose to
Hi there,
What version of Kafka are you using? Can you share your config files and
any sample code?
Thanks
Tom Crayford
Heroku Kafka
On Monday, 1 August 2016, Benny Ho <blazebenn...@yahoo.com.invalid> wrote:
> Hello,
> I'm receiving an error while publishing messages to a
Hi there,
It's enabled with the config log.cleaner.enable
Thanks
On Wed, Jul 20, 2016 at 5:29 PM, Anderson Goulart <
anderson.goul...@boxever.com> wrote:
> Hi,
>
> How can I see if log compaction is enabled? And how can I enable it? I
> didn't find it on kafka docs.
>
>
> Thanks, Anderson
>
>
Manikumar,
How will that help? Increasing the number of log cleaner threads will lead
to *less* memory for the buffer per thread, as it's divided up among
available threads.
Lawrence, I'm reasonably sure you're hitting KAFKA-3587 here, and should
upgrade to 0.10 ASAP. As far as I'm aware Kafka
Anderson,
The metric `UnderReplicatedPartitions` gives you the number of partitions
that are out of the ISR, but doesn't expose that per topic-partition. I'd
welcome the addition of that metric to Kafka (it shouldn't be that much
work in the source code I don't think), and I think others would as
Also which version of Kafka are you using?
On Mon, Jul 18, 2016 at 7:16 PM, Guozhang Wang wrote:
> This is un-expected. Any error logs / exceptions did you see from the
> clients when they can no longer send / fetch from brokers?
>
> Guozhang
>
> On Mon, Jul 18, 2016 at 8:59
Hi Jack,
No, kafka doesn't support not writing to disk. If you're really 100% sure
of yourself you could use a ramdisk and mount Kafka on it, but that's not
supported. I'd recommend "just" writing to disk, it's plenty fast enough
for nearly all use cases.
Thanks
Tom Crayford
He
Also note that there were a number of bugs fixed in the log cleaner thread
between 0.8 and the latest release. I wouldn't be comfortable relying on
kafka committed offsets on a version under 0.10 for a new production
system, and would carefully consider an upgrade all the way to the latest
, you have to
remediate by increasing the dedupe buffer size
We're exploring solutions in KAFKA-3894, and would love your feedback there
if you have any thoughts.
Thanks
Tom Crayford
Heroku Kafka
On Wednesday, 13 July 2016, Rakesh Vidyadharan <rvidyadha...@gracenote.com>
wrote:
> We ran into
ce is set to true, so rebalancing may be happening at
> that time. Is there any issue tracker that I can refer to?
> On 12 Jul 2016 21:48, "Tom Crayford" <tcrayf...@heroku.com
> <javascript:_e(%7B%7D,'cvml','tcrayf...@heroku.com');>> wrote:
>
>> Hi,
>>
>&
Hi,
Were you rebalancing that topic or partition at that time? If there are
rebalancing bugs this might point at that.
Thanks
Tom
On Tue, Jul 12, 2016 at 6:47 AM, Gokul wrote:
> We had an issue last week when kafka cluster reported under replicated
> partitions for quite a
eap memory, yet the actual process memory
is more like 10GB.
Thanks
Tom Crayford
Heroku Kafka
not perfect and
not great throttling, and I agree that it's something Kafka desperately
needs to work on.
Thanks
Tom Crayford
Heroku Kafka
On Sun, Jul 3, 2016 at 2:00 AM, Charity Majors <char...@hound.sh> wrote:
> Hi there,
>
> I'm curious if there's anything on the Kafka roadmap fo
d do we need to clean those folders? is there any configuration?
>
> - Original Message -
> From: "Tom Crayford" <tcrayf...@heroku.com>
> To: "Users" <users@kafka.apache.org>
> Sent: Thursday, June 30, 2016 6:11:03 PM
> Subject: Re: wh
Hi there,
Kafka uses this topic internally for consumer offset commits.
Thanks
Tom Crayford
Heroku Kafka
On Thu, Jun 30, 2016 at 1:36 PM, Snehalata Nagaje <
snehalata.nag...@harbingergroup.com> wrote:
>
> Hi All,
>
>
> I am using kafka 9 version with publish subscribe
The default cleanup policy is delete, which is the regular time based
retention.
On Thursday, 30 June 2016, Sathyakumar Seshachalam <
sathyakumar_seshacha...@trimble.com> wrote:
> Or may be am wrong, and Log cleaner only picks up topics with a
> cleanup.policy.
> From the documentation it is not
I think you'll be far better off using EBS and Kafka's inbuilt distribution
than NFS mounts. Kafka's designed for distributing data natively, and not
for NFS style mounts.
On Wed, Jun 29, 2016 at 11:46 AM, Ben Davison
wrote:
> Does anyone have any opinions on this?
>
>
> be deleted by Kafka?
>
>
>
> --
> κρισhναν
>
> On Thu, Jun 23, 2016 at 6:16 PM, Tom Crayford <tcrayf...@heroku.com>
> wrote:
>
>> Hi,
>>
>> A pretty reasonable thing to do here would be to have a consumer that
>> moved "old" events to
ven higher (LinkedIn use 4 days, the default is
7 days).
Thanks
Tom Crayford
Heroku Kafka
On Thu, Jun 23, 2016 at 10:45 AM, Krish <krishnan.k.i...@gmail.com> wrote:
> Hi,
> I am trying to design a real-time application where message timeout can be
> as low as a minute or two (messag
Cleaned 100.0 MB in 1.0 seconds (103.6 Mb/sec, 48.5% of total time)
> Start size: 100.0 MB (928,011 messages)
> End size: 0.0 MB (97 messages)
> 100.0% size reduction (100.0% fewer messages)
> (kafka.log.LogCleaner)
>
> But no actual delete messages like a properly-working broker is showing of
> a different partition.
>
> Lawrence Weikum
>
>
> On 6/22/16, 11:28 AM, "Tom Crayford" <tcrayf...@heroku.com> wrote:
>
> kafka-log-cleaner-thread-0
>
>
Is the log cleaner thread running? We've seen issues where the log cleaner
thread dies after too much logged data. You'll see a message like this:
[kafka-log-cleaner-thread-0], Error due to
java.lang.IllegalArgumentException: requirement failed: 9750860 messages in
segment
Did you check if the controller is active in the cluster? If the controller
isn't active (there are known 0.8 bugs that can lead to this), then this
could cause this kind of data loss issue. I recommend upgrading to 0.9 ASAP.
Thanks
Tom Crayford
Heroku Kafka
On Friday, 17 June 2016, Gulia
+1
We can't promise security features whilst using a deprecated version of the
JDK and relying on it for underlying security functionality (e.g. SSL).
This impacts both clients and brokers. Java 7 has been deprecated for over
a year, and software that isn't up to date with that is at fault with
Hi Mudit,
Sorry this is not possible. The only deletion Kafka offers is retention or
whole topic deletion.
Thanks
Tom Crayford
Heroku Kafka
On Tuesday, 14 June 2016, Mudit Kumar <mudit.ku...@askme.in> wrote:
> Hey,
>
> How can I delete particular messages from p
Kafka itself handles distribution among brokers and which broker consumers
and producers connect to. There's no need for an ELB, and you have to
directly expose all brokers to producers and consumers.
On Friday, 10 June 2016, Ram Chander wrote:
> Hi,
>
>
> I am trying to
or disk bandwidth, so colocating brokers makes
that much more likely.
Thanks
Tom Crayford, Heroku Kafka
On Fri, Jun 10, 2016 at 7:02 AM, Barry Kaplan <bkap...@memelet.com> wrote:
> If too much heap cause problems, would it make sense to run multiple
> brokers on a box with lots memory?
Hi Lawrence,
What JVM options were you using? There's a few pages in the confluent docs
on JVM tuning iirc. We simply use the G1 and a 4GB Max heap and things work
well (running many thousands of clusters).
Thanks
Tom Crayford
Heroku Kafka
On Thursday, 9 June 2016, Lawrence Weikum <l
No. These versions and all versions 0.8 onwards rely on Zookeeper.
On Wednesday, 8 June 2016, Subhash Agrawal wrote:
> Hi,
> I am currently using Kafka 0.7.1 without zookeeper. We have single node
> kafka server.
> To enhance security, we have decided to support SSL. As
Tom Crayford
Heroku Kafka
On Fri, Jun 3, 2016 at 1:15 PM, cs user <acldstk...@gmail.com> wrote:
> Hi All,
>
> Does anyone have any experience of using kafka behind a load balancer?
>
> Would this work? Are there any reasons why you would not want to do it?
>
> Thanks!
>
to the few tens of KB for most use cases.
Thanks
Tom Crayford
Heroku Kafka
On Wed, Jun 1, 2016 at 9:49 PM, prateek arora <prateek.arora...@gmail.com>
wrote:
> I have 4 node kafka broker with following configuration :
>
> Default Number of Partitions : num.partitions : 1
> Default
That looks like somebody is killing the process. I'd suspect either the
linux OOM killer or something else automatically killing the JVM for some
reason.
For the OOM killer, assuming you're on ubuntu, it's pretty easy to find in
/var/log/syslog (depending on your setup). I don't know about other
we've found that the kinds of uses of Kafka that require in
message encryption (alongside full disk encryption and SSL which we provide
as standard) don't have such high throughput needs that they worry about
compression etc. That clearly isn't true for all use cases though.
Thanks
Tom Crayford
Ok. I'd recommend upgrading to 0.9 asap to fix the known bugs in 0.8 here.
Thanks
Tom Crayford
Heroku Kafka
On Wed, Jun 1, 2016 at 3:27 AM, Fredo Lee <buaatianwa...@gmail.com> wrote:
> we use 0.8.2.2.
>
> 2016-05-31 20:14 GMT+08:00 Tom Crayford <tcrayf...@heroku.com>:
&g
Nope. You should upgrade to Kafka 0.9, assuming that your
ActiveControllerCount across all brokers is 0 or more than 1 (which is
typically the failure case we see).
Thanks
Tom Crayford
Heroku Kafka
On Wed, Jun 1, 2016 at 3:22 AM, Fredo Lee <buaatianwa...@gmail.com> wrote:
> we us
Is this under 0.8? There are a few known bugs in 0.8 that can lead to this
situation. I'd recommend upgrading to 0.9 as soon as is viable to prevent
this and many other kinds of issues that were fixed in 0.9.
Thanks
Tom Crayford
Heroku Kafka
On Tue, May 31, 2016 at 6:19 AM, Fredo Lee
Hi,
Which version of Kafka are you running? We run thousands of clusters, and
typically use this mechanism for replacing damaged hardware, and we've only
seen this issue under Kafka 0.8, where the controller can get stuck (due to
a few bugs in Kafka) and not be functioning. If you are on 0.8, I'd
issues.
Thanks
Tom Crayford
Heroku Kafka
On Fri, May 27, 2016 at 1:33 PM, Jan Algermissen <algermissen1...@icloud.com
> wrote:
> Hi,
>
> I have a producer question: Is the producer (specifically the normal Java
> producer) using the file system in any way?
>
> If it does
limiter here.
Thanks
Tom Crayford
Heroku Kafka
On Wednesday, 25 May 2016, Joe San <codeintheo...@gmail.com> wrote:
> I do not mind the ordering as I have a Timestamp in all my messages and all
> my messaged land in a Timeseries database. So I understand that it is
> better to have ju
By process I mean a JVM process (if you're using the JVM clients and for
your app).
Thanks
Tom Crayford
Heroku Kafka
On Wednesday, 25 May 2016, Hafsa Asif <hafsa.a...@matchinguu.com> wrote:
> A very good question from Joe. I have also the same question.
>
> Hafsa
>
> 2016-
ch.bytes", "128");
> consumer = new KafkaConsumer<String, String>(props);
>
> I am setting the max.partition.fetch.bytes to 128, because I only want to
> process one record for each poll.
>
> Thank a lot for your help. I really appreciate it.
>
&g
If you're using EBS then it's a single flag to use encrypted drives at the
provision time of the volume. I don't know about the other storage options,
I'd recommend looking at the AWS documentation.
Thanks
Tom Crayford
Heroku Kafka
On Wednesday, 25 May 2016, Snehalata Nagaje <
snehalata.
Stepanov, Ishita Mandhan,
> > > Ismael Juma, Jaikiran Pai, Jakub Nowak, James Cheng, Jason Gustafson,
> > > Jay Kreps, Jeff Klukas, Jeremy Custenborder, Jesse Anderson, jholoman,
> > > Jiangjie Qin, Jin Xing, jinxing, Jonathan Bond, Jun Rao, Ján Koščo,
> > > Kaufman
Aha, yep that helped a lot.
One producer per process. There's not really a per producer topic limit.
There's buffering and batching space, but assuming you have sufficient
memory (which is by the partition, not by topic), you'll be fine.
Thanks
Tom Crayford
Heroku Kafka
On Tue, May 24, 2016
t; can convince my team in a good way.
>
> Best Regards,
> Hafsa
>
> 2016-05-24 16:11 GMT+02:00 Tom Crayford <tcrayf...@heroku.com>:
>
> > Is that "one instance of the producer class per topic"? I'd recommend
> just
> > having a single producer shared per pr
No real precautions need to be taken on starting a down instance, assuming
you have replication in play and the controller is up and active. We
routinely restart downed broker processes, and have never had an issue with
it (running thousands of clusters with an ops team of ~3).
Thanks
Tom
What's your server setup for the brokers and consumers? Generally I'd
expect something to be exhausted here and that to end up being the
bottleneck.
Thanks
Tom Crayford
Heroku Kafka
On Mon, May 23, 2016 at 7:32 PM, Yazeed Alabdulkarim <
y.alabdulka...@gmail.com> wrote:
> Hi,
>
ond on our clusters). Or maybe your load is spikier than
that?
Generally if you have multiple producer instances they will fail slightly
differently, but most failures that hit one (e.g. a broker going down and
the controller not changing over the leader fast enough).
Thanks
Tom Crayford
He
be the biggest issue. Before we lost our
> metrics I noticed a lot of leader change activity - could this be a symptom
> of the offline partitions?
>
> Kind regards,
>
> Jahn Roux
>
>
> -Original Message-
> From: Tom Crayford [mailto:tcrayf...@heroku.com]
>
?
Are you monitoring Kafka's internal metrics on each broker? Issues with
e.g. offline partitions and other things could cause that kind of impact.
Thanks
Tom Crayford
Heroku Kafka
On Tue, May 24, 2016 at 9:56 AM, Jahn Roux <j...@comprsa.com> wrote:
> Thank you for the response. Yes, we
Hi,
There's no encryption at rest. It's recommended to use filesystem
encryption, or encryption of each individual message before producing it
for this.
Only the new producer and consumers have SSL support.
Thanks
Tom Crayford
Heroku Kafka
On Tue, May 24, 2016 at 11:33 AM, Snehalata Nagaje
producer!
>
> On Mon, May 23, 2016 at 2:22 PM, Tom Crayford <tcrayf...@heroku.com>
> wrote:
>
> > That's accurate. Why are you creating so many producers? The Kafka
> producer
> > is thread safe and *should* be shared to take advantage of batching, so
> I'd
&g
That's accurate. Why are you creating so many producers? The Kafka producer
is thread safe and *should* be shared to take advantage of batching, so I'd
recommend just having a single producer.
Thanks
Tom Crayford
Heroku Kafka
On Mon, May 23, 2016 at 10:41 AM, Joe San <codeintheo...@gmail.
H there,
You could probably wrangle this with log4j and filters. A single broker
doesn't really have a consistent view of "if the cluster goes down", so
it'd be hard to log that, but you could write some external monitoring that
checked brokers were up via JMX and log from there.
T
Hi there,
The missing piece is the config `log.roll.hours` or it's alternative `
log.roll.ms`. Log segments are by default rolled once a week, regardless of
activity, but you can tune that down as you like.
Thanks
Tom Crayford
Heroku Kafka
On Fri, May 20, 2016 at 12:49 AM, James Cheng <wush
of consumers as partitions (but each consumer can have multiple
partitions).
That error message seems pretty bad - it sounds like the broker is falling
over repeatedly. I'd check the broker logs and metrics (see
http://docs.confluent.io/2.0.1/kafka/monitoring.html).
Thanks
Tom Crayford
Heroku
at
all.
The ACL for sure only works on the new consumer API, because the old one
talks directly to zookeeper so there's no good way to apply the same ACLs
there.
Thanks
Tom Crayford
Heroku Kafka
On Thu, May 19, 2016 at 1:28 AM, David Hawes <dha...@vt.edu> wrote:
> I have been playi
belong on a laptop that goes to sleep).
Lastly, because of this and many other issues we've seen in production with
0.8.X, I'd recommend upgrading to 0.9 ASAP - in our experience (running
thousands of production clusters), 0.9 is much more stable than 0.8.
Thanks
Tom Crayford,
Heroku Kafka
On Wed
The issue I have with that log.cleanup.policy approach is how it impacts
with replication. Different replicas often have different sets of segment
files. There are effectively two options then:
1. All replicas upload all their segments. This has issues with storage
space, but is easy to implement.
out well in production
for years for many companies. Partition or Topic per device is not a thing
that will work well, and definitely will never work on indefinite growth,
as zookeeper puts a max limit on the number of partitions in a Kafka
cluster.
Thanks
Tom Crayford
Heroku Kafka
>
> Thanks
Go developer so when possible please avoid Java centric
> terminology.
Please to note that the node and go clients are notably less mature than
the JVM clients, and that running Kafka in production means knowing enough
about the JVM and Zookeeper to handle that.
Thanks!
Tom Crayford
Heroku Kafka
>
> Thanks!
>
> - Oli
>
> --
> - Oli
>
> Olivier Lalonde
> http://www.syskall.com <-- connect with me!
>
Hi there,
The Kafka producer and consumer are libraries you run inside your
application. As such, the beans from them do not exist on the brokers.
Thanks
Tom Crayford,
Heroku Kafka
On Mon, May 16, 2016 at 8:29 PM, Russ Lavoie <russlav...@gmail.com> wrote:
> I am using JMX to gat
Hi,
Producers don't track offsets in the same way, so there is no producer
offset API.
Thanks
Tom Crayford
Heroku Kafka
On Mon, May 16, 2016 at 5:25 PM, Kanagha <er.kana...@gmail.com> wrote:
> Hi,
>
> I am trying to find out the API for committing producer offset for Ka
Hi there,
Generally you don't use a single topic per device in this use case, but one
topic with some number of partitions and the key distribution based on
device id. Kafka isn't designed for millions of low volume topics, but a
few high volume ones.
Thanks
Tom Crayford
Heroku Kafka
On Mon
Yep, confirm.
On Thu, May 12, 2016 at 9:37 PM, Gwen Shapira <g...@confluent.io> wrote:
> Just to confirm:
> You tested both versions with plain text and saw no performance drop?
>
>
> On Thu, May 12, 2016 at 1:26 PM, Tom Crayford <tcrayf...@heroku.com>
> wrote:
We've started running our usual suite of performance tests against Kafka
0.10.0.0 RC. These tests orchestrate multiple consumer/producer machines to
run a fairly normal mixed workload of producers and consumers (each
producer/consumer are just instances of kafka's inbuilt consumer/producer
perf
1 - 100 of 114 matches
Mail list logo