[druid-user] Keep waiting for handoff after middlemanager is restarted

Hi community,

I increased druid.worker.capacity and then restarted MiddleManager without suspending all the running supervisors. I guess I did something wrong here as tasks have failed after that.

Specifically Kafka ingestion tasks seem to work fine until the last moment: I can query the result from the kafka ingestion task. However, even after segments are successfully pushed to HDFS, the task keeps waiting for handoff to be done and then fails. Attached is the entire log from that task. After that failure, a new task is scheduled for that partition but it also keep waiting for handoff to be done at last and then fails again.

I searched coordinate-overlord-service.log and found the following messages regarding a segment produced by the task in log:

2022-01-11T10:14:42,900 INFO [qtp1953069155-168] org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator - Allocated pending segment [navi-gps_2022-01-11T09:00:00.000Z_2022-01-11T10:00:00.000Z_2022-01-11T09:11:38.515Z_8] for sequence[index_kafka_navi-gps_ae030ea793e6992_0] in DB

2022-01-11T10:14:42,905 INFO [qtp1953069155-173] org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator - Found existing pending segment [navi-gps_2022-01-11T09:00:00.000Z_2022-01-11T10:00:00.000Z_2022-01-11T09:11:38.515Z_8] for sequence[index_kafka_navi-gps_ae030ea793e6992_0] in DB

2022-01-11T11:14:43,300 INFO [qtp1953069155-138] org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator - Published segments to DB: [navi-gps_2022-01-11T09:00:00.000Z_2022-01-11T10:00:00.000Z_2022-01-11T09:11:38.515Z_8, navi-gps_2022-01-11T10:00:00.000Z_2022-01-11T11:00:00.000Z_2022-01-11T10:20:19.334Z_1, navi-gps_2022-01-11T11:00:00.000Z_2022-01-11T12:00:00.000Z_2022-01-11T11:00:00.031Z_7]

Can I get some advice on this? FYI, I’m using Druid-0.22.1 and HDFS for the deep storage and MariaDB for the metadata store.

Best,

Dongwon

(Attachment task.log is missing)

Not sure what went wrong, but we have seen situations similar to these get resolved by restarting the coordinator service.

Hi Sergio,

Not sure what went wrong, but we have seen situations similar to these get resolved by restarting the coordinator service.

It works only for segments that have already been created before restarting the coordinator service.

I found that new tasks launched after the coordinator service are also suffering from the delayed handoff.
I also found that my historicals are busy with unannouncing segments caused by compaction tasks I recently launched.
Do unannouncing take precedence during a normal situation over announcing recent segments created by kafka ingestion tasks?

A related question is how to speed up unannouncing segments from both the coordinator and historicals?

Best,

Dongwon

Hey Dongwon;

I spotted this in the task log:
2022-01-11T10:14:41,374 ERROR [main-EventThread] org.apache.curator.framework.imps.EnsembleTracker - Invalid config event received: {server.1=mobdata-druid-master01.dakao.io:2888:3888:participant, version=0, server.3=mobdata-druid-master03.dakao.io:2888:3888:participant, server.2=mobdata-druid-master02.dakao.io:2888:3888:participant}

Curator is the code responsible for Zookeeper communication: I’m wondering if you have a Zookeeper issue…

Hi Peter :slight_smile:

I opened an issue and summarized what I’ve been through [1]; now all the segments are available on Druid.

But I didn’t notice that error message. It’s really weird because Druid has been working fine for more than a year except the last two days due to heavy bulk segment deletion.

[1] https://github.com/apache/druid/issues/12149

Best,

Dongwon

Somehow yesterday I pressed the wrong button and replied to myself (!) this message below, instead of to you!

Hi peter!

I wonder if your Historicals are out of space? In streaming, the task will wait for Historicals to load the new segments – could you check if you have got enough space available / configured?

I was also suspecting that but each Historical is configured with 8TB and they have a plenty of spaces for segments.

The real problem is, as I said in the previous email, Historical servers were busy with unannouncing deleted segments. I was monitoring each Historical using tail -f var/svhistorical-service.log and found that they were showing lines like

2022-01-12T09:44:42,332 INFO [ZKCoordinator–3] org.apache.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[…]
2022-01-12T09:44:42,332 INFO [ZKCoordinator–3] org.apache.druid.server.coordination.SegmentLoadDropHandler - Completely removing […] in [30,000] millis
2022-01-12T09:44:42,333 INFO [ZkCoordinator] org.apache.druid.server.coordination.ZkCoordinator - zNode[…] was removed
2022-01-12T09:44:42,333 INFO [ZKCoordinator–3] org.apache.druid.server.coordination.ZkCoordinator - Completed request […]

I’d rather increase the value of druid.segmehtCache.numLoadingThreads to speed up Historical’s announcing/unannouncing segments. Also I want the newly created issue to be figured out anytime soon: https://github.com/apache/druid/issues/12149

Best,

Dongwon