Merge branch 'main' into branch-3.5
diff --git a/.github/workflows/ci-pr-validation.yaml b/.github/workflows/ci-pr-validation.yaml
index 56309e9..d9dc2ca 100644
--- a/.github/workflows/ci-pr-validation.yaml
+++ b/.github/workflows/ci-pr-validation.yaml
@@ -29,8 +29,20 @@
 
 jobs:
 
+  formatting-check:
+    name: Formatting Check
+    runs-on: ubuntu-latest
+    steps:
+    - uses: actions/checkout@v3
+    - name: Run clang-format style check for C/C++/Protobuf programs.
+      uses: jidicula/clang-format-action@v4.11.0
+      with:
+        clang-format-version: '11'
+        exclude-regex: '.*\.(proto|hpp)'
+
   wireshark-dissector-build:
     name: Build the Wireshark dissector
+    needs: formatting-check
     runs-on: ${{ matrix.os }}
     timeout-minutes: 60
     strategy:
@@ -61,6 +73,7 @@
 
   unit-tests:
     name: Run unit tests
+    needs: formatting-check
     runs-on: ubuntu-22.04
     timeout-minutes: 120
 
@@ -81,8 +94,8 @@
           ./vcpkg/vcpkg format-manifest vcpkg.json
           if [[ $(git diff | wc -l) -gt 0 ]]; then
               echo "Please run `./vcpkg/vcpkg format-manifest vcpkg.json` to reformat vcpkg.json"
+              exit 1
           fi
-          make check-format
 
       - name: Build tests
         run: |
@@ -103,6 +116,7 @@
 
   cpp20-build:
     name: Build with the C++20 standard
+    needs: formatting-check
     runs-on: ubuntu-22.04
     timeout-minutes: 60
 
@@ -288,8 +302,8 @@
   cpp-build-macos:
     timeout-minutes: 120
     name: Build CPP Client on macOS
+    needs: formatting-check
     runs-on: macos-12
-    needs: unit-tests
     steps:
       - name: checkout
         uses: actions/checkout@v3
@@ -306,6 +320,12 @@
         run: |
           cmake --build ./build-macos --parallel --config Release
 
+      - name: Build with C++20
+        shell: bash
+        run: |
+          cmake -B build-macos-cpp20 -DCMAKE_CXX_STANDARD=20
+          cmake --build build-macos-cpp20 -j8
+
   cpp-build-macos-static:
     timeout-minutes: 120
     name: Build CPP Client on macOS with static dependencies
@@ -332,7 +352,7 @@
   check-completion:
     name: Check Completion
     runs-on: ubuntu-latest
-    needs: [wireshark-dissector-build, unit-tests, cpp20-build, cpp-build-windows, package, cpp-build-macos]
+    needs: [formatting-check, wireshark-dissector-build, unit-tests, cpp20-build, cpp-build-windows, package, cpp-build-macos]
 
     steps:
       - run: true
diff --git a/lib/ClientConnection.cc b/lib/ClientConnection.cc
index abd38b4..04202d3 100644
--- a/lib/ClientConnection.cc
+++ b/lib/ClientConnection.cc
@@ -1321,7 +1321,7 @@
     }
     // Remove the connection from the pool before completing any promise
     if (detach) {
-        pool_.remove(logicalAddress_ + "-" + std::to_string(poolIndex_), this);
+        pool_.remove(logicalAddress_, physicalAddress_, poolIndex_, this);
     }
 
     auto self = shared_from_this();
diff --git a/lib/ConnectionPool.cc b/lib/ConnectionPool.cc
index 4cc8883..9a614ed 100644
--- a/lib/ConnectionPool.cc
+++ b/lib/ConnectionPool.cc
@@ -66,6 +66,13 @@
     return true;
 }
 
+static const std::string getKey(const std::string& logicalAddress, const std::string& physicalAddress,
+                                size_t keySuffix) {
+    std::stringstream ss;
+    ss << logicalAddress << '-' << physicalAddress << '-' << keySuffix;
+    return ss.str();
+}
+
 Future<Result, ClientConnectionWeakPtr> ConnectionPool::getConnectionAsync(const std::string& logicalAddress,
                                                                            const std::string& physicalAddress,
                                                                            size_t keySuffix) {
@@ -77,9 +84,7 @@
 
     std::unique_lock<std::recursive_mutex> lock(mutex_);
 
-    std::stringstream ss;
-    ss << logicalAddress << '-' << keySuffix;
-    const std::string key = ss.str();
+    auto key = getKey(logicalAddress, physicalAddress, keySuffix);
 
     PoolMap::iterator cnxIt = pool_.find(key);
     if (cnxIt != pool_.end()) {
@@ -127,7 +132,9 @@
     return future;
 }
 
-void ConnectionPool::remove(const std::string& key, ClientConnection* value) {
+void ConnectionPool::remove(const std::string& logicalAddress, const std::string& physicalAddress,
+                            size_t keySuffix, ClientConnection* value) {
+    auto key = getKey(logicalAddress, physicalAddress, keySuffix);
     std::lock_guard<std::recursive_mutex> lock(mutex_);
     auto it = pool_.find(key);
     if (it != pool_.end() && it->second.get() == value) {
diff --git a/lib/ConnectionPool.h b/lib/ConnectionPool.h
index a51205b..e044d62 100644
--- a/lib/ConnectionPool.h
+++ b/lib/ConnectionPool.h
@@ -51,7 +51,8 @@
      */
     bool close();
 
-    void remove(const std::string& key, ClientConnection* value);
+    void remove(const std::string& logicalAddress, const std::string& physicalAddress, size_t keySuffix,
+                ClientConnection* value);
 
     /**
      * Get a connection from the pool.
diff --git a/lib/ConsumerImpl.cc b/lib/ConsumerImpl.cc
index fa33e28..ebc8518 100644
--- a/lib/ConsumerImpl.cc
+++ b/lib/ConsumerImpl.cc
@@ -236,16 +236,14 @@
     // sending the subscribe request.
     cnx->registerConsumer(consumerId_, get_shared_this_ptr());
 
-    if (duringSeek_) {
+    if (duringSeek()) {
         ackGroupingTrackerPtr_->flushAndClean();
     }
 
     Lock lockForMessageId(mutexForMessageId_);
-    // Update startMessageId so that we can discard messages after delivery restarts
-    const auto startMessageId = clearReceiveQueue();
+    clearReceiveQueue();
     const auto subscribeMessageId =
-        (subscriptionMode_ == Commands::SubscriptionModeNonDurable) ? startMessageId : boost::none;
-    startMessageId_ = startMessageId;
+        (subscriptionMode_ == Commands::SubscriptionModeNonDurable) ? startMessageId_.get() : boost::none;
     lockForMessageId.unlock();
 
     unAckedMessageTrackerPtr_->clear();
@@ -1048,14 +1046,21 @@
  * Clear the internal receiver queue and returns the message id of what was the 1st message in the queue that
  * was
  * not seen by the application
+ * `startMessageId_` is updated so that we can discard messages after delivery restarts.
  */
-boost::optional<MessageId> ConsumerImpl::clearReceiveQueue() {
-    bool expectedDuringSeek = true;
-    if (duringSeek_.compare_exchange_strong(expectedDuringSeek, false)) {
-        return seekMessageId_.get();
+void ConsumerImpl::clearReceiveQueue() {
+    if (duringSeek()) {
+        startMessageId_ = seekMessageId_.get();
+        SeekStatus expected = SeekStatus::COMPLETED;
+        if (seekStatus_.compare_exchange_strong(expected, SeekStatus::NOT_STARTED)) {
+            auto seekCallback = seekCallback_.release();
+            executor_->postWork([seekCallback] { seekCallback(ResultOk); });
+        }
+        return;
     } else if (subscriptionMode_ == Commands::SubscriptionModeDurable) {
-        return startMessageId_.get();
+        return;
     }
+
     Message nextMessageInQueue;
     if (incomingMessages_.peekAndClear(nextMessageInQueue)) {
         // There was at least one message pending in the queue
@@ -1071,16 +1076,12 @@
                                            .ledgerId(nextMessageId.ledgerId())
                                            .entryId(nextMessageId.entryId() - 1)
                                            .build();
-        return previousMessageId;
+        startMessageId_ = previousMessageId;
     } else if (lastDequedMessageId_ != MessageId::earliest()) {
         // If the queue was empty we need to restart from the message just after the last one that has been
         // dequeued
         // in the past
-        return lastDequedMessageId_;
-    } else {
-        // No message was received or dequeued by this consumer. Next message would still be the
-        // startMessageId
-        return startMessageId_.get();
+        startMessageId_ = lastDequedMessageId_;
     }
 }
 
@@ -1500,18 +1501,15 @@
 
 bool ConsumerImpl::isReadCompacted() { return readCompacted_; }
 
-inline bool hasMoreMessages(const MessageId& lastMessageIdInBroker, const MessageId& messageId) {
-    return lastMessageIdInBroker > messageId && lastMessageIdInBroker.entryId() != -1;
-}
-
 void ConsumerImpl::hasMessageAvailableAsync(HasMessageAvailableCallback callback) {
-    const auto startMessageId = startMessageId_.get();
-    Lock lock(mutexForMessageId_);
-    const auto messageId =
-        (lastDequedMessageId_ == MessageId::earliest()) ? startMessageId.value() : lastDequedMessageId_;
-
-    if (messageId == MessageId::latest()) {
-        lock.unlock();
+    bool compareMarkDeletePosition;
+    {
+        std::lock_guard<std::mutex> lock{mutexForMessageId_};
+        compareMarkDeletePosition =
+            (lastDequedMessageId_ == MessageId::earliest()) &&
+            (startMessageId_.get().value_or(MessageId::earliest()) == MessageId::latest());
+    }
+    if (compareMarkDeletePosition) {
         auto self = get_shared_this_ptr();
         getLastMessageIdAsync([self, callback](Result result, const GetLastMessageIdResponse& response) {
             if (result != ResultOk) {
@@ -1543,16 +1541,15 @@
             }
         });
     } else {
-        if (hasMoreMessages(lastMessageIdInBroker_, messageId)) {
-            lock.unlock();
+        if (hasMoreMessages()) {
             callback(ResultOk, true);
             return;
         }
-        lock.unlock();
-
-        getLastMessageIdAsync([callback, messageId](Result result, const GetLastMessageIdResponse& response) {
-            callback(result, (result == ResultOk) && hasMoreMessages(response.getLastMessageId(), messageId));
-        });
+        auto self = get_shared_this_ptr();
+        getLastMessageIdAsync(
+            [this, self, callback](Result result, const GetLastMessageIdResponse& response) {
+                callback(result, (result == ResultOk) && hasMoreMessages());
+            });
     }
 }
 
@@ -1656,9 +1653,18 @@
         return;
     }
 
+    auto expected = SeekStatus::NOT_STARTED;
+    if (!seekStatus_.compare_exchange_strong(expected, SeekStatus::IN_PROGRESS)) {
+        LOG_ERROR(getName() << " attempted to seek (" << seekId << ", " << timestamp << " when the status is "
+                            << static_cast<int>(expected));
+        callback(ResultNotAllowedError);
+        return;
+    }
+
     const auto originalSeekMessageId = seekMessageId_.get();
     seekMessageId_ = seekId;
-    duringSeek_ = true;
+    seekStatus_ = SeekStatus::IN_PROGRESS;
+    seekCallback_ = std::move(callback);
     if (timestamp > 0) {
         LOG_INFO(getName() << " Seeking subscription to " << timestamp);
     } else {
@@ -1682,12 +1688,19 @@
                 Lock lock(mutexForMessageId_);
                 lastDequedMessageId_ = MessageId::earliest();
                 lock.unlock();
+                if (getCnx().expired()) {
+                    // It's during reconnection, complete the seek future after connection is established
+                    seekStatus_ = SeekStatus::COMPLETED;
+                } else {
+                    startMessageId_ = seekMessageId_.get();
+                    seekCallback_.release()(result);
+                }
             } else {
                 LOG_ERROR(getName() << "Failed to seek: " << result);
                 seekMessageId_ = originalSeekMessageId;
-                duringSeek_ = false;
+                seekStatus_ = SeekStatus::NOT_STARTED;
+                seekCallback_.release()(result);
             }
-            callback(result);
         });
 }
 
diff --git a/lib/ConsumerImpl.h b/lib/ConsumerImpl.h
index 524acb8..82e323b 100644
--- a/lib/ConsumerImpl.h
+++ b/lib/ConsumerImpl.h
@@ -75,6 +75,13 @@
 const static std::string PROPERTY_ORIGIN_MESSAGE_ID = "ORIGIN_MESSAGE_ID";
 const static std::string DLQ_GROUP_TOPIC_SUFFIX = "-DLQ";
 
+enum class SeekStatus : std::uint8_t
+{
+    NOT_STARTED,
+    IN_PROGRESS,
+    COMPLETED
+};
+
 class ConsumerImpl : public ConsumerImplBase {
    public:
     ConsumerImpl(const ClientImplPtr client, const std::string& topic, const std::string& subscriptionName,
@@ -193,7 +200,7 @@
                                        const DeadlineTimerPtr& timer,
                                        BrokerGetLastMessageIdCallback callback);
 
-    boost::optional<MessageId> clearReceiveQueue();
+    void clearReceiveQueue();
     void seekAsyncInternal(long requestId, SharedBuffer seek, const MessageId& seekId, long timestamp,
                            ResultCallback callback);
     void processPossibleToDLQ(const MessageId& messageId, ProcessDLQCallBack cb);
@@ -239,10 +246,13 @@
     MessageId lastDequedMessageId_{MessageId::earliest()};
     MessageId lastMessageIdInBroker_{MessageId::earliest()};
 
-    std::atomic_bool duringSeek_{false};
+    std::atomic<SeekStatus> seekStatus_{SeekStatus::NOT_STARTED};
+    Synchronized<ResultCallback> seekCallback_{[](Result) {}};
     Synchronized<boost::optional<MessageId>> startMessageId_;
     Synchronized<MessageId> seekMessageId_{MessageId::earliest()};
 
+    bool duringSeek() const { return seekStatus_ != SeekStatus::NOT_STARTED; }
+
     class ChunkedMessageCtx {
        public:
         ChunkedMessageCtx() : totalChunks_(0) {}
@@ -332,6 +342,23 @@
                                                       const proto::MessageIdData& messageIdData,
                                                       const ClientConnectionPtr& cnx, MessageId& messageId);
 
+    bool hasMoreMessages() const {
+        std::lock_guard<std::mutex> lock{mutexForMessageId_};
+        if (lastMessageIdInBroker_.entryId() == -1L) {
+            return false;
+        }
+
+        const auto inclusive = config_.isStartMessageIdInclusive();
+        if (lastDequedMessageId_ == MessageId::earliest()) {
+            // If startMessageId_ is none, use latest so that this method will return false
+            const auto startMessageId = startMessageId_.get().value_or(MessageId::latest());
+            return inclusive ? (lastMessageIdInBroker_ >= startMessageId)
+                             : (lastMessageIdInBroker_ > startMessageId);
+        } else {
+            return lastMessageIdInBroker_ > lastDequedMessageId_;
+        }
+    }
+
     friend class PulsarFriend;
     friend class MultiTopicsConsumerImpl;
 
diff --git a/lib/MultiTopicsConsumerImpl.cc b/lib/MultiTopicsConsumerImpl.cc
index a0854cf..1484785 100644
--- a/lib/MultiTopicsConsumerImpl.cc
+++ b/lib/MultiTopicsConsumerImpl.cc
@@ -475,7 +475,7 @@
     };
     const auto state = state_.load();
     if (state == Closing || state == Closed) {
-        callback(ResultAlreadyClosed);
+        callback(ResultOk);
         return;
     }
 
@@ -488,7 +488,7 @@
     if (consumers.empty()) {
         LOG_DEBUG("TopicsConsumer have no consumers to close "
                   << " topic" << topic() << " subscription - " << subscriptionName_);
-        callback(ResultAlreadyClosed);
+        callback(ResultOk);
         return;
     }
 
diff --git a/lib/NamespaceName.cc b/lib/NamespaceName.cc
index f493db2..d635480 100644
--- a/lib/NamespaceName.cc
+++ b/lib/NamespaceName.cc
@@ -93,7 +93,7 @@
     return std::shared_ptr<NamespaceName>(this);
 }
 
-bool NamespaceName::operator==(const NamespaceName& namespaceName) {
+bool NamespaceName::operator==(const NamespaceName& namespaceName) const {
     return this->namespace_.compare(namespaceName.namespace_) == 0;
 }
 
diff --git a/lib/NamespaceName.h b/lib/NamespaceName.h
index ce451a2..d1f5765 100644
--- a/lib/NamespaceName.h
+++ b/lib/NamespaceName.h
@@ -37,7 +37,7 @@
     static std::shared_ptr<NamespaceName> get(const std::string& property, const std::string& cluster,
                                               const std::string& namespaceName);
     static std::shared_ptr<NamespaceName> get(const std::string& property, const std::string& namespaceName);
-    bool operator==(const NamespaceName& namespaceName);
+    bool operator==(const NamespaceName& namespaceName) const;
     bool isV2();
     std::string toString();
 
diff --git a/lib/Synchronized.h b/lib/Synchronized.h
index a98c08d..5449a9f 100644
--- a/lib/Synchronized.h
+++ b/lib/Synchronized.h
@@ -30,6 +30,11 @@
         return value_;
     }
 
+    T&& release() {
+        std::lock_guard<std::mutex> lock(mutex_);
+        return std::move(value_);
+    }
+
     Synchronized& operator=(const T& value) {
         std::lock_guard<std::mutex> lock(mutex_);
         value_ = value;
diff --git a/lib/TopicName.cc b/lib/TopicName.cc
index 5b892fc..487eee5 100644
--- a/lib/TopicName.cc
+++ b/lib/TopicName.cc
@@ -164,7 +164,7 @@
 
 std::string TopicName::getEncodedLocalName() const { return getEncodedName(localName_); }
 
-bool TopicName::operator==(const TopicName& other) {
+bool TopicName::operator==(const TopicName& other) const {
     return (this->topicName_.compare(other.topicName_) == 0);
 }
 
diff --git a/lib/TopicName.h b/lib/TopicName.h
index 8cc9cb5..bee8138 100644
--- a/lib/TopicName.h
+++ b/lib/TopicName.h
@@ -65,7 +65,7 @@
     NamespaceNamePtr getNamespaceName();
     int getPartitionIndex() const noexcept { return partition_; }
     static std::shared_ptr<TopicName> get(const std::string& topicName);
-    bool operator==(const TopicName& other);
+    bool operator==(const TopicName& other) const;
     static std::string getEncodedName(const std::string& nameBeforeEncoding);
     static std::string removeDomain(const std::string& topicName);
     static bool containsDomain(const std::string& topicName);
diff --git a/tests/BasicEndToEndTest.cc b/tests/BasicEndToEndTest.cc
index 5dbccbf..42c072a 100644
--- a/tests/BasicEndToEndTest.cc
+++ b/tests/BasicEndToEndTest.cc
@@ -1694,7 +1694,7 @@
     ASSERT_EQ(expected.str(), msgReceived.getDataAsString());
     ASSERT_EQ(ResultOk, consumer.acknowledge(msgReceived));
     ASSERT_EQ(ResultOk, consumer.unsubscribe());
-    ASSERT_EQ(ResultAlreadyClosed, consumer.close());
+    ASSERT_EQ(ResultOk, consumer.close());
     ASSERT_EQ(ResultOk, producer.close());
     ASSERT_EQ(ResultOk, client.close());
 }
diff --git a/tests/ReaderTest.cc b/tests/ReaderTest.cc
index 723972d..9d2fe5f 100644
--- a/tests/ReaderTest.cc
+++ b/tests/ReaderTest.cc
@@ -752,6 +752,8 @@
     producer.close();
 }
 
+class ReaderSeekTest : public ::testing::TestWithParam<bool> {};
+
 TEST(ReaderSeekTest, testStartAtLatestMessageId) {
     Client client(serviceUrl);
 
@@ -784,4 +786,57 @@
     producer.close();
 }
 
+TEST(ReaderTest, testSeekInProgress) {
+    Client client(serviceUrl);
+    const auto topic = "test-seek-in-progress-" + std::to_string(time(nullptr));
+    Reader reader;
+    ASSERT_EQ(ResultOk, client.createReader(topic, MessageId::earliest(), {}, reader));
+
+    reader.seekAsync(MessageId::earliest(), [](Result) {});
+    Promise<Result, Result> promise;
+    reader.seekAsync(MessageId::earliest(), [promise](Result result) { promise.setValue(result); });
+    Result result;
+    promise.getFuture().get(result);
+    ASSERT_EQ(result, ResultNotAllowedError);
+    client.close();
+}
+
+TEST_P(ReaderSeekTest, testHasMessageAvailableAfterSeekToEnd) {
+    Client client(serviceUrl);
+    const auto topic = "test-has-message-available-after-seek-to-end-" + std::to_string(time(nullptr));
+    Producer producer;
+    ASSERT_EQ(ResultOk, client.createProducer(topic, producer));
+    Reader reader;
+    ASSERT_EQ(ResultOk, client.createReader(topic, MessageId::earliest(), {}, reader));
+
+    producer.send(MessageBuilder().setContent("msg-0").build());
+    producer.send(MessageBuilder().setContent("msg-1").build());
+
+    bool hasMessageAvailable;
+    if (GetParam()) {
+        // Test the case when `ConsumerImpl.lastMessageIdInBroker_` has been initialized
+        ASSERT_EQ(ResultOk, reader.hasMessageAvailable(hasMessageAvailable));
+    }
+
+    ASSERT_EQ(ResultOk, reader.seek(MessageId::latest()));
+    ASSERT_EQ(ResultOk, reader.hasMessageAvailable(hasMessageAvailable));
+    ASSERT_FALSE(hasMessageAvailable);
+
+    producer.send(MessageBuilder().setContent("msg-2").build());
+    ASSERT_EQ(ResultOk, reader.hasMessageAvailable(hasMessageAvailable));
+    ASSERT_TRUE(hasMessageAvailable);
+
+    Message msg;
+    ASSERT_EQ(ResultOk, reader.readNext(msg, 1000));
+    ASSERT_EQ("msg-2", msg.getDataAsString());
+
+    // Test the 2nd seek
+    ASSERT_EQ(ResultOk, reader.seek(MessageId::latest()));
+    ASSERT_EQ(ResultOk, reader.hasMessageAvailable(hasMessageAvailable));
+    ASSERT_FALSE(hasMessageAvailable);
+
+    client.close();
+}
+
 INSTANTIATE_TEST_SUITE_P(Pulsar, ReaderTest, ::testing::Values(true, false));
+INSTANTIATE_TEST_SUITE_P(Pulsar, ReaderSeekTest, ::testing::Values(true, false));