You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by vi...@apache.org on 2016/07/22 18:51:48 UTC

[1/4] mesos git commit: Disabled the `--registry_strict` master flag.

Repository: mesos
Updated Branches:
  refs/heads/1.0.x c09ee618c -> 3931cef73


Disabled the `--registry_strict` master flag.

This flag was always marked as experimental. This flag controlled
whether the master will behave the same, regardless of whether the
master has failed over. Notably, in Mesos 1.0, enabling this flag would
mean that a partitioned agent would always be refused reregistration,
regardless of whether the master has failed over (by default, the agent
would be allowed to reregister iff the master failed over in the
interim).

For Mesos 1.1, we'd like to change the behavior so that partitioned
agents are always allowed to reregister (regardless of master
failover). To prevent developers from writing frameworks that depend on
the old, "strict" behavior (partitioned agents are always rerefused
reregistration), we'd like to disable the "--registry_strict" flag.

The code that implements the strict registry remains, as do the test
cases that depend on this behavior. However, `mesos-master` will refuse
to start if the flag has been specified.

Review: https://reviews.apache.org/r/49955/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/0d51860c
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/0d51860c
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/0d51860c

Branch: refs/heads/1.0.x
Commit: 0d51860c68d59fb9b52b63eb499d5f63eace6536
Parents: c09ee61
Author: Neil Conway <ne...@gmail.com>
Authored: Fri Jul 22 09:24:13 2016 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Jul 22 11:19:43 2016 -0700

----------------------------------------------------------------------
 CHANGELOG                                 |  2 ++
 docs/configuration.md                     | 14 --------------
 docs/high-availability-framework-guide.md |  3 +--
 docs/upgrades.md                          |  5 +++++
 src/local/local.cpp                       |  5 -----
 src/master/flags.cpp                      | 17 +++++++++++++----
 src/master/main.cpp                       |  5 -----
 7 files changed, 21 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0d51860c/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index fa6b0b4..9e02e6b 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -195,6 +195,8 @@ Additional API Changes:
     `--authenticate_http_readonly`, which enables authentication for those
     authenticatable endpoints that cannot be used to modify the cluster state.
 
+  * [MESOS-5833] - Disable the experimental `registry_strict` master flag.
+
 3rd Party Upgrades:
   * [MESOS-4805] - Upgraded vendored ry-http-parser-1c3624a to nodejs/http-parser 2.6.1.
   * [MESOS-4678] - Upgraded vendored protobuf 2.5.0 to 2.6.1.

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d51860c/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 666fb87..afe2378 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -802,20 +802,6 @@ after which the operation is considered a failure. (default: 20secs)
 </tr>
 <tr>
   <td>
-    --[no-]registry_strict
-  </td>
-  <td>
-Whether the master will take actions based on the persistent
-information stored in the Registry. Setting this to false means
-that the Registrar will never reject the admission, readmission,
-or removal of a agent. Consequently, <code>false</code> can be used to
-bootstrap the persistent state on a running cluster.
-<b>NOTE</b>: This flag is *experimental* and should not be used in
-production yet. (default: false)
-  </td>
-</tr>
-<tr>
-  <td>
     --roles=VALUE
   </td>
   <td>

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d51860c/docs/high-availability-framework-guide.md
----------------------------------------------------------------------
diff --git a/docs/high-availability-framework-guide.md b/docs/high-availability-framework-guide.md
index ae5617b..73743ab 100644
--- a/docs/high-availability-framework-guide.md
+++ b/docs/high-availability-framework-guide.md
@@ -349,5 +349,4 @@ one difference: by default, agents are _allowed to reconnect_ following master
 failover, even after the `agent_reregister_timeout` has fired. This means that
 frameworks might see a `TASK_LOST` update for a task but then later discover
 that the task is running (because the agent where it was running was allowed to
-reconnect). This behavior can be avoided by enabling the `--registry_strict`
-configuration flag, which will be the default in a future version of Mesos.
+reconnect).

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d51860c/docs/upgrades.md
----------------------------------------------------------------------
diff --git a/docs/upgrades.md b/docs/upgrades.md
index 82b9e26..d3e7cdd 100644
--- a/docs/upgrades.md
+++ b/docs/upgrades.md
@@ -65,6 +65,7 @@ We categorize the changes as follows:
       <li>R <a href="#1-0-x-workdir">work_dir default value</a></li>
       <li>D <a href="#1-0-x-deprecated-ssl-env-variables">SSL environment variables</a></li>
       <li>ACD <a href="#1-0-x-http-authentication-flags">HTTP authentication</a></li>
+      <li>R <a href="#1-0-x-registry-strict">registry_strict</a></li>
     </ul>
   </td>
   <td style="word-wrap: break-word; overflow-wrap: break-word;"><!--Framework API-->
@@ -220,6 +221,10 @@ We categorize the changes as follows:
 
 * Mesos 1.0 removes the default value for the agent's `work_dir` command-line flag. This flag is now required; the agent will exit immediately if it is not provided.
 
+<a name="1-0-x-registry-strict"></a>
+
+* Mesos 1.0 disables support for the master's `registry_strict` command-line flag. If this flag is specified, the master will exit immediately. Note that this flag was previously marked as experimental and not recommended for production use.
+
 <a name="1-0-x-credentials-file"></a>
 
 * Mesos 1.0 deprecates the use of plain text credential files in favor of JSON-formatted credential files.

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d51860c/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index a1a9a6b..d6fe4d0 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -192,11 +192,6 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
     }
 
     if (flags.registry == "in_memory") {
-      if (flags.registry_strict) {
-        EXIT(EXIT_FAILURE)
-          << "Cannot use '--registry_strict' when using in-memory storage"
-          << " based registry";
-      }
       storage = new mesos::state::InMemoryStorage();
     } else if (flags.registry == "replicated_log") {
       // For local runs, we use a temporary work directory.

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d51860c/src/master/flags.cpp
----------------------------------------------------------------------
diff --git a/src/master/flags.cpp b/src/master/flags.cpp
index 176133f..19ae6c1 100644
--- a/src/master/flags.cpp
+++ b/src/master/flags.cpp
@@ -84,7 +84,8 @@ mesos::internal::master::Flags::Flags()
       "ZooKeeper session timeout.",
       ZOOKEEPER_SESSION_TIMEOUT);
 
-  // TODO(bmahler): Set the default to true in 0.20.0.
+  // TODO(neilc): This flag is deprecated in 1.0 and will be removed 6
+  // months later.
   add(&Flags::registry_strict,
       "registry_strict",
       "Whether the master will take actions based on the persistent\n"
@@ -92,9 +93,17 @@ mesos::internal::master::Flags::Flags()
       "that the Registrar will never reject the admission, readmission,\n"
       "or removal of an agent. Consequently, `false` can be used to\n"
       "bootstrap the persistent state on a running cluster.\n"
-      "NOTE: This flag is *experimental* and should not be used in\n"
-      "production yet.",
-      false);
+      "NOTE: This flag is *disabled* and will be removed in a future\n"
+      "version of Mesos.",
+      false,
+      [](bool value) -> Option<Error> {
+        if (value) {
+          return Error("Support for '--registry_strict' has been "
+                       "disabled and will be removed in a future "
+                       "version of Mesos");
+        }
+        return None();
+      });
 
   add(&Flags::registry_fetch_timeout,
       "registry_fetch_timeout",

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d51860c/src/master/main.cpp
----------------------------------------------------------------------
diff --git a/src/master/main.cpp b/src/master/main.cpp
index 6146c31..e202973 100644
--- a/src/master/main.cpp
+++ b/src/master/main.cpp
@@ -373,11 +373,6 @@ int main(int argc, char** argv)
   Log* log = nullptr;
 
   if (flags.registry == "in_memory") {
-    if (flags.registry_strict) {
-      EXIT(EXIT_FAILURE)
-        << "Cannot use '--registry_strict' when using in-memory storage"
-        << " based registry";
-    }
     storage = new InMemoryStorage();
   } else if (flags.registry == "replicated_log" ||
              flags.registry == "log_storage") {


[2/4] mesos git commit: Fixed the incomplete `TaskStatus` message of docker executor.

Posted by vi...@apache.org.
Fixed the incomplete `TaskStatus` message of docker executor.

When we enable health check for the tasks run in docker container,
their `TaskStatus` messages generated by docker executor may miss
`NetworkInfo` field and agent would fill the host ip as default value.
In this change, we cache the `NetworkInfo` of the task and reuse it
when generating `TaskStatus` messages which the task's health is
updated.

Review: https://reviews.apache.org/r/50103/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/ff5e084d
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/ff5e084d
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/ff5e084d

Branch: refs/heads/1.0.x
Commit: ff5e084da779dc2294bb9ee50fb515a85b1dbe8c
Parents: 0d51860
Author: haosdent huang <ha...@gmail.com>
Authored: Mon Jul 18 18:01:42 2016 +0200
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Jul 22 11:41:49 2016 -0700

----------------------------------------------------------------------
 src/docker/executor.cpp | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ff5e084d/src/docker/executor.cpp
----------------------------------------------------------------------
diff --git a/src/docker/executor.cpp b/src/docker/executor.cpp
index 9fcf83c..0d1fd65 100644
--- a/src/docker/executor.cpp
+++ b/src/docker/executor.cpp
@@ -201,6 +201,8 @@ public:
 
             NetworkInfo::IPAddress* ipAddress = networkInfo->add_ip_addresses();
             ipAddress->set_ip_address(container.ipAddress.get());
+
+            containerNetworkInfo = *networkInfo;
           }
           driver->sendStatusUpdate(status);
         }
@@ -295,6 +297,12 @@ protected:
     status.mutable_task_id()->CopyFrom(taskID);
     status.set_healthy(healthy);
     status.set_state(TASK_RUNNING);
+
+    if (containerNetworkInfo.isSome()) {
+      status.mutable_container_status()->add_network_infos()->CopyFrom(
+          containerNetworkInfo.get());
+    }
+
     driver.get()->sendStatusUpdate(status);
 
     if (initiateTaskKill) {
@@ -576,6 +584,7 @@ private:
   Option<ExecutorDriver*> driver;
   Option<FrameworkInfo> frameworkInfo;
   Option<TaskID> taskId;
+  Option<NetworkInfo> containerNetworkInfo;
 };
 
 


[4/4] mesos git commit: Updated CHANGELOG for 1.0.0-rc3.

Posted by vi...@apache.org.
Updated CHANGELOG for 1.0.0-rc3.


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/3931cef7
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/3931cef7
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/3931cef7

Branch: refs/heads/1.0.x
Commit: 3931cef7368ba320ad1b067b67746300da246847
Parents: 614ffcd
Author: Vinod Kone <vi...@gmail.com>
Authored: Fri Jul 22 11:41:37 2016 -0700
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Jul 22 11:50:29 2016 -0700

----------------------------------------------------------------------
 CHANGELOG | 69 ++++++++++++++++++++++++++++++++++++++++++++++++++++++++--
 1 file changed, 67 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/3931cef7/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 9e02e6b..17e51ff 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -222,9 +222,12 @@ All Issues:
     * [MESOS-2201] - ReplicaTest.Restore fails with leveldb greater than v1.7.
     * [MESOS-2331] - MasterSlaveReconciliationTest.ReconcileRace is flaky
     * [MESOS-2858] - FetcherCacheHttpTest.HttpMixed is flaky.
+    * [MESOS-3181] - Implement package rebundling for Windows
     * [MESOS-3319] - Mesos will not build when configured with gperftools enabled
     * [MESOS-3402] - mesos-execute does not support credentials
     * [MESOS-3573] - Mesos does not kill orphaned docker containers
+    * [MESOS-3714] - `os::environ` collides with the `environ` macro in Windows headers.
+    * [MESOS-3737] - `limiter.hpp` causes template specialization error on Windows 10/MSVC 1900
     * [MESOS-3739] - Mesos does not set Content-Type for 400 Bad Request
     * [MESOS-3881] - Implement `stout/os/pstree.hpp` on Windows
     * [MESOS-3902] - The Location header when non-leading master redirects to leading master is incomplete.
@@ -237,10 +240,16 @@ All Issues:
     * [MESOS-4434] - Install 3rdparty package boost, glog, protobuf and picojson when installing Mesos
     * [MESOS-4447] - Renamed reserved() API to reservations()
     * [MESOS-4462] - Port `gmtime_r`
+    * [MESOS-4463] - Implement `hsterror`
+    * [MESOS-4464] - Implement cpu count facilities on Windows
     * [MESOS-4465] - Implement pagesize facilities in Windows
     * [MESOS-4466] - Implement `waitpid` in Windows
+    * [MESOS-4469] - Implement memory querying in Windows
     * [MESOS-4470] - Implement `uname` in Windows
     * [MESOS-4471] - Implement process querying/counting in Windows
+    * [MESOS-4472] - Implement `getenv` in Windows
+    * [MESOS-4473] - Implement `shell` in Windows
+    * [MESOS-4474] - Implement `sendfile` in Windows
     * [MESOS-4580] - Consider returning `202` (Accepted) for /reserve and related endpoints
     * [MESOS-4611] - Passing a lambda to dispatch() always matches the template returning void
     * [MESOS-4633] - Tests will dereference stack allocated agent objects upon assertion/expectation failure.
@@ -271,6 +280,7 @@ All Issues:
     * [MESOS-4957] - Typo in Mesos portal
     * [MESOS-4961] - ContainerLoggerTest.LOGROTATE_RotateInSandbox is flaky
     * [MESOS-4963] - Incorrect CXXFLAGS with GCC 6
+    * [MESOS-4972] - Implement `os::rename`
     * [MESOS-4978] - Update mesos-execute with Appc changes.
     * [MESOS-4981] - Framework (re-)register metric counters broken for calls made via scheduler driver
     * [MESOS-4984] - MasterTest.SlavesEndpointTwoSlaves is flaky
@@ -324,6 +334,7 @@ All Issues:
     * [MESOS-5282] - Destroy container while provisioning volume images may lead to a race.
     * [MESOS-5287] - boto is no longer a Mesos dependency.
     * [MESOS-5293] - Endpoint handlers for master and agent are implemented surprisingly differently.
+    * [MESOS-5294] - Status updates after a health check are incomplete or invalid
     * [MESOS-5295] - The task launched by non-checkpointed HTTP command executor will keep running till executor shutdown grace period (5s) after agent process exits.
     * [MESOS-5304] - /metrics/snapshot endpoint help disappeared on agent.
     * [MESOS-5308] - ROOT_XFS_QuotaTest.NoCheckpointRecovery failed.
@@ -353,6 +364,7 @@ All Issues:
     * [MESOS-5416] - make check of stout fails.
     * [MESOS-5422] - Website README.md is out of dated
     * [MESOS-5423] - Updating the website section in release-guide is out of dated
+    * [MESOS-5428] - Update the mechanism to define flags in FlagsBase derived clases
     * [MESOS-5429] - Enhance error message for mesos-ps
     * [MESOS-5432] - Javadoc in project website didn't include the generated protobuf
     * [MESOS-5434] - Incomplete bootstrap 3.3.6 upgrade in webui
@@ -419,9 +431,11 @@ All Issues:
     * [MESOS-5766] - Missing License Information for Bundled NVML headers
     * [MESOS-5794] - Agent's /containers endpoint should skip terminated executors.
     * [MESOS-5799] - docker::inspect() may get wrong output when a docker container is not in "running" state
-    * [MESOS-5806] - CNI isolator should prepare network related /etc/* files for containers using host mode but specify container images
+    * [MESOS-5806] - CNI isolator should prepare network related /etc/* files for containers using host mode but specify container images.
+    * [MESOS-5844] - PersistentVolumeEndpointsTest.OfferCreateThenEndpointRemove test is flaky
     * [MESOS-5845] - The fetcher can access any local file as root
     * [MESOS-5848] - Docker health checks are malformed.
+    * [MESOS-5851] - Create mechanism to control authentication between different HTTP endpoints
     * [MESOS-5863] - Enabling SSL causes fetcher fail to fetch from HTTPS sites.
 
 ** Documentation
@@ -457,6 +471,7 @@ All Issues:
     * [MESOS-2154] - Port CFS quota support to Docker Containerizer
     * [MESOS-2281] - Deprecate plain text Credential format.
     * [MESOS-2372] - Test script for verifying compatibility between Mesos components
+    * [MESOS-2602] - Provide a way to "push" cluster state updates to a registered service.
     * [MESOS-2720] - Publish the schema for operator endpoints
     * [MESOS-3243] - Replace NULL with nullptr
     * [MESOS-3690] - Make Apache Mesos' website mobile friendly
@@ -570,6 +585,7 @@ All Issues:
     * [MESOS-5457] - Create a small testing doc for the v1 Scheduler/Executor API
     * [MESOS-5459] - Update RUN_TASK_WITH_USER to use additional metadata
     * [MESOS-5519] - Refresh Mesos project website homepage
+    * [MESOS-5532] - Maven build is too verbose for batch builds
     * [MESOS-5540] - Support building with non-GNU libc
     * [MESOS-5550] - Remove Nvidia GPU Isolator's link-time dependence on `libnvidia-ml`
     * [MESOS-5551] - Move the Nvidia GPU isolator from `cgroups/devices/gpu/nvidia` to `gpu/nvidia`
@@ -604,6 +620,7 @@ All Issues:
     * [MESOS-5782] - Renamed 'commands' to 'pre_exec_commands' in ContainerLaunchInfo.
     * [MESOS-5787] - Add ability to set framework capabilities in 'mesos-execute'
     * [MESOS-5793] - Add ability to inject Nvidia devices into a container
+    * [MESOS-5833] - Disable '--registry_strict' master flag
 
 ** Task
     * [MESOS-338] - Mesos 1.0
@@ -611,21 +628,66 @@ All Issues:
     * [MESOS-2408] - Slave should reclaim storage for destroyed persistent volumes.
     * [MESOS-2950] - Implement current mesos Authorizer in terms of generalized Authorizer interface
     * [MESOS-3063] - Add an example framework using dynamic reservation
+    * [MESOS-3103] - Separate OS-specific code in the libprocess library
     * [MESOS-3214] - Replace boost foreach with range-based for
     * [MESOS-3368] - Add device support in cgroups abstraction
+    * [MESOS-3371] - Implement process::subprocess on Windows
+    * [MESOS-3436] - Port dynamiclibrary_test.cpp to Windows
+    * [MESOS-3438] - Port gzip_test to Windows
+    * [MESOS-3439] - Port ip_tests
     * [MESOS-3443] - Windows: Port protobuf_tests.hpp
     * [MESOS-3541] - Add CMakeLists that builds the Mesos master
     * [MESOS-3558] - Implement  HTTPCommandExecutor that uses the Executor Library 
     * [MESOS-3559] - Make the Command Scheduler use the HTTP Scheduler Library
+    * [MESOS-3609] - Port slave/gc.cpp
+    * [MESOS-3610] - Port slave/flags.cpp to Windows
+    * [MESOS-3611] - Port slave/http.cpp to Windows
+    * [MESOS-3612] - Port slave/metrics.cpp to Windows
+    * [MESOS-3614] - Port slave/slave.cpp to Windows
+    * [MESOS-3616] - Port slave/status_update_manager.cpp to Windows
+    * [MESOS-3617] - Port slave/containerizer/containerizer.cpp to Windows
     * [MESOS-3618] - Port slave/containerizer/fetcher.cpp
     * [MESOS-3619] - Port slave/containerizer/isolator.cpp to Windows
     * [MESOS-3620] - Create slave/containerizer/isolators/filesystem/windows.cpp
+    * [MESOS-3622] - Port slave/containerizer/launcher.cpp to Windows
     * [MESOS-3623] - Port slave/containerizer/mesos/containerizer.cpp to Windows
     * [MESOS-3624] - Port slave/containerizer/mesos/launch.cpp to Windows
+    * [MESOS-3634] - Port process/protobuf.hpp
+    * [MESOS-3635] - Port process/defer.hpp to Windows
+    * [MESOS-3636] - Port process/dispatch.hpp
     * [MESOS-3637] - Port process/process.hpp to Windows
     * [MESOS-3639] - Implement stout/os/windows/killtree.hpp
     * [MESOS-3641] - Implement stout/os/windows/read.hpp and write.hpp
+    * [MESOS-3642] - Implement stout/os/windows/sendfile.hpp
+    * [MESOS-3646] - Port process/clock.hpp to Windows
+    * [MESOS-3647] - Port process/time.hpp to Windows
+    * [MESOS-3648] - Port stout/duration.hpp to Windows
+    * [MESOS-3649] - Port process/future.hpp to Windows
+    * [MESOS-3650] - Port process/event.hpp to Windows
+    * [MESOS-3651] - Port process/latch.hpp to Windows
+    * [MESOS-3652] - Port process/http.hpp to Windows
+    * [MESOS-3653] - Port process/message.hpp to Windows
+    * [MESOS-3654] - Port process/filter.hpp to Windows
+    * [MESOS-3657] - Port process/deferred.hpp to Windows
+    * [MESOS-3661] - Port slave/metrics.hpp to Windows
+    * [MESOS-3662] - Port slave/slave.hpp to Windows
+    * [MESOS-3663] - Port process/metrics/gauge.hpp to Windows
+    * [MESOS-3664] - Port process/metrics/metric.hpp to Windows
+    * [MESOS-3666] - Port process/metrics/metrics.hpp to Windows
+    * [MESOS-3668] - Port process/delay.hpp to Windows
+    * [MESOS-3669] - Port process/clock.hpp to Windows
+    * [MESOS-3670] - Port process/time.hpp to Windows
+    * [MESOS-3671] - Port stout/duration.hpp to Windows
+    * [MESOS-3672] - Port process/timer.hpp to Windows
+    * [MESOS-3673] - Port process/timeout.hpp to Windows
+    * [MESOS-3674] - Port process/async.hpp to Windows
+    * [MESOS-3675] - Port process/check.hpp to Windows
+    * [MESOS-3679] - Port slave/containerizer/containerizer.hpp to Windows
+    * [MESOS-3680] - Port process/subprocess.hpp to Windows
+    * [MESOS-3681] - Port slave/containerizer/fetcher.hpp to Windows
+    * [MESOS-3682] - Port slave/containerizer/launcher.hpp to Windows
     * [MESOS-3683] - Port slave/containerizer/isolator.hpp to Windows
+    * [MESOS-3685] - Port process/io.hpp to Windows
     * [MESOS-3779] - Slave/Agent Rename Phase I - Update terms in Web UI.
     * [MESOS-3781] - Replace Master/Slave Terminology Phase I - Rename flag names and deprecate old ones
     * [MESOS-3782] - Slave/Agent Rename Phase I - Add duplicate binaries (or create symlinks)
@@ -720,7 +782,7 @@ All Issues:
     * [MESOS-5494] - Implement GET_ROLES Call in v1 master API.
     * [MESOS-5495] - Implement GET_WEIGHTS Call in v1 master API.
     * [MESOS-5496] - Implement UPDATE_WEIGHTS Call in v1 master API.
-    * [MESOS-5497] - Implement GET_LEADING_MASTER Call in v1 master API.
+    * [MESOS-5497] - Implement GET_MASTER Call in v1 master API.
     * [MESOS-5498] - Implement SUBSCRIBE Call in v1 master API.
     * [MESOS-5499] - Implement RESERVE_RESOURCES Call in v1 master API.
     * [MESOS-5500] - Implement UNRESERVE_RESOURCES Call in v1 master API.
@@ -737,6 +799,7 @@ All Issues:
     * [MESOS-5511] - Implement GET_METRICS Call in v1 agent API.
     * [MESOS-5512] - Implement GET_LOGGING_LEVEL Call in v1 agent API.
     * [MESOS-5513] - Implement SET_LOGGING_LEVEL Call in v1 agent API.
+    * [MESOS-5514] - Implement LIST_FILES Call in v1 agent API.
     * [MESOS-5517] - Implement GET_RESOURCE_STATISTICS Call in v1 agent API.
     * [MESOS-5518] - Implement GET_CONTAINERS Call in v1 agent API.
     * [MESOS-5549] - Document aufs provisioner backend.
@@ -772,6 +835,8 @@ All Issues:
   * [MESOS-5748] - Potential segfault in `link` when linking to a remote process.
   * [MESOS-5073] - Mesos allocator leaks role sorter and quota role sorters.
   * [MESOS-5698] - Quota sorter not updated for resource changes at agent.
+  * [MESOS-5740] - Consider adding `relink` functionality to libprocess.
+  * [MESOS-5576] - Masters may drop the first message they send between masters after a network partition.
 
 
 Release Notes - Mesos - Version 0.28.2


[3/4] mesos git commit: Fixed flakiness in persistent volume test case.

Posted by vi...@apache.org.
Fixed flakiness in persistent volume test case.

In PersistentVolumeEndpointsTest.OfferCreateThenEndpointRemove,
add an extra offer cycle. Along the way, clean up some comments.

Review: https://reviews.apache.org/r/50023/


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/614ffcd8
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/614ffcd8
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/614ffcd8

Branch: refs/heads/1.0.x
Commit: 614ffcd835845d8b2f04e0a5ef07cd58191c03d6
Parents: ff5e084
Author: Neil Conway <ne...@gmail.com>
Authored: Mon Jul 18 17:38:03 2016 +0200
Committer: Vinod Kone <vi...@gmail.com>
Committed: Fri Jul 22 11:48:40 2016 -0700

----------------------------------------------------------------------
 src/tests/persistent_volume_endpoints_tests.cpp | 35 +++++++++++++++-----
 1 file changed, 26 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/614ffcd8/src/tests/persistent_volume_endpoints_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/persistent_volume_endpoints_tests.cpp b/src/tests/persistent_volume_endpoints_tests.cpp
index 2348f13..266c2a0 100644
--- a/src/tests/persistent_volume_endpoints_tests.cpp
+++ b/src/tests/persistent_volume_endpoints_tests.cpp
@@ -1588,7 +1588,7 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   // Reserve the resources.
   driver.acceptOffers({offer.id()}, {RESERVE(dynamicallyReserved)}, filters);
 
-  // In the next offer, expect an offer with reserved resources.
+  // Expect an offer with reserved resources.
   AWAIT_READY(offers);
 
   ASSERT_EQ(1u, offers.get().size());
@@ -1607,13 +1607,12 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
       frameworkInfo.principal());
 
   EXPECT_CALL(sched, resourceOffers(&driver, _))
-    .WillOnce(FutureArg<1>(&offers))
-    .WillRepeatedly(Return()); // Ignore subsequent offers.
+    .WillOnce(FutureArg<1>(&offers));
 
   // Create the volume.
   driver.acceptOffers({offer.id()}, {CREATE(volume)}, filters);
 
-  // In the next offer, expect an offer with a persistent volume.
+  // Expect an offer with a persistent volume.
   AWAIT_READY(offers);
 
   ASSERT_EQ(1u, offers.get().size());
@@ -1626,6 +1625,10 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   EXPECT_CALL(sched, offerRescinded(&driver, _))
     .WillOnce(FutureArg<1>(&rescindedOfferId));
 
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers));
+
+  // Destroy the volume using HTTP operator endpoint.
   Future<Response> destroyResponse = process::http::post(
       master.get()->pid,
       "destroy-volumes",
@@ -1635,13 +1638,23 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
   AWAIT_EXPECT_RESPONSE_STATUS_EQ(Accepted().status, destroyResponse);
 
   AWAIT_READY(rescindedOfferId);
-
   EXPECT_EQ(rescindedOfferId.get(), offer.id());
 
-  // Expect an offer containing only unreserved resources.
+  // Expect an offer containing reserved resources.
+  AWAIT_READY(offers);
+
+  ASSERT_EQ(1u, offers.get().size());
+  offer = offers.get()[0];
+
+  EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
+
+  EXPECT_CALL(sched, offerRescinded(&driver, _))
+    .WillOnce(FutureArg<1>(&rescindedOfferId));
+
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
 
+  // Unreserve the resources using HTTP operator endpoint.
   Future<Response> unreserveResponse = process::http::post(
       master.get()->pid,
       "unreserve",
@@ -1650,6 +1663,10 @@ TEST_F(PersistentVolumeEndpointsTest, OfferCreateThenEndpointRemove)
 
   AWAIT_EXPECT_RESPONSE_STATUS_EQ(Accepted().status, unreserveResponse);
 
+  AWAIT_READY(rescindedOfferId);
+  EXPECT_EQ(rescindedOfferId.get(), offer.id());
+
+  // Expect an offer containing only unreserved resources.
   AWAIT_READY(offers);
 
   ASSERT_EQ(1u, offers.get().size());
@@ -1728,12 +1745,12 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
 
   EXPECT_CALL(sched, registered(&driver, _, _));
 
-  // Expect an offer containing the persistent volume.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
 
   driver.start();
 
+  // Expect an offer containing the persistent volume.
   AWAIT_READY(offers);
 
   ASSERT_EQ(1u, offers.get().size());
@@ -1746,7 +1763,6 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
   Filters filters;
   filters.set_refuse_seconds(0);
 
-  // Expect an offer containing the dynamic reservation.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
 
@@ -1755,6 +1771,7 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
       {DESTROY(volume)},
       filters);
 
+  // Expect an offer containing the dynamic reservation.
   AWAIT_READY(offers);
 
   ASSERT_EQ(1u, offers.get().size());
@@ -1762,13 +1779,13 @@ TEST_F(PersistentVolumeEndpointsTest, EndpointCreateThenOfferRemove)
 
   EXPECT_TRUE(Resources(offer.resources()).contains(dynamicallyReserved));
 
-  // Expect an offer containing only unreserved resources.
   EXPECT_CALL(sched, resourceOffers(&driver, _))
     .WillOnce(FutureArg<1>(&offers));
 
   // Unreserve the resources.
   driver.acceptOffers({offer.id()}, {UNRESERVE(dynamicallyReserved)}, filters);
 
+  // Expect an offer containing only unreserved resources.
   AWAIT_READY(offers);
 
   ASSERT_EQ(1u, offers.get().size());