You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pegasus.apache.org by GitBox <gi...@apache.org> on 2022/08/04 10:53:47 UTC

[GitHub] [incubator-pegasus] hycdong opened a new pull request, #1102: feat(backup): 2. update and refactor meta backup engine class

hycdong opened a new pull request, #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102

   https://github.com/apache/incubator-pegasus/issues/1081
   As the issue shows, meta_backup_engine will  intertact with replica server and contorl backup lifecycle and status switch. This pr is the first part about it, including:
   - update `init_backup` and `start` functions
   - update `backup_status` and `backup_item` structures
   - update varieties of `meta_backup_engine.h`
   - rename `backup_engine` into `meta_backup_engine`
   - refactor `meta_backup_engine.h` including public/private, function definition order etc.


-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942072801


##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}

Review Comment:
   OK, in this case you can pass is_periodic to false, not needed to add another cstor, and make all member variables inited clearly.



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#issuecomment-1210462450

   the failed test case `meta_bulk_load_http_test.start_compaction_test` is not related (has been issued by https://github.com/apache/incubator-pegasus/issues/1060), we can fix it later.


-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941012247


##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}
+
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc, bool is_periodic)
+    : _meta_svc(meta_svc), _is_periodic_backup(is_periodic)
 {
 }
 
-backup_engine::~backup_engine() { _tracker.cancel_outstanding_tasks(); }
+meta_backup_engine::~meta_backup_engine() { _tracker.cancel_outstanding_tasks(); }
 
-error_code backup_engine::init_backup(int32_t app_id)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::init_backup(int32_t app_id,
+                                     int32_t partition_count,
+                                     const std::string &app_name,
+                                     const std::string &provider,
+                                     const std::string &backup_root_path)
 {
-    std::string app_name;
-    int partition_count;
-    {
-        zauto_read_lock l;
-        _backup_service->get_state()->lock_read(l);
-        std::shared_ptr<app_state> app = _backup_service->get_state()->get_app(app_id);
-        if (app == nullptr || app->status != app_status::AS_AVAILABLE) {
-            derror_f("app {} is not available, couldn't do backup now.", app_id);
-            return ERR_INVALID_STATE;
-        }
-        app_name = app->app_name;
-        partition_count = app->partition_count;
-    }
-
-    zauto_lock lock(_lock);
+    zauto_write_lock l(_lock);
     _backup_status.clear();
     for (int i = 0; i < partition_count; ++i) {
-        _backup_status.emplace(i, backup_status::UNALIVE);
+        _backup_status.emplace(i, backup_status::INVALID);
     }
     _cur_backup.app_id = app_id;
     _cur_backup.app_name = app_name;
     _cur_backup.backup_id = static_cast<int64_t>(dsn_now_ms());
     _cur_backup.start_time_ms = _cur_backup.backup_id;
-    return ERR_OK;
+    _cur_backup.backup_provider_type = provider;
+    _cur_backup.backup_path = backup_root_path;
+    _cur_backup.status = backup_status::INVALID;
+    _is_backup_failed = false;
+    _is_backup_canceled = false;
 }
 
-error_code backup_engine::set_block_service(const std::string &provider)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::start()
 {
-    _provider_type = provider;
-    _block_service = _backup_service->get_meta_service()
-                         ->get_block_service_manager()
-                         .get_or_create_block_filesystem(provider);
-    if (_block_service == nullptr) {
-        return ERR_INVALID_PARAMETERS;
+    ddebug_f("App[{}] start {} backup[{}] on {}, root_path = {}",
+             _cur_backup.app_name,
+             _is_periodic_backup ? "periodic" : "onetime",
+             _cur_backup.backup_id,
+             _cur_backup.backup_provider_type,
+             _cur_backup.backup_path);
+    error_code err = write_app_info();
+    if (err != ERR_OK) {
+        derror_f("backup_id({}): backup meta data for app {} failed, error {}",
+                 _cur_backup.backup_id,
+                 _cur_backup.app_id,
+                 err);
+        update_backup_item_on_remote_storage(backup_status::FAILED, dsn_now_ms());
+        return;
+    }
+    update_backup_item_on_remote_storage(backup_status::CHECKPOINTING);

Review Comment:
   do we still storage the status on zk?



##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,
+    CHECKPOINTING,
+    CHECKPOINTED,
+    UPLOADING,

Review Comment:
   if it has no other logic between `CHECKPOINTED` and `UPLOADING`, I think delete `CHECKPOINTED` is ok. checkpoint complete also mean it step to uploading



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941930492


##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}

Review Comment:
   When meta server restart, it will sync backup information from zk, in this case, it won't initialize `is_periodic` by constructor but other function, `meta_backup_engine(meta_service *meta_svc)` will be used in this case.



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941928066


##########
src/rdsn/include/dsn/dist/replication/replication_enums.h:
##########
@@ -157,4 +157,14 @@ ENUM_REG(replication::manual_compaction_status::QUEUING)
 ENUM_REG(replication::manual_compaction_status::RUNNING)
 ENUM_REG(replication::manual_compaction_status::FINISHED)
 ENUM_END2(replication::manual_compaction_status::type, manual_compaction_status)
+
+ENUM_BEGIN2(replication::backup_status::type, backup_status, replication::backup_status::INVALID)
+ENUM_REG(replication::backup_status::INVALID)
+ENUM_REG(replication::backup_status::CHECKPOINTING)
+ENUM_REG(replication::backup_status::CHECKPOINTED)
+ENUM_REG(replication::backup_status::UPLOADING)
+ENUM_REG(replication::backup_status::SUCCEED)
+ENUM_REG(replication::backup_status::FAILED)
+ENUM_REG(replication::backup_status::CANCELED)

Review Comment:
   Well, canceled is the past tense of cancel, not cancelled. 



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941928945


##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,
+    CHECKPOINTING,
+    CHECKPOINTED,
+    UPLOADING,

Review Comment:
   CHECKPOINTED is used for replica server, when all partitions status is CHECKPOINTED, app status will turn into UPLOADING.



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942077972


##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,
+    CHECKPOINTING,
+    CHECKPOINTED,
+    UPLOADING,

Review Comment:
   I mean the checkpointing can be step into uploading directly, not add the checkpointed status



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941928066


##########
src/rdsn/include/dsn/dist/replication/replication_enums.h:
##########
@@ -157,4 +157,14 @@ ENUM_REG(replication::manual_compaction_status::QUEUING)
 ENUM_REG(replication::manual_compaction_status::RUNNING)
 ENUM_REG(replication::manual_compaction_status::FINISHED)
 ENUM_END2(replication::manual_compaction_status::type, manual_compaction_status)
+
+ENUM_BEGIN2(replication::backup_status::type, backup_status, replication::backup_status::INVALID)
+ENUM_REG(replication::backup_status::INVALID)
+ENUM_REG(replication::backup_status::CHECKPOINTING)
+ENUM_REG(replication::backup_status::CHECKPOINTED)
+ENUM_REG(replication::backup_status::UPLOADING)
+ENUM_REG(replication::backup_status::SUCCEED)
+ENUM_REG(replication::backup_status::FAILED)
+ENUM_REG(replication::backup_status::CANCELED)

Review Comment:
   Actually, canceled is the past tense of cancel, not cancelled. 



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong merged pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong merged PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102


-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941934402


##########
src/rdsn/src/meta/meta_backup_engine.h:
##########
@@ -18,74 +18,107 @@
 #pragma once
 
 #include <dsn/cpp/json_helper.h>
-#include <dsn/dist/block_service.h>
 #include <dsn/tool-api/zlocks.h>
 
+#include "common/backup_common.h"
+#include "meta_service.h"
+#include "server_state.h"
+#include "meta_backup_service.h"
+
 namespace dsn {
 namespace replication {
 
-enum backup_status
-{
-    UNALIVE = 1,
-    ALIVE = 2,
-    COMPLETED = 3,
-    FAILED = 4
-};
-
+// backup_info file written into block service
 struct app_backup_info
 {
     int64_t backup_id;
     int64_t start_time_ms;
     int64_t end_time_ms;
-
     int32_t app_id;
     std::string app_name;
-
     app_backup_info() : backup_id(0), start_time_ms(0), end_time_ms(0) {}
-
     DEFINE_JSON_SERIALIZATION(backup_id, start_time_ms, end_time_ms, app_id, app_name)
 };
 
-class app_state;
-class backup_service;
-
-class backup_engine
+///
+///           Meta backup status
+///
+///              start backup
+///                  |
+///                  v       Error/Cancel
+///            Checkpointing ------------->|
+///                  |                     |
+///                  v       Error/Cancel  |
+///              Uploading  -------------->|
+///                  |                     |
+///                  v                     v
+///               Succeed          Failed/Canceled
+///
+class meta_backup_engine
 {
 public:
-    backup_engine(backup_service *service);
-    ~backup_engine();
-
-    error_code init_backup(int32_t app_id);
-    error_code set_block_service(const std::string &provider);
-    error_code set_backup_path(const std::string &path);
-
-    error_code start();
+    explicit meta_backup_engine(meta_service *meta_svc);
+    explicit meta_backup_engine(meta_service *meta_svc, bool is_periodic);
+    ~meta_backup_engine();
 
     int64_t get_current_backup_id() const { return _cur_backup.backup_id; }
     int32_t get_backup_app_id() const { return _cur_backup.app_id; }
-    bool is_in_progress() const;
 
-    backup_item get_backup_item() const;
+    backup_item get_backup_item() const
+    {
+        zauto_read_lock l(_lock);
+        backup_item item = _cur_backup;
+        return item;
+    }
 
-private:
-    friend class backup_engine_test;
-    friend class backup_service_test;
+    bool is_in_progress() const
+    {
+        zauto_read_lock l(_lock);
+        return _cur_backup.end_time_ms == 0 && !_is_backup_failed && !_is_backup_canceled;
+    }
 
-    FRIEND_TEST(backup_engine_test, test_on_backup_reply);
-    FRIEND_TEST(backup_engine_test, test_backup_completed);
-    FRIEND_TEST(backup_engine_test, test_write_backup_info_failed);
+private:
+    void init_backup(int32_t app_id,
+                     int32_t partition_count,
+                     const std::string &app_name,
+                     const std::string &provider,
+                     const std::string &backup_root_path);
+    void start();
 
-    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
-    error_code backup_app_meta();
     void backup_app_partition(const gpid &pid);
     void on_backup_reply(error_code err,
                          const backup_response &response,
                          gpid pid,
                          const rpc_address &primary);
+    void retry_backup(const dsn::gpid pid);
+    void handle_replica_backup_failed(const backup_response &response, const gpid pid);
+
+    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
+    error_code write_app_info();
     void write_backup_info();
+
+    void update_backup_item_on_remote_storage(backup_status::type new_status, int64_t end_time = 0);
+
+private:
+    friend class meta_backup_engine_test;
+
+    meta_service *_meta_svc;
+    task_tracker _tracker;
+
+    mutable zrwlock_nr _lock; // {
+    bool _is_periodic_backup;
+    bool _is_backup_failed{false};
+    bool _is_backup_canceled{false};
+    backup_item _cur_backup;
+    // partition_id -> backup_status
+    std::map<int32_t, backup_status::type> _backup_status;

Review Comment:
   Good idea~ This original structure is map, vector is more suitable, I will update it.



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942077972


##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,
+    CHECKPOINTING,
+    CHECKPOINTED,
+    UPLOADING,

Review Comment:
   I mean the checkpointing can step into uploading directly, not add the checkpointed status



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941930990


##########
src/rdsn/src/meta/meta_backup_engine.h:
##########
@@ -18,74 +18,107 @@
 #pragma once
 
 #include <dsn/cpp/json_helper.h>
-#include <dsn/dist/block_service.h>
 #include <dsn/tool-api/zlocks.h>
 
+#include "common/backup_common.h"
+#include "meta_service.h"
+#include "server_state.h"
+#include "meta_backup_service.h"
+
 namespace dsn {
 namespace replication {
 
-enum backup_status
-{
-    UNALIVE = 1,
-    ALIVE = 2,
-    COMPLETED = 3,
-    FAILED = 4
-};
-
+// backup_info file written into block service
 struct app_backup_info
 {
     int64_t backup_id;
     int64_t start_time_ms;
     int64_t end_time_ms;
-
     int32_t app_id;
     std::string app_name;
-
     app_backup_info() : backup_id(0), start_time_ms(0), end_time_ms(0) {}
-
     DEFINE_JSON_SERIALIZATION(backup_id, start_time_ms, end_time_ms, app_id, app_name)
 };
 
-class app_state;
-class backup_service;
-
-class backup_engine
+///
+///           Meta backup status
+///
+///              start backup
+///                  |
+///                  v       Error/Cancel
+///            Checkpointing ------------->|
+///                  |                     |
+///                  v       Error/Cancel  |
+///              Uploading  -------------->|
+///                  |                     |
+///                  v                     v
+///               Succeed          Failed/Canceled
+///
+class meta_backup_engine
 {
 public:
-    backup_engine(backup_service *service);
-    ~backup_engine();
-
-    error_code init_backup(int32_t app_id);
-    error_code set_block_service(const std::string &provider);
-    error_code set_backup_path(const std::string &path);
-
-    error_code start();
+    explicit meta_backup_engine(meta_service *meta_svc);
+    explicit meta_backup_engine(meta_service *meta_svc, bool is_periodic);
+    ~meta_backup_engine();
 
     int64_t get_current_backup_id() const { return _cur_backup.backup_id; }
     int32_t get_backup_app_id() const { return _cur_backup.app_id; }
-    bool is_in_progress() const;
 
-    backup_item get_backup_item() const;
+    backup_item get_backup_item() const
+    {
+        zauto_read_lock l(_lock);
+        backup_item item = _cur_backup;
+        return item;
+    }
 
-private:
-    friend class backup_engine_test;
-    friend class backup_service_test;
+    bool is_in_progress() const
+    {
+        zauto_read_lock l(_lock);
+        return _cur_backup.end_time_ms == 0 && !_is_backup_failed && !_is_backup_canceled;
+    }
 
-    FRIEND_TEST(backup_engine_test, test_on_backup_reply);
-    FRIEND_TEST(backup_engine_test, test_backup_completed);
-    FRIEND_TEST(backup_engine_test, test_write_backup_info_failed);
+private:
+    void init_backup(int32_t app_id,
+                     int32_t partition_count,
+                     const std::string &app_name,
+                     const std::string &provider,
+                     const std::string &backup_root_path);
+    void start();
 
-    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
-    error_code backup_app_meta();
     void backup_app_partition(const gpid &pid);
     void on_backup_reply(error_code err,
                          const backup_response &response,
                          gpid pid,
                          const rpc_address &primary);
+    void retry_backup(const dsn::gpid pid);
+    void handle_replica_backup_failed(const backup_response &response, const gpid pid);
+
+    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
+    error_code write_app_info();
     void write_backup_info();
+
+    void update_backup_item_on_remote_storage(backup_status::type new_status, int64_t end_time = 0);
+
+private:
+    friend class meta_backup_engine_test;
+
+    meta_service *_meta_svc;
+    task_tracker _tracker;
+
+    mutable zrwlock_nr _lock; // {
+    bool _is_periodic_backup;

Review Comment:
   As the comment below, in most case, _is_periodic_backup will be initilization by calling `meta_backup_engine(meta_service *meta_svc, bool is_periodic)`, not suitable to be false as default.



##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,
+    CHECKPOINTING,
+    CHECKPOINTED,
+    UPLOADING,
+    SUCCEED,
+    FAILED,
+    CANCELED

Review Comment:
   Same



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941933827


##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,

Review Comment:
   Good idea~ Done.



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942076474


##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}
+
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc, bool is_periodic)
+    : _meta_svc(meta_svc), _is_periodic_backup(is_periodic)
 {
 }
 
-backup_engine::~backup_engine() { _tracker.cancel_outstanding_tasks(); }
+meta_backup_engine::~meta_backup_engine() { _tracker.cancel_outstanding_tasks(); }
 
-error_code backup_engine::init_backup(int32_t app_id)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::init_backup(int32_t app_id,
+                                     int32_t partition_count,
+                                     const std::string &app_name,
+                                     const std::string &provider,
+                                     const std::string &backup_root_path)
 {
-    std::string app_name;
-    int partition_count;
-    {
-        zauto_read_lock l;
-        _backup_service->get_state()->lock_read(l);
-        std::shared_ptr<app_state> app = _backup_service->get_state()->get_app(app_id);
-        if (app == nullptr || app->status != app_status::AS_AVAILABLE) {
-            derror_f("app {} is not available, couldn't do backup now.", app_id);
-            return ERR_INVALID_STATE;
-        }
-        app_name = app->app_name;
-        partition_count = app->partition_count;
-    }
-
-    zauto_lock lock(_lock);
+    zauto_write_lock l(_lock);
     _backup_status.clear();
     for (int i = 0; i < partition_count; ++i) {
-        _backup_status.emplace(i, backup_status::UNALIVE);
+        _backup_status.emplace(i, backup_status::INVALID);
     }
     _cur_backup.app_id = app_id;
     _cur_backup.app_name = app_name;
     _cur_backup.backup_id = static_cast<int64_t>(dsn_now_ms());
     _cur_backup.start_time_ms = _cur_backup.backup_id;
-    return ERR_OK;
+    _cur_backup.backup_provider_type = provider;
+    _cur_backup.backup_path = backup_root_path;
+    _cur_backup.status = backup_status::INVALID;
+    _is_backup_failed = false;
+    _is_backup_canceled = false;
 }
 
-error_code backup_engine::set_block_service(const std::string &provider)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::start()
 {
-    _provider_type = provider;
-    _block_service = _backup_service->get_meta_service()
-                         ->get_block_service_manager()
-                         .get_or_create_block_filesystem(provider);
-    if (_block_service == nullptr) {
-        return ERR_INVALID_PARAMETERS;
+    ddebug_f("App[{}] start {} backup[{}] on {}, root_path = {}",
+             _cur_backup.app_name,
+             _is_periodic_backup ? "periodic" : "onetime",
+             _cur_backup.backup_id,
+             _cur_backup.backup_provider_type,
+             _cur_backup.backup_path);
+    error_code err = write_app_info();
+    if (err != ERR_OK) {
+        derror_f("backup_id({}): backup meta data for app {} failed, error {}",
+                 _cur_backup.backup_id,
+                 _cur_backup.app_id,
+                 err);
+        update_backup_item_on_remote_storage(backup_status::FAILED, dsn_now_ms());
+        return;
+    }
+    update_backup_item_on_remote_storage(backup_status::CHECKPOINTING);

Review Comment:
   is it for ? you want to support recovery when  the progress is terminated ?



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942129777


##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,
+    CHECKPOINTING,
+    CHECKPOINTED,
+    UPLOADING,

Review Comment:
   CHECKPOINTED is not used for meta server, but is used for replica server. It will be used in further pr.



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r940866475


##########
src/rdsn/src/meta/meta_backup_engine.h:
##########
@@ -18,74 +18,107 @@
 #pragma once
 
 #include <dsn/cpp/json_helper.h>
-#include <dsn/dist/block_service.h>
 #include <dsn/tool-api/zlocks.h>
 
+#include "common/backup_common.h"
+#include "meta_service.h"
+#include "server_state.h"
+#include "meta_backup_service.h"
+
 namespace dsn {
 namespace replication {
 
-enum backup_status
-{
-    UNALIVE = 1,
-    ALIVE = 2,
-    COMPLETED = 3,
-    FAILED = 4
-};
-
+// backup_info file written into block service
 struct app_backup_info
 {
     int64_t backup_id;
     int64_t start_time_ms;
     int64_t end_time_ms;
-
     int32_t app_id;
     std::string app_name;
-
     app_backup_info() : backup_id(0), start_time_ms(0), end_time_ms(0) {}
-
     DEFINE_JSON_SERIALIZATION(backup_id, start_time_ms, end_time_ms, app_id, app_name)
 };
 
-class app_state;
-class backup_service;
-
-class backup_engine
+///
+///           Meta backup status
+///
+///              start backup
+///                  |
+///                  v       Error/Cancel
+///            Checkpointing ------------->|
+///                  |                     |
+///                  v       Error/Cancel  |
+///              Uploading  -------------->|
+///                  |                     |
+///                  v                     v
+///               Succeed          Failed/Canceled
+///
+class meta_backup_engine
 {
 public:
-    backup_engine(backup_service *service);
-    ~backup_engine();
-
-    error_code init_backup(int32_t app_id);
-    error_code set_block_service(const std::string &provider);
-    error_code set_backup_path(const std::string &path);
-
-    error_code start();
+    explicit meta_backup_engine(meta_service *meta_svc);
+    explicit meta_backup_engine(meta_service *meta_svc, bool is_periodic);
+    ~meta_backup_engine();
 
     int64_t get_current_backup_id() const { return _cur_backup.backup_id; }
     int32_t get_backup_app_id() const { return _cur_backup.app_id; }
-    bool is_in_progress() const;
 
-    backup_item get_backup_item() const;
+    backup_item get_backup_item() const
+    {
+        zauto_read_lock l(_lock);
+        backup_item item = _cur_backup;
+        return item;
+    }
 
-private:
-    friend class backup_engine_test;
-    friend class backup_service_test;
+    bool is_in_progress() const
+    {
+        zauto_read_lock l(_lock);
+        return _cur_backup.end_time_ms == 0 && !_is_backup_failed && !_is_backup_canceled;
+    }
 
-    FRIEND_TEST(backup_engine_test, test_on_backup_reply);
-    FRIEND_TEST(backup_engine_test, test_backup_completed);
-    FRIEND_TEST(backup_engine_test, test_write_backup_info_failed);
+private:
+    void init_backup(int32_t app_id,
+                     int32_t partition_count,
+                     const std::string &app_name,
+                     const std::string &provider,
+                     const std::string &backup_root_path);
+    void start();
 
-    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
-    error_code backup_app_meta();
     void backup_app_partition(const gpid &pid);
     void on_backup_reply(error_code err,
                          const backup_response &response,
                          gpid pid,
                          const rpc_address &primary);
+    void retry_backup(const dsn::gpid pid);
+    void handle_replica_backup_failed(const backup_response &response, const gpid pid);
+
+    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
+    error_code write_app_info();
     void write_backup_info();
+
+    void update_backup_item_on_remote_storage(backup_status::type new_status, int64_t end_time = 0);
+
+private:
+    friend class meta_backup_engine_test;
+
+    meta_service *_meta_svc;
+    task_tracker _tracker;
+
+    mutable zrwlock_nr _lock; // {
+    bool _is_periodic_backup;
+    bool _is_backup_failed{false};
+    bool _is_backup_canceled{false};
+    backup_item _cur_backup;
+    // partition_id -> backup_status
+    std::map<int32_t, backup_status::type> _backup_status;

Review Comment:
   can we use a vector instead, the index is the partition id?



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942272423


##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,
+    CHECKPOINTING,
+    CHECKPOINTED,
+    UPLOADING,

Review Comment:
   Oh, I see



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941934402


##########
src/rdsn/src/meta/meta_backup_engine.h:
##########
@@ -18,74 +18,107 @@
 #pragma once
 
 #include <dsn/cpp/json_helper.h>
-#include <dsn/dist/block_service.h>
 #include <dsn/tool-api/zlocks.h>
 
+#include "common/backup_common.h"
+#include "meta_service.h"
+#include "server_state.h"
+#include "meta_backup_service.h"
+
 namespace dsn {
 namespace replication {
 
-enum backup_status
-{
-    UNALIVE = 1,
-    ALIVE = 2,
-    COMPLETED = 3,
-    FAILED = 4
-};
-
+// backup_info file written into block service
 struct app_backup_info
 {
     int64_t backup_id;
     int64_t start_time_ms;
     int64_t end_time_ms;
-
     int32_t app_id;
     std::string app_name;
-
     app_backup_info() : backup_id(0), start_time_ms(0), end_time_ms(0) {}
-
     DEFINE_JSON_SERIALIZATION(backup_id, start_time_ms, end_time_ms, app_id, app_name)
 };
 
-class app_state;
-class backup_service;
-
-class backup_engine
+///
+///           Meta backup status
+///
+///              start backup
+///                  |
+///                  v       Error/Cancel
+///            Checkpointing ------------->|
+///                  |                     |
+///                  v       Error/Cancel  |
+///              Uploading  -------------->|
+///                  |                     |
+///                  v                     v
+///               Succeed          Failed/Canceled
+///
+class meta_backup_engine
 {
 public:
-    backup_engine(backup_service *service);
-    ~backup_engine();
-
-    error_code init_backup(int32_t app_id);
-    error_code set_block_service(const std::string &provider);
-    error_code set_backup_path(const std::string &path);
-
-    error_code start();
+    explicit meta_backup_engine(meta_service *meta_svc);
+    explicit meta_backup_engine(meta_service *meta_svc, bool is_periodic);
+    ~meta_backup_engine();
 
     int64_t get_current_backup_id() const { return _cur_backup.backup_id; }
     int32_t get_backup_app_id() const { return _cur_backup.app_id; }
-    bool is_in_progress() const;
 
-    backup_item get_backup_item() const;
+    backup_item get_backup_item() const
+    {
+        zauto_read_lock l(_lock);
+        backup_item item = _cur_backup;
+        return item;
+    }
 
-private:
-    friend class backup_engine_test;
-    friend class backup_service_test;
+    bool is_in_progress() const
+    {
+        zauto_read_lock l(_lock);
+        return _cur_backup.end_time_ms == 0 && !_is_backup_failed && !_is_backup_canceled;
+    }
 
-    FRIEND_TEST(backup_engine_test, test_on_backup_reply);
-    FRIEND_TEST(backup_engine_test, test_backup_completed);
-    FRIEND_TEST(backup_engine_test, test_write_backup_info_failed);
+private:
+    void init_backup(int32_t app_id,
+                     int32_t partition_count,
+                     const std::string &app_name,
+                     const std::string &provider,
+                     const std::string &backup_root_path);
+    void start();
 
-    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
-    error_code backup_app_meta();
     void backup_app_partition(const gpid &pid);
     void on_backup_reply(error_code err,
                          const backup_response &response,
                          gpid pid,
                          const rpc_address &primary);
+    void retry_backup(const dsn::gpid pid);
+    void handle_replica_backup_failed(const backup_response &response, const gpid pid);
+
+    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
+    error_code write_app_info();
     void write_backup_info();
+
+    void update_backup_item_on_remote_storage(backup_status::type new_status, int64_t end_time = 0);
+
+private:
+    friend class meta_backup_engine_test;
+
+    meta_service *_meta_svc;
+    task_tracker _tracker;
+
+    mutable zrwlock_nr _lock; // {
+    bool _is_periodic_backup;
+    bool _is_backup_failed{false};
+    bool _is_backup_canceled{false};
+    backup_item _cur_backup;
+    // partition_id -> backup_status
+    std::map<int32_t, backup_status::type> _backup_status;

Review Comment:
   Good idea~ This original structure is map, vector is more suitable, I have already updated it.



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942131849


##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}
+
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc, bool is_periodic)
+    : _meta_svc(meta_svc), _is_periodic_backup(is_periodic)
 {
 }
 
-backup_engine::~backup_engine() { _tracker.cancel_outstanding_tasks(); }
+meta_backup_engine::~meta_backup_engine() { _tracker.cancel_outstanding_tasks(); }
 
-error_code backup_engine::init_backup(int32_t app_id)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::init_backup(int32_t app_id,
+                                     int32_t partition_count,
+                                     const std::string &app_name,
+                                     const std::string &provider,
+                                     const std::string &backup_root_path)
 {
-    std::string app_name;
-    int partition_count;
-    {
-        zauto_read_lock l;
-        _backup_service->get_state()->lock_read(l);
-        std::shared_ptr<app_state> app = _backup_service->get_state()->get_app(app_id);
-        if (app == nullptr || app->status != app_status::AS_AVAILABLE) {
-            derror_f("app {} is not available, couldn't do backup now.", app_id);
-            return ERR_INVALID_STATE;
-        }
-        app_name = app->app_name;
-        partition_count = app->partition_count;
-    }
-
-    zauto_lock lock(_lock);
+    zauto_write_lock l(_lock);
     _backup_status.clear();
     for (int i = 0; i < partition_count; ++i) {
-        _backup_status.emplace(i, backup_status::UNALIVE);
+        _backup_status.emplace(i, backup_status::INVALID);
     }
     _cur_backup.app_id = app_id;
     _cur_backup.app_name = app_name;
     _cur_backup.backup_id = static_cast<int64_t>(dsn_now_ms());
     _cur_backup.start_time_ms = _cur_backup.backup_id;
-    return ERR_OK;
+    _cur_backup.backup_provider_type = provider;
+    _cur_backup.backup_path = backup_root_path;
+    _cur_backup.status = backup_status::INVALID;
+    _is_backup_failed = false;
+    _is_backup_canceled = false;
 }
 
-error_code backup_engine::set_block_service(const std::string &provider)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::start()
 {
-    _provider_type = provider;
-    _block_service = _backup_service->get_meta_service()
-                         ->get_block_service_manager()
-                         .get_or_create_block_filesystem(provider);
-    if (_block_service == nullptr) {
-        return ERR_INVALID_PARAMETERS;
+    ddebug_f("App[{}] start {} backup[{}] on {}, root_path = {}",
+             _cur_backup.app_name,
+             _is_periodic_backup ? "periodic" : "onetime",
+             _cur_backup.backup_id,
+             _cur_backup.backup_provider_type,
+             _cur_backup.backup_path);
+    error_code err = write_app_info();
+    if (err != ERR_OK) {
+        derror_f("backup_id({}): backup meta data for app {} failed, error {}",
+                 _cur_backup.backup_id,
+                 _cur_backup.app_id,
+                 err);
+        update_backup_item_on_remote_storage(backup_status::FAILED, dsn_now_ms());
+        return;
+    }
+    update_backup_item_on_remote_storage(backup_status::CHECKPOINTING);

Review Comment:
   When meta server restart, it will load all in-process backup and finished backup. If replica server is doing backup, but meta server crash, backup won't stop, because meta server record old status. Besides, backup can be failed or canceled, those status also need to be recorded on zk facility for user to query.



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#issuecomment-1209400247

   > > [github-actions](https://github.com/apps/github-actions) bot added [cpp](https://github.com/apache/incubator-pegasus/labels/cpp) [thrift](https://github.com/apache/incubator-pegasus/labels/thrift) labels [5 days ago](https://github.com/apache/incubator-pegasus/pull/1102#event-7126785824)
   > 
   > The action can add more detail info, for example, we update `src/backup` folder, it can add `backup` label? @acelyc111
   
   It depends on the source code are well organized, but currently they are not, we can do that after several refactors.


-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941928566


##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}
+
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc, bool is_periodic)
+    : _meta_svc(meta_svc), _is_periodic_backup(is_periodic)
 {
 }
 
-backup_engine::~backup_engine() { _tracker.cancel_outstanding_tasks(); }
+meta_backup_engine::~meta_backup_engine() { _tracker.cancel_outstanding_tasks(); }
 
-error_code backup_engine::init_backup(int32_t app_id)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::init_backup(int32_t app_id,
+                                     int32_t partition_count,
+                                     const std::string &app_name,
+                                     const std::string &provider,
+                                     const std::string &backup_root_path)
 {
-    std::string app_name;
-    int partition_count;
-    {
-        zauto_read_lock l;
-        _backup_service->get_state()->lock_read(l);
-        std::shared_ptr<app_state> app = _backup_service->get_state()->get_app(app_id);
-        if (app == nullptr || app->status != app_status::AS_AVAILABLE) {
-            derror_f("app {} is not available, couldn't do backup now.", app_id);
-            return ERR_INVALID_STATE;
-        }
-        app_name = app->app_name;
-        partition_count = app->partition_count;
-    }
-
-    zauto_lock lock(_lock);
+    zauto_write_lock l(_lock);
     _backup_status.clear();
     for (int i = 0; i < partition_count; ++i) {
-        _backup_status.emplace(i, backup_status::UNALIVE);
+        _backup_status.emplace(i, backup_status::INVALID);
     }
     _cur_backup.app_id = app_id;
     _cur_backup.app_name = app_name;
     _cur_backup.backup_id = static_cast<int64_t>(dsn_now_ms());
     _cur_backup.start_time_ms = _cur_backup.backup_id;
-    return ERR_OK;
+    _cur_backup.backup_provider_type = provider;
+    _cur_backup.backup_path = backup_root_path;
+    _cur_backup.status = backup_status::INVALID;
+    _is_backup_failed = false;
+    _is_backup_canceled = false;
 }
 
-error_code backup_engine::set_block_service(const std::string &provider)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::start()
 {
-    _provider_type = provider;
-    _block_service = _backup_service->get_meta_service()
-                         ->get_block_service_manager()
-                         .get_or_create_block_filesystem(provider);
-    if (_block_service == nullptr) {
-        return ERR_INVALID_PARAMETERS;
+    ddebug_f("App[{}] start {} backup[{}] on {}, root_path = {}",
+             _cur_backup.app_name,
+             _is_periodic_backup ? "periodic" : "onetime",
+             _cur_backup.backup_id,
+             _cur_backup.backup_provider_type,
+             _cur_backup.backup_path);
+    error_code err = write_app_info();
+    if (err != ERR_OK) {
+        derror_f("backup_id({}): backup meta data for app {} failed, error {}",
+                 _cur_backup.backup_id,
+                 _cur_backup.app_id,
+                 err);
+        update_backup_item_on_remote_storage(backup_status::FAILED, dsn_now_ms());
+        return;
+    }
+    update_backup_item_on_remote_storage(backup_status::CHECKPOINTING);

Review Comment:
   Yes



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942131849


##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}
+
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc, bool is_periodic)
+    : _meta_svc(meta_svc), _is_periodic_backup(is_periodic)
 {
 }
 
-backup_engine::~backup_engine() { _tracker.cancel_outstanding_tasks(); }
+meta_backup_engine::~meta_backup_engine() { _tracker.cancel_outstanding_tasks(); }
 
-error_code backup_engine::init_backup(int32_t app_id)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::init_backup(int32_t app_id,
+                                     int32_t partition_count,
+                                     const std::string &app_name,
+                                     const std::string &provider,
+                                     const std::string &backup_root_path)
 {
-    std::string app_name;
-    int partition_count;
-    {
-        zauto_read_lock l;
-        _backup_service->get_state()->lock_read(l);
-        std::shared_ptr<app_state> app = _backup_service->get_state()->get_app(app_id);
-        if (app == nullptr || app->status != app_status::AS_AVAILABLE) {
-            derror_f("app {} is not available, couldn't do backup now.", app_id);
-            return ERR_INVALID_STATE;
-        }
-        app_name = app->app_name;
-        partition_count = app->partition_count;
-    }
-
-    zauto_lock lock(_lock);
+    zauto_write_lock l(_lock);
     _backup_status.clear();
     for (int i = 0; i < partition_count; ++i) {
-        _backup_status.emplace(i, backup_status::UNALIVE);
+        _backup_status.emplace(i, backup_status::INVALID);
     }
     _cur_backup.app_id = app_id;
     _cur_backup.app_name = app_name;
     _cur_backup.backup_id = static_cast<int64_t>(dsn_now_ms());
     _cur_backup.start_time_ms = _cur_backup.backup_id;
-    return ERR_OK;
+    _cur_backup.backup_provider_type = provider;
+    _cur_backup.backup_path = backup_root_path;
+    _cur_backup.status = backup_status::INVALID;
+    _is_backup_failed = false;
+    _is_backup_canceled = false;
 }
 
-error_code backup_engine::set_block_service(const std::string &provider)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::start()
 {
-    _provider_type = provider;
-    _block_service = _backup_service->get_meta_service()
-                         ->get_block_service_manager()
-                         .get_or_create_block_filesystem(provider);
-    if (_block_service == nullptr) {
-        return ERR_INVALID_PARAMETERS;
+    ddebug_f("App[{}] start {} backup[{}] on {}, root_path = {}",
+             _cur_backup.app_name,
+             _is_periodic_backup ? "periodic" : "onetime",
+             _cur_backup.backup_id,
+             _cur_backup.backup_provider_type,
+             _cur_backup.backup_path);
+    error_code err = write_app_info();
+    if (err != ERR_OK) {
+        derror_f("backup_id({}): backup meta data for app {} failed, error {}",
+                 _cur_backup.backup_id,
+                 _cur_backup.app_id,
+                 err);
+        update_backup_item_on_remote_storage(backup_status::FAILED, dsn_now_ms());
+        return;
+    }
+    update_backup_item_on_remote_storage(backup_status::CHECKPOINTING);

Review Comment:
   When meta server restart, it will load all in-process backup and finished backup. If replica server is doing backup, but meta server crash, backup won't stop, because meta server record old status. Besides, backup can be failed or canceled, those status also need to be recorded on zk.



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942142622


##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}

Review Comment:
   Sounds reasonable~ Done.



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#issuecomment-1209102487

   > [github-actions](https://github.com/apps/github-actions) bot added [cpp](https://github.com/apache/incubator-pegasus/labels/cpp) [thrift](https://github.com/apache/incubator-pegasus/labels/thrift) labels [5 days ago](https://github.com/apache/incubator-pegasus/pull/1102#event-7126785824)
   
   The action can add more detail info, for example, we update `src/backup` folder, it can add `backup` label? @acelyc111 


-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941934402


##########
src/rdsn/src/meta/meta_backup_engine.h:
##########
@@ -18,74 +18,107 @@
 #pragma once
 
 #include <dsn/cpp/json_helper.h>
-#include <dsn/dist/block_service.h>
 #include <dsn/tool-api/zlocks.h>
 
+#include "common/backup_common.h"
+#include "meta_service.h"
+#include "server_state.h"
+#include "meta_backup_service.h"
+
 namespace dsn {
 namespace replication {
 
-enum backup_status
-{
-    UNALIVE = 1,
-    ALIVE = 2,
-    COMPLETED = 3,
-    FAILED = 4
-};
-
+// backup_info file written into block service
 struct app_backup_info
 {
     int64_t backup_id;
     int64_t start_time_ms;
     int64_t end_time_ms;
-
     int32_t app_id;
     std::string app_name;
-
     app_backup_info() : backup_id(0), start_time_ms(0), end_time_ms(0) {}
-
     DEFINE_JSON_SERIALIZATION(backup_id, start_time_ms, end_time_ms, app_id, app_name)
 };
 
-class app_state;
-class backup_service;
-
-class backup_engine
+///
+///           Meta backup status
+///
+///              start backup
+///                  |
+///                  v       Error/Cancel
+///            Checkpointing ------------->|
+///                  |                     |
+///                  v       Error/Cancel  |
+///              Uploading  -------------->|
+///                  |                     |
+///                  v                     v
+///               Succeed          Failed/Canceled
+///
+class meta_backup_engine
 {
 public:
-    backup_engine(backup_service *service);
-    ~backup_engine();
-
-    error_code init_backup(int32_t app_id);
-    error_code set_block_service(const std::string &provider);
-    error_code set_backup_path(const std::string &path);
-
-    error_code start();
+    explicit meta_backup_engine(meta_service *meta_svc);
+    explicit meta_backup_engine(meta_service *meta_svc, bool is_periodic);
+    ~meta_backup_engine();
 
     int64_t get_current_backup_id() const { return _cur_backup.backup_id; }
     int32_t get_backup_app_id() const { return _cur_backup.app_id; }
-    bool is_in_progress() const;
 
-    backup_item get_backup_item() const;
+    backup_item get_backup_item() const
+    {
+        zauto_read_lock l(_lock);
+        backup_item item = _cur_backup;
+        return item;
+    }
 
-private:
-    friend class backup_engine_test;
-    friend class backup_service_test;
+    bool is_in_progress() const
+    {
+        zauto_read_lock l(_lock);
+        return _cur_backup.end_time_ms == 0 && !_is_backup_failed && !_is_backup_canceled;
+    }
 
-    FRIEND_TEST(backup_engine_test, test_on_backup_reply);
-    FRIEND_TEST(backup_engine_test, test_backup_completed);
-    FRIEND_TEST(backup_engine_test, test_write_backup_info_failed);
+private:
+    void init_backup(int32_t app_id,
+                     int32_t partition_count,
+                     const std::string &app_name,
+                     const std::string &provider,
+                     const std::string &backup_root_path);
+    void start();
 
-    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
-    error_code backup_app_meta();
     void backup_app_partition(const gpid &pid);
     void on_backup_reply(error_code err,
                          const backup_response &response,
                          gpid pid,
                          const rpc_address &primary);
+    void retry_backup(const dsn::gpid pid);
+    void handle_replica_backup_failed(const backup_response &response, const gpid pid);
+
+    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
+    error_code write_app_info();
     void write_backup_info();
+
+    void update_backup_item_on_remote_storage(backup_status::type new_status, int64_t end_time = 0);
+
+private:
+    friend class meta_backup_engine_test;
+
+    meta_service *_meta_svc;
+    task_tracker _tracker;
+
+    mutable zrwlock_nr _lock; // {
+    bool _is_periodic_backup;
+    bool _is_backup_failed{false};
+    bool _is_backup_canceled{false};
+    backup_item _cur_backup;
+    // partition_id -> backup_status
+    std::map<int32_t, backup_status::type> _backup_status;

Review Comment:
   Good idea~



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r941933741


##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,
+    CHECKPOINTING,
+    CHECKPOINTED,

Review Comment:
   I think CHECKPOINT is a noun, the original is better.



##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,

Review Comment:
   Good idea~



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] hycdong commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
hycdong commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942142622


##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}

Review Comment:
   Sounds reasonable, done



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r940854781


##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,
+    CHECKPOINTING,
+    CHECKPOINTED,
+    UPLOADING,
+    SUCCEED,
+    FAILED,
+    CANCELED

Review Comment:
   typo



##########
src/rdsn/include/dsn/dist/replication/replication_enums.h:
##########
@@ -157,4 +157,14 @@ ENUM_REG(replication::manual_compaction_status::QUEUING)
 ENUM_REG(replication::manual_compaction_status::RUNNING)
 ENUM_REG(replication::manual_compaction_status::FINISHED)
 ENUM_END2(replication::manual_compaction_status::type, manual_compaction_status)
+
+ENUM_BEGIN2(replication::backup_status::type, backup_status, replication::backup_status::INVALID)
+ENUM_REG(replication::backup_status::INVALID)
+ENUM_REG(replication::backup_status::CHECKPOINTING)
+ENUM_REG(replication::backup_status::CHECKPOINTED)
+ENUM_REG(replication::backup_status::UPLOADING)
+ENUM_REG(replication::backup_status::SUCCEED)
+ENUM_REG(replication::backup_status::FAILED)
+ENUM_REG(replication::backup_status::CANCELED)

Review Comment:
   typo: CANCELLED



##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}

Review Comment:
   I'm a bit of confused can it be replaced by the other version `meta_backup_engine(meta_service *meta_svc, bool is_periodic)`?



##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,
+    CHECKPOINTING,
+    CHECKPOINTED,

Review Comment:
   maybe CHECKING_POINT and CHECKED_POINT ?



##########
src/rdsn/src/meta/meta_backup_engine.h:
##########
@@ -18,74 +18,107 @@
 #pragma once
 
 #include <dsn/cpp/json_helper.h>
-#include <dsn/dist/block_service.h>
 #include <dsn/tool-api/zlocks.h>
 
+#include "common/backup_common.h"
+#include "meta_service.h"
+#include "server_state.h"
+#include "meta_backup_service.h"
+
 namespace dsn {
 namespace replication {
 
-enum backup_status
-{
-    UNALIVE = 1,
-    ALIVE = 2,
-    COMPLETED = 3,
-    FAILED = 4
-};
-
+// backup_info file written into block service
 struct app_backup_info
 {
     int64_t backup_id;
     int64_t start_time_ms;
     int64_t end_time_ms;
-
     int32_t app_id;
     std::string app_name;
-
     app_backup_info() : backup_id(0), start_time_ms(0), end_time_ms(0) {}
-
     DEFINE_JSON_SERIALIZATION(backup_id, start_time_ms, end_time_ms, app_id, app_name)
 };
 
-class app_state;
-class backup_service;
-
-class backup_engine
+///
+///           Meta backup status
+///
+///              start backup
+///                  |
+///                  v       Error/Cancel
+///            Checkpointing ------------->|
+///                  |                     |
+///                  v       Error/Cancel  |
+///              Uploading  -------------->|
+///                  |                     |
+///                  v                     v
+///               Succeed          Failed/Canceled
+///
+class meta_backup_engine
 {
 public:
-    backup_engine(backup_service *service);
-    ~backup_engine();
-
-    error_code init_backup(int32_t app_id);
-    error_code set_block_service(const std::string &provider);
-    error_code set_backup_path(const std::string &path);
-
-    error_code start();
+    explicit meta_backup_engine(meta_service *meta_svc);
+    explicit meta_backup_engine(meta_service *meta_svc, bool is_periodic);
+    ~meta_backup_engine();
 
     int64_t get_current_backup_id() const { return _cur_backup.backup_id; }
     int32_t get_backup_app_id() const { return _cur_backup.app_id; }
-    bool is_in_progress() const;
 
-    backup_item get_backup_item() const;
+    backup_item get_backup_item() const
+    {
+        zauto_read_lock l(_lock);
+        backup_item item = _cur_backup;
+        return item;
+    }
 
-private:
-    friend class backup_engine_test;
-    friend class backup_service_test;
+    bool is_in_progress() const
+    {
+        zauto_read_lock l(_lock);
+        return _cur_backup.end_time_ms == 0 && !_is_backup_failed && !_is_backup_canceled;
+    }
 
-    FRIEND_TEST(backup_engine_test, test_on_backup_reply);
-    FRIEND_TEST(backup_engine_test, test_backup_completed);
-    FRIEND_TEST(backup_engine_test, test_write_backup_info_failed);
+private:
+    void init_backup(int32_t app_id,
+                     int32_t partition_count,
+                     const std::string &app_name,
+                     const std::string &provider,
+                     const std::string &backup_root_path);
+    void start();
 
-    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
-    error_code backup_app_meta();
     void backup_app_partition(const gpid &pid);
     void on_backup_reply(error_code err,
                          const backup_response &response,
                          gpid pid,
                          const rpc_address &primary);
+    void retry_backup(const dsn::gpid pid);
+    void handle_replica_backup_failed(const backup_response &response, const gpid pid);
+
+    error_code write_backup_file(const std::string &file_name, const dsn::blob &write_buffer);
+    error_code write_app_info();
     void write_backup_info();
+
+    void update_backup_item_on_remote_storage(backup_status::type new_status, int64_t end_time = 0);
+
+private:
+    friend class meta_backup_engine_test;
+
+    meta_service *_meta_svc;
+    task_tracker _tracker;
+
+    mutable zrwlock_nr _lock; // {
+    bool _is_periodic_backup;

Review Comment:
   also init as `false`?



##########
src/rdsn/src/common/backup.thrift:
##########
@@ -20,6 +20,30 @@ include "../../../../idl/dsn.layer2.thrift"
 
 namespace cpp dsn.replication
 
+enum backup_status
+{
+    INVALID,

Review Comment:
   how about UNINITIALIZED?



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] acelyc111 commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942072801


##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}

Review Comment:
   OK, in this case you can pass is_periodic to false, not needed to add another cstor.



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org


[GitHub] [incubator-pegasus] foreverneverer commented on a diff in pull request #1102: feat(backup): 2. update and refactor meta backup engine class

Posted by GitBox <gi...@apache.org>.
foreverneverer commented on code in PR #1102:
URL: https://github.com/apache/incubator-pegasus/pull/1102#discussion_r942271224


##########
src/rdsn/src/meta/meta_backup_engine.cpp:
##########
@@ -16,74 +16,93 @@
 // under the License.
 
 #include <dsn/dist/fmt_logging.h>
+#include <dsn/utility/fail_point.h>
 #include <dsn/utility/filesystem.h>
 
-#include "common/backup_common.h"
-#include "common/replication_common.h"
-#include "server_state.h"
+#include "meta_backup_engine.h"
 
 namespace dsn {
 namespace replication {
 
-backup_engine::backup_engine(backup_service *service)
-    : _backup_service(service), _block_service(nullptr), _backup_path(""), _is_backup_failed(false)
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc) : _meta_svc(meta_svc) {}
+
+meta_backup_engine::meta_backup_engine(meta_service *meta_svc, bool is_periodic)
+    : _meta_svc(meta_svc), _is_periodic_backup(is_periodic)
 {
 }
 
-backup_engine::~backup_engine() { _tracker.cancel_outstanding_tasks(); }
+meta_backup_engine::~meta_backup_engine() { _tracker.cancel_outstanding_tasks(); }
 
-error_code backup_engine::init_backup(int32_t app_id)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::init_backup(int32_t app_id,
+                                     int32_t partition_count,
+                                     const std::string &app_name,
+                                     const std::string &provider,
+                                     const std::string &backup_root_path)
 {
-    std::string app_name;
-    int partition_count;
-    {
-        zauto_read_lock l;
-        _backup_service->get_state()->lock_read(l);
-        std::shared_ptr<app_state> app = _backup_service->get_state()->get_app(app_id);
-        if (app == nullptr || app->status != app_status::AS_AVAILABLE) {
-            derror_f("app {} is not available, couldn't do backup now.", app_id);
-            return ERR_INVALID_STATE;
-        }
-        app_name = app->app_name;
-        partition_count = app->partition_count;
-    }
-
-    zauto_lock lock(_lock);
+    zauto_write_lock l(_lock);
     _backup_status.clear();
     for (int i = 0; i < partition_count; ++i) {
-        _backup_status.emplace(i, backup_status::UNALIVE);
+        _backup_status.emplace(i, backup_status::INVALID);
     }
     _cur_backup.app_id = app_id;
     _cur_backup.app_name = app_name;
     _cur_backup.backup_id = static_cast<int64_t>(dsn_now_ms());
     _cur_backup.start_time_ms = _cur_backup.backup_id;
-    return ERR_OK;
+    _cur_backup.backup_provider_type = provider;
+    _cur_backup.backup_path = backup_root_path;
+    _cur_backup.status = backup_status::INVALID;
+    _is_backup_failed = false;
+    _is_backup_canceled = false;
 }
 
-error_code backup_engine::set_block_service(const std::string &provider)
+// ThreadPool: THREAD_POOL_DEFAULT
+void meta_backup_engine::start()
 {
-    _provider_type = provider;
-    _block_service = _backup_service->get_meta_service()
-                         ->get_block_service_manager()
-                         .get_or_create_block_filesystem(provider);
-    if (_block_service == nullptr) {
-        return ERR_INVALID_PARAMETERS;
+    ddebug_f("App[{}] start {} backup[{}] on {}, root_path = {}",
+             _cur_backup.app_name,
+             _is_periodic_backup ? "periodic" : "onetime",
+             _cur_backup.backup_id,
+             _cur_backup.backup_provider_type,
+             _cur_backup.backup_path);
+    error_code err = write_app_info();
+    if (err != ERR_OK) {
+        derror_f("backup_id({}): backup meta data for app {} failed, error {}",
+                 _cur_backup.backup_id,
+                 _cur_backup.app_id,
+                 err);
+        update_backup_item_on_remote_storage(backup_status::FAILED, dsn_now_ms());
+        return;
+    }
+    update_backup_item_on_remote_storage(backup_status::CHECKPOINTING);

Review Comment:
   Ok, good!



-- 
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: dev-unsubscribe@pegasus.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@pegasus.apache.org
For additional commands, e-mail: dev-help@pegasus.apache.org