Uploaded image for project: 'Aurora'
  1. Aurora
  2. AURORA-1780

Offers with unknown resources types to Aurora crash the scheduler

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 0.17.0
    • None
    • None
    • vagrant

    Description

      Taking offers from Agents which have resources that are not known to Aurora cause the Scheduler to crash.

      Steps to reproduce:

      vagrant up
      sudo service mesos-slave stop
      echo "cpus(aurora-role):0.5;cpus(*):3.5;mem(aurora-role):1024;disk:20000;gpus(*):4;test:200" | sudo tee /etc/mesos-slave/resources
      sudo rm -f /var/lib/mesos/meta/slaves/latest
      sudo service mesos-slave start
      

      Wait around a few moments for the offer to be made to Aurora

      I0922 02:41:57.839 [Thread-19, MesosSchedulerImpl:142] Received notification of lost agent: value: "cadaf569-171d-42fc-a417-fbd608ea5bab-S0"
      
      I0922 02:42:30.585597  2999 log.cpp:577] Attempting to append 109 bytes to the log
      I0922 02:42:30.585654  2999 coordinator.cpp:348] Coordinator attempting to write APPEND action at position 4
      I0922 02:42:30.585747  2999 replica.cpp:537] Replica received write request for position 4 from (10)@192.168.33.7:8083
      I0922 02:42:30.586858  2999 leveldb.cpp:341] Persisting action (125 bytes) to leveldb took 1.086601ms
      I0922 02:42:30.586897  2999 replica.cpp:712] Persisted action at 4
      I0922 02:42:30.587020  2999 replica.cpp:691] Replica received learned notice for position 4 from @0.0.0.0:0
      I0922 02:42:30.587785  2999 leveldb.cpp:341] Persisting action (127 bytes) to leveldb took 746999ns
      I0922 02:42:30.587805  2999 replica.cpp:712] Persisted action at 4
      I0922 02:42:30.587811  2999 replica.cpp:697] Replica learned APPEND action at position 4
      I0922 02:42:30.601 [SchedulerImpl-0, OfferManager$OfferManagerImpl:185] Returning offers for cadaf569-171d-42fc-a417-fbd608ea5bab-S1 for compaction.
      Sep 22, 2016 2:42:38 AM com.google.common.util.concurrent.ServiceManager$ServiceListener failed
      SEVERE: Service SlotSizeCounterService [FAILED] has failed in the RUNNING state.
      java.lang.NullPointerException: Unknown Mesos resource: name: "test"
      type: SCALAR
      scalar {
        value: 200.0
      }
      role: "*"
      
      	at java.util.Objects.requireNonNull(Objects.java:228)
      	at org.apache.aurora.scheduler.resources.ResourceType.fromResource(ResourceType.java:355)
      	at org.apache.aurora.scheduler.resources.ResourceManager.lambda$static$0(ResourceManager.java:52)
      	at com.google.common.collect.Iterators$7.computeNext(Iterators.java:675)
      	at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
      	at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
      	at java.util.Iterator.forEachRemaining(Iterator.java:115)
      	at java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801)
      	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481)
      	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471)
      	at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
      	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
      	at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499)
      	at org.apache.aurora.scheduler.resources.ResourceManager.bagFromResources(ResourceManager.java:274)
      	at org.apache.aurora.scheduler.resources.ResourceManager.bagFromMesosResources(ResourceManager.java:239)
      	at org.apache.aurora.scheduler.stats.AsyncStatsModule$OfferAdapter.get(AsyncStatsModule.java:153)
      	at org.apache.aurora.scheduler.stats.SlotSizeCounter.run(SlotSizeCounter.java:168)
      	at org.apache.aurora.scheduler.stats.AsyncStatsModule$SlotSizeCounterService.runOneIteration(AsyncStatsModule.java:130)
      	at com.google.common.util.concurrent.AbstractScheduledService$ServiceDelegate$Task.run(AbstractScheduledService.java:189)
      	at com.google.common.util.concurrent.Callables$3.run(Callables.java:100)
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      	at java.lang.Thread.run(Thread.java:745)
      
      E0922 02:42:38.353 [SlotSizeCounterService RUNNING, GuavaUtils$LifecycleShutdownListener:55] Service: SlotSizeCounterService [FAILED] failed unexpectedly. Triggering shutdown.
      I0922 02:42:38.353 [SlotSizeCounterService RUNNING, Lifecycle:84] Shutting down application
      I0922 02:42:38.354 [SlotSizeCounterService RUNNING, ShutdownRegistry$ShutdownRegistryImpl:77] Executing 4 shutdown commands.
      I0922 02:42:38.356 [SlotSizeCounterService RUNNING, StateMachine$Builder:389] SchedulerLifecycle state machine transition ACTIVE -> DEAD
      I0922 02:42:38.373028  4029 sched.cpp:1987] Asked to stop the driver
      I0922 02:42:38.373152  3000 sched.cpp:1187] Stopping framework 'cadaf569-171d-42fc-a417-fbd608ea5bab-0000'
      I0922 02:42:38.373 [BlockingDriverJoin, SchedulerLifecycle$6:267] Driver exited, terminating lifecycle.
      I0922 02:42:38.374 [BlockingDriverJoin, StateMachine$Builder:389] SchedulerLifecycle state machine transition DEAD -> DEAD
      I0922 02:42:38.374 [BlockingDriverJoin, SchedulerLifecycle$7:287] Shutdown already invoked, ignoring extra call.
      I0922 02:42:38.375 [SlotSizeCounterService RUNNING, StateMachine$Builder:389] storage state machine transition READY -> STOPPED
      I0922 02:42:38.392 [CronLifecycle STOPPING, CronLifecycle:90] Shutting down Quartz cron scheduler.
      I0922 02:42:38.392 [CronLifecycle STOPPING, QuartzScheduler:694] Scheduler QuartzScheduler_$_aurora-cron-1 shutting down.
      I0922 02:42:38.392 [CronLifecycle STOPPING, QuartzScheduler:613] Scheduler QuartzScheduler_$_aurora-cron-1 paused.
      I0922 02:42:38.394 [CronLifecycle STOPPING, QuartzScheduler:771] Scheduler QuartzScheduler_$_aurora-cron-1 shutdown complete.
      W0922 02:42:43.450 [SlotSizeCounterService RUNNING, ShutdownRegistry$ShutdownRegistryImpl:87] Shutdown action failed. java.util.concurrent.TimeoutException: Timeout waiting for the services to stop. The following services have not stopped: {STOPPING=[TaskGroupBatchWorker [STOPPING], TaskEventBatchWorker [STOPPING], CronBatchWorker [STOPPING]]}
      	at com.google.common.util.concurrent.ServiceManager$ServiceManagerState.awaitStopped(ServiceManager.java:571) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.ServiceManager.awaitStopped(ServiceManager.java:349) ~[guava-19.0.jar:na]
      	at org.apache.aurora.GuavaUtils$1.awaitStopped(GuavaUtils.java:139) ~[aurora-0.17.0-SNAPSHOT.jar:na]
      	at org.apache.aurora.scheduler.SchedulerLifecycle$3.execute(SchedulerLifecycle.java:221) ~[aurora-0.17.0-SNAPSHOT.jar:na]
      	at org.apache.aurora.common.application.ShutdownRegistry$ShutdownRegistryImpl.execute(ShutdownRegistry.java:85) ~[commons-0.17.0-SNAPSHOT.jar:na]
      	at org.apache.aurora.common.application.Lifecycle.shutdown(Lifecycle.java:85) [commons-0.17.0-SNAPSHOT.jar:na]
      	at org.apache.aurora.GuavaUtils$LifecycleShutdownListener.failure(GuavaUtils.java:56) [aurora-0.17.0-SNAPSHOT.jar:na]
      	at com.google.common.util.concurrent.ServiceManager$ServiceManagerState$2.call(ServiceManager.java:695) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.ServiceManager$ServiceManagerState$2.call(ServiceManager.java:693) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.ListenerCallQueue.run(ListenerCallQueue.java:118) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java:456) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.ListenerCallQueue.execute(ListenerCallQueue.java:86) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.ServiceManager$ServiceManagerState.executeListeners(ServiceManager.java:706) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.ServiceManager$ServiceManagerState.transitionService(ServiceManager.java:677) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.ServiceManager$ServiceListener.failed(ServiceManager.java:781) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.AbstractService$5.call(AbstractService.java:509) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.AbstractService$5.call(AbstractService.java:507) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.ListenerCallQueue.run(ListenerCallQueue.java:118) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java:456) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.ListenerCallQueue.execute(ListenerCallQueue.java:86) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.AbstractService.executeListeners(AbstractService.java:458) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.AbstractService.notifyFailed(AbstractService.java:407) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.AbstractScheduledService$ServiceDelegate$Task.run(AbstractScheduledService.java:197) [guava-19.0.jar:na]
      	at com.google.common.util.concurrent.Callables$3.run(Callables.java:100) [guava-19.0.jar:na]
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_91]
      	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_91]
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_91]
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_91]
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_91]
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_91]
      	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_91]
      
      I0922 02:42:43.453 [Curator-Framework-0, CuratorFrameworkImpl:821] backgroundOperationsLoop exiting
      I0922 02:42:43.458 [main-EventThread, ClientCnxn$EventThread:519] EventThread shut down for session: 0x1574fa2880c000a
      I0922 02:42:43.458 [SlotSizeCounterService RUNNING, ZooKeeper:684] Session: 0x1574fa2880c000a closed
      I0922 02:42:43.459 [main, SchedulerMain:101] Stopping scheduler services.
      I0922 02:42:43.459 [TimeSeriesRepositoryImpl STOPPING, TimeSeriesRepositoryImpl:168] Variable sampler shut down
      I0922 02:42:43.462 [TearDownShutdownRegistry STOPPING, ShutdownRegistry$ShutdownRegistryImpl:95] Action controller has already completed, subsequent calls ignored.
      I0922 02:42:43.464 [HttpServerLauncher STOPPING, JettyServerModule$HttpServerLauncher:413] Shutting down embedded http server
      I0922 02:42:43.470 [HttpServerLauncher STOPPING, AbstractConnector:310] Stopped ServerConnector@5c738e5{HTTP/1.1,[http/1.1]}{0.0.0.0:8081}
      I0922 02:42:43.473 [HttpServerLauncher STOPPING, ContextHandler:910] Stopped o.e.j.s.ServletContextHandler@5edf9dde{/,null,UNAVAILABLE}
      I0922 02:42:43.476 [main, SchedulerMain:187] Application run() exited.
      2016-09-22 02:42:58,164:2942(0x7f307f9b5700):ZOO_WARN@zookeeper_interest@1570: Exceeded deadline by 11ms
      E0922 02:43:38.369 [SlaStat-0, AsyncUtil:159] java.util.concurrent.ExecutionException: org.apache.aurora.scheduler.storage.Storage$TransientStorageException: Storage is not READY java.util.concurrent.ExecutionException: org.apache.aurora.scheduler.storage.Storage$TransientStorageException: Storage is not READY
      	at java.util.concurrent.FutureTask.report(FutureTask.java:122) ~[na:1.8.0_91]
      	at java.util.concurrent.FutureTask.get(FutureTask.java:192) ~[na:1.8.0_91]
      	at org.apache.aurora.scheduler.base.AsyncUtil.evaluateResult(AsyncUtil.java:154) [aurora-0.17.0-SNAPSHOT.jar:na]
      	at org.apache.aurora.scheduler.base.AsyncUtil.access$000(AsyncUtil.java:35) [aurora-0.17.0-SNAPSHOT.jar:na]
      	at org.apache.aurora.scheduler.base.AsyncUtil$1.afterExecute(AsyncUtil.java:65) [aurora-0.17.0-SNAPSHOT.jar:na]
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1150) [na:1.8.0_91]
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_91]
      	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_91]
      Caused by: org.apache.aurora.scheduler.storage.Storage$TransientStorageException: Storage is not READY
      	at org.apache.aurora.scheduler.storage.CallOrderEnforcingStorage.checkInState(CallOrderEnforcingStorage.java:79) ~[aurora-0.17.0-SNAPSHOT.jar:na]
      	at org.apache.aurora.scheduler.storage.CallOrderEnforcingStorage.read(CallOrderEnforcingStorage.java:112) ~[aurora-0.17.0-SNAPSHOT.jar:na]
      	at org.apache.aurora.scheduler.storage.Storage$Util.fetchTasks(Storage.java:299) ~[aurora-0.17.0-SNAPSHOT.jar:na]
      	at org.apache.aurora.scheduler.sla.MetricCalculator.run(MetricCalculator.java:183) ~[aurora-0.17.0-SNAPSHOT.jar:na]
      	at org.apache.aurora.common.inject.TimedInterceptor.invoke(TimedInterceptor.java:83) ~[commons-0.17.0-SNAPSHOT.jar:na]
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_91]
      	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) ~[na:1.8.0_91]
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) ~[na:1.8.0_91]
      	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) ~[na:1.8.0_91]
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_91]
      	... 2 common frames omitted
      

      Attachments

        Activity

          People

            renan Renan DelValle
            renan Renan DelValle
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: