You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@mesos.apache.org by "Joseph Wu (JIRA)" <ji...@apache.org> on 2016/03/31 20:53:25 UTC

[jira] [Commented] (MESOS-5074) SlaveTest.RemoveUnregisteredTerminatedExecutor is flaky

    [ https://issues.apache.org/jira/browse/MESOS-5074?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15220445#comment-15220445 ] 

Joseph Wu commented on MESOS-5074:
----------------------------------

This appears to be a race between two operations on the {{TestContainerizer}}
* Here, the {{TestContainerizer}} creates a promise on a future termination.  https://github.com/apache/mesos/blob/b2e6228378f93b4c21c22ae9a1b6fdd7c12b90a6/src/tests/slave_tests.cpp#L299
* Later in the test, we satisfy the promise and immediately delete it.  https://github.com/apache/mesos/blob/b2e6228378f93b4c21c22ae9a1b6fdd7c12b90a6/src/tests/slave_tests.cpp#L311-L312 

The {{TestContainerizer}} is unfortunately *not* a libprocess process, so it can be interleaved.
The following interleaving can lead to this failure:
| {{TestContainerizer::_wait}} | {{TestContainerizer::_destroy}} |
| {code}
  if (!promises.contains(containerId)) {
    return Failure("Unknown container: " + stringify(containerId));
  }

  // ----->




  

  return promises[containerId]->future();
{code} | {code}
  if (promises.contains(containerId)) {
    containerizer::Termination termination;
    termination.set_message("Killed executor");
    termination.set_status(0);

    promises[containerId]->set(termination);
    promises.erase(containerId);

    // <----
  }
{code}

> SlaveTest.RemoveUnregisteredTerminatedExecutor is flaky
> -------------------------------------------------------
>
>                 Key: MESOS-5074
>                 URL: https://issues.apache.org/jira/browse/MESOS-5074
>             Project: Mesos
>          Issue Type: Bug
>    Affects Versions: 0.28.0
>         Environment: Ubuntu 14.04, using clang, with libevent and SSL enabled
>            Reporter: Greg Mann
>              Labels: mesosphere, tests
>
> Observed on the ASF CI:
> {code}
> [ RUN      ] SlaveTest.RemoveUnregisteredTerminatedExecutor
> I0331 17:09:45.304322  1299 cluster.cpp:139] Creating default 'local' authorizer
> I0331 17:09:45.310246  1299 leveldb.cpp:174] Opened db in 5.653849ms
> I0331 17:09:45.311182  1299 leveldb.cpp:181] Compacted db in 897714ns
> I0331 17:09:45.311238  1299 leveldb.cpp:196] Created db iterator in 35116ns
> I0331 17:09:45.311264  1299 leveldb.cpp:202] Seeked to beginning of db in 11547ns
> I0331 17:09:45.311280  1299 leveldb.cpp:271] Iterated through 0 keys in the db in 8343ns
> I0331 17:09:45.311334  1299 replica.cpp:779] Replica recovered with log positions 0 -> 0 with 1 holes and 0 unlearned
> I0331 17:09:45.312374  1328 recover.cpp:447] Starting replica recovery
> I0331 17:09:45.312685  1327 recover.cpp:473] Replica is in EMPTY status
> I0331 17:09:45.313987  1318 replica.cpp:673] Replica in EMPTY status received a broadcasted recover request from (12029)@172.17.0.2:47264
> I0331 17:09:45.314481  1321 recover.cpp:193] Received a recover response from a replica in EMPTY status
> I0331 17:09:45.314913  1324 recover.cpp:564] Updating replica status to STARTING
> I0331 17:09:45.315553  1319 leveldb.cpp:304] Persisting metadata (8 bytes) to leveldb took 527754ns
> I0331 17:09:45.315587  1319 replica.cpp:320] Persisted replica status to STARTING
> I0331 17:09:45.315762  1326 recover.cpp:473] Replica is in STARTING status
> I0331 17:09:45.317013  1322 replica.cpp:673] Replica in STARTING status received a broadcasted recover request from (12030)@172.17.0.2:47264
> I0331 17:09:45.317533  1322 recover.cpp:193] Received a recover response from a replica in STARTING status
> I0331 17:09:45.317993  1319 recover.cpp:564] Updating replica status to VOTING
> I0331 17:09:45.318450  1322 leveldb.cpp:304] Persisting metadata (8 bytes) to leveldb took 354962ns
> I0331 17:09:45.318478  1322 replica.cpp:320] Persisted replica status to VOTING
> I0331 17:09:45.318559  1319 recover.cpp:578] Successfully joined the Paxos group
> I0331 17:09:45.318720  1319 recover.cpp:462] Recover process terminated
> I0331 17:09:45.320158  1317 master.cpp:376] Master 9b06f6db-72f1-42be-b262-98eb1ddb86e2 (29ba6f7aca97) started on 172.17.0.2:47264
> I0331 17:09:45.320196  1317 master.cpp:378] Flags at startup: --acls="" --allocation_interval="1secs" --allocator="HierarchicalDRF" --authenticate="true" --authenticate_http="true" --authenticate_slaves="true" --authenticators="crammd5" --authorizers="local" --credentials="/tmp/HoiV0t/credentials" --framework_sorter="drf" --help="false" --hostname_lookup="true" --http_authenticators="basic" --initialize_driver_logging="true" --log_auto_initialize="true" --logbufsecs="0" --logging_level="INFO" --max_completed_frameworks="50" --max_completed_tasks_per_framework="1000" --max_slave_ping_timeouts="5" --quiet="false" --recovery_slave_removal_limit="100%" --registry="replicated_log" --registry_fetch_timeout="1mins" --registry_store_timeout="100secs" --registry_strict="true" --root_submissions="true" --slave_ping_timeout="15secs" --slave_reregister_timeout="10mins" --user_sorter="drf" --version="false" --webui_dir="/mesos/mesos-0.29.0/_inst/share/mesos/webui" --work_dir="/tmp/HoiV0t/master" --zk_session_timeout="10secs"
> I0331 17:09:45.320572  1317 master.cpp:427] Master only allowing authenticated frameworks to register
> I0331 17:09:45.320585  1317 master.cpp:432] Master only allowing authenticated slaves to register
> I0331 17:09:45.320595  1317 credentials.hpp:37] Loading credentials for authentication from '/tmp/HoiV0t/credentials'
> I0331 17:09:45.321111  1317 master.cpp:474] Using default 'crammd5' authenticator
> I0331 17:09:45.321318  1317 master.cpp:545] Using default 'basic' HTTP authenticator
> I0331 17:09:45.321517  1317 master.cpp:583] Authorization enabled
> I0331 17:09:45.321770  1325 hierarchical.cpp:144] Initialized hierarchical allocator process
> I0331 17:09:45.321846  1325 whitelist_watcher.cpp:77] No whitelist given
> I0331 17:09:45.325551  1317 master.cpp:1826] The newly elected leader is master@172.17.0.2:47264 with id 9b06f6db-72f1-42be-b262-98eb1ddb86e2
> I0331 17:09:45.325592  1317 master.cpp:1839] Elected as the leading master!
> I0331 17:09:45.325604  1317 master.cpp:1526] Recovering from registrar
> I0331 17:09:45.325875  1331 registrar.cpp:307] Recovering registrar
> I0331 17:09:45.326637  1331 log.cpp:659] Attempting to start the writer
> I0331 17:09:45.328130  1324 replica.cpp:493] Replica received implicit promise request from (12032)@172.17.0.2:47264 with proposal 1
> I0331 17:09:45.328989  1324 leveldb.cpp:304] Persisting metadata (8 bytes) to leveldb took 832457ns
> I0331 17:09:45.329020  1324 replica.cpp:342] Persisted promised to 1
> I0331 17:09:45.329779  1329 coordinator.cpp:238] Coordinator attempting to fill missing positions
> I0331 17:09:45.332118  1329 replica.cpp:388] Replica received explicit promise request from (12033)@172.17.0.2:47264 for position 0 with proposal 2
> I0331 17:09:45.332484  1329 leveldb.cpp:341] Persisting action (8 bytes) to leveldb took 333132ns
> I0331 17:09:45.332510  1329 replica.cpp:712] Persisted action at 0
> I0331 17:09:45.333724  1318 replica.cpp:537] Replica received write request for position 0 from (12034)@172.17.0.2:47264
> I0331 17:09:45.333792  1318 leveldb.cpp:436] Reading position from leveldb took 45226ns
> I0331 17:09:45.334182  1318 leveldb.cpp:341] Persisting action (14 bytes) to leveldb took 358530ns
> I0331 17:09:45.334213  1318 replica.cpp:712] Persisted action at 0
> I0331 17:09:45.335052  1318 replica.cpp:691] Replica received learned notice for position 0 from @0.0.0.0:0
> I0331 17:09:45.335985  1318 leveldb.cpp:341] Persisting action (16 bytes) to leveldb took 920454ns
> I0331 17:09:45.336016  1318 replica.cpp:712] Persisted action at 0
> I0331 17:09:45.336040  1318 replica.cpp:697] Replica learned NOP action at position 0
> I0331 17:09:45.336995  1327 log.cpp:675] Writer started with ending position 0
> I0331 17:09:45.338444  1319 leveldb.cpp:436] Reading position from leveldb took 66714ns
> I0331 17:09:45.339751  1323 registrar.cpp:340] Successfully fetched the registry (0B) in 13.831936ms
> I0331 17:09:45.339889  1323 registrar.cpp:439] Applied 1 operations in 45178ns; attempting to update the 'registry'
> I0331 17:09:45.340608  1325 log.cpp:683] Attempting to append 170 bytes to the log
> I0331 17:09:45.340701  1319 coordinator.cpp:348] Coordinator attempting to write APPEND action at position 1
> I0331 17:09:45.341581  1320 replica.cpp:537] Replica received write request for position 1 from (12035)@172.17.0.2:47264
> I0331 17:09:45.342003  1320 leveldb.cpp:341] Persisting action (189 bytes) to leveldb took 395996ns
> I0331 17:09:45.342036  1320 replica.cpp:712] Persisted action at 1
> I0331 17:09:45.343068  1320 replica.cpp:691] Replica received learned notice for position 1 from @0.0.0.0:0
> I0331 17:09:45.343983  1320 leveldb.cpp:341] Persisting action (191 bytes) to leveldb took 781249ns
> I0331 17:09:45.344081  1320 replica.cpp:712] Persisted action at 1
> I0331 17:09:45.344189  1320 replica.cpp:697] Replica learned APPEND action at position 1
> I0331 17:09:45.345816  1320 registrar.cpp:484] Successfully updated the 'registry' in 5.862144ms
> I0331 17:09:45.346009  1319 log.cpp:702] Attempting to truncate the log to 1
> I0331 17:09:45.346058  1320 registrar.cpp:370] Successfully recovered registrar
> I0331 17:09:45.346191  1319 coordinator.cpp:348] Coordinator attempting to write TRUNCATE action at position 2
> I0331 17:09:45.347513  1317 replica.cpp:537] Replica received write request for position 2 from (12036)@172.17.0.2:47264
> I0331 17:09:45.347892  1320 master.cpp:1634] Recovered 0 slaves from the Registry (131B) ; allowing 10mins for slaves to re-register
> I0331 17:09:45.347992  1317 leveldb.cpp:341] Persisting action (16 bytes) to leveldb took 434355ns
> I0331 17:09:45.348021  1317 replica.cpp:712] Persisted action at 2
> I0331 17:09:45.348026  1323 hierarchical.cpp:171] Skipping recovery of hierarchical allocator: nothing to recover
> I0331 17:09:45.348672  1326 replica.cpp:691] Replica received learned notice for position 2 from @0.0.0.0:0
> I0331 17:09:45.349053  1326 leveldb.cpp:341] Persisting action (18 bytes) to leveldb took 362152ns
> I0331 17:09:45.349113  1326 leveldb.cpp:399] Deleting ~1 keys from leveldb took 44449ns
> I0331 17:09:45.349138  1326 replica.cpp:712] Persisted action at 2
> I0331 17:09:45.349161  1326 replica.cpp:697] Replica learned TRUNCATE action at position 2
> I0331 17:09:45.364617  1331 slave.cpp:200] Slave started on 357)@172.17.0.2:47264
> I0331 17:09:45.364680  1331 slave.cpp:201] Flags at startup: --appc_simple_discovery_uri_prefix="http://" --appc_store_dir="/tmp/mesos/store/appc" --authenticate_http="true" --authenticatee="crammd5" --cgroups_cpu_enable_pids_and_tids_count="false" --cgroups_enable_cfs="false" --cgroups_hierarchy="/sys/fs/cgroup" --cgroups_limit_swap="false" --cgroups_root="mesos" --container_disk_watch_interval="15secs" --containerizers="mesos" --credential="/tmp/SlaveTest_RemoveUnregisteredTerminatedExecutor_r87Psi/credential" --default_role="*" --disk_watch_interval="1mins" --docker="docker" --docker_kill_orphans="true" --docker_registry="https://registry-1.docker.io" --docker_remove_delay="6hrs" --docker_socket="/var/run/docker.sock" --docker_stop_timeout="0ns" --docker_store_dir="/tmp/mesos/store/docker" --enforce_container_disk_quota="false" --executor_registration_timeout="1mins" --executor_shutdown_grace_period="5secs" --fetcher_cache_dir="/tmp/SlaveTest_RemoveUnregisteredTerminatedExecutor_r87Psi/fetch" --fetcher_cache_size="2GB" --frameworks_home="" --gc_delay="1weeks" --gc_disk_headroom="0.1" --hadoop_home="" --help="false" --hostname_lookup="true" --http_authenticators="basic" --http_credentials="/tmp/SlaveTest_RemoveUnregisteredTerminatedExecutor_r87Psi/http_credentials" --image_provisioner_backend="copy" --initialize_driver_logging="true" --isolation="posix/cpu,posix/mem" --launcher_dir="/mesos/mesos-0.29.0/_build/src" --logbufsecs="0" --logging_level="INFO" --oversubscribed_resources_interval="15secs" --perf_duration="10secs" --perf_interval="1mins" --qos_correction_interval_min="0ns" --quiet="false" --recover="reconnect" --recovery_timeout="15mins" --registration_backoff_factor="10ms" --resources="cpus:2;mem:1024;disk:1024;ports:[31000-32000]" --revocable_cpu_low_priority="true" --sandbox_directory="/mnt/mesos/sandbox" --strict="true" --switch_user="true" --systemd_enable_support="true" --systemd_runtime_directory="/run/systemd/system" --version="false" --work_dir="/tmp/SlaveTest_RemoveUnregisteredTerminatedExecutor_r87Psi"
> I0331 17:09:45.365267  1331 credentials.hpp:86] Loading credential for authentication from '/tmp/SlaveTest_RemoveUnregisteredTerminatedExecutor_r87Psi/credential'
> I0331 17:09:45.365537  1331 slave.cpp:338] Slave using credential for: test-principal
> I0331 17:09:45.365581  1331 credentials.hpp:37] Loading credentials for authentication from '/tmp/SlaveTest_RemoveUnregisteredTerminatedExecutor_r87Psi/http_credentials'
> I0331 17:09:45.365871  1331 slave.cpp:390] Using default 'basic' HTTP authenticator
> I0331 17:09:45.366381  1331 resources.cpp:572] Parsing resources as JSON failed: cpus:2;mem:1024;disk:1024;ports:[31000-32000]
> Trying semicolon-delimited string format instead
> I0331 17:09:45.366503  1299 sched.cpp:222] Version: 0.29.0
> I0331 17:09:45.366998  1331 slave.cpp:589] Slave resources: cpus(*):2; mem(*):1024; disk(*):1024; ports(*):[31000-32000]
> I0331 17:09:45.367086  1331 slave.cpp:597] Slave attributes: [  ]
> I0331 17:09:45.367101  1331 slave.cpp:602] Slave hostname: 29ba6f7aca97
> I0331 17:09:45.367518  1321 sched.cpp:326] New master detected at master@172.17.0.2:47264
> I0331 17:09:45.367595  1321 sched.cpp:382] Authenticating with master master@172.17.0.2:47264
> I0331 17:09:45.367617  1321 sched.cpp:389] Using default CRAM-MD5 authenticatee
> I0331 17:09:45.367931  1321 authenticatee.cpp:121] Creating new client SASL connection
> I0331 17:09:45.368347  1321 master.cpp:5679] Authenticating scheduler-750dafdb-b99a-49b6-883c-3de70916a485@172.17.0.2:47264
> I0331 17:09:45.368533  1321 authenticator.cpp:413] Starting authentication session for crammd5_authenticatee(753)@172.17.0.2:47264
> I0331 17:09:45.368839  1321 authenticator.cpp:98] Creating new server SASL connection
> I0331 17:09:45.369026  1317 state.cpp:57] Recovering state from '/tmp/SlaveTest_RemoveUnregisteredTerminatedExecutor_r87Psi/meta'
> I0331 17:09:45.369580  1317 authenticatee.cpp:212] Received SASL authentication mechanisms: CRAM-MD5
> I0331 17:09:45.369616  1317 authenticatee.cpp:238] Attempting to authenticate with mechanism 'CRAM-MD5'
> I0331 17:09:45.369716  1321 authenticator.cpp:203] Received SASL authentication start
> I0331 17:09:45.369781  1321 authenticator.cpp:325] Authentication requires more steps
> I0331 17:09:45.369848  1320 status_update_manager.cpp:200] Recovering status update manager
> I0331 17:09:45.369868  1321 authenticatee.cpp:258] Received SASL authentication step
> I0331 17:09:45.370017  1320 authenticator.cpp:231] Received SASL authentication step
> I0331 17:09:45.370057  1320 auxprop.cpp:107] Request to lookup properties for user: 'test-principal' realm: '29ba6f7aca97' server FQDN: '29ba6f7aca97' SASL_AUXPROP_VERIFY_AGAINST_HASH: false SASL_AUXPROP_OVERRIDE: false SASL_AUXPROP_AUTHZID: false 
> I0331 17:09:45.370074  1320 auxprop.cpp:179] Looking up auxiliary property '*userPassword'
> I0331 17:09:45.370127  1320 auxprop.cpp:179] Looking up auxiliary property '*cmusaslsecretCRAM-MD5'
> I0331 17:09:45.370151  1321 slave.cpp:4773] Finished recovery
> I0331 17:09:45.370156  1320 auxprop.cpp:107] Request to lookup properties for user: 'test-principal' realm: '29ba6f7aca97' server FQDN: '29ba6f7aca97' SASL_AUXPROP_VERIFY_AGAINST_HASH: false SASL_AUXPROP_OVERRIDE: false SASL_AUXPROP_AUTHZID: true 
> I0331 17:09:45.370182  1320 auxprop.cpp:129] Skipping auxiliary property '*userPassword' since SASL_AUXPROP_AUTHZID == true
> I0331 17:09:45.370193  1320 auxprop.cpp:129] Skipping auxiliary property '*cmusaslsecretCRAM-MD5' since SASL_AUXPROP_AUTHZID == true
> I0331 17:09:45.370211  1320 authenticator.cpp:317] Authentication success
> I0331 17:09:45.370369  1320 authenticatee.cpp:298] Authentication success
> I0331 17:09:45.370455  1320 master.cpp:5709] Successfully authenticated principal 'test-principal' at scheduler-750dafdb-b99a-49b6-883c-3de70916a485@172.17.0.2:47264
> I0331 17:09:45.370546  1320 authenticator.cpp:431] Authentication session cleanup for crammd5_authenticatee(753)@172.17.0.2:47264
> I0331 17:09:45.370789  1321 slave.cpp:4945] Querying resource estimator for oversubscribable resources
> I0331 17:09:45.371309  1321 status_update_manager.cpp:174] Pausing sending status updates
> I0331 17:09:45.371325  1326 slave.cpp:928] New master detected at master@172.17.0.2:47264
> I0331 17:09:45.371408  1326 slave.cpp:991] Authenticating with master master@172.17.0.2:47264
> I0331 17:09:45.371428  1326 slave.cpp:996] Using default CRAM-MD5 authenticatee
> I0331 17:09:45.371578  1326 slave.cpp:964] Detecting new master
> I0331 17:09:45.371677  1321 authenticatee.cpp:121] Creating new client SASL connection
> I0331 17:09:45.372030  1317 master.cpp:5679] Authenticating slave(357)@172.17.0.2:47264
> I0331 17:09:45.372042  1330 slave.cpp:4959] Received oversubscribable resources  from the resource estimator
> I0331 17:09:45.372134  1330 authenticator.cpp:413] Starting authentication session for crammd5_authenticatee(754)@172.17.0.2:47264
> I0331 17:09:45.372319  1322 authenticator.cpp:98] Creating new server SASL connection
> I0331 17:09:45.372548  1321 authenticatee.cpp:212] Received SASL authentication mechanisms: CRAM-MD5
> I0331 17:09:45.372582  1321 authenticatee.cpp:238] Attempting to authenticate with mechanism 'CRAM-MD5'
> I0331 17:09:45.372684  1322 authenticator.cpp:203] Received SASL authentication start
> I0331 17:09:45.372746  1322 authenticator.cpp:325] Authentication requires more steps
> I0331 17:09:45.372835  1322 authenticatee.cpp:258] Received SASL authentication step
> I0331 17:09:45.372928  1322 authenticator.cpp:231] Received SASL authentication step
> I0331 17:09:45.373037  1322 auxprop.cpp:107] Request to lookup properties for user: 'test-principal' realm: '29ba6f7aca97' server FQDN: '29ba6f7aca97' SASL_AUXPROP_VERIFY_AGAINST_HASH: false SASL_AUXPROP_OVERRIDE: false SASL_AUXPROP_AUTHZID: false 
> I0331 17:09:45.373059  1322 auxprop.cpp:179] Looking up auxiliary property '*userPassword'
> I0331 17:09:45.373107  1322 auxprop.cpp:179] Looking up auxiliary property '*cmusaslsecretCRAM-MD5'
> I0331 17:09:45.373134  1322 auxprop.cpp:107] Request to lookup properties for user: 'test-principal' realm: '29ba6f7aca97' server FQDN: '29ba6f7aca97' SASL_AUXPROP_VERIFY_AGAINST_HASH: false SASL_AUXPROP_OVERRIDE: false SASL_AUXPROP_AUTHZID: true 
> I0331 17:09:45.373147  1322 auxprop.cpp:129] Skipping auxiliary property '*userPassword' since SASL_AUXPROP_AUTHZID == true
> I0331 17:09:45.373155  1322 auxprop.cpp:129] Skipping auxiliary property '*cmusaslsecretCRAM-MD5' since SASL_AUXPROP_AUTHZID == true
> I0331 17:09:45.373173  1322 authenticator.cpp:317] Authentication success
> I0331 17:09:45.373255  1320 authenticatee.cpp:298] Authentication success
> I0331 17:09:45.373327  1322 master.cpp:5709] Successfully authenticated principal 'test-principal' at slave(357)@172.17.0.2:47264
> I0331 17:09:45.373345  1320 authenticator.cpp:431] Authentication session cleanup for crammd5_authenticatee(754)@172.17.0.2:47264
> I0331 17:09:45.373495  1329 sched.cpp:472] Successfully authenticated with master master@172.17.0.2:47264
> I0331 17:09:45.373517  1329 sched.cpp:777] Sending SUBSCRIBE call to master@172.17.0.2:47264
> I0331 17:09:45.373615  1329 sched.cpp:810] Will retry registration in 1.487722594secs if necessary
> I0331 17:09:45.373731  1320 master.cpp:2346] Received SUBSCRIBE call for framework 'default' at scheduler-750dafdb-b99a-49b6-883c-3de70916a485@172.17.0.2:47264
> I0331 17:09:45.373754  1320 master.cpp:1865] Authorizing framework principal 'test-principal' to receive offers for role '*'
> I0331 17:09:45.374069  1320 master.cpp:2417] Subscribing framework default with checkpointing disabled and capabilities [  ]
> I0331 17:09:45.374737  1320 hierarchical.cpp:266] Added framework 9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000
> I0331 17:09:45.374830  1320 hierarchical.cpp:1490] No resources available to allocate!
> I0331 17:09:45.374868  1320 hierarchical.cpp:1585] No inverse offers to send out!
> I0331 17:09:45.374927  1320 hierarchical.cpp:1141] Performed allocation for 0 slaves in 176144ns
> I0331 17:09:45.375205  1317 sched.cpp:704] Framework registered with 9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000
> I0331 17:09:45.375283  1317 sched.cpp:718] Scheduler::registered took 32966ns
> I0331 17:09:45.375393  1322 slave.cpp:1061] Successfully authenticated with master master@172.17.0.2:47264
> I0331 17:09:45.375533  1322 slave.cpp:1457] Will retry registration in 15.182386ms if necessary
> I0331 17:09:45.375825  1322 master.cpp:4390] Registering slave at slave(357)@172.17.0.2:47264 (29ba6f7aca97) with id 9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0
> I0331 17:09:45.376332  1320 registrar.cpp:439] Applied 1 operations in 81112ns; attempting to update the 'registry'
> I0331 17:09:45.377719  1322 log.cpp:683] Attempting to append 339 bytes to the log
> I0331 17:09:45.377935  1322 coordinator.cpp:348] Coordinator attempting to write APPEND action at position 3
> I0331 17:09:45.379067  1330 replica.cpp:537] Replica received write request for position 3 from (12042)@172.17.0.2:47264
> I0331 17:09:45.379992  1330 leveldb.cpp:341] Persisting action (358 bytes) to leveldb took 884750ns
> I0331 17:09:45.380023  1330 replica.cpp:712] Persisted action at 3
> I0331 17:09:45.381299  1330 replica.cpp:691] Replica received learned notice for position 3 from @0.0.0.0:0
> I0331 17:09:45.381980  1330 leveldb.cpp:341] Persisting action (360 bytes) to leveldb took 604022ns
> I0331 17:09:45.382009  1330 replica.cpp:712] Persisted action at 3
> I0331 17:09:45.382030  1330 replica.cpp:697] Replica learned APPEND action at position 3
> I0331 17:09:45.391926  1321 registrar.cpp:484] Successfully updated the 'registry' in 15520us
> I0331 17:09:45.392407  1322 log.cpp:702] Attempting to truncate the log to 3
> I0331 17:09:45.392673  1322 coordinator.cpp:348] Coordinator attempting to write TRUNCATE action at position 4
> I0331 17:09:45.392875  1321 master.cpp:4458] Registered slave 9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0 at slave(357)@172.17.0.2:47264 (29ba6f7aca97) with cpus(*):2; mem(*):1024; disk(*):1024; ports(*):[31000-32000]
> I0331 17:09:45.393252  1321 hierarchical.cpp:476] Added slave 9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0 (29ba6f7aca97) with cpus(*):2; mem(*):1024; disk(*):1024; ports(*):[31000-32000] (allocated: )
> I0331 17:09:45.393406  1317 slave.cpp:1105] Registered with master master@172.17.0.2:47264; given slave ID 9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0
> I0331 17:09:45.393548  1317 fetcher.cpp:81] Clearing fetcher cache
> I0331 17:09:45.394038  1319 status_update_manager.cpp:181] Resuming sending status updates
> I0331 17:09:45.394340  1317 slave.cpp:1128] Checkpointing SlaveInfo to '/tmp/SlaveTest_RemoveUnregisteredTerminatedExecutor_r87Psi/meta/slaves/9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0/slave.info'
> I0331 17:09:45.394834  1325 master.cpp:5508] Sending 1 offers to framework 9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000 (default) at scheduler-750dafdb-b99a-49b6-883c-3de70916a485@172.17.0.2:47264
> I0331 17:09:45.395146  1317 slave.cpp:1165] Forwarding total oversubscribed resources 
> I0331 17:09:45.395349  1318 sched.cpp:874] Scheduler::resourceOffers took 145774ns
> I0331 17:09:45.395462  1317 slave.cpp:3664] Received ping from slave-observer(340)@172.17.0.2:47264
> I0331 17:09:45.395781  1325 master.cpp:4802] Received update of slave 9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0 at slave(357)@172.17.0.2:47264 (29ba6f7aca97) with total oversubscribed resources 
> I0331 17:09:45.396335  1321 hierarchical.cpp:1585] No inverse offers to send out!
> I0331 17:09:45.396401  1321 hierarchical.cpp:1164] Performed allocation for slave 9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0 in 3.112603ms
> I0331 17:09:45.396800  1321 hierarchical.cpp:534] Slave 9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0 (29ba6f7aca97) updated with oversubscribed resources  (total: cpus(*):2; mem(*):1024; disk(*):1024; ports(*):[31000-32000], allocated: cpus(*):2; mem(*):1024; disk(*):1024; ports(*):[31000-32000])
> I0331 17:09:45.396965  1320 replica.cpp:537] Replica received write request for position 4 from (12043)@172.17.0.2:47264
> I0331 17:09:45.396978  1321 hierarchical.cpp:1490] No resources available to allocate!
> I0331 17:09:45.397666  1320 leveldb.cpp:341] Persisting action (16 bytes) to leveldb took 556870ns
> I0331 17:09:45.397699  1320 replica.cpp:712] Persisted action at 4
> I0331 17:09:45.397799  1321 hierarchical.cpp:1585] No inverse offers to send out!
> I0331 17:09:45.397964  1321 hierarchical.cpp:1164] Performed allocation for slave 9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0 in 1.122823ms
> I0331 17:09:45.399468  1318 master.cpp:3288] Processing ACCEPT call for offers: [ 9b06f6db-72f1-42be-b262-98eb1ddb86e2-O0 ] on slave 9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0 at slave(357)@172.17.0.2:47264 (29ba6f7aca97) for framework 9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000 (default) at scheduler-750dafdb-b99a-49b6-883c-3de70916a485@172.17.0.2:47264
> I0331 17:09:45.399530  1318 master.cpp:2891] Authorizing framework principal 'test-principal' to launch task 1 as user 'mesos'
> W0331 17:09:45.401288  1318 validation.cpp:416] Executor default for task 1 uses less CPUs (None) than the minimum required (0.01). Please update your executor, as this will be mandatory in future releases.
> W0331 17:09:45.401373  1318 validation.cpp:428] Executor default for task 1 uses less memory (None) than the minimum required (32MB). Please update your executor, as this will be mandatory in future releases.
> I0331 17:09:45.401914  1318 master.hpp:177] Adding task 1 with resources cpus(*):2; mem(*):1024; disk(*):1024; ports(*):[31000-32000] on slave 9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0 (29ba6f7aca97)
> I0331 17:09:45.403225  1318 master.cpp:3773] Launching task 1 of framework 9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000 (default) at scheduler-750dafdb-b99a-49b6-883c-3de70916a485@172.17.0.2:47264 with resources cpus(*):2; mem(*):1024; disk(*):1024; ports(*):[31000-32000] on slave 9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0 at slave(357)@172.17.0.2:47264 (29ba6f7aca97)
> I0331 17:09:45.403780  1322 slave.cpp:1497] Got assigned task 1 for framework 9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000
> I0331 17:09:45.404628  1322 slave.cpp:1616] Launching task 1 for framework 9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000
> I0331 17:09:45.404757  1318 replica.cpp:691] Replica received learned notice for position 4 from @0.0.0.0:0
> I0331 17:09:45.405338  1318 leveldb.cpp:341] Persisting action (18 bytes) to leveldb took 557752ns
> I0331 17:09:45.405519  1318 leveldb.cpp:399] Deleting ~2 keys from leveldb took 132165ns
> I0331 17:09:45.405654  1318 replica.cpp:712] Persisted action at 4
> I0331 17:09:45.405788  1318 replica.cpp:697] Replica learned TRUNCATE action at position 4
> I0331 17:09:45.406699  1322 paths.cpp:528] Trying to chown '/tmp/SlaveTest_RemoveUnregisteredTerminatedExecutor_r87Psi/slaves/9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0/frameworks/9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000/executors/default/runs/64ca6446-70ef-4912-98e0-fe9349b5c4f0' to user 'mesos'
> I0331 17:09:45.418769  1322 slave.cpp:5575] Launching executor default of framework 9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000 with resources  in work directory '/tmp/SlaveTest_RemoveUnregisteredTerminatedExecutor_r87Psi/slaves/9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0/frameworks/9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000/executors/default/runs/64ca6446-70ef-4912-98e0-fe9349b5c4f0'
> I0331 17:09:45.422309  1322 exec.cpp:150] Version: 0.29.0
> I0331 17:09:45.422802  1320 exec.cpp:200] Executor started at: executor(102)@172.17.0.2:47264 with pid 1299
> I0331 17:09:45.423281  1322 slave.cpp:1834] Queuing task '1' for executor 'default' of framework 9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000
> I0331 17:09:45.423583  1322 slave.cpp:881] Successfully attached file '/tmp/SlaveTest_RemoveUnregisteredTerminatedExecutor_r87Psi/slaves/9b06f6db-72f1-42be-b262-98eb1ddb86e2-S0/frameworks/9b06f6db-72f1-42be-b262-98eb1ddb86e2-0000/executors/default/runs/64ca6446-70ef-4912-98e0-fe9349b5c4f0'
> F0331 17:09:45.425576  1322 owned.hpp:110] Check failed: 'get()' Must be non NULL 
> *** Check failure stack trace: ***
>     @     0x2b84020b04bd  google::LogMessage::Fail()
>     @     0x2b84020af89e  google::LogMessage::SendToLog()
>     @     0x2b84020b017d  google::LogMessage::Flush()
>     @     0x2b84020b35f8  google::LogMessageFatal::~LogMessageFatal()
>     @           0xa07364  google::CheckNotNull<>()
>     @           0x9f69a2  process::Owned<>::operator->()
>     @           0x9f4a6d  mesos::internal::tests::TestContainerizer::_wait()
>     @           0x9faf66  _ZN7testing8internal12InvokeHelperIN7process6FutureIN5mesos13containerizer11TerminationEEESt5tupleIJRKNS4_11ContainerIDEEEE12InvokeMethodINS4_8internal5tests17TestContainerizerEMSH_FS7_SB_EEES7_PT_T0_RKSC_
>     @           0x9faeb4  _ZNK7testing8internal18InvokeMethodActionIN5mesos8internal5tests17TestContainerizerEMS5_FN7process6FutureINS2_13containerizer11TerminationEEERKNS2_11ContainerIDEEE7PerformISA_St5tupleIJSD_EEEET_RKT0_
>     @           0x9fae56  _ZN7testing17PolymorphicActionINS_8internal18InvokeMethodActionIN5mesos8internal5tests17TestContainerizerEMS6_FN7process6FutureINS3_13containerizer11TerminationEEERKNS3_11ContainerIDEEEEE15MonomorphicImplIFSB_SE_EE7PerformERKSt5tupleIJSE_EE
>     @           0xa30758  _ZNK7testing6ActionIFN7process6FutureIN5mesos13containerizer11TerminationEEERKNS3_11ContainerIDEEE7PerformERKSt5tupleIJS9_EE
>     @           0xa305ba  testing::internal::ActionResultHolder<>::PerformAction<>()
>     @           0xa2f19c  testing::internal::FunctionMockerBase<>::UntypedPerformAction()
>     @          0x197a679  testing::internal::UntypedFunctionMockerBase::UntypedInvokeWith()
>     @           0xa1d09b  _ZN7testing8internal18FunctionMockerBaseIFN7process6FutureIN5mesos13containerizer11TerminationEEERKNS4_11ContainerIDEEE10InvokeWithERKSt5tupleIJSA_EE
>     @           0xa1d064  testing::internal::FunctionMocker<>::Invoke()
>     @           0x9f98c0  mesos::internal::tests::TestContainerizer::wait()
>     @     0x2b84012ec6a5  mesos::internal::slave::Slave::executorLaunched()
>     @     0x2b84013225ea  _ZZN7process8dispatchIN5mesos8internal5slave5SlaveERKNS1_11FrameworkIDERKNS1_10ExecutorIDERKNS1_11ContainerIDERKNS_6FutureIbEES5_S8_SB_SF_EEvRKNS_3PIDIT_EEMSJ_FvT0_T1_T2_T3_ET4_T5_T6_T7_ENKUlPNS_11ProcessBaseEE_clESY_
>     @     0x2b8401322112  _ZNSt17_Function_handlerIFvPN7process11ProcessBaseEEZNS0_8dispatchIN5mesos8internal5slave5SlaveERKNS5_11FrameworkIDERKNS5_10ExecutorIDERKNS5_11ContainerIDERKNS0_6FutureIbEES9_SC_SF_SJ_EEvRKNS0_3PIDIT_EEMSN_FvT0_T1_T2_T3_ET4_T5_T6_T7_EUlS2_E_E9_M_invokeERKSt9_Any_dataS2_
>     @     0x2b8401fe72e8  std::function<>::operator()()
>     @     0x2b8401fd1534  process::ProcessBase::visit()
>     @     0x2b84020279ae  process::DispatchEvent::visit()
>     @           0x86b4f1  process::ProcessBase::serve()
>     @     0x2b8401fcf254  process::ProcessManager::resume()
>     @     0x2b8401fd7375  process::ProcessManager::init_threads()::$_1::operator()()
>     @     0x2b8401fd72b3  _ZNSt5_BindIFZN7process14ProcessManager12init_threadsEvE3$_1St17reference_wrapperIKSt11atomic_boolEEE6__callIvJEJLm0EEEET_OSt5tupleIJDpT0_EESt12_Index_tupleIJXspT1_EEE
>     @     0x2b8401fd7266  _ZNSt5_BindIFZN7process14ProcessManager12init_threadsEvE3$_1St17reference_wrapperIKSt11atomic_boolEEEclIJEvEET0_DpOT_
>     @     0x2b8401fd7215  _ZNSt12_Bind_simpleIFSt5_BindIFZN7process14ProcessManager12init_threadsEvE3$_1St17reference_wrapperIKSt11atomic_boolEEEvEE9_M_invokeIJEEEvSt12_Index_tupleIJXspT_EEE
>     @     0x2b8401fd71e5  std::_Bind_simple<>::operator()()
>     @     0x2b8401fd71bc  std::thread::_Impl<>::_M_run()
>     @     0x2b8403840a60  (unknown)
> make[4]: Leaving directory `/mesos/mesos-0.29.0/_build/src'
> make[4]: *** [check-local] Aborted
> make[3]: Leaving directory `/mesos/mesos-0.29.0/_build/src'
> make[3]: *** [check-am] Error 2
> make[2]: *** [check] Error 2
> make[2]: Leaving directory `/mesos/mesos-0.29.0/_build/src'
> make[1]: *** [check-recursive] Error 1
> make[1]: Leaving directory `/mesos/mesos-0.29.0/_build'
> make: *** [distcheck] Error 1
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)