You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by mp...@apache.org on 2018/01/03 07:17:25 UTC

[1/5] mesos git commit: Pulled out `upgradeResources` out of `validateAndNormalizeResources`.

Repository: mesos
Updated Branches:
  refs/heads/1.5.x 67eb44d91 -> 0ad8f8d08


Pulled out `upgradeResources` out of `validateAndNormalizeResources`.

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


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

Branch: refs/heads/1.5.x
Commit: 12e3d7fab18ebe6746b1d6a6df508098c23eec37
Parents: 67eb44d
Author: Michael Park <mp...@apache.org>
Authored: Fri Dec 15 18:49:52 2017 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Tue Jan 2 23:04:23 2018 -0800

----------------------------------------------------------------------
 src/common/resources_utils.cpp | 200 ++++++++++++++++++++++--------------
 1 file changed, 123 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/12e3d7fa/src/common/resources_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources_utils.cpp b/src/common/resources_utils.cpp
index 8a2652b..47ba885 100644
--- a/src/common/resources_utils.cpp
+++ b/src/common/resources_utils.cpp
@@ -388,6 +388,115 @@ void convertResourceFormat(
 }
 
 
+void upgradeResources(Offer::Operation* operation)
+{
+  CHECK_NOTNULL(operation);
+
+  switch (operation->type()) {
+    case Offer::Operation::RESERVE: {
+      convertResourceFormat(
+          operation->mutable_reserve()->mutable_resources(),
+          POST_RESERVATION_REFINEMENT);
+
+      return;
+    }
+    case Offer::Operation::UNRESERVE: {
+      convertResourceFormat(
+          operation->mutable_unreserve()->mutable_resources(),
+          POST_RESERVATION_REFINEMENT);
+
+      return;
+    }
+    case Offer::Operation::CREATE: {
+      convertResourceFormat(
+          operation->mutable_create()->mutable_volumes(),
+          POST_RESERVATION_REFINEMENT);
+
+      return;
+    }
+    case Offer::Operation::DESTROY: {
+      convertResourceFormat(
+          operation->mutable_destroy()->mutable_volumes(),
+          POST_RESERVATION_REFINEMENT);
+
+      return;
+    }
+    case Offer::Operation::LAUNCH: {
+      foreach (
+          TaskInfo& task, *operation->mutable_launch()->mutable_task_infos()) {
+        convertResourceFormat(
+            task.mutable_resources(),
+            POST_RESERVATION_REFINEMENT);
+
+        if (task.has_executor()) {
+          convertResourceFormat(
+              task.mutable_executor()->mutable_resources(),
+              POST_RESERVATION_REFINEMENT);
+        }
+      }
+
+      return;
+    }
+    case Offer::Operation::LAUNCH_GROUP: {
+      Offer::Operation::LaunchGroup* launchGroup =
+        operation->mutable_launch_group();
+
+      if (launchGroup->has_executor()) {
+        convertResourceFormat(
+            launchGroup->mutable_executor()->mutable_resources(),
+            POST_RESERVATION_REFINEMENT);
+      }
+
+      foreach (
+          TaskInfo& task, *launchGroup->mutable_task_group()->mutable_tasks()) {
+        convertResourceFormat(
+            task.mutable_resources(), POST_RESERVATION_REFINEMENT);
+
+        if (task.has_executor()) {
+          convertResourceFormat(
+              task.mutable_executor()->mutable_resources(),
+              POST_RESERVATION_REFINEMENT);
+        }
+      }
+
+      return;
+    }
+    case Offer::Operation::CREATE_VOLUME: {
+      convertResourceFormat(
+          operation->mutable_create_volume()->mutable_source(),
+          POST_RESERVATION_REFINEMENT);
+
+      return;
+    }
+    case Offer::Operation::DESTROY_VOLUME: {
+      convertResourceFormat(
+          operation->mutable_destroy_volume()->mutable_volume(),
+          POST_RESERVATION_REFINEMENT);
+
+      return;
+    }
+    case Offer::Operation::CREATE_BLOCK: {
+      convertResourceFormat(
+          operation->mutable_create_block()->mutable_source(),
+          POST_RESERVATION_REFINEMENT);
+
+      return;
+    }
+    case Offer::Operation::DESTROY_BLOCK: {
+      convertResourceFormat(
+          operation->mutable_destroy_block()->mutable_block(),
+          POST_RESERVATION_REFINEMENT);
+
+      return;
+    }
+    case Offer::Operation::UNKNOWN: {
+      return;
+    }
+  }
+  UNREACHABLE();
+}
+
+
 Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
 {
   CHECK_NOTNULL(operation);
@@ -410,11 +519,7 @@ Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
         return error;
       }
 
-      convertResourceFormat(
-          operation->mutable_reserve()->mutable_resources(),
-          POST_RESERVATION_REFINEMENT);
-
-      return None();
+      break;
     }
     case Offer::Operation::UNRESERVE: {
       // TODO(mpark): Once we perform a sanity check validation for
@@ -433,11 +538,7 @@ Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
         return error;
       }
 
-      convertResourceFormat(
-          operation->mutable_unreserve()->mutable_resources(),
-          POST_RESERVATION_REFINEMENT);
-
-      return None();
+      break;
     }
     case Offer::Operation::CREATE: {
       // TODO(mpark): Once we perform a sanity check validation for
@@ -456,11 +557,7 @@ Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
         return error;
       }
 
-      convertResourceFormat(
-          operation->mutable_create()->mutable_volumes(),
-          POST_RESERVATION_REFINEMENT);
-
-      return None();
+      break;
     }
     case Offer::Operation::DESTROY: {
       // TODO(mpark): Once we perform a sanity check validation for
@@ -479,11 +576,7 @@ Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
         return error;
       }
 
-      convertResourceFormat(
-          operation->mutable_destroy()->mutable_volumes(),
-          POST_RESERVATION_REFINEMENT);
-
-      return None();
+      break;
     }
     case Offer::Operation::LAUNCH: {
       // TODO(mpark): Once we perform a sanity check validation for
@@ -512,21 +605,7 @@ Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
         }
       }
 
-      // Normalize resources in LAUNCH.
-      foreach (
-          TaskInfo& task, *operation->mutable_launch()->mutable_task_infos()) {
-        convertResourceFormat(
-            task.mutable_resources(),
-            POST_RESERVATION_REFINEMENT);
-
-        if (task.has_executor()) {
-          convertResourceFormat(
-              task.mutable_executor()->mutable_resources(),
-              POST_RESERVATION_REFINEMENT);
-        }
-      }
-
-      return None();
+      break;
     }
     case Offer::Operation::LAUNCH_GROUP: {
       // TODO(mpark): Once we perform a sanity check validation for
@@ -567,27 +646,7 @@ Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
         }
       }
 
-      // Normalize resources in LAUNCH_GROUP.
-      if (launchGroup->has_executor()) {
-        convertResourceFormat(
-            launchGroup->mutable_executor()->mutable_resources(),
-            POST_RESERVATION_REFINEMENT);
-      }
-
-      foreach (
-          TaskInfo& task, *launchGroup->mutable_task_group()->mutable_tasks()) {
-        convertResourceFormat(
-            task.mutable_resources(),
-            POST_RESERVATION_REFINEMENT);
-
-        if (task.has_executor()) {
-          convertResourceFormat(
-              task.mutable_executor()->mutable_resources(),
-              POST_RESERVATION_REFINEMENT);
-        }
-      }
-
-      return None();
+      break;
     }
     case Offer::Operation::CREATE_VOLUME: {
       // TODO(mpark): Once we perform a sanity check validation for
@@ -606,11 +665,7 @@ Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
         return error;
       }
 
-      convertResourceFormat(
-          operation->mutable_create_volume()->mutable_source(),
-          POST_RESERVATION_REFINEMENT);
-
-      return None();
+      break;
     }
     case Offer::Operation::DESTROY_VOLUME: {
       // TODO(mpark): Once we perform a sanity check validation for
@@ -629,11 +684,7 @@ Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
         return error;
       }
 
-      convertResourceFormat(
-          operation->mutable_destroy_volume()->mutable_volume(),
-          POST_RESERVATION_REFINEMENT);
-
-      return None();
+      break;
     }
     case Offer::Operation::CREATE_BLOCK: {
       // TODO(mpark): Once we perform a sanity check validation for
@@ -652,11 +703,7 @@ Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
         return error;
       }
 
-      convertResourceFormat(
-          operation->mutable_create_block()->mutable_source(),
-          POST_RESERVATION_REFINEMENT);
-
-      return None();
+      break;
     }
     case Offer::Operation::DESTROY_BLOCK: {
       // TODO(mpark): Once we perform a sanity check validation for
@@ -675,11 +722,7 @@ Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
         return error;
       }
 
-      convertResourceFormat(
-          operation->mutable_destroy_block()->mutable_block(),
-          POST_RESERVATION_REFINEMENT);
-
-      return None();
+      break;
     }
     case Offer::Operation::UNKNOWN: {
       // TODO(mpark): Once we perform a sanity check validation for
@@ -688,7 +731,10 @@ Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
       return Error("Unknown offer operation");
     }
   }
-  UNREACHABLE();
+
+  upgradeResources(operation);
+
+  return None();
 }
 
 


[5/5] mesos git commit: Upgraded resources that come from `protobuf::read`.

Posted by mp...@apache.org.
Upgraded resources that come from `protobuf::read`.

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


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

Branch: refs/heads/1.5.x
Commit: 0ad8f8d084d4ce19654d19b3b9f5473e943fb94a
Parents: 0dbb4cf
Author: Michael Park <mp...@apache.org>
Authored: Fri Dec 15 18:12:57 2017 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Tue Jan 2 23:04:47 2018 -0800

----------------------------------------------------------------------
 src/resource_provider/storage/provider.cpp | 20 +++++++++++++++++---
 src/slave/containerizer/mesos/paths.cpp    | 23 +++++++++++++++++++++--
 2 files changed, 38 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0ad8f8d0/src/resource_provider/storage/provider.cpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/storage/provider.cpp b/src/resource_provider/storage/provider.cpp
index ee9ac90..e260872 100644
--- a/src/resource_provider/storage/provider.cpp
+++ b/src/resource_provider/storage/provider.cpp
@@ -714,9 +714,13 @@ Future<Nothing> StorageLocalResourceProviderProcess::recoverServices()
               configPath + "': " + config.error());
         }
 
-        if (config.isSome() &&
-            getCSIPluginContainerInfo(info, containerId) == config.get()) {
-          continue;
+        if (config.isSome()) {
+          convertResourceFormat(
+              config->mutable_resources(), POST_RESERVATION_REFINEMENT);
+
+          if (getCSIPluginContainerInfo(info, containerId) == config.get()) {
+            continue;
+          }
         }
       }
     }
@@ -912,6 +916,16 @@ StorageLocalResourceProviderProcess::recoverResourceProviderState()
     }
 
     if (resourceProviderState.isSome()) {
+      foreach (
+          Operation& operation,
+          resourceProviderState->mutable_operations()) {
+        upgradeResources(operation.mutable_info());
+      }
+
+      convertResourceFormat(
+          resourceProviderState->mutable_resources(),
+          POST_RESERVATION_REFINEMENT);
+
       foreach (const Operation& operation,
                resourceProviderState->operations()) {
         Try<id::UUID> uuid = id::UUID::fromBytes(operation.uuid().value());

http://git-wip-us.apache.org/repos/asf/mesos/blob/0ad8f8d0/src/slave/containerizer/mesos/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/paths.cpp b/src/slave/containerizer/mesos/paths.cpp
index 8a188a9..d6ea618 100644
--- a/src/slave/containerizer/mesos/paths.cpp
+++ b/src/slave/containerizer/mesos/paths.cpp
@@ -20,6 +20,7 @@
 #include <stout/protobuf.hpp>
 
 #include "common/protobuf_utils.hpp"
+#include "common/resources_utils.hpp"
 
 #include "slave/containerizer/mesos/paths.hpp"
 
@@ -275,7 +276,7 @@ Result<ContainerTermination> getContainerTermination(
     return None();
   }
 
-  const Result<ContainerTermination>& termination =
+  Result<ContainerTermination> termination =
     ::protobuf::read<ContainerTermination>(path);
 
   if (termination.isError()) {
@@ -283,6 +284,11 @@ Result<ContainerTermination> getContainerTermination(
                  termination.error());
   }
 
+  if (termination.isSome()) {
+    convertResourceFormat(
+        termination->mutable_limited_resources(), POST_RESERVATION_REFINEMENT);
+  }
+
   return termination;
 }
 
@@ -323,7 +329,7 @@ Result<ContainerConfig> getContainerConfig(
     return None();
   }
 
-  const Result<ContainerConfig>& containerConfig =
+  Result<ContainerConfig> containerConfig =
     ::protobuf::read<ContainerConfig>(path);
 
   if (containerConfig.isError()) {
@@ -331,6 +337,19 @@ Result<ContainerConfig> getContainerConfig(
                  containerConfig.error());
   }
 
+  if (containerConfig.isSome()) {
+    convertResourceFormat(
+        containerConfig->mutable_executor_info()->mutable_resources(),
+        POST_RESERVATION_REFINEMENT);
+
+    convertResourceFormat(
+        containerConfig->mutable_task_info()->mutable_resources(),
+        POST_RESERVATION_REFINEMENT);
+
+    convertResourceFormat(
+        containerConfig->mutable_resources(), POST_RESERVATION_REFINEMENT);
+  }
+
   return containerConfig;
 }
 


[4/5] mesos git commit: Used the protobuf-reflection-based 'downgradeResources' utility.

Posted by mp...@apache.org.
Used the protobuf-reflection-based 'downgradeResources' utility.

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


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

Branch: refs/heads/1.5.x
Commit: 0dbb4cf7c7ce8ff6803f7e30dbd467c9837c8c6e
Parents: 6c07f4d
Author: Michael Park <mp...@apache.org>
Authored: Mon Dec 11 16:09:41 2017 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Tue Jan 2 23:04:44 2018 -0800

----------------------------------------------------------------------
 src/master/master.cpp              |  43 +++-------
 src/slave/slave.cpp                | 137 +++++++-------------------------
 src/slave/state.hpp                |  41 ++++++++--
 src/tests/slave_recovery_tests.cpp |   7 +-
 4 files changed, 82 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0dbb4cf7/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index bba70c3..282fdf8 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -5004,20 +5004,13 @@ void Master::_accept(
                       slave->info));
             }
 
-            // If the agent does not support reservation refinement,
-            // downgrade the task and executor resources to the
-            // "pre-reservation-refinement" format. This cannot fail
-            // since the master rejects attempts to create refined
-            // reservations on non-capable agents.
+            // If the agent does not support reservation refinement, downgrade
+            // the task / executor resources to the "pre-reservation-refinement"
+            // format. This cannot contain any refined reservations since
+            // the master rejects attempts to create refined reservations
+            // on non-capable agents.
             if (!slave->capabilities.reservationRefinement) {
-              TaskInfo& task = *message.mutable_task();
-
-              CHECK_SOME(downgradeResources(task.mutable_resources()));
-
-              if (task.has_executor()) {
-                CHECK_SOME(downgradeResources(
-                    task.mutable_executor()->mutable_resources()));
-              }
+              CHECK_SOME(downgradeResources(&message));
             }
 
             // TODO(bmahler): Consider updating this log message to
@@ -5208,21 +5201,11 @@ void Master::_accept(
 
         // If the agent does not support reservation refinement, downgrade
         // the task and executor resources to the "pre-reservation-refinement"
-        // format. This cannot fail since the master rejects attempts to
-        // create refined reservations on non-capable agents.
+        // format. This cannot contain any refined reservations since
+        // the master rejects attempts to create refined reservations
+        // on non-capable agents.
         if (!slave->capabilities.reservationRefinement) {
-          CHECK_SOME(downgradeResources(
-              message.mutable_executor()->mutable_resources()));
-
-          foreach (
-              TaskInfo& task, *message.mutable_task_group()->mutable_tasks()) {
-            CHECK_SOME(downgradeResources(task.mutable_resources()));
-
-            if (task.has_executor()) {
-              CHECK_SOME(downgradeResources(
-                  task.mutable_executor()->mutable_resources()));
-            }
-          }
+          CHECK_SOME(downgradeResources(&message));
         }
 
         LOG(INFO) << "Launching task group " << stringify(taskIds)
@@ -7148,9 +7131,9 @@ void Master::___reregisterSlave(
       //
       // TODO(neilc): It would probably be better to prevent the agent
       // from re-registering in this scenario.
-      Try<Nothing> result = downgradeResources(message.mutable_resources());
+      Try<Nothing> result = downgradeResources(&message);
       if (result.isError()) {
-        LOG(WARNING) << "Not sending updated checkpointed resouces "
+        LOG(WARNING) << "Not sending updated checkpointed resources "
                      << slave->checkpointedResources
                      << " with refined reservations, since agent " << *slave
                      << " is not RESERVATION_REFINEMENT-capable.";
@@ -10653,7 +10636,7 @@ void Master::_apply(
       //
       // TODO(neilc): It would probably be better to prevent the agent
       // from re-registering in this scenario.
-      Try<Nothing> result = downgradeResources(message.mutable_resources());
+      Try<Nothing> result = downgradeResources(&message);
       if (result.isError()) {
         LOG(WARNING) << "Not sending updated checkpointed resources "
                      << slave->checkpointedResources

http://git-wip-us.apache.org/repos/asf/mesos/blob/0dbb4cf7/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index e0806ab..318906c 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -1294,17 +1294,7 @@ void Slave::registered(
 
       VLOG(1) << "Checkpointing SlaveInfo to '" << path << "'";
 
-      {
-        // The `SlaveInfo.resources` does not include dynamic reservations,
-        // which means it cannot contain reservation refinements, so
-        // `downgradeResources` should always succeed.
-        SlaveInfo info_ = info;
-
-        Try<Nothing> result = downgradeResources(info_.mutable_resources());
-        CHECK_SOME(result);
-
-        CHECK_SOME(state::checkpoint(path, info_));
-      }
+      CHECK_SOME(state::checkpoint(path, info));
 
       // We start the local resource providers daemon once the agent is
       // running, so the resource providers can use the agent API.
@@ -1531,43 +1521,31 @@ void Slave::doReliableRegistration(Duration maxBackoff)
   // See MESOS-5330.
   link(master.get());
 
-  SlaveInfo slaveInfo = info;
-
-  // The `SlaveInfo.resources` does not include dynamic reservations,
-  // which means it cannot contain reservation refinements, so
-  // `downgradeResources` should always succeed.
-  Try<Nothing> result = downgradeResources(slaveInfo.mutable_resources());
-  CHECK_SOME(result);
-
-  RepeatedPtrField<Resource> checkpointedResources_ = checkpointedResources;
-
-  // If the checkpointed resources don't have reservation refinements,
-  // send them to the master in "pre-reservation-refinement" format
-  // for backward compatibility with old masters. If downgrading is
-  // not possible without losing information, send the resources in
-  // the "post-reservation-refinement" format. We ignore the return
-  // value of `downgradeResources` because for now, we send the result
-  // either way.
-  //
-  // TODO(mpark): Do something smarter with the result once something
-  // like a master capability is introduced.
-  downgradeResources(&checkpointedResources_);
-
-  if (!slaveInfo.has_id()) {
+  if (!info.has_id()) {
     // Registering for the first time.
     RegisterSlaveMessage message;
     message.set_version(MESOS_VERSION);
-    message.mutable_slave()->CopyFrom(slaveInfo);
+    message.mutable_slave()->CopyFrom(info);
 
     message.mutable_agent_capabilities()->CopyFrom(
         capabilities.toRepeatedPtrField());
 
     // Include checkpointed resources.
-    message.mutable_checkpointed_resources()->CopyFrom(checkpointedResources_);
+    message.mutable_checkpointed_resources()->CopyFrom(checkpointedResources);
 
     message.mutable_resource_version_uuid()->set_value(
         resourceVersion.toBytes());
 
+    // If the `Try` from `downgradeResources` returns an `Error`, we currently
+    // continue to send the resources to the master in a partially downgraded
+    // state. This implies that an agent with refined reservations cannot work
+    // with versions of master before reservation refinement support, which was
+    // introduced in 1.4.0.
+    //
+    // TODO(mpark): Do something smarter with the result once something
+    //              like a master capability is introduced.
+    downgradeResources(&message);
+
     send(master.get(), message);
   } else {
     // Re-registering, so send tasks running.
@@ -1578,12 +1556,12 @@ void Slave::doReliableRegistration(Duration maxBackoff)
         capabilities.toRepeatedPtrField());
 
     // Include checkpointed resources.
-    message.mutable_checkpointed_resources()->CopyFrom(checkpointedResources_);
+    message.mutable_checkpointed_resources()->CopyFrom(checkpointedResources);
 
     message.mutable_resource_version_uuid()->set_value(
         resourceVersion.toBytes());
 
-    message.mutable_slave()->CopyFrom(slaveInfo);
+    message.mutable_slave()->CopyFrom(info);
 
     foreachvalue (Framework* framework, frameworks) {
       message.add_frameworks()->CopyFrom(framework->info);
@@ -1671,29 +1649,15 @@ void Slave::doReliableRegistration(Duration maxBackoff)
       }
     }
 
-    // If the resources don't have reservation refinements, send them
-    // to the master in "pre-reservation-refinement" format for backward
-    // compatibility with old masters. If downgrading is not possible
-    // without losing information, send the resources in the
-    // "post-reservation-refinement" format. We ignore the return value of
-    // `downgradeResources` because for now, we send the result either way.
+    // If the `Try` from `downgradeResources` returns an `Error`, we currently
+    // continue to send the resources to the master in a partially downgraded
+    // state. This implies that an agent with refined reservations cannot work
+    // with versions of master before reservation refinement support, which was
+    // introduced in 1.4.0.
     //
     // TODO(mpark): Do something smarter with the result once something
     // like a master capability is introduced.
-    foreach (Task& task, *message.mutable_tasks()) {
-      downgradeResources(task.mutable_resources());
-    }
-
-    foreach (ExecutorInfo& executor, *message.mutable_executor_infos()) {
-      downgradeResources(executor.mutable_resources());
-    }
-
-    foreach (Archive::Framework& completedFramework,
-             *message.mutable_completed_frameworks()) {
-      foreach (Task& task, *completedFramework.mutable_tasks()) {
-        downgradeResources(task.mutable_resources());
-      }
-    }
+    downgradeResources(&message);
 
     CHECK_SOME(master);
     send(master.get(), message);
@@ -3683,27 +3647,10 @@ void Slave::checkpointResources(
   // we avoid a case of inconsistency between the master and the agent if
   // the agent restarts during handling of CheckpointResourcesMessage.
 
-  {
-    // If the checkpointed resources don't have reservation refinements,
-    // checkpoint them on the agent in "pre-reservation-refinement" format
-    // for backward compatibility with old agents. If downgrading is
-    // not possible without losing information, checkpoint the resources in
-    // the "post-reservation-refinement" format. We ignore the return
-    // value of `downgradeResources` because for now, we checkpoint the result
-    // either way.
-    //
-    // TODO(mpark): Do something smarter with the result once something
-    // like agent capability requirements is introduced.
-    RepeatedPtrField<Resource> newCheckpointedResources_ =
-      newCheckpointedResources;
-
-    downgradeResources(&newCheckpointedResources_);
-
-    CHECK_SOME(state::checkpoint(
-        paths::getResourcesTargetPath(metaDir),
-        newCheckpointedResources_))
-      << "Failed to checkpoint resources target " << newCheckpointedResources_;
-  }
+  CHECK_SOME(state::checkpoint(
+      paths::getResourcesTargetPath(metaDir),
+      newCheckpointedResources))
+    << "Failed to checkpoint resources target " << newCheckpointedResources;
 
   Try<Nothing> syncResult = syncCheckpointedResources(
       newCheckpointedResources);
@@ -8992,21 +8939,7 @@ void Executor::checkpointExecutor()
 
   VLOG(1) << "Checkpointing ExecutorInfo to '" << path << "'";
 
-  {
-    // If the checkpointed resources don't have reservation refinements,
-    // checkpoint them on the agent in "pre-reservation-refinement" format
-    // for backward compatibility with old agents. If downgrading is
-    // not possible without losing information, checkpoint the resources in
-    // the "post-reservation-refinement" format. We ignore the return
-    // value of `downgradeResources` because for now, we checkpoint the result
-    // either way.
-    //
-    // TODO(mpark): Do something smarter with the result once something
-    // like agent capability requirements is introduced.
-    ExecutorInfo info_ = info;
-    downgradeResources(info_.mutable_resources());
-    CHECK_SOME(state::checkpoint(path, info_));
-  }
+  CHECK_SOME(state::checkpoint(path, info));
 
   // Create the meta executor directory.
   // NOTE: This creates the 'latest' symlink in the meta directory.
@@ -9035,21 +8968,7 @@ void Executor::checkpointTask(const Task& task)
 
   VLOG(1) << "Checkpointing TaskInfo to '" << path << "'";
 
-  {
-    // If the checkpointed resources don't have reservation refinements,
-    // checkpoint them on the agent in "pre-reservation-refinement" format
-    // for backward compatibility with old agents. If downgrading is
-    // not possible without losing information, checkpoint the resources in
-    // the "post-reservation-refinement" format. We ignore the return
-    // value of `downgradeResources` because for now, we checkpoint the result
-    // either way.
-    //
-    // TODO(mpark): Do something smarter with the result once something
-    // like agent capability requirements is introduced.
-    Task task_ = task;
-    downgradeResources(task_.mutable_resources());
-    CHECK_SOME(state::checkpoint(path, task_));
-  }
+  CHECK_SOME(state::checkpoint(path, task));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0dbb4cf7/src/slave/state.hpp
----------------------------------------------------------------------
diff --git a/src/slave/state.hpp b/src/slave/state.hpp
index aaf8e5c..01abb50 100644
--- a/src/slave/state.hpp
+++ b/src/slave/state.hpp
@@ -84,20 +84,49 @@ inline Try<Nothing> checkpoint(
 }
 
 
+template <
+    typename T,
+    typename std::enable_if<
+        std::is_convertible<T*, google::protobuf::Message*>::value,
+        int>::type = 0>
+inline Try<Nothing> checkpoint(const std::string& path, T message)
+{
+  // If the `Try` from `downgradeResources` returns an `Error`, we currently
+  // continue to checkpoint the resources in a partially downgraded state.
+  // This implies that an agent with refined reservations cannot be downgraded
+  // to versions before reservation refinement support, which was introduced
+  // in 1.4.0.
+  //
+  // TODO(mpark): Do something smarter with the result once
+  // something like an agent recovery capability is introduced.
+  downgradeResources(&message);
+  return ::protobuf::write(path, message);
+}
+
+
 inline Try<Nothing> checkpoint(
     const std::string& path,
-    const google::protobuf::Message& message)
+    google::protobuf::RepeatedPtrField<Resource> resources)
 {
-  return ::protobuf::write(path, message);
+  // If the `Try` from `downgradeResources` returns an `Error`, we currently
+  // continue to checkpoint the resources in a partially downgraded state.
+  // This implies that an agent with refined reservations cannot be downgraded
+  // to versions before reservation refinement support, which was introduced
+  // in 1.4.0.
+  //
+  // TODO(mpark): Do something smarter with the result once
+  // something like an agent recovery capability is introduced.
+  downgradeResources(&resources);
+  return ::protobuf::write(path, resources);
 }
 
 
-template <typename T>
-Try<Nothing> checkpoint(
+inline Try<Nothing> checkpoint(
     const std::string& path,
-    const google::protobuf::RepeatedPtrField<T>& messages)
+    const Resources& resources)
 {
-  return ::protobuf::write(path, messages);
+  const google::protobuf::RepeatedPtrField<Resource>& messages = resources;
+  return checkpoint(path, messages);
 }
 
 }  // namespace internal {

http://git-wip-us.apache.org/repos/asf/mesos/blob/0dbb4cf7/src/tests/slave_recovery_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_recovery_tests.cpp b/src/tests/slave_recovery_tests.cpp
index bf2c5fc..387c2ff 100644
--- a/src/tests/slave_recovery_tests.cpp
+++ b/src/tests/slave_recovery_tests.cpp
@@ -138,7 +138,7 @@ TEST_F(SlaveStateTest, CheckpointProtobufMessage)
 TEST_F(SlaveStateTest, CheckpointRepeatedProtobufMessages)
 {
   // Checkpoint resources.
-  const google::protobuf::RepeatedPtrField<Resource> expected =
+  const Resources expected =
     Resources::parse("cpus:2;mem:512;cpus(role):4;mem(role):1024").get();
 
   const string file = "resources-file";
@@ -149,6 +149,11 @@ TEST_F(SlaveStateTest, CheckpointRepeatedProtobufMessages)
 
   ASSERT_SOME(actual);
 
+  // We convert the `actual` back to "post-reservation-refinement"
+  // since `state::checkpoint` always downgrades whatever resources
+  // are downgradable.
+  convertResourceFormat(&actual.get(), POST_RESERVATION_REFINEMENT);
+
   EXPECT_SOME_EQ(expected, actual);
 }
 


[3/5] mesos git commit: Modified `downgradeResources` to use protobuf reflection.

Posted by mp...@apache.org.
Modified `downgradeResources` to use protobuf reflection.

Previously, our `downgradeResources` function only took a pointer to
`RepeatedPtrField<Resource>`. This wasn't great since we needed manually
invoke `downgradeResources` on every `Resource`s field of every message.

This patch makes it such that `downgradeResources` can take any
`protobuf::Message` or `protobuf::RepeatedPtrField<Resource>`.

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


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

Branch: refs/heads/1.5.x
Commit: 6c07f4d241015fc54e677db2e15997703ed5affc
Parents: 7b1a10a
Author: Michael Park <mp...@apache.org>
Authored: Mon Nov 20 22:03:17 2017 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Tue Jan 2 23:04:41 2018 -0800

----------------------------------------------------------------------
 src/common/resources_utils.cpp | 132 +++++++++++++++++++++++++--
 src/common/resources_utils.hpp |  24 ++++-
 src/tests/resources_tests.cpp  | 172 ++++++++++++++++++++++++++++++++++++
 3 files changed, 315 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6c07f4d2/src/common/resources_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources_utils.cpp b/src/common/resources_utils.cpp
index 465bd54..bd9e025 100644
--- a/src/common/resources_utils.cpp
+++ b/src/common/resources_utils.cpp
@@ -21,6 +21,8 @@
 
 using std::vector;
 
+using google::protobuf::Descriptor;
+using google::protobuf::Message;
 using google::protobuf::RepeatedPtrField;
 
 namespace mesos {
@@ -738,24 +740,136 @@ Option<Error> validateAndUpgradeResources(Offer::Operation* operation)
 }
 
 
+Try<Nothing> downgradeResource(Resource* resource)
+{
+  CHECK(!resource->has_role());
+  CHECK(!resource->has_reservation());
+
+  if (Resources::hasRefinedReservations(*resource)) {
+    return Error("Cannot downgrade resources containing refined reservations");
+  }
+
+  convertResourceFormat(resource, PRE_RESERVATION_REFINEMENT);
+  return Nothing();
+}
+
+
 Try<Nothing> downgradeResources(RepeatedPtrField<Resource>* resources)
 {
-  foreach (const Resource& resource, *resources) {
-    CHECK(!resource.has_role());
-    CHECK(!resource.has_reservation());
+  CHECK_NOTNULL(resources);
+
+  foreach (Resource& resource, *resources) {
+    Try<Nothing> result = downgradeResource(&resource);
+    if (result.isError()) {
+      return result;
+    }
   }
 
-  foreach (const Resource& resource, *resources) {
-    if (Resources::hasRefinedReservations(resource)) {
-      return Error(
-          "Invalid resources downgrade: resource " + stringify(resource) +
-          " with refined reservations cannot be downgraded");
+  return Nothing();
+}
+
+namespace internal {
+
+// Given a protobuf descriptor `descriptor`, returns `true` if `descriptor`
+// is a `mesos::Resource`, or contains a `mesos::Resource` somewhere within.
+//
+// The provided `result` is recursively populated, where the keys are the
+// message descriptors within `descriptor`'s schema (including itself), and
+//  the corresponding value is `true` if the key contains a `mesos::Resource`.
+static void precomputeResourcesContainment(
+    const Descriptor* descriptor,
+    hashmap<const Descriptor*, bool>* result)
+{
+  CHECK_NOTNULL(descriptor);
+  CHECK_NOTNULL(result);
+
+  if (result->contains(descriptor)) {
+    return;
+  }
+
+  if (descriptor == mesos::Resource::descriptor()) {
+    result->insert({descriptor, true});
+  }
+
+  result->insert({descriptor, false});
+  for (int i = 0; i < descriptor->field_count(); ++i) {
+    // `message_type()` returns `nullptr` if the field is not a message type.
+    const Descriptor* messageDescriptor = descriptor->field(i)->message_type();
+    if (messageDescriptor == nullptr) {
+      continue;
     }
+    precomputeResourcesContainment(messageDescriptor, result);
+    result->at(descriptor) |= result->at(messageDescriptor);
   }
+}
+
 
-  convertResourceFormat(resources, PRE_RESERVATION_REFINEMENT);
+static Try<Nothing> downgradeResourcesImpl(
+    Message* message,
+    const hashmap<const Descriptor*, bool>& resourcesContainment)
+{
+  CHECK_NOTNULL(message);
+
+  const Descriptor* descriptor = message->GetDescriptor();
+
+  if (descriptor == mesos::Resource::descriptor()) {
+    return downgradeResource(static_cast<mesos::Resource*>(message));
+  }
+
+  const google::protobuf::Reflection* reflection = message->GetReflection();
+
+  for (int i = 0; i < descriptor->field_count(); ++i) {
+    const google::protobuf::FieldDescriptor* field = descriptor->field(i);
+    const Descriptor* messageDescriptor = field->message_type();
+
+    if (messageDescriptor == nullptr ||
+        !resourcesContainment.at(messageDescriptor)) {
+      continue;
+    }
+
+    if (!field->is_repeated()) {
+      if (reflection->HasField(*message, field)) {
+        Try<Nothing> result = downgradeResourcesImpl(
+            reflection->MutableMessage(message, field), resourcesContainment);
+
+        if (result.isError()) {
+          return result;
+        }
+      }
+    } else {
+      const int size = reflection->FieldSize(*message, field);
+
+      for (int j = 0; j < size; ++j) {
+        Try<Nothing> result = downgradeResourcesImpl(
+            reflection->MutableRepeatedMessage(message, field, j),
+            resourcesContainment);
+
+        if (result.isError()) {
+          return result;
+        }
+      }
+    }
+  }
 
   return Nothing();
 }
 
+}  // namespace internal {
+
+Try<Nothing> downgradeResources(Message* message)
+{
+  CHECK_NOTNULL(message);
+
+  const Descriptor* descriptor = message->GetDescriptor();
+
+  hashmap<const Descriptor*, bool> resourcesContainment;
+  internal::precomputeResourcesContainment(descriptor, &resourcesContainment);
+
+  if (!resourcesContainment.at(descriptor)) {
+    return Nothing();
+  }
+
+  return internal::downgradeResourcesImpl(message, resourcesContainment);
+}
+
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/6c07f4d2/src/common/resources_utils.hpp
----------------------------------------------------------------------
diff --git a/src/common/resources_utils.hpp b/src/common/resources_utils.hpp
index a946f0a..115efc2 100644
--- a/src/common/resources_utils.hpp
+++ b/src/common/resources_utils.hpp
@@ -169,13 +169,29 @@ void convertResourceFormat(
 Option<Error> validateAndUpgradeResources(Offer::Operation* operation);
 
 
-// Convert the given resources to the "pre-reservation-refinement" format
-// if none of the resources have refined reservations. Returns an `Error`
-// if there are any refined reservations present; in this case, the resources
-// are left in the "post-reservation-refinement" format.
+// Convert any resources contained in the given message(s)
+// to the "pre-reservation-refinement" format, if possible.
+//
+// These functions do not provide "all-or-nothing" semantics.
+// The resources are downgraded to "pre-" format until either
+//   (1) there are no more resources, or
+//   (2) a non-downgradable resource is encountered.
+//
+// For (1), `Nothing` is returned.
+// For (2), `Error` is returned, and the rest of the resources are untouched.
+//
+// This implies that components that have refined resources created cannot
+// be downgraded to a version that does not support reservation refinement.
+Try<Nothing> downgradeResource(Resource* resource);
+
+
 Try<Nothing> downgradeResources(
     google::protobuf::RepeatedPtrField<Resource>* resources);
 
+
+Try<Nothing> downgradeResources(google::protobuf::Message* message);
+
+
 } // namespace mesos {
 
 #endif // __RESOURCES_UTILS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/6c07f4d2/src/tests/resources_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resources_tests.cpp b/src/tests/resources_tests.cpp
index 64bde85..bd328b2 100644
--- a/src/tests/resources_tests.cpp
+++ b/src/tests/resources_tests.cpp
@@ -31,6 +31,8 @@
 
 #include <mesos/v1/resources.hpp>
 
+#include "common/resources_utils.hpp"
+
 #include "internal/evolve.hpp"
 
 #include "master/master.hpp"
@@ -3014,6 +3016,176 @@ TEST(ResourceFormatTest, Endpoint)
 }
 
 
+TEST(ResourceFormatTest, DowngradeWithoutResources)
+{
+  FrameworkInfo frameworkInfo = DEFAULT_FRAMEWORK_INFO;
+  EXPECT_SOME(downgradeResources(&frameworkInfo));
+  EXPECT_EQ(DEFAULT_FRAMEWORK_INFO, frameworkInfo);
+}
+
+
+TEST(ResourceFormatTest, DowngradeWithResourcesWithoutRefinedReservations)
+{
+  SlaveID slaveId;
+  slaveId.set_value("agent");
+
+  TaskInfo actual;
+  {
+    actual.set_name("task");
+    actual.mutable_task_id()->set_value("task_id");
+    actual.mutable_slave_id()->CopyFrom(slaveId);
+
+    Resource resource;
+    resource.set_name("cpus");
+    resource.set_type(Value::SCALAR);
+    resource.mutable_scalar()->set_value(555.5);
+
+    // Add "post-reservation-refinement" resources.
+
+    // Unreserved resource.
+    actual.add_resources()->CopyFrom(resource);
+
+    Resource::ReservationInfo* reservation = resource.add_reservations();
+
+    // Statically reserved resource.
+    reservation->set_type(Resource::ReservationInfo::STATIC);
+    reservation->set_role("foo");
+    actual.add_resources()->CopyFrom(resource);
+
+    // Dynamically reserved resource.
+    reservation->set_type(Resource::ReservationInfo::DYNAMIC);
+    reservation->set_role("bar");
+    reservation->set_principal("principal1");
+    actual.add_resources()->CopyFrom(resource);
+  }
+
+  TaskInfo expected;
+  {
+    expected.set_name("task");
+    expected.mutable_task_id()->set_value("task_id");
+    expected.mutable_slave_id()->CopyFrom(slaveId);
+
+    Resource resource;
+    resource.set_name("cpus");
+    resource.set_type(Value::SCALAR);
+    resource.mutable_scalar()->set_value(555.5);
+
+    // Add "pre-reservation-refinement" resources.
+
+    // Unreserved resource.
+    resource.set_role("*");
+    expected.add_resources()->CopyFrom(resource);
+
+    // Statically reserved resource.
+    resource.set_role("foo");
+    expected.add_resources()->CopyFrom(resource);
+
+    // Dynamically reserved resource.
+    resource.set_role("bar");
+    Resource::ReservationInfo* reservation = resource.mutable_reservation();
+    reservation->set_principal("principal1");
+    expected.add_resources()->CopyFrom(resource);
+  }
+
+  EXPECT_SOME(downgradeResources(&actual));
+  EXPECT_EQ(expected, actual);
+}
+
+
+TEST(ResourceFormatTest, DowngradeWithResourcesWithRefinedReservations)
+{
+  SlaveID slaveId;
+  slaveId.set_value("agent");
+
+  TaskInfo actual;
+  {
+    actual.set_name("task");
+    actual.mutable_task_id()->set_value("task_id");
+    actual.mutable_slave_id()->CopyFrom(slaveId);
+
+    Resource resource;
+    resource.set_name("cpus");
+    resource.set_type(Value::SCALAR);
+    resource.mutable_scalar()->set_value(555.5);
+
+    // Add "post-reservation-refinement" resources.
+
+    // Unreserved resource.
+    actual.add_resources()->CopyFrom(resource);
+
+    Resource::ReservationInfo* reservation = resource.add_reservations();
+
+    // Statically reserved resource.
+    reservation->set_type(Resource::ReservationInfo::STATIC);
+    reservation->set_role("foo");
+    actual.add_resources()->CopyFrom(resource);
+
+    // Dynamically reserved resource.
+    reservation->set_type(Resource::ReservationInfo::DYNAMIC);
+    reservation->set_role("bar");
+    reservation->set_principal("principal1");
+    actual.add_resources()->CopyFrom(resource);
+
+    // Dynamically refined reservation on top of dynamic reservation.
+    Resource::ReservationInfo* refinedReservation = resource.add_reservations();
+    refinedReservation->set_type(Resource::ReservationInfo::DYNAMIC);
+    refinedReservation->set_role("bar/baz");
+    refinedReservation->set_principal("principal2");
+    actual.add_resources()->CopyFrom(resource);
+  }
+
+  TaskInfo expected;
+  {
+    expected.set_name("task");
+    expected.mutable_task_id()->set_value("task_id");
+    expected.mutable_slave_id()->CopyFrom(slaveId);
+
+    Resource resource;
+    resource.set_name("cpus");
+    resource.set_type(Value::SCALAR);
+    resource.mutable_scalar()->set_value(555.5);
+
+    // Add "pre-reservation-refinement" resources.
+
+    // Unreserved resource.
+    resource.set_role("*");
+    expected.add_resources()->CopyFrom(resource);
+
+    // Statically reserved resource.
+    resource.set_role("foo");
+    expected.add_resources()->CopyFrom(resource);
+
+    // Dynamically reserved resource.
+    resource.set_role("bar");
+    Resource::ReservationInfo* reservation = resource.mutable_reservation();
+    reservation->set_principal("principal1");
+    expected.add_resources()->CopyFrom(resource);
+
+    // Add non-downgradable resources. Note that the non-downgradable
+    // resources remain in "post-reservation-refinement" format.
+
+    // Dynamically refined reservation on top of dynamic reservation.
+    resource.clear_role();
+    resource.clear_reservation();
+
+    Resource::ReservationInfo* dynamicReservation = resource.add_reservations();
+    dynamicReservation->set_type(Resource::ReservationInfo::DYNAMIC);
+    dynamicReservation->set_role("bar");
+    dynamicReservation->set_principal("principal1");
+
+    Resource::ReservationInfo* refinedReservation = resource.add_reservations();
+    refinedReservation->set_type(Resource::ReservationInfo::DYNAMIC);
+    refinedReservation->set_role("bar/baz");
+    refinedReservation->set_principal("principal2");
+
+    expected.add_resources()->CopyFrom(resource);
+  }
+
+  EXPECT_ERROR(downgradeResources(&actual));
+  EXPECT_EQ(expected, actual);
+}
+
+
 TEST(ResourcesTest, Count)
 {
   // The summation of identical shared resources is valid and


[2/5] mesos git commit: Renamed "normalize" to "upgrade" in resources adjustment operations.

Posted by mp...@apache.org.
Renamed "normalize" to "upgrade" in resources adjustment operations.

We have introduced the notion of "downgrading" resources in order to
support downgrades of various components. The opposite operation used
to be called "normalize", but it seems "upgrade" would be simpler.

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


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

Branch: refs/heads/1.5.x
Commit: 7b1a10a7888f947f3976b299172a178c6d0d505e
Parents: 12e3d7f
Author: Michael Park <mp...@apache.org>
Authored: Tue Jan 2 11:20:45 2018 -0800
Committer: Michael Park <mp...@apache.org>
Committed: Tue Jan 2 23:04:32 2018 -0800

----------------------------------------------------------------------
 src/common/resources_utils.cpp | 2 +-
 src/common/resources_utils.hpp | 2 +-
 src/master/http.cpp            | 8 ++++----
 src/master/master.cpp          | 2 +-
 4 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7b1a10a7/src/common/resources_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/resources_utils.cpp b/src/common/resources_utils.cpp
index 47ba885..465bd54 100644
--- a/src/common/resources_utils.cpp
+++ b/src/common/resources_utils.cpp
@@ -497,7 +497,7 @@ void upgradeResources(Offer::Operation* operation)
 }
 
 
-Option<Error> validateAndNormalizeResources(Offer::Operation* operation)
+Option<Error> validateAndUpgradeResources(Offer::Operation* operation)
 {
   CHECK_NOTNULL(operation);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/7b1a10a7/src/common/resources_utils.hpp
----------------------------------------------------------------------
diff --git a/src/common/resources_utils.hpp b/src/common/resources_utils.hpp
index 5b74ff2..a946f0a 100644
--- a/src/common/resources_utils.hpp
+++ b/src/common/resources_utils.hpp
@@ -166,7 +166,7 @@ void convertResourceFormat(
 // NOTE: The validate and upgrade steps are bundled because currently
 // it would be an error to validate but not upgrade or to upgrade
 // without validating.
-Option<Error> validateAndNormalizeResources(Offer::Operation* operation);
+Option<Error> validateAndUpgradeResources(Offer::Operation* operation);
 
 
 // Convert the given resources to the "pre-reservation-refinement" format

http://git-wip-us.apache.org/repos/asf/mesos/blob/7b1a10a7/src/master/http.cpp
----------------------------------------------------------------------
diff --git a/src/master/http.cpp b/src/master/http.cpp
index d7276e4..bc29faf 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -1311,7 +1311,7 @@ Future<Response> Master::Http::_createVolumes(
   operation.set_type(Offer::Operation::CREATE);
   operation.mutable_create()->mutable_volumes()->CopyFrom(volumes);
 
-  Option<Error> error = validateAndNormalizeResources(&operation);
+  Option<Error> error = validateAndUpgradeResources(&operation);
   if (error.isSome()) {
     return BadRequest(error->message);
   }
@@ -1485,7 +1485,7 @@ Future<Response> Master::Http::_destroyVolumes(
   operation.set_type(Offer::Operation::DESTROY);
   operation.mutable_destroy()->mutable_volumes()->CopyFrom(volumes);
 
-  Option<Error> error = validateAndNormalizeResources(&operation);
+  Option<Error> error = validateAndUpgradeResources(&operation);
   if (error.isSome()) {
     return BadRequest(error->message);
   }
@@ -2473,7 +2473,7 @@ Future<Response> Master::Http::_reserve(
   operation.set_type(Offer::Operation::RESERVE);
   operation.mutable_reserve()->mutable_resources()->CopyFrom(resources);
 
-  Option<Error> error = validateAndNormalizeResources(&operation);
+  Option<Error> error = validateAndUpgradeResources(&operation);
   if (error.isSome()) {
     return BadRequest(error->message);
   }
@@ -5245,7 +5245,7 @@ Future<Response> Master::Http::_unreserve(
   operation.set_type(Offer::Operation::UNRESERVE);
   operation.mutable_unreserve()->mutable_resources()->CopyFrom(resources);
 
-  Option<Error> error = validateAndNormalizeResources(&operation);
+  Option<Error> error = validateAndUpgradeResources(&operation);
   if (error.isSome()) {
     return BadRequest(error->message);
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/7b1a10a7/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 03eb178..bba70c3 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -4094,7 +4094,7 @@ void Master::accept(
     accept.clear_operations();
 
     foreach (Offer::Operation& operation, operations) {
-      Option<Error> error = validateAndNormalizeResources(&operation);
+      Option<Error> error = validateAndUpgradeResources(&operation);
       if (error.isSome()) {
         switch (operation.type()) {
           case Offer::Operation::RESERVE: