You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@kvrocks.apache.org by "git-hulk (via GitHub)" <gi...@apache.org> on 2023/02/28 15:00:34 UTC

[GitHub] [incubator-kvrocks] git-hulk opened a new pull request, #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

git-hulk opened a new pull request, #1287:
URL: https://github.com/apache/incubator-kvrocks/pull/1287

   By default, the writes cannot be seen in transactions if the WriteBatch
   wasn't committed. To mitigate this issue, RocksDB offers the WriteBatchWithIndex
   to make the uncommitted writes visible if needed.
   
   For more information can see: https://rocksdb.org/blog/2015/02/27/write-batch-with-index.html
   
   This PR looks not easy to review at first glance, but it will be easier if review commits one by one:
   
   - The first commit adds the function `GetWriteBatch` in storage to wrap the WriteBatch for the write operations, making it possible to group and write all write operations at once in the EXEC command
   - The second commit supports using WriteBatchIndex in transaction mode so that those uncommitted writes can be seen in the same transaction
   - The third commit replaces the DB's Get and NewIterator API with WriteBatchIndex one


-- 
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 #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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

   > Other LGTM. Though I still think put txn_write_batch_ in storage in a bit trickey, can we add some description here, for it can only used by exclusive command?
   
   Thanks for your kind review. Agreed to add more comments to explain why the current implementation is right. I'm also thinking about how to allow multi transactions to improve performance because it's unnecessary to be exclusive command if we can promise the commit all or none. But can retrospect this after this PR and the watch/unwatch feature are ready.


-- 
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 #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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

   > Just parallel exec can be put into a "Session.Context" rather than Storage. A exec can write to it own session, and read first it own session.
   
   Yes, the transaction state indeed is a connection state, but it needs some refactoring work to support binding transaction conetext in underlying storage type implementation. We won't do that until this PR and watch/unwatch are supported.


-- 
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 #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/storage/storage.cc:
##########
@@ -500,10 +500,48 @@ Status Storage::GetWALIter(rocksdb::SequenceNumber seq, std::unique_ptr<rocksdb:
 
 rocksdb::SequenceNumber Storage::LatestSeq() { return db_->GetLatestSequenceNumber(); }
 
+rocksdb::Status Storage::Get(const rocksdb::ReadOptions &options, const rocksdb::Slice &key, std::string *value) {
+  return Get(options, db_->DefaultColumnFamily(), key, value);
+}
+
+rocksdb::Status Storage::Get(const rocksdb::ReadOptions &options, rocksdb::ColumnFamilyHandle *column_family,
+                             const rocksdb::Slice &key, std::string *value) {
+  if (is_txn_mode_ && txn_write_batch_->GetWriteBatch()->Count() > 0) {
+    return txn_write_batch_->GetFromBatchAndDB(db_, options, column_family, key, value);
+  }
+  return db_->Get(options, column_family, key, value);
+}
+
+rocksdb::Iterator *Storage::NewIterator(const rocksdb::ReadOptions &options) {
+  return NewIterator(options, db_->DefaultColumnFamily());
+}

Review Comment:
   A little formatting here: add a new line after `}`.



##########
src/storage/storage.cc:
##########
@@ -639,12 +677,41 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+Status Storage::BeginTxn() {
+  if (is_txn_mode_) {
+    return Status{Status::NotOK, "cannot begin a new transaction while already in transaction mode"};
+  }
+  // The EXEC command is exclusive and shouldn't have multi transaction at the same time,
+  // so it's fine to reset the global write batch without any lock.
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();
+  return Status::OK();
+}
 
+Status Storage::CommitTxn() {
+  if (!is_txn_mode_) {
+    return Status{Status::NotOK, "cannot commit while not in transaction mode"};
+  }
+  is_txn_mode_ = false;
+  auto s = Write(write_opts_, txn_write_batch_->GetWriteBatch());

Review Comment:
   Could we reset `txn_write_batch` after the transaction commit? Otherwise, it will be reset only on the next call to `BeginTxn`.



##########
src/storage/storage.cc:
##########
@@ -639,12 +677,41 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+Status Storage::BeginTxn() {
+  if (is_txn_mode_) {
+    return Status{Status::NotOK, "cannot begin a new transaction while already in transaction mode"};
+  }
+  // The EXEC command is exclusive and shouldn't have multi transaction at the same time,
+  // so it's fine to reset the global write batch without any lock.
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();
+  return Status::OK();
+}
 
+Status Storage::CommitTxn() {
+  if (!is_txn_mode_) {
+    return Status{Status::NotOK, "cannot commit while not in transaction mode"};
+  }
+  is_txn_mode_ = false;
+  auto s = Write(write_opts_, txn_write_batch_->GetWriteBatch());
+  if (s.ok()) {
+    return {Status::cOK};

Review Comment:
   You can just return `Status::OK();` here instead of `{Status::cOK}`.



##########
src/storage/storage.cc:
##########
@@ -639,12 +677,41 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+Status Storage::BeginTxn() {
+  if (is_txn_mode_) {
+    return Status{Status::NotOK, "cannot begin a new transaction while already in transaction mode"};
+  }
+  // The EXEC command is exclusive and shouldn't have multi transaction at the same time,
+  // so it's fine to reset the global write batch without any lock.
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();
+  return Status::OK();
+}
 
+Status Storage::CommitTxn() {
+  if (!is_txn_mode_) {
+    return Status{Status::NotOK, "cannot commit while not in transaction mode"};
+  }
+  is_txn_mode_ = false;

Review Comment:
   I think `is_txn_mode_` should be set to `false` only **after** the transaction was committed (**after** the call to `Write`). Otherwise `Write` could be called concurrently. Am I right?



-- 
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 #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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

   Thanks all, merging…


-- 
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] caipengbo commented on a diff in pull request #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/storage/storage.h:
##########
@@ -183,7 +198,18 @@ class Storage {
 
   std::atomic<bool> db_in_retryable_io_error_{false};
 
+  std::atomic<bool> is_txn_mode_ = false;
+  // txn_write_batch_ is used as the global write batch for the transaction mode,
+  // all writes will be grouped in this write batch when entering the transaction mode,
+  // then write it at once when committing.
+  //
+  // Notice: the reason why we can use the global transaction? because the EXEC is an exclusive

Review Comment:
   ```suggestion
     // Notice: the reason why we can use the global transaction is that the EXEC is an exclusive
   ```



-- 
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 a diff in pull request #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/storage/storage.cc:
##########
@@ -639,12 +677,41 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+Status Storage::BeginTxn() {
+  if (is_txn_mode_) {
+    return Status{Status::NotOK, "cannot begin a new transaction while already in transaction mode"};
+  }
+  // The EXEC command is exclusive and shouldn't have multi transaction at the same time,
+  // so it's fine to reset the global write batch without any lock.
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();
+  return Status::OK();
+}
 
+Status Storage::CommitTxn() {
+  if (!is_txn_mode_) {
+    return Status{Status::NotOK, "cannot commit while not in transaction mode"};
+  }
+  is_txn_mode_ = false;

Review Comment:
   The origin Write function will check is_txn_mode status. If true, it won't write the batch to db, so the transaction needs to be reset before committing. But it won't have the data race since the exec command is exclusive.
   
   There's a bit tricky in this implementation, I add a new function `writeToDB` which won't check the transaction status and it should be more clear. @torwig You can take a look at the commit: https://github.com/apache/incubator-kvrocks/pull/1287/commits/b8b5a6bb4332fedd622f8f8f4248e925d122c84b



-- 
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 #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/commands/cmd_txn.cc:
##########
@@ -68,13 +69,16 @@ class CommandExec : public Commander {
       return Status::OK();
     }
 
+    auto storage = svr->storage_;
     // Reply multi length first
     conn->Reply(Redis::MultiLen(conn->GetMultiExecCommands()->size()));
     // Execute multi-exec commands
     conn->SetInExec();
+    storage->BeginTxn();
     conn->ExecuteCommands(conn->GetMultiExecCommands());
+    auto s = storage->CommitTxn();

Review Comment:
   By the way, do we have a flag to check that `ExecuteCommands` triggers error. If it has 5 commands, and the 4th command error. Here it will still try to commit. Am I right?



-- 
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 #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/storage/storage.cc:
##########
@@ -639,12 +677,33 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+void Storage::BeginTxn() {
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();
+}
+
+Status Storage::CommitTxn() {
+  is_txn_mode_ = false;
+  auto s = Write(write_opts_, txn_write_batch_->GetWriteBatch());
+  txn_write_batch_.reset();
+  if (s.ok()) {
+    return {Status::cOK};
+  }
+  return {Status::NotOK, s.ToString()};
+}
+
+std::shared_ptr<rocksdb::WriteBatchBase> Storage::GetWriteBatch() {

Review Comment:
   Sounds ok. I just want to distinct `rocksdb::WriteBatchBase` and the real WriteBatch. But current solution is ok.



-- 
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 #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/storage/storage.cc:
##########
@@ -639,12 +677,33 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+void Storage::BeginTxn() {
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();
+}
+
+Status Storage::CommitTxn() {
+  is_txn_mode_ = false;
+  auto s = Write(write_opts_, txn_write_batch_->GetWriteBatch());
+  txn_write_batch_.reset();
+  if (s.ok()) {
+    return {Status::cOK};
+  }
+  return {Status::NotOK, s.ToString()};
+}
+
+std::shared_ptr<rocksdb::WriteBatchBase> Storage::GetWriteBatch() {

Review Comment:
   I think the `Base` suffix seems to be redundant.
   
   > offered a way to work around this
   
   @mwish You can access the prototype via:
   https://godbolt.org/z/T1xzfvWMa



-- 
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 a diff in pull request #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/commands/cmd_txn.cc:
##########
@@ -68,13 +69,16 @@ class CommandExec : public Commander {
       return Status::OK();
     }
 
+    auto storage = svr->storage_;
     // Reply multi length first
     conn->Reply(Redis::MultiLen(conn->GetMultiExecCommands()->size()));
     // Execute multi-exec commands
     conn->SetInExec();
+    storage->BeginTxn();
     conn->ExecuteCommands(conn->GetMultiExecCommands());
+    auto s = storage->CommitTxn();

Review Comment:
   Kvrocks won't throw any exception itself, but if the underlying components do, then Kvrocks should exit without committing if we didn't catch it. Thats to say, for the current implementation, all write operations are grouped into one WriteBatch, so it will be committed all or none.



-- 
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 merged pull request #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


-- 
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 pull request #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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

   In general, LGTM.


-- 
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 pull request #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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

   Just parallel exec can be put into a "Session.Context" rather than `Storage`. A exec can write to it own session, and read first it own session.
   
   Though it's ok, but it needs detail considering. For know, I think txn related is all under exclusive mode, because they only used by "Exec", which is exclusive.


-- 
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 a diff in pull request #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/storage/storage.cc:
##########
@@ -639,12 +677,33 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+void Storage::BeginTxn() {
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();
+}
+
+Status Storage::CommitTxn() {
+  is_txn_mode_ = false;
+  auto s = Write(write_opts_, txn_write_batch_->GetWriteBatch());
+  txn_write_batch_.reset();
+  if (s.ok()) {
+    return {Status::cOK};
+  }
+  return {Status::NotOK, s.ToString()};
+}
+
+std::shared_ptr<rocksdb::WriteBatchBase> Storage::GetWriteBatch() {

Review Comment:
   Yes, WriteBatchBase is a better name.
   
   >  And why we use shared_ptr rather than raw pointer here
   
   It's ok to use the raw pointer for the global transaction WriteBatch, but we should use the unique_ptr if it's not in transaction mode. So it needs to use shared_ptr if we want to put them together. BTW @PragmaTwice has offered a way to work around this, will have a try this week.



-- 
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 a diff in pull request #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/commands/cmd_txn.cc:
##########
@@ -68,13 +69,16 @@ class CommandExec : public Commander {
       return Status::OK();
     }
 
+    auto storage = svr->storage_;
     // Reply multi length first
     conn->Reply(Redis::MultiLen(conn->GetMultiExecCommands()->size()));
     // Execute multi-exec commands
     conn->SetInExec();
+    storage->BeginTxn();
     conn->ExecuteCommands(conn->GetMultiExecCommands());
+    auto s = storage->CommitTxn();

Review Comment:
   Yes, that's right. Redis also don't check the execute command status, for example:
   ```
   ❯ redis-cli
   127.0.0.1:6379> MULTI
   OK
   127.0.0.1:6379(TX)> SET k1 v1
   QUEUED
   127.0.0.1:6379(TX)> hset k1 field value
   QUEUED
   127.0.0.1:6379(TX)> EXEC
   1) OK
   2) (error) WRONGTYPE Operation against a key holding the wrong kind of 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 a diff in pull request #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/storage/storage.cc:
##########
@@ -639,12 +677,33 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+void Storage::BeginTxn() {
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();
+}
+
+Status Storage::CommitTxn() {
+  is_txn_mode_ = false;
+  auto s = Write(write_opts_, txn_write_batch_->GetWriteBatch());
+  txn_write_batch_.reset();
+  if (s.ok()) {
+    return {Status::cOK};
+  }
+  return {Status::NotOK, s.ToString()};
+}
+
+std::shared_ptr<rocksdb::WriteBatchBase> Storage::GetWriteBatch() {

Review Comment:
   I think the `Base` suffix seems to be redundant.



-- 
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 #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/storage/storage.cc:
##########
@@ -639,12 +677,33 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+void Storage::BeginTxn() {
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();

Review Comment:
   oh it has snapshot, ignore 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] mapleFU commented on a diff in pull request #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/storage/storage.cc:
##########
@@ -639,12 +677,33 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+void Storage::BeginTxn() {
+  is_txn_mode_ = true;

Review Comment:
   Here can we check `is_txn_mode_` is false? If not false, at lease we should add some warn or error log



##########
src/storage/storage.cc:
##########
@@ -639,12 +677,33 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+void Storage::BeginTxn() {
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();
+}
+
+Status Storage::CommitTxn() {
+  is_txn_mode_ = false;

Review Comment:
   can we check `is_txn_mode_` is true here? Seems currently we cannot has multiple CommitTxn. At lease we should add some warn or error log



-- 
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 #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/storage/storage.cc:
##########
@@ -639,12 +677,33 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+void Storage::BeginTxn() {
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();
+}
+
+Status Storage::CommitTxn() {
+  is_txn_mode_ = false;
+  auto s = Write(write_opts_, txn_write_batch_->GetWriteBatch());
+  txn_write_batch_.reset();
+  if (s.ok()) {
+    return {Status::cOK};
+  }
+  return {Status::NotOK, s.ToString()};
+}
+
+std::shared_ptr<rocksdb::WriteBatchBase> Storage::GetWriteBatch() {

Review Comment:
   Should we name it `GetWriteBatchBase` here? And why we use shared_ptr rather than raw pointer here?



##########
src/commands/cmd_txn.cc:
##########
@@ -68,13 +69,16 @@ class CommandExec : public Commander {
       return Status::OK();
     }
 
+    auto storage = svr->storage_;
     // Reply multi length first
     conn->Reply(Redis::MultiLen(conn->GetMultiExecCommands()->size()));
     // Execute multi-exec commands
     conn->SetInExec();
+    storage->BeginTxn();
     conn->ExecuteCommands(conn->GetMultiExecCommands());
+    auto s = storage->CommitTxn();

Review Comment:
   Just a naive question. If a exception is thrown in `conn->ExecuteCommands` (I didn't review carefully, so didn't know would it throw). The transaction will never recover until another txn begin and do the thing



##########
src/storage/storage.cc:
##########
@@ -639,12 +677,33 @@ void Storage::SetIORateLimit(int64_t max_io_mb) {
 
 rocksdb::DB *Storage::GetDB() { return db_; }
 
-Status Storage::WriteToPropagateCF(const std::string &key, const std::string &value) {
-  rocksdb::WriteBatch batch;
+void Storage::BeginTxn() {
+  is_txn_mode_ = true;
+  txn_write_batch_ = std::make_unique<rocksdb::WriteBatchWithIndex>();

Review Comment:
   Does this txn has consistent read snapshot?



-- 
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 #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/cluster/slot_import.cc:
##########
@@ -27,9 +27,8 @@ SlotImport::SlotImport(Server *svr)
       import_status_(kImportNone),
       import_fd_(-1) {
   std::lock_guard<std::mutex> guard(mutex_);
-  // Let db_ and metadata_cf_handle_ be nullptr, then get them in real time while use them.
+  // Let metadata_cf_handle_ be nullptr, then get them in real time while use them.
   // See comments in SlotMigrate::SlotMigrate for detailed reason.
-  db_ = nullptr;

Review Comment:
   db_ always ref system db here? Would this cause bug in transaction?



-- 
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 a diff in pull request #1287: Use the RocksDB WriteBatchWithIndex to implement the read-your-own-writes in transaction

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


##########
src/cluster/slot_import.cc:
##########
@@ -27,9 +27,8 @@ SlotImport::SlotImport(Server *svr)
       import_status_(kImportNone),
       import_fd_(-1) {
   std::lock_guard<std::mutex> guard(mutex_);
-  // Let db_ and metadata_cf_handle_ be nullptr, then get them in real time while use them.
+  // Let metadata_cf_handle_ be nullptr, then get them in real time while use them.
   // See comments in SlotMigrate::SlotMigrate for detailed reason.
-  db_ = nullptr;

Review Comment:
   No, it won't. It's right for the migration should always use the raw DB instead of the wrapped one. Because we don't expect to read uncommitted writes when migrating.



-- 
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