gianm opened a new issue, #13132:
URL: https://github.com/apache/druid/issues/13132

   Reported in Slack here: 
https://apachedruidworkspace.slack.com/archives/C0309C9L90D/p1663633728125109. 
Logs reproduced below.
   
   I think what is happening here is that the changes in #12404 make the task 
`type` from the Overlord tasks API return `null` until the metadata store 
update finishes in the background. (See 
`SQLMetadataStorageActionHandler#populateTaskTypeAndGroupId`.) It happens 
automatically when the Overlord starts up and logs messages like:
   
   > Populate fields task and group_id of task entry table [%s] from payload
   
   And:
   
   > Task migration for table [druid_tasks] successful
   
   During this time, the Coordinator is unable to properly filter out 
non-compaction tasks from the task list. It tries to deserialize non-compaction 
tasks as ClientTaskQuery, which it can't do. (That class only supports a subset 
of task types.)
   
   I believe this situation would resolve on its own once the metadata store 
update is complete.
   
   Logs:
   
   ```
   2022-09-19T23:08:35,342 ERROR [Coordinator-Exec--0] 
org.apache.druid.server.coordinator.DruidCoordinator - Caught exception, 
ignoring so that schedule keeps going.: 
{class=org.apache.druid.server.coordinator.DruidCoordinator, 
exceptionType=class java.lang.RuntimeException, 
exceptionMessage=com.fasterxml.jackson.databind.exc.InvalidTypeIdException: 
Please make sure to load all the necessary extensions and jars with type 
'index_kafka' on 'druid/coordinator' service. Could not resolve type id 
'index_kafka' as a subtype of 
`org.apache.druid.client.indexing.ClientTaskQuery` known type ids = [compact, 
kill] (for POJO property 'payload')
    at [Source: 
(String)"{"task":"index_kafka_druid_v2_30a330c471e9073_afjlnglj","payload":{"type":"index_kafka","id":"index_kafka_druid_v2_30a330c471e9073_afjlnglj","resource":{"availabilityGroup":"index_kafka_druid_v2_30a330c471e9073","requiredCapacity":1},"dataSchema":{"dataSource":"druid_v2","timestampSpec":{"column":"timestamp","format":"millis","missingValue":null},"dimensionsSpec":{"dimensions":[{"type":"string","name":"companyID","multiValueHandling":"SORTED_ARRAY","createBitmapIndex":true},{"type":"string","nam"[truncated
 5767 chars]; line: 1, column: 75] (through reference chain: 
org.apache.druid.client.indexing.TaskPayloadResponse["payload"])}
   java.lang.RuntimeException: 
com.fasterxml.jackson.databind.exc.InvalidTypeIdException: Please make sure to 
load all the necessary extensions and jars with type 'index_kafka' on 
'druid/coordinator' service. Could not resolve type id 'index_kafka' as a 
subtype of `org.apache.druid.client.indexing.ClientTaskQuery` known type ids = 
[compact, kill] (for POJO property 'payload')
    at [Source: 
(String)"{"task":"index_kafka_druid_v2_30a330c471e9073_afjlnglj","payload":{"type":"index_kafka","id":"index_kafka_druid_v2_30a330c471e9073_afjlnglj","resource":{"availabilityGroup":"index_kafka_druid_v2_30a330c471e9073","requiredCapacity":1},"dataSchema":{"dataSource":"druid_v2","timestampSpec":{"column":"timestamp","format":"millis","missingValue":null},"dimensionsSpec":{"dimensions":[{"type":"string","name":"companyID","multiValueHandling":"SORTED_ARRAY","createBitmapIndex":true},{"type":"string","nam"[truncated
 5767 chars]; line: 1, column: 75] (through reference chain: 
org.apache.druid.client.indexing.TaskPayloadResponse["payload"])
        at 
org.apache.druid.client.indexing.HttpIndexingServiceClient.getTaskPayload(HttpIndexingServiceClient.java:351)
 ~[druid-server-24.0.0.jar:24.0.0]
        at 
org.apache.druid.server.coordinator.duty.CompactSegments.run(CompactSegments.java:143)
 ~[druid-server-24.0.0.jar:24.0.0]
        at 
org.apache.druid.server.coordinator.DruidCoordinator$DutiesRunnable.run(DruidCoordinator.java:948)
 ~[druid-server-24.0.0.jar:24.0.0]
        at 
org.apache.druid.server.coordinator.DruidCoordinator$2.call(DruidCoordinator.java:721)
 ~[druid-server-24.0.0.jar:24.0.0]
        at 
org.apache.druid.server.coordinator.DruidCoordinator$2.call(DruidCoordinator.java:714)
 ~[druid-server-24.0.0.jar:24.0.0]
        at 
org.apache.druid.java.util.common.concurrent.ScheduledExecutors$4.run(ScheduledExecutors.java:163)
 ~[druid-core-24.0.0.jar:24.0.0]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[?:1.8.0_275]
        at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
~[?:1.8.0_275]
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
 ~[?:1.8.0_275]
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
 ~[?:1.8.0_275]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
~[?:1.8.0_275]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
~[?:1.8.0_275]
        at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_275]
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to