Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

coordinator throwing exception trying to load segments #8137

Closed
pdeva opened this issue Jul 23, 2019 · 13 comments · Fixed by #8140
Closed

coordinator throwing exception trying to load segments #8137

pdeva opened this issue Jul 23, 2019 · 13 comments · Fixed by #8140

Comments

@pdeva
Copy link
Contributor

pdeva commented Jul 23, 2019

Affected Version

0.15

Description

started happening all of a sudden. no segments can be loaded in the historical!

2019-07-23 18:17:41,742 ERROR o.a.d.s.c.DruidCoordinator [Coordinator-Exec--0] Caught exception, ignoring so that schedule keeps going.: {class=org.apache.druid.server.coordinator.DruidCoordinator, exceptionType=class java.util.concurrent.RejectedExecutionException, exceptionMessage=Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@57722b5c rejected from java.util.concurrent.ScheduledThreadPoolExecutor@40995ea7[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 34991]}
java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@57722b5c rejected from java.util.concurrent.ScheduledThreadPoolExecutor@40995ea7[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 34991]
	at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2063) ~[?:1.8.0_181]
	at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830) ~[?:1.8.0_181]
	at java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:326) ~[?:1.8.0_181]
	at java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:533) ~[?:1.8.0_181]
	at java.util.concurrent.ScheduledThreadPoolExecutor.submit(ScheduledThreadPoolExecutor.java:632) ~[?:1.8.0_181]
	at org.apache.druid.server.coordinator.CuratorLoadQueuePeon.loadSegment(CuratorLoadQueuePeon.java:181) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.rules.LoadRule.assignPrimary(LoadRule.java:216) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.rules.LoadRule.assign(LoadRule.java:104) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.rules.LoadRule.run(LoadRule.java:77) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.helper.DruidCoordinatorRuleRunner.run(DruidCoordinatorRuleRunner.java:122) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.DruidCoordinator$CoordinatorRunnable.run(DruidCoordinator.java:716) [druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.DruidCoordinator$2.call(DruidCoordinator.java:601) [druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.DruidCoordinator$2.call(DruidCoordinator.java:594) [druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.java.util.common.concurrent.ScheduledExecutors$2.run(ScheduledExecutors.java:92) [druid-core-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_181]
	at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_181]
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_181]
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [?:1.8.0_181]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_181]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_181]
	at java.lang.Thread.run(Thread.java:748) [?:1.8.0_181]

also seeing this:

2019-07-23 18:17:41,733 ERROR o.a.d.s.c.ReplicationThrottler [Coordinator-Exec--0] [_default_tier]: Replicant create queue stuck after 15+ runs!: {class=org.apache.druid.server.coordinator.ReplicationThrottler, segments=[apm-minute_2019-07-23T06:00:00.000Z_2019-07-23T12:00:00.000Z_2019-07-23T05:58:50.795Z ON 192.168.44.101:8080]}
@pdeva
Copy link
Contributor Author

pdeva commented Jul 23, 2019

seems this could be related to #3362

@pdeva
Copy link
Contributor Author

pdeva commented Jul 23, 2019

is there any workaround for this?
restarting things didnt help

@jihoonson
Copy link
Contributor

Hi @pdeva, it looks like a bug, but I have no clue how it could happen. Would you please give some more context? Are you using the supervisor script provided in the druid binary? And, is your ZooKeeper cluster fine?

The error means the executorService is terminated when loadSegment() is called, and the executorService is managed by the coordinator lifecycle which means, the executorService is supposed to terminate when the coordinator stops. I'm not sure how this bug could be related to #3362.

@pdeva
Copy link
Contributor Author

pdeva commented Jul 23, 2019

the reason for #3362 reference is because of the msg "Replicant create queue stuck after 15+ runs".

so literally nothing in my cluster has changed. zookeeper is perfectly fine.
we had decomissioned 2 historical nodes (out of 4) earlier and increased the replication throttle, maxsegments to move, mergeSegmentsLimit.
things were going fine. in fact, we were 90% of the way there to the nodes being decomissioned, when we start seeing this error prop up

@pdeva
Copy link
Contributor Author

pdeva commented Jul 23, 2019

and no i am not using the supervisor script. just starting via
/opt/druid/bin/coordinator.sh start

@pdeva
Copy link
Contributor Author

pdeva commented Jul 23, 2019

here is another more detailed exception:

2019-07-23 18:33:15,490 ERROR o.a.d.s.c.DruidCoordinator [Coordinator-Exec--0] Exception moving segment kube-metrics-second_2019-06-20T11:00:00.000Z_2019-06-20T12:00:00.000Z_2019-06-20T11:00:00.093Z_1: {class=org.apache.druid.server.coordinator.DruidCoordinator, exceptionType=class java.lang.RuntimeException, exceptionMessage=java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@65bf8472 rejected from java.util.concurrent.ScheduledThreadPoolExecutor@63faa5b[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 570]}
java.lang.RuntimeException: java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@65bf8472 rejected from java.util.concurrent.ScheduledThreadPoolExecutor@63faa5b[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 570]
	at org.apache.druid.server.coordinator.DruidCoordinator.moveSegment(DruidCoordinator.java:480) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer.moveSegment(DruidCoordinatorBalancer.java:271) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer.balanceServers(DruidCoordinatorBalancer.java:222) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer.balanceTier(DruidCoordinatorBalancer.java:161) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer.lambda$run$0(DruidCoordinatorBalancer.java:84) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at java.util.HashMap.forEach(HashMap.java:1289) [?:1.8.0_181]
	at org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer.run(DruidCoordinatorBalancer.java:83) [druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.DruidCoordinator$CoordinatorRunnable.run(DruidCoordinator.java:716) [druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.DruidCoordinator$2.call(DruidCoordinator.java:601) [druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.DruidCoordinator$2.call(DruidCoordinator.java:594) [druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.java.util.common.concurrent.ScheduledExecutors$2.run(ScheduledExecutors.java:92) [druid-core-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_181]
	at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_181]
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_181]
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [?:1.8.0_181]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_181]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_181]
	at java.lang.Thread.run(Thread.java:748) [?:1.8.0_181]
Caused by: java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@65bf8472 rejected from java.util.concurrent.ScheduledThreadPoolExecutor@63faa5b[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 570]
	at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2063) ~[?:1.8.0_181]
	at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830) ~[?:1.8.0_181]
	at java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:326) ~[?:1.8.0_181]
	at java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:533) ~[?:1.8.0_181]
	at java.util.concurrent.ScheduledThreadPoolExecutor.submit(ScheduledThreadPoolExecutor.java:632) ~[?:1.8.0_181]
	at org.apache.druid.server.coordinator.CuratorLoadQueuePeon.loadSegment(CuratorLoadQueuePeon.java:181) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	at org.apache.druid.server.coordinator.DruidCoordinator.moveSegment(DruidCoordinator.java:460) ~[druid-server-0.15.0-incubating-iap2.jar:0.15.0-incubating-iap2]
	... 17 more

@jihoonson
Copy link
Contributor

I noticed that it could happen once the coordinator loses leadership and gets it back. Do you see this kind of behavior from the coordinator logs?

@pdeva
Copy link
Contributor Author

pdeva commented Jul 23, 2019

we only have 1 coordinator node

@jihoonson
Copy link
Contributor

Oh, now I notice it can also happen if one of historical has gone..

@pdeva
Copy link
Contributor Author

pdeva commented Jul 23, 2019

i restarted the coordinator (previously just the historicals were restarted) and it seems to be loading again... for now..

@clintropolis
Copy link
Member

I am able to replicate this issue by running a coordinator and killing historical servers. This looks related to #7088, where some executors are getting shutdown by the load queue peons, but the peons don't own the executors. Investigating a fix.

@pdeva
Copy link
Contributor Author

pdeva commented Jul 23, 2019

i am also begining to see this msg. not sure if related. should i open a different bug for this:

2019-07-23 19:26:13,086 ERROR o.a.d.m.IndexerSQLMetadataStorageCoordinator [qtp2120404899-84] Not updating metadata, existing state[KafkaDataSourceMetadata{SeekableStreamStartSequenceNumbers=SeekableStreamEndSequenceNumbers{stream='infraserver', partitionSequenceNumberMap={0=28736041678}}}] in metadata store doesn't match to the new start state[KafkaDataSourceMetadata{SeekableStreamStartSequenceNumbers=SeekableStreamStartSequenceNumbers{stream='infraserver', partitionSequenceNumberMap={0=28730133985}, exclusivePartitions=[]}}].

@jihoonson
Copy link
Contributor

Not sure how come it could be related to this issue yet, so yes. Please open another issue for it.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants