Re: Mirrormaker consumption slowness

2017-12-06 Thread Steve Miller
This kind of sounds to me like there’s packet loss somewhere and TCP is closing the window to try to limit congestion. But from the snippets you posted, I didn’t see any sacks in the tcpdump output. If there *are* sacks, that’d be a strong indicator of loss somewhere, whether it’s in the netwo

Re: Too Many Open Files

2016-08-01 Thread Steve Miller
Can you run lsof -p (pid) for whatever the pid is for your Kafka process? For the fd limits you've set, I don't think subtlety is required: if there's a millionish lines in the output, the fd limit you set is where you think it is, and if it's a lot lower than that, the limit isn't being applied

Re: Debugging high log flush latency on a broker.

2015-09-22 Thread Steve Miller
There may be more elegant ways to do this, but I'd think that you could just ls all the directories specified in log.dirs in your server.properties file for Kafka. You should see directories for each topicname-partitionnumber there. Offhand it sounds to me like maybe something's evicting

Re: MirrorMaker - Not consuming from all partitions

2015-09-11 Thread Steve Miller
I have a vague feeling that I've seen stuff like this when the network on the broker that's disappearing is actually unreachable from time to time -- though I'd like to believe that's not such an issue when talking to AWS (though there could be a lot of screwed-up Internet between you and it,

Disaster Recovery (was: Re: Suggestions when all replicas of a partition are dead?)

2015-08-08 Thread Steve Miller
re states here and how to recover from them. -Steve On Fri, Aug 07, 2015 at 08:36:28PM +, Daniel Compton wrote: > I would have thought you'd want ZK up before Kafka started, but I don't > have any strong data to back that up. > On Sat, 8 Aug 2015 at 7:59 AM Steve Mi

Suggestions when all replicas of a partition are dead?

2015-08-07 Thread Steve Miller
So... we had an extensive recabling exercise, during which we had to shut down and derack and rerack a whole Kafka cluster. Then when we brought it back up, we discovered the hard way that two hosts had their "rebuild on reboot" flag set in Cobbler. Everything on those hosts is gone as a

Re: kafka-python message offset?

2015-07-29 Thread Steve Miller
Are you using mumrah/kafka-python? I think so from context but I know there's at least one other implementation rattling around these days. (-: If that's what you're using, I can see two potential problems you might be having. You can set the offset to some approximation of wherever you

When in-sync isn't in sync?

2015-05-04 Thread Steve Miller
[ BTW, after some more research, I think what might be happening here is that we had some de-facto network partitioning happen as a side-effect of us renaming some network interfaces, though if that's the case, I'd like to know how to get everything back into sync. ] Hi. I'm seeing somethin

Re: [DISCUSS] KIP-14 Tools Standardization

2015-04-09 Thread Steve Miller
FWIW I like the standardization idea but just making the old switches fail seems like it's not the best plan. People wrap this sort of thing for any number of reasons, and breaking all of their stuff all at once is not going to make them happy. And it's not like keeping the old switches workin

Re: consumer groups in python

2015-03-17 Thread Steve Miller
It's possible that I just haven't used it but I am reasonably sure that the python API doesn't have a way to store offsets in ZK. You would need to implement something more or less compatible with what the Scala/Java API does, presumably. On the plus side the python API -- possibly just becaus

Re: The purpose of key in kafka

2014-12-19 Thread Steve Miller
Also, if log.cleaner.enable is true in your broker config, that enables the log-compaction retention strategy. Then, for topics with the per-topic "cleanup.policy=compact" config parameter set, Kafka will scan the topic periodically, nuking old versions of the data with the same key.

Long-delayed followup on weird topic corruption issue

2014-11-17 Thread Steve Miller
Several months ago, I'd posted a lot of stuff here about an issue I was seeing, where the data segments for some topics were getting corrupted, for no obvious reason. We tried a bunch of stuff and got a bunch of advice, none of which helped, other than to cause there to be more documentat

Re: Keep on getting kafka.common.OffsetOutOfRangeException: Random times

2014-08-20 Thread Steve Miller
ad > > -Original Message- > From: Manjunath Shivakumar [mailto:manjunath.shivaku...@betfair.com] > Sent: Wednesday, August 20, 2014 1:31 PM > To: users@kafka.apache.org > Subject: RE: Keep on getting kafka.common.OffsetOutOfRangeException: Random > times &g

Re: Keep on getting kafka.common.OffsetOutOfRangeException: Random times

2014-08-20 Thread Steve Miller
ours have affect on this? I didn't > knew this. I have log.retention.hours set to 1, and during development we > test this once a 15 mins or hour or 2. So do you think this is causing the > issue? > > Thanks, > Pradeep Simha > Technical Lead > > -Or

Re: Keep on getting kafka.common.OffsetOutOfRangeException: Random times

2014-08-19 Thread Steve Miller
Also, what do you have log.retention.hours set to? How often do you publish messages? I can envision a scenario in which you don't publish to a topic often, and in fact publish so infrequently that everything in the topic ages out from log.retention.hours first. I don't know exactly

Re: Strange topic-corruption issue?

2014-08-17 Thread Steve Miller
:33PM -0700, Jun Rao wrote: > What's in there seems to be still the output for deep iteration. For > shallow iteration, the compression codec for each message should be Snappy. > > Thanks, > > Jun > > > On Fri, Aug 15, 2014 at 5:27 AM, Steve Miller > wrote: >

Re: Strange topic-corruption issue?

2014-08-15 Thread Steve Miller
Oh, yeah, sorry about that. I threw a copy of that up at: https://newbie.idrathernotsay.com/full.txt.gz (you'll get a cert error, for the four times I put something on the home website each year, I didn't feel like getting a real cert (-: ). If that doesn't work I'm sure I can fi

Re: Strange topic-corruption issue?

2014-08-14 Thread Steve Miller
I've attached the full output. The only other thing it produced was our old favorite: Non-secutive offsets in :/home/steve/mytopic-9/.log 1327 is followed by 1327 For the first time, earlier today, we've seen this happen from one of our other producers; offhand I'm

Re: Strange topic-corruption issue?

2014-08-13 Thread Steve Miller
n Wed, Aug 13, 2014 at 08:15:21AM -0700, Jun Rao wrote: > Interesting, could you run DumpLogSegments with and w/o deep-iteration and > send the output around offset 1327? > > Thanks, > > Jun > > > On Tue, Aug 12, 2014 at 5:42 PM, Steve Miller > wrote: > > &

Strange topic-corruption issue?

2014-08-12 Thread Steve Miller
[ "Aha!", you say, "now I know why this guy's been doing so much tshark stuff!" (-: ] Hi. I'm running into a strange situation, in which more or less all of the topics on our Kafka server behave exactly as expected... but the data produced by one family of applications is producing fairly

Using the kafka dissector in wireshark/tshark 1.12

2014-08-12 Thread Steve Miller
I'd seen references to there being a Kafka protocol dissector built into wireshark/tshark 1.12, but what I could find on that was a bit light on the specifics as to how to get it to do anything -- at least for someone (like me) who might use tcpdump a lot but who doesn't use tshark a lot.