results from kafka-indexing-service unavailable after publishing

424 views
Skip to first unread message

balazs.k...@s4m.io

unread,
Jun 16, 2016, 12:28:44 PM6/16/16
to Druid User
Hi,

I'm trying to use the new kafka-indexing-service of druid 0.9.1-rc3 to ingest one day of our data. The topic contains about 5M events as a JSON string, and I'm using 16 tasks to index them.
The tasks rapidly consume and index all the data, and I can query it in Pivot, but as soon as the tasks end and the segments are supposed to be hand off to historical nodes, the datasource becomes unavailable (query error: no such datasource). All the tasks end with statusCode SUCCESS, and their log looks like this:


2016-06-16T15:09:35,355 INFO [qtp499339307-149] io.druid.indexing.kafka.KafkaIndexTask - Stopping gracefully.
2016-06-16T15:09:35,355 INFO [qtp499339307-149] io.druid.indexing.kafka.KafkaIndexTask - Interrupting run thread (status: [PUBLISHING])
2016-06-16T15:09:35,361 INFO [task-runner-0-priority-0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Shutting down...
2016-06-16T15:09:35,364 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events2_2016-03-01T06:00:00.000Z_2016-03-01T07:00:00.000Z_2016-06-16T13:39:52.443Z_2] at path[/druid/segments/server1:8101/server1:8101_indexer-executor__default_tier_2016-06-16T13:39:30.692Z_a7c606f99b174702837cc5f850ca3e3a0]
2016-06-16T15:09:35,364 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events2_2016-03-01T06:00:00.000Z_2016-03-01T07:00:00.000Z_2016-06-16T13:39:52.443Z_2].
2016-06-16T15:09:35,366 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events2_2016-02-29T23:00:00.000Z_2016-03-01T00:00:00.000Z_2016-06-16T13:39:30.561Z_4] at path[/druid/segments/server1:8101/server1:8101_indexer-executor__default_tier_2016-06-16T13:39:30.692Z_a7c606f99b174702837cc5f850ca3e3a0]
2016-06-16T15:09:35,366 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events2_2016-02-29T23:00:00.000Z_2016-03-01T00:00:00.000Z_2016-06-16T13:39:30.561Z_4].
2016-06-16T15:09:35,367 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events2_2016-03-01T12:00:00.000Z_2016-03-01T13:00:00.000Z_2016-06-16T13:40:08.959Z_6] at path[/druid/segments/server1:8101/server1:8101_indexer-executor__default_tier_2016-06-16T13:39:30.692Z_a7c606f99b174702837cc5f850ca3e3a0]
2016-06-16T15:09:35,367 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events2_2016-03-01T12:00:00.000Z_2016-03-01T13:00:00.000Z_2016-06-16T13:40:08.959Z_6].
2016-06-16T15:09:35,368 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events2_2016-03-01T08:00:00.000Z_2016-03-01T09:00:00.000Z_2016-06-16T13:39:56.971Z_13] at path[/druid/segments/server1:8101/server1:8101_indexer-executor__default_tier_2016-06-16T13:39:30.692Z_a7c606f99b174702837cc5f850ca3e3a0]
[...]
2016-06-16T15:09:35,413 INFO [appenderator_persist_0] io.druid.curator.announcement.Announcer - unannouncing [/druid/segments/server1:8101/server1:8101_indexer-executor__default_tier_2016-06-16T13:39:30.692Z_a7c606f99b174702837cc5f850ca3e3a0]
2016-06-16T15:09:35,467 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events2_2016-03-01T23:00:00.000Z_2016-03-02T00:00:00.000Z_2016-06-16T13:40:51.175Z_14].
2016-06-16T15:09:35,468 INFO [task-runner-0-priority-0] io.druid.indexing.kafka.KafkaIndexTask - The task was asked to stop before completing
2016-06-16T15:09:35,469 INFO [task-runner-0-priority-0] io.druid.indexing.overlord.TaskRunnerUtils - Task [index_kafka_events2_0688efdab4e62c4_onilphmm] status changed to [SUCCESS].
2016-06-16T15:09:35,471 INFO [task-runner-0-priority-0] io.druid.indexing.worker.executor.ExecutorLifecycle - Task completed with status: {
 
"id" : "index_kafka_events2_0688efdab4e62c4_onilphmm",
 
"status" : "SUCCESS",
 
"duration" : 5426688
}



In the coordinator console the datasource is shown as disabled. The table pendingSegments contains also a lot of lines. Can you provide us some hints how could we debug this? 
Our initial thought was that as we haven't configured the retention policy, the segments were automatically dropped instead of being stored on the cold storage, but we couldn't find anything related in the logs. How could we confirm this idea?

Best regards,
Balazs

David Lim

unread,
Jun 16, 2016, 2:15:29 PM6/16/16
to Druid User
Hey Balazs,

Based on the duration in the log you posted, I'm guessing what happened is that the task ran for 60 minutes, published a set of segments, and then waited for historical nodes to pick up the segments which never happened so it timed out 30 minutes later.

The first thing I would do is make sure that you're using the mysql-metadata-storage / postgresql extension that came with the 0.9.1-rc3 instead of the 0.9 or earlier one. More than one other person reported similar symptoms and the issue was a subtle difference between the extensions (that maddeningly doesn't seem to log any errors) that prevented the coordinator from reading the segment metadata table.

Once you've confirmed this, if you're still seeing issues, I would check that the segments table (not the pendingSegments table) has entries that correspond to the data that was ingested. If there are no entries here, the issue is likely with the indexing task, and if there are entries then the issue is likely with the coordinator. If there are entries, the 'used' column will probably be marked as false which corresponds with the coordinator not loading them. In that case, you should make sure that your load rules are not preventing them from being loaded (http://druid.io/docs/latest/operations/rule-configuration.html) and once that's done, try enabling the datasource (you might have to use the old coordinator console to do this, there's a link to the old console from the default one). The rules should be set before enabling the datasource, otherwise it'll just become disabled again when the coordinator next scans the segment metadata to apply the rules.

balazs.k...@s4m.io

unread,
Jun 20, 2016, 11:42:42 AM6/20/16
to Druid User

Thanks David,

Changing the metadata storage extension solved our problems, but we still have some failed ingestion tasks, although I'm not sure it's related:


2016-06-20T08:35:58,597 INFO [task-runner-0-priority-0] io.druid.indexing.common.actions.RemoteTaskActionClient - Submitting action for task[index_kafka_events_05646ab17626d9b_amgnlild] to overlord[http://server37:8090/druid/indexer/v1/action]: SegmentListUsedAction{dataSource='events', intervals=[2016-03-04T16:00:00.000Z/2016-03-05T00:00:00.000Z]} 2016-06-20T08:35:58,597 INFO [task-runner-0-priority-0] com.metamx.http.client.pool.ChannelResourceFactory - Generating: http://server37:8090 2016-06-20T08:35:58,602 WARN [task-runner-0-priority-0] io.druid.segment.realtime.appenderator.FiniteAppenderatorDriver - Our segments don't exist, giving up. 2016-06-20T08:35:58,605 INFO [task-runner-0-priority-0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Shutting down... 2016-06-20T08:35:58,607 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events_2016-03-04T20:00:00.000Z_2016-03-04T21:00:00.000Z_2016-06-20T08:31:48.977Z_1] at path[/druid/segments/server39:8106/server39:8106_indexer-executor__default_tier_2016-06-20T08:30:44.951Z_fecb0733b9d44835bad8441041af98f20] 2016-06-20T08:35:58,608 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events_2016-03-04T20:00:00.000Z_2016-03-04T21:00:00.000Z_2016-06-20T08:31:48.977Z_1]. 2016-06-20T08:35:58,610 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events_2016-03-04T23:00:00.000Z_2016-03-05T00:00:00.000Z_2016-06-20T08:32:39.820Z_2] at path[/druid/segments/server39:8106/server39:8106_indexer-executor__default_tier_2016-06-20T08:30:44.951Z_fecb0733b9d44835bad8441041af98f20] 2016-06-20T08:35:58,610 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events_2016-03-04T23:00:00.000Z_2016-03-05T00:00:00.000Z_2016-06-20T08:32:39.820Z_2]. 2016-06-20T08:35:58,611 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events_2016-03-04T22:00:00.000Z_2016-03-04T23:00:00.000Z_2016-06-20T08:32:25.921Z_3] at path[/druid/segments/server39:8106/server39:8106_indexer-executor__default_tier_2016-06-20T08:30:44.951Z_fecb0733b9d44835bad8441041af98f20] 2016-06-20T08:35:58,611 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events_2016-03-04T22:00:00.000Z_2016-03-04T23:00:00.000Z_2016-06-20T08:32:25.921Z_3]. 2016-06-20T08:35:58,611 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events_2016-03-04T21:00:00.000Z_2016-03-04T22:00:00.000Z_2016-06-20T08:32:07.957Z_5] at path[/druid/segments/server39:8106/server39:8106_indexer-executor__default_tier_2016-06-20T08:30:44.951Z_fecb0733b9d44835bad8441041af98f20] 2016-06-20T08:35:58,612 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events_2016-03-04T21:00:00.000Z_2016-03-04T22:00:00.000Z_2016-06-20T08:32:07.957Z_5]. 2016-06-20T08:35:58,612 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events_2016-03-04T17:00:00.000Z_2016-03-04T18:00:00.000Z_2016-06-20T08:30:50.192Z_9] at path[/druid/segments/server39:8106/server39:8106_indexer-executor__default_tier_2016-06-20T08:30:44.951Z_fecb0733b9d44835bad8441041af98f20] 2016-06-20T08:35:58,612 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events_2016-03-04T17:00:00.000Z_2016-03-04T18:00:00.000Z_2016-06-20T08:30:50.192Z_9]. 2016-06-20T08:35:58,613 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events_2016-03-04T18:00:00.000Z_2016-03-04T19:00:00.000Z_2016-06-20T08:31:03.625Z_8] at path[/druid/segments/server39:8106/server39:8106_indexer-executor__default_tier_2016-06-20T08:30:44.951Z_fecb0733b9d44835bad8441041af98f20] 2016-06-20T08:35:58,613 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events_2016-03-04T18:00:00.000Z_2016-03-04T19:00:00.000Z_2016-06-20T08:31:03.625Z_8]. 2016-06-20T08:35:58,613 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events_2016-03-04T19:00:00.000Z_2016-03-04T20:00:00.000Z_2016-06-20T08:31:25.988Z_1] at path[/druid/segments/server39:8106/server39:8106_indexer-executor__default_tier_2016-06-20T08:30:44.951Z_fecb0733b9d44835bad8441041af98f20] 2016-06-20T08:35:58,613 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events_2016-03-04T19:00:00.000Z_2016-03-04T20:00:00.000Z_2016-06-20T08:31:25.988Z_1]. 2016-06-20T08:35:58,614 INFO [appenderator_persist_0] io.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[events_2016-03-04T16:00:00.000Z_2016-03-04T17:00:00.000Z_2016-06-20T08:30:32.682Z_10] at path[/druid/segments/server39:8106/server39:8106_indexer-executor__default_tier_2016-06-20T08:30:44.951Z_fecb0733b9d44835bad8441041af98f20] 2016-06-20T08:35:58,614 INFO [appenderator_persist_0] io.druid.curator.announcement.Announcer - unannouncing [/druid/segments/server39:8106/server39:8106_indexer-executor__default_tier_2016-06-20T08:30:44.951Z_fecb0733b9d44835bad8441041af98f20] 2016-06-20T08:35:58,658 INFO [appenderator_persist_0] io.druid.segment.realtime.appenderator.AppenderatorImpl - Removing sink for segment[events_2016-03-04T16:00:00.000Z_2016-03-04T17:00:00.000Z_2016-06-20T08:30:32.682Z_10]. 2016-06-20T08:35:58,660 ERROR [task-runner-0-priority-0] io.druid.indexing.overlord.ThreadPoolTaskRunner - Exception while running task[KafkaIndexTask{id=index_kafka_events_05646ab17626d9b_amgnlild, type=index_kafka, dataSource=events}] com.metamx.common.ISE: Transaction failure publishing segments, aborting at io.druid.indexing.kafka.KafkaIndexTask.run(KafkaIndexTask.java:506) ~[?:?] at io.druid.indexing.overlord.ThreadPoolTaskRunner$ThreadPoolTaskRunnerCallable.call(ThreadPoolTaskRunner.java:436) [druid-indexing-service-0.9.1-rc4.jar:0.9.1-rc4] at io.druid.indexing.overlord.ThreadPoolTaskRunner$ThreadPoolTaskRunnerCallable.call(ThreadPoolTaskRunner.java:408) [druid-indexing-service-0.9.1-rc4.jar:0.9.1-rc4] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_91] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [?:1.8.0_91] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [?:1.8.0_91] at java.lang.Thread.run(Thread.java:745) [?:1.8.0_91] 2016-06-20T08:35:58,664 INFO [task-runner-0-priority-0] io.druid.indexing.overlord.TaskRunnerUtils - Task [index_kafka_events_05646ab17626d9b_amgnlild] status changed to [FAILED]. 2016-06-20T08:35:58,666 INFO [task-runner-0-priority-0] io.druid.indexing.worker.executor.ExecutorLifecycle - Task completed with status: { "id" : "index_kafka_events_05646ab17626d9b_amgnlild", "status" : "FAILED", "duration" : 314328 }



Does it look familiar to you? For the record, I set the task duration to 5 minutes in our tests.



David Lim

unread,
Jun 20, 2016, 11:58:45 AM6/20/16
to Druid User
Hey Balazs,

Good to hear updating the extension helped.

Are you seeing this failure for all your tasks or only some of them? I'm wondering if your metadata tables may have gotten into a weird state as a result of the bad extension. If you're able to do so, shutdown the supervisor, kill the Kafka indexing tasks, and try removing the entries in the druid_dataSource and druid_pendingSegments before resubmitting the supervisor and see if that helps.

Balázs Kossovics

unread,
Jun 20, 2016, 1:02:44 PM6/20/16
to druid...@googlegroups.com

Now I double checked, and we only had one task failing with this error. I asked our ops, and around the time of the error they had to remount the coldstorage's partition on this server because of a problem, so probably this was the source of the failure. Sorry for the false alert.

Actually we had to wipe clean our Druid cluster after changing the extension, because it was the only way of resurrect it (well, as we had no data on it, it maybe wasn't the only way, but it was certainly the easiest)

-- You received this message because you are subscribed to a topic in the Google Groups "Druid User" group. To unsubscribe from this topic, visit https://groups.google.com/d/topic/druid-user/mERo4LQfSBs/unsubscribe. To unsubscribe from this group and all its topics, send an email to druid-user+...@googlegroups.com. To post to this group, send email to druid...@googlegroups.com. To view this discussion on the web visit https://groups.google.com/d/msgid/druid-user/f9585fb9-c271-4d31-b96c-22c0db37b82b%40googlegroups.com. For more options, visit https://groups.google.com/d/optout.

David Lim

unread,
Jun 20, 2016, 1:15:06 PM6/20/16
to Druid User
Cool, let us know how things go!
Reply all
Reply to author
Forward
0 new messages