You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2016/09/02 02:36:30 UTC

[1/4] kudu git commit: KUDU-1495. Maintenance manager should not schedule new ops during unregister

Repository: kudu
Updated Branches:
  refs/heads/master 93be1310d -> d13ac79a4


KUDU-1495. Maintenance manager should not schedule new ops during unregister

This fixes a bug where deleting a tablet could block for an extended
time on compactions being scheduled if the maintenance manager was
configured with multiple threads.

The issue was that Unregister() waited for the number of running
operations to reach 0 before removing the op from the candidate list of
ops, but did not prevent new instances of that op from being scheduled.

Change-Id: I3675705caf5b73f8a480036b974e4db6c205616a
Reviewed-on: http://gerrit.cloudera.org:8080/4295
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 531e7d99c68a7767cfc900447179fcc116f32f1c
Parents: 93be131
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Sep 1 17:06:59 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Sep 2 01:56:12 2016 +0000

----------------------------------------------------------------------
 src/kudu/tablet/tablet.cc                 |  6 ++++
 src/kudu/util/maintenance_manager-test.cc | 47 +++++++++++++++++++++++---
 src/kudu/util/maintenance_manager.cc      |  9 +++--
 src/kudu/util/maintenance_manager.h       | 19 +++++++++++
 4 files changed, 74 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/531e7d99/src/kudu/tablet/tablet.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/tablet.cc b/src/kudu/tablet/tablet.cc
index 387d6f5..faf1ae1 100644
--- a/src/kudu/tablet/tablet.cc
+++ b/src/kudu/tablet/tablet.cc
@@ -1206,6 +1206,12 @@ void Tablet::RegisterMaintenanceOps(MaintenanceManager* maint_mgr) {
 }
 
 void Tablet::UnregisterMaintenanceOps() {
+  // First cancel all of the operations, so that while we're waiting for one
+  // operation to finish in Unregister(), a different one can't get re-scheduled.
+  for (MaintenanceOp* op : maintenance_ops_) {
+    op->CancelAndDisable();
+  }
+
   for (MaintenanceOp* op : maintenance_ops_) {
     op->Unregister();
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/531e7d99/src/kudu/util/maintenance_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/maintenance_manager-test.cc b/src/kudu/util/maintenance_manager-test.cc
index 60a9d51..73e11a6 100644
--- a/src/kudu/util/maintenance_manager-test.cc
+++ b/src/kudu/util/maintenance_manager-test.cc
@@ -85,7 +85,8 @@ class TestMaintenanceOp : public MaintenanceOp {
       metric_entity_(METRIC_ENTITY_test.Instantiate(&metric_registry_, "test")),
       maintenance_op_duration_(METRIC_maintenance_op_duration.Instantiate(metric_entity_)),
       maintenance_ops_running_(METRIC_maintenance_ops_running.Instantiate(metric_entity_, 0)),
-      remaining_runs_(1) {
+      remaining_runs_(1),
+      sleep_time_(MonoDelta::FromSeconds(0)) {
   }
 
   virtual ~TestMaintenanceOp() {}
@@ -100,10 +101,14 @@ class TestMaintenanceOp : public MaintenanceOp {
   }
 
   virtual void Perform() OVERRIDE {
-    DLOG(INFO) << "Performing op " << name();
-    std::lock_guard<Mutex> guard(lock_);
-    CHECK_GE(remaining_runs_, 1);
-    remaining_runs_--;
+    {
+      std::lock_guard<Mutex> guard(lock_);
+      DLOG(INFO) << "Performing op " << name();
+      CHECK_GE(remaining_runs_, 1);
+      remaining_runs_--;
+    }
+
+    SleepFor(sleep_time_);
   }
 
   virtual void UpdateStats(MaintenanceOpStats* stats) OVERRIDE {
@@ -119,6 +124,11 @@ class TestMaintenanceOp : public MaintenanceOp {
     remaining_runs_ = runs;
   }
 
+  void set_sleep_time(MonoDelta time) {
+    std::lock_guard<Mutex> guard(lock_);
+    sleep_time_ = time;
+  }
+
   void set_ram_anchored(uint64_t ram_anchored) {
     std::lock_guard<Mutex> guard(lock_);
     consumption_.Reset(ram_anchored);
@@ -156,6 +166,9 @@ class TestMaintenanceOp : public MaintenanceOp {
   // The number of remaining times this operation will run before disabling
   // itself.
   int remaining_runs_;
+
+  // The amount of time each op invocation will sleep.
+  MonoDelta sleep_time_;
 };
 
 // Create an op and wait for it to start running.  Unregister it while it is
@@ -179,6 +192,30 @@ TEST_F(MaintenanceManagerTest, TestRegisterUnregister) {
   ThreadJoiner(thread.get()).Join();
 }
 
+// Regression test for KUDU-1495: when an operation is being unregistered,
+// new instances of that operation should not be scheduled.
+TEST_F(MaintenanceManagerTest, TestNewOpsDontGetScheduledDuringUnregister) {
+  TestMaintenanceOp op1("1", MaintenanceOp::HIGH_IO_USAGE, test_tracker_);
+  op1.set_ram_anchored(1001);
+
+  // Set the op to run up to 10 times, and each time should sleep for a second.
+  op1.set_remaining_runs(10);
+  op1.set_sleep_time(MonoDelta::FromSeconds(1));
+  manager_->RegisterOp(&op1);
+
+  // Wait until two instances of the ops start running, since we have two MM threads.
+  AssertEventually([&]() {
+      ASSERT_EQ(op1.RunningGauge()->value(), 2);
+    });
+
+  // Trigger Unregister while they are running. This should wait for the currently-
+  // running operations to complete, but no new operations should be scheduled.
+  manager_->UnregisterOp(&op1);
+
+  // Hence, we should have run only the original two that we saw above.
+  ASSERT_LE(op1.DurationHistogram()->TotalCount(), 2);
+}
+
 // Test that we'll run an operation that doesn't improve performance when memory
 // pressure gets high.
 TEST_F(MaintenanceManagerTest, TestMemoryPressure) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/531e7d99/src/kudu/util/maintenance_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/maintenance_manager.cc b/src/kudu/util/maintenance_manager.cc
index b596e0e..7628fbb 100644
--- a/src/kudu/util/maintenance_manager.cc
+++ b/src/kudu/util/maintenance_manager.cc
@@ -72,7 +72,11 @@ void MaintenanceOpStats::Clear() {
 }
 
 MaintenanceOp::MaintenanceOp(std::string name, IOUsage io_usage)
-    : name_(std::move(name)), running_(0), io_usage_(io_usage) {}
+    : name_(std::move(name)),
+      running_(0),
+      cancel_(false),
+      io_usage_(io_usage) {
+}
 
 MaintenanceOp::~MaintenanceOp() {
   CHECK(!manager_.get()) << "You must unregister the " << name_
@@ -165,6 +169,7 @@ void MaintenanceManager::UnregisterOp(MaintenanceOp* op) {
       VLOG_AND_TRACE("maintenance", 1) << "Waiting for op " << op->name() << " to finish so "
             << "we can unregister it.";
     }
+    op->CancelAndDisable();
     while (iter->first->running_ > 0) {
       op->cond_->Wait();
       iter = ops_.find(op);
@@ -270,7 +275,7 @@ MaintenanceOp* MaintenanceManager::FindBestOp() {
     // Update op stats.
     stats.Clear();
     op->UpdateStats(&stats);
-    if (!stats.valid() || !stats.runnable()) {
+    if (op->cancelled() || !stats.valid() || !stats.runnable()) {
       continue;
     }
     if (stats.logs_retained_bytes() > low_io_most_logs_retained_bytes &&

http://git-wip-us.apache.org/repos/asf/kudu/blob/531e7d99/src/kudu/util/maintenance_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/maintenance_manager.h b/src/kudu/util/maintenance_manager.h
index c55d62b..2ae4ccb 100644
--- a/src/kudu/util/maintenance_manager.h
+++ b/src/kudu/util/maintenance_manager.h
@@ -26,6 +26,7 @@
 #include <vector>
 
 #include "kudu/gutil/macros.h"
+#include "kudu/util/atomic.h"
 #include "kudu/util/condition_variable.h"
 #include "kudu/util/maintenance_manager.pb.h"
 #include "kudu/util/monotime.h"
@@ -177,6 +178,19 @@ class MaintenanceOp {
 
   IOUsage io_usage() const { return io_usage_; }
 
+  // Return true if the operation has been cancelled due to Unregister() pending.
+  bool cancelled() const {
+    return cancel_.Load();
+  }
+
+  // Cancel this operation, which prevents new instances of it from being scheduled
+  // regardless of whether the statistics indicate it is runnable. Instances may also
+  // optionally poll 'cancelled()' on a periodic basis to know if they should abort a
+  // lengthy operation in the middle of Perform().
+  void CancelAndDisable() {
+    cancel_.Store(true);
+  }
+
  private:
   DISALLOW_COPY_AND_ASSIGN(MaintenanceOp);
 
@@ -186,6 +200,11 @@ class MaintenanceOp {
   // The number of times that this op is currently running.
   uint32_t running_;
 
+  // Set when we are trying to unregister the maintenance operation.
+  // Ongoing operations could read this boolean and cancel themselves.
+  // New operations will not be scheduled when this boolean is set.
+  AtomicBool cancel_;
+
   // Condition variable which the UnregisterOp function can wait on.
   //
   // Note: 'cond_' is used with the MaintenanceManager's mutex. As such,


[3/4] kudu git commit: cache: dynamically determine the number of cache shards

Posted by ad...@apache.org.
cache: dynamically determine the number of cache shards

I noticed in a heavily cache-bound workload on a 48-core machine that
there was a lot of contention on the cache shard spin locks. Manually
bumping the number of cache shards to 64 instead of the default 16
seemed to reduce the amount of contention significantly.

Change-Id: I2181ba4863065a554068a9c6d390b7f49e094abc
Reviewed-on: http://gerrit.cloudera.org:8080/4292
Tested-by: Kudu Jenkins
Reviewed-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/master
Commit: bfb6f2338cd57a1c7f8405c97b66c3050c8920ff
Parents: 22d68a0
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Sep 1 16:18:20 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Sep 2 01:57:37 2016 +0000

----------------------------------------------------------------------
 src/kudu/util/cache.cc | 26 +++++++++++++++++++-------
 1 file changed, 19 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/bfb6f233/src/kudu/util/cache.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/cache.cc b/src/kudu/util/cache.cc
index 7dd8e8e..964d93c 100644
--- a/src/kudu/util/cache.cc
+++ b/src/kudu/util/cache.cc
@@ -10,9 +10,11 @@
 #include <vector>
 
 #include "kudu/gutil/atomic_refcount.h"
+#include "kudu/gutil/bits.h"
 #include "kudu/gutil/hash/city.h"
 #include "kudu/gutil/stl_util.h"
 #include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/sysinfo.h"
 #include "kudu/util/alignment.h"
 #include "kudu/util/atomic.h"
 #include "kudu/util/cache.h"
@@ -368,8 +370,13 @@ void LRUCache::Erase(const Slice& key, uint32_t hash) {
   }
 }
 
-static const int kNumShardBits = 4;
-static const int kNumShards = 1 << kNumShardBits;
+// Determine the number of bits of the hash that should be used to determine
+// the cache shard. This, in turn, determines the number of shards.
+int DetermineShardBits() {
+  int bits = Bits::Log2Ceiling(base::NumCPUs());
+  VLOG(1) << "Will use " << (1 << bits) << " shards for LRU cache.";
+  return bits;
+}
 
 class ShardedLRUCache : public Cache {
  private:
@@ -379,26 +386,31 @@ class ShardedLRUCache : public Cache {
   MutexType id_mutex_;
   uint64_t last_id_;
 
+  // Number of bits of hash used to determine the shard.
+  const int shard_bits_;
+
   static inline uint32_t HashSlice(const Slice& s) {
     return util_hash::CityHash64(
       reinterpret_cast<const char *>(s.data()), s.size());
   }
 
-  static uint32_t Shard(uint32_t hash) {
-    return hash >> (32 - kNumShardBits);
+  uint32_t Shard(uint32_t hash) {
+    return hash >> (32 - shard_bits_);
   }
 
  public:
   explicit ShardedLRUCache(size_t capacity, const string& id)
-      : last_id_(0) {
+      : last_id_(0),
+        shard_bits_(DetermineShardBits()) {
     // A cache is often a singleton, so:
     // 1. We reuse its MemTracker if one already exists, and
     // 2. It is directly parented to the root MemTracker.
     mem_tracker_ = MemTracker::FindOrCreateTracker(
         -1, strings::Substitute("$0-sharded_lru_cache", id));
 
-    const size_t per_shard = (capacity + (kNumShards - 1)) / kNumShards;
-    for (int s = 0; s < kNumShards; s++) {
+    int num_shards = 1 << shard_bits_;
+    const size_t per_shard = (capacity + (num_shards - 1)) / num_shards;
+    for (int s = 0; s < num_shards; s++) {
       gscoped_ptr<LRUCache> shard(new LRUCache(mem_tracker_.get()));
       shard->SetCapacity(per_shard);
       shards_.push_back(shard.release());


[4/4] kudu git commit: docs: updates to kudu_impala_integration.adoc

Posted by ad...@apache.org.
docs: updates to kudu_impala_integration.adoc

Note that this doc has diverged pretty substantially from its downstream
Cloudera counterpart, which is far more featureful.

Change-Id: I02a91f1c33be3f0d6fbffaef5e03832e21b6db70
Reviewed-on: http://gerrit.cloudera.org:8080/4138
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Reviewed-by: Dan Burkert <da...@cloudera.com>


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

Branch: refs/heads/master
Commit: d13ac79a46602b135cb7dffcf484c65ccb898099
Parents: bfb6f23
Author: Adar Dembo <ad...@cloudera.com>
Authored: Fri Aug 26 16:52:06 2016 -0700
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Fri Sep 2 02:35:57 2016 +0000

----------------------------------------------------------------------
 docs/kudu_impala_integration.adoc | 16 +++++++++++-----
 1 file changed, 11 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/d13ac79a/docs/kudu_impala_integration.adoc
----------------------------------------------------------------------
diff --git a/docs/kudu_impala_integration.adoc b/docs/kudu_impala_integration.adoc
index 16be5cd..e2fe89c 100755
--- a/docs/kudu_impala_integration.adoc
+++ b/docs/kudu_impala_integration.adoc
@@ -174,12 +174,12 @@ hosted on `cloudera.com`.
 
 .Procedure
 
-- Download the `deploy.py` from https://github.com/cloudera/impala-kudu/blob/feature/kudu/infra/deploy/deploy.py
+- Download the `deploy.py` from https://github.com/apache/incubator-impala/blob/master/infra/deploy/deploy.py
 using `curl` or another utility of your choice.
 +
 [source,bash]
 ----
-$ curl -O https://raw.githubusercontent.com/cloudera/impala-kudu/feature/kudu/infra/deploy/deploy.py
+$ curl -O https://raw.githubusercontent.com/apache/incubator-impala/master/infra/deploy/deploy.py
 ----
 +
 - Run the `deploy.py` script. The syntax below creates a standalone IMPALA_KUDU
@@ -238,7 +238,10 @@ alongside another Impala instance if you use packages.
 [cols=">s,<,<",options="header"]
 |===
 | OS  | Repository  | Individual Packages
-| RHEL | link:http://archive.cloudera.com/beta/impala-kudu/redhat/6/x86_64/impala-kudu/cloudera-impala-kudu.repo[RHEL 6] |  link:http://archive.cloudera.com/beta/impala-kudu/redhat/6/x86_64/impala-kudu/0.5.0/RPMS/x86_64/[RHEL 6]
+| RHEL or CentOS | link:http://archive.cloudera.com/beta/impala-kudu/redhat/6/x86_64/impala-kudu/cloudera-impala-kudu.repo[RHEL 6 or CentOS 6],
+                   link:http://archive.cloudera.com/beta/impala-kudu/redhat/7/x86_64/impala-kudu/cloudera-impala-kudu.repo[RHEL 7 or CentOS 7] |
+                   link:http://archive.cloudera.com/beta/impala-kudu/redhat/6/x86_64/impala-kudu/0/RPMS/x86_64/[RHEL 6 or CentOS 6],
+                   link:http://archive.cloudera.com/beta/impala-kudu/redhat/7/x86_64/impala-kudu/0/RPMS/x86_64/[RHEL 7 or CentOS 7]
 | Ubuntu | link:http://archive.cloudera.com/beta/impala-kudu/ubuntu/trusty/amd64/impala-kudu/cloudera.list[Trusty] |  http://archive.cloudera.com/beta/impala-kudu/ubuntu/trusty/amd64/impala-kudu/pool/contrib/i/impala-kudu/[Trusty]
 |===
 
@@ -271,7 +274,10 @@ Instead, follow <<install_impala_kudu_packages>>.
 [cols=">s,<,<",options="header"]
 |===
 | OS  | Repository  | Individual Packages
-| RHEL | link:http://archive.cloudera.com/beta/impala-kudu/redhat/6/x86_64/impala-kudu/cloudera-impala-kudu.repo[RHEL 6] |  link:http://archive.cloudera.com/beta/impala-kudu/redhat/6/x86_64/impala-kudu/0.5.0/RPMS/x86_64/[RHEL 6]
+| RHEL or CentOS | link:http://archive.cloudera.com/beta/impala-kudu/redhat/6/x86_64/impala-kudu/cloudera-impala-kudu.repo[RHEL 6 or CentOS 6],
+                   link:http://archive.cloudera.com/beta/impala-kudu/redhat/7/x86_64/impala-kudu/cloudera-impala-kudu.repo[RHEL 7 or CentOS 7] |
+                   link:http://archive.cloudera.com/beta/impala-kudu/redhat/6/x86_64/impala-kudu/0/RPMS/x86_64/[RHEL 6 or CentOS 6],
+                   link:http://archive.cloudera.com/beta/impala-kudu/redhat/7/x86_64/impala-kudu/0/RPMS/x86_64/[RHEL 7 or CentOS 7]
 | Ubuntu | link:http://archive.cloudera.com/beta/impala-kudu/ubuntu/trusty/amd64/impala-kudu/cloudera.list[Trusty] |  http://archive.cloudera.com/beta/impala-kudu/ubuntu/trusty/amd64/impala-kudu/pool/contrib/i/impala-kudu/[Trusty]
 |===
 
@@ -305,7 +311,7 @@ provides the Impala query to map to an existing Kudu table in the web UI.
 
 - Be sure you are using the `impala-shell` binary provided by the Impala_Kudu package,
 rather than the default CDH Impala binary. The following shows how to verify this
-using the `alternatives` command on a RHEL 6 host.
+using the `alternatives` command on a RHEL or CentOS host.
 +
 [source,bash]
 ----


[2/4] kudu git commit: KUDU-1557. maintenance_manager: return most recently completed ops first

Posted by ad...@apache.org.
KUDU-1557. maintenance_manager: return most recently completed ops first

This changes the order of maintenance manager operations in the
maintenance manager status dump so that the most recently completed
operation is first in the list.

Change-Id: I7783bc8cee1beb0f7accaedbd46fb1e3a49168cc
Reviewed-on: http://gerrit.cloudera.org:8080/4296
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 22d68a0c30a72e29b3fe83e31bd80a417a2f1101
Parents: 531e7d9
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Sep 1 17:21:36 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Sep 2 01:56:15 2016 +0000

----------------------------------------------------------------------
 src/kudu/util/maintenance_manager-test.cc | 5 +++--
 src/kudu/util/maintenance_manager.cc      | 6 +++++-
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/22d68a0c/src/kudu/util/maintenance_manager-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/maintenance_manager-test.cc b/src/kudu/util/maintenance_manager-test.cc
index 73e11a6..2448766 100644
--- a/src/kudu/util/maintenance_manager-test.cc
+++ b/src/kudu/util/maintenance_manager-test.cc
@@ -293,8 +293,9 @@ TEST_F(MaintenanceManagerTest, TestCompletedOpsHistory) {
     manager_->GetMaintenanceManagerStatusDump(&status_pb);
     // The size should be at most the history_size.
     ASSERT_GE(kHistorySize, status_pb.completed_operations_size());
-    // See that we have the right name, even if we wrap around.
-    ASSERT_EQ(name, status_pb.completed_operations(i % 4).name());
+    // The most recently completed op should always be first, even if we wrap
+    // around.
+    ASSERT_EQ(name, status_pb.completed_operations(0).name());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d68a0c/src/kudu/util/maintenance_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/maintenance_manager.cc b/src/kudu/util/maintenance_manager.cc
index 7628fbb..11fa614 100644
--- a/src/kudu/util/maintenance_manager.cc
+++ b/src/kudu/util/maintenance_manager.cc
@@ -411,7 +411,11 @@ void MaintenanceManager::GetMaintenanceManagerStatusDump(MaintenanceManagerStatu
     }
   }
 
-  for (const CompletedOp& completed_op : completed_ops_) {
+  for (int n = 1; n <= completed_ops_.size(); n++) {
+    int i = completed_ops_count_ - n;
+    if (i < 0) break;
+    const auto& completed_op = completed_ops_[i % completed_ops_.size()];
+
     if (!completed_op.name.empty()) {
       MaintenanceManagerStatusPB_CompletedOpPB* completed_pb = out_pb->add_completed_operations();
       completed_pb->set_name(completed_op.name);