You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by as...@apache.org on 2020/10/27 20:15:22 UTC

[mesos] branch master updated (3a81cc9 -> c1e7160)

This is an automated email from the ASF dual-hosted git repository.

asekretenko pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git.


    from 3a81cc9  Updated Postgres URL in CentOS 6 Dockerfile.
     new 4150559  Moved failover timeout validation to stateless FrameworkInfo validation.
     new 3fcc148  Consolidated creation and validation of `allocator::Framework` options.
     new c1e7160  Added validation that offer constraints are set only for existing roles.

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/master/master.cpp                       | 191 ++++++++++++++--------------
 src/master/master.hpp                       |  13 +-
 src/master/validation.cpp                   |  59 ++++++++-
 src/master/validation.hpp                   |   8 ++
 src/tests/master/update_framework_tests.cpp |  87 +++++++++++++
 5 files changed, 253 insertions(+), 105 deletions(-)


[mesos] 02/03: Consolidated creation and validation of `allocator::Framework` options.

Posted by as...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

asekretenko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 3fcc148d5d28405f4256e4cfa7005786de10b4f9
Author: Andrei Sekretenko <as...@apache.org>
AuthorDate: Mon Oct 12 20:47:01 2020 +0200

    Consolidated creation and validation of `allocator::Framework` options.
    
    This merges three near-identical pieces of scattered code in SUBSCRIBE
    and UPDATE_FRAMEWORK execution paths in the Master that validate
    and construct parts of `allocator::FrameworkOptions` (the set of
    suppressed roles and the offer constraints filter) into a single
    function.
    
    This is a prerequisite to adding validation of offer constraint roles.
    
    Review: https://reviews.apache.org/r/72955
---
 src/master/master.cpp     | 172 +++++++++++++++++++++++-----------------------
 src/master/master.hpp     |  13 ++--
 src/master/validation.cpp |  20 ++++++
 src/master/validation.hpp |   4 ++
 4 files changed, 118 insertions(+), 91 deletions(-)

diff --git a/src/master/master.cpp b/src/master/master.cpp
index 6c0523d..531b971 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -2551,8 +2551,7 @@ void Master::reregisterFramework(
 
 
 Option<Error> Master::validateFramework(
-  const FrameworkInfo& frameworkInfo,
-  const google::protobuf::RepeatedPtrField<std::string>& suppressedRoles) const
+    const FrameworkInfo& frameworkInfo) const
 {
   Option<Error> validationError =
     validation::framework::validate(frameworkInfo);
@@ -2583,17 +2582,6 @@ Option<Error> Master::validateFramework(
                  " are not present in the master's --roles");
   }
 
-  // Ensure each of the suppressed role is contained in the list of roles.
-  set<string> frameworkRoles = protobuf::framework::getRoles(frameworkInfo);
-  // The suppressed roles must be contained within the list of all
-  // roles for the framwork.
-  foreach (const string& role, suppressedRoles) {
-    if (!frameworkRoles.count(role)) {
-      return Error("Suppressed role '" + role +
-                   "' is not contained in the list of roles");
-    }
-  }
-
   // TODO(vinod): Deprecate this in favor of authorization.
   if (frameworkInfo.user() == "root" && !flags.root_submissions) {
     return Error("User 'root' is not allowed to run frameworks"
@@ -2614,6 +2602,37 @@ Option<Error> Master::validateFramework(
 }
 
 
+static Try<allocator::FrameworkOptions> createAllocatorFrameworkOptions(
+    const set<string>& validFrameworkRoles,
+    const OfferConstraintsFilter::Options filterOptions,
+    google::protobuf::RepeatedPtrField<std::string>&& suppressedRoles,
+    OfferConstraints offerConstraints)
+{
+  set<string> suppressedRolesSet(
+      make_move_iterator(suppressedRoles.begin()),
+      make_move_iterator(suppressedRoles.end()));
+
+  Option<Error> error = validation::framework::validateSuppressedRoles(
+      validFrameworkRoles, suppressedRolesSet);
+
+  if (error.isSome()) {
+    return *error;
+  }
+
+  // TODO(asekretenko): Validate roles in offer constraints (see MESOS-10176).
+  Try<OfferConstraintsFilter> filter = OfferConstraintsFilter::create(
+      filterOptions, std::move(offerConstraints));
+
+  if (filter.isError()) {
+    return Error(
+        "Offer constraints are not valid: " + std::move(filter.error()));
+  }
+
+  return allocator::FrameworkOptions{
+    std::move(suppressedRolesSet), std::move(*filter)};
+}
+
+
 // Returns None if the framework object approvers are ready and the scheduler
 // trying to SUBSCRIBE is authorized to do so with provided framework info.
 // Otherwise, returns an error to be sent to the scheduler trying to subscribe.
@@ -2665,40 +2684,34 @@ void Master::subscribe(
   LOG(INFO) << "Received subscription request for"
             << " HTTP framework '" << frameworkInfo.name() << "'";
 
-  Option<Error> validationError =
-    validateFramework(frameworkInfo, subscribe.suppressed_roles());
-
-  allocator::FrameworkOptions allocatorOptions;
-
-  // TODO(asekretenko): Validate roles in offer constraints (see MESOS-10176).
-  if (validationError.isNone()) {
-    Try<OfferConstraintsFilter> filter = OfferConstraintsFilter::create(
-        offerConstraintsFilterOptions,
-        OfferConstraints(subscribe.offer_constraints()));
-
-    if (filter.isError()) {
-      validationError = Error(std::move(filter.error()));
-    } else {
-      allocatorOptions.offerConstraintsFilter = std::move(*filter);
-    }
-  }
-
-  if (validationError.isSome()) {
+  auto refuseSubscription = [&](const string& error) {
     LOG(INFO) << "Refusing subscription of framework"
-              << " '" << frameworkInfo.name() << "': "
-              << validationError->message;
+              << " '" << frameworkInfo.name() << "': " << error;
 
     FrameworkErrorMessage message;
-    message.set_message(validationError->message);
+    message.set_message(error);
 
     http.send(message);
     http.close();
+  };
+
+  const Option<Error> validationError = validateFramework(frameworkInfo);
+  if (validationError.isSome()) {
+    refuseSubscription(validationError->message);
     return;
   }
 
-  allocatorOptions.suppressedRoles = set<string>(
-      make_move_iterator(subscribe.mutable_suppressed_roles()->begin()),
-      make_move_iterator(subscribe.mutable_suppressed_roles()->end()));
+  Try<allocator::FrameworkOptions> allocatorOptions =
+    createAllocatorFrameworkOptions(
+        protobuf::framework::getRoles(frameworkInfo),
+        offerConstraintsFilterOptions,
+        std::move(*subscribe.mutable_suppressed_roles()),
+        subscribe.offer_constraints());
+
+  if (allocatorOptions.isError()) {
+    refuseSubscription(allocatorOptions.error());
+    return;
+  }
 
   // Need to disambiguate for the compiler.
   void (Master::*_subscribe)(
@@ -2719,7 +2732,7 @@ void Master::subscribe(
       std::move(frameworkInfo),
       std::move(*subscribe.mutable_offer_constraints()),
       subscribe.force(),
-      std::move(allocatorOptions),
+      std::move(*allocatorOptions),
       lambda::_1));
 }
 
@@ -2918,37 +2931,37 @@ void Master::subscribe(
     return;
   }
 
-  Option<Error> validationError =
-    validateFramework(frameworkInfo, subscribe.suppressed_roles());
+  auto refuseSubscription = [&](const string& error) {
+    LOG(INFO) << "Refusing subscription of framework"
+              << " '" << frameworkInfo.name() << "' at " << from << ": "
+              << error;
+
+    FrameworkErrorMessage message;
+    message.set_message(error);
+    send(from, message);
+  };
+
+  Option<Error> validationError = validateFramework(frameworkInfo);
 
   // Note that re-authentication errors are already handled above.
   if (validationError.isNone()) {
     validationError = validateFrameworkAuthentication(frameworkInfo, from);
   }
 
-  allocator::FrameworkOptions allocatorOptions;
-
-  // TODO(asekretenko): Validate roles in offer constraints (see MESOS-10176).
-  if (validationError.isNone()) {
-    Try<OfferConstraintsFilter> filter = OfferConstraintsFilter::create(
-        offerConstraintsFilterOptions,
-        OfferConstraints(subscribe.offer_constraints()));
-
-    if (filter.isError()) {
-      validationError = Error(std::move(filter.error()));
-    } else {
-      allocatorOptions.offerConstraintsFilter = std::move(*filter);
-    }
+  if (validationError.isSome()) {
+    refuseSubscription(validationError->message);
+    return;
   }
 
-  if (validationError.isSome()) {
-    LOG(INFO) << "Refusing subscription of framework"
-              << " '" << frameworkInfo.name() << "' at " << from << ": "
-              << validationError->message;
+  Try<allocator::FrameworkOptions> allocatorOptions =
+    createAllocatorFrameworkOptions(
+        protobuf::framework::getRoles(frameworkInfo),
+        offerConstraintsFilterOptions,
+        std::move(*subscribe.mutable_suppressed_roles()),
+        subscribe.offer_constraints());
 
-    FrameworkErrorMessage message;
-    message.set_message(validationError->message);
-    send(from, message);
+  if (allocatorOptions.isError()) {
+    refuseSubscription(allocatorOptions.error());
     return;
   }
 
@@ -2968,10 +2981,6 @@ void Master::subscribe(
     frameworkInfo.set_principal(authenticated[from]);
   }
 
-  allocatorOptions.suppressedRoles = set<string>(
-      make_move_iterator(subscribe.mutable_suppressed_roles()->begin()),
-      make_move_iterator(subscribe.mutable_suppressed_roles()->end()));
-
   // Need to disambiguate for the compiler.
   void (Master::*_subscribe)(
       const UPID&,
@@ -2991,7 +3000,7 @@ void Master::subscribe(
       std::move(frameworkInfo),
       std::move(*subscribe.mutable_offer_constraints()),
       subscribe.force(),
-      std::move(allocatorOptions),
+      std::move(*allocatorOptions),
       lambda::_1));
 }
 
@@ -3262,8 +3271,7 @@ Future<process::http::Response> Master::updateFramework(
   LOG(INFO) << "Processing UPDATE_FRAMEWORK call for framework "
             << call.framework_info().id();
 
-  Option<Error> error =
-    validateFramework(call.framework_info(), call.suppressed_roles());
+  Option<Error> error = validateFramework(call.framework_info());
 
   if (error.isSome()) {
     return process::http::BadRequest(
@@ -3281,19 +3289,17 @@ Future<process::http::Response> Master::updateFramework(
   const bool frameworkInfoChanged =
     !typeutils::equivalent(framework->info, call.framework_info());
 
-  allocator::FrameworkOptions allocatorOptions;
-  // TODO(asekretenko): Validate roles in offer constraints (see MESOS-10176).
-  Try<OfferConstraintsFilter> filter = OfferConstraintsFilter::create(
-      offerConstraintsFilterOptions,
-      OfferConstraints(call.offer_constraints()));
-
-    if (filter.isError()) {
-      return process::http::BadRequest(
-          "'UpdateFramework.offer_constraints' are not valid: " +
-          filter.error());
-    }
+  Try<allocator::FrameworkOptions> allocatorOptions =
+    createAllocatorFrameworkOptions(
+        protobuf::framework::getRoles(call.framework_info()),
+        offerConstraintsFilterOptions,
+        std::move(*call.mutable_suppressed_roles()),
+        call.offer_constraints());
 
-  allocatorOptions.offerConstraintsFilter = std::move(*filter);
+  if (allocatorOptions.isError()) {
+    return process::http::BadRequest(
+        "'UpdateFramework' call is not valid: " + allocatorOptions.error());
+  }
 
   ActionObject actionObject =
     ActionObject::frameworkRegistration(call.framework_info());
@@ -3312,15 +3318,11 @@ Future<process::http::Response> Master::updateFramework(
         "Not authorized to " + stringify(actionObject));
   }
 
-  allocatorOptions.suppressedRoles = set<string>(
-    make_move_iterator(call.mutable_suppressed_roles()->begin()),
-    make_move_iterator(call.mutable_suppressed_roles()->end()));
-
   updateFramework(
       framework,
       call.framework_info(),
       std::move(*call.mutable_offer_constraints()),
-      std::move(allocatorOptions));
+      std::move(*allocatorOptions));
 
   if (frameworkInfoChanged) {
     // NOTE: Among the framework properties that can be changed by this call
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 83d8190..c200c32 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -1058,12 +1058,13 @@ private:
     const std::string role;
   };
 
-  // Performs validations of the FrameworkInfo and suppressed roles set
-  // which do not depend on the current state of this framework.
-  Option<Error> validateFramework(
-      const FrameworkInfo& frameworkInfo,
-      const google::protobuf::RepeatedPtrField<std::string>& suppressedRoles)
-    const;
+  // Performs stateless and stateful validation of the FrameworkInfo.
+  // The stateful validation only uses the master flags and whether
+  // the framework is completed.
+
+  // TODO(asekretenko): Pass in the state explicitly to move this function into
+  // validation.hpp for unit testability and better separation of concerns.
+  Option<Error> validateFramework(const FrameworkInfo& frameworkInfo) const;
 
   /**
    * Inner class used to namespace the handling of quota requests.
diff --git a/src/master/validation.cpp b/src/master/validation.cpp
index feeea8e..aafffd5 100644
--- a/src/master/validation.cpp
+++ b/src/master/validation.cpp
@@ -37,6 +37,7 @@
 #include <stout/hashset.hpp>
 #include <stout/lambda.hpp>
 #include <stout/none.hpp>
+#include <stout/set.hpp>
 #include <stout/stringify.hpp>
 
 #include "checks/checker.hpp"
@@ -677,6 +678,25 @@ void preserveImmutableFields(
 }
 
 
+Option<Error> validateSuppressedRoles(
+    const set<string>& validFrameworkRoles,
+    const set<string>& suppressedRoles)
+{
+  // Needed to prevent shadowing of the template '::operator-<std::set<T>>'
+  // by a non-template '::mesos::operator-'
+  using ::operator-;
+
+  set<string> invalidSuppressedRoles = suppressedRoles - validFrameworkRoles;
+  if (!invalidSuppressedRoles.empty()) {
+    return Error(
+        "Suppressed roles " + stringify(invalidSuppressedRoles) +
+        " are not contained in the set of roles");
+  }
+
+  return None();
+}
+
+
 } // namespace framework {
 
 
diff --git a/src/master/validation.hpp b/src/master/validation.hpp
index 7fe8f08..6239492 100644
--- a/src/master/validation.hpp
+++ b/src/master/validation.hpp
@@ -130,6 +130,10 @@ void preserveImmutableFields(
     const FrameworkInfo& oldInfo,
     FrameworkInfo* newInfo);
 
+Option<Error> validateSuppressedRoles(
+    const std::set<std::string>& validFrameworkRoles,
+    const std::set<std::string>& suppressedRoles);
+
 } // namespace framework {
 
 


[mesos] 01/03: Moved failover timeout validation to stateless FrameworkInfo validation.

Posted by as...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

asekretenko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 4150559ab29e37c8b5d65024ce3a728b359de2f1
Author: Andrei Sekretenko <as...@apache.org>
AuthorDate: Wed Oct 14 17:46:46 2020 +0200

    Moved failover timeout validation to stateless FrameworkInfo validation.
    
    This turns the validation of the failover timeout in `FrameworkInfo`
    into  a part of `validation::framework::validate()` that performs
    all the other validations that depend on `FrameworkInfo` only.
    
    Review: https://reviews.apache.org/r/72964
---
 src/master/master.cpp     | 13 -------------
 src/master/validation.cpp | 20 +++++++++++++++++++-
 2 files changed, 19 insertions(+), 14 deletions(-)

diff --git a/src/master/master.cpp b/src/master/master.cpp
index d6d3ea7..6c0523d 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -157,8 +157,6 @@ using mesos::master::detector::MasterDetector;
 
 using mesos::scheduler::OfferConstraints;
 
-static bool isValidFailoverTimeout(const FrameworkInfo& frameworkInfo);
-
 
 class SlaveObserver : public ProtobufProcess<SlaveObserver>
 {
@@ -2612,11 +2610,6 @@ Option<Error> Master::validateFramework(
     return Error("Framework has been removed");
   }
 
-  if (!isValidFailoverTimeout(frameworkInfo)) {
-    return Error("The framework failover_timeout (" +
-                 stringify(frameworkInfo.failover_timeout()) +
-                 ") is invalid");
-  }
   return Option<Error>::none();
 }
 
@@ -12319,12 +12312,6 @@ double Master::_resources_revocable_percent(const string& name)
 }
 
 
-static bool isValidFailoverTimeout(const FrameworkInfo& frameworkInfo)
-{
-  return Duration::create(frameworkInfo.failover_timeout()).isSome();
-}
-
-
 void Master::Subscribers::send(
     const mesos::master::Event& event,
     const Option<FrameworkInfo>& frameworkInfo,
diff --git a/src/master/validation.cpp b/src/master/validation.cpp
index 5b1bcb5..feeea8e 100644
--- a/src/master/validation.cpp
+++ b/src/master/validation.cpp
@@ -560,13 +560,25 @@ Option<Error> validateOfferFilters(const FrameworkInfo& frameworkInfo)
   return None();
 }
 
+
+Option<Error> validateFailoverTimeout(const FrameworkInfo& frameworkInfo)
+{
+  if (Duration::create(frameworkInfo.failover_timeout()).isSome()) {
+    return None();
+  }
+
+  return Error(
+      "The framework failover_timeout (" +
+      stringify(frameworkInfo.failover_timeout()) + ") is invalid");
+}
+
 } // namespace internal {
 
 
 Option<Error> validate(const mesos::FrameworkInfo& frameworkInfo)
 {
   // TODO(jay_guo): This currently only validates the role(s),
-  // framework ID and offer filters, validate more fields!
+  // framework ID, offer filters and failover timeout, validate more fields!
   Option<Error> error = internal::validateRoles(frameworkInfo);
 
   if (error.isSome()) {
@@ -585,6 +597,12 @@ Option<Error> validate(const mesos::FrameworkInfo& frameworkInfo)
     return error;
   }
 
+  error = internal::validateFailoverTimeout(frameworkInfo);
+
+  if(error.isSome()) {
+    return error;
+  }
+
   return None();
 }
 


[mesos] 03/03: Added validation that offer constraints are set only for existing roles.

Posted by as...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

asekretenko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit c1e716054d8dead61074c8619ffa7b33f3064152
Author: Andrei Sekretenko <as...@apache.org>
AuthorDate: Mon Oct 12 22:06:51 2020 +0200

    Added validation that offer constraints are set only for existing roles.
    
    This patch makes SUBSCRIBE/UPDATE_FRAMEWORK calls validate that
    the framework does not specify offer constraints for roles to which
    it is not going to be subscribed.
    
    Review: https://reviews.apache.org/r/72956
---
 src/master/master.cpp                       |  8 ++-
 src/master/validation.cpp                   | 19 +++++++
 src/master/validation.hpp                   |  4 ++
 src/tests/master/update_framework_tests.cpp | 87 +++++++++++++++++++++++++++++
 4 files changed, 117 insertions(+), 1 deletion(-)

diff --git a/src/master/master.cpp b/src/master/master.cpp
index 531b971..164720a 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -2619,7 +2619,13 @@ static Try<allocator::FrameworkOptions> createAllocatorFrameworkOptions(
     return *error;
   }
 
-  // TODO(asekretenko): Validate roles in offer constraints (see MESOS-10176).
+  error = validation::framework::validateOfferConstraintsRoles(
+      validFrameworkRoles, offerConstraints);
+
+  if (error.isSome()) {
+    return *error;
+  }
+
   Try<OfferConstraintsFilter> filter = OfferConstraintsFilter::create(
       filterOptions, std::move(offerConstraints));
 
diff --git a/src/master/validation.cpp b/src/master/validation.cpp
index aafffd5..6bdab54 100644
--- a/src/master/validation.cpp
+++ b/src/master/validation.cpp
@@ -59,6 +59,8 @@ using std::vector;
 
 using google::protobuf::RepeatedPtrField;
 
+using mesos::scheduler::OfferConstraints;
+
 namespace mesos {
 namespace internal {
 namespace master {
@@ -697,6 +699,23 @@ Option<Error> validateSuppressedRoles(
 }
 
 
+Option<Error> validateOfferConstraintsRoles(
+    const set<string>& validFrameworkRoles,
+    const OfferConstraints& offerConstraints)
+{
+  for (const auto& pair : offerConstraints.role_constraints()) {
+    const string& role = pair.first;
+    if (validFrameworkRoles.count(role) < 1) {
+      return Error(
+          "Offer constraints specify `role_constraints` for a role '" + role +
+          "'not contained in the set of roles");
+    }
+  }
+
+  return None();
+}
+
+
 } // namespace framework {
 
 
diff --git a/src/master/validation.hpp b/src/master/validation.hpp
index 6239492..17652e3 100644
--- a/src/master/validation.hpp
+++ b/src/master/validation.hpp
@@ -134,6 +134,10 @@ Option<Error> validateSuppressedRoles(
     const std::set<std::string>& validFrameworkRoles,
     const std::set<std::string>& suppressedRoles);
 
+Option<Error> validateOfferConstraintsRoles(
+    const std::set<std::string>& validFrameworkRoles,
+    const scheduler::OfferConstraints& offerConstraints);
+
 } // namespace framework {
 
 
diff --git a/src/tests/master/update_framework_tests.cpp b/src/tests/master/update_framework_tests.cpp
index 3f86573..ce5a51c 100644
--- a/src/tests/master/update_framework_tests.cpp
+++ b/src/tests/master/update_framework_tests.cpp
@@ -981,6 +981,93 @@ TEST_F(UpdateFrameworkTest, OfferConstraints)
 }
 
 
+// This test ensures that an UPDATE_FRAMEWORK call trying to set offer
+// constraints for a role to which the framework will not be subscribed
+// fails validation and is rejected as a whole.
+TEST_F(UpdateFrameworkTest, OfferConstraintsForUnsubscribedRole)
+{
+  using ::mesos::v1::scheduler::OfferConstraints;
+
+  const Try<JSON::Object> constraintsJson = JSON::parse<JSON::Object>(
+      R"~(
+        {
+          "role_constraints": {
+            ")~" + DEFAULT_FRAMEWORK_INFO.roles(0) + R"~(": {
+              "groups": [{
+                "attribute_constraints": [{
+                  "selector": {"attribute_name": "foo"},
+                  "predicate": {"exists": {}}
+                }]
+              }]
+            }
+          }
+        })~");
+
+  ASSERT_SOME(constraintsJson);
+
+  const Try<OfferConstraints> constraints =
+    ::protobuf::parse<OfferConstraints>(*constraintsJson);
+
+  ASSERT_SOME(constraints);
+
+  Try<Owned<cluster::Master>> master = StartMaster(CreateMasterFlags());
+  ASSERT_SOME(master);
+
+  auto scheduler = std::make_shared<MockHTTPScheduler>();
+
+  Future<Nothing> connected;
+  EXPECT_CALL(*scheduler, connected(_))
+    .WillOnce(Invoke([constraints](Mesos* mesos) {
+      Call call;
+      call.set_type(Call::SUBSCRIBE);
+      *call.mutable_subscribe()->mutable_framework_info() =
+        DEFAULT_FRAMEWORK_INFO;
+
+      *call.mutable_subscribe()->mutable_offer_constraints() = *constraints;
+
+      mesos->send(call);
+    }));
+
+  EXPECT_CALL(*scheduler, heartbeat(_))
+    .WillRepeatedly(Return()); // Ignore heartbeats.
+
+  Future<Event::Subscribed> subscribed;
+
+  EXPECT_CALL(*scheduler, subscribed(_, _))
+    .WillOnce(FutureArg<1>(&subscribed));
+
+  TestMesos mesos(master->get()->pid, ContentType::PROTOBUF, scheduler);
+
+  AWAIT_READY(subscribed);
+
+  // Try to change framework's role but specify constraints for the old role.
+  FrameworkInfo frameworkInfoWithNewRole = DEFAULT_FRAMEWORK_INFO;
+  frameworkInfoWithNewRole.clear_roles();
+  frameworkInfoWithNewRole.add_roles(DEFAULT_FRAMEWORK_INFO.roles(0) + "_new");
+  *frameworkInfoWithNewRole.mutable_id() = subscribed->framework_id();
+
+  Future<APIResult> result =
+    callUpdateFramework(&mesos, frameworkInfoWithNewRole, {}, *constraints);
+
+  AWAIT_READY(result);
+  EXPECT_EQ(result->status_code(), 400u);
+  EXPECT_TRUE(strings::contains(result->error(), "Offer constraints"));
+
+  Future<v1::master::Response::GetFrameworks> frameworks =
+    getFrameworks(master->get()->pid);
+
+  AWAIT_READY(frameworks);
+
+  ASSERT_EQ(frameworks->frameworks_size(), 1);
+
+  // The framework info should have remained the same, despite
+  // `updatedFrameworkInfo` on its own not containing any invalid updates.
+  FrameworkInfo expected = DEFAULT_FRAMEWORK_INFO;
+  *expected.mutable_id() = subscribed->framework_id();
+  EXPECT_NONE(::mesos::v1::typeutils::diff(
+      frameworks->frameworks(0).framework_info(), expected));
+}
+
 } // namespace scheduler {
 } // namespace v1 {