Re: ISR differs between Kafka Metadata and Zookeeper

2014-09-19 Thread Andrew Otto
I am seeing this behavior using librdkafka, as is another user. Listing the topic metadata with the tool provided with Kafka (kafka-topic.sh) shows all replicas in the ISR. However, using kafkacat[1] (built with librdkafka) shows that many ISRs are missing some replicas. I talked with Magnus (au

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

2014-09-29 Thread Andrew Otto
is case GC threads) to > flush the dirty pages. So, all those time in real was spent on disk I/Os, > rather than real GCs. The fix is to tune dirty_expire_centisecs and > dirty_writeback_centisecs > to flush dirty pages more frequently to avoid such drafting. > > Thanks, > > J

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

2014-10-01 Thread Andrew Otto
ISR? On Sep 30, 2014, at 7:17 PM, Jun Rao wrote: > With ack=1, acked messages could be lost when the leader fails. > > Thanks, > > Jun > > On Mon, Sep 29, 2014 at 8:04 AM, Andrew Otto wrote: > >> This happened again to me this weekend. I've done some sleuth

Re: Zookeeper reconnect failed due to 'state changed (Expired)'

2014-10-01 Thread Andrew Otto
ader. So, any unreplicated messages that didn't reach the > new leader are lost. If the old leader rejoins ISR, it will also truncate > it's log to follow the new leader's log. > > Thanks, > Neha > > On Wed, Oct 1, 2014 at 5:48 AM, Andrew Otto wrote: > >>

Re: Cross-Data-Center Mirroring, and Guaranteed Minimum Time Period on Data

2014-10-16 Thread Andrew Otto
Check out Camus. It was built to do parallel loads from Kafka into time bucketed directories in HDFS. On Oct 16, 2014, at 9:32 AM, Gwen Shapira wrote: > I assume the messages themselves contain the timestamp? > > If you use Flume, you can configure a Kafka source to pull data from > Kafka,

Re: powered by kafka

2014-11-10 Thread Andrew Otto
Oo, add us too! The Wikimedia Foundation (http://wikimediafoundation.org/wiki/Our_projects) uses Kafka as a log transport for analytics data from production webservers and applications. This data is consumed into Hadoop using Camus and to other processors of analytics data. > On Nov 10, 2

Re: How to push metrics to graphite - jmxtrans does not work

2014-12-02 Thread Andrew Otto
Maybe also set: -Dcom.sun.management.jmxremote.port= ? > On Dec 2, 2014, at 02:59, David Montgomery wrote: > > Hi, > > I am having a very difficult time trying to report kafka 8 metrics to > Graphite. Nothing is listening on and and no data in graphite. If > this method of graphi

Re: kafka jmxtrans config

2015-02-12 Thread Andrew Otto
Hm, I’m sure how long ago the kafka jmxtrans example has been updated on the puppet-jmxtrans module. Wikimedia is currently using Kafka 0.8.1.1 with this puppet module’s jmxtrans setup: https://github.com/wikimedia/puppet-kafka/blob/master/manifests/server/jmxtrans.pp > On Feb 12, 2015, at 10

Re: Announcing the Confluent Platform built on Apache Kafka

2015-02-25 Thread Andrew Otto
Wow, .deb packages. I love you. > On Feb 25, 2015, at 14:48, Joseph Lawson wrote: > > This is really awesome stuff. It's great to see y'all growing! Thank you > and congratulations! > > > From: Neha Narkhede > Sent: Wednesday, February 25, 2015 1:3

Re: REST/Proxy Consumer access

2015-03-05 Thread Andrew Otto
BTW, Wikimedia uses varnishkafka to produce http requests to Kafka, and we are pretty happy with it. https://github.com/wikimedia/varnishkafka > On Mar 5, 2015, at 13:09, Ewen Cheslack-Postava wrote: > > Yes, Confluent built a REST proxy that gives access to cluster metadata > (e.g. list to

Re: integrate Camus and Hive?

2015-03-11 Thread Andrew Otto
> Hive provides the ability to provide custom patterns for partitions. You > can use this in combination with MSCK REPAIR TABLE to automatically detect > and load the partitions into the metastore. I tried this yesterday, and as far as I can tell it doesn’t work with a custom partition layout. A

Re: integrate Camus and Hive?

2015-03-11 Thread Andrew Otto
TION '${env:TABLE_LOCATION_CAMUS_OUTPUT}' > ; > > > I hope this will help ! You will have to construct hive query according > to partition define. > > Thanks, > > Bhavesh > > On Wed, Mar 11, 2015 at 7:24 AM, Andrew Otto wrote: > >>> Hiv

Re: integrate Camus and Hive?

2015-03-11 Thread Andrew Otto
api/src/main/java/com/linkedin/camus/etl/Partitioner.java > and use configuration "etl.partitioner.class=CLASSNAME" then you can > organize any way you like. > > I hope this helps. > > > Thanks, > > Bhavesh > > > On Wed, Mar 11, 2015 at 8:36 AM, And

Re: integrate Camus and Hive?

2015-03-12 Thread Andrew Otto
generatePartitionedPath*(JobContext context, String > topic, String brokerId, int partitionId, String *encodedPartition*) { >StringBuilder sb = new StringBuilder(); >sb.append("Create your HDFS custom path here"); >return sb.toString(); >} > &

Re: Alternative to camus

2015-03-13 Thread Andrew Otto
> We are currently using spark streaming 1.2.1 with kafka and write-ahead log. > I will only say one thing : "a nightmare". ;-) I’d be really interested in hearing about your experience here. I’m exploring streaming frameworks a bit, and Spark Streaming is just so easy to use and set up. I’d be

Re: Kafka/Hadoop consumers and producers

2013-08-09 Thread Andrew Otto
For the last 6 months, we've been using this: https://github.com/wikimedia-incubator/kafka-hadoop-consumer In combination with this wrapper script: https://github.com/wikimedia/kraken/blob/master/bin/kafka-hadoop-consume It's not great, but it works! On Aug 9, 2013, at 2:06 PM, Felix GV wrot

Re: Kafka 08 clients

2013-08-12 Thread Andrew Otto
This is the Kafka C client for 0.8 we are using at Wikimedia: https://github.com/edenhill/librdkafka If you're using Debian/Ubuntu: you use the debian branch here to build a .deb: https://github.com/paravoid/librdkafka/tree/debian On Aug 12, 2013, at 12:06 AM, Jun Rao wrote: > At Linked

Re: Kafka/Hadoop consumers and producers

2013-08-12 Thread Andrew Otto
We've done a bit of work over at Wikimedia to debianize Kafka and make it behave like a regular service. https://github.com/wikimedia/operations-debs-kafka/blob/debian/debian Most relevant, Ken, is an init script for Kafka: https://github.com/wikimedia/operations-debs-kafka/blob/debian/debian

Re: Kafka/Hadoop consumers and producers

2013-08-13 Thread Andrew Otto
Andrew, I'm about to dive into figuring out how to use Camus without Avro. Perhaps we should join forces? (Be warned thought! My java fu is low at the moment. :) ). -Ao On Aug 12, 2013, at 11:20 PM, Andrew Psaltis wrote: > Kam, > I am perfectly fine if you pick this up. After thinking abo

Re: Kafka/Hadoop consumers and producers

2013-08-13 Thread Andrew Otto
y clean and simple. > What installs all the kafka dependencies under /usr/share/java? > > From: Andrew Otto > To: Kam Kasravi > Cc: "d...@kafka.apache.org" ; Ken Goodhope > ; Andrew Psaltis ; > "dibyendu.bhattacha...@pearson.com" ; > "camus

Re: Kafka/Hadoop consumers and producers

2013-08-13 Thread Andrew Otto
you. I do > know the bigtop folks > would like to see sbt support. > > From: Andrew Otto > To: Kam Kasravi > Cc: "d...@kafka.apache.org" ; Ken Goodhope > ; Andrew Psaltis ; > "dibyendu.bhattacha...@pearson.com" ; > "camus_...@googlegroups.

Kafka Mirroring setup

2013-08-20 Thread Andrew Otto
Hi all! Wikimedia is investigating how best to set up Broker clusters in multiple data centers. Our main analytics Broker cluster is currently in our main datacenter. It is possible for all of the main DC's frontend producers to produce directly to our analytics cluster, but we're not sure if

Re: Kafka Mirroring setup

2013-08-20 Thread Andrew Otto
centers relatively common. > > -Jay > > > > > On Tue, Aug 20, 2013 at 10:35 AM, Andrew Otto wrote: > >> Hi all! >> >> Wikimedia is investigating how best to set up Broker clusters in multiple >> data centers. Our main analytics Broker cluster is cu

Re: Kafka Mirroring setup

2013-08-22 Thread Andrew Otto
ore flexible if you need that. > On Aug 20, 2013 7:57 PM, "Andrew Otto" wrote: > >> In our case, our aggregator/analytics cluster is in our main datacenter, >> so there's no risk of the main producers becoming disconnected from it. It >> seems nicer to have

Re: Ganglia Metrics Reporter

2013-08-22 Thread Andrew Otto
Cool! At WMF, we use jmxtrans to do this: https://github.com/jmxtrans/jmxtrans And, if you use puppet, here's a nice little module to help generate jmxtrans json files, and an example of metrics we were sending to Ganglia from Kafka 0.7.2. https://github.com/wikimedia/puppet-jmxtrans#kafka-bro

Re: Ganglia Metrics Reporter

2013-08-23 Thread Andrew Otto
Jun, Note that the puppet module README I liked to isn't a full jmxtrans example JSON query. It is a jmxtrans puppet module usage example. So, using that in puppet will generate a .json file containing the query. We'll be moving to 0.8 in the next coming months, and I'll try to get some real

Kafka git branches

2013-08-23 Thread Andrew Otto
Hi all, I'm building packages for testing 0.8. Which branch should I base my builds off of? - 0.8 - 0.8.0-beta1-candidate1 - trunk ? Thanks! -Andrew Otto

Re: Kafka git branches

2013-08-23 Thread Andrew Otto
********/ > > > On Aug 23, 2013, at 4:15 PM, Andrew Otto wrote: > >> Hi all, >> >> I'm building packages for testing 0.8. Which branch should I base my builds >> off of? >> >> - 0.8 >> - 0.8.0-beta1-candidate1 >> - trunk >> >> ? >> >> Thanks! >> -Andrew Otto >> >>

Re: delete a topic

2013-08-23 Thread Andrew Otto
mmand class and script that trunk has should be merged into 0.8 sooner rather than later? (I'm really only mentioning this because TopicCommand makes my wrapper script a bit simpler. :) ) -Andrew Otto On Aug 23, 2013, at 5:28 PM, Jay Kreps wrote: > I mean I think the meta point N

Re: JMX

2013-08-29 Thread Andrew Otto
I use JConsole, works fine. On Aug 29, 2013, at 7:03 PM, Surendranauth Hiraman wrote: > I can't remember my success with JConsole but I know it works with jmxterm > - http://wiki.cyclopsgroup.org/jmxterm/tutorial > > > On Thu, Aug 29, 2013 at 7:02 PM, Mark wrote: > >> I should note this is

Re: Kafka -> HDFS

2013-09-03 Thread Andrew Otto
Mark, I had the same question! Camus is super awesome, but doesn't have out of the box support for just writing Strings into HDFS. I submitted this pull request to support that: https://github.com/linkedin/camus/pull/28 You can clone this directly from the wikimedia branch of Camus: https:/

Re: How far off is branch 0.8 to -0.8.0-beta1-candidate1? Is branch 0.8 stable enough to be used in production?

2013-09-23 Thread Andrew Otto
I've been building the 0.8 branch and running tests in a semi-production environment and haven't had any troubles yet. I had one problem resulting from KAFKA-1046 where the Annotations*.scala files all were loaded by our custom build system, resulting in duplicate class declarations. We had to

Re: Ganglia Metrics Reporter

2013-10-29 Thread Andrew Otto
hould I proceed? :) -Andrew Otto (Thanks for writing this, btw!) On Aug 22, 2013, at 11:42 AM, Maxime Brugidou wrote: > Hi all, > > Since I couldn't find any other way to publish kafka metrics to ganglia > from kafka 0.8 (beta), I just published on github a super-simple ganglia >

Re: Ganglia Metrics Reporter

2013-11-01 Thread Andrew Otto
using this small plugin and let me know what you plan to do. > > Cheers > On Oct 29, 2013 2:00 PM, "Andrew Otto" wrote: > >> Hi Maxime, >> >> I'm using this at the Wikimedia Foundation to send Kafka Broker metrics to >> Ganglia. However, we use

Incorrect JMX MBean name on Kafka doc page

2013-11-01 Thread Andrew Otto
In http://kafka.apache.org/documentation.html#monitoring, ISR expansion rate "kafka.server":name="ISRShrinksPerSec",type="ReplicaManager"See above I believe this should be "kafka.server":name="IsrExpandsPerSec",type="ReplicaManager" -Andrew Otto

Kafka 0.8 jmxtrans + puppet

2013-11-06 Thread Andrew Otto
to monitor. Hope this is useful to someone! -Andrew Otto

Kafka IPv6

2013-11-06 Thread Andrew Otto
Hm, Does Kafka support IPv6? I'm trying it now, but I'm just using the console-producer, which seems to not be able to read in --broker-list with IPv6 addresses. It looks like it is interpreting the colons in the address as the addy:port separator. -Andrew

Re: Kafka 0.8 jmxtrans + puppet

2013-11-06 Thread Andrew Otto
gshadoop <http://www.twitter.com/allthingshadoop> > / > > > On Wed, Nov 6, 2013 at 9:29 AM, Neha Narkhede wrote: > >> Cool, thanks for sharing this! >> >> -Neha >> On Nov 6, 2013 6:16 AM, "Andrew Otto" wrote: >>

Re: Kafka IPv6

2013-11-06 Thread Andrew Otto
Jun > > > On Wed, Nov 6, 2013 at 7:43 AM, Andrew Otto wrote: > >> Hm, >> >> Does Kafka support IPv6? I'm trying it now, but I'm just using the >> console-producer, which seems to not be able to read in --broker-list with >> IPv6 addresses.

Re: List of topics with JMX?

2013-11-19 Thread Andrew Otto
Would "kafka.server":type="BrokerTopicMetrics",name="AllTopicsMessagesInPerSec" count be easier? Also, correct me if I am wrong, but I believe that these count values are the total number of messages seen for a topic (or all topics) since the Broker was started, not the total number of message

Re: List of topics with JMX?

2013-11-19 Thread Andrew Otto
Hi, > The count in the Mbean > "kafka.server":type="BrokerTopicMetrics",name="AllTopicsMessagesInPerSec" > gives the total no of messages for all the topics on the broker. > > > On Tue, Nov 19, 2013 at 8:13 PM, Andrew Otto wrote: > >>

Re: How to get monitoring stats

2013-11-20 Thread Andrew Otto
For Kafka 0.8, you could use http://www.jmxtrans.org/ and a variation of this json file: https://github.com/wikimedia/puppet-kafka/blob/master/kafka-jmxtrans.json.md Just change the ganglia output writers to graphite ones. On Nov 19, 2013, at 8:25 PM, Benjamin Black wrote: > https://www.goo

Kafka Cluster Failover

2013-11-27 Thread Andrew Otto
producers and their metadata? Would they be able to handle a total switch of cluster metadata? Thanks! -Andrew Otto

Re: Kafka Cluster Failover

2013-11-27 Thread Andrew Otto
ill need to get it registered in a ZooKeeper cluster right? Also, > where are the events going to be ultimately consumed? I'm assuming in > the main DC - in which case you would anyway need to ship your Kafka > logs from the remote DC to the main DC correct? > > Joel >

Kafka partitions unbalanced

2015-05-27 Thread Andrew Otto
Hi all, I’ve recently noticed that our broker log.dirs are using up different amounts of storage. We use JBOD for our brokers, with 12 log.dirs, 1 on each disk. One of our topics is larger than the others, and has 12 partitions. Replication factor is 3, and we have 4 brokers. Each broker th

Controlled Shutdown Tool?

2015-07-27 Thread Andrew Otto
is supposed to set this for a running broker. Do I need a special JMX tool in order to flick this switch? I’d like to add a command to my kafka bin wrapper script so that I can easily use this when restarting brokers. What is the proper way to set controlled.shutdown.enable? Thanks! -Andrew

Re: Controlled Shutdown Tool?

2015-07-27 Thread Andrew Otto
x retries using controlled.shutdown.max.retries > defaults to 3 . > > > Thanks, > Harsha > > > On July 27, 2015 at 11:42:32 AM, Andrew Otto (ao...@wikimedia.org > <mailto:ao...@wikimedia.org>) wrote: > >> I’m working on packaging 0.8.2.1 for Wikimedia, and in

Re: Controlled Shutdown Tool?

2015-07-27 Thread Andrew Otto
oker to tell it to do the controlled shutdown. > I also got confused before and had to look at the code to figure that out. > I think it is better if we can add this to the document. > > -Binh > > On Mon, Jul 27, 2015 at 11:50 AM, Andrew Otto wrote: > >> Thanks! &g

Decomissioning a broker

2015-07-30 Thread Andrew Otto
I know that broker will still be registered in Zookeeper. Should I just delete the znode for that broker once it has been shut down? Thanks! -Andrew Otto

Re: Decomissioning a broker

2015-08-04 Thread Andrew Otto
eaders should happen for you as well. Though, manual reassignment may be > preferred in your case. > > Here is some extra information on controlled shutdowns: > http://kafka.apache.org/documentation.html#basic_ops_restarting > > Thanks, > Grant > > On Thu, Jul 30, 2015 at

Re: Kafka Simple Consumer Replicas versus ISR

2015-08-05 Thread Andrew Otto
Hi, I’m not sure, but it is possible the discrepancy you are seeing is related to this: https://issues.apache.org/jira/browse/KAFKA-1367 I’m pretty sure the CLI talks to Zookeeper directly, whereas likely the SimpleConsumer talks to the Brokers. > On Aug 5, 2015, at 13:13, d...@ariens.ca wro

Re: Kafka metadata

2015-08-10 Thread Andrew Otto
Note that broker metadata is not necessarily kept in sync with zookeeper on all brokers at all times: https://issues.apache.org/jira/browse/KAFKA-1367 This looks like it is fixed in the upcoming 0.8.3 > On Aug 8, 2015, at 01:08, Abdoulaye Diallo wrote: > > @Rahul > >> If this is true, why do

Re: 0.8.2.1 upgrade causes much more IO

2015-08-13 Thread Andrew Otto
s to confirm this). > > > > Thanks, > > Matt Bruce > > > > > > *From:* Andrew Otto [mailto:ao...@wikimedia.org] > *Sent:* Tuesday, August 11, 2015 3:15 PM > *To:* users@kafka.apache.org > *Cc:* Dan Andreescu ; Joseph Allemandou < > jalleman...@wiki

Replica not available...when it is!

2015-08-15 Thread Andrew Otto
I am having trouble with a single broker causing consumers to lag. As I am troubleshooting this issue, I have stopped this broker in the hopes that other replicas will take over as leader for this broker’s preferred partitions. However, when I do so, Camus reports: kafka.CamusJob: Skipping

Re: Replica not available...when it is!

2015-08-15 Thread Andrew Otto
this broker being offline makes the rest of the cluster think that its none of its replicas are available. > On Aug 15, 2015, at 11:18, Andrew Otto wrote: > > I am having trouble with a single broker causing consumers to lag. As I am > troubleshooting this issue, I have stopped t

Re: Replica not available...when it is!

2015-08-15 Thread Andrew Otto
eans that there > is a replica that is down. > > If you get Leader not available that means the partition is offline. > > -Clark > > Sent from my iPhone > >> On Aug 15, 2015, at 8:41 AM, Andrew Otto wrote: >> >> Also strange: If I start this bro

Re: Replica not available...when it is!

2015-08-15 Thread Andrew Otto
running the latest version of Camus? > > -Clark > > Sent from my iPhone > >> On Aug 15, 2015, at 10:25 AM, Andrew Otto wrote: >> >> Hm, interesting. So my real issue is more with Camus than with cluster >> problems? It seems that Camus won’t co

Re: [DISCUSSION] Kafka 0.8.2.2 release?

2015-08-18 Thread Andrew Otto
I agree: keep it simple :) The latest stable version of Kafka right now has a critical bug in it. Fixing that would be good enough. 0.8.2.2 should probably just a maintenance/bugfix release. > On Aug 18, 2015, at 14:29, Edward Ribeiro wrote: > > I sort of follow Daniel Nelson on this issue

Re: API to query cluster metadata on-demand

2015-09-03 Thread Andrew Otto
If you don’t mind doing it with a C CLI: https://github.com/edenhill/kafkacat $ kafkacat -L -b mybroker But, uhhh, you probably want a something in the Java API. :) > On Sep 3, 2015, at 13:58, Gwen Shapira wrote: > > Ah, I wish. > > We are working on it :) > > On Thu, Sep 3, 2015 at 9:10

Replica Fetcher Reset Its Offset to beginning

2015-10-29 Thread Andrew Otto
Hi all, This morning I woke up to see a very high max replica lag on one of my brokers. I looked at logs, and it seems that one of the replica fetchers for a partition just decided that its offset was out of range, so it reset its offset to the beginning of the leader’s log and started replica

Re: best python library to use?

2016-01-11 Thread Andrew Otto
pykafka’s balanced consumer is very useful. pykafka also has Python bindings to the librdkafka C library that you can optionally enable, which might get you some speed boosts. python-kafka (oh, I just saw this 0.9x version, hm!) was better at producing than pykafka for us, so we am currently using

Re: best python library to use?

2016-01-12 Thread Andrew Otto
idential and may also be privileged. If youare not > the > > intended recipient, please notify the sender immediately, and do > > notdisclose > > the contents to another person, use it for any purpose, or store, or > > copythe > > information in any medium. Please also d

MirrorMaker —new.producer

2016-01-19 Thread Andrew Otto
Hi all, I finally have a need to understand MirrorMaker well. I’m running Kafka 0.8.2.2. I see that my version of MirrorMaker has a —new.producer option, which uses NewShinyProducer instead of OldProducer. Without —new.producer, kafka-console-producer.sh seems to produce byte messages that caus

Re: MirrorMaker —new.producer

2016-01-19 Thread Andrew Otto
ass=kafka.serializer.StringEncoder. But still, should I be using —new.producer? On Tue, Jan 19, 2016 at 11:50 AM, Andrew Otto wrote: > Hi all, > > I finally have a need to understand MirrorMaker well. I’m running Kafka > 0.8.2.2. I see that my version of MirrorMaker has a —new.producer

Re: Apache Kafka Case Studies

2016-02-03 Thread Andrew Otto
Talk I gave about Kafka at the Wikimedia Foundation at Kafka NYC Meetup in 2014. https://www.hakkalabs.co/articles/apache-kafka-wikimedia On Wed, Feb 3, 2016 at 1:56 PM, Joe San wrote: > The OReilly online training seems to be interesting! Is there anything else > that is similiar to this that

Upgrade by replacing brokers?

2017-07-26 Thread Andrew Otto
anything I’m missing? Are there gotchas related with on disk log file formats that might cause some issues? Thanks! - Andrew Otto Systems Engineer, Wikimedia Foundation

Re: Kafka Monitoring..

2017-11-09 Thread Andrew Otto
We’ve recently started using Prometheus, and use Prometheus JMX Exporter to get Kafka metrics into prometheus. Here’s our JMX Exporter config: https://github.com/wikimedia/puppet/blob/production/modules/profile/files/kafka/broker_prometheus_jmx_exporter

super.users or kafka-acls —cluster for broker ACLs?

2017-12-04 Thread Andrew Otto
Hi all, Is there any reason not to list broker principals as super.users? I know there is the —cluster shortcut for adding broker ACL permissions via the kafka-acls CLI. Adding them to super.users would be simpler, as it can more easily be done via configuration management. Thanks! - Andrew

Re: offsetsForTimes API performance

2018-01-22 Thread Andrew Otto
Speaking of, has there been any talk of combining those two requests into a single API call? I’d assume that offsetForTimes + consumer seek is probably the most common use case of offsetForTimes. Maybe a round trip could be avoided if the broker could just auto-assign the consumer to the offset f

JSONSchema Kafka Connect Converter

2018-01-23 Thread Andrew Otto
to a ConnectRecord, the messages could be used with any Connector out there, right? I might have space in the next year to work on something like this, but I thought I’d ask here first to see what others thought. Would this be useful? If so, is this something that might be upstreamed into Apache Kafka?

Re: Cross-cluster mirror making

2018-02-12 Thread Andrew Otto
by the Kafka cluster. If the buffer gets too full, it will have to start dropping messages. - Andrew Otto On Thu, Feb 8, 2018 at 5:21 PM, Husna Hadi wrote: > Hi, I read on The Definitive Guide to Kafka that when using cross-cluster > kafka mirroring, when possible, consume from a

Kafka 0.9 MirrorMaker failing with Batch Expired when producing to Kafka 1.0 cluster

2018-03-12 Thread Andrew Otto
://gist.github.com/ottomata/5324fc3becdd20e9a678d5d37c2db872 Any help is appreciated, thanks! -Andrew Otto Senior Systems Engineer Wikimedia Foundation

Re: Kafka 0.9 MirrorMaker failing with Batch Expired when producing to Kafka 1.0 cluster

2018-03-13 Thread Andrew Otto
anging MirrorMaker instances, I think looking at stack dumps would > help you get closer to the root cause. > > Best regards, > Andras > > On Mon, Mar 12, 2018 at 7:56 PM, Andrew Otto wrote: > > > Hi all, > > > > I’m troubleshooting a MirrorMaker issue, and

Re: Kafka Mirrormaker issue

2018-03-26 Thread Andrew Otto
I’ve had similar problems, but I don’t have an explanation for ya :/ On Sun, Mar 25, 2018 at 12:19 PM, Siva A wrote: > Hi, > > We have 3 nodes Kafka cluster(0.10.0.1) and its mirroring the data from > another 3 node cluster of same Kafka version. > Both the clusters are Kerberized and we are run

Re: Kafka Mirrormaker issue

2018-03-30 Thread Andrew Otto
sure what the default in 0.10 is. On Fri, Mar 30, 2018 at 11:40 AM, Siva A wrote: > Any other update on this? > > On Mon, Mar 26, 2018, 7:42 PM Andrew Otto wrote: > > > I’ve had similar problems, but I don’t have an explanation for ya :/ > > > > On Sun, Ma

Re: Kafka Mirrormaker issue

2018-04-09 Thread Andrew Otto
t of our mirrormakers is the last > one we will upgrade to 1.x, at which point we can just upgrade the > mirrormakers to 1.x as well. > > On 4/6/18, 1:09 PM, "Jeff Field" wrote: > > I'm hitting the same problem, even with the new consumer, on > MirrorMake

Re: Filter plugins in Kafka

2016-05-02 Thread Andrew Otto
If you want something really simple and hacky, you could use kafkatee[1] and kafkacat[2] together: kafkatee.conf: input [encoding=string] pipe tail -f a.log output pipe 1 grep -v ’not this’ | kafkacat -P -b b1:9092 -t mytopic [1] https://github.com/wikimedia/analytics-kafkatee [2] https://github

Brokers changing mtime on data files during startup?

2016-05-25 Thread Andrew Otto
Hiya, We’ve recently upgraded to 0.9. In 0.8, when we restarted a broker, data log file mtimes were not changed. In 0.9, any data log file that was on disk before the broker has it’s mtime modified to the time of the broker restart. This causes problems with log retention, as all the files then

Re: Brokers changing mtime on data files during startup?

2016-05-25 Thread Andrew Otto
“We use the default log retention of 7 *days*" :)* On Wed, May 25, 2016 at 12:34 PM, Andrew Otto wrote: > Hiya, > > We’ve recently upgraded to 0.9. In 0.8, when we restarted a broker, data > log file mtimes were not changed. In 0.9, any data log file that was on > disk b

Re: Relaying UDP packets into Kafka

2016-05-25 Thread Andrew Otto
Super old, but: https://github.com/atdt/UdpKafka On Wed, May 25, 2016 at 4:20 PM, Joe San wrote: > What about this one: https://github.com/agaoglu/udp-kafka-bridge > > On Wed, May 25, 2016 at 6:48 PM, Sunil Saggar > wrote: > > > Hi All, > > > > I am looking for a kafka producer to receive UDP p

Re: Changing default logger to RollingFileAppender (KAFKA-2394)

2016-06-02 Thread Andrew Otto
+1, this is what Wikimedia uses in production. On Thu, Jun 2, 2016 at 10:38 AM, Tauzell, Dave wrote: > I haven't started using this in production but this is how I will likely > setup the logging as it is easier to manage. > > -Dave > > -Original Message- > From: Dustin Cote [mailto:dus.

Re: Brokers changing mtime on data files during startup?

2016-06-07 Thread Andrew Otto
The settings controls the max size in bytes of a partion oft he > specified > > > topic. So you can find a good size by checking the size of a partition > > with > > > du –b and use this value. > > > > > > This way you do not loose ~7 days of data and can

Re: [VOTE] Add REST Server to Apache Kafka

2016-10-26 Thread Andrew Otto
-1 for http kafka client in core Although a read only management interface, perhaps via http, sounds kinda useful for things like health checks as mentioned. On Wed, Oct 26, 2016 at 2:00 PM, Zakee wrote: > -1 > > Thanks. > > On Oct 25, 2016, at 2:16 PM, Harsha Chintalapani > wrote: > > > > Hi

Re: Using Kafka CLI without specifying the URLs every single time?

2018-04-23 Thread Andrew Otto
Us too: https://github.com/wikimedia/puppet/blob/production/modules/confluent/files/kafka/kafka.sh This requires that the various kafka-* scrips are in your PATH. And then this gets rendered into /etc/profile.d to set env variables. https://github.com/wikimedia/puppet/blob/production/modules/con

Re: Kafka mirror maker help

2018-04-27 Thread Andrew Otto
Hiya, Saravanan, I saw you emailed my colleague Alex about WMF’s old debian packaging. I’ll reply here. We now use Confluent’s Kafka debian packaging which does not (or did not?) ship with init scripts. We don’t use Sys V init.d scripts anymore either, but use systemd instead. Our systemd serv

Re: Kafka Connect REST connector with additional logging to kafka

2018-08-23 Thread Andrew Otto
Hiya, this doesn’t help answer your question, but as an FYI, Wikimedia has implemented https://github.com/wikimedia/change-propagation to do what you are trying to do: issue HTTP requests (and other things) triggered by incoming messages in Kafka. On Thu, Aug 23, 2018 at 9:45 AM Andrea Spina wrot

Re: Exposing Kafka on WAN

2018-08-30 Thread Andrew Otto
The trouble is that the producer and consumer clients need to discover the broker hostnames and address the individual brokers directly. There is an advertised.listeners setting that will allow you to tell clients to connect to external proxy hostnames instead of your internal ones, but those prox

Re: Where to run MM2? Source or destination DC/region?

2020-01-09 Thread Andrew Otto
Hi Peter, My understanding here comes from MirrorMaker 1, but I believe it holds for MM2 (someone correct me if I am wrong!) For the most part, if you have no latency or connectivity issues, running MM at the source will be fine. However, the failure scenario is different if something goes wrong.

Re: Where to run MM2? Source or destination DC/region?

2020-01-09 Thread Andrew Otto
ource this event is happing. > > -----Original Message- > From: Andrew Otto > Sent: Thursday, January 9, 2020 8:32 AM > To: users@kafka.apache.org > Subject: Re: Where to run MM2? Source or destination DC/region? > > ---External Email--- > > Hi Peter, > > My understa

log.message.timestamp.difference.max.ms and future timestamps?

2020-04-16 Thread Andrew Otto
egative in this case. log.message.timestamp.difference.max.ms - futureTimestamp == -bigNumber Will the message be rejected or accepted in this case? Thanks! -Andrew Otto @Wikimedia Foundation

Re: log.message.timestamp.difference.max.ms and future timestamps?

2020-05-08 Thread Andrew Otto
fference works both ways. > > > -Matthias > > > On 4/16/20 9:39 AM, Andrew Otto wrote: > > log.message.timestamp.difference.max.ms is the "The maximum difference > > allowed between the timestamp when a broker receives a message and the > > timestamp specifie

Re: ACLs - How To Allow Anyone To Access of A Topic

2020-05-18 Thread Andrew Otto
If I understand correctly, if your client authenticates, there must be an ACL for that principal, otherwise it will fail authorization. If you are going to allow everything anyway, perhaps you don't need to authenticate?

Re: Disk space - sharp increase in usage

2020-06-02 Thread Andrew Otto
WMF recently had an issue where Kafka broker disks were filling up with log segment data. It turned out that Kafka was not deleting old log segments because the oldest log segment had a message with a Kafka timestamp a year in the future. Since

Wikimedia's Event Data Platform

2020-09-14 Thread Andrew Otto
be 2 more posts coming out over the next couple of weeks too. Thanks for reading! :) - Andrew Otto Engineer @ The Wikimedia Foundation

Re: Introducing JulieOps, a tool to help you build your GitOps and self-service with Apache Kafka and related

2021-03-01 Thread Andrew Otto
This looks amazing, thanks Pere! I just filed a ticket for Wikimedia to consider adopting JulieOps. On Sat, Feb 27, 2021 at 2:05 PM Pere Urbón Bayes wrote: > Hi, >I hope this message finds everyone healthy and good. I wanted to reach > out to th

Re: Standard way to get http POST request into a Kafka topic?

2021-04-28 Thread Andrew Otto
/24/wikimedias-event-data-platform-event-intake/> explains how we use EventGate. -Andrew Otto SRE, Wikimedia Foundation On Tue, Apr 27, 2021 at 11:42 PM Ran Lupovich wrote: > Hi, have a look for Rest Proxy component as part of the kafka eco system > > בתאריך יום ד׳, 28 באפר׳ 2021,

Re: Spark Streams vs Kafka Streams

2021-04-28 Thread Andrew Otto
I'd assume this is because Kafka Streams is positioned for building streaming applications, rather than doing analytics, whereas Spark is more often used for analytics purposes.

Re: Spark Streams vs Kafka Streams

2021-04-28 Thread Andrew Otto
2021 at 3:50 PM Parthasarathy, Mohan wrote: > Andrew, > > I am not sure I understand. We have built several analytics applications. > We typically use custom aggregations as they are not available directly in > the library. > > -mohan > > > On 4/28/21, 12:12 PM, &q

Re: Newbie looking for a connector I can configure on my mac

2022-03-29 Thread Andrew Otto
ions are CCL and not FOSS, means that organizations that use purely FOSS software (like the Wikimedia Foundation) makes Kafka Connect effectively unusable. Okay carry on! :) - Andrew Otto On Tue, Mar 29, 2022 at 8:27 PM Liam Clarke-Hutchinson wrote: > Hi Andrew, > > So if you

Kafka Stretch Clusters

2022-05-09 Thread Andrew Otto
re the potential issues with partition leaders is worth it! Thanks for any insight y'all have, -Andrew Otto Wikimedia Foundation

  1   2   >