Attached a transcript that explains what I'm seeing
On Fri, Jul 25, 2014 at 10:52 AM, Kashyap Paidimarri <kashy...@gmail.com> wrote: > No, we haven't configured that. We have a few hundred topics but this > seems to be the only one affected (I did a quick check, not thorough). > > The relevant config params that we have set in server.properties. > > log.dir=/var/lib/fk-3p-kafka/logs > log.flush.interval.messages=10000 > log.flush.interval.ms=1000 > log.retention.hours=168 > log.segment.bytes=536870912 > log.cleanup.interval.mins=1 > log.retention.hours=336 > > > > On Fri, Jul 25, 2014 at 10:11 AM, Jun Rao <jun...@gmail.com> wrote: > >> Have you configured log.retention.bytes? >> >> Thanks, >> >> Jun >> >> >> On Thu, Jul 24, 2014 at 10:04 AM, Kashyap Paidimarri <kashy...@gmail.com> >> wrote: >> >> > We just noticed that one of our topics has been horribly misbehaving. >> > >> > *retention.ms <http://retention.ms>* for the topic is set to >> 1209600000 ms >> > >> > However, segments are getting schedule for deletetion as soon as a new >> one >> > is rolled over. And naturally consumers are running into a >> > kafka.common.OffsetOutOfRangeException whenever this happens. >> > >> > Is this a known bug? It is incredibly serious. We seem to have lost >> about >> > 40 million messages on a single topic and are yet to figure out what all >> > topics are affected. >> > >> > I thought of restarting Kafka but figured I'd leave it untouched while I >> > figure out what I can capture for finding the root cause. >> > >> > Meanwhile in order to keep from losing any more data, I have a periodic >> job >> > that is doing a *'cp -al' *of the partitions into a separate folder. >> That >> > way Kafka goes ahead and deletes the segment but the data is not lost >> from >> > the filesystem. >> > >> > If this is a unseen bug, what should I save from the running instance. >> > >> > By the way, this has affected all partitions and replicas of the topic >> and >> > not on a specific host. >> > >> > > > > -- > “ The difference between ramen and varelse is not in the creature judged, > but in the creature judging. When we declare an alien species to be ramen, > it does not mean that *they* have passed a threshold of moral maturity. > It means that *we* have. > > —Demosthenes, *Letter to the Framlings* > ” > -- “ The difference between ramen and varelse is not in the creature judged, but in the creature judging. When we declare an alien species to be ramen, it does not mean that *they* have passed a threshold of moral maturity. It means that *we* have. —Demosthenes, *Letter to the Framlings* ”
Topic with `retention.ms=1209600000` ------------------------------------ Relevant config parameters ``` log.dir=/var/lib/fk-3p-kafka/logs log.flush.interval.messages=10000 log.flush.interval.ms=1000 log.segment.bytes=536870912 log.cleanup.interval.mins=1 log.retention.hours=336 ``` ``` kashyapp@bigfoot-kafka-1:~$ /usr/share/fk-3p-kafka/bin/kafka-topics.sh --zookeeper bigfoot-zk3:2181/bigfoot/dart/kafka --describe --topic dart.fkint.cp.user.Address /bin/bash: warning: setlocale: LC_ALL: cannot change locale (en_US.UTF-8) /bin/bash: warning: setlocale: LC_ALL: cannot change locale (en_US.UTF-8) Topic:dart.fkint.cp.user.Address PartitionCount:2 ReplicationFactor:2 Configs:retention.ms=1209600000 Topic: dart.fkint.cp.user.Address Partition: 0 Leader: 1 Replicas: 3,1 Isr: 1,3 Topic: dart.fkint.cp.user.Address Partition: 1 Leader: 1 Replicas: 1,2 Isr: 1,2 ``` How it is --------- ``` kashyapp@bigfoot-kafka-1:~$ ls -l /var/lib/fk-3p-kafka/logs/dart.fkint.cp.user.Address* /var/lib/fk-3p-kafka/logs/dart.fkint.cp.user.Address-0: total 578904 -rw-r--r-- 2 fk-3p-kafka fk-pf 935528 Jul 24 22:33 00000000000036990624.index.deleted -rw-r--r-- 2 fk-3p-kafka fk-pf 536871185 Jul 24 22:33 00000000000036990624.log.deleted -rw-r--r-- 2 fk-3p-kafka fk-pf 10485760 Jul 24 22:36 00000000000037800733.index -rw-r--r-- 2 fk-3p-kafka fk-pf 54877502 Jul 24 22:36 00000000000037800733.log /var/lib/fk-3p-kafka/logs/dart.fkint.cp.user.Address-1: total 579544 -rw-r--r-- 2 fk-3p-kafka fk-pf 935432 Jul 24 22:33 00000000000037763015.index.deleted -rw-r--r-- 2 fk-3p-kafka fk-pf 536871489 Jul 24 22:33 00000000000037763015.log.deleted -rw-r--r-- 2 fk-3p-kafka fk-pf 10485760 Jul 24 22:36 00000000000038573139.index -rw-r--r-- 2 fk-3p-kafka fk-pf 55529779 Jul 24 22:36 00000000000038573139.log ``` How it should have been ----------------------- I'm using a periodic `cp -al` command to keep segments from actually getting lost. ``` $ while true; do cp -val /var/lib/fk-3p-kafka/logs/dart.fkint.cp.user.Address-* ~/Address-backup/;sleep 120;done ``` ``` ls -lh /home/kashyapp/Address-backup/dart.fkint.cp.user.Address-* /home/kashyapp/Address-backup/dart.fkint.cp.user.Address-0: total 1.9G -rw-r--r-- 1 fk-3p-kafka fk-pf 554K Jul 24 20:31 00000000000034988917.index -rw-r--r-- 1 fk-3p-kafka fk-pf 290M Jul 24 20:29 00000000000034988917.log -rw-r--r-- 1 fk-3p-kafka fk-pf 975K Jul 24 21:25 00000000000035417074.index -rw-r--r-- 1 fk-3p-kafka fk-pf 513M Jul 24 21:25 00000000000035417074.log -rw-r--r-- 1 fk-3p-kafka fk-pf 914K Jul 24 21:59 00000000000036179981.index -rw-r--r-- 1 fk-3p-kafka fk-pf 513M Jul 24 21:59 00000000000036179981.log -rw-r--r-- 2 fk-3p-kafka fk-pf 914K Jul 24 22:33 00000000000036990624.index -rw-r--r-- 2 fk-3p-kafka fk-pf 513M Jul 24 22:33 00000000000036990624.log -rw-r--r-- 2 fk-3p-kafka fk-pf 10M Jul 24 22:35 00000000000037800733.index -rw-r--r-- 2 fk-3p-kafka fk-pf 32M Jul 24 22:35 00000000000037800733.log /home/kashyapp/Address-backup/dart.fkint.cp.user.Address-1: total 1.9G -rw-r--r-- 1 fk-3p-kafka fk-pf 553K Jul 24 20:31 00000000000035761499.index -rw-r--r-- 1 fk-3p-kafka fk-pf 289M Jul 24 20:29 00000000000035761499.log -rw-r--r-- 1 fk-3p-kafka fk-pf 975K Jul 24 21:26 00000000000036188875.index -rw-r--r-- 1 fk-3p-kafka fk-pf 513M Jul 24 21:26 00000000000036188875.log -rw-r--r-- 1 fk-3p-kafka fk-pf 914K Jul 24 21:59 00000000000036952421.index -rw-r--r-- 1 fk-3p-kafka fk-pf 513M Jul 24 21:59 00000000000036952421.log -rw-r--r-- 2 fk-3p-kafka fk-pf 914K Jul 24 22:33 00000000000037763015.index -rw-r--r-- 2 fk-3p-kafka fk-pf 513M Jul 24 22:33 00000000000037763015.log -rw-r--r-- 2 fk-3p-kafka fk-pf 10M Jul 24 22:35 00000000000038573139.index -rw-r--r-- 2 fk-3p-kafka fk-pf 32M Jul 24 22:35 00000000000038573139.log ``` From `server.log` ----------------- ``` INFO [2014-07-24 20:31:54,596] [kafka-scheduler-3][] kafka.log.Log - Rolled new log segment for 'dart.fkint.cp.user.Address-1' in 2 ms. INFO [2014-07-24 20:31:54,596] [kafka-scheduler-3][] kafka.log.Log - Scheduling log segment 35761499 for log dart.fkint.cp.user.Address-1 for deletion. INFO [2014-07-24 20:31:54,599] [kafka-scheduler-3][] kafka.log.Log - Rolled new log segment for 'dart.fkint.cp.user.Address-0' in 1 ms. INFO [2014-07-24 20:31:54,599] [kafka-scheduler-3][] kafka.log.Log - Scheduling log segment 34988917 for log dart.fkint.cp.user.Address-0 for deletion. ERROR [2014-07-24 20:31:54,705] [kafka-request-handler-1][] kafka.server.KafkaApis - [KafkaApi-1] Error when processing fetch request for partition [dart.fkint.cp.user.Address,0] offset 35306748 from consumer with correlation id -1 ERROR [2014-07-24 20:31:54,735] [kafka-request-handler-0][] kafka.server.KafkaApis - [KafkaApi-1] Error when processing fetch request for partition [dart.fkint.cp.user.Address,1] offset 36086743 from consumer with correlation id -1 ERROR [2014-07-24 20:31:55,706] [request-expiration-task][] kafka.server.KafkaApis - [KafkaApi-1] Error when processing fetch request for partition [dart.fkint.cp.user.Address,0] offset 35306748 from consumer with correlation id -1 ERROR [2014-07-24 20:31:55,736] [request-expiration-task][] kafka.server.KafkaApis - [KafkaApi-1] Error when processing fetch request for partition [dart.fkint.cp.user.Address,1] offset 36086743 from consumer with correlation id -1 INFO [2014-07-24 20:32:54,597] [kafka-scheduler-1][] kafka.log.Log - Deleting segment 35761499 from log dart.fkint.cp.user.Address-1. INFO [2014-07-24 20:32:54,597] [kafka-scheduler-1][] kafka.log.OffsetIndex - Deleting index /var/lib/fk-3p-kafka/logs/dart.fkint.cp.user.Address-1/00000000000035761499.index.deleted INFO [2014-07-24 20:32:54,599] [kafka-scheduler-0][] kafka.log.Log - Deleting segment 34988917 from log dart.fkint.cp.user.Address-0. INFO [2014-07-24 20:32:54,600] [kafka-scheduler-0][] kafka.log.OffsetIndex - Deleting index /var/lib/fk-3p-kafka/logs/dart.fkint.cp.user.Address-0/00000000000034988917.index.deleted INFO [2014-07-24 21:25:59,575] [kafka-request-handler-1][] kafka.log.Log - Rolled new log segment for 'dart.fkint.cp.user.Address-0' in 2 ms. INFO [2014-07-24 21:26:00,227] [kafka-request-handler-0][] kafka.log.Log - Rolled new log segment for 'dart.fkint.cp.user.Address-1' in 2 ms. INFO [2014-07-24 21:31:54,595] [kafka-scheduler-3][] kafka.log.Log - Scheduling log segment 36188875 for log dart.fkint.cp.user.Address-1 for deletion. INFO [2014-07-24 21:31:54,596] [kafka-scheduler-3][] kafka.log.Log - Scheduling log segment 35417074 for log dart.fkint.cp.user.Address-0 for deletion. INFO [2014-07-24 21:32:54,595] [kafka-scheduler-1][] kafka.log.Log - Deleting segment 36188875 from log dart.fkint.cp.user.Address-1. INFO [2014-07-24 21:32:54,596] [kafka-scheduler-1][] kafka.log.OffsetIndex - Deleting index /var/lib/fk-3p-kafka/logs/dart.fkint.cp.user.Address-1/00000000000036188875.index.deleted INFO [2014-07-24 21:32:54,596] [kafka-scheduler-1][] kafka.log.Log - Deleting segment 35417074 from log dart.fkint.cp.user.Address-0. INFO [2014-07-24 21:32:54,597] [kafka-scheduler-1][] kafka.log.OffsetIndex - Deleting index /var/lib/fk-3p-kafka/logs/dart.fkint.cp.user.Address-0/00000000000035417074.index.deleted INFO [2014-07-24 21:59:12,922] [kafka-request-handler-0][] kafka.log.Log - Rolled new log segment for 'dart.fkint.cp.user.Address-0' in 2 ms. INFO [2014-07-24 21:59:13,494] [kafka-request-handler-1][] kafka.log.Log - Rolled new log segment for 'dart.fkint.cp.user.Address-1' in 2 ms. INFO [2014-07-24 22:01:54,595] [kafka-scheduler-0][] kafka.log.Log - Scheduling log segment 36952421 for log dart.fkint.cp.user.Address-1 for deletion. INFO [2014-07-24 22:01:54,596] [kafka-scheduler-0][] kafka.log.Log - Scheduling log segment 36179981 for log dart.fkint.cp.user.Address-0 for deletion. INFO [2014-07-24 22:02:54,595] [kafka-scheduler-0][] kafka.log.Log - Deleting segment 36952421 from log dart.fkint.cp.user.Address-1. INFO [2014-07-24 22:02:54,596] [kafka-scheduler-0][] kafka.log.OffsetIndex - Deleting index /var/lib/fk-3p-kafka/logs/dart.fkint.cp.user.Address-1/00000000000036952421.index.deleted INFO [2014-07-24 22:02:54,596] [kafka-scheduler-0][] kafka.log.Log - Deleting segment 36179981 from log dart.fkint.cp.user.Address-0. INFO [2014-07-24 22:02:54,597] [kafka-scheduler-0][] kafka.log.OffsetIndex - Deleting index /var/lib/fk-3p-kafka/logs/dart.fkint.cp.user.Address-0/00000000000036179981.index.deleted INFO [2014-07-24 22:33:20,605] [kafka-request-handler-0][] kafka.log.Log - Rolled new log segment for 'dart.fkint.cp.user.Address-0' in 2 ms. INFO [2014-07-24 22:33:21,483] [kafka-request-handler-0][] kafka.log.Log - Rolled new log segment for 'dart.fkint.cp.user.Address-1' in 2 ms. INFO [2014-07-24 22:36:54,595] [kafka-scheduler-0][] kafka.log.Log - Scheduling log segment 37763015 for log dart.fkint.cp.user.Address-1 for deletion. INFO [2014-07-24 22:36:54,596] [kafka-scheduler-0][] kafka.log.Log - Scheduling log segment 36990624 for log dart.fkint.cp.user.Address-0 for deletion. ```