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

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


The following commit(s) were added to refs/heads/master by this push:
     new a5674dd18 [consensus] clean-up on OpId utilities
a5674dd18 is described below

commit a5674dd186f8df5f8c7d67003af960513af7c0b3
Author: Alexey Serbin <ale...@apache.org>
AuthorDate: Fri Aug 4 17:47:30 2023 -0700

    [consensus] clean-up on OpId utilities
    
    This patch cleans up the code in opid_util.{cc,h} by removing unused
    methods and introducing human-readable operators (there are in-lined)
    to compare OpId instances.
    
    The MinimumOpId() and MaximumOpId() helper functions have been updated
    as well to return constant references to statically initialized
    constants instead of creating OpId instances every time, setting
    the term and index fields, and returning a copy.  Related call sites
    and methods of other classes have been updated correspondingly.
    
    Change-Id: Iea1e7fba9d91ac9527d88721a867a856e8a858b3
    Reviewed-on: http://gerrit.cloudera.org:8080/20320
    Reviewed-by: Abhishek Chennaka <achenn...@cloudera.com>
    Tested-by: Abhishek Chennaka <achenn...@cloudera.com>
    Tested-by: Alexey Serbin <ale...@apache.org>
    Reviewed-by: Mahesh Reddy <mre...@cloudera.com>
---
 src/kudu/consensus/consensus-test-util.h           |   4 +-
 src/kudu/consensus/consensus_queue.cc              |  28 +++---
 src/kudu/consensus/opid_util.cc                    | 110 ++++-----------------
 src/kudu/consensus/opid_util.h                     |  96 +++++++-----------
 src/kudu/consensus/pending_rounds.cc               |   4 +-
 src/kudu/consensus/pending_rounds.h                |   2 +-
 src/kudu/consensus/raft_consensus.cc               |  10 +-
 src/kudu/consensus/raft_consensus.h                |   2 +-
 src/kudu/consensus/raft_consensus_quorum-test.cc   |  24 ++++-
 src/kudu/integration-tests/raft_consensus-itest.cc |   2 +-
 src/kudu/tablet/tablet_bootstrap.cc                |   5 +-
 src/kudu/tablet/tablet_metadata.cc                 |   4 +-
 12 files changed, 101 insertions(+), 190 deletions(-)

diff --git a/src/kudu/consensus/consensus-test-util.h 
b/src/kudu/consensus/consensus-test-util.h
index 2a8c20576..5cf6003b6 100644
--- a/src/kudu/consensus/consensus-test-util.h
+++ b/src/kudu/consensus/consensus-test-util.h
@@ -51,7 +51,7 @@
   do { \
     const consensus::OpId& TOKENPASTE2(_left, __LINE__) = (left); \
     const consensus::OpId& TOKENPASTE2(_right, __LINE__) = (right); \
-    if (!consensus::OpIdEquals(TOKENPASTE2(_left, __LINE__), 
TOKENPASTE2(_right, __LINE__))) { \
+    if (TOKENPASTE2(_left, __LINE__) != TOKENPASTE2(_right, __LINE__)) { \
       FAIL() << "Expected: " \
             << pb_util::SecureShortDebugString(TOKENPASTE2(_left, __LINE__)) 
<< "\n" \
             << "Value: " \
@@ -354,7 +354,7 @@ class NoOpTestPeerProxy : public TestPeerProxy {
     response->Clear();
     {
       std::lock_guard<simple_spinlock> lock(lock_);
-      if (OpIdLessThan(last_received_, request.preceding_id())) {
+      if (last_received_ < request.preceding_id()) {
         ConsensusErrorPB* error = response->mutable_status()->mutable_error();
         error->set_code(ConsensusErrorPB::PRECEDING_ENTRY_DIDNT_MATCH);
         StatusToPB(Status::IllegalState(""), error->mutable_status());
diff --git a/src/kudu/consensus/consensus_queue.cc 
b/src/kudu/consensus/consensus_queue.cc
index 135e62e09..01368616a 100644
--- a/src/kudu/consensus/consensus_queue.cc
+++ b/src/kudu/consensus/consensus_queue.cc
@@ -29,7 +29,6 @@
 #include <utility>
 
 #include <gflags/gflags.h>
-#include <google/protobuf/stubs/port.h>
 
 #include "kudu/common/common.pb.h"
 #include "kudu/common/timestamp.h"
@@ -443,7 +442,7 @@ Status 
PeerMessageQueue::AppendOperations(vector<ReplicateRefPtr> msgs,
       }));
   lock.lock();
   DCHECK(last_id.IsInitialized());
-  queue_state_.last_appended = last_id;
+  queue_state_.last_appended = std::move(last_id);
   UpdateMetricsUnlocked();
 
   return Status::OK();
@@ -1080,13 +1079,14 @@ void PeerMessageQueue::PromoteIfNeeded(TrackedPeer* 
peer, const TrackedPeer& pre
     // time we contact them to try to promote them.
     if (prev_peer_state.last_received.index() == 0) return;
 
-    int64_t last_batch_size =
-        std::max<int64_t>(0, peer->last_received.index() - 
prev_peer_state.last_received.index());
-    bool peer_caught_up =
-        !OpIdEquals(status.last_received_current_leader(), MinimumOpId()) &&
+    int64_t last_batch_size = std::max<int64_t>(0, peer->last_received.index() 
-
+        prev_peer_state.last_received.index());
+    if (status.last_received_current_leader() == MinimumOpId() ||
         status.last_received_current_leader().index() + last_batch_size
-            >= queue_state_.committed_index;
-    if (!peer_caught_up) return;
+            < queue_state_.committed_index) {
+      // The peer hasn't caught up yet.
+      return;
+    }
 
     // TODO(mpercy): Implement a SafeToPromote() check to ensure that we only
     // try to promote a NON_VOTER to VOTER if we will be able to commit the
@@ -1127,10 +1127,9 @@ void PeerMessageQueue::TransferLeadershipIfNeeded(const 
TrackedPeer& peer,
     return;
   }
 
-  bool peer_caught_up =
-      !OpIdEquals(status.last_received_current_leader(), MinimumOpId()) &&
-      OpIdEquals(status.last_received_current_leader(), 
queue_state_.last_appended);
-  if (!peer_caught_up) {
+  if (status.last_received_current_leader() == MinimumOpId() ||
+      status.last_received_current_leader() != queue_state_.last_appended) {
+    // The peer hasn't caught up yet.
     return;
   }
 
@@ -1205,13 +1204,12 @@ bool PeerMessageQueue::ResponseFromPeer(const 
std::string& peer_uuid,
       PromoteIfNeeded(peer, prev_peer_state, status);
 
       TransferLeadershipIfNeeded(*peer, status);
-    } else if (!OpIdEquals(status.last_received_current_leader(), 
MinimumOpId())) {
+    } else if (status.last_received_current_leader() != MinimumOpId()) {
       // Their log may have diverged from ours, however we are in the process
       // of replicating our ops to them, so continue doing so. Eventually, we
       // will cause the divergent entry in their log to be overwritten.
       peer->last_received = status.last_received_current_leader();
       peer->next_index = peer->last_received.index() + 1;
-
     } else {
       // The peer is divergent and they have not (successfully) received
       // anything from us yet. Start sending from their last committed index.
@@ -1473,7 +1471,7 @@ bool PeerMessageQueue::IsOpInLog(const OpId& desired_op) 
const {
   OpId log_op;
   Status s = log_cache_.LookupOpId(desired_op.index(), &log_op);
   if (PREDICT_TRUE(s.ok())) {
-    return OpIdEquals(desired_op, log_op);
+    return desired_op == log_op;
   }
   if (PREDICT_TRUE(s.IsNotFound() || s.IsIncomplete())) {
     return false;
diff --git a/src/kudu/consensus/opid_util.cc b/src/kudu/consensus/opid_util.cc
index c76fcc27a..f29904d85 100644
--- a/src/kudu/consensus/opid_util.cc
+++ b/src/kudu/consensus/opid_util.cc
@@ -18,14 +18,13 @@
 #include "kudu/consensus/opid_util.h"
 
 #include <limits>
+#include <mutex>
 #include <ostream>
-#include <utility>
 
 #include <glog/logging.h>
 
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/consensus/opid.pb.h"
-#include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/substitute.h"
 
 namespace kudu {
@@ -35,99 +34,28 @@ const int64_t kMinimumTerm = 0;
 const int64_t kMinimumOpIdIndex = 0;
 const int64_t kInvalidOpIdIndex = -1;
 
-int OpIdCompare(const OpId& left, const OpId& right) {
-  DCHECK(left.IsInitialized());
-  DCHECK(right.IsInitialized());
-  if (PREDICT_TRUE(left.term() == right.term())) {
-    return left.index() < right.index() ? -1 : left.index() == right.index() ? 
0 : 1;
-  }
-  return left.term() < right.term() ? -1 : 1;
-}
-
-bool OpIdEquals(const OpId& left, const OpId& right) {
-  DCHECK(left.IsInitialized());
-  DCHECK(right.IsInitialized());
-  return left.term() == right.term() && left.index() == right.index();
-}
-
-bool OpIdLessThan(const OpId& left, const OpId& right) {
-  DCHECK(left.IsInitialized());
-  DCHECK(right.IsInitialized());
-  if (left.term() < right.term()) return true;
-  if (left.term() > right.term()) return false;
-  return left.index() < right.index();
-}
-
-bool OpIdBiggerThan(const OpId& left, const OpId& right) {
-  DCHECK(left.IsInitialized());
-  DCHECK(right.IsInitialized());
-  if (left.term() > right.term()) return true;
-  if (left.term() < right.term()) return false;
-  return left.index() > right.index();
-}
-
-bool CopyIfOpIdLessThan(const consensus::OpId& to_compare, consensus::OpId* 
target) {
-  if (to_compare.IsInitialized() &&
-      (!target->IsInitialized() || OpIdLessThan(to_compare, *target))) {
-    target->CopyFrom(to_compare);
-    return true;
-  }
-  return false;
-}
-
-size_t OpIdHashFunctor::operator() (const OpId& id) const {
-  return (id.term() + 31) ^ id.index();
-}
+const OpId& MinimumOpId() {
+  static std::once_flag once;
+  static OpId kMinOpId;
+  std::call_once(once, [&]() {
+    kMinOpId.set_term(0);
+    kMinOpId.set_index(0);
+  });
 
-bool OpIdEqualsFunctor::operator() (const OpId& left, const OpId& right) const 
{
-  return OpIdEquals(left, right);
+  return kMinOpId;
 }
 
-bool OpIdLessThanPtrFunctor::operator() (const OpId* left, const OpId* right) 
const {
-  return OpIdLessThan(*left, *right);
-}
-
-bool OpIdIndexLessThanPtrFunctor::operator() (const OpId* left, const OpId* 
right) const {
-  return left->index() < right->index();
-}
-
-bool OpIdCompareFunctor::operator() (const OpId& left, const OpId& right) 
const {
-  return OpIdLessThan(left, right);
-}
-
-bool OpIdBiggerThanFunctor::operator() (const OpId& left, const OpId& right) 
const {
-  return OpIdBiggerThan(left, right);
-}
+const OpId& MaximumOpId() {
+  static std::once_flag once;
+  static OpId kMaxOpId;
+  std::call_once(once, [&] {
+    kMaxOpId.set_term(std::numeric_limits<int64_t>::max());
+    kMaxOpId.set_index(std::numeric_limits<int64_t>::max());
+  });
 
-OpId MinimumOpId() {
-  OpId op_id;
-  op_id.set_term(0);
-  op_id.set_index(0);
-  return op_id;
+  return kMaxOpId;
 }
 
-OpId MaximumOpId() {
-  OpId op_id;
-  op_id.set_term(std::numeric_limits<int64_t>::max());
-  op_id.set_index(std::numeric_limits<int64_t>::max());
-  return op_id;
-}
-
-// helper hash functor for delta store ids
-struct DeltaIdHashFunction {
-  size_t operator()(const std::pair<int64_t, int64_t >& id) const {
-    return (id.first + 31) ^ id.second;
-  }
-};
-
-// helper equals functor for delta store ids
-struct DeltaIdEqualsTo {
-  bool operator()(const std::pair<int64_t, int64_t >& left,
-                  const std::pair<int64_t, int64_t >& right) const {
-    return left.first == right.first && left.second == right.second;
-  }
-};
-
 std::ostream& operator<<(std::ostream& os, const consensus::OpId& op_id) {
   os << OpIdToString(op_id);
   return os;
@@ -142,7 +70,7 @@ std::string OpIdToString(const OpId& id) {
 
 std::string OpsRangeString(const ConsensusRequestPB& req) {
   std::string ret;
-  ret.reserve(100);
+  ret.reserve(82);  // 4 * 19 + 3 + 2 + 1
   ret.push_back('[');
   if (req.ops_size() > 0) {
     const OpId& first_op = req.ops(0).id();
@@ -165,4 +93,4 @@ OpId MakeOpId(int64_t term, int64_t index) {
 }
 
 } // namespace consensus
-}  // namespace kudu
+} // namespace kudu
diff --git a/src/kudu/consensus/opid_util.h b/src/kudu/consensus/opid_util.h
index 507b36912..c45975e4c 100644
--- a/src/kudu/consensus/opid_util.h
+++ b/src/kudu/consensus/opid_util.h
@@ -15,19 +15,18 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef KUDU_CONSENSUS_OPID_UTIL_H_
-#define KUDU_CONSENSUS_OPID_UTIL_H_
+#pragma once
 
-#include <cstddef>
 #include <cstdint>
 #include <iosfwd>
 #include <string>
 
+#include "kudu/consensus/opid.pb.h"
+
 namespace kudu {
 namespace consensus {
 
 class ConsensusRequestPB;
-class OpId;
 
 // Minimum possible term.
 extern const int64_t kMinimumTerm;
@@ -38,68 +37,43 @@ extern const int64_t kMinimumOpIdIndex;
 // Log index that is lower than the minimum index (and so will never occur).
 extern const int64_t kInvalidOpIdIndex;
 
-// Returns true iff left == right.
-bool OpIdEquals(const OpId& left, const OpId& right);
-
-// Returns true iff left < right.
-bool OpIdLessThan(const OpId& left, const OpId& right);
-
-// Returns true iff left > right.
-bool OpIdBiggerThan(const OpId& left, const OpId& right);
-
-// Copies to_compare into target under the following conditions:
-// - If to_compare is initialized and target is not.
-// - If they are both initialized and to_compare is less than target.
-// Otherwise, does nothing.
-// If to_compare is copied into target, returns true, else false.
-bool CopyIfOpIdLessThan(const OpId& to_compare, OpId* target);
-
-// Return -1 if left < right,
-//         0 if equal,
-//         1 if left > right.
-int OpIdCompare(const OpId& left, const OpId& right);
-
-// OpId hash functor. Suitable for use with std::unordered_map.
-struct OpIdHashFunctor {
-  size_t operator() (const OpId& id) const;
-};
-
-// OpId equals functor. Suitable for use with std::unordered_map.
-struct OpIdEqualsFunctor {
-  bool operator() (const OpId& left, const OpId& right) const;
-};
-
-// OpId less than functor for pointers.. Suitable for use with std::sort and 
std::map.
-struct OpIdLessThanPtrFunctor {
-  // Returns true iff left < right.
-  bool operator() (const OpId* left, const OpId* right) const;
-};
-
-// Sorts op id's by index only, disregarding the term.
-struct OpIdIndexLessThanPtrFunctor {
-  // Returns true iff left.index() < right.index().
-  bool operator() (const OpId* left, const OpId* right) const;
-};
-
-// OpId compare() functor. Suitable for use with std::sort and std::map.
-struct OpIdCompareFunctor {
-  // Returns true iff left < right.
-  bool operator() (const OpId& left, const OpId& right) const;
-};
-
-// OpId comparison functor that returns true iff left > right. Suitable for use
-// with std::sort and std::map to sort keys in increasing order.
-struct OpIdBiggerThanFunctor {
-  bool operator() (const OpId& left, const OpId& right) const;
-};
+inline bool operator==(const OpId& lhs, const OpId& rhs) {
+  return lhs.term() == rhs.term() && lhs.index() == rhs.index();
+}
+
+inline bool operator!=(const OpId& lhs, const OpId& rhs) {
+  return !(lhs == rhs);
+}
+
+inline bool operator<(const OpId& lhs, const OpId& rhs) {
+  if (lhs.term() < rhs.term()) {
+    return true;
+  }
+  if (lhs.term() > rhs.term()) {
+    return false;
+  }
+  return lhs.index() < rhs.index();
+}
+
+inline bool operator<=(const OpId& lhs, const OpId& rhs) {
+  return lhs < rhs || lhs == rhs;
+}
+
+inline bool operator>(const OpId& lhs, const OpId& rhs) {
+  return !(lhs <= rhs);
+}
+
+inline bool operator>=(const OpId& lhs, const OpId& rhs) {
+  return !(lhs < rhs);
+}
 
 std::ostream& operator<<(std::ostream& os, const consensus::OpId& op_id);
 
 // Return the minimum possible OpId.
-OpId MinimumOpId();
+const OpId& MinimumOpId();
 
 // Return the maximum possible OpId.
-OpId MaximumOpId();
+const OpId& MaximumOpId();
 
 std::string OpIdToString(const OpId& id);
 
@@ -109,5 +83,3 @@ OpId MakeOpId(int64_t term, int64_t index);
 
 }  // namespace consensus
 }  // namespace kudu
-
-#endif /* KUDU_CONSENSUS_OPID_UTIL_H_ */
diff --git a/src/kudu/consensus/pending_rounds.cc 
b/src/kudu/consensus/pending_rounds.cc
index cd275dc1c..80a5d8660 100644
--- a/src/kudu/consensus/pending_rounds.cc
+++ b/src/kudu/consensus/pending_rounds.cc
@@ -133,7 +133,7 @@ bool PendingRounds::IsOpCommittedOrPending(const OpId& 
op_id, bool* term_mismatc
   return true;
 }
 
-OpId PendingRounds::GetLastPendingOpOpId() const {
+const OpId& PendingRounds::GetLastPendingOpOpId() const {
   return pending_ops_.empty()
       ? MinimumOpId() : (--pending_ops_.end())->second->id();
 }
@@ -175,7 +175,7 @@ Status PendingRounds::AdvanceCommittedIndex(int64_t 
committed_index) {
     DCHECK(round);
     const OpId& current_id = round->id();
 
-    if (PREDICT_TRUE(!OpIdEquals(last_committed_op_id_, MinimumOpId()))) {
+    if (PREDICT_TRUE(last_committed_op_id_ != MinimumOpId())) {
       CHECK_OK(CheckOpInSequence(last_committed_op_id_, current_id));
     }
 
diff --git a/src/kudu/consensus/pending_rounds.h 
b/src/kudu/consensus/pending_rounds.h
index 530db2f0e..e3f0e4376 100644
--- a/src/kudu/consensus/pending_rounds.h
+++ b/src/kudu/consensus/pending_rounds.h
@@ -75,7 +75,7 @@ class PendingRounds {
 
   // Returns the id of the latest pending op (i.e. the one with the latest
   // index). This must be called under the lock.
-  OpId GetLastPendingOpOpId() const;
+  const OpId& GetLastPendingOpOpId() const;
 
   // Used by replicas to cancel pending ops. Pending ops are those that have
   // completed prepare/replicate but are waiting on the LEADER's commit to
diff --git a/src/kudu/consensus/raft_consensus.cc 
b/src/kudu/consensus/raft_consensus.cc
index 16a16a0e7..9d227e8b3 100644
--- a/src/kudu/consensus/raft_consensus.cc
+++ b/src/kudu/consensus/raft_consensus.cc
@@ -1180,7 +1180,7 @@ void 
RaftConsensus::DeduplicateLeaderRequestUnlocked(const ConsensusRequestPB* r
 
       // If the OpIds match, i.e. if they have the same term and id, then this 
is just
       // duplicate, we skip...
-      if (OpIdEquals(round->replicate_msg()->id(), leader_msg->id())) {
+      if (round->replicate_msg()->id() == leader_msg->id()) {
         VLOG_WITH_PREFIX_UNLOCKED(2) << "Skipping op id " << leader_msg->id()
                                      << " (already replicated)";
         deduplicated_req->preceding_opid = &leader_msg->id();
@@ -1256,8 +1256,8 @@ Status 
RaftConsensus::EnforceLogMatchingPropertyMatchesUnlocked(const LeaderRequ
                              Status::IllegalState(error_msg));
 
   // Adding a check to eliminate an unnecessary log message in the
-  // scenario where this is the first message from the Leader of a new tablet.
-  if (!OpIdEquals(MakeOpId(1,1), *req.preceding_opid)) {
+  // scenario where this is the first message from the leader of a new tablet.
+  if (MakeOpId(1, 1) != *req.preceding_opid) {
     LOG_WITH_PREFIX_UNLOCKED(INFO) << "Refusing update from remote peer "
                                    << req.leader_uuid << ": " << error_msg;
   }
@@ -1830,8 +1830,8 @@ Status RaftConsensus::RequestVote(const VoteRequestPB* 
request,
 
   // Candidate must have last-logged OpId at least as large as our own to get
   // our vote.
-  bool vote_yes = !OpIdLessThan(request->candidate_status().last_received(),
-                                local_last_logged_opid);
+  const bool vote_yes =
+      request->candidate_status().last_received() >= local_last_logged_opid;
 
   // Record the term advancement if necessary. We don't do so in the case of
   // pre-elections because it's possible that the node who called the 
pre-election
diff --git a/src/kudu/consensus/raft_consensus.h 
b/src/kudu/consensus/raft_consensus.h
index 7e42a3993..47e86f814 100644
--- a/src/kudu/consensus/raft_consensus.h
+++ b/src/kudu/consensus/raft_consensus.h
@@ -1043,7 +1043,7 @@ class ConsensusRound : public 
RefCountedThreadSafe<ConsensusRound> {
 
   // Returns the id of the (replicate) operation this context
   // refers to. This is only set _after_ RaftConsensus::Replicate(context).
-  OpId id() const {
+  const OpId& id() const {
     return replicate_msg_->get()->id();
   }
 
diff --git a/src/kudu/consensus/raft_consensus_quorum-test.cc 
b/src/kudu/consensus/raft_consensus_quorum-test.cc
index 6275fd863..ee613e1fd 100644
--- a/src/kudu/consensus/raft_consensus_quorum-test.cc
+++ b/src/kudu/consensus/raft_consensus_quorum-test.cc
@@ -16,6 +16,7 @@
 // under the License.
 
 #include <algorithm>
+#include <cstddef>
 #include <cstdint>
 #include <functional>
 #include <memory>
@@ -333,7 +334,7 @@ class RaftConsensusQuorumTest : public KuduTest {
     shared_ptr<RaftConsensus> peer;
     CHECK_OK(peers_->GetPeerByIdx(peer_idx, &peer));
     while (true) {
-      if (OpIdCompare(peer->queue_->GetLastOpIdInLog(), to_wait_for) >= 0) {
+      if (peer->queue_->GetLastOpIdInLog() >= to_wait_for) {
         return;
       }
       SleepFor(MonoDelta::FromMilliseconds(1));
@@ -518,9 +519,22 @@ class RaftConsensusQuorumTest : public KuduTest {
     }
   }
 
-  void VerifyNoCommitsBeforeReplicates(const LogEntries& entries) {
-    std::unordered_set<OpId, OpIdHashFunctor, OpIdEqualsFunctor> 
replication_ops;
+  static void VerifyNoCommitsBeforeReplicates(const LogEntries& entries) {
+    // OpId equals functor for std::unordered_map.
+    struct OpIdEqualsFunctor {
+      bool operator()(const OpId& lhs, const OpId& rhs) const {
+        return lhs == rhs;
+      }
+    };
 
+    // OpId hash functor for std::unordered_map.
+    struct OpIdHashFunctor {
+      size_t operator()(const OpId& id) const {
+        return (id.term() + 31) ^ id.index();
+      }
+    };
+
+    std::unordered_set<OpId, OpIdHashFunctor, OpIdEqualsFunctor> 
replication_ops;
     for (const auto& entry : entries) {
       if (entry->has_replicate()) {
         ASSERT_TRUE(InsertIfNotPresent(&replication_ops, 
entry->replicate().id()))
@@ -985,7 +999,7 @@ TEST_F(RaftConsensusQuorumTest, 
TestReplicasEnforceTheLogMatchingProperty) {
   // Appending this message to peer0 should work and update
   // its 'last_received' to 'id'.
   ASSERT_OK(follower->Update(&req, &resp));
-  ASSERT_TRUE(OpIdEquals(resp.status().last_received(), *id));
+  ASSERT_EQ(resp.status().last_received(), *id);
   ASSERT_EQ(0, follower->queue_->metrics_.num_ops_behind_leader->value());
 
   // Now skip one message in the same term. The replica should
@@ -1019,7 +1033,7 @@ TEST_F(RaftConsensusQuorumTest, TestRequestVote) {
   WaitForCommitIfNotAlreadyPresent(last_op_id.index(), 1, 2);
 
   // Ensure last-logged OpId is > (0,0).
-  ASSERT_TRUE(OpIdLessThan(MinimumOpId(), last_op_id));
+  ASSERT_LT(MinimumOpId(), last_op_id);
 
   const int kPeerIndex = 1;
   shared_ptr<RaftConsensus> peer;
diff --git a/src/kudu/integration-tests/raft_consensus-itest.cc 
b/src/kudu/integration-tests/raft_consensus-itest.cc
index 07eff7105..9d5c1526f 100644
--- a/src/kudu/integration-tests/raft_consensus-itest.cc
+++ b/src/kudu/integration-tests/raft_consensus-itest.cc
@@ -3026,7 +3026,7 @@ TEST_F(RaftConsensusITest, TestLogIOErrorIsFatal) {
     ASSERT_OK(GetLastOpIdForReplica(tablet_id_, tservers[0], 
consensus::RECEIVED_OPID,
                                     MonoDelta::FromSeconds(10), &cur_opid));
     VLOG(1) << "Current OpId on server 0: " << OpIdToString(cur_opid);
-  } while (consensus::OpIdEquals(prev_opid, cur_opid));
+  } while (prev_opid == cur_opid);
   workload.StopAndJoin();
   ASSERT_OK(cluster_->SetFlag(ext_tservers[0],
             "log_inject_io_error_on_append_fraction", "1.0"));
diff --git a/src/kudu/tablet/tablet_bootstrap.cc 
b/src/kudu/tablet/tablet_bootstrap.cc
index a5af0f0ec..9fa32fc85 100644
--- a/src/kudu/tablet/tablet_bootstrap.cc
+++ b/src/kudu/tablet/tablet_bootstrap.cc
@@ -115,7 +115,6 @@ using kudu::consensus::ConsensusBootstrapInfo;
 using kudu::consensus::MinimumOpId;
 using kudu::consensus::NO_OP;
 using kudu::consensus::OpId;
-using kudu::consensus::OpIdEquals;
 using kudu::consensus::OpIdToString;
 using kudu::consensus::OperationType;
 using kudu::consensus::OperationType_Name;
@@ -967,7 +966,7 @@ Status TabletBootstrap::HandleReplicateMessage(ReplayState* 
state,
     const auto& existing_entry = existing_entry_iter->second;
 
     auto iter = state->pending_replicates.lower_bound(index);
-    DCHECK(OpIdEquals(iter->second->replicate().id(), 
existing_entry->replicate().id()));
+    DCHECK(iter->second->replicate().id() == existing_entry->replicate().id());
 
     const auto& last_entry = state->pending_replicates.rbegin()->second;
     VLOG_WITH_PREFIX(1) << "Overwriting operations starting at: "
@@ -1090,7 +1089,7 @@ Status TabletBootstrap::ApplyCommitMessage(const 
IOContext* io_context,
   if (PREDICT_TRUE(pending_replicate_entry)) {
     // We found a replicate with the same index, make sure it also has the 
same term.
     const auto& replicate = pending_replicate_entry->replicate();
-    if (PREDICT_FALSE(!OpIdEquals(committed_op_id, replicate.id()))) {
+    if (PREDICT_FALSE(committed_op_id != replicate.id())) {
       string error_msg = Substitute("Committed operation's OpId: $0 didn't 
match the"
           "commit message's committed OpId: $1. Pending operation: $2, Commit 
message: $3",
           SecureShortDebugString(replicate.id()),
diff --git a/src/kudu/tablet/tablet_metadata.cc 
b/src/kudu/tablet/tablet_metadata.cc
index 979958edb..3aae9500f 100644
--- a/src/kudu/tablet/tablet_metadata.cc
+++ b/src/kudu/tablet/tablet_metadata.cc
@@ -491,7 +491,7 @@ Status TabletMetadata::LoadFromSuperBlock(const 
TabletSuperBlockPB& superblock)
     // consider it equal to "not present".
     if (superblock.has_tombstone_last_logged_opid() &&
         superblock.tombstone_last_logged_opid().IsInitialized() &&
-        !OpIdEquals(MinimumOpId(), superblock.tombstone_last_logged_opid())) {
+        superblock.tombstone_last_logged_opid() != MinimumOpId()) {
       tombstone_last_logged_opid_ = superblock.tombstone_last_logged_opid();
     } else {
       tombstone_last_logged_opid_.reset();
@@ -771,7 +771,7 @@ Status 
TabletMetadata::ToSuperBlockUnlocked(TabletSuperBlockPB* super_block,
 
   pb.set_tablet_data_state(tablet_data_state_);
   if (tombstone_last_logged_opid_ &&
-      !OpIdEquals(MinimumOpId(), *tombstone_last_logged_opid_)) {
+      *tombstone_last_logged_opid_ != MinimumOpId()) {
     *pb.mutable_tombstone_last_logged_opid() = *tombstone_last_logged_opid_;
   }
 

Reply via email to