You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by al...@apache.org on 2018/08/21 06:43:34 UTC

[mesos] branch master updated (9c4b4d5 -> c6c078e)

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

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


    from 9c4b4d5  Added MESOS-8917 to 1.5.2 CHANGELOG.
     new a6c6298  Added const versions of slave-counting functions in Master.
     new 943aa2c  Moved state serialization into separate function.
     new 8e2ea9d  Changed vector of requests to be mutable.
     new c6c078e  Added 'handler' field to batched requests.

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


Summary of changes:
 src/master/http.cpp   | 274 ++++++++++++++++++++++++++------------------------
 src/master/master.cpp |  39 ++++++-
 src/master/master.hpp |  70 ++++++++++---
 3 files changed, 230 insertions(+), 153 deletions(-)


[mesos] 04/04: Added 'handler' field to batched requests.

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

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

commit c6c078e94865ad07abb4a12c5478a2bc46fc9eb4
Author: Benno Evers <be...@mesosphere.com>
AuthorDate: Tue Aug 21 08:10:38 2018 +0200

    Added 'handler' field to batched requests.
    
    This commit adds a handler field to be able to batch
    arbitrary requests, instead of having a hard-coded
    dispatch to `state`.
    
    Review: https://reviews.apache.org/r/68296/
---
 src/master/http.cpp   | 32 +++++++++++++++++++-------------
 src/master/master.hpp | 35 ++++++++++++++++++++---------------
 2 files changed, 39 insertions(+), 28 deletions(-)

diff --git a/src/master/http.cpp b/src/master/http.cpp
index 48944e2..ae28d52 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -2843,27 +2843,31 @@ Future<Response> Master::Http::state(
     .then(defer(
         master->self(),
         [this, request](const Owned<ObjectApprovers>& approvers) {
-          return deferStateRequest(request, approvers);
+          return deferBatchedRequest(
+              &Master::ReadOnlyHandler::state,
+              request,
+              approvers);
         }));
 }
 
 
-Future<Response> Master::Http::deferStateRequest(
+Future<Response> Master::Http::deferBatchedRequest(
+    ReadOnlyRequestHandler handler,
     const Request& request,
     const Owned<ObjectApprovers>& approvers) const
 {
-  bool scheduleBatch = batchedStateRequests.empty();
+  bool scheduleBatch = batchedRequests.empty();
 
   // Add an element to the batched state requests.
   Promise<Response> promise;
   Future<Response> future = promise.future();
-  batchedStateRequests.push_back(
-      BatchedStateRequest{request, approvers, std::move(promise)});
+  batchedRequests.push_back(
+      BatchedRequest{handler, request, approvers, std::move(promise)});
 
   // Schedule processing of batched requests if not yet scheduled.
   if (scheduleBatch) {
     dispatch(master->self(), [this]() {
-      processStateRequestsBatch();
+      processRequestsBatch();
     });
   }
 
@@ -3013,9 +3017,9 @@ process::http::Response Master::ReadOnlyHandler::state(
 }
 
 
-void Master::Http::processStateRequestsBatch() const
+void Master::Http::processRequestsBatch() const
 {
-  CHECK(!batchedStateRequests.empty())
+  CHECK(!batchedRequests.empty())
     << "Bug in state batching logic: No requests to process";
 
   // Produce the responses in parallel.
@@ -3025,12 +3029,14 @@ void Master::Http::processStateRequestsBatch() const
   //
   // TODO(alexr): Consider moving `BatchedStateRequest`'s fields into
   // `process::async` once it supports moving.
-  foreach (BatchedStateRequest& request, batchedStateRequests) {
+  foreach (BatchedRequest& request, batchedRequests) {
     request.promise.associate(process::async(
-        [this](const process::http::Request& request,
+        [this](ReadOnlyRequestHandler handler,
+               const process::http::Request& request,
                const process::Owned<ObjectApprovers>& approvers) {
-           return readonlyHandler.state(request, approvers);
+          return (readonlyHandler.*handler)(request, approvers);
         },
+        request.handler,
         request.request,
         request.approvers));
   }
@@ -3042,12 +3048,12 @@ void Master::Http::processStateRequestsBatch() const
   // NOTE: There is the potential for deadlock since we are blocking 1 working
   // thread here, see MESOS-8256.
   vector<Future<Response>> responses;
-  foreach (const BatchedStateRequest& request, batchedStateRequests) {
+  foreach (const BatchedRequest& request, batchedRequests) {
     responses.push_back(request.promise.future());
   }
   process::await(responses).await();
 
-  batchedStateRequests.clear();
+  batchedRequests.clear();
 }
 
 
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 7faaaae..dc0080b 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -1578,17 +1578,6 @@ private:
         const Option<process::http::authentication::Principal>&
             principal) const;
 
-    // A continuation for `state()`. Schedules request processing in a batch
-    // of other '/state' requests.
-    process::Future<process::http::Response> deferStateRequest(
-        const process::http::Request& request,
-        const process::Owned<ObjectApprovers>& approvers) const;
-
-    // A helper that responds to batched, i.e., accumulated, '/state'
-    // requests in parallel, i.e., a continuation for `deferStateRequest()`.
-    // See also `BatchedStateRequest`.
-    void processStateRequestsBatch() const;
-
     process::Future<std::vector<const Task*>> _tasks(
         const size_t limit,
         const size_t offset,
@@ -1870,16 +1859,32 @@ private:
     // out into this separate class.
     WeightsHandler weightsHandler;
 
-    // TODO(alexr): Consider adding a `type` or `handler` field to expand
-    // batching to other heavy read-only requests, e.g., '/state-summary'.
-    struct BatchedStateRequest
+    // Since the Master actor is one of the most loaded in a typical Mesos
+    // installation, we take some extra care to keep the backlog small.
+    // In particular, all read-only requests are batched and executed in
+    // parallel, instead of going through the master queue separately.
+
+    typedef process::http::Response
+      (Master::ReadOnlyHandler::*ReadOnlyRequestHandler)(
+          const process::http::Request&,
+          const process::Owned<ObjectApprovers>&) const;
+
+    process::Future<process::http::Response> deferBatchedRequest(
+        ReadOnlyRequestHandler handler,
+        const process::http::Request& request,
+        const process::Owned<ObjectApprovers>& approvers) const;
+
+    void processRequestsBatch() const;
+
+    struct BatchedRequest
     {
+      ReadOnlyRequestHandler handler;
       process::http::Request request;
       process::Owned<ObjectApprovers> approvers;
       process::Promise<process::http::Response> promise;
     };
 
-    mutable std::vector<BatchedStateRequest> batchedStateRequests;
+    mutable std::vector<BatchedRequest> batchedRequests;
   };
 
   Master(const Master&);              // No copying.


[mesos] 02/04: Moved state serialization into separate function.

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

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

commit 943aa2c984bfab59e327e96e955a2de72b91218f
Author: Benno Evers <be...@mesosphere.com>
AuthorDate: Tue Aug 21 08:10:09 2018 +0200

    Moved state serialization into separate function.
    
    In preparation for the changes in the subsequent commit,
    the serialization logic was moved from an inner lambda
    to a separate function.
    
    Review: https://reviews.apache.org/r/68343/
---
 src/master/http.cpp   | 246 +++++++++++++++++++++++++-------------------------
 src/master/master.hpp |  25 +++++
 2 files changed, 150 insertions(+), 121 deletions(-)

diff --git a/src/master/http.cpp b/src/master/http.cpp
index e2773ed..1b6a840 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -2871,154 +2871,153 @@ Future<Response> Master::Http::deferStateRequest(
 }
 
 
-void Master::Http::processStateRequestsBatch()
+process::http::Response Master::ReadOnlyHandler::state(
+    const process::http::Request& request,
+    const process::Owned<ObjectApprovers>& approvers) const
 {
-  CHECK(!batchedStateRequests.empty())
-    << "Bug in state batching logic: No requests to process";
-
-  // This lambda is consumed before the enclosed function returns,
-  // hence capturing `this` is fine here.
-  auto produceResponse = [this](
-      const Request& request,
-      const Owned<ObjectApprovers>& approvers) -> Response {
-    // This lambda is consumed before the outer lambda returns,
-    // hence capturing a reference is fine here.
-    auto calculateState = [this, &approvers](JSON::ObjectWriter* writer) {
-      writer->field("version", MESOS_VERSION);
-
-      if (build::GIT_SHA.isSome()) {
-        writer->field("git_sha", build::GIT_SHA.get());
-      }
+  const Master* master = this->master;
+  auto calculateState = [master, &approvers](JSON::ObjectWriter* writer) {
+    writer->field("version", MESOS_VERSION);
 
-      if (build::GIT_BRANCH.isSome()) {
-        writer->field("git_branch", build::GIT_BRANCH.get());
-      }
+    if (build::GIT_SHA.isSome()) {
+      writer->field("git_sha", build::GIT_SHA.get());
+    }
 
-      if (build::GIT_TAG.isSome()) {
-        writer->field("git_tag", build::GIT_TAG.get());
-      }
+    if (build::GIT_BRANCH.isSome()) {
+      writer->field("git_branch", build::GIT_BRANCH.get());
+    }
 
-      writer->field("build_date", build::DATE);
-      writer->field("build_time", build::TIME);
-      writer->field("build_user", build::USER);
-      writer->field("start_time", master->startTime.secs());
+    if (build::GIT_TAG.isSome()) {
+      writer->field("git_tag", build::GIT_TAG.get());
+    }
 
-      if (master->electedTime.isSome()) {
-        writer->field("elected_time", master->electedTime->secs());
-      }
+    writer->field("build_date", build::DATE);
+    writer->field("build_time", build::TIME);
+    writer->field("build_user", build::USER);
+    writer->field("start_time", master->startTime.secs());
 
-      writer->field("id", master->info().id());
-      writer->field("pid", string(master->self()));
-      writer->field("hostname", master->info().hostname());
-      writer->field("capabilities", master->info().capabilities());
-      writer->field("activated_slaves", master->_slaves_active());
-      writer->field("deactivated_slaves", master->_slaves_inactive());
-      writer->field("unreachable_slaves", master->_slaves_unreachable());
+    if (master->electedTime.isSome()) {
+      writer->field("elected_time", master->electedTime->secs());
+    }
 
-      if (master->info().has_domain()) {
-        writer->field("domain", master->info().domain());
-      }
+    writer->field("id", master->info().id());
+    writer->field("pid", string(master->self()));
+    writer->field("hostname", master->info().hostname());
+    writer->field("capabilities", master->info().capabilities());
+    writer->field("activated_slaves", master->_const_slaves_active());
+    writer->field("deactivated_slaves", master->_const_slaves_inactive());
+    writer->field("unreachable_slaves", master->_const_slaves_unreachable());
 
-      // TODO(haosdent): Deprecated this in favor of `leader_info` below.
-      if (master->leader.isSome()) {
-        writer->field("leader", master->leader->pid());
-      }
+    if (master->info().has_domain()) {
+      writer->field("domain", master->info().domain());
+    }
 
-      if (master->leader.isSome()) {
-        writer->field("leader_info", [this](JSON::ObjectWriter* writer) {
-          json(writer, master->leader.get());
-        });
-      }
+    // TODO(haosdent): Deprecated this in favor of `leader_info` below.
+    if (master->leader.isSome()) {
+      writer->field("leader", master->leader->pid());
+    }
 
-      if (approvers->approved<VIEW_FLAGS>()) {
-        if (master->flags.cluster.isSome()) {
-          writer->field("cluster", master->flags.cluster.get());
-        }
+    if (master->leader.isSome()) {
+      writer->field("leader_info", [master](JSON::ObjectWriter* writer) {
+        json(writer, master->leader.get());
+      });
+    }
 
-        if (master->flags.log_dir.isSome()) {
-          writer->field("log_dir", master->flags.log_dir.get());
-        }
+    if (approvers->approved<VIEW_FLAGS>()) {
+      if (master->flags.cluster.isSome()) {
+        writer->field("cluster", master->flags.cluster.get());
+      }
 
-        if (master->flags.external_log_file.isSome()) {
-          writer->field("external_log_file",
-                        master->flags.external_log_file.get());
-        }
+      if (master->flags.log_dir.isSome()) {
+        writer->field("log_dir", master->flags.log_dir.get());
+      }
 
-        writer->field("flags", [this](JSON::ObjectWriter* writer) {
-            foreachvalue (const flags::Flag& flag, master->flags) {
-              Option<string> value = flag.stringify(master->flags);
-              if (value.isSome()) {
-                writer->field(flag.effective_name().value, value.get());
-              }
-            }
-          });
+      if (master->flags.external_log_file.isSome()) {
+        writer->field("external_log_file",
+                      master->flags.external_log_file.get());
       }
 
-      // Model all of the registered slaves.
-      writer->field(
-          "slaves",
-          [this, &approvers](JSON::ArrayWriter* writer) {
-            foreachvalue (Slave* slave, master->slaves.registered) {
-              writer->element(SlaveWriter(*slave, approvers));
+      writer->field("flags", [master](JSON::ObjectWriter* writer) {
+          foreachvalue (const flags::Flag& flag, master->flags) {
+            Option<string> value = flag.stringify(master->flags);
+            if (value.isSome()) {
+              writer->field(flag.effective_name().value, value.get());
             }
-          });
+          }
+        });
+    }
 
-      // Model all of the recovered slaves.
-      writer->field(
-          "recovered_slaves",
-          [this](JSON::ArrayWriter* writer) {
-            foreachvalue (
-                const SlaveInfo& slaveInfo, master->slaves.recovered) {
-              writer->element([&slaveInfo](JSON::ObjectWriter* writer) {
-                json(writer, slaveInfo);
-              });
-            }
-          });
+    // Model all of the registered slaves.
+    writer->field(
+        "slaves",
+        [master, &approvers](JSON::ArrayWriter* writer) {
+          foreachvalue (Slave* slave, master->slaves.registered) {
+            writer->element(SlaveWriter(*slave, approvers));
+          }
+        });
 
-      // Model all of the frameworks.
-      writer->field(
-          "frameworks",
-          [this, &approvers](JSON::ArrayWriter* writer) {
-            foreachvalue (
-                Framework* framework, master->frameworks.registered) {
-              // Skip unauthorized frameworks.
-              if (!approvers->approved<VIEW_FRAMEWORK>(framework->info)) {
-                continue;
-              }
+    // Model all of the recovered slaves.
+    writer->field(
+        "recovered_slaves",
+        [master](JSON::ArrayWriter* writer) {
+          foreachvalue (
+              const SlaveInfo& slaveInfo, master->slaves.recovered) {
+            writer->element([&slaveInfo](JSON::ObjectWriter* writer) {
+              json(writer, slaveInfo);
+            });
+          }
+        });
 
-              writer->element(FullFrameworkWriter(approvers, framework));
+    // Model all of the frameworks.
+    writer->field(
+        "frameworks",
+        [master, &approvers](JSON::ArrayWriter* writer) {
+          foreachvalue (
+              Framework* framework, master->frameworks.registered) {
+            // Skip unauthorized frameworks.
+            if (!approvers->approved<VIEW_FRAMEWORK>(framework->info)) {
+              continue;
             }
-          });
 
-      // Model all of the completed frameworks.
-      writer->field(
-          "completed_frameworks",
-          [this, &approvers](JSON::ArrayWriter* writer) {
-            foreachvalue (
-                const Owned<Framework>& framework,
-                master->frameworks.completed) {
-              // Skip unauthorized frameworks.
-              if (!approvers->approved<VIEW_FRAMEWORK>(framework->info)) {
-                continue;
-              }
+            writer->element(FullFrameworkWriter(approvers, framework));
+          }
+        });
 
-              writer->element(
-                  FullFrameworkWriter(approvers, framework.get()));
+    // Model all of the completed frameworks.
+    writer->field(
+        "completed_frameworks",
+        [master, &approvers](JSON::ArrayWriter* writer) {
+          foreachvalue (
+              const Owned<Framework>& framework,
+              master->frameworks.completed) {
+            // Skip unauthorized frameworks.
+            if (!approvers->approved<VIEW_FRAMEWORK>(framework->info)) {
+              continue;
             }
-          });
 
-      // Orphan tasks are no longer possible. We emit an empty array
-      // for the sake of backward compatibility.
-      writer->field("orphan_tasks", [](JSON::ArrayWriter*) {});
+            writer->element(
+                FullFrameworkWriter(approvers, framework.get()));
+          }
+        });
 
-      // Unregistered frameworks are no longer possible. We emit an
-      // empty array for the sake of backward compatibility.
-      writer->field("unregistered_frameworks", [](JSON::ArrayWriter*) {});
-    };
+    // Orphan tasks are no longer possible. We emit an empty array
+    // for the sake of backward compatibility.
+    writer->field("orphan_tasks", [](JSON::ArrayWriter*) {});
 
-    return OK(jsonify(calculateState), request.url.query.get("jsonp"));
+    // Unregistered frameworks are no longer possible. We emit an
+    // empty array for the sake of backward compatibility.
+    writer->field("unregistered_frameworks", [](JSON::ArrayWriter*) {});
   };
 
+  return OK(jsonify(calculateState), request.url.query.get("jsonp"));
+}
+
+
+void Master::Http::processStateRequestsBatch()
+{
+  CHECK(!batchedStateRequests.empty())
+    << "Bug in state batching logic: No requests to process";
+
   // Produce the responses in parallel.
   //
   // TODO(alexr): Consider abstracting this into `parallel_async` or
@@ -3028,7 +3027,12 @@ void Master::Http::processStateRequestsBatch()
   // `process::async` once it supports moving.
   foreach (BatchedStateRequest& request, batchedStateRequests) {
     request.promise.associate(process::async(
-        produceResponse, request.request, request.approvers));
+        [this](const process::http::Request& request,
+               const process::Owned<ObjectApprovers>& approvers) {
+           return readonlyHandler.state(request, approvers);
+        },
+        request.request,
+        request.approvers));
   }
 
   // Block the master actor until all workers have generated state responses.
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 1020626..6312c29 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -1384,12 +1384,35 @@ private:
     Master* master;
   };
 
+  // Inner class used to namespace HTTP handlers that do not change the
+  // underlying master object.
+  //
+  // NOTE: Most member functions of this class are not routed directly but
+  // dispatched from their corresponding handlers in the outer `Http` class.
+  // This is because deciding whether an incoming request is read-only often
+  // requires some inspection, e.g. distinguishing between "GET" and "POST"
+  // requests to the same endpoint.
+  class ReadOnlyHandler
+  {
+  public:
+    explicit ReadOnlyHandler(const Master* _master) : master(_master) {}
+
+    // /state
+    process::http::Response state(
+        const process::http::Request& request,
+        const process::Owned<ObjectApprovers>& approvers) const;
+
+  private:
+    const Master* master;
+  };
+
   // Inner class used to namespace HTTP route handlers (see
   // master/http.cpp for implementations).
   class Http
   {
   public:
     explicit Http(Master* _master) : master(_master),
+                                     readonlyHandler(_master),
                                      quotaHandler(_master),
                                      weightsHandler(_master) {}
 
@@ -1837,6 +1860,8 @@ private:
 
     Master* master;
 
+    ReadOnlyHandler readonlyHandler;
+
     // NOTE: The quota specific pieces of the Operator API are factored
     // out into this separate class.
     QuotaHandler quotaHandler;


[mesos] 01/04: Added const versions of slave-counting functions in Master.

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

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

commit a6c62980c012dc18cbcda475c59ddef46b5eb59b
Author: Benno Evers <be...@mesosphere.com>
AuthorDate: Tue Aug 21 08:09:54 2018 +0200

    Added const versions of slave-counting functions in Master.
    
    The family of functions '_slaves_active()', etc. is logically
    const, although the function signatures do not currently
    reflect this due to MESOS-4995.
    
    Since a const version of this functionality is required for
    the changes introduced in the subsequent commits, it is
    added with different names as a workaround.
    
    Review: https://reviews.apache.org/r/68342/
---
 src/master/master.cpp | 39 ++++++++++++++++++++++++++++++++++-----
 src/master/master.hpp |  8 ++++++++
 2 files changed, 42 insertions(+), 5 deletions(-)

diff --git a/src/master/master.cpp b/src/master/master.cpp
index ad83be2..cc38a37 100644
--- a/src/master/master.cpp
+++ b/src/master/master.cpp
@@ -11532,7 +11532,7 @@ SlaveID Master::newSlaveId()
 }
 
 
-double Master::_slaves_connected()
+double Master::_const_slaves_connected() const
 {
   double count = 0.0;
   foreachvalue (Slave* slave, slaves.registered) {
@@ -11543,8 +11543,13 @@ double Master::_slaves_connected()
   return count;
 }
 
+double Master::_slaves_connected()
+{
+  return _const_slaves_connected();
+}
+
 
-double Master::_slaves_disconnected()
+double Master::_const_slaves_disconnected() const
 {
   double count = 0.0;
   foreachvalue (Slave* slave, slaves.registered) {
@@ -11556,7 +11561,13 @@ double Master::_slaves_disconnected()
 }
 
 
-double Master::_slaves_active()
+double Master::_slaves_disconnected()
+{
+  return _const_slaves_disconnected();
+}
+
+
+double Master::_const_slaves_active() const
 {
   double count = 0.0;
   foreachvalue (Slave* slave, slaves.registered) {
@@ -11568,7 +11579,13 @@ double Master::_slaves_active()
 }
 
 
-double Master::_slaves_inactive()
+double Master::_slaves_active()
+{
+  return _const_slaves_active();
+}
+
+
+double Master::_const_slaves_inactive() const
 {
   double count = 0.0;
   foreachvalue (Slave* slave, slaves.registered) {
@@ -11580,12 +11597,24 @@ double Master::_slaves_inactive()
 }
 
 
-double Master::_slaves_unreachable()
+double Master::_slaves_inactive()
+{
+  return _const_slaves_inactive();
+}
+
+
+double Master::_const_slaves_unreachable() const
 {
   return static_cast<double>(slaves.unreachable.size());
 }
 
 
+double Master::_slaves_unreachable()
+{
+  return _const_slaves_unreachable();
+}
+
+
 double Master::_frameworks_connected()
 {
   double count = 0.0;
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 36994a1..1020626 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -2227,6 +2227,14 @@ private:
   double _slaves_inactive();
   double _slaves_unreachable();
 
+  // TODO(bevers): Remove these and make the above functions
+  // const instead after MESOS-4995 is resolved.
+  double _const_slaves_connected() const;
+  double _const_slaves_disconnected() const;
+  double _const_slaves_active() const;
+  double _const_slaves_inactive() const;
+  double _const_slaves_unreachable() const;
+
   double _frameworks_connected();
   double _frameworks_disconnected();
   double _frameworks_active();


[mesos] 03/04: Changed vector of requests to be mutable.

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

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

commit 8e2ea9d588440a054cb3e5ed9d72dd33985f7084
Author: Benno Evers <be...@mesosphere.com>
AuthorDate: Tue Aug 21 08:10:21 2018 +0200

    Changed vector of requests to be mutable.
    
    Changed the `batchedRequest` vector inside the `Master::Http`
    class to be mutable, in order to be able to restore the
    `const` qualifier to the `/state`-handler.
    
    While this may be a debatable benefit for this
    endpoint alone, the subsequent commits introduce
    additional work to extend this mechanism to all
    read-only endpoint of `Master`.
    
    Without this change, this would create a paradoxical
    situation where all mutating endpoints are marked as
    const and all read-only endpoints are marked as
    non-const.
    
    Review: https://reviews.apache.org/r/68378/
---
 src/master/http.cpp   | 6 +++---
 src/master/master.hpp | 8 ++++----
 2 files changed, 7 insertions(+), 7 deletions(-)

diff --git a/src/master/http.cpp b/src/master/http.cpp
index 1b6a840..48944e2 100644
--- a/src/master/http.cpp
+++ b/src/master/http.cpp
@@ -2816,7 +2816,7 @@ string Master::Http::STATE_HELP()
 
 Future<Response> Master::Http::state(
     const Request& request,
-    const Option<Principal>& principal)
+    const Option<Principal>& principal) const
 {
   // TODO(greggomann): Remove this check once the `Principal` type is used in
   // `ReservationInfo`, `DiskInfo`, and within the master's `principals` map.
@@ -2850,7 +2850,7 @@ Future<Response> Master::Http::state(
 
 Future<Response> Master::Http::deferStateRequest(
     const Request& request,
-    const Owned<ObjectApprovers>& approvers)
+    const Owned<ObjectApprovers>& approvers) const
 {
   bool scheduleBatch = batchedStateRequests.empty();
 
@@ -3013,7 +3013,7 @@ process::http::Response Master::ReadOnlyHandler::state(
 }
 
 
-void Master::Http::processStateRequestsBatch()
+void Master::Http::processStateRequestsBatch() const
 {
   CHECK(!batchedStateRequests.empty())
     << "Bug in state batching logic: No requests to process";
diff --git a/src/master/master.hpp b/src/master/master.hpp
index 6312c29..7faaaae 100644
--- a/src/master/master.hpp
+++ b/src/master/master.hpp
@@ -1490,7 +1490,7 @@ private:
     process::Future<process::http::Response> state(
         const process::http::Request& request,
         const Option<process::http::authentication::Principal>&
-            principal);
+            principal) const;
 
     // /master/state-summary
     process::Future<process::http::Response> stateSummary(
@@ -1582,12 +1582,12 @@ private:
     // of other '/state' requests.
     process::Future<process::http::Response> deferStateRequest(
         const process::http::Request& request,
-        const process::Owned<ObjectApprovers>& approvers);
+        const process::Owned<ObjectApprovers>& approvers) const;
 
     // A helper that responds to batched, i.e., accumulated, '/state'
     // requests in parallel, i.e., a continuation for `deferStateRequest()`.
     // See also `BatchedStateRequest`.
-    void processStateRequestsBatch();
+    void processStateRequestsBatch() const;
 
     process::Future<std::vector<const Task*>> _tasks(
         const size_t limit,
@@ -1879,7 +1879,7 @@ private:
       process::Promise<process::http::Response> promise;
     };
 
-    std::vector<BatchedStateRequest> batchedStateRequests;
+    mutable std::vector<BatchedStateRequest> batchedStateRequests;
   };
 
   Master(const Master&);              // No copying.