KAFKA-3802 does seem plausible; I had to restart the brokers again after the 0.10.1.0 upgrade to change some JVM settings; maybe that touched the mtime on the files? Not sure why that would make them *more* likely to be deleted, though, since their mtime should've gone into the future, not into the past...
On Mon, Oct 31, 2016 at 5:02 PM, Jun Rao <j...@confluent.io> wrote: > Hi, James, > > Thanks for testing and reporting this. What you observed is actually not > the expected behavior in 0.10.1 based on the design. The way that retention > works in 0.10.1 is that if a log segment has at least one message with a > timestamp, we will use the largest timestamp in that segment to determine > if the segment should be retained. If no message in a segment has a > timestamp (which is your case), we will fall back to use the last modified > time of the segment, which is the old behavior. > > I tested this locally and didn't see old log segments being deleted > immediately after upgrade. It's also a bit weird that you only saw that in > the leader broker. The retention logic is orthogonal to a replica being a > leader or a follower. > > I am wondering if you could be hitting > https://issues.apache.org/jira/browse/KAFKA-3802 ? If not, is there a way > to reproduce this reliably? > > Jun > > On Mon, Oct 31, 2016 at 4:14 PM, James Brown <jbr...@easypost.com> wrote: > > > I just finished upgrading our main production cluster to 0.10.1.0 (from > > 0.9.0.1) with an on-line rolling upgrade, and I noticed something > strange — > > the leader for one of our big partitions just decided to expire all of > the > > logs from before the upgrade. I have log.retention.hours set to 336 in my > > config, and the replicas still have data going back to October 17, but > > after restarting for 0.10.1.0, the topic leader deleted all segments more > > than a couple of hours old (approximately 2TB of data on that box). > > > > inter.broker.protocol.version and log.message.format.version are both > still > > set to 0.9.0.1 in my config > > > > Before the upgrade, the oldest available offset in this topic/partition > > was 812555925; now it's 848947551. > > > > I assume this is some bug with upgrading to 0.10.1.0 when the extant data > > doesn't have any associated timestamps, but it seems, uh, really > > unexpected, and if I'd had any consumers which were behind, I could've > > ended up losing quite a lot of data here. It's particularly bizarre that > > this didn't affect anything except the leader (yet). > > > > It may be that this is expected behavior, but I guess I just assumed that > > the code would fall back to using the mtime if timestamps were not > present > > in the log rather than assuming that the timestamp of a given segment was > > 0. If this is expected behavior, I would recommend adding a specific note > > the the "Potential breaking changes in 0.10.1.0" section of the manual > > indicating that upgrading from 0.9.0.1 might immediately truncate all of > > your data. > > > > > > Debugging output is below: > > > > > > % kafka-topics.sh --zookeeper localhost:40169 --describe --topic > > easypost.request_log > > Topic:easypost.request_log PartitionCount:4 ReplicationFactor:3 Configs: > > Topic: easypost.request_log Partition: 0 Leader: 1 Replicas: 1,4,2 Isr: > > 4,2,1 > > Topic: easypost.request_log Partition: 1 Leader: 4 Replicas: 4,1,2 Isr: > > 4,2,1 > > Topic: easypost.request_log Partition: 2 Leader: 5 Replicas: 5,2,3 Isr: > > 5,3,2 > > Topic: easypost.request_log Partition: 3 Leader: 3 Replicas: 3,5,2 Isr: > > 5,3,2 > > > > (on broker #1): > > > > % ls -l /srv/var/kafka/logs/easypost.request_log-0/ | wc -l > > 25 > > > > (on broker #4): > > > > % ls -l /srv/var/kafka/logs/easypost.request_log-0/ | wc -l > > 3391 > > > > > > When the actual deletion occurred, there were no errors in the log; just > a > > lot messages like > > > > INFO Scheduling log segment 811849601 for log easypost.request_log-0 for > > deletion. (kafka.log.Log) > > > > > > I suspect it's too late to un-do anything related to this, and I don't > > actually think any of our consumers were relying on this data, but I > > figured I'd send along this report and see if anybody else has seen > > behavior like this. > > > > Thanks, > > -- > > James Brown > > Engineer > > > -- James Brown Engineer