You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@kvrocks.apache.org by "ellutionist (via GitHub)" <gi...@apache.org> on 2023/05/04 12:45:46 UTC

[GitHub] [incubator-kvrocks] ellutionist opened a new pull request, #1418: support sync migration

ellutionist opened a new pull request, #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418

   This PR closes [1350](https://github.com/apache/incubator-kvrocks/issues/1350).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] git-hulk commented on pull request #1418: Support blocking migration for the cluster migrate command

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1545005100

   Looks like the migration test case has a segment fault. https://github.com/apache/incubator-kvrocks/actions/runs/4951001502/jobs/8855626634?pr=1418


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] ellutionist commented on a diff in pull request #1418: support sync migration

Posted by "ellutionist (via GitHub)" <gi...@apache.org>.
ellutionist commented on code in PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#discussion_r1187592915


##########
src/commands/cmd_cluster.cc:
##########
@@ -126,11 +127,28 @@ class CommandClusterX : public Commander {
     if (subcommand_ == "setnodeid" && args_.size() == 3 && args_[2].size() == kClusterNodeIdLen) return Status::OK();
 
     if (subcommand_ == "migrate") {
-      if (args.size() != 4) return {Status::RedisParseErr, errWrongNumOfArguments};
+      if (args.size() < 4) return {Status::RedisParseErr, errWrongNumOfArguments};
 
       slot_ = GET_OR_RET(ParseInt<int64_t>(args[2], 10));
 
       dst_node_id_ = args[3];
+
+      if (args.size() >= 5) {
+        auto sync_flag = util::ToLower(args[4]);
+        if (sync_flag == "async") {
+          sync_migrate_ = false;
+        } else if (sync_flag == "sync") {
+          sync_migrate_ = true;
+
+          if (args.size() == 6) {
+            sync_migrate_timeout_ = GET_OR_RET(ParseInt<int64_t>(args[5], 10));
+          } else if (args.size() > 6) {
+            return {Status::RedisParseErr, "Wrong number of arguments for MIGRATE SYNC option"};
+          }
+        } else {
+          return {Status::RedisParseErr, "Invalid sync flag"};
+        }
+      }

Review Comment:
   There will be some duplicate code checking the `sync_flag` like:
   ```c++
   if (args.size() == 5) {
    // check sync_flag = args[4]
   }
   
   if (args.size() == 6) {
    // check sync_flag = args[4]
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] ellutionist commented on pull request #1418: support sync migration

Posted by "ellutionist (via GitHub)" <gi...@apache.org>.
ellutionist commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1536378582

   > @ellutionist Thanks for your explanation.
   > 
   > I have not dived into these changes deeply, but if you just want to make lifetime of SyncMigrateContext as long as the command object, you can store an instance of SyncMigrateContext in CommandClusterX directly (`class CommandClusterX { SyncMigrateContext ctx_; ... }`), and pass some non-owning pointer (like raw pointer, no ownership) to the migrator.
   
   @PragmaTwice @xiaobiaozhao Thank you for the advice. This is another reasonable solution. However, there are two things to note here:
   1. The `CommandClusterX` does not always have a `SyncMigrateContext` (only when migrating with a "sync" flag). Making the context a non-pointer member of the `CommandClusterX` object may not make good sense here. If we decide to do this, I suggest to adopt `std::optional<SyncMigrateContext>`.
   2. Due to the concern of memory safety, I am not a fan of raw pointers. Of course, in this PR we could use it, but we may set up some potential risk for the future. Because not every maintainer could get to know  that "the`CommandClusterX` object must live long enough to complete the migration".


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] xiaobiaozhao commented on a diff in pull request #1418: support sync migration

Posted by "xiaobiaozhao (via GitHub)" <gi...@apache.org>.
xiaobiaozhao commented on code in PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#discussion_r1186022617


##########
src/commands/cmd_cluster.cc:
##########
@@ -126,11 +127,28 @@ class CommandClusterX : public Commander {
     if (subcommand_ == "setnodeid" && args_.size() == 3 && args_[2].size() == kClusterNodeIdLen) return Status::OK();
 
     if (subcommand_ == "migrate") {
-      if (args.size() != 4) return {Status::RedisParseErr, errWrongNumOfArguments};
+      if (args.size() < 4) return {Status::RedisParseErr, errWrongNumOfArguments};

Review Comment:
   I think we can do a strong check here
   eg : args.size() must be 5/6



##########
src/cluster/slot_migrate.h:
##########
@@ -80,8 +82,8 @@ class SlotMigrator : public redis::Database {
   ~SlotMigrator();
 
   Status CreateMigrationThread();
-  Status PerformSlotMigration(const std::string &node_id, std::string &dst_ip, int dst_port, int slot_id, int speed,
-                              int pipeline_size, int seq_gap);
+  Status PerformSlotMigration(const std::string &node_id, std::string &dst_ip, int dst_port, int slot_id,
+                              const std::shared_ptr<SyncMigrateContext> &blocking_ctx = nullptr);

Review Comment:
   I think std::shared_ptr<T> blocking_ctx is enough



##########
src/commands/cmd_cluster.cc:
##########
@@ -126,11 +127,28 @@ class CommandClusterX : public Commander {
     if (subcommand_ == "setnodeid" && args_.size() == 3 && args_[2].size() == kClusterNodeIdLen) return Status::OK();
 
     if (subcommand_ == "migrate") {
-      if (args.size() != 4) return {Status::RedisParseErr, errWrongNumOfArguments};
+      if (args.size() < 4) return {Status::RedisParseErr, errWrongNumOfArguments};
 
       slot_ = GET_OR_RET(ParseInt<int64_t>(args[2], 10));
 
       dst_node_id_ = args[3];
+
+      if (args.size() >= 5) {
+        auto sync_flag = util::ToLower(args[4]);
+        if (sync_flag == "async") {
+          sync_migrate_ = false;
+        } else if (sync_flag == "sync") {
+          sync_migrate_ = true;
+
+          if (args.size() == 6) {
+            sync_migrate_timeout_ = GET_OR_RET(ParseInt<int64_t>(args[5], 10));
+          } else if (args.size() > 6) {
+            return {Status::RedisParseErr, "Wrong number of arguments for MIGRATE SYNC option"};
+          }
+        } else {
+          return {Status::RedisParseErr, "Invalid sync flag"};
+        }
+      }

Review Comment:
   I think it's a little bit clearer
   
   if (args.size() == 5) {
      ...
   }
   
   if (args.size() == 6) {
      ...
   }



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] mapleFU commented on a diff in pull request #1418: Support blocking migration for the cluster migrate command

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on code in PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#discussion_r1189367973


##########
src/cluster/sync_migrate_context.cc:
##########
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include "cluster/sync_migrate_context.h"
+
+SyncMigrateContext::~SyncMigrateContext() {
+  if (timer_) {
+    event_free(timer_);
+    timer_ = nullptr;
+  }
+}
+
+void SyncMigrateContext::StartBlock() {
+  auto bev = conn_->GetBufferEvent();
+  bufferevent_setcb(bev, nullptr, WriteCB, EventCB, this);
+
+  if (timeout_) {
+    timer_ = evtimer_new(bufferevent_get_base(bev), TimerCB, this);
+    timeval tm = {timeout_, 0};
+    evtimer_add(timer_, &tm);
+  }
+}
+
+void SyncMigrateContext::Wakeup(const Status &migrate_result) {
+  migrate_result_ = migrate_result;
+  auto s = conn_->Owner()->EnableWriteEvent(conn_->GetFD());
+  if (!s.IsOK()) {
+    LOG(ERROR) << "[server] Failed to enable write event on the sync migrate connection " << conn_->GetFD() << ": "
+               << s.Msg();
+  }
+}
+
+void SyncMigrateContext::EventCB(bufferevent *bev, int16_t events, void *ctx) {
+  auto self = reinterpret_cast<SyncMigrateContext *>(ctx);
+  auto &&slot_migrator = self->svr_->slot_migrator;
+
+  if (events & (BEV_EVENT_EOF | BEV_EVENT_ERROR)) {
+    if (self->timer_ != nullptr) {
+      event_free(self->timer_);
+      self->timer_ = nullptr;
+    }
+
+    slot_migrator->CancelBlocking();
+  }
+  redis::Connection::OnEvent(bev, events, self->conn_);
+}
+
+void SyncMigrateContext::TimerCB(int, int16_t events, void *ctx) {
+  auto self = reinterpret_cast<SyncMigrateContext *>(ctx);
+  auto &&slot_migrator = self->svr_->slot_migrator;

Review Comment:
   ditto



##########
src/cluster/sync_migrate_context.cc:
##########
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include "cluster/sync_migrate_context.h"
+
+SyncMigrateContext::~SyncMigrateContext() {
+  if (timer_) {
+    event_free(timer_);
+    timer_ = nullptr;
+  }
+}
+
+void SyncMigrateContext::StartBlock() {
+  auto bev = conn_->GetBufferEvent();
+  bufferevent_setcb(bev, nullptr, WriteCB, EventCB, this);
+
+  if (timeout_) {
+    timer_ = evtimer_new(bufferevent_get_base(bev), TimerCB, this);
+    timeval tm = {timeout_, 0};
+    evtimer_add(timer_, &tm);
+  }
+}
+
+void SyncMigrateContext::Wakeup(const Status &migrate_result) {
+  migrate_result_ = migrate_result;
+  auto s = conn_->Owner()->EnableWriteEvent(conn_->GetFD());
+  if (!s.IsOK()) {
+    LOG(ERROR) << "[server] Failed to enable write event on the sync migrate connection " << conn_->GetFD() << ": "
+               << s.Msg();
+  }
+}
+
+void SyncMigrateContext::EventCB(bufferevent *bev, int16_t events, void *ctx) {
+  auto self = reinterpret_cast<SyncMigrateContext *>(ctx);
+  auto &&slot_migrator = self->svr_->slot_migrator;

Review Comment:
   Why using `auto &&` here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on pull request #1418: support sync migration

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1536401895

   > The CommandClusterX does not always have a SyncMigrateContext (only when migrating with a "sync" flag). Making the context a non-pointer member of the CommandClusterX object may not make good sense here. If we decide to do this, I suggest to adopt std::optional<SyncMigrateContext>.
   
   Sure, you can use something like `std::optional` or `std::unique_ptr`.
   
   > Due to the concern of memory safety, I am not a fan of raw pointers. Of course, in this PR we could use it, but we may set up some potential risk for the future. Because not every maintainer could get to know that "theCommandClusterX object must live long enough to complete the migration". If some change is made to "how the command objects work" in the future, it may lead to crash.
   
   I think in kvrock, wse use `std::shared_ptr` if and only if the object need to be **"shared"** between multiple references, and the lifetime of this object cannot be determined **statically** (e.g. given an object `o` and two reference `a` and `b`, the lifetime of `o` is equal to the maximum of lifetime of `a` and `b`, which need to be determined **dynamically**).
   
   So currently, just *"in the future ..."* seems not to be an accredited reason, since if this reason holds, we may need to change many raw pointer in the codebase to `shared_ptr`. If you just need a non-owning pointer (or called an observer/view/span, in modern C++), raw pointer is currently the available choice, since [`std::observer_ptr`](https://en.cppreference.com/w/cpp/experimental/observer_ptr) or `std::optional<T&>` (optional with reference type) is not in the standard.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] xiaobiaozhao commented on pull request #1418: support sync migration

Posted by "xiaobiaozhao (via GitHub)" <gi...@apache.org>.
xiaobiaozhao commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1536341427

   > @ellutionist Thanks for your explanation.
   > 
   > I have not dived into these changes deeply, but if you just want to make lifetime of SyncMigrateContext as long as the command object, you can store an instance of SyncMigrateContext in CommandClusterX directly (`class CommandClusterX { SyncMigrateContext ctx_; ... }`), and pass some non-owning pointer (like raw pointer, no ownership) to the migrator.
   
   I agree with that


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] git-hulk commented on pull request #1418: Support blocking migration for the cluster migrate command

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1541290041

   Hi @ellutionist Need to use `./x.py` format to correct the code format 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] git-hulk commented on pull request #1418: Support blocking migration for the cluster migrate command

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1542401986

   The implementation generally looks good to me, left a few comments:
   
   - The blockingLock function looks unnecessary, it's fine to use `std::unique_lock<std::mutex>lock(blocking_mutex_);` directly
   - For the naming issue, SyncMigrateContext::(StartBlock|Wakeup) is a bit inconsistent with Migrator::(StartBlocking|WakeupBlocking|CancelBlocking).
   - Those functions in SyncMigrateContext can be private:
     void OnWrite(bufferevent *bev); => void onWrite(bufferevent *bev);
     void OnEvent(bufferevent *bev, int16_t events); => void onEvent(bufferevent *bev, int16_t events);
     void TimerCB(int, int16_t events); => void timerCB(int, int16_t events);


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] ellutionist commented on a diff in pull request #1418: Support blocking migration for the cluster migrate command

Posted by "ellutionist (via GitHub)" <gi...@apache.org>.
ellutionist commented on code in PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#discussion_r1189530650


##########
src/cluster/sync_migrate_context.cc:
##########
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+#include "cluster/sync_migrate_context.h"
+
+SyncMigrateContext::~SyncMigrateContext() {
+  if (timer_) {
+    event_free(timer_);
+    timer_ = nullptr;
+  }
+}
+
+void SyncMigrateContext::StartBlock() {
+  auto bev = conn_->GetBufferEvent();
+  bufferevent_setcb(bev, nullptr, WriteCB, EventCB, this);
+
+  if (timeout_) {
+    timer_ = evtimer_new(bufferevent_get_base(bev), TimerCB, this);
+    timeval tm = {timeout_, 0};
+    evtimer_add(timer_, &tm);
+  }
+}
+
+void SyncMigrateContext::Wakeup(const Status &migrate_result) {
+  migrate_result_ = migrate_result;
+  auto s = conn_->Owner()->EnableWriteEvent(conn_->GetFD());
+  if (!s.IsOK()) {
+    LOG(ERROR) << "[server] Failed to enable write event on the sync migrate connection " << conn_->GetFD() << ": "
+               << s.Msg();
+  }
+}
+
+void SyncMigrateContext::EventCB(bufferevent *bev, int16_t events, void *ctx) {
+  auto self = reinterpret_cast<SyncMigrateContext *>(ctx);
+  auto &&slot_migrator = self->svr_->slot_migrator;

Review Comment:
   This is a reference to `std::unique_ptr<SlotMigrator>`, which cannot be copied.
   
   Technically `auto&` would suffice, but I personally prefer `auto&&` since it accepts more kinds of value (e.g. `const`, not this case though).
   
   Of course, it is also okay to use `auto slot_migrator = self->svr_->slot_migrator.get()` (raw pointer) here. Just a choice of style.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] ellutionist commented on pull request #1418: support sync migration

Posted by "ellutionist (via GitHub)" <gi...@apache.org>.
ellutionist commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1537156966

   > > The CommandClusterX does not always have a SyncMigrateContext (only when migrating with a "sync" flag). Making the context a non-pointer member of the CommandClusterX object may not make good sense here. If we decide to do this, I suggest to adopt std::optional.
   > 
   > Sure, you can use something like `std::optional` or `std::unique_ptr`.
   > 
   > > Due to the concern of memory safety, I am not a fan of raw pointers. Of course, in this PR we could use it, but we may set up some potential risk for the future. Because not every maintainer could get to know that "theCommandClusterX object must live long enough to complete the migration". If some change is made to "how the command objects work" in the future, it may lead to crash.
   > 
   > I think in kvrock, we use `std::shared_ptr` if and only if the object need to be **"shared"** between multiple references, and the lifetime of this object cannot be determined **statically** (e.g. given an object `o` and two reference `a` and `b`, the lifetime of `o` is equal to the maximum of lifetime of `a` and `b`, which need to be determined **dynamically**).
   > 
   > So currently, just _"in the future ..."_ seems not to be an accredited reason to me, since if this reason holds, we may need to change many raw pointer in the codebase to `shared_ptr`. If you just need a non-owning pointer (or called an observer/view/span, in modern C++), raw pointer is currently the available choice, since [`std::observer_ptr`](https://en.cppreference.com/w/cpp/experimental/observer_ptr) or `std::optional<T&>` (optional with reference type) is not in the standard.
   
   Thank you. I got the point of the difference between dynamic and static lifetime. From this point of view the `share_ptr` is not proper in this case. Perhaps the raw pointer is the best solution here.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] torwig commented on a diff in pull request #1418: support sync migration

Posted by "torwig (via GitHub)" <gi...@apache.org>.
torwig commented on code in PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#discussion_r1185110418


##########
src/commands/cmd_cluster.cc:
##########
@@ -253,6 +278,10 @@ class CommandClusterX : public Commander {
   int64_t slot_ = -1;
   int slot_id_ = -1;
   bool force_ = false;
+
+  bool sync_migrate_ = false;
+  int sync_migrate_timeout_ = 0;
+  std::shared_ptr<SyncMigrateContext> sync_migrate_ctx_ = nullptr;

Review Comment:
   @ellutionist Great job!
   I just thinking if we need to have `sync_migrate_ctx_` inside CommandClusterX because you just create it and pass it to the `MigrateSlot` function without further use of this object.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] ellutionist commented on pull request #1418: support sync migration

Posted by "ellutionist (via GitHub)" <gi...@apache.org>.
ellutionist commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1534720651

   @git-hulk @xiaobiaozhao @PragmaTwice @torwig @ShooterIT 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] ellutionist commented on a diff in pull request #1418: support sync migration

Posted by "ellutionist (via GitHub)" <gi...@apache.org>.
ellutionist commented on code in PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#discussion_r1187586820


##########
src/commands/cmd_cluster.cc:
##########
@@ -126,11 +127,28 @@ class CommandClusterX : public Commander {
     if (subcommand_ == "setnodeid" && args_.size() == 3 && args_[2].size() == kClusterNodeIdLen) return Status::OK();
 
     if (subcommand_ == "migrate") {
-      if (args.size() != 4) return {Status::RedisParseErr, errWrongNumOfArguments};
+      if (args.size() < 4) return {Status::RedisParseErr, errWrongNumOfArguments};

Review Comment:
   size should be between 4 to 6, will fix this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on pull request #1418: Support blocking migration for the cluster migrate command

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1547059020

   Merging... Thanks for your contribution!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1418: support sync migration

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#discussion_r1185126759


##########
src/commands/cmd_cluster.cc:
##########
@@ -253,6 +278,10 @@ class CommandClusterX : public Commander {
   int64_t slot_ = -1;
   int slot_id_ = -1;
   bool force_ = false;
+
+  bool sync_migrate_ = false;
+  int sync_migrate_timeout_ = 0;
+  std::shared_ptr<SyncMigrateContext> sync_migrate_ctx_ = nullptr;

Review Comment:
   Could you share some reason to use `std::shared_ptr`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] ellutionist commented on pull request #1418: support sync migration

Posted by "ellutionist (via GitHub)" <gi...@apache.org>.
ellutionist commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1534930672

   @torwig @PragmaTwice The reason why I use `shared_ptr` of `SyncMigrateContext` as a field of  `CommandClusterX` is that I have to keep the lifetime of `SyncMigrateContext` as long as the command object.
   
   When the `slot_migrator` wake up the `SyncMigrateContext` with a migration result status, it has to release the pointer so that no context will "occupy" the migrator any more (see [here](https://github.com/apache/incubator-kvrocks/pull/1418/files#diff-6a88559b7b94f9543cde3449bda77668eba37d8696153da4f244b250054e809aR1113). 
   However, at this time point, the `WriteCB` have not been triggered by the event loop yet, thus the context cannot be deconstructed yet. Using a shared pointer will bind the life cycle of the `SyncMigrateContext` with the `CommandClusterX` object, which will solve this issue.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] ellutionist commented on pull request #1418: Support blocking migration for the cluster migrate command

Posted by "ellutionist (via GitHub)" <gi...@apache.org>.
ellutionist commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1545016575

   > Looks like the migration test case has a segment fault. https://github.com/apache/incubator-kvrocks/actions/runs/4951001502/jobs/8855626634?pr=1418
   
   I think I found the reason. When building with ASAN the pointer of context won't initialize with `nullptr` by default. Should explicitly assign it a default value.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on pull request #1418: Support blocking migration for the cluster migrate command

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1542533001

   Seems need run `./x.py check golangci-lint`.
   
   > Those functions in SyncMigrateContext can be private:
   
   Currently we make the public for the CRTP base classes to use them.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] ellutionist commented on pull request #1418: Support blocking migration for the cluster migrate command

Posted by "ellutionist (via GitHub)" <gi...@apache.org>.
ellutionist commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1542344682

   > Hi @ellutionist Need to use `./x.py` format to correct the code format
   
   > Hi @ellutionist Need to use `./x.py` format to correct the code format
   
   Done. It took me a while to adapt to the recent changes from [#1420]. 
   
   Also some more go test cases were added.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice merged pull request #1418: Support blocking migration for the cluster migrate command

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice merged PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] git-hulk commented on pull request #1418: Support blocking migration for the cluster migrate command

Posted by "git-hulk (via GitHub)" <gi...@apache.org>.
git-hulk commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1544004397

   > Seems need run `./x.py check golangci-lint`.
   > 
   > > Those functions in SyncMigrateContext can be private:
   > 
   > Currently we need to make them public for the CRTP base classes to use them.
   
   OK, I didn't notice this limitation.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on a diff in pull request #1418: support sync migration

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on code in PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#discussion_r1185126759


##########
src/commands/cmd_cluster.cc:
##########
@@ -253,6 +278,10 @@ class CommandClusterX : public Commander {
   int64_t slot_ = -1;
   int slot_id_ = -1;
   bool force_ = false;
+
+  bool sync_migrate_ = false;
+  int sync_migrate_timeout_ = 0;
+  std::shared_ptr<SyncMigrateContext> sync_migrate_ctx_ = nullptr;

Review Comment:
   Could you share some reason to use `std::shared_ptr`? Seems except here (not necessary in my opinion) I cannot find where the object is "shared".



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] PragmaTwice commented on pull request #1418: support sync migration

Posted by "PragmaTwice (via GitHub)" <gi...@apache.org>.
PragmaTwice commented on PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#issuecomment-1535035392

   @ellutionist Thanks for your explanation.
   
   I have not dove into these changes deeply, but if you just want to make lifetime of SyncMigrateContext as long as the command object, you can store a instance of SyncMigrateContext in CommandClusterX directly (`class CommandClusterX { SyncMigrateContext ctx_; ... }`), and pass some non-owning pointer (like raw pointer, no ownership) to the migrator.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-kvrocks] ellutionist commented on a diff in pull request #1418: support sync migration

Posted by "ellutionist (via GitHub)" <gi...@apache.org>.
ellutionist commented on code in PR #1418:
URL: https://github.com/apache/incubator-kvrocks/pull/1418#discussion_r1187593433


##########
src/cluster/slot_migrate.h:
##########
@@ -80,8 +82,8 @@ class SlotMigrator : public redis::Database {
   ~SlotMigrator();
 
   Status CreateMigrationThread();
-  Status PerformSlotMigration(const std::string &node_id, std::string &dst_ip, int dst_port, int slot_id, int speed,
-                              int pipeline_size, int seq_gap);
+  Status PerformSlotMigration(const std::string &node_id, std::string &dst_ip, int dst_port, int slot_id,
+                              const std::shared_ptr<SyncMigrateContext> &blocking_ctx = nullptr);

Review Comment:
   Changed to raw pointer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@kvrocks.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org