You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gr...@apache.org on 2017/12/08 08:04:22 UTC

[1/3] mesos git commit: Renamed the resource provider PUBLISH event to PUBLISH_RESOURCES.

Repository: mesos
Updated Branches:
  refs/heads/master 884459168 -> c78496fd5


Renamed the resource provider PUBLISH event to PUBLISH_RESOURCES.

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


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

Branch: refs/heads/master
Commit: 2975d7e6bae6dc0b1ddb730ef3da2938b4fc5e01
Parents: 8844591
Author: Greg Mann <gr...@gmail.com>
Authored: Wed Dec 6 15:28:14 2017 -0800
Committer: Greg Mann <gr...@gmail.com>
Committed: Thu Dec 7 23:06:14 2017 -0800

----------------------------------------------------------------------
 .../resource_provider/resource_provider.proto   |  6 ++--
 .../resource_provider/resource_provider.proto   |  6 ++--
 src/resource_provider/manager.cpp               | 17 ++++++-----
 src/resource_provider/manager.hpp               |  2 +-
 src/resource_provider/storage/provider.cpp      | 11 ++++---
 src/slave/slave.cpp                             |  2 +-
 src/tests/mesos.hpp                             | 14 +++++----
 src/tests/resource_provider_manager_tests.cpp   | 32 ++++++++++----------
 src/tests/slave_tests.cpp                       |  6 ++--
 9 files changed, 50 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2975d7e6/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 d2b9c79..9c69813 100644
--- a/include/mesos/resource_provider/resource_provider.proto
+++ b/include/mesos/resource_provider/resource_provider.proto
@@ -34,7 +34,7 @@ message Event {
 
     SUBSCRIBED = 1;                  // See 'Subscribed' below.
     OPERATION = 2;                   // See 'Operation' below.
-    PUBLISH = 3;                     // See 'Publish' below.
+    PUBLISH_RESOURCES = 3;           // See 'PublishResources' below.
     ACKNOWLEDGE_OFFER_OPERATION = 4; // See 'AcknowledgeOfferOperation' below.
     RECONCILE_OFFER_OPERATIONS = 5;  // See 'ReconcileOfferOperations' below.
   }
@@ -73,7 +73,7 @@ message Event {
 
   // Received when the master wants to launch a task using resources
   // of this resource provider.
-  message Publish {
+  message PublishResources {
     required bytes uuid = 1;
     repeated Resource resources = 2;
   }
@@ -99,7 +99,7 @@ message Event {
   optional Type type = 1;
   optional Subscribed subscribed = 2;
   optional Operation operation = 3;
-  optional Publish publish = 4;
+  optional PublishResources publish_resources = 4;
   optional AcknowledgeOfferOperation acknowledge_offer_operation = 5;
   optional ReconcileOfferOperations reconcile_offer_operations = 6;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/2975d7e6/include/mesos/v1/resource_provider/resource_provider.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/resource_provider/resource_provider.proto b/include/mesos/v1/resource_provider/resource_provider.proto
index 0c23e91..8340316 100644
--- a/include/mesos/v1/resource_provider/resource_provider.proto
+++ b/include/mesos/v1/resource_provider/resource_provider.proto
@@ -34,7 +34,7 @@ message Event {
 
     SUBSCRIBED = 1;                  // See 'Subscribed' below.
     OPERATION = 2;                   // See 'Operation' below.
-    PUBLISH = 3;                     // See 'Publish' below.
+    PUBLISH_RESOURCES = 3;           // See 'PublishResources' below.
     ACKNOWLEDGE_OFFER_OPERATION = 4; // See 'AcknowledgeOfferOperation' below.
     RECONCILE_OFFER_OPERATIONS = 5;  // See 'ReconcileOfferOperations' below.
   }
@@ -73,7 +73,7 @@ message Event {
 
   // Received when the master wants to launch a task using resources
   // of this resource provider.
-  message Publish {
+  message PublishResources {
     required bytes uuid = 1;
     repeated Resource resources = 2;
   }
@@ -99,7 +99,7 @@ message Event {
   optional Type type = 1;
   optional Subscribed subscribed = 2;
   optional Operation operation = 3;
-  optional Publish publish = 4;
+  optional PublishResources publish_resources = 4;
   optional AcknowledgeOfferOperation acknowledge_offer_operation = 5;
   optional ReconcileOfferOperations reconcile_offer_operations = 6;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/2975d7e6/src/resource_provider/manager.cpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/manager.cpp b/src/resource_provider/manager.cpp
index e75d528..c12fd38 100644
--- a/src/resource_provider/manager.cpp
+++ b/src/resource_provider/manager.cpp
@@ -163,7 +163,7 @@ public:
 
   void reconcileOfferOperations(const ReconcileOfferOperationsMessage& message);
 
-  Future<Nothing> publish(const Resources& resources);
+  Future<Nothing> publishResources(const Resources& resources);
 
   Queue<ResourceProviderMessage> messages;
 
@@ -496,7 +496,7 @@ void ResourceProviderManagerProcess::reconcileOfferOperations(
 }
 
 
-Future<Nothing> ResourceProviderManagerProcess::publish(
+Future<Nothing> ResourceProviderManagerProcess::publishResources(
     const Resources& resources)
 {
   hashmap<ResourceProviderID, Resources> providedResources;
@@ -531,9 +531,9 @@ Future<Nothing> ResourceProviderManagerProcess::publish(
     UUID uuid = UUID::random();
 
     Event event;
-    event.set_type(Event::PUBLISH);
-    event.mutable_publish()->set_uuid(uuid.toBytes());
-    event.mutable_publish()->mutable_resources()->CopyFrom(resources);
+    event.set_type(Event::PUBLISH_RESOURCES);
+    event.mutable_publish_resources()->set_uuid(uuid.toBytes());
+    event.mutable_publish_resources()->mutable_resources()->CopyFrom(resources);
 
     ResourceProvider* resourceProvider =
       resourceProviders.subscribed.at(resourceProviderId).get();
@@ -544,7 +544,7 @@ Future<Nothing> ResourceProviderManagerProcess::publish(
 
     if (!resourceProvider->http.send(event)) {
       return Failure(
-          "Failed to send PUBLISH event to resource provider " +
+          "Failed to send PUBLISH_RESOURCES event to resource provider " +
           stringify(resourceProviderId) + ": connection closed");
     }
 
@@ -765,11 +765,12 @@ void ResourceProviderManager::reconcileOfferOperations(
 }
 
 
-Future<Nothing> ResourceProviderManager::publish(const Resources& resources)
+Future<Nothing> ResourceProviderManager::publishResources(
+    const Resources& resources)
 {
   return dispatch(
       process.get(),
-      &ResourceProviderManagerProcess::publish,
+      &ResourceProviderManagerProcess::publishResources,
       resources);
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/2975d7e6/src/resource_provider/manager.hpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/manager.hpp b/src/resource_provider/manager.hpp
index e94b836..c5c2d52 100644
--- a/src/resource_provider/manager.hpp
+++ b/src/resource_provider/manager.hpp
@@ -64,7 +64,7 @@ public:
       const ReconcileOfferOperationsMessage& message) const;
 
   // Ensure that the resources are ready for use.
-  process::Future<Nothing> publish(const Resources& resources);
+  process::Future<Nothing> publishResources(const Resources& resources);
 
   // Returns a stream of messages from the resource provider manager.
   process::Queue<ResourceProviderMessage> messages() const;

http://git-wip-us.apache.org/repos/asf/mesos/blob/2975d7e6/src/resource_provider/storage/provider.cpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/storage/provider.cpp b/src/resource_provider/storage/provider.cpp
index a029421..d43c8bd 100644
--- a/src/resource_provider/storage/provider.cpp
+++ b/src/resource_provider/storage/provider.cpp
@@ -330,7 +330,7 @@ private:
   // Functions for received events.
   void subscribed(const Event::Subscribed& subscribed);
   void operation(const Event::Operation& operation);
-  void publish(const Event::Publish& publish);
+  void publishResources(const Event::PublishResources& publish);
   void acknowledgeOfferOperation(
       const Event::AcknowledgeOfferOperation& acknowledge);
   void reconcileOfferOperations(
@@ -453,9 +453,9 @@ void StorageLocalResourceProviderProcess::received(const Event& event)
       operation(event.operation());
       break;
     }
-    case Event::PUBLISH: {
-      CHECK(event.has_publish());
-      publish(event.publish());
+    case Event::PUBLISH_RESOURCES: {
+      CHECK(event.has_publish_resources());
+      publishResources(event.publish_resources());
       break;
     }
     case Event::ACKNOWLEDGE_OFFER_OPERATION: {
@@ -997,7 +997,8 @@ void StorageLocalResourceProviderProcess::operation(
 }
 
 
-void StorageLocalResourceProviderProcess::publish(const Event::Publish& publish)
+void StorageLocalResourceProviderProcess::publishResources(
+    const Event::PublishResources& publish)
 {
   Option<Error> error;
   hashset<string> volumeIds;

http://git-wip-us.apache.org/repos/asf/mesos/blob/2975d7e6/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 1bdc9d8..98370f9 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -7363,7 +7363,7 @@ Future<Nothing> Slave::publishResources(
     resources += additionalResources.get();
   }
 
-  return resourceProviderManager.publish(resources);
+  return resourceProviderManager.publishResources(resources);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/2975d7e6/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 76fbd90..3b9403c 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -2850,7 +2850,7 @@ public:
               Source>::operationDefault));
     EXPECT_CALL(*this, operation(_)).WillRepeatedly(DoDefault());
 
-    ON_CALL(*this, publish(_))
+    ON_CALL(*this, publishResources(_))
       .WillByDefault(Invoke(
           this,
           &MockResourceProvider<
@@ -2864,14 +2864,16 @@ public:
               OfferOperationState,
               Operation,
               Source>::publishDefault));
-    EXPECT_CALL(*this, publish(_)).WillRepeatedly(DoDefault());
+    EXPECT_CALL(*this, publishResources(_)).WillRepeatedly(DoDefault());
   }
 
   MOCK_METHOD0_T(connected, void());
   MOCK_METHOD0_T(disconnected, void());
   MOCK_METHOD1_T(subscribed, void(const typename Event::Subscribed&));
   MOCK_METHOD1_T(operation, void(const typename Event::Operation&));
-  MOCK_METHOD1_T(publish, void(const typename Event::Publish&));
+  MOCK_METHOD1_T(
+      publishResources,
+      void(const typename Event::PublishResources&));
   MOCK_METHOD1_T(
       acknowledgeOfferOperation,
       void(const typename Event::AcknowledgeOfferOperation&));
@@ -2892,8 +2894,8 @@ public:
         case Event::OPERATION:
           operation(event.operation());
           break;
-        case Event::PUBLISH:
-          publish(event.publish());
+        case Event::PUBLISH_RESOURCES:
+          publishResources(event.publish_resources());
           break;
         case Event::ACKNOWLEDGE_OFFER_OPERATION:
           acknowledgeOfferOperation(event.acknowledge_offer_operation());
@@ -3075,7 +3077,7 @@ public:
     driver->send(call);
   }
 
-  void publishDefault(const typename Event::Publish& publish)
+  void publishDefault(const typename Event::PublishResources& publish)
   {
     CHECK(info.has_id());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/2975d7e6/src/tests/resource_provider_manager_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resource_provider_manager_tests.cpp b/src/tests/resource_provider_manager_tests.cpp
index a4c19ca..117003d 100644
--- a/src/tests/resource_provider_manager_tests.cpp
+++ b/src/tests/resource_provider_manager_tests.cpp
@@ -460,9 +460,9 @@ TEST_P(ResourceProviderManagerHttpApiTest, UpdateOfferOperationStatus)
 
 
 // This test verifies that the pending future returned by
-// `ResourceProviderManager::publish()` becomes ready when the manager
+// `ResourceProviderManager::publishResources()` becomes ready when the manager
 // receives an publish status update with an `OK` status.
-TEST_P(ResourceProviderManagerHttpApiTest, PublishSuccess)
+TEST_P(ResourceProviderManagerHttpApiTest, PublishResourcesSuccess)
 {
   const ContentType contentType = GetParam();
 
@@ -531,19 +531,19 @@ TEST_P(ResourceProviderManagerHttpApiTest, PublishSuccess)
       resource.mutable_provider_id()->CopyFrom(resourceProviderId.get());
     }
 
-    Future<Nothing> published = manager.publish(devolve(resources));
+    Future<Nothing> published = manager.publishResources(devolve(resources));
 
     Future<Result<Event>> event = responseDecoder->read();
     AWAIT_READY(event);
     ASSERT_SOME(event.get());
-    ASSERT_EQ(Event::PUBLISH, event->get().type());
+    ASSERT_EQ(Event::PUBLISH_RESOURCES, event->get().type());
 
     Call call;
     call.set_type(Call::UPDATE_PUBLISH_STATUS);
     call.mutable_resource_provider_id()->CopyFrom(resourceProviderId.get());
 
     Call::UpdatePublishStatus* update = call.mutable_update_publish_status();
-    update->set_uuid(event->get().publish().uuid());
+    update->set_uuid(event->get().publish_resources().uuid());
     update->set_status(Call::UpdatePublishStatus::OK);
 
     http::Request request;
@@ -565,9 +565,9 @@ TEST_P(ResourceProviderManagerHttpApiTest, PublishSuccess)
 
 
 // This test verifies that the pending future returned by
-// `ResourceProviderManager::publish()` becomes failed when the manager
+// `ResourceProviderManager::publishResources()` becomes failed when the manager
 // receives an publish status update with a `FAILED` status.
-TEST_P(ResourceProviderManagerHttpApiTest, PublishFailure)
+TEST_P(ResourceProviderManagerHttpApiTest, PublishResourcesFailure)
 {
   const ContentType contentType = GetParam();
 
@@ -636,19 +636,19 @@ TEST_P(ResourceProviderManagerHttpApiTest, PublishFailure)
       resource.mutable_provider_id()->CopyFrom(resourceProviderId.get());
     }
 
-    Future<Nothing> published = manager.publish(devolve(resources));
+    Future<Nothing> published = manager.publishResources(devolve(resources));
 
     Future<Result<Event>> event = responseDecoder->read();
     AWAIT_READY(event);
     ASSERT_SOME(event.get());
-    ASSERT_EQ(Event::PUBLISH, event->get().type());
+    ASSERT_EQ(Event::PUBLISH_RESOURCES, event->get().type());
 
     Call call;
     call.set_type(Call::UPDATE_PUBLISH_STATUS);
     call.mutable_resource_provider_id()->CopyFrom(resourceProviderId.get());
 
     Call::UpdatePublishStatus* update = call.mutable_update_publish_status();
-    update->set_uuid(event->get().publish().uuid());
+    update->set_uuid(event->get().publish_resources().uuid());
     update->set_status(Call::UpdatePublishStatus::FAILED);
 
     http::Request request;
@@ -670,9 +670,9 @@ TEST_P(ResourceProviderManagerHttpApiTest, PublishFailure)
 
 
 // This test verifies that the pending future returned by
-// `ResourceProviderManager::publish()` becomes failed when the resource
-// provider is disconnected.
-TEST_P(ResourceProviderManagerHttpApiTest, PublishDisconnected)
+// `ResourceProviderManager::publishResources()` becomes failed when the
+// resource provider is disconnected.
+TEST_P(ResourceProviderManagerHttpApiTest, PublishResourcesDisconnected)
 {
   const ContentType contentType = GetParam();
 
@@ -727,7 +727,7 @@ TEST_P(ResourceProviderManagerHttpApiTest, PublishDisconnected)
     EXPECT_FALSE(resourceProviderId->value().empty());
   }
 
-  // Then, close the connection after receiving a publish event.
+  // Then, close the connection after receiving a publish resources event.
   {
     vector<v1::Resource> resources =
       v1::Resources::fromString("disk:4").get();
@@ -735,12 +735,12 @@ TEST_P(ResourceProviderManagerHttpApiTest, PublishDisconnected)
       resource.mutable_provider_id()->CopyFrom(resourceProviderId.get());
     }
 
-    Future<Nothing> published = manager.publish(devolve(resources));
+    Future<Nothing> published = manager.publishResources(devolve(resources));
 
     Future<Result<Event>> event = responseDecoder->read();
     AWAIT_READY(event);
     ASSERT_SOME(event.get());
-    ASSERT_EQ(Event::PUBLISH, event->get().type());
+    ASSERT_EQ(Event::PUBLISH_RESOURCES, event->get().type());
 
     reader->close();
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/2975d7e6/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index 6640620..8739f0b 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -8883,11 +8883,11 @@ TEST_F(SlaveTest, ResourceProviderPublishAll)
     AWAIT_READY(offers);
     ASSERT_FALSE(offers->empty());
 
-    Future<mesos::v1::resource_provider::Event::Publish> publish;
+    Future<mesos::v1::resource_provider::Event::PublishResources> publish;
 
-    // Two PUBLISH events will be received: one for launching the
+    // Two PUBLISH_RESOURCES events will be received: one for launching the
     // executor, and the other for launching the task.
-    EXPECT_CALL(resourceProvider, publish(_))
+    EXPECT_CALL(resourceProvider, publishResources(_))
       .WillOnce(
           Invoke(&resourceProvider,
                  &v1::MockResourceProvider::publishDefault))


[3/3] mesos git commit: Changed the resource provider OPERATION event to APPLY_OFFER_OPERATION.

Posted by gr...@apache.org.
Changed the resource provider OPERATION event to APPLY_OFFER_OPERATION.

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


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

Branch: refs/heads/master
Commit: c78496fd5c2f48f2989449e3edc032af18524802
Parents: fb47d1c
Author: Greg Mann <gr...@gmail.com>
Authored: Thu Dec 7 22:45:24 2017 -0800
Committer: Greg Mann <gr...@gmail.com>
Committed: Thu Dec 7 23:18:12 2017 -0800

----------------------------------------------------------------------
 .../resource_provider/resource_provider.proto   |  6 ++---
 .../resource_provider/resource_provider.proto   |  6 ++---
 src/messages/messages.proto                     |  2 +-
 src/resource_provider/manager.cpp               | 12 +++++----
 src/resource_provider/state.proto               |  2 +-
 src/resource_provider/storage/provider.cpp      | 28 +++++++++++---------
 src/tests/mesos.hpp                             | 14 +++++-----
 src/tests/slave_tests.cpp                       |  4 +--
 8 files changed, 41 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c78496fd/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 2619cd0..360d862 100644
--- a/include/mesos/resource_provider/resource_provider.proto
+++ b/include/mesos/resource_provider/resource_provider.proto
@@ -33,7 +33,7 @@ message Event {
     UNKNOWN = 0;
 
     SUBSCRIBED = 1;                  // See 'Subscribed' below.
-    OPERATION = 2;                   // See 'Operation' below.
+    APPLY_OFFER_OPERATION = 2;       // See 'ApplyOfferOperation' below.
     PUBLISH_RESOURCES = 3;           // See 'PublishResources' below.
     ACKNOWLEDGE_OFFER_OPERATION = 4; // See 'AcknowledgeOfferOperation' below.
     RECONCILE_OFFER_OPERATIONS = 5;  // See 'ReconcileOfferOperations' below.
@@ -47,7 +47,7 @@ message Event {
 
   // Received when the master wants to send an operation to the
   // resource provider.
-  message Operation {
+  message ApplyOfferOperation {
     required FrameworkID framework_id = 1;
     required Offer.Operation info = 2;
 
@@ -98,7 +98,7 @@ message Event {
 
   optional Type type = 1;
   optional Subscribed subscribed = 2;
-  optional Operation operation = 3;
+  optional ApplyOfferOperation apply_offer_operation = 3;
   optional PublishResources publish_resources = 4;
   optional AcknowledgeOfferOperation acknowledge_offer_operation = 5;
   optional ReconcileOfferOperations reconcile_offer_operations = 6;

http://git-wip-us.apache.org/repos/asf/mesos/blob/c78496fd/include/mesos/v1/resource_provider/resource_provider.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/resource_provider/resource_provider.proto b/include/mesos/v1/resource_provider/resource_provider.proto
index 135729b..3e799f3 100644
--- a/include/mesos/v1/resource_provider/resource_provider.proto
+++ b/include/mesos/v1/resource_provider/resource_provider.proto
@@ -33,7 +33,7 @@ message Event {
     UNKNOWN = 0;
 
     SUBSCRIBED = 1;                  // See 'Subscribed' below.
-    OPERATION = 2;                   // See 'Operation' below.
+    APPLY_OFFER_OPERATION = 2;       // See 'ApplyOfferOperation' below.
     PUBLISH_RESOURCES = 3;           // See 'PublishResources' below.
     ACKNOWLEDGE_OFFER_OPERATION = 4; // See 'AcknowledgeOfferOperation' below.
     RECONCILE_OFFER_OPERATIONS = 5;  // See 'ReconcileOfferOperations' below.
@@ -47,7 +47,7 @@ message Event {
 
   // Received when the master wants to send an operation to the
   // resource provider.
-  message Operation {
+  message ApplyOfferOperation {
     required FrameworkID framework_id = 1;
     required Offer.Operation info = 2;
 
@@ -98,7 +98,7 @@ message Event {
 
   optional Type type = 1;
   optional Subscribed subscribed = 2;
-  optional Operation operation = 3;
+  optional ApplyOfferOperation apply_offer_operation = 3;
   optional PublishResources publish_resources = 4;
   optional AcknowledgeOfferOperation acknowledge_offer_operation = 5;
   optional ReconcileOfferOperations reconcile_offer_operations = 6;

http://git-wip-us.apache.org/repos/asf/mesos/blob/c78496fd/src/messages/messages.proto
----------------------------------------------------------------------
diff --git a/src/messages/messages.proto b/src/messages/messages.proto
index 1a70967..44b45d5 100644
--- a/src/messages/messages.proto
+++ b/src/messages/messages.proto
@@ -779,7 +779,7 @@ message OfferOperationStatusUpdateRecord {
  * the master for external resource providers) to apply an offer
  * operation.
  *
- * See resource_provider::Event::OPERATION.
+ * See resource_provider::Event::APPLY_OFFER_OPERATION.
  */
 message ApplyOfferOperationMessage {
   optional FrameworkID framework_id = 1;

http://git-wip-us.apache.org/repos/asf/mesos/blob/c78496fd/src/resource_provider/manager.cpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/manager.cpp b/src/resource_provider/manager.cpp
index 9c0d19c..879caba 100644
--- a/src/resource_provider/manager.cpp
+++ b/src/resource_provider/manager.cpp
@@ -389,11 +389,13 @@ void ResourceProviderManagerProcess::applyOfferOperation(
     << resourceProviderId.get();
 
   Event event;
-  event.set_type(Event::OPERATION);
-  event.mutable_operation()->mutable_framework_id()->CopyFrom(frameworkId);
-  event.mutable_operation()->mutable_info()->CopyFrom(operation);
-  event.mutable_operation()->set_operation_uuid(message.operation_uuid());
-  event.mutable_operation()->set_resource_version_uuid(
+  event.set_type(Event::APPLY_OFFER_OPERATION);
+  event.mutable_apply_offer_operation()
+    ->mutable_framework_id()->CopyFrom(frameworkId);
+  event.mutable_apply_offer_operation()->mutable_info()->CopyFrom(operation);
+  event.mutable_apply_offer_operation()
+    ->set_operation_uuid(message.operation_uuid());
+  event.mutable_apply_offer_operation()->set_resource_version_uuid(
       message.resource_version_uuid().uuid());
 
   if (!resourceProvider->http.send(event)) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/c78496fd/src/resource_provider/state.proto
----------------------------------------------------------------------
diff --git a/src/resource_provider/state.proto b/src/resource_provider/state.proto
index ae30d16..321201e 100644
--- a/src/resource_provider/state.proto
+++ b/src/resource_provider/state.proto
@@ -30,7 +30,7 @@ message ResourceProviderState {
   // This includes only pending operations. Operations that have
   // unacknowledged statuses should be recovered through the status
   // update manager.
-  repeated Event.Operation operations = 1;
+  repeated Event.ApplyOfferOperation operations = 1;
 
   // The total resources provided by this resource provider.
   repeated Resource resources = 2;

http://git-wip-us.apache.org/repos/asf/mesos/blob/c78496fd/src/resource_provider/storage/provider.cpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/storage/provider.cpp b/src/resource_provider/storage/provider.cpp
index b6db618..2193866 100644
--- a/src/resource_provider/storage/provider.cpp
+++ b/src/resource_provider/storage/provider.cpp
@@ -329,7 +329,7 @@ private:
 
   // Functions for received events.
   void subscribed(const Event::Subscribed& subscribed);
-  void operation(const Event::Operation& operation);
+  void applyOfferOperation(const Event::ApplyOfferOperation& operation);
   void publishResources(const Event::PublishResources& publish);
   void acknowledgeOfferOperation(
       const Event::AcknowledgeOfferOperation& acknowledge);
@@ -411,7 +411,7 @@ private:
   // NOTE: We store the list of pending operations in a `LinkedHashMap`
   // to preserve the order we receive the operations. This is useful
   // when we replay depending operations during recovery.
-  LinkedHashMap<UUID, Event::Operation> pendingOperations;
+  LinkedHashMap<UUID, Event::ApplyOfferOperation> pendingOperations;
   Resources totalResources;
   Option<UUID> resourceVersion;
   hashmap<string, VolumeData> volumes;
@@ -448,9 +448,9 @@ void StorageLocalResourceProviderProcess::received(const Event& event)
       subscribed(event.subscribed());
       break;
     }
-    case Event::OPERATION: {
-      CHECK(event.has_operation());
-      operation(event.operation());
+    case Event::APPLY_OFFER_OPERATION: {
+      CHECK(event.has_apply_offer_operation());
+      applyOfferOperation(event.apply_offer_operation());
       break;
     }
     case Event::PUBLISH_RESOURCES: {
@@ -587,7 +587,7 @@ Future<Nothing> StorageLocalResourceProviderProcess::recover()
         }
 
         if (resourceProviderState.isSome()) {
-          foreach (const Event::Operation& operation,
+          foreach (const Event::ApplyOfferOperation& operation,
                    resourceProviderState->operations()) {
             Try<UUID> uuid = UUID::fromBytes(operation.operation_uuid());
             CHECK_SOME(uuid);
@@ -965,8 +965,8 @@ void StorageLocalResourceProviderProcess::subscribed(
 }
 
 
-void StorageLocalResourceProviderProcess::operation(
-    const Event::Operation& operation)
+void StorageLocalResourceProviderProcess::applyOfferOperation(
+    const Event::ApplyOfferOperation& operation)
 {
   Future<Resources> converted;
 
@@ -2026,7 +2026,8 @@ Future<Nothing> StorageLocalResourceProviderProcess::applyOfferOperation(
   Option<Error> error;
 
   CHECK(pendingOperations.contains(operationUuid));
-  const Event::Operation& operation = pendingOperations.at(operationUuid);
+  const Event::ApplyOfferOperation& operation =
+    pendingOperations.at(operationUuid);
 
   Try<UUID> operationVersion =
     UUID::fromBytes(operation.resource_version_uuid());
@@ -2295,7 +2296,8 @@ Try<Nothing> StorageLocalResourceProviderProcess::applyResourceConversions(
   Option<Error> error;
 
   CHECK(pendingOperations.contains(operationUuid));
-  const Event::Operation& operation = pendingOperations.at(operationUuid);
+  const Event::ApplyOfferOperation& operation =
+    pendingOperations.at(operationUuid);
 
   if (conversions.isSome()) {
     // Strip away the allocation info when applying the convertion to
@@ -2388,7 +2390,9 @@ void StorageLocalResourceProviderProcess::checkpointResourceProviderState()
 {
   ResourceProviderState state;
 
-  foreachvalue (const Event::Operation& operation, pendingOperations) {
+  foreachvalue (
+      const Event::ApplyOfferOperation& operation,
+      pendingOperations) {
     state.add_operations()->CopyFrom(operation);
   }
 
@@ -2414,7 +2418,7 @@ void StorageLocalResourceProviderProcess::sendResourceProviderStateUpdate()
   Call::UpdateState* update = call.mutable_update_state();
 
   foreachpair (const UUID& uuid,
-               const Event::Operation& operation,
+               const Event::ApplyOfferOperation& operation,
                pendingOperations) {
     // TODO(chhsiao): Maintain a list of terminated but unacknowledged
     // offer operations in memory and reconstruc that during recovery

http://git-wip-us.apache.org/repos/asf/mesos/blob/c78496fd/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index be976c4..9506889 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -2834,7 +2834,7 @@ public:
               Source>::subscribedDefault));
     EXPECT_CALL(*this, subscribed(_)).WillRepeatedly(DoDefault());
 
-    ON_CALL(*this, operation(_))
+    ON_CALL(*this, applyOfferOperation(_))
       .WillByDefault(Invoke(
           this,
           &MockResourceProvider<
@@ -2848,7 +2848,7 @@ public:
               OfferOperationState,
               Operation,
               Source>::operationDefault));
-    EXPECT_CALL(*this, operation(_)).WillRepeatedly(DoDefault());
+    EXPECT_CALL(*this, applyOfferOperation(_)).WillRepeatedly(DoDefault());
 
     ON_CALL(*this, publishResources(_))
       .WillByDefault(Invoke(
@@ -2870,7 +2870,9 @@ public:
   MOCK_METHOD0_T(connected, void());
   MOCK_METHOD0_T(disconnected, void());
   MOCK_METHOD1_T(subscribed, void(const typename Event::Subscribed&));
-  MOCK_METHOD1_T(operation, void(const typename Event::Operation&));
+  MOCK_METHOD1_T(
+      applyOfferOperation,
+      void(const typename Event::ApplyOfferOperation&));
   MOCK_METHOD1_T(
       publishResources,
       void(const typename Event::PublishResources&));
@@ -2891,8 +2893,8 @@ public:
         case Event::SUBSCRIBED:
           subscribed(event.subscribed());
           break;
-        case Event::OPERATION:
-          operation(event.operation());
+        case Event::APPLY_OFFER_OPERATION:
+          applyOfferOperation(event.apply_offer_operation());
           break;
         case Event::PUBLISH_RESOURCES:
           publishResources(event.publish_resources());
@@ -3000,7 +3002,7 @@ public:
     }
   }
 
-  void operationDefault(const typename Event::Operation& operation)
+  void operationDefault(const typename Event::ApplyOfferOperation& operation)
   {
     CHECK(info.has_id());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c78496fd/src/tests/slave_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/slave_tests.cpp b/src/tests/slave_tests.cpp
index 8739f0b..ee490a0 100644
--- a/src/tests/slave_tests.cpp
+++ b/src/tests/slave_tests.cpp
@@ -9214,8 +9214,8 @@ TEST_F(SlaveTest, ResourceProviderReconciliation)
 
   // We now perform a `RESERVE` operation on the offered resources,
   // but let the operation fail in the resource provider.
-  Future<v1::resource_provider::Event::Operation> operation;
-  EXPECT_CALL(resourceProvider, operation(_))
+  Future<v1::resource_provider::Event::ApplyOfferOperation> operation;
+  EXPECT_CALL(resourceProvider, applyOfferOperation(_))
     .WillOnce(FutureArg<0>(&operation));
 
   {


[2/3] mesos git commit: Renamed a resource provider call to UPDATE_PUBLISH_RESOURCES_STATUS.

Posted by gr...@apache.org.
Renamed a resource provider call to UPDATE_PUBLISH_RESOURCES_STATUS.

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


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

Branch: refs/heads/master
Commit: fb47d1c548685b3efbee677454829c0c98d66d12
Parents: 2975d7e
Author: Greg Mann <gr...@gmail.com>
Authored: Wed Dec 6 16:42:14 2017 -0800
Committer: Greg Mann <gr...@gmail.com>
Committed: Thu Dec 7 23:06:22 2017 -0800

----------------------------------------------------------------------
 .../resource_provider/resource_provider.hpp     |  4 +--
 .../resource_provider/resource_provider.proto   | 12 +++----
 .../v1/resource_provider/resource_provider.hpp  |  4 +--
 .../resource_provider/resource_provider.proto   | 12 +++----
 src/resource_provider/manager.cpp               | 35 +++++++++++---------
 src/resource_provider/storage/provider.cpp      |  8 +++--
 src/resource_provider/validation.cpp            |  7 ++--
 src/tests/mesos.hpp                             |  8 ++---
 src/tests/resource_provider_manager_tests.cpp   | 14 ++++----
 9 files changed, 56 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fb47d1c5/include/mesos/resource_provider/resource_provider.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/resource_provider/resource_provider.hpp b/include/mesos/resource_provider/resource_provider.hpp
index 45cda42..36d3e6d 100644
--- a/include/mesos/resource_provider/resource_provider.hpp
+++ b/include/mesos/resource_provider/resource_provider.hpp
@@ -27,9 +27,9 @@ namespace resource_provider {
 
 inline std::ostream& operator<<(
     std::ostream& stream,
-    const Call::UpdatePublishStatus::Status& status)
+    const Call::UpdatePublishResourcesStatus::Status& status)
 {
-  return stream << Call::UpdatePublishStatus::Status_Name(status);
+  return stream << Call::UpdatePublishResourcesStatus::Status_Name(status);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb47d1c5/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 9c69813..2619cd0 100644
--- a/include/mesos/resource_provider/resource_provider.proto
+++ b/include/mesos/resource_provider/resource_provider.proto
@@ -113,10 +113,10 @@ message Call {
     // in a backwards-compatible way. See: MESOS-4997.
     UNKNOWN = 0;
 
-    SUBSCRIBE = 1;                     // See 'Subscribe'.
-    UPDATE_OFFER_OPERATION_STATUS = 2; // See 'UpdateOfferOperationStatus'.
-    UPDATE_STATE = 3;                  // See 'UpdateState'.
-    UPDATE_PUBLISH_STATUS = 4;         // See 'UpdatePublishStatus'.
+    SUBSCRIBE = 1;                       // See 'Subscribe'.
+    UPDATE_OFFER_OPERATION_STATUS = 2;   // See 'UpdateOfferOperationStatus'.
+    UPDATE_STATE = 3;                    // See 'UpdateState'.
+    UPDATE_PUBLISH_RESOURCES_STATUS = 4; // See 'UpdatePublishResourcesStatus'.
   }
 
   // Request to subscribe with the master.
@@ -160,7 +160,7 @@ message Call {
     required bytes resource_version_uuid = 3;
   }
 
-  message UpdatePublishStatus {
+  message UpdatePublishResourcesStatus {
     enum Status {
       UNKNOWN = 0;
       OK = 1;
@@ -185,5 +185,5 @@ message Call {
   optional Subscribe subscribe = 3;
   optional UpdateOfferOperationStatus update_offer_operation_status = 4;
   optional UpdateState update_state = 5;
-  optional UpdatePublishStatus update_publish_status = 6;
+  optional UpdatePublishResourcesStatus update_publish_resources_status = 6;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb47d1c5/include/mesos/v1/resource_provider/resource_provider.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/v1/resource_provider/resource_provider.hpp b/include/mesos/v1/resource_provider/resource_provider.hpp
index 3fc45c9..70d8176 100644
--- a/include/mesos/v1/resource_provider/resource_provider.hpp
+++ b/include/mesos/v1/resource_provider/resource_provider.hpp
@@ -28,9 +28,9 @@ namespace resource_provider {
 
 inline std::ostream& operator<<(
     std::ostream& stream,
-    const Call::UpdatePublishStatus::Status& status)
+    const Call::UpdatePublishResourcesStatus::Status& status)
 {
-  return stream << Call::UpdatePublishStatus::Status_Name(status);
+  return stream << Call::UpdatePublishResourcesStatus::Status_Name(status);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb47d1c5/include/mesos/v1/resource_provider/resource_provider.proto
----------------------------------------------------------------------
diff --git a/include/mesos/v1/resource_provider/resource_provider.proto b/include/mesos/v1/resource_provider/resource_provider.proto
index 8340316..135729b 100644
--- a/include/mesos/v1/resource_provider/resource_provider.proto
+++ b/include/mesos/v1/resource_provider/resource_provider.proto
@@ -113,10 +113,10 @@ message Call {
     // in a backwards-compatible way. See: MESOS-4997.
     UNKNOWN = 0;
 
-    SUBSCRIBE = 1;                     // See 'Subscribe'.
-    UPDATE_OFFER_OPERATION_STATUS = 2; // See 'UpdateOfferOperationStatus'.
-    UPDATE_STATE = 3;                  // See 'UpdateState'.
-    UPDATE_PUBLISH_STATUS = 4;         // See 'UpdatePublishStatus'.
+    SUBSCRIBE = 1;                       // See 'Subscribe'.
+    UPDATE_OFFER_OPERATION_STATUS = 2;   // See 'UpdateOfferOperationStatus'.
+    UPDATE_STATE = 3;                    // See 'UpdateState'.
+    UPDATE_PUBLISH_RESOURCES_STATUS = 4; // See 'UpdatePublishResourcesStatus'.
   }
 
   // Request to subscribe with the master.
@@ -160,7 +160,7 @@ message Call {
     required bytes resource_version_uuid = 3;
   }
 
-  message UpdatePublishStatus {
+  message UpdatePublishResourcesStatus {
     enum Status {
       UNKNOWN = 0;
       OK = 1;
@@ -185,5 +185,5 @@ message Call {
   optional Subscribe subscribe = 3;
   optional UpdateOfferOperationStatus update_offer_operation_status = 4;
   optional UpdateState update_state = 5;
-  optional UpdatePublishStatus update_publish_status = 6;
+  optional UpdatePublishResourcesStatus update_publish_resources_status = 6;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb47d1c5/src/resource_provider/manager.cpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/manager.cpp b/src/resource_provider/manager.cpp
index c12fd38..9c0d19c 100644
--- a/src/resource_provider/manager.cpp
+++ b/src/resource_provider/manager.cpp
@@ -180,9 +180,9 @@ private:
       ResourceProvider* resourceProvider,
       const Call::UpdateState& update);
 
-  void updatePublishStatus(
+  void updatePublishResourcesStatus(
       ResourceProvider* resourceProvider,
-      const Call::UpdatePublishStatus& update);
+      const Call::UpdatePublishResourcesStatus& update);
 
   ResourceProviderID newResourceProviderId();
 
@@ -330,8 +330,10 @@ Future<http::Response> ResourceProviderManagerProcess::api(
       return Accepted();
     }
 
-    case Call::UPDATE_PUBLISH_STATUS: {
-      updatePublishStatus(resourceProvider, call.update_publish_status());
+    case Call::UPDATE_PUBLISH_RESOURCES_STATUS: {
+      updatePublishResourcesStatus(
+          resourceProvider,
+          call.update_publish_resources_status());
       return Accepted();
     }
   }
@@ -663,34 +665,35 @@ void ResourceProviderManagerProcess::updateState(
 }
 
 
-void ResourceProviderManagerProcess::updatePublishStatus(
+void ResourceProviderManagerProcess::updatePublishResourcesStatus(
     ResourceProvider* resourceProvider,
-    const Call::UpdatePublishStatus& update)
+    const Call::UpdatePublishResourcesStatus& update)
 {
   Try<UUID> uuid = UUID::fromBytes(update.uuid());
   if (uuid.isError()) {
-    LOG(ERROR) << "Invalid UUID in UpdatePublishStatus from resource provider "
-               << resourceProvider->info.id() << ": " << uuid.error();
+    LOG(ERROR) << "Invalid UUID in UpdatePublishResourcesStatus from resource"
+               << " provider " << resourceProvider->info.id()
+               << ": " << uuid.error();
     return;
   }
 
   if (!resourceProvider->publishes.contains(uuid.get())) {
-    LOG(ERROR) << "Ignoring UpdatePublishStatus from resource provider "
-               << resourceProvider->info.id() << " because UUID "
-               << uuid->toString() << " is unknown";
+    LOG(ERROR) << "Ignoring UpdatePublishResourcesStatus from resource"
+               << " provider " << resourceProvider->info.id()
+               << " because UUID " << uuid->toString() << " is unknown";
     return;
   }
 
   LOG(INFO)
-    << "Received UPDATE_PUBLISH_STATUS call for PUBLISH event " << uuid.get()
-    << " with " << update.status() << " status from resource provider "
-    << resourceProvider->info.id();
+    << "Received UPDATE_PUBLISH_RESOURCES_STATUS call for PUBLISH_RESOURCES"
+    << " event " << uuid.get() << " with " << update.status()
+    << " status from resource provider " << resourceProvider->info.id();
 
-  if (update.status() == Call::UpdatePublishStatus::OK) {
+  if (update.status() == Call::UpdatePublishResourcesStatus::OK) {
     resourceProvider->publishes.at(uuid.get())->set(Nothing());
   } else {
     // TODO(jieyu): Consider to include an error message in
-    // 'UpdatePublishStatus' and surface that to the caller.
+    // 'UpdatePublishResourcesStatus' and surface that to the caller.
     resourceProvider->publishes.at(uuid.get())->fail(
         "Failed to publish resources for resource provider " +
         stringify(resourceProvider->info.id()) + ": Received " +

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb47d1c5/src/resource_provider/storage/provider.cpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/storage/provider.cpp b/src/resource_provider/storage/provider.cpp
index d43c8bd..b6db618 100644
--- a/src/resource_provider/storage/provider.cpp
+++ b/src/resource_provider/storage/provider.cpp
@@ -1111,12 +1111,14 @@ void StorageLocalResourceProviderProcess::publishResources(
 
       Call call;
       call.mutable_resource_provider_id()->CopyFrom(info.id());
-      call.set_type(Call::UPDATE_PUBLISH_STATUS);
+      call.set_type(Call::UPDATE_PUBLISH_RESOURCES_STATUS);
 
-      Call::UpdatePublishStatus* update = call.mutable_update_publish_status();
+      Call::UpdatePublishResourcesStatus* update =
+        call.mutable_update_publish_resources_status();
       update->set_uuid(publish.uuid());
       update->set_status(future.isReady()
-        ? Call::UpdatePublishStatus::OK : Call::UpdatePublishStatus::FAILED);
+        ? Call::UpdatePublishResourcesStatus::OK
+        : Call::UpdatePublishResourcesStatus::FAILED);
 
       auto err = [](const string& uuid, const string& message) {
         LOG(ERROR)

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb47d1c5/src/resource_provider/validation.cpp
----------------------------------------------------------------------
diff --git a/src/resource_provider/validation.cpp b/src/resource_provider/validation.cpp
index 984d593..5b7de74 100644
--- a/src/resource_provider/validation.cpp
+++ b/src/resource_provider/validation.cpp
@@ -74,13 +74,14 @@ Option<Error> validate(const Call& call)
       return None();
     }
 
-    case Call::UPDATE_PUBLISH_STATUS: {
+    case Call::UPDATE_PUBLISH_RESOURCES_STATUS: {
       if (!call.has_resource_provider_id()) {
         return Error("Expecting 'resource_provider_id' to be present");
       }
 
-      if (!call.has_update_publish_status()) {
-        return Error("Expecting 'update_publish_status' to be present.");
+      if (!call.has_update_publish_resources_status()) {
+        return Error(
+            "Expecting 'update_publish_resources_status' to be present.");
       }
 
       return None();

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb47d1c5/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index 3b9403c..be976c4 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -3082,13 +3082,13 @@ public:
     CHECK(info.has_id());
 
     Call call;
-    call.set_type(Call::UPDATE_PUBLISH_STATUS);
+    call.set_type(Call::UPDATE_PUBLISH_RESOURCES_STATUS);
     call.mutable_resource_provider_id()->CopyFrom(info.id());
 
-    typename Call::UpdatePublishStatus* update =
-      call.mutable_update_publish_status();
+    typename Call::UpdatePublishResourcesStatus* update =
+      call.mutable_update_publish_resources_status();
     update->set_uuid(publish.uuid());
-    update->set_status(Call::UpdatePublishStatus::OK);
+    update->set_status(Call::UpdatePublishResourcesStatus::OK);
 
     driver->send(call);
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/fb47d1c5/src/tests/resource_provider_manager_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/resource_provider_manager_tests.cpp b/src/tests/resource_provider_manager_tests.cpp
index 117003d..7c34b66 100644
--- a/src/tests/resource_provider_manager_tests.cpp
+++ b/src/tests/resource_provider_manager_tests.cpp
@@ -539,12 +539,13 @@ TEST_P(ResourceProviderManagerHttpApiTest, PublishResourcesSuccess)
     ASSERT_EQ(Event::PUBLISH_RESOURCES, event->get().type());
 
     Call call;
-    call.set_type(Call::UPDATE_PUBLISH_STATUS);
+    call.set_type(Call::UPDATE_PUBLISH_RESOURCES_STATUS);
     call.mutable_resource_provider_id()->CopyFrom(resourceProviderId.get());
 
-    Call::UpdatePublishStatus* update = call.mutable_update_publish_status();
+    Call::UpdatePublishResourcesStatus* update =
+      call.mutable_update_publish_resources_status();
     update->set_uuid(event->get().publish_resources().uuid());
-    update->set_status(Call::UpdatePublishStatus::OK);
+    update->set_status(Call::UpdatePublishResourcesStatus::OK);
 
     http::Request request;
     request.method = "POST";
@@ -644,12 +645,13 @@ TEST_P(ResourceProviderManagerHttpApiTest, PublishResourcesFailure)
     ASSERT_EQ(Event::PUBLISH_RESOURCES, event->get().type());
 
     Call call;
-    call.set_type(Call::UPDATE_PUBLISH_STATUS);
+    call.set_type(Call::UPDATE_PUBLISH_RESOURCES_STATUS);
     call.mutable_resource_provider_id()->CopyFrom(resourceProviderId.get());
 
-    Call::UpdatePublishStatus* update = call.mutable_update_publish_status();
+    Call::UpdatePublishResourcesStatus* update =
+      call.mutable_update_publish_resources_status();
     update->set_uuid(event->get().publish_resources().uuid());
-    update->set_status(Call::UpdatePublishStatus::FAILED);
+    update->set_status(Call::UpdatePublishResourcesStatus::FAILED);
 
     http::Request request;
     request.method = "POST";