You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bb...@apache.org on 2018/03/06 16:31:19 UTC

[2/2] mesos git commit: Used proto UUID instead stout UUID internally for operation IDs.

Used proto UUID instead stout UUID internally for operation IDs.

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


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

Branch: refs/heads/master
Commit: 0d247c3887ea08b6273992218cd5899010d89fed
Parents: 4c4ee45
Author: Benjamin Bannier <be...@mesosphere.io>
Authored: Tue Mar 6 16:02:00 2018 +0100
Committer: Benjamin Bannier <bb...@apache.org>
Committed: Tue Mar 6 16:02:00 2018 +0100

----------------------------------------------------------------------
 .../resource_provider/resource_provider.proto   |   1 -
 src/common/protobuf_utils.cpp                   |  24 ++-
 src/common/protobuf_utils.hpp                   |   8 +-
 src/master/master.cpp                           | 157 +++++++------------
 src/master/master.hpp                           |  34 ++--
 src/resource_provider/manager.cpp               |  44 ++----
 src/resource_provider/message.hpp               |   4 +-
 src/resource_provider/storage/provider.cpp      |  20 ++-
 src/slave/slave.cpp                             | 146 +++++++----------
 src/slave/slave.hpp                             |  14 +-
 10 files changed, 181 insertions(+), 271 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0d247c38/include/mesos/resource_provider/resource_provider.proto
----------------------------------------------------------------------
diff --git a/include/mesos/resource_provider/resource_provider.proto b/include/mesos/resource_provider/resource_provider.proto
index e96a404..db7c751 100644
--- a/include/mesos/resource_provider/resource_provider.proto
+++ b/include/mesos/resource_provider/resource_provider.proto
@@ -23,7 +23,6 @@ package mesos.resource_provider;
 option java_package = "org.apache.mesos.resource_provider";
 option java_outer_classname = "Protos";
 
-
 message Event {
   enum Type {
     // This must be the first enum value in this list, to

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d247c38/src/common/protobuf_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/protobuf_utils.cpp b/src/common/protobuf_utils.cpp
index f516091..9c5fb97 100644
--- a/src/common/protobuf_utils.cpp
+++ b/src/common/protobuf_utils.cpp
@@ -450,7 +450,7 @@ Operation createOperation(
     const OperationStatus& latestStatus,
     const Option<FrameworkID>& frameworkId,
     const Option<SlaveID>& slaveId,
-    const Option<id::UUID>& operationUUID)
+    const Option<UUID>& operationUUID)
 {
   Operation operation;
   if (frameworkId.isSome()) {
@@ -462,7 +462,7 @@ Operation createOperation(
   operation.mutable_info()->CopyFrom(info);
   operation.mutable_latest_status()->CopyFrom(latestStatus);
   if (operationUUID.isSome()) {
-    operation.mutable_uuid()->set_value(operationUUID->toBytes());
+    operation.mutable_uuid()->CopyFrom(operationUUID.get());
   } else {
     operation.mutable_uuid()->set_value(id::UUID::random().toBytes());
   }
@@ -472,7 +472,7 @@ Operation createOperation(
 
 
 UpdateOperationStatusMessage createUpdateOperationStatusMessage(
-    const id::UUID& operationUUID,
+    const UUID& operationUUID,
     const OperationStatus& status,
     const Option<OperationStatus>& latestStatus,
     const Option<FrameworkID>& frameworkId,
@@ -489,7 +489,7 @@ UpdateOperationStatusMessage createUpdateOperationStatusMessage(
   if (latestStatus.isSome()) {
     update.mutable_latest_status()->CopyFrom(latestStatus.get());
   }
-  update.mutable_operation_uuid()->set_value(operationUUID.toBytes());
+  update.mutable_operation_uuid()->CopyFrom(operationUUID);
 
   return update;
 }
@@ -856,30 +856,30 @@ bool isSpeculativeOperation(const Offer::Operation& operation)
 
 
 RepeatedPtrField<ResourceVersionUUID> createResourceVersions(
-    const hashmap<Option<ResourceProviderID>, id::UUID>& resourceVersions)
+    const hashmap<Option<ResourceProviderID>, UUID>& resourceVersions)
 {
   RepeatedPtrField<ResourceVersionUUID> result;
 
   foreachpair (
       const Option<ResourceProviderID>& resourceProviderId,
-      const id::UUID& uuid,
+      const UUID& uuid,
       resourceVersions) {
     ResourceVersionUUID* entry = result.Add();
 
     if (resourceProviderId.isSome()) {
       entry->mutable_resource_provider_id()->CopyFrom(resourceProviderId.get());
     }
-    entry->mutable_uuid()->set_value(uuid.toBytes());
+    entry->mutable_uuid()->CopyFrom(uuid);
   }
 
   return result;
 }
 
 
-hashmap<Option<ResourceProviderID>, id::UUID> parseResourceVersions(
+hashmap<Option<ResourceProviderID>, UUID> parseResourceVersions(
     const RepeatedPtrField<ResourceVersionUUID>& resourceVersionUUIDs)
 {
-  hashmap<Option<ResourceProviderID>, id::UUID> result;
+  hashmap<Option<ResourceProviderID>, UUID> result;
 
   foreach (
       const ResourceVersionUUID& resourceVersionUUID,
@@ -891,11 +891,7 @@ hashmap<Option<ResourceProviderID>, id::UUID> parseResourceVersions(
 
     CHECK(!result.contains(resourceProviderId));
 
-    const Try<id::UUID> uuid =
-      id::UUID::fromBytes(resourceVersionUUID.uuid().value());
-    CHECK_SOME(uuid);
-
-    result.insert({std::move(resourceProviderId), std::move(uuid.get())});
+    result.insert({std::move(resourceProviderId), resourceVersionUUID.uuid()});
   }
 
   return result;

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d247c38/src/common/protobuf_utils.hpp
----------------------------------------------------------------------
diff --git a/src/common/protobuf_utils.hpp b/src/common/protobuf_utils.hpp
index 9a940db..2ef1c9a 100644
--- a/src/common/protobuf_utils.hpp
+++ b/src/common/protobuf_utils.hpp
@@ -173,11 +173,11 @@ Operation createOperation(
     const OperationStatus& latestStatus,
     const Option<FrameworkID>& frameworkId,
     const Option<SlaveID>& slaveId,
-    const Option<id::UUID>& operationUUID = None());
+    const Option<UUID>& operationUUID = None());
 
 
 UpdateOperationStatusMessage createUpdateOperationStatusMessage(
-    const id::UUID& operationUUID,
+    const UUID& operationUUID,
     const OperationStatus& status,
     const Option<OperationStatus>& latestStatus = None(),
     const Option<FrameworkID>& frameworkId = None(),
@@ -224,11 +224,11 @@ bool isSpeculativeOperation(const Offer::Operation& operation);
 
 // Helper function to pack a protobuf list of resource versions.
 google::protobuf::RepeatedPtrField<ResourceVersionUUID> createResourceVersions(
-    const hashmap<Option<ResourceProviderID>, id::UUID>& resourceVersions);
+    const hashmap<Option<ResourceProviderID>, UUID>& resourceVersions);
 
 
 // Helper function to unpack a protobuf list of resource versions.
-hashmap<Option<ResourceProviderID>, id::UUID> parseResourceVersions(
+hashmap<Option<ResourceProviderID>, UUID> parseResourceVersions(
     const google::protobuf::RepeatedPtrField<ResourceVersionUUID>&
       resourceVersionUUIDs);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d247c38/src/master/master.cpp
----------------------------------------------------------------------
diff --git a/src/master/master.cpp b/src/master/master.cpp
index 9cea7bb..e666b66 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -6296,13 +6296,9 @@ void Master::__registerSlave(
   vector<Resource> checkpointedResources = google::protobuf::convert(
       std::move(*registerSlaveMessage.mutable_checkpointed_resources()));
 
-  Option<id::UUID> resourceVersion;
+  Option<UUID> resourceVersion;
   if (registerSlaveMessage.has_resource_version_uuid()) {
-    Try<id::UUID> uuid = id::UUID::fromBytes(
-        registerSlaveMessage.resource_version_uuid().value());
-
-    CHECK_SOME(uuid);
-    resourceVersion = uuid.get();
+    resourceVersion = registerSlaveMessage.resource_version_uuid();
   }
 
   Slave* slave = new Slave(
@@ -6852,13 +6848,9 @@ void Master::__reregisterSlave(
   vector<ExecutorInfo> executorInfos = google::protobuf::convert(
       std::move(*reregisterSlaveMessage.mutable_executor_infos()));
 
-  Option<id::UUID> resourceVersion;
+  Option<UUID> resourceVersion;
   if (reregisterSlaveMessage.has_resource_version_uuid()) {
-    Try<id::UUID> uuid = id::UUID::fromBytes(
-        reregisterSlaveMessage.resource_version_uuid().value());
-
-    CHECK_SOME(uuid);
-    resourceVersion = uuid.get();
+    resourceVersion = reregisterSlaveMessage.resource_version_uuid();
   }
 
   slaves.recovered.erase(slaveInfo.id());
@@ -7000,13 +6992,9 @@ void Master::___reregisterSlave(
   const vector<SlaveInfo::Capability> agentCapabilities =
     google::protobuf::convert(reregisterSlaveMessage.agent_capabilities());
 
-  Option<id::UUID> resourceVersion;
+  Option<UUID> resourceVersion;
   if (reregisterSlaveMessage.has_resource_version_uuid()) {
-    Try<id::UUID> uuid = id::UUID::fromBytes(
-        reregisterSlaveMessage.resource_version_uuid().value());
-
-    CHECK_SOME(uuid);
-    resourceVersion = uuid.get();
+    resourceVersion = reregisterSlaveMessage.resource_version_uuid();
   }
 
   // Update our view of checkpointed agent resources for resource
@@ -7308,13 +7296,9 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
     message.has_resource_providers();
 
   if (message.has_resource_version_uuid()) {
-    hashmap<Option<ResourceProviderID>, id::UUID> resourceVersions;
+    hashmap<Option<ResourceProviderID>, UUID> resourceVersions;
 
-    const Try<id::UUID> slaveResourceVersion =
-      id::UUID::fromBytes(message.resource_version_uuid().value());
-
-    CHECK_SOME(slaveResourceVersion);
-    resourceVersions.insert({None(), slaveResourceVersion.get()});
+    resourceVersions.insert({None(), message.resource_version_uuid()});
 
     foreach (
         const UpdateSlaveMessage::ResourceProvider& resourceProvider,
@@ -7323,18 +7307,14 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
         continue;
       }
 
-      Try<id::UUID> resourceVersion =
-        id::UUID::fromBytes(resourceProvider.resource_version_uuid().value());
-
-      CHECK_SOME(resourceVersion);
-
       CHECK(resourceProvider.info().has_id());
 
       const ResourceProviderID& resourceProviderId =
         resourceProvider.info().id();
 
       CHECK(!resourceVersions.contains(resourceProviderId));
-      resourceVersions.insert({resourceProviderId, resourceVersion.get()});
+      resourceVersions.insert(
+          {resourceProviderId, resourceProvider.resource_version_uuid()});
     }
 
     updated = updated || slave->resourceVersions != resourceVersions;
@@ -7342,14 +7322,11 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
   }
 
   // Check if the known operations for this agent changed.
-  const hashset<id::UUID> knownOperations = slave->operations.keys();
-  hashset<id::UUID> receivedOperations;
+  const hashset<UUID> knownOperations = slave->operations.keys();
+  hashset<UUID> receivedOperations;
 
   foreach (const Operation& operation, message.operations().operations()) {
-    Try<id::UUID> operationUuid = id::UUID::fromBytes(operation.uuid().value());
-    CHECK_SOME(operationUuid);
-
-    receivedOperations.insert(operationUuid.get());
+    receivedOperations.insert(operation.uuid());
   }
 
   if (message.has_resource_providers()) {
@@ -7358,11 +7335,7 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
         message.resource_providers().providers()) {
       foreach (const Operation& operation,
                resourceProvider.operations().operations()) {
-        Try<id::UUID> operationUuid =
-          id::UUID::fromBytes(operation.uuid().value());
-        CHECK_SOME(operationUuid);
-
-        receivedOperations.insert(operationUuid.get());
+        receivedOperations.insert(operation.uuid());
       }
     }
   }
@@ -7379,8 +7352,8 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
   {
     Option<Resources> oldTotal;
     Option<Resources> newTotal;
-    Option<hashmap<id::UUID, Operation>> oldOperations;
-    Option<hashmap<id::UUID, Operation>> newOperations;
+    Option<hashmap<UUID, Operation>> oldOperations;
+    Option<hashmap<UUID, Operation>> newOperations;
     Option<ResourceProviderInfo> info;
   };
 
@@ -7417,7 +7390,7 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
     }
 
     // Process known operations.
-    foreachpair (const id::UUID& uuid,
+    foreachpair (const UUID& uuid,
                  Operation* operation,
                  slave->operations) {
       Result<ResourceProviderID> providerId_ =
@@ -7436,7 +7409,7 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
       // create a record for this resource provider if needed.
       if (resourceProviders[providerId].oldOperations.isNone()) {
         resourceProviders.at(providerId).oldOperations =
-          hashmap<id::UUID, Operation>();
+          hashmap<UUID, Operation>();
       }
 
       resourceProviders.at(providerId)
@@ -7449,16 +7422,11 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
 
     // Process received agent operations.
     resourceProviders[None()].newOperations =
-      hashmap<id::UUID, Operation>();
+      hashmap<UUID, Operation>();
 
     foreach (const Operation& operation, message.operations().operations()) {
-      Try<id::UUID> uuid = id::UUID::fromBytes(operation.uuid().value());
-
-      CHECK_SOME(uuid)
-        << "Could not deserialize operation id when reconciling operations";
-
       resourceProviders.at(None())
-        .newOperations->emplace(uuid.get(), operation);
+        .newOperations->emplace(operation.uuid(), operation);
     }
 
     // Process explicitly received resource provider information.
@@ -7476,17 +7444,12 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
 
         provider.newTotal = resourceProvider.total_resources();
         if (provider.newOperations.isNone()) {
-          provider.newOperations = hashmap<id::UUID, Operation>();
+          provider.newOperations = hashmap<UUID, Operation>();
         }
 
         foreach (const Operation& operation,
                  resourceProvider.operations().operations()) {
-          Try<id::UUID> uuid = id::UUID::fromBytes(operation.uuid().value());
-
-          CHECK_SOME(uuid)
-            << "Could not deserialize operation id when reconciling operations";
-
-          provider.newOperations->emplace(uuid.get(), operation);
+          provider.newOperations->emplace(operation.uuid(), operation);
         }
       }
     }
@@ -7514,9 +7477,9 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
         // acknowledgement, or the agent initiates an unrelated
         // `UpdateSlaveMessage`.
         auto extractPendingOperations =
-          [](const hashmap<id::UUID, Operation>& source,
-             hashset<id::UUID>* target) {
-            foreachpair (const id::UUID& uuid,
+          [](const hashmap<UUID, Operation>& source,
+             hashset<UUID>* target) {
+            foreachpair (const UUID& uuid,
                          const Operation& operation,
                          source) {
               if (!protobuf::isTerminalState(
@@ -7526,8 +7489,8 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
             }
           };
 
-        hashset<id::UUID> oldPendingOperations;
-        hashset<id::UUID> newPendingOperations;
+        hashset<UUID> oldPendingOperations;
+        hashset<UUID> newPendingOperations;
 
         if (provider.oldOperations.isSome()) {
           extractPendingOperations(
@@ -7539,10 +7502,10 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
               provider.newOperations.get(), &newPendingOperations);
         }
 
-        foreach (const id::UUID& uuid, newPendingOperations) {
+        foreach (const UUID& uuid, newPendingOperations) {
           CHECK(oldPendingOperations.contains(uuid))
             << "Agent tried to reconcile unknown non-terminal operation "
-            << uuid.toString();
+            << uuid;
         }
       }
 
@@ -7619,12 +7582,12 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
       hashmap<FrameworkID, Resources> usedByOperations;
 
       if (provider.newOperations.isSome()) {
-        foreachpair (const id::UUID& uuid,
+        foreachpair (const UUID& uuid,
                      const Operation& operation,
                      provider.newOperations.get()) {
           // Update to bookkeeping of operations.
           CHECK(!slave->operations.contains(uuid))
-            << "New operation " << uuid.toString() << " is already known";
+            << "New operation " << uuid << " is already known";
 
           Framework* framework = nullptr;
           if (operation.has_framework_id()) {
@@ -7657,18 +7620,17 @@ void Master::updateSlave(UpdateSlaveMessage&& message)
       // status update will be generated and the master will remove
       // the operation from its state upon receipt of that update.
       if (provider.oldOperations.isSome()) {
-        foreachkey (const id::UUID& uuid, provider.oldOperations.get()) {
+        foreachkey (const UUID& uuid, provider.oldOperations.get()) {
           if (provider.newOperations.isNone() ||
               !provider.newOperations->contains(uuid)) {
             LOG(WARNING) << "Performing explicit reconciliation with agent for"
-                         << " known operation " << uuid.toString()
+                         << " known operation " << uuid
                          << " since it was not present in original"
                          << " reconciliation message from agent";
 
             ReconcileOperationsMessage::Operation* reconcileOperation =
               reconcile.add_operations();
-            reconcileOperation->mutable_operation_uuid()->set_value(
-                uuid.toBytes());
+            reconcileOperation->mutable_operation_uuid()->CopyFrom(uuid);
 
             if (providerId.isSome()) {
               reconcileOperation->mutable_resource_provider_id()
@@ -7983,19 +7945,10 @@ void Master::updateOperationStatus(
     ? update.framework_id()
     : Option<FrameworkID>::none();
 
-  Try<id::UUID> uuid = id::UUID::fromBytes(update.operation_uuid().value());
-  if (uuid.isError()) {
-    LOG(ERROR) << "Failed to parse operation UUID for operation "
-               << "'" << update.status().operation_id() << "' for "
-               << (frameworkId.isSome()
-                     ? "framework " + stringify(frameworkId.get())
-                     : "an operator API call")
-               << " from agent " << slaveId << ": " << uuid.error();
-    return;
-  }
-
   Slave* slave = slaves.registered.get(slaveId);
 
+  const UUID& uuid = update.operation_uuid();
+
   // This is possible if the agent is marked as unreachable or gone,
   // or has initiated a graceful shutdown. In either of those cases,
   // ignore the operation status update.
@@ -8010,7 +7963,7 @@ void Master::updateOperationStatus(
   if (slave == nullptr) {
     LOG(WARNING) << "Ignoring status update for operation '"
                  << update.status().operation_id()
-                 << "' (uuid: " << uuid->toString() << ") for "
+                 << "' (uuid: " << uuid << ") for "
                  << (frameworkId.isSome()
                        ? "framework " + stringify(frameworkId.get())
                        : "an operator API call")
@@ -8018,12 +7971,11 @@ void Master::updateOperationStatus(
     return;
   }
 
-  Operation* operation = slave->getOperation(uuid.get());
+  Operation* operation = slave->getOperation(update.operation_uuid());
   if (operation == nullptr) {
     LOG(ERROR) << "Failed to find the operation '"
-               << update.status().operation_id() << "' (uuid: "
-               << uuid->toString() << ") for "
-               << (frameworkId.isSome()
+               << update.status().operation_id() << "' (uuid: " << uuid << ")"
+               << " for " << (frameworkId.isSome()
                      ? "framework " + stringify(frameworkId.get())
                      : "an operator API call")
                << " on agent " << slaveId;
@@ -8041,7 +7993,7 @@ void Master::updateOperationStatus(
     if (framework == nullptr || !framework->connected()) {
       LOG(WARNING) << "Received status update for operation '"
                    << update.status().operation_id()
-                   << "' (uuid: " << uuid->toString() << ") "
+                   << "' (uuid: " << uuid << ") "
                    << "for framework " << frameworkId.get()
                    << ", but the framework is "
                    << (framework == nullptr ? "unknown" : "disconnected");
@@ -10617,7 +10569,7 @@ void Master::_apply(
     // This must have been validated by the caller.
     CHECK(!resourceProviderId.isError());
 
-    Option<id::UUID> resourceVersion = resourceProviderId.isSome()
+    Option<UUID> resourceVersion = resourceProviderId.isSome()
       ? slave->resourceVersions.get(resourceProviderId.get())
       : slave->resourceVersions.get(None());
 
@@ -10662,8 +10614,9 @@ void Master::_apply(
         ->mutable_resource_provider_id()
         ->CopyFrom(resourceProviderId.get());
     }
-    message.mutable_resource_version_uuid()->mutable_uuid()->set_value(
-        resourceVersion->toBytes());
+
+    message.mutable_resource_version_uuid()->mutable_uuid()->CopyFrom(
+        resourceVersion.get());
 
     LOG(INFO) << "Sending operation '" << operation->info().id()
               << "' (uuid: " << operation->uuid() << ") "
@@ -11439,7 +11392,7 @@ Slave::Slave(
     vector<SlaveInfo::Capability> _capabilites,
     const Time& _registeredTime,
     vector<Resource> _checkpointedResources,
-    const Option<id::UUID>& resourceVersion,
+    const Option<UUID>& resourceVersion,
     vector<ExecutorInfo> executorInfos,
     vector<Task> tasks)
   : master(_master),
@@ -11591,10 +11544,7 @@ void Slave::removeTask(Task* task)
 
 void Slave::addOperation(Operation* operation)
 {
-  Try<id::UUID> uuid = id::UUID::fromBytes(operation->uuid().value());
-  CHECK_SOME(uuid);
-
-  operations.put(uuid.get(), operation);
+  operations.put(operation->uuid(), operation);
 
   if (!protobuf::isSpeculativeOperation(operation->info()) &&
       !protobuf::isTerminalState(operation->latest_status().state())) {
@@ -11642,11 +11592,10 @@ void Slave::recoverResources(Operation* operation)
 
 void Slave::removeOperation(Operation* operation)
 {
-  Try<id::UUID> uuid = id::UUID::fromBytes(operation->uuid().value());
-  CHECK_SOME(uuid);
+  const UUID& uuid = operation->uuid();
 
-  CHECK(operations.contains(uuid.get()))
-    << "Unknown operation (uuid: " << uuid->toString() << ")"
+  CHECK(operations.contains(uuid))
+    << "Unknown operation (uuid: " << uuid << ")"
     << " to agent " << *this;
 
   if (!protobuf::isSpeculativeOperation(operation->info()) &&
@@ -11654,11 +11603,11 @@ void Slave::removeOperation(Operation* operation)
     recoverResources(operation);
   }
 
-  operations.erase(uuid.get());
+  operations.erase(uuid);
 }
 
 
-Operation* Slave::getOperation(const id::UUID& uuid) const
+Operation* Slave::getOperation(const UUID& uuid) const
 {
   if (operations.contains(uuid)) {
     return operations.at(uuid);
@@ -11764,7 +11713,7 @@ Try<Nothing> Slave::update(
     const string& _version,
     const vector<SlaveInfo::Capability>& _capabilities,
     const Resources& _checkpointedResources,
-    const Option<id::UUID>& resourceVersion)
+    const Option<UUID>& resourceVersion)
 {
   Try<Resources> resources = applyCheckpointedResources(
       _info.resources(),

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d247c38/src/master/master.hpp
----------------------------------------------------------------------
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 1fadbe6..c9c8a96 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -124,7 +124,7 @@ Slave(Master* const _master,
         std::vector<SlaveInfo::Capability> _capabilites,
         const process::Time& _registeredTime,
         std::vector<Resource> _checkpointedResources,
-        const Option<id::UUID>& resourceVersion,
+        const Option<UUID>& resourceVersion,
         std::vector<ExecutorInfo> executorInfos = std::vector<ExecutorInfo>(),
         std::vector<Task> tasks = std::vector<Task>());
 
@@ -152,7 +152,7 @@ Slave(Master* const _master,
 
   void removeOperation(Operation* operation);
 
-  Operation* getOperation(const id::UUID& uuid) const;
+  Operation* getOperation(const UUID& uuid) const;
 
   void addOffer(Offer* offer);
 
@@ -181,7 +181,7 @@ Slave(Master* const _master,
       const std::string& _version,
       const std::vector<SlaveInfo::Capability>& _capabilites,
       const Resources& _checkpointedResources,
-      const Option<id::UUID>& resourceVersion);
+      const Option<UUID>& resourceVersion);
 
   Master* const master;
   const SlaveID id;
@@ -245,7 +245,7 @@ Slave(Master* const _master,
 
   // Pending operations or terminal operations that have
   // unacknowledged status updates on this agent.
-  hashmap<id::UUID, Operation*> operations;
+  hashmap<UUID, Operation*> operations;
 
   // Active offers on this slave.
   hashset<Offer*> offers;
@@ -278,7 +278,7 @@ Slave(Master* const _master,
 
   SlaveObserver* observer;
 
-  hashmap<Option<ResourceProviderID>, id::UUID> resourceVersions;
+  hashmap<Option<ResourceProviderID>, UUID> resourceVersions;
   hashmap<ResourceProviderID, ResourceProviderInfo> resourceProviders;
 
 private:
@@ -2508,18 +2508,17 @@ struct Framework
 
     const FrameworkID& frameworkId = operation->framework_id();
 
-    Try<id::UUID> uuid = id::UUID::fromBytes(operation->uuid().value());
-    CHECK_SOME(uuid);
+    const UUID& uuid = operation->uuid();
 
-    CHECK(!operations.contains(uuid.get()))
+    CHECK(!operations.contains(uuid))
       << "Duplicate operation '" << operation->info().id()
-      << "' (uuid: " << uuid->toString() << ") "
+      << "' (uuid: " << uuid << ") "
       << "of framework " << frameworkId;
 
-    operations.put(uuid.get(), operation);
+    operations.put(uuid, operation);
 
     if (operation->info().has_id()) {
-      operationUUIDs.put(operation->info().id(), uuid.get());
+      operationUUIDs.put(operation->info().id(), uuid);
     }
 
     if (!protobuf::isSpeculativeOperation(operation->info()) &&
@@ -2595,12 +2594,11 @@ struct Framework
 
   void removeOperation(Operation* operation)
   {
-    Try<id::UUID> uuid = id::UUID::fromBytes(operation->uuid().value());
-    CHECK_SOME(uuid);
+    const UUID& uuid = operation->uuid();
 
-    CHECK(operations.contains(uuid.get()))
+    CHECK(operations.contains(uuid))
       << "Unknown operation '" << operation->info().id()
-      << "' (uuid: " << uuid->toString() << ") "
+      << "' (uuid: " << uuid << ") "
       << "of framework " << operation->framework_id();
 
     if (!protobuf::isSpeculativeOperation(operation->info()) &&
@@ -2608,7 +2606,7 @@ struct Framework
       recoverResources(operation);
     }
 
-    operations.erase(uuid.get());
+    operations.erase(uuid);
   }
 
   const FrameworkID id() const { return info.id(); }
@@ -2865,11 +2863,11 @@ struct Framework
 
   // Pending operations or terminal operations that have
   // unacknowledged status updates.
-  hashmap<id::UUID, Operation*> operations;
+  hashmap<UUID, Operation*> operations;
 
   // The map from the framework-specified operation ID to the
   // corresponding internal operation UUID.
-  hashmap<OperationID, id::UUID> operationUUIDs;
+  hashmap<OperationID, UUID> operationUUIDs;
 
   // NOTE: For the used and offered resources below, we keep the
   // total as well as partitioned by SlaveID.

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d247c38/src/resource_provider/manager.cpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/manager.cpp b/src/resource_provider/manager.cpp
index cc195a3..a4fdb95 100644
--- a/src/resource_provider/manager.cpp
+++ b/src/resource_provider/manager.cpp
@@ -148,7 +148,7 @@ struct ResourceProvider
 
   ResourceProviderInfo info;
   HttpConnection http;
-  hashmap<id::UUID, Owned<Promise<Nothing>>> publishes;
+  hashmap<UUID, Owned<Promise<Nothing>>> publishes;
 };
 
 
@@ -545,12 +545,12 @@ Future<Nothing> ResourceProviderManagerProcess::publishResources(
   foreachpair (const ResourceProviderID& resourceProviderId,
                const Resources& resources,
                providedResources) {
-    id::UUID uuid = id::UUID::random();
+    UUID uuid;
+    uuid.set_value(id::UUID::random().toBytes());
 
     Event event;
     event.set_type(Event::PUBLISH_RESOURCES);
-    event.mutable_publish_resources()
-      ->mutable_uuid()->set_value(uuid.toBytes());
+    event.mutable_publish_resources()->mutable_uuid()->CopyFrom(uuid);
     event.mutable_publish_resources()->mutable_resources()->CopyFrom(resources);
 
     ResourceProvider* resourceProvider =
@@ -676,19 +676,9 @@ void ResourceProviderManagerProcess::updateState(
 
   // TODO(chhsiao): Report pending operations.
 
-  Try<id::UUID> resourceVersion =
-    id::UUID::fromBytes(update.resource_version_uuid().value());
-
-  CHECK_SOME(resourceVersion)
-    << "Could not deserialize version of resource provider "
-    << resourceProvider->info.id() << ": " << resourceVersion.error();
-
-  hashmap<id::UUID, Operation> operations;
+  hashmap<UUID, Operation> operations;
   foreach (const Operation &operation, update.operations()) {
-    Try<id::UUID> uuid = id::UUID::fromBytes(operation.uuid().value());
-    CHECK_SOME(uuid);
-
-    operations.put(uuid.get(), operation);
+    operations.put(operation.uuid(), operation);
   }
 
   LOG(INFO)
@@ -698,7 +688,7 @@ void ResourceProviderManagerProcess::updateState(
 
   ResourceProviderMessage::UpdateState updateState{
       resourceProvider->info,
-      resourceVersion.get(),
+      update.resource_version_uuid(),
       update.resources(),
       std::move(operations)};
 
@@ -714,38 +704,32 @@ void ResourceProviderManagerProcess::updatePublishResourcesStatus(
     ResourceProvider* resourceProvider,
     const Call::UpdatePublishResourcesStatus& update)
 {
-  Try<id::UUID> uuid = id::UUID::fromBytes(update.uuid().value());
-  if (uuid.isError()) {
-    LOG(ERROR) << "Invalid UUID in UpdatePublishResourcesStatus from resource"
-               << " provider " << resourceProvider->info.id()
-               << ": " << uuid.error();
-    return;
-  }
+  const UUID& uuid = update.uuid();
 
-  if (!resourceProvider->publishes.contains(uuid.get())) {
+  if (!resourceProvider->publishes.contains(uuid)) {
     LOG(ERROR) << "Ignoring UpdatePublishResourcesStatus from resource"
                << " provider " << resourceProvider->info.id()
-               << " because UUID " << uuid->toString() << " is unknown";
+               << " because UUID " << uuid << " is unknown";
     return;
   }
 
   LOG(INFO)
     << "Received UPDATE_PUBLISH_RESOURCES_STATUS call for PUBLISH_RESOURCES"
-    << " event " << uuid.get() << " with " << update.status()
+    << " event " << uuid << " with " << update.status()
     << " status from resource provider " << resourceProvider->info.id();
 
   if (update.status() == Call::UpdatePublishResourcesStatus::OK) {
-    resourceProvider->publishes.at(uuid.get())->set(Nothing());
+    resourceProvider->publishes.at(uuid)->set(Nothing());
   } else {
     // TODO(jieyu): Consider to include an error message in
     // 'UpdatePublishResourcesStatus' and surface that to the caller.
-    resourceProvider->publishes.at(uuid.get())->fail(
+    resourceProvider->publishes.at(uuid)->fail(
         "Failed to publish resources for resource provider " +
         stringify(resourceProvider->info.id()) + ": Received " +
         stringify(update.status()) + " status");
   }
 
-  resourceProvider->publishes.erase(uuid.get());
+  resourceProvider->publishes.erase(uuid);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d247c38/src/resource_provider/message.hpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/message.hpp b/src/resource_provider/message.hpp
index 2511af6..ae752fc 100644
--- a/src/resource_provider/message.hpp
+++ b/src/resource_provider/message.hpp
@@ -48,9 +48,9 @@ struct ResourceProviderMessage
   struct UpdateState
   {
     ResourceProviderInfo info;
-    id::UUID resourceVersion;
+    UUID resourceVersion;
     Resources totalResources;
-    hashmap<id::UUID, Operation> operations;
+    hashmap<UUID, Operation> operations;
   };
 
   struct UpdateOperationStatus

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d247c38/src/resource_provider/storage/provider.cpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/storage/provider.cpp b/src/resource_provider/storage/provider.cpp
index 63dde51..bb19ed4 100644
--- a/src/resource_provider/storage/provider.cpp
+++ b/src/resource_provider/storage/provider.cpp
@@ -1162,9 +1162,12 @@ Future<Nothing> StorageLocalResourceProviderProcess::reconcileStatusUpdates()
             ? statusUpdateManagerState.streams.at(uuid)->updates.size() : 0;
 
         for (int i = numStatuses; i < operation.statuses().size(); i++) {
+          UUID uuid_;
+          uuid_.set_value(uuid.toBytes());
+
           UpdateOperationStatusMessage update =
             protobuf::createUpdateOperationStatusMessage(
-                uuid,
+                uuid_,
                 operation.statuses(i),
                 None(),
                 operation.has_framework_id()
@@ -1433,13 +1436,16 @@ void StorageLocalResourceProviderProcess::applyOperation(
         " (expected: " + stringify(resourceVersion) + ")");
   }
 
+  UUID uuid_;
+  uuid_.set_value(uuid->toBytes());
+
   CHECK(!operations.contains(uuid.get()));
   operations[uuid.get()] = protobuf::createOperation(
       operation.info(),
       protobuf::createOperationStatus(OPERATION_PENDING, operationId),
       frameworkId,
       slaveId,
-      uuid.get());
+      uuid_);
 
   checkpointResourceProviderState();
 
@@ -2716,9 +2722,12 @@ void StorageLocalResourceProviderProcess::dropOperation(
   LOG(WARNING)
     << "Dropping operation (uuid: " << operationUuid << "): " << message;
 
+  UUID operationUuid_;
+  operationUuid_.set_value(operationUuid.toBytes());
+
   UpdateOperationStatusMessage update =
     protobuf::createUpdateOperationStatusMessage(
-       operationUuid,
+       operationUuid_,
        protobuf::createOperationStatus(
            OPERATION_DROPPED,
            operationId,
@@ -3018,10 +3027,13 @@ Try<Nothing> StorageLocalResourceProviderProcess::updateOperationStatus(
 
   checkpointResourceProviderState();
 
+  UUID operationUuid_;
+  operationUuid_.set_value(operationUuid.toBytes());
+
   // Send out the status update for the operation.
   UpdateOperationStatusMessage update =
     protobuf::createUpdateOperationStatusMessage(
-        operationUuid,
+        operationUuid_,
         operation.latest_status(),
         None(),
         operation.has_framework_id()

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d247c38/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index e2facb3..8cb6899 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -225,8 +225,10 @@ Slave::Slave(const string& id,
     resourceEstimator(_resourceEstimator),
     qosController(_qosController),
     secretGenerator(_secretGenerator),
-    authorizer(_authorizer),
-    resourceVersion(id::UUID::random()) {}
+    authorizer(_authorizer)
+    {
+      resourceVersion.set_value(id::UUID::random().toBytes());
+    }
 
 
 Slave::~Slave()
@@ -1743,8 +1745,7 @@ void Slave::doReliableRegistration(Duration maxBackoff)
     // Include checkpointed resources.
     message.mutable_checkpointed_resources()->CopyFrom(checkpointedResources);
 
-    message.mutable_resource_version_uuid()->set_value(
-        resourceVersion.toBytes());
+    message.mutable_resource_version_uuid()->CopyFrom(resourceVersion);
 
     // If the `Try` from `downgradeResources` returns an `Error`, we currently
     // continue to send the resources to the master in a partially downgraded
@@ -1768,9 +1769,7 @@ void Slave::doReliableRegistration(Duration maxBackoff)
     // Include checkpointed resources.
     message.mutable_checkpointed_resources()->CopyFrom(checkpointedResources);
 
-    message.mutable_resource_version_uuid()->set_value(
-        resourceVersion.toBytes());
-
+    message.mutable_resource_version_uuid()->CopyFrom(resourceVersion);
     message.mutable_slave()->CopyFrom(info);
 
     foreachvalue (Framework* framework, frameworks) {
@@ -2546,7 +2545,7 @@ void Slave::__run(
       }
     }
 
-    const hashmap<Option<ResourceProviderID>, id::UUID>
+    const hashmap<Option<ResourceProviderID>, UUID>
       receivedResourceVersions = protobuf::parseResourceVersions(
           {resourceVersionUuids.begin(), resourceVersionUuids.end()});
 
@@ -4269,24 +4268,15 @@ void Slave::applyOperation(const ApplyOperationMessage& message)
     ? message.framework_id()
     : Option<FrameworkID>::none();
 
-  Try<id::UUID> uuid = id::UUID::fromBytes(message.operation_uuid().value());
-  if (uuid.isError()) {
-    LOG(ERROR) << "Failed to parse operation UUID for operation "
-               << "'" << message.operation_info().id() << "' from "
-               << (frameworkId.isSome()
-                     ? "framework " + stringify(frameworkId.get())
-                     : "an operator API call")
-               << ": " << uuid.error();
-    return;
-  }
-
   Result<ResourceProviderID> resourceProviderId =
     getResourceProviderId(message.operation_info());
 
+  const UUID& uuid = message.operation_uuid();
+
   if (resourceProviderId.isError()) {
     LOG(ERROR) << "Failed to get the resource provider ID of operation "
                << "'" << message.operation_info().id() << "' "
-               << "(uuid: " << uuid->toString() << ") from "
+               << "(uuid: " << uuid << ") from "
                << (frameworkId.isSome()
                      ? "framework " + stringify(frameworkId.get())
                      : "an operator API call")
@@ -4300,7 +4290,7 @@ void Slave::applyOperation(const ApplyOperationMessage& message)
           protobuf::createOperationStatus(OPERATION_PENDING),
           frameworkId,
           info.id(),
-          uuid.get()));
+          uuid));
 
   addOperation(operation);
 
@@ -4330,7 +4320,7 @@ void Slave::applyOperation(const ApplyOperationMessage& message)
 
   UpdateOperationStatusMessage update =
     protobuf::createUpdateOperationStatusMessage(
-        uuid.get(),
+        uuid,
         protobuf::createOperationStatus(OPERATION_FINISHED),
         None(),
         frameworkId,
@@ -4358,16 +4348,12 @@ void Slave::reconcileOperations(const ReconcileOperationsMessage& message)
       continue;
     }
 
-    Try<id::UUID> operationUuid =
-      id::UUID::fromBytes(operation.operation_uuid().value());
-    CHECK_SOME(operationUuid);
-
     // The master reconciles when it notices that an operation is missing from
     // an `UpdateSlaveMessage`. If we cannot find an operation in the agent
     // state, we send an update to inform the master. If we do find the
     // operation, then the master and agent state are consistent and we do not
     // need to do anything.
-    Operation* storedOperation = getOperation(operationUuid.get());
+    Operation* storedOperation = getOperation(operation.operation_uuid());
     if (storedOperation == nullptr) {
       // For agent default resources, we send best-effort operation status
       // updates to the master. This is satisfactory because a dropped message
@@ -4375,7 +4361,7 @@ void Slave::reconcileOperations(const ReconcileOperationsMessage& message)
       // `UpdateSlaveMessage` would be sent with pending operations.
       UpdateOperationStatusMessage update =
         protobuf::createUpdateOperationStatusMessage(
-            operationUuid.get(),
+            operation.operation_uuid(),
             protobuf::createOperationStatus(OPERATION_DROPPED),
             None(),
             None(),
@@ -4428,6 +4414,9 @@ void Slave::statusUpdateAcknowledgement(
     }
   }
 
+  UUID uuid_;
+  uuid_.set_value(uuid);
+
   taskStatusUpdateManager->acknowledgement(
       taskId, frameworkId, id::UUID::fromBytes(uuid).get())
     .onAny(defer(self(),
@@ -4435,7 +4424,7 @@ void Slave::statusUpdateAcknowledgement(
                  lambda::_1,
                  taskId,
                  frameworkId,
-                 id::UUID::fromBytes(uuid).get()));
+                 uuid_));
 }
 
 
@@ -4443,7 +4432,7 @@ void Slave::_statusUpdateAcknowledgement(
     const Future<bool>& future,
     const TaskID& taskId,
     const FrameworkID& frameworkId,
-    const id::UUID& uuid)
+    const UUID& uuid)
 {
   // The future could fail if this is a duplicate status update acknowledgement.
   if (!future.isReady()) {
@@ -4513,11 +4502,7 @@ void Slave::operationStatusAcknowledgement(
     const UPID& from,
     const AcknowledgeOperationStatusMessage& acknowledgement)
 {
-  Try<id::UUID> operationUuid =
-    id::UUID::fromBytes(acknowledgement.operation_uuid().value());
-  CHECK_SOME(operationUuid);
-
-  Operation* operation = getOperation(operationUuid.get());
+  Operation* operation = getOperation(acknowledgement.operation_uuid());
   if (operation != nullptr) {
     resourceProviderManager.acknowledgeOperationStatus(acknowledgement);
 
@@ -7506,7 +7491,7 @@ UpdateSlaveMessage Slave::generateResourceProviderUpdate() const
   UpdateSlaveMessage message;
   message.mutable_slave_id()->CopyFrom(info.id());
   message.set_update_oversubscribed_resources(false);
-  message.mutable_resource_version_uuid()->set_value(resourceVersion.toBytes());
+  message.mutable_resource_version_uuid()->CopyFrom(resourceVersion);
   message.mutable_operations();
 
   foreachvalue (const Operation* operation, operations) {
@@ -7527,8 +7512,8 @@ UpdateSlaveMessage Slave::generateResourceProviderUpdate() const
         resourceProvider->info);
     provider->mutable_total_resources()->CopyFrom(
         resourceProvider->totalResources);
-    provider->mutable_resource_version_uuid()->set_value(
-        resourceProvider->resourceVersion.toBytes());
+    provider->mutable_resource_version_uuid()->CopyFrom(
+        resourceProvider->resourceVersion);
 
     provider->mutable_operations();
 
@@ -7623,10 +7608,8 @@ void Slave::handleResourceProviderMessage(
         // We only update operations which are not contained in both
         // the known and just received sets. All other operations will
         // be updated via relayed operation status updates.
-        const hashset<id::UUID> knownUuids =
-          resourceProvider->operations.keys();
-
-        const hashset<id::UUID> receivedUuids = updateState.operations.keys();
+        const hashset<UUID> knownUuids = resourceProvider->operations.keys();
+        const hashset<UUID> receivedUuids = updateState.operations.keys();
 
         // Handle operations known to the agent but not reported by
         // the resource provider. These could be operations where the
@@ -7636,16 +7619,14 @@ void Slave::handleResourceProviderMessage(
         //
         // NOTE: We do not mutate operations statuses here; this would
         // be the responsibility of a operation status update handler.
-        hashset<id::UUID> disappearedOperations;
-        std::set_difference(
-            knownUuids.begin(),
-            knownUuids.end(),
-            receivedUuids.begin(),
-            receivedUuids.end(),
-            std::inserter(
-                disappearedOperations, disappearedOperations.begin()));
-
-        foreach (const id::UUID& uuid, disappearedOperations) {
+        hashset<UUID> disappearedOperations;
+        foreach (const UUID& knownUuid, knownUuids) {
+          if (!receivedUuids.contains(knownUuid)) {
+            disappearedOperations.insert(knownUuid);
+          }
+        }
+
+        foreach (const UUID& uuid, disappearedOperations) {
           // TODO(bbannier): Instead of simply dropping an operation
           // with `removeOperation` here we should instead send a
           // `Reconcile` message with a failed state to the resource
@@ -7658,15 +7639,14 @@ void Slave::handleResourceProviderMessage(
         // Handle operations known to the resource provider but not
         // the agent. This can happen if the agent failed over and the
         // resource provider reregistered.
-        hashset<id::UUID> reappearedOperations;
-        std::set_difference(
-            receivedUuids.begin(),
-            receivedUuids.end(),
-            knownUuids.begin(),
-            knownUuids.end(),
-            std::inserter(reappearedOperations, reappearedOperations.begin()));
-
-        foreach (const id::UUID& uuid, reappearedOperations) {
+        hashset<UUID> reappearedOperations;
+        foreach (const UUID& receivedUuid, receivedUuids) {
+          if (!knownUuids.contains(receivedUuid)) {
+            reappearedOperations.insert(receivedUuid);
+          }
+        }
+
+        foreach (const UUID& uuid, reappearedOperations) {
           // Start tracking this operation.
           //
           // NOTE: We do not need to update total resources here as its
@@ -7707,11 +7687,9 @@ void Slave::handleResourceProviderMessage(
       const UpdateOperationStatusMessage& update =
         message->updateOperationStatus->update;
 
-      Try<id::UUID> operationUUID =
-        id::UUID::fromBytes(update.operation_uuid().value());
-      CHECK_SOME(operationUUID);
+      const UUID& operationUUID = update.operation_uuid();
 
-      Operation* operation = getOperation(operationUUID.get());
+      Operation* operation = getOperation(operationUUID);
 
       if (operation != nullptr) {
         // The agent might not know about the operation in the
@@ -7750,7 +7728,7 @@ void Slave::handleResourceProviderMessage(
             << (update.status().has_operation_id()
                  ? " '" + stringify(update.status().operation_id()) + "'"
                  : " with no ID")
-            << " (operation_uuid: " << operationUUID->toString() << ")"
+            << " (operation_uuid: " << operationUUID << ")"
             << (update.has_framework_id()
                  ? " for framework " + stringify(update.framework_id())
                  : " for an operator API call")
@@ -7764,7 +7742,7 @@ void Slave::handleResourceProviderMessage(
             << (update.status().has_operation_id()
                  ? " '" + stringify(update.status().operation_id()) + "'"
                  : " with no ID")
-            << " (operation_uuid: " << operationUUID->toString() << ")"
+            << " (operation_uuid: " << operationUUID << ")"
             << (update.has_framework_id()
                  ? " for framework " + stringify(update.framework_id())
                  : " for an operator API call");
@@ -7841,10 +7819,7 @@ void Slave::handleResourceProviderMessage(
 
 void Slave::addOperation(Operation* operation)
 {
-  Try<id::UUID> uuid = id::UUID::fromBytes(operation->uuid().value());
-  CHECK_SOME(uuid);
-
-  operations.put(uuid.get(), operation);
+  operations.put(operation->uuid(), operation);
 
   Result<ResourceProviderID> resourceProviderId =
     getResourceProviderId(operation->info());
@@ -7965,8 +7940,7 @@ void Slave::updateOperation(
 
 void Slave::removeOperation(Operation* operation)
 {
-  Try<id::UUID> uuid = id::UUID::fromBytes(operation->uuid().value());
-  CHECK_SOME(uuid);
+  const UUID& uuid = operation->uuid();
 
   Result<ResourceProviderID> resourceProviderId =
     getResourceProviderId(operation->info());
@@ -7984,15 +7958,15 @@ void Slave::removeOperation(Operation* operation)
     resourceProvider->removeOperation(operation);
   }
 
-  CHECK(operations.contains(uuid.get()))
-    << "Unknown operation (uuid: " << uuid->toString() << ")";
+  CHECK(operations.contains(uuid))
+    << "Unknown operation (uuid: " << uuid << ")";
 
-  operations.erase(uuid.get());
+  operations.erase(uuid);
   delete operation;
 }
 
 
-Operation* Slave::getOperation(const id::UUID& uuid) const
+Operation* Slave::getOperation(const UUID& uuid) const
 {
   if (operations.contains(uuid)) {
     return operations.at(uuid);
@@ -9775,25 +9749,23 @@ Resources Executor::allocatedResources() const
 
 void ResourceProvider::addOperation(Operation* operation)
 {
-  Try<id::UUID> uuid = id::UUID::fromBytes(operation->uuid().value());
-  CHECK_SOME(uuid);
+  const UUID& uuid = operation->uuid();
 
-  CHECK(!operations.contains(uuid.get()))
-    << "Operation (uuid: " << uuid->toString() << ") already exists";
+  CHECK(!operations.contains(uuid))
+    << "Operation (uuid: " << uuid << ") already exists";
 
-  operations.put(uuid.get(), operation);
+  operations.put(uuid, operation);
 }
 
 
 void ResourceProvider::removeOperation(Operation* operation)
 {
-  Try<id::UUID> uuid = id::UUID::fromBytes(operation->uuid().value());
-  CHECK_SOME(uuid);
+  const UUID& uuid = operation->uuid();
 
-  CHECK(operations.contains(uuid.get()))
-    << "Unknown operation (uuid: " << uuid->toString() << ")";
+  CHECK(operations.contains(uuid))
+    << "Unknown operation (uuid: " << uuid << ")";
 
-  operations.erase(uuid.get());
+  operations.erase(uuid);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0d247c38/src/slave/slave.hpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.hpp b/src/slave/slave.hpp
index 42c3ebc..f161369 100644
--- a/src/slave/slave.hpp
+++ b/src/slave/slave.hpp
@@ -355,7 +355,7 @@ public:
       const process::Future<bool>& future,
       const TaskID& taskId,
       const FrameworkID& frameworkId,
-      const id::UUID& uuid);
+      const UUID& uuid);
 
   void operationStatusAcknowledgement(
       const process::UPID& from,
@@ -649,7 +649,7 @@ private:
 
   void removeOperation(Operation* operation);
 
-  Operation* getOperation(const id::UUID& uuid) const;
+  Operation* getOperation(const UUID& uuid) const;
 
   void addResourceProvider(ResourceProvider* resourceProvider);
   ResourceProvider* getResourceProvider(const ResourceProviderID& id) const;
@@ -824,14 +824,14 @@ private:
   // different resource version UUID than that it maintains, because
   // this means the operation is operating on resources that might
   // have already been invalidated.
-  id::UUID resourceVersion;
+  UUID resourceVersion;
 
   // Keeps track of the following:
   // (1) Pending operations for resources from the agent.
   // (2) Pending operations or terminal operations that have
   //     unacknowledged status updates for resource provider
   //     provided resources.
-  hashmap<id::UUID, Operation*> operations;
+  hashmap<UUID, Operation*> operations;
 };
 
 
@@ -1160,7 +1160,7 @@ struct ResourceProvider
   ResourceProvider(
       const ResourceProviderInfo& _info,
       const Resources& _totalResources,
-      const id::UUID& _resourceVersion)
+      const UUID& _resourceVersion)
     : info(_info),
       totalResources(_totalResources),
       resourceVersion(_resourceVersion) {}
@@ -1182,11 +1182,11 @@ struct ResourceProvider
   // different resource version UUID than that it maintains, because
   // this means the operation is operating on resources that might
   // have already been invalidated.
-  id::UUID resourceVersion;
+  UUID resourceVersion;
 
   // Pending operations or terminal operations that have
   // unacknowledged status updates.
-  hashmap<id::UUID, Operation*> operations;
+  hashmap<UUID, Operation*> operations;
 };