You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@mesos.apache.org by "Benno Evers (JIRA)" <ji...@apache.org> on 2018/01/26 16:20:00 UTC

[jira] [Commented] (MESOS-8485) MasterTest.RegistryGcByCount is flaky

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

Benno Evers commented on MESOS-8485:
------------------------------------

This is fairly reproducible when putting the test machine under heavy load (i.e. ca. 1 failure per 3000 runs when I'm compiling Mesos with 24 threads at the same time)

 

What happens is the following:

The test case is starting two different instances of `mesos-agent`, marking both of them as gone, and forcing one of them to be garbage collected. It expects that after this is done, one of the slaves will be marked as "gone" and the other be unknown. To get the agent id of the agents it registers, the following code is used:

 
{noformat}
  Future<SlaveRegisteredMessage> slaveRegisteredMessage =
    FUTURE_PROTOBUF(SlaveRegisteredMessage(), master.get()->pid, _);
  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), slaveFlags);
  AWAIT_READY(slaveRegisteredMessage);

  [...] (the slave is marked as gone here)

  Future<SlaveRegisteredMessage> slaveRegisteredMessage2 =
    FUTURE_PROTOBUF(SlaveRegisteredMessage(), master.get()->pid, _);
  Try<Owned<cluster::Slave>> slave2 = StartSlave(detector.get(), slaveFlags2);
  AWAIT_READY(slaveRegisteredMessage2);{noformat}
 

In the failure case, the registration of the first agent works as follows:
{noformat}
agent0: Sends RegisterSlaveMessage
master: Does registration, adds SlaveRegisteredMessage to outbound message queue
agent0: Didn't get an answer after timeout, resends RegisterSlaveMessage
agent0: Gets the previously sent SlaveRegisteredMessage
master: Gets the second RegisterSlaveMessage, notices that agent0 is already registered and just resends the Slave
test: Proceeds to mark agent0 as gone, creates the Future<SlaveRegisteredMessage> for agent1
test: The future is satisfied by the second SlaveRegisteredMessage sent by the master{noformat}
Leading the test code to think that agent1 has the agent id of agent0, which leads to the subsequent test failure.

 

Mesos basically works correctly here, so the correct fix seems to be to rewrite the test to wait for a `SlaveRegisteredMessage` that is actually destined for the correct pid.

 

 

> MasterTest.RegistryGcByCount is flaky
> -------------------------------------
>
>                 Key: MESOS-8485
>                 URL: https://issues.apache.org/jira/browse/MESOS-8485
>             Project: Mesos
>          Issue Type: Bug
>          Components: test
>    Affects Versions: 1.5.0
>            Reporter: Vinod Kone
>            Assignee: Benno Evers
>            Priority: Major
>              Labels: flaky-test
>
> Observed this while testing Mesos 1.5.0-rc1 in ASF CI.
>  
> {code}
> 3: [ RUN      ] MasterTest.RegistryGcByCount
> ..............snip...........................
> 3: I0123 19:22:05.929347 15994 slave.cpp:1201] Detecting new master
> 3: I0123 19:22:05.931701 15988 slave.cpp:1228] Authenticating with master master@172.17.0.2:45634
> 3: I0123 19:22:05.931838 15988 slave.cpp:1237] Using default CRAM-MD5 authenticatee
> 3: I0123 19:22:05.932153 15999 authenticatee.cpp:121] Creating new client SASL connection
> 3: I0123 19:22:05.932580 15992 master.cpp:8958] Authenticating slave(442)@172.17.0.2:45634
> 3: I0123 19:22:05.932822 15990 authenticator.cpp:414] Starting authentication session for crammd5-authenticatee(870)@172.17.0.2:45634
> 3: I0123 19:22:05.933163 15989 authenticator.cpp:98] Creating new server SASL connection
> 3: I0123 19:22:05.933465 16001 authenticatee.cpp:213] Received SASL authentication mechanisms: CRAM-MD5
> 3: I0123 19:22:05.933495 16001 authenticatee.cpp:239] Attempting to authenticate with mechanism 'CRAM-MD5'
> 3: I0123 19:22:05.933631 15987 authenticator.cpp:204] Received SASL authentication start
> 3: I0123 19:22:05.933712 15987 authenticator.cpp:326] Authentication requires more steps
> 3: I0123 19:22:05.933851 15987 authenticatee.cpp:259] Received SASL authentication step
> 3: I0123 19:22:05.934006 15987 authenticator.cpp:232] Received SASL authentication step
> 3: I0123 19:22:05.934041 15987 auxprop.cpp:109] Request to lookup properties for user: 'test-principal' realm: '455912973e2c' server FQDN: '455912973e2c' SASL_AUXPROP_VERIFY_AGAINST_HASH: false SASL_AUXPROP_OVERRIDE: false SASL_AUXPROP_AUTHZID: false 
> 3: I0123 19:22:05.934095 15987 auxprop.cpp:181] Looking up auxiliary property '*userPassword'
> 3: I0123 19:22:05.934147 15987 auxprop.cpp:181] Looking up auxiliary property '*cmusaslsecretCRAM-MD5'
> 3: I0123 19:22:05.934279 15987 auxprop.cpp:109] Request to lookup properties for user: 'test-principal' realm: '455912973e2c' server FQDN: '455912973e2c' SASL_AUXPROP_VERIFY_AGAINST_HASH: false SASL_AUXPROP_OVERRIDE: false SASL_AUXPROP_AUTHZID: true 
> 3: I0123 19:22:05.934298 15987 auxprop.cpp:131] Skipping auxiliary property '*userPassword' since SASL_AUXPROP_AUTHZID == true
> 3: I0123 19:22:05.934307 15987 auxprop.cpp:131] Skipping auxiliary property '*cmusaslsecretCRAM-MD5' since SASL_AUXPROP_AUTHZID == true
> 3: I0123 19:22:05.934324 15987 authenticator.cpp:318] Authentication success
> 3: I0123 19:22:05.934463 15995 authenticatee.cpp:299] Authentication success
> 3: I0123 19:22:05.934563 16002 master.cpp:8988] Successfully authenticated principal 'test-principal' at slave(442)@172.17.0.2:45634
> 3: I0123 19:22:05.934708 15993 authenticator.cpp:432] Authentication session cleanup for crammd5-authenticatee(870)@172.17.0.2:45634
> 3: I0123 19:22:05.934891 15995 slave.cpp:1320] Successfully authenticated with master master@172.17.0.2:45634
> 3: I0123 19:22:05.935261 15995 slave.cpp:1764] Will retry registration in 2.234083ms if necessary
> 3: I0123 19:22:05.935436 15999 master.cpp:6061] Received register agent message from slave(442)@172.17.0.2:45634 (455912973e2c)
> 3: I0123 19:22:05.935662 15999 master.cpp:3867] Authorizing agent with principal 'test-principal'
> 3: I0123 19:22:05.936161 15992 master.cpp:6123] Authorized registration of agent at slave(442)@172.17.0.2:45634 (455912973e2c)
> 3: I0123 19:22:05.936261 15992 master.cpp:6234] Registering agent at slave(442)@172.17.0.2:45634 (455912973e2c) with id eef8ea11-9247-44f3-84cf-340b24df3a52-S0
> 3: I0123 19:22:05.936993 15989 registrar.cpp:495] Applied 1 operations in 227911ns; attempting to update the registry
> 3: I0123 19:22:05.937814 15989 registrar.cpp:552] Successfully updated the registry in 743168ns
> 3: I0123 19:22:05.938057 15991 master.cpp:6282] Admitted agent eef8ea11-9247-44f3-84cf-340b24df3a52-S0 at slave(442)@172.17.0.2:45634 (455912973e2c)
> 3: I0123 19:22:05.938891 15991 master.cpp:6331] Registered agent eef8ea11-9247-44f3-84cf-340b24df3a52-S0 at slave(442)@172.17.0.2:45634 (455912973e2c) with cpus:2; mem:1024; disk:1024; ports:[31000-32000]
> 3: I0123 19:22:05.939159 16002 slave.cpp:1764] Will retry registration in 26.332876ms if necessary
> 3: I0123 19:22:05.939349 15994 master.cpp:6061] Received register agent message from slave(442)@172.17.0.2:45634 (455912973e2c)
> 3: I0123 19:22:05.939347 15998 hierarchical.cpp:574] Added agent eef8ea11-9247-44f3-84cf-340b24df3a52-S0 (455912973e2c) with cpus:2; mem:1024; disk:1024; ports:[31000-32000] (allocated: {})
> 3: I0123 19:22:05.939574 15994 master.cpp:3867] Authorizing agent with principal 'test-principal'
> 3: I0123 19:22:05.939704 16002 slave.cpp:1366] Registered with master master@172.17.0.2:45634; given agent ID eef8ea11-9247-44f3-84cf-340b24df3a52-S0
> 3: I0123 19:22:05.939894 15999 task_status_update_manager.cpp:188] Resuming sending task status updates
> 3: I0123 19:22:05.940163 15998 hierarchical.cpp:1517] Performed allocation for 1 agents in 231470ns
> 3: I0123 19:22:05.940194 16001 master.cpp:6123] Authorized registration of agent at slave(442)@172.17.0.2:45634 (455912973e2c)
> 3: I0123 19:22:05.940263 16001 master.cpp:6213] Agent eef8ea11-9247-44f3-84cf-340b24df3a52-S0 at slave(442)@172.17.0.2:45634 (455912973e2c) already registered, resending acknowledgement
> 3: I0123 19:22:05.942983 15994 process.cpp:3515] Handling HTTP event for process 'master' with path: '/master/api/v1'
> 3: I0123 19:22:05.944905 15995 http.cpp:1185] HTTP POST for /master/api/v1 from 172.17.0.2:33442
> 3: I0123 19:22:05.945107 15995 http.cpp:682] Processing call MARK_AGENT_GONE
> 3: I0123 19:22:05.945749 16001 http.cpp:5363] Marking agent 'eef8ea11-9247-44f3-84cf-340b24df3a52-S0' as gone
> 3: I0123 19:22:05.946480 15997 registrar.cpp:495] Applied 1 operations in 186752ns; attempting to update the registry
> 3: I0123 19:22:05.947284 15997 registrar.cpp:552] Successfully updated the registry in 730112ns
> 3: I0123 19:22:05.948225 15988 hierarchical.cpp:609] Removed agent eef8ea11-9247-44f3-84cf-340b24df3a52-S0
> 3: I0123 19:22:05.952500 16002 slave.cpp:1386] Checkpointing SlaveInfo to '/tmp/MasterTest_RegistryGcByCount_HbzHl2/meta/slaves/eef8ea11-9247-44f3-84cf-340b24df3a52-S0/slave.info'
> 3: I0123 19:22:05.953299 16002 slave.cpp:1433] Forwarding agent update \{"operations":{},"resource_version_uuid":\{"value":"nekTyNfGT1S5DNQZxKJ72A=="},"slave_id":\{"value":"eef8ea11-9247-44f3-84cf-340b24df3a52-S0"},"update_oversubscribed_resources":true}
> 3: W0123 19:22:05.953675 16002 slave.cpp:1415] Already registered with master master@172.17.0.2:45634
> 3: I0123 19:22:05.953790 16002 slave.cpp:1433] Forwarding agent update \{"operations":{},"resource_version_uuid":\{"value":"nekTyNfGT1S5DNQZxKJ72A=="},"slave_id":\{"value":"eef8ea11-9247-44f3-84cf-340b24df3a52-S0"},"update_oversubscribed_resources":true}
> 3: I0123 19:22:05.954031 16002 slave.cpp:964] Agent asked to shut down by master@172.17.0.2:45634 because 'Agent has been marked gone'
> 3: I0123 19:22:05.954082 16002 slave.cpp:931] Agent terminating
> 3: W0123 19:22:05.954145 15993 master.cpp:7235] Ignoring update on removed agent eef8ea11-9247-44f3-84cf-340b24df3a52-S0
> 3: W0123 19:22:05.954636 15993 master.cpp:7235] Ignoring update on removed agent eef8ea11-9247-44f3-84cf-340b24df3a52-S0
> 3: W0123 19:22:05.955550 15986 process.cpp:2756] Attempted to spawn already running process files@172.17.0.2:45634
> 3: I0123 19:22:05.956634 15986 containerizer.cpp:304] Using isolation \{ environment_secret, posix/cpu, posix/mem, filesystem/posix, network/cni }
> 3: W0123 19:22:05.957228 15986 backend.cpp:76] Failed to create 'aufs' backend: AufsBackend requires root privileges
> 3: W0123 19:22:05.957363 15986 backend.cpp:76] Failed to create 'bind' backend: BindBackend requires root privileges
> 3: I0123 19:22:05.957401 15986 provisioner.cpp:299] Using default backend 'copy'
> 3: I0123 19:22:05.959393 15986 cluster.cpp:460] Creating default 'local' authorizer
> 3: I0123 19:22:05.961545 15998 slave.cpp:262] Mesos agent started on (443)@172.17.0.2:45634
> 3: I0123 19:22:05.961560 15998 slave.cpp:263] Flags at startup: --acls="" --appc_simple_discovery_uri_prefix="http://" --appc_store_dir="/tmp/MasterTest_RegistryGcByCount_2Nh5JR/store/appc" --authenticate_http_readonly="true" --authenticate_http_readwrite="false" --authenticatee="crammd5" --authentication_backoff_factor="1secs" --authorizer="local" --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/MasterTest_RegistryGcByCount_2Nh5JR/credential" --default_role="*" --disallow_sharing_agent_pid_namespace="false" --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/MasterTest_RegistryGcByCount_2Nh5JR/store/docker" --docker_volume_checkpoint_dir="/var/run/mesos/isolators/docker/volume" --enforce_container_disk_quota="false" --executor_registration_timeout="1mins" --executor_reregistration_timeout="2secs" --executor_shutdown_grace_period="5secs" --fetcher_cache_dir="/tmp/MasterTest_RegistryGcByCount_2Nh5JR/fetch" --fetcher_cache_size="2GB" --frameworks_home="" --gc_delay="1weeks" --gc_disk_headroom="0.1" --hadoop_home="" --help="false" --hostname_lookup="true" --http_command_executor="false" --http_credentials="/tmp/MasterTest_RegistryGcByCount_2Nh5JR/http_credentials" --http_heartbeat_interval="30secs" --initialize_driver_logging="true" --isolation="posix/cpu,posix/mem" --launcher="posix" --launcher_dir="/mesos/build/src" --logbufsecs="0" --logging_level="INFO" --max_completed_executors_per_framework="150" --oversubscribed_resources_interval="15secs" --perf_duration="10secs" --perf_interval="1mins" --port="5051" --qos_correction_interval_min="0ns" --quiet="false" --reconfiguration_policy="equal" --recover="reconnect" --recovery_timeout="15mins" --registration_backoff_factor="10ms" --resources="cpus:2;gpus:0;mem:1024;disk:1024;ports:[31000-32000]" --revocable_cpu_low_priority="true" --runtime_dir="/tmp/MasterTest_RegistryGcByCount_2Nh5JR" --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/MasterTest_RegistryGcByCount_FyzIlU" --zk_session_timeout="10secs"
> 3: I0123 19:22:05.961917 15998 credentials.hpp:86] Loading credential for authentication from '/tmp/MasterTest_RegistryGcByCount_2Nh5JR/credential'
> 3: I0123 19:22:05.962070 15998 slave.cpp:295] Agent using credential for: test-principal
> 3: I0123 19:22:05.962090 15998 credentials.hpp:37] Loading credentials for authentication from '/tmp/MasterTest_RegistryGcByCount_2Nh5JR/http_credentials'
> 3: I0123 19:22:05.962347 15998 http.cpp:1045] Creating default 'basic' HTTP authenticator for realm 'mesos-agent-readonly'
> 3: I0123 19:22:05.964292 15988 process.cpp:3515] Handling HTTP event for process 'master' with path: '/master/api/v1'
> 3: I0123 19:22:05.964375 15998 slave.cpp:612] Agent resources: [\{"name":"cpus","scalar":{"value":2.0},"type":"SCALAR"},\{"name":"mem","scalar":{"value":1024.0},"type":"SCALAR"},\{"name":"disk","scalar":{"value":1024.0},"type":"SCALAR"},\{"name":"ports","ranges":{"range":[{"begin":31000,"end":32000}]},"type":"RANGES"}]
> 3: I0123 19:22:05.964591 15998 slave.cpp:620] Agent attributes: [  ]
> 3: I0123 19:22:05.964601 15998 slave.cpp:629] Agent hostname: 455912973e2c
> 3: I0123 19:22:05.964753 15995 task_status_update_manager.cpp:181] Pausing sending task status updates
> 3: I0123 19:22:05.966159 16002 http.cpp:1185] HTTP POST for /master/api/v1 from 172.17.0.2:33443
> 3: I0123 19:22:05.966327 16002 http.cpp:682] Processing call MARK_AGENT_GONE
> 3: I0123 19:22:05.966898 15990 http.cpp:5363] Marking agent 'eef8ea11-9247-44f3-84cf-340b24df3a52-S0' as gone
> 3: W0123 19:22:05.966939 15990 http.cpp:5366] Not marking agent 'eef8ea11-9247-44f3-84cf-340b24df3a52-S0' as gone because it has already transitioned to gone
> 3: I0123 19:22:05.966969 15992 state.cpp:66] Recovering state from '/tmp/MasterTest_RegistryGcByCount_FyzIlU/meta'
> 3: I0123 19:22:05.967445 15995 task_status_update_manager.cpp:207] Recovering task status update manager
> 3: I0123 19:22:05.967747 15991 containerizer.cpp:674] Recovering containerizer
> 3: I0123 19:22:05.969804 15999 provisioner.cpp:493] Provisioner recovery complete
> 3: I0123 19:22:05.970371 16002 slave.cpp:6822] Finished recovery
> 3: I0123 19:22:05.971616 16000 task_status_update_manager.cpp:181] Pausing sending task status updates
> 3: I0123 19:22:05.971608 15987 slave.cpp:1146] New master detected at master@172.17.0.2:45634
> 3: I0123 19:22:05.971737 15987 slave.cpp:1201] Detecting new master
> 3: I0123 19:22:05.971755 15990 hierarchical.cpp:1517] Performed allocation for 0 agents in 134021ns
> 3: I0123 19:22:05.972729 15999 slave.cpp:6360] Current disk usage 16.56%. Max allowed age: 5.140805523372986days
> 3: I0123 19:22:05.972985 15988 master.cpp:1878] Skipping periodic registry garbage collection: no agents qualify for removal
> 3: I0123 19:22:05.974480 16001 slave.cpp:1228] Authenticating with master master@172.17.0.2:45634
> 3: I0123 19:22:05.974581 16001 slave.cpp:1237] Using default CRAM-MD5 authenticatee
> 3: I0123 19:22:05.975023 16002 authenticatee.cpp:121] Creating new client SASL connection
> 3: I0123 19:22:05.975343 15998 master.cpp:8958] Authenticating slave(443)@172.17.0.2:45634
> 3: I0123 19:22:05.975476 16000 authenticator.cpp:414] Starting authentication session for crammd5-authenticatee(871)@172.17.0.2:45634
> 3: I0123 19:22:05.975735 15991 authenticator.cpp:98] Creating new server SASL connection
> 3: I0123 19:22:05.976027 15995 authenticatee.cpp:213] Received SASL authentication mechanisms: CRAM-MD5
> 3: I0123 19:22:05.976143 15995 authenticatee.cpp:239] Attempting to authenticate with mechanism 'CRAM-MD5'
> 3: I0123 19:22:05.976356 15989 authenticator.cpp:204] Received SASL authentication start
> 3: I0123 19:22:05.976420 15989 authenticator.cpp:326] Authentication requires more steps
> 3: I0123 19:22:05.976552 15990 authenticatee.cpp:259] Received SASL authentication step
> 3: I0123 19:22:05.976698 15987 authenticator.cpp:232] Received SASL authentication step
> 3: I0123 19:22:05.976750 15987 auxprop.cpp:109] Request to lookup properties for user: 'test-principal' realm: '455912973e2c' server FQDN: '455912973e2c' SASL_AUXPROP_VERIFY_AGAINST_HASH: false SASL_AUXPROP_OVERRIDE: false SASL_AUXPROP_AUTHZID: false 
> 3: I0123 19:22:05.976773 15987 auxprop.cpp:181] Looking up auxiliary property '*userPassword'
> 3: I0123 19:22:05.976821 15987 auxprop.cpp:181] Looking up auxiliary property '*cmusaslsecretCRAM-MD5'
> 3: I0123 19:22:05.976898 15987 auxprop.cpp:109] Request to lookup properties for user: 'test-principal' realm: '455912973e2c' server FQDN: '455912973e2c' SASL_AUXPROP_VERIFY_AGAINST_HASH: false SASL_AUXPROP_OVERRIDE: false SASL_AUXPROP_AUTHZID: true 
> 3: I0123 19:22:05.976924 15987 auxprop.cpp:131] Skipping auxiliary property '*userPassword' since SASL_AUXPROP_AUTHZID == true
> 3: I0123 19:22:05.976935 15987 auxprop.cpp:131] Skipping auxiliary property '*cmusaslsecretCRAM-MD5' since SASL_AUXPROP_AUTHZID == true
> 3: I0123 19:22:05.976953 15987 authenticator.cpp:318] Authentication success
> 3: I0123 19:22:05.977094 15996 authenticatee.cpp:299] Authentication success
> 3: I0123 19:22:05.977233 15994 master.cpp:8988] Successfully authenticated principal 'test-principal' at slave(443)@172.17.0.2:45634
> 3: I0123 19:22:05.977321 15987 authenticator.cpp:432] Authentication session cleanup for crammd5-authenticatee(871)@172.17.0.2:45634
> 3: I0123 19:22:05.977475 15996 slave.cpp:1320] Successfully authenticated with master master@172.17.0.2:45634
> 3: I0123 19:22:05.977953 15996 slave.cpp:1764] Will retry registration in 6.841446ms if necessary
> 3: I0123 19:22:05.978238 15992 master.cpp:6061] Received register agent message from slave(443)@172.17.0.2:45634 (455912973e2c)
> 3: I0123 19:22:05.978591 15992 master.cpp:3867] Authorizing agent with principal 'test-principal'
> 3: I0123 19:22:05.979161 16000 master.cpp:6123] Authorized registration of agent at slave(443)@172.17.0.2:45634 (455912973e2c)
> 3: I0123 19:22:05.979320 16000 master.cpp:6234] Registering agent at slave(443)@172.17.0.2:45634 (455912973e2c) with id eef8ea11-9247-44f3-84cf-340b24df3a52-S1
> 3: I0123 19:22:05.980505 15991 registrar.cpp:495] Applied 1 operations in 455955ns; attempting to update the registry
> 3: I0123 19:22:05.981642 15991 registrar.cpp:552] Successfully updated the registry in 0ns
> 3: I0123 19:22:05.981912 15988 master.cpp:6282] Admitted agent eef8ea11-9247-44f3-84cf-340b24df3a52-S1 at slave(443)@172.17.0.2:45634 (455912973e2c)
> 3: I0123 19:22:05.982857 15988 master.cpp:6331] Registered agent eef8ea11-9247-44f3-84cf-340b24df3a52-S1 at slave(443)@172.17.0.2:45634 (455912973e2c) with cpus:2; mem:1024; disk:1024; ports:[31000-32000]
> 3: I0123 19:22:05.982964 16001 slave.cpp:1366] Registered with master master@172.17.0.2:45634; given agent ID eef8ea11-9247-44f3-84cf-340b24df3a52-S1
> 3: I0123 19:22:05.983130 15996 task_status_update_manager.cpp:188] Resuming sending task status updates
> 3: I0123 19:22:05.983392 16001 slave.cpp:1386] Checkpointing SlaveInfo to '/tmp/MasterTest_RegistryGcByCount_FyzIlU/meta/slaves/eef8ea11-9247-44f3-84cf-340b24df3a52-S1/slave.info'
> 3: I0123 19:22:05.983423 15994 hierarchical.cpp:574] Added agent eef8ea11-9247-44f3-84cf-340b24df3a52-S1 (455912973e2c) with cpus:2; mem:1024; disk:1024; ports:[31000-32000] (allocated: {})
> 3: I0123 19:22:05.983815 15994 hierarchical.cpp:1517] Performed allocation for 1 agents in 171516ns
> 3: I0123 19:22:05.984135 16001 slave.cpp:1433] Forwarding agent update \{"operations":{},"resource_version_uuid":\{"value":"1HNo1ICkRY24eDUqFmb6+Q=="},"slave_id":\{"value":"eef8ea11-9247-44f3-84cf-340b24df3a52-S1"},"update_oversubscribed_resources":true}
> 3: I0123 19:22:05.984762 15997 master.cpp:7265] Received update of agent eef8ea11-9247-44f3-84cf-340b24df3a52-S1 at slave(443)@172.17.0.2:45634 (455912973e2c) with total oversubscribed resources {}
> 3: I0123 19:22:05.985123 15997 master.cpp:7359] Ignoring update on agent eef8ea11-9247-44f3-84cf-340b24df3a52-S1 at slave(443)@172.17.0.2:45634 (455912973e2c) as it reports no changes
> 3: W0123 19:22:05.985782 15986 process.cpp:2756] Attempted to spawn already running process version@172.17.0.2:45634
> 3: I0123 19:22:05.986744 15986 sched.cpp:232] Version: 1.5.0
> 3: I0123 19:22:05.987470 15990 sched.cpp:336] New master detected at master@172.17.0.2:45634
> 3: I0123 19:22:05.987567 15990 sched.cpp:396] Authenticating with master master@172.17.0.2:45634
> 3: I0123 19:22:05.987582 15990 sched.cpp:403] Using default CRAM-MD5 authenticatee
> 3: I0123 19:22:05.987869 15999 authenticatee.cpp:121] Creating new client SASL connection
> 3: I0123 19:22:05.988121 15991 master.cpp:8958] Authenticating scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634
> 3: I0123 19:22:05.988296 15987 authenticator.cpp:414] Starting authentication session for crammd5-authenticatee(872)@172.17.0.2:45634
> 3: I0123 19:22:05.988575 15988 authenticator.cpp:98] Creating new server SASL connection
> 3: I0123 19:22:05.988821 15996 authenticatee.cpp:213] Received SASL authentication mechanisms: CRAM-MD5
> 3: I0123 19:22:05.988852 15996 authenticatee.cpp:239] Attempting to authenticate with mechanism 'CRAM-MD5'
> 3: I0123 19:22:05.988967 15994 authenticator.cpp:204] Received SASL authentication start
> 3: I0123 19:22:05.989023 15994 authenticator.cpp:326] Authentication requires more steps
> 3: I0123 19:22:05.989145 15993 authenticatee.cpp:259] Received SASL authentication step
> 3: I0123 19:22:05.989271 16001 authenticator.cpp:232] Received SASL authentication step
> 3: I0123 19:22:05.989316 16001 auxprop.cpp:109] Request to lookup properties for user: 'test-principal' realm: '455912973e2c' server FQDN: '455912973e2c' SASL_AUXPROP_VER:
> IFY_AGAINST_HASH: false SASL_AUXPROP_OVERRIDE: false SASL_AUXPROP_AUTHZID: false 
> 3: I0123 19:22:05.989334 16001 auxprop.cpp:181] Looking up auxiliary property '*userPassword'
> 3: I0123 19:22:05.989377 16001 auxprop.cpp:181] Looking up auxiliary property '*cmusaslsecretCRAM-MD5'
> 3: I0123 19:22:05.989400 16001 auxprop.cpp:109] Request to lookup properties for user: 'test-principal' realm: '455912973e2c' server FQDN: '455912973e2c' SASL_AUXPROP_VERIFY_AGAINST_HASH: false SASL_AUXPROP_OVERRIDE: false SASL_AUXPROP_AUTHZID: true 
> 3: I0123 19:22:05.989415 16001 auxprop.cpp:131] Skipping auxiliary property '*userPassword' since SASL_AUXPROP_AUTHZID == true
> 3: I0123 19:22:05.989423 16001 auxprop.cpp:131] Skipping auxiliary property '*cmusaslsecretCRAM-MD5' since SASL_AUXPROP_AUTHZID == true
> 3: I0123 19:22:05.989441 16001 authenticator.cpp:318] Authentication success
> 3: I0123 19:22:05.989531 15997 authenticatee.cpp:299] Authentication success
> 3: I0123 19:22:05.989719 15992 master.cpp:8988] Successfully authenticated principal 'test-principal' at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634
> 3: I0123 19:22:05.989751 16000 authenticator.cpp:432] Authentication session cleanup for crammd5-authenticatee(872)@172.17.0.2:45634
> 3: I0123 19:22:05.989894 15998 sched.cpp:502] Successfully authenticated with master master@172.17.0.2:45634
> 3: I0123 19:22:05.989914 15998 sched.cpp:824] Sending SUBSCRIBE call to master@172.17.0.2:45634
> 3: I0123 19:22:05.990039 15998 sched.cpp:857] Will retry registration in 1.379182754secs if necessary
> 3: I0123 19:22:05.990229 15991 master.cpp:2958] Received SUBSCRIBE call for framework 'default' at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634
> 3: I0123 19:22:05.990311 15991 master.cpp:2275] Authorizing framework principal 'test-principal' to receive offers for roles '\{ * }'
> 3: I0123 19:22:05.990876 15987 master.cpp:3038] Subscribing framework default with checkpointing disabled and capabilities [ MULTI_ROLE, RESERVATION_REFINEMENT, PARTITION_AWARE ]
> 3: I0123 19:22:05.991102 15987 master.cpp:9179] Adding framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000 (default) at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634 with roles {  } suppressed
> 3: I0123 19:22:05.991621 15994 sched.cpp:751] Framework registered with eef8ea11-9247-44f3-84cf-340b24df3a52-0000
> 3: I0123 19:22:05.991731 15988 hierarchical.cpp:297] Added framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000
> 3: I0123 19:22:05.991816 15994 sched.cpp:765] Scheduler::registered took 25842ns
> 3: I0123 19:22:05.993170 15988 hierarchical.cpp:1517] Performed allocation for 1 agents in 1.236264ms
> 3: I0123 19:22:05.993611 15997 master.cpp:8788] Sending 1 offers to framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000 (default) at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634
> 3: 
> 3: GMOCK WARNING:
> 3: Uninteresting mock function call - returning directly.
> 3:     Function call: resourceOffers(0x7ffcbfa4e140, @0x7f82fc814860 \{ 160-byte object <10-22 C5-0C 83-7F 00-00 00-00 00-00 00-00 00-00 5F-00 00-00 00-00 00-00 00-00 00-00 00-00 00-00 04-00 00-00 04-00 00-00 80-C0 03-C8 82-7F 00-00 00-00 00-00 00-00 00-00 00-00 00-00 00-00 00-00 ... 10-38 04-C8 82-7F 00-00 E0-C4 03-C8 82-7F 00-00 10-B3 00-C8 82-7F 00-00 60-A7 03-C8 82-7F 00-00 10-C3 00-C8 82-7F 00-00 00-00 00-00 00-00 00-00 10-DA 00-C8 82-7F 00-00 00-00 00-00 00-00 00-00> })
> 3: NOTE: You can safely ignore the above warning unless this call should not happen.  Do not suppress it by blindly adding an EXPECT_CALL() if you don't mean to enforce the call.  See https://github.com/google/googletest/blob/master/googlemock/docs/CookBook.md#knowing-when-to-expect for details.
> 3: I0123 19:22:05.994109 16001 sched.cpp:921] Scheduler::resourceOffers took 77564ns
> 3: I0123 19:22:05.994665 15989 master.cpp:8425] Performing explicit task state reconciliation for 2 tasks of framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000 (default) at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634
> 3: I0123 19:22:05.994779 15989 master.cpp:8573] Sending explicit reconciliation state TASK_GONE_BY_OPERATOR for task 7a4ff1bf-488b-4152-a7e4-cf0876008c4d of framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000 (default) at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634
> 3: I0123 19:22:05.994936 15989 master.cpp:8573] Sending explicit reconciliation state TASK_GONE_BY_OPERATOR for task 9d4e66ef-86c7-428a-b110-ba949c0c19b8 of framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000 (default) at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634
> 3: I0123 19:22:05.995215 15989 sched.cpp:1029] Scheduler::statusUpdate took 36257ns
> 3: I0123 19:22:05.995381 15989 sched.cpp:1029] Scheduler::statusUpdate took 36862ns
> 3: /mesos/src/tests/master_tests.cpp:8606: Failure
> 3:       Expected: TASK_UNKNOWN
> 3: To be equal to: reconcileUpdate1->state()
> 3:       Which is: TASK_GONE_BY_OPERATOR
> 3: I0123 19:22:05.995779 16000 master.cpp:1420] Framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000 (default) at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634 disconnected
> 3: I0123 19:22:05.995802 16000 master.cpp:3328] Deactivating framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000 (default) at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634
> 3: I0123 19:22:05.996014 15991 hierarchical.cpp:405] Deactivated framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000
> 3: I0123 19:22:05.996299 15994 slave.cpp:931] Agent terminating
> 3: I0123 19:22:05.996402 16000 master.cpp:10703] Removing offer eef8ea11-9247-44f3-84cf-340b24df3a52-O0
> 3: I0123 19:22:05.996474 16000 master.cpp:3305] Disconnecting framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000 (default) at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634
> 3: I0123 19:22:05.996522 16000 master.cpp:1435] Giving framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000 (default) at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634 0ns to failover
> 3: I0123 19:22:05.996711 16000 master.cpp:1306] Agent eef8ea11-9247-44f3-84cf-340b24df3a52-S1 at slave(443)@172.17.0.2:45634 (455912973e2c) disconnected
> 3: I0123 19:22:05.996732 16000 master.cpp:3365] Disconnecting agent eef8ea11-9247-44f3-84cf-340b24df3a52-S1 at slave(443)@172.17.0.2:45634 (455912973e2c)
> 3: I0123 19:22:05.996789 16000 master.cpp:3384] Deactivating agent eef8ea11-9247-44f3-84cf-340b24df3a52-S1 at slave(443)@172.17.0.2:45634 (455912973e2c)
> 3: I0123 19:22:05.997368 15991 hierarchical.cpp:1192] Recovered cpus(allocated: *):2; mem(allocated: *):1024; disk(allocated: *):1024; ports(allocated: *):[31000-32000] (total: cpus:2; mem:1024; disk:1024; ports:[31000-32000], allocated: {}) on agent eef8ea11-9247-44f3-84cf-340b24df3a52-S1 from framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000
> 3: I0123 19:22:05.997454 15991 hierarchical.cpp:766] Agent eef8ea11-9247-44f3-84cf-340b24df3a52-S1 deactivated
> 3: I0123 19:22:05.998085 15992 master.cpp:8603] Framework failover timeout, removing framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000 (default) at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634
> 3: I0123 19:22:05.998239 15992 master.cpp:9480] Removing framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000 (default) at scheduler-be5d74bf-c4bf-4504-9e67-6a4f8df93425@172.17.0.2:45634
> 3: I0123 19:22:05.998915 15995 hierarchical.cpp:344] Removed framework eef8ea11-9247-44f3-84cf-340b24df3a52-0000
> 3: I0123 19:22:06.013475 15986 master.cpp:1148] Master terminating
> 3: I0123 19:22:06.014463 15994 hierarchical.cpp:609] Removed agent eef8ea11-9247-44f3-84cf-340b24df3a52-S1
> 3: [  FAILED  ] MasterTest.RegistryGcByCount (172 ms)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)