diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index ff9c11a970f6b9..ec25edf676e57c 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -65,6 +65,8 @@ DEFINE_Int32(brpc_port, "8060"); DEFINE_Int32(arrow_flight_sql_port, "8050"); +DEFINE_Int32(cdc_client_port, "9096"); + // If the external client cannot directly access priority_networks, set public_host to be accessible // to external client. // There are usually two usage scenarios: @@ -629,6 +631,9 @@ DEFINE_mBool(enable_stream_load_commit_txn_on_be, "false"); // The buffer size to store stream table function schema info DEFINE_Int64(stream_tvf_buffer_size, "1048576"); // 1MB +// request cdc client timeout +DEFINE_mInt32(request_cdc_client_timeout_ms, "60000"); + // OlapTableSink sender's send interval, should be less than the real response time of a tablet writer rpc. // You may need to lower the speed when the sink receiver bes are too busy. DEFINE_mInt32(olap_table_sink_send_interval_microseconds, "1000"); diff --git a/be/src/common/config.h b/be/src/common/config.h index 446b7f10f55a19..3da56c305232ed 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -100,6 +100,9 @@ DECLARE_Int32(brpc_port); // Default -1, do not start arrow flight sql server. DECLARE_Int32(arrow_flight_sql_port); +// port for cdc client scan oltp cdc data +DECLARE_Int32(cdc_client_port); + // If the external client cannot directly access priority_networks, set public_host to be accessible // to external client. // There are usually two usage scenarios: @@ -667,6 +670,9 @@ DECLARE_mBool(enable_stream_load_commit_txn_on_be); // The buffer size to store stream table function schema info DECLARE_Int64(stream_tvf_buffer_size); +// request cdc client timeout +DECLARE_mInt32(request_cdc_client_timeout_ms); + // OlapTableSink sender's send interval, should be less than the real response time of a tablet writer rpc. // You may need to lower the speed when the sink receiver bes are too busy. DECLARE_mInt32(olap_table_sink_send_interval_microseconds); diff --git a/be/src/runtime/cdc_client_mgr.cpp b/be/src/runtime/cdc_client_mgr.cpp new file mode 100644 index 00000000000000..e1c7ba976e7913 --- /dev/null +++ b/be/src/runtime/cdc_client_mgr.cpp @@ -0,0 +1,269 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "runtime/cdc_client_mgr.h" + +#include +#include +#include +#include +#include +#include +#include + +#include +#ifndef __APPLE__ +#include +#endif + +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/logging.h" +#include "common/status.h" +#include "http/http_client.h" + +namespace doris { + +namespace { +// Handle SIGCHLD signal to prevent zombie processes +void handle_sigchld(int sig_no) { + int status = 0; + pid_t pid; + while ((pid = waitpid(-1, &status, WNOHANG)) > 0) { + } +} + +// Check CDC client health +#ifndef BE_TEST +Status check_cdc_client_health(int retry_times, int sleep_time, std::string& health_response) { + const std::string cdc_health_url = + "http://127.0.0.1:" + std::to_string(doris::config::cdc_client_port) + + "/actuator/health"; + + auto health_request = [cdc_health_url, &health_response](HttpClient* client) { + RETURN_IF_ERROR(client->init(cdc_health_url)); + client->set_timeout_ms(5000); + RETURN_IF_ERROR(client->execute(&health_response)); + return Status::OK(); + }; + + Status status = HttpClient::execute_with_retry(retry_times, sleep_time, health_request); + + if (!status.ok()) { + return Status::InternalError("CDC client health check failed"); + } + + bool is_up = health_response.find("UP") != std::string::npos; + + if (!is_up) { + return Status::InternalError(fmt::format("CDC client unhealthy: {}", health_response)); + } + + return Status::OK(); +} +#endif + +} // anonymous namespace + +CdcClientMgr::CdcClientMgr() = default; + +CdcClientMgr::~CdcClientMgr() { + stop(); +} + +void CdcClientMgr::stop() { + pid_t pid = _child_pid.load(); + if (pid > 0) { + // Check if process is still alive + if (kill(pid, 0) == 0) { + LOG(INFO) << "Stopping CDC client process, pid=" << pid; + // Send SIGTERM for graceful shutdown + kill(pid, SIGTERM); + // Wait a short time for graceful shutdown + std::this_thread::sleep_for(std::chrono::milliseconds(200)); + // Force kill if still alive + if (kill(pid, 0) == 0) { + LOG(INFO) << "Force killing CDC client process, pid=" << pid; + kill(pid, SIGKILL); + int status = 0; + waitpid(pid, &status, 0); + } + } + _child_pid.store(0); + } + + LOG(INFO) << "CdcClientMgr is stopped"; +} + +Status CdcClientMgr::start_cdc_client(PRequestCdcClientResult* result) { + std::lock_guard lock(_start_mutex); + + Status st = Status::OK(); + pid_t exist_pid = _child_pid.load(); + if (exist_pid > 0) { +#ifdef BE_TEST + // In test mode, directly return OK if PID exists + LOG(INFO) << "cdc client already started (BE_TEST mode), pid=" << exist_pid; + return Status::OK(); +#else + // Check if process is still alive + if (kill(exist_pid, 0) == 0) { + // Process exists, verify it's actually our CDC client by health check + std::string check_response; + auto check_st = check_cdc_client_health(1, 0, check_response); + if (check_st.ok()) { + // Process exists and responding, CDC client is running + return Status::OK(); + } else { + // Process exists but CDC client not responding + // Either it's a different process (PID reused) or CDC client is unhealthy + // Reset PID and return error + _child_pid.store(0); + st = Status::InternalError(fmt::format("CDC client {} unresponsive", exist_pid)); + st.to_protobuf(result->mutable_status()); + return st; + } + } else { + // Process is dead, reset PID and continue to start + _child_pid.store(0); + } +#endif + } + + const char* doris_home = getenv("DORIS_HOME"); + const char* log_dir = getenv("LOG_DIR"); + const std::string cdc_jar_path = std::string(doris_home) + "/lib/cdc_client/cdc-client.jar"; + const std::string cdc_jar_port = + "--server.port=" + std::to_string(doris::config::cdc_client_port); + const std::string backend_http_port = + "--backend.http.port=" + std::to_string(config::webserver_port); + const std::string java_opts = "-Dlog.path=" + std::string(log_dir); + + // check cdc jar exists + struct stat buffer; + if (stat(cdc_jar_path.c_str(), &buffer) != 0) { + st = Status::InternalError("Can not find cdc-client.jar."); + st.to_protobuf(result->mutable_status()); + return st; + } + + // Ready to start cdc client + LOG(INFO) << "Ready to start cdc client"; + const auto* java_home = getenv("JAVA_HOME"); + if (!java_home) { + st = Status::InternalError("Can not find JAVA_HOME"); + st.to_protobuf(result->mutable_status()); + return st; + } + std::string path(java_home); + std::string java_bin = path + "/bin/java"; + // Capture signal to prevent child process from becoming a zombie process + struct sigaction act; + act.sa_flags = 0; + act.sa_handler = handle_sigchld; + sigaction(SIGCHLD, &act, NULL); + LOG(INFO) << "Start to fork cdc client process with " << path; +#ifdef BE_TEST + _child_pid.store(99999); + st = Status::OK(); + return st; +#else + pid_t pid = fork(); + if (pid < 0) { + // Fork failed + st = Status::InternalError("Fork cdc client failed."); + st.to_protobuf(result->mutable_status()); + return st; + } else if (pid == 0) { + // Child process + // When the parent process is killed, the child process also needs to exit +#ifndef __APPLE__ + prctl(PR_SET_PDEATHSIG, SIGKILL); +#endif + // java -jar -Dlog.path=xx cdc-client.jar --server.port=9096 --backend.http.port=8040 + execlp(java_bin.c_str(), "java", java_opts.c_str(), "-jar", cdc_jar_path.c_str(), + cdc_jar_port.c_str(), backend_http_port.c_str(), (char*)NULL); + // If execlp returns, it means it failed + perror("Cdc client child process error"); + exit(1); + } else { + // Parent process: save PID and wait for startup + _child_pid.store(pid); + + // Waiting for cdc to start, failed after more than 3 * 10 seconds + std::string health_response; + Status status = check_cdc_client_health(3, 10, health_response); + if (!status.ok()) { + // Reset PID if startup failed + _child_pid.store(0); + st = Status::InternalError("Start cdc client failed."); + st.to_protobuf(result->mutable_status()); + } else { + LOG(INFO) << "Start cdc client success, pid=" << pid + << ", status=" << status.to_string() << ", response=" << health_response; + } + } +#endif //BE_TEST + return st; +} + +void CdcClientMgr::request_cdc_client_impl(const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) { + brpc::ClosureGuard closure_guard(done); + + // Start CDC client if not started + Status start_st = start_cdc_client(result); + if (!start_st.ok()) { + LOG(ERROR) << "Failed to start CDC client, status=" << start_st.to_string(); + start_st.to_protobuf(result->mutable_status()); + return; + } + + std::string cdc_response; + Status st = send_request_to_cdc_client(request->api(), request->params(), &cdc_response); + result->set_response(cdc_response); + st.to_protobuf(result->mutable_status()); +} + +Status CdcClientMgr::send_request_to_cdc_client(const std::string& api, + const std::string& params_body, + std::string* response) { + std::string remote_url_prefix = + fmt::format("http://127.0.0.1:{}{}", doris::config::cdc_client_port, api); + + auto cdc_request = [&remote_url_prefix, response, ¶ms_body](HttpClient* client) { + RETURN_IF_ERROR(client->init(remote_url_prefix)); + client->set_timeout_ms(doris::config::request_cdc_client_timeout_ms); + if (!params_body.empty()) { + client->set_payload(params_body); + } + client->set_content_type("application/json"); + client->set_method(POST); + RETURN_IF_ERROR(client->execute(response)); + return Status::OK(); + }; + + return HttpClient::execute_with_retry(3, 1, cdc_request); +} + +} // namespace doris diff --git a/be/src/runtime/cdc_client_mgr.h b/be/src/runtime/cdc_client_mgr.h new file mode 100644 index 00000000000000..077097086daba2 --- /dev/null +++ b/be/src/runtime/cdc_client_mgr.h @@ -0,0 +1,63 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#pragma once + +#include + +#include +#include +#include + +#include "common/status.h" + +namespace google::protobuf { +class Closure; +class RpcController; +} // namespace google::protobuf + +namespace doris { + +class CdcClientMgr { +public: + CdcClientMgr(); + ~CdcClientMgr(); + + void stop(); + + // Request CDC client to handle a request + void request_cdc_client_impl(const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, google::protobuf::Closure* done); + + Status send_request_to_cdc_client(const std::string& api, const std::string& params_body, + std::string* response); + + Status start_cdc_client(PRequestCdcClientResult* result); + +#ifdef BE_TEST + // For testing only: get current child PID + pid_t get_child_pid() const { return _child_pid.load(); } + // For testing only: set child PID directly + void set_child_pid_for_test(pid_t pid) { _child_pid.store(pid); } +#endif + +private: + std::mutex _start_mutex; + std::atomic _child_pid {0}; +}; + +} // namespace doris diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 9200f800c0b2aa..48435bc718241a 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -116,6 +116,7 @@ class HeartbeatFlags; class FrontendServiceClient; class FileMetaCache; class GroupCommitMgr; +class CdcClientMgr; class TabletSchemaCache; class TabletColumnObjectPool; class UserFunctionCache; @@ -275,6 +276,7 @@ class ExecEnv { SmallFileMgr* small_file_mgr() { return _small_file_mgr; } doris::vectorized::SpillStreamManager* spill_stream_mgr() { return _spill_stream_mgr; } GroupCommitMgr* group_commit_mgr() { return _group_commit_mgr; } + CdcClientMgr* cdc_client_mgr() { return _cdc_client_mgr; } const std::vector& store_paths() const { return _store_paths; } @@ -505,6 +507,7 @@ class ExecEnv { // ip:brpc_port -> frontend_indo std::map _frontends; GroupCommitMgr* _group_commit_mgr = nullptr; + CdcClientMgr* _cdc_client_mgr = nullptr; // Maybe we should use unique_ptr, but it need complete type, which means we need // to include many headers, and for some cpp file that do not need class like TabletSchemaCache, diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index dce07a71aa8a8a..10cbcf92661505 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -70,6 +70,7 @@ #include "pipeline/task_scheduler.h" #include "runtime/broker_mgr.h" #include "runtime/cache/result_cache.h" +#include "runtime/cdc_client_mgr.h" #include "runtime/client_cache.h" #include "runtime/exec_env.h" #include "runtime/external_scan_context_mgr.h" @@ -335,6 +336,7 @@ Status ExecEnv::_init(const std::vector& store_paths, RETURN_IF_ERROR(_routine_load_task_executor->init(MemInfo::mem_limit())); _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); _group_commit_mgr = new GroupCommitMgr(this); + _cdc_client_mgr = new CdcClientMgr(); _memtable_memory_limiter = std::make_unique(); _load_stream_map_pool = std::make_unique(); _delta_writer_v2_pool = std::make_unique(); @@ -830,6 +832,7 @@ void ExecEnv::destroy() { SAFE_DELETE(_result_mgr); SAFE_DELETE(_file_meta_cache); SAFE_DELETE(_group_commit_mgr); + SAFE_DELETE(_cdc_client_mgr); SAFE_DELETE(_routine_load_task_executor); SAFE_DELETE(_stream_load_recorder_manager); // _stream_load_executor diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 03aef2669d5fbd..6c89be3e143066 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -83,6 +83,7 @@ #include "olap/txn_manager.h" #include "olap/wal/wal_manager.h" #include "runtime/cache/result_cache.h" +#include "runtime/cdc_client_mgr.h" #include "runtime/descriptors.h" #include "runtime/exec_env.h" #include "runtime/fold_constant_executor.h" @@ -2395,5 +2396,19 @@ void PInternalService::get_tablet_rowsets(google::protobuf::RpcController* contr Status::OK().to_protobuf(response->mutable_status()); } +void PInternalService::request_cdc_client(google::protobuf::RpcController* controller, + const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) { + bool ret = _heavy_work_pool.try_offer([this, request, result, done]() { + _exec_env->cdc_client_mgr()->request_cdc_client_impl(request, result, done); + }); + + if (!ret) { + offer_failed(result, done, _heavy_work_pool); + return; + } +} + #include "common/compile_check_avoid_end.h" } // namespace doris diff --git a/be/src/service/internal_service.h b/be/src/service/internal_service.h index d73501bfc80861..8535cc2dc98698 100644 --- a/be/src/service/internal_service.h +++ b/be/src/service/internal_service.h @@ -232,6 +232,11 @@ class PInternalService : public PBackendService { PGetTabletRowsetsResponse* response, google::protobuf::Closure* done) override; + void request_cdc_client(google::protobuf::RpcController* controller, + const PRequestCdcClientRequest* request, + PRequestCdcClientResult* result, + google::protobuf::Closure* done) override; + private: void _exec_plan_fragment_in_pthread(google::protobuf::RpcController* controller, const PExecPlanFragmentRequest* request, diff --git a/be/test/runtime/cdc_client_mgr_test.cpp b/be/test/runtime/cdc_client_mgr_test.cpp new file mode 100644 index 00000000000000..4e9f880d18eb25 --- /dev/null +++ b/be/test/runtime/cdc_client_mgr_test.cpp @@ -0,0 +1,641 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "runtime/cdc_client_mgr.h" + +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include "common/config.h" +#include "common/status.h" + +namespace doris { + +class CdcClientMgrTest : public testing::Test { +public: + void SetUp() override { + // Save original environment variables + _original_doris_home = getenv("DORIS_HOME"); + _original_log_dir = getenv("LOG_DIR"); + _original_java_home = getenv("JAVA_HOME"); + + // Use existing DORIS_HOME + const char* doris_home = std::getenv("DORIS_HOME"); + if (doris_home) { + _doris_home = doris_home; + _lib_dir = _doris_home + "/lib/cdc_client"; + _jar_path = _lib_dir + "/cdc-client.jar"; + + // Create lib directory and jar file if they don't exist + [[maybe_unused]] int ret1 = system(("mkdir -p " + _lib_dir).c_str()); + // Create a dummy jar file for testing if it doesn't exist + if (access(_jar_path.c_str(), F_OK) != 0) { + [[maybe_unused]] int ret2 = system(("touch " + _jar_path).c_str()); + _jar_created = true; + } + } + + // Use existing LOG_DIR or set a default + const char* log_dir = std::getenv("LOG_DIR"); + if (!log_dir) { + _log_dir = "/tmp/doris_test_log"; + setenv("LOG_DIR", _log_dir.c_str(), 1); + _log_dir_set = true; + } + } + + void TearDown() override { + // Restore original environment variables + if (_original_doris_home) { + setenv("DORIS_HOME", _original_doris_home, 1); + } else { + unsetenv("DORIS_HOME"); + } + + if (_original_log_dir) { + setenv("LOG_DIR", _original_log_dir, 1); + } else if (_log_dir_set) { + unsetenv("LOG_DIR"); + } + + if (_original_java_home) { + setenv("JAVA_HOME", _original_java_home, 1); + } else { + unsetenv("JAVA_HOME"); + } + + // Clean up created jar file if we created it + if (_jar_created && !_jar_path.empty()) { + [[maybe_unused]] int cleanup_ret = system(("rm -f " + _jar_path).c_str()); + } + } + +protected: + std::string _doris_home; + std::string _log_dir; + std::string _lib_dir; + std::string _jar_path; + const char* _original_doris_home = nullptr; + const char* _original_log_dir = nullptr; + const char* _original_java_home = nullptr; + bool _jar_created = false; + bool _log_dir_set = false; +}; + +// Test stop method when there's no child process +TEST_F(CdcClientMgrTest, StopWithoutChild) { + CdcClientMgr mgr; + // Should not crash + mgr.stop(); +} + +// Test stop when child process is already dead (covers lines 98-111: kill(pid, 0) == 0 is false) +TEST_F(CdcClientMgrTest, StopWhenProcessDead) { + CdcClientMgr mgr; + + // Start CDC client (sets PID to 99999 in BE_TEST mode) + PRequestCdcClientResult result; + Status status = mgr.start_cdc_client(&result); + EXPECT_TRUE(status.ok()); + EXPECT_GT(mgr.get_child_pid(), 0); + + // Stop - since PID 99999 doesn't exist, kill(99999, 0) will fail + // This should trigger the branch where kill(pid, 0) != 0 (process already dead) + mgr.stop(); + + // PID should be reset to 0 + EXPECT_EQ(mgr.get_child_pid(), 0); +} + +// Test stop with real process that exits gracefully (covers lines 98-111: graceful shutdown) +TEST_F(CdcClientMgrTest, StopWithRealProcessGraceful) { + CdcClientMgr mgr; + + // Use popen to start a background sleep process and get its PID + // This avoids fork() which conflicts with gcov/coverage tools + FILE* pipe = popen("sleep 10 & echo $!", "r"); + if (pipe) { + char buffer[128]; + if (fgets(buffer, sizeof(buffer), pipe) != nullptr) { + pid_t real_pid = std::atoi(buffer); + pclose(pipe); + + if (real_pid > 0) { + // Set the PID in the manager + mgr.set_child_pid_for_test(real_pid); + + // Call stop - process will respond to SIGTERM and exit + // This covers the graceful shutdown path + mgr.stop(); + + // Verify PID is reset + EXPECT_EQ(mgr.get_child_pid(), 0); + + // Clean up: make sure child is dead + kill(real_pid, SIGKILL); + waitpid(real_pid, nullptr, WNOHANG); + } + } else { + pclose(pipe); + } + } +} + +// Test stop with real process that requires force kill (covers lines 98-111: force kill path) +TEST_F(CdcClientMgrTest, StopWithRealProcessForceKill) { + CdcClientMgr mgr; + + // Start a bash process that ignores SIGTERM by trapping it + // This process will not exit on SIGTERM, requiring SIGKILL + const char* script = "bash -c 'trap \"\" TERM; while true; do sleep 1; done' & echo $!"; + FILE* pipe = popen(script, "r"); + if (pipe) { + char buffer[128]; + if (fgets(buffer, sizeof(buffer), pipe) != nullptr) { + pid_t real_pid = std::atoi(buffer); + pclose(pipe); + + if (real_pid > 0) { + // Give the process a moment to start + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + + // Set the PID + mgr.set_child_pid_for_test(real_pid); + + // Call stop - should try graceful shutdown first, then force kill + // Since process ignores SIGTERM, it will still be alive after 200ms + // This should trigger the force kill path (lines 105-110) + mgr.stop(); + + // Verify PID is reset + EXPECT_EQ(mgr.get_child_pid(), 0); + + // Clean up: make sure child is dead + kill(real_pid, SIGKILL); + waitpid(real_pid, nullptr, WNOHANG); + } + } else { + pclose(pipe); + } + } +} + +// Test start_cdc_client with missing jar file +TEST_F(CdcClientMgrTest, StartCdcClientMissingJar) { + [[maybe_unused]] int rm_ret = system(("rm -f " + _jar_path).c_str()); + + CdcClientMgr mgr; + PRequestCdcClientResult result; + Status status = mgr.start_cdc_client(&result); + + EXPECT_FALSE(status.ok()); + EXPECT_TRUE(status.to_string().find("cdc-client.jar") != std::string::npos); + EXPECT_TRUE(result.has_status()); + + [[maybe_unused]] int touch_ret = system(("touch " + _jar_path).c_str()); +} + +// Test start_cdc_client with missing JAVA_HOME +TEST_F(CdcClientMgrTest, StartCdcClientMissingJavaHome) { + unsetenv("JAVA_HOME"); + + CdcClientMgr mgr; + PRequestCdcClientResult result; + Status status = mgr.start_cdc_client(&result); + + EXPECT_FALSE(status.ok()); + EXPECT_TRUE(status.to_string().find("JAVA_HOME") != std::string::npos); + EXPECT_TRUE(result.has_status()); +} + +// Test multiple calls to start_cdc_client +TEST_F(CdcClientMgrTest, StartCdcClientMultipleTimes) { + CdcClientMgr mgr; + PRequestCdcClientResult result1; + PRequestCdcClientResult result2; + + // Initially no child process + EXPECT_EQ(mgr.get_child_pid(), 0); + + // First call - should start CDC client + Status status1 = mgr.start_cdc_client(&result1); + EXPECT_TRUE(status1.ok()); + pid_t pid_after_first = mgr.get_child_pid(); + EXPECT_GT(pid_after_first, 0); // PID should be set + + // Second call - should detect already started and not restart + Status status2 = mgr.start_cdc_client(&result2); + EXPECT_TRUE(status2.ok()); + EXPECT_EQ(mgr.get_child_pid(), pid_after_first); // PID should not change! +} + +// Test destructor calls stop +TEST_F(CdcClientMgrTest, DestructorCallsStop) { + { + CdcClientMgr mgr; + PRequestCdcClientResult result; + [[maybe_unused]] Status status = mgr.start_cdc_client(&result); + // Destructor will be called when mgr goes out of scope + } + // If no crash, test passes + SUCCEED(); +} + +// Test send_request_to_cdc_client with explicitly empty params (covers params_body.empty() branch) +TEST_F(CdcClientMgrTest, SendRequestExplicitlyEmptyParams) { + CdcClientMgr mgr; + std::string response; + + // Explicitly test empty params_body path (line 309) + std::string empty_params = ""; + Status status = mgr.send_request_to_cdc_client("/health", empty_params, &response); + + // Expected to fail but should not crash + EXPECT_FALSE(status.ok()); +} + +// Test request_cdc_client_impl when start fails (covers lines 288-291) +TEST_F(CdcClientMgrTest, RequestCdcClientImplStartFailed) { + CdcClientMgr mgr; + PRequestCdcClientRequest request; + PRequestCdcClientResult result; + + request.set_api("/test/api"); + request.set_params("{}"); + + // Remove JAVA_HOME to make start_cdc_client fail + unsetenv("JAVA_HOME"); + + struct TestClosure : public google::protobuf::Closure { + void Run() override { called = true; } + bool called = false; + }; + TestClosure closure; + + mgr.request_cdc_client_impl(&request, &result, &closure); + + // Should have error status in result + EXPECT_TRUE(result.has_status()); + EXPECT_NE(result.status().status_code(), 0); + EXPECT_TRUE(closure.called); + + // Restore JAVA_HOME + if (_original_java_home) { + setenv("JAVA_HOME", _original_java_home, 1); + } +} + +// Test start_cdc_client with result +TEST_F(CdcClientMgrTest, StartCdcClientWithResult) { + CdcClientMgr mgr; + PRequestCdcClientResult result; + + EXPECT_EQ(mgr.get_child_pid(), 0); + + Status status = mgr.start_cdc_client(&result); + + // Should succeed + EXPECT_TRUE(status.ok()); + EXPECT_GT(mgr.get_child_pid(), 0); // PID should be set +} + +// Test start_cdc_client when environment is missing +TEST_F(CdcClientMgrTest, StartCdcClientMissingEnv) { + unsetenv("JAVA_HOME"); + + CdcClientMgr mgr; + PRequestCdcClientResult result; + Status status = mgr.start_cdc_client(&result); + + EXPECT_FALSE(status.ok()); + EXPECT_TRUE(status.to_string().find("JAVA_HOME") != std::string::npos); + EXPECT_TRUE(result.has_status()); + + // Restore JAVA_HOME + if (_original_java_home) { + setenv("JAVA_HOME", _original_java_home, 1); + } +} + +// Test concurrent start attempts +TEST_F(CdcClientMgrTest, ConcurrentStartAttempts) { + CdcClientMgr mgr; + std::vector threads; + std::atomic success_count {0}; + + // Initially no child process + EXPECT_EQ(mgr.get_child_pid(), 0); + + // Launch multiple threads trying to start CDC client + for (int i = 0; i < 5; ++i) { + threads.emplace_back([&mgr, &success_count]() { + PRequestCdcClientResult result; + Status status = mgr.start_cdc_client(&result); + if (status.ok()) { + success_count++; + } + }); + } + + for (auto& t : threads) { + t.join(); + } + + // All attempts should succeed (subsequent ones detect already started) + EXPECT_EQ(success_count.load(), 5); + + // CDC client should be started exactly once (PID is set) + EXPECT_GT(mgr.get_child_pid(), 0); +} + +// Test send_request with various HTTP methods implicitly +TEST_F(CdcClientMgrTest, SendRequestVariousEndpoints) { + CdcClientMgr mgr; + std::string response; + + // Test different API endpoints (reduced from 4 to 2 for speed) + std::vector apis = {"/actuator/health", "/api/submit"}; + + for (const auto& api : apis) { + Status status = mgr.send_request_to_cdc_client(api, "{}", &response); + // Expected to fail (no real CDC client running) but should not crash + EXPECT_FALSE(status.ok()); + } +} + +// Test that stop is idempotent +TEST_F(CdcClientMgrTest, StopIdempotent) { + CdcClientMgr mgr; + + // Multiple stops should not cause issues + mgr.stop(); + mgr.stop(); + mgr.stop(); + + // Should still be able to start after multiple stops + PRequestCdcClientResult result; + Status status = mgr.start_cdc_client(&result); + EXPECT_TRUE(status.ok()); +} + +// Test rapid start/stop cycles +TEST_F(CdcClientMgrTest, RapidStartStopCycles) { + CdcClientMgr mgr; + PRequestCdcClientResult result; + + for (int i = 0; i < 5; ++i) { + EXPECT_EQ(mgr.get_child_pid(), 0); // Should be stopped + + Status status = mgr.start_cdc_client(&result); + EXPECT_TRUE(status.ok()); + EXPECT_GT(mgr.get_child_pid(), 0); // Should be started + + mgr.stop(); + EXPECT_EQ(mgr.get_child_pid(), 0); // Should be stopped again + } +} + +// Test that multiple request_cdc_client_impl calls work +TEST_F(CdcClientMgrTest, MultipleRequestCalls) { + CdcClientMgr mgr; + + // Reduced from 5 to 2 iterations for speed + for (int i = 0; i < 2; ++i) { + PRequestCdcClientRequest request; + PRequestCdcClientResult result; + struct TestClosure : public google::protobuf::Closure { + void Run() override {} + }; + TestClosure closure; + + request.set_api("/test"); + request.set_params("{\"index\":" + std::to_string(i) + "}"); + + mgr.request_cdc_client_impl(&request, &result, &closure); + EXPECT_TRUE(result.has_status()); + } +} + +// Test behavior when DORIS_HOME has trailing slash +TEST_F(CdcClientMgrTest, DorisHomeWithTrailingSlash) { + if (_original_doris_home) { + std::string doris_home_with_slash = std::string(_original_doris_home) + "/"; + setenv("DORIS_HOME", doris_home_with_slash.c_str(), 1); + + CdcClientMgr mgr; + PRequestCdcClientResult result; + Status status = mgr.start_cdc_client(&result); + + // Should still work fine + EXPECT_TRUE(status.ok()); + + // Restore original + setenv("DORIS_HOME", _original_doris_home, 1); + } +} + +// Test send_request_to_cdc_client with very long API path +TEST_F(CdcClientMgrTest, SendRequestLongApiPath) { + CdcClientMgr mgr; + std::string response; + + std::string long_api = "/api/v1/very/long/path/to/endpoint/with/many/segments"; + Status status = mgr.send_request_to_cdc_client(long_api, "{\"data\":\"test\"}", &response); + + EXPECT_FALSE(status.ok()); +} + +// Test concurrent stop calls +TEST_F(CdcClientMgrTest, ConcurrentStopCalls) { + CdcClientMgr mgr; + + // Start CDC client first + PRequestCdcClientResult result; + Status status = mgr.start_cdc_client(&result); + EXPECT_TRUE(status.ok()); + + std::vector threads; + // Launch multiple threads trying to stop simultaneously + for (int i = 0; i < 3; ++i) { + threads.emplace_back([&mgr]() { mgr.stop(); }); + } + + for (auto& t : threads) { + t.join(); + } + + // Should be stopped cleanly + EXPECT_EQ(mgr.get_child_pid(), 0); +} + +// Test concurrent requests +TEST_F(CdcClientMgrTest, ConcurrentRequests) { + CdcClientMgr mgr; + std::vector threads; + std::atomic completed {0}; + + // Launch multiple threads making requests + for (int i = 0; i < 3; ++i) { + threads.emplace_back([&mgr, &completed, i]() { + PRequestCdcClientRequest request; + PRequestCdcClientResult result; + + request.set_api("/test/" + std::to_string(i)); + request.set_params("{\"id\":" + std::to_string(i) + "}"); + + struct TestClosure : public google::protobuf::Closure { + void Run() override {} + }; + TestClosure closure; + + mgr.request_cdc_client_impl(&request, &result, &closure); + completed++; + }); + } + + for (auto& t : threads) { + t.join(); + } + + EXPECT_EQ(completed.load(), 3); +} + +// Test start after failed start attempt +TEST_F(CdcClientMgrTest, StartAfterFailedStart) { + CdcClientMgr mgr; + PRequestCdcClientResult result1; + + // First attempt: remove JAVA_HOME to fail + unsetenv("JAVA_HOME"); + Status status1 = mgr.start_cdc_client(&result1); + EXPECT_FALSE(status1.ok()); + + // Restore JAVA_HOME + if (_original_java_home) { + setenv("JAVA_HOME", _original_java_home, 1); + } + + // Second attempt: should succeed now + PRequestCdcClientResult result2; + Status status2 = mgr.start_cdc_client(&result2); + EXPECT_TRUE(status2.ok()); +} + +// Test send_request with special characters in params +TEST_F(CdcClientMgrTest, SendRequestSpecialCharsInParams) { + CdcClientMgr mgr; + std::string response; + + std::string special_params = "{\"data\":\"test\\n\\t\\r\\\"\"}"; + Status status = mgr.send_request_to_cdc_client("/test", special_params, &response); + + EXPECT_FALSE(status.ok()); +} + +// Test request_cdc_client_impl with long params +TEST_F(CdcClientMgrTest, RequestWithLongParams) { + CdcClientMgr mgr; + PRequestCdcClientRequest request; + PRequestCdcClientResult result; + + request.set_api("/test"); + // Create a long params string + std::string long_params = "{\"data\":\""; + for (int i = 0; i < 1000; ++i) { + long_params += "x"; + } + long_params += "\"}"; + request.set_params(long_params); + + struct TestClosure : public google::protobuf::Closure { + void Run() override {} + }; + TestClosure closure; + + mgr.request_cdc_client_impl(&request, &result, &closure); + EXPECT_TRUE(result.has_status()); +} + +// Test multiple managers with concurrent operations +TEST_F(CdcClientMgrTest, MultipleManagersConcurrent) { + std::vector> managers; + std::vector threads; + + // Create 3 managers + for (int i = 0; i < 3; ++i) { + managers.push_back(std::make_unique()); + } + + // Each manager tries to start CDC client concurrently + for (int i = 0; i < 3; ++i) { + threads.emplace_back([&managers, i]() { + PRequestCdcClientResult result; + Status status = managers[i]->start_cdc_client(&result); + EXPECT_TRUE(status.ok()); + }); + } + + for (auto& t : threads) { + t.join(); + } + + // All should have succeeded + for (auto& mgr : managers) { + EXPECT_GT(mgr->get_child_pid(), 0); + } +} + +// Test start_cdc_client with result having pre-existing status +TEST_F(CdcClientMgrTest, StartWithPreExistingResultStatus) { + CdcClientMgr mgr; + PRequestCdcClientResult result; + + // Pre-populate result with some status + auto* status_pb = result.mutable_status(); + status_pb->set_status_code(999); + status_pb->add_error_msgs("pre-existing error"); + + Status status = mgr.start_cdc_client(&result); + + // Should succeed + EXPECT_TRUE(status.ok()); + // Note: start_cdc_client only updates result status on error, not on success + // So the pre-existing status (999) will remain unchanged + EXPECT_EQ(result.status().status_code(), 999); +} + +// Test send_request_to_cdc_client with empty API +TEST_F(CdcClientMgrTest, SendRequestEmptyApi) { + CdcClientMgr mgr; + std::string response; + + Status status = mgr.send_request_to_cdc_client("", "{\"test\":\"data\"}", &response); + + EXPECT_FALSE(status.ok()); +} + +} // namespace doris diff --git a/build.sh b/build.sh index 58c938e83c4d59..ef848701565051 100755 --- a/build.sh +++ b/build.sh @@ -142,6 +142,7 @@ if ! OPTS="$(getopt \ -l 'spark-dpp' \ -l 'hive-udf' \ -l 'be-java-extensions' \ + -l 'be-cdc-client' \ -l 'be-extension-ignore:' \ -l 'clean' \ -l 'coverage' \ @@ -165,6 +166,7 @@ BUILD_INDEX_TOOL='OFF' BUILD_BENCHMARK='OFF' BUILD_TASK_EXECUTOR_SIMULATOR='OFF' BUILD_BE_JAVA_EXTENSIONS=0 +BUILD_BE_CDC_CLIENT=0 BUILD_OBS_DEPENDENCIES=1 BUILD_COS_DEPENDENCIES=1 BUILD_HIVE_UDF=0 @@ -189,6 +191,7 @@ if [[ "$#" == 1 ]]; then BUILD_BENCHMARK='OFF' BUILD_HIVE_UDF=1 BUILD_BE_JAVA_EXTENSIONS=1 + BUILD_BE_CDC_CLIENT=1 CLEAN=0 else while true; do @@ -202,6 +205,7 @@ else --be) BUILD_BE=1 BUILD_BE_JAVA_EXTENSIONS=1 + BUILD_BE_CDC_CLIENT=1 shift ;; --cloud) @@ -247,6 +251,10 @@ else BUILD_BE_JAVA_EXTENSIONS=1 shift ;; + --be-cdc-client) + BUILD_BE_CDC_CLIENT=1 + shift + ;; --exclude-obs-dependencies) BUILD_OBS_DEPENDENCIES=0 shift @@ -306,6 +314,7 @@ else BUILD_TASK_EXECUTOR_SIMULATOR='OFF' BUILD_HIVE_UDF=1 BUILD_BE_JAVA_EXTENSIONS=1 + BUILD_BE_CDC_CLIENT=1 CLEAN=0 fi fi @@ -436,6 +445,14 @@ if [[ -n "${DISABLE_BE_JAVA_EXTENSIONS}" ]]; then fi fi +if [[ -n "${DISABLE_BE_CDC_CLIENT}" ]]; then + if [[ "${DISABLE_BE_CDC_CLIENT}" == "ON" ]]; then + BUILD_BE_CDC_CLIENT=0 + else + BUILD_BE_CDC_CLIENT=1 + fi +fi + if [[ -n "${DISABLE_BUILD_UI}" ]]; then if [[ "${DISABLE_BUILD_UI}" == "ON" ]]; then BUILD_UI=0 @@ -502,6 +519,7 @@ echo "Get params: BUILD_BENCHMARK -- ${BUILD_BENCHMARK} BUILD_TASK_EXECUTOR_SIMULATOR -- ${BUILD_TASK_EXECUTOR_SIMULATOR} BUILD_BE_JAVA_EXTENSIONS -- ${BUILD_BE_JAVA_EXTENSIONS} + BUILD_BE_CDC_CLIENT -- ${BUILD_BE_CDC_CLIENT} BUILD_HIVE_UDF -- ${BUILD_HIVE_UDF} PARALLEL -- ${PARALLEL} CLEAN -- ${CLEAN} @@ -993,6 +1011,15 @@ if [[ "${BUILD_BROKER}" -eq 1 ]]; then cd "${DORIS_HOME}" fi +if [[ "${BUILD_BE_CDC_CLIENT}" -eq 1 ]]; then + install -d "${DORIS_OUTPUT}/be/lib/cdc_client" + cd "${DORIS_HOME}/fs_brokers/cdc_client" + ./build.sh + rm -rf "${DORIS_OUTPUT}/be/lib/cdc_client"/* + cp -r -p "${DORIS_HOME}/fs_brokers/cdc_client/target/cdc-client.jar" "${DORIS_OUTPUT}/be/lib/cdc_client/" + cd "${DORIS_HOME}" +fi + if [[ ${BUILD_CLOUD} -eq 1 ]]; then rm -rf "${DORIS_HOME}/output/ms" rm -rf "${DORIS_HOME}/cloud/output/lib/hadoop_hdfs" @@ -1018,4 +1045,4 @@ if [[ -n "${DORIS_POST_BUILD_HOOK}" ]]; then eval "${DORIS_POST_BUILD_HOOK}" fi -exit 0 +exit 0 \ No newline at end of file diff --git a/docker/thirdparties/docker-compose/mysql/my.cnf b/docker/thirdparties/docker-compose/mysql/my.cnf index a6ef77e8f1910e..1f9664e8afc2b5 100644 --- a/docker/thirdparties/docker-compose/mysql/my.cnf +++ b/docker/thirdparties/docker-compose/mysql/my.cnf @@ -17,6 +17,9 @@ character-set-server=utf8 collation-server=utf8_bin max_connections=1000 +log-bin=mysql-bin +binlog_format=ROW +server-id=1 [client] default-character-set=utf8 [mysql] diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java index eb33df5003c10e..f887d3e9a5d172 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java @@ -1269,6 +1269,12 @@ public class Config extends ConfigBase { @ConfField(mutable = true, masterOnly = true) public static int routine_load_task_min_timeout_sec = 60; + /** + * streaming task load timeout is equal to maxIntervalS * streaming_task_timeout_multiplier. + */ + @ConfField(mutable = true, masterOnly = true) + public static int streaming_task_timeout_multiplier = 10; + /** * the max timeout of get kafka meta. */ diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/DataSourceConfigKeys.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/DataSourceConfigKeys.java new file mode 100644 index 00000000000000..074c100579c154 --- /dev/null +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/DataSourceConfigKeys.java @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.cdc; + +public class DataSourceConfigKeys { + public static final String JDBC_URL = "jdbc_url"; + public static final String DRIVER_URL = "driver_url"; + public static final String DRIVER_CLASS = "driver_class"; + public static final String USER = "user"; + public static final String PASSWORD = "password"; + public static final String DATABASE = "database"; + public static final String INCLUDE_TABLES = "include_tables"; + public static final String EXCLUDE_TABLES = "exclude_tables"; + // initial,earliest,latest,{binlog,postion},\d{13} + public static final String OFFSET = "offset"; + public static final String OFFSET_INITIAL = "initial"; + public static final String OFFSET_EARLIEST = "earliest"; + public static final String OFFSET_LATEST = "latest"; + public static final String SPLIT_SIZE = "split_size"; +} diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/CompareOffsetRequest.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/CompareOffsetRequest.java new file mode 100644 index 00000000000000..8449afd96281e1 --- /dev/null +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/CompareOffsetRequest.java @@ -0,0 +1,44 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.cdc.request; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; + +import java.util.Map; + +@Getter +@Setter +@AllArgsConstructor +@NoArgsConstructor +public class CompareOffsetRequest extends JobBaseConfig { + private Map offsetFirst; + private Map offsetSecond; + + public CompareOffsetRequest(Long jobId, + String sourceType, + Map sourceProperties, + Map offsetFirst, + Map offsetSecond) { + super(jobId, sourceType, sourceProperties); + this.offsetFirst = offsetFirst; + this.offsetSecond = offsetSecond; + } +} diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchRecordRequest.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchRecordRequest.java new file mode 100644 index 00000000000000..f11539e68324d1 --- /dev/null +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchRecordRequest.java @@ -0,0 +1,38 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.cdc.request; + +import lombok.Data; +import lombok.EqualsAndHashCode; + +@Data +@EqualsAndHashCode(callSuper = true) +public class FetchRecordRequest extends JobBaseRecordRequest { + private boolean reload = true; + private int fetchSize; + + @Override + public boolean isReload() { + return reload; + } + + @Override + public int getFetchSize() { + return fetchSize; + } +} diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchTableSplitsRequest.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchTableSplitsRequest.java new file mode 100644 index 00000000000000..f855e373958687 --- /dev/null +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/FetchTableSplitsRequest.java @@ -0,0 +1,41 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.cdc.request; + +import lombok.AllArgsConstructor; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; + +import java.util.Map; + +@Getter +@Setter +@EqualsAndHashCode(callSuper = true) +@AllArgsConstructor +@NoArgsConstructor +public class FetchTableSplitsRequest extends JobBaseConfig { + private String snapshotTable; + + public FetchTableSplitsRequest(Long jobId, String name, + Map sourceProperties, String snapshotTable) { + super(jobId, name, sourceProperties); + this.snapshotTable = snapshotTable; + } +} diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseConfig.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseConfig.java new file mode 100644 index 00000000000000..bfdbf6a34558a9 --- /dev/null +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseConfig.java @@ -0,0 +1,33 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.cdc.request; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +import java.util.Map; + +@Data +@AllArgsConstructor +@NoArgsConstructor +public class JobBaseConfig { + private Long jobId; + private String dataSource; + private Map config; +} diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseRecordRequest.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseRecordRequest.java new file mode 100644 index 00000000000000..a9a1be374dbede --- /dev/null +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/JobBaseRecordRequest.java @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.cdc.request; + +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.Setter; + +import java.util.Map; + +@Getter +@Setter +@EqualsAndHashCode(callSuper = true) +public abstract class JobBaseRecordRequest extends JobBaseConfig { + protected Map meta; + + public abstract boolean isReload(); + + public abstract int getFetchSize(); +} diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/WriteRecordRequest.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/WriteRecordRequest.java new file mode 100644 index 00000000000000..a75edfcf7fb718 --- /dev/null +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/request/WriteRecordRequest.java @@ -0,0 +1,41 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.cdc.request; + +import lombok.Data; +import lombok.EqualsAndHashCode; + +@Data +@EqualsAndHashCode(callSuper = true) +public class WriteRecordRequest extends JobBaseRecordRequest { + private long maxInterval; + private String targetDb; + private String token; + private String frontendAddress; + private String taskId; + + @Override + public boolean isReload() { + return true; + } + + @Override + public int getFetchSize() { + return Integer.MAX_VALUE; + } +} diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/AbstractSourceSplit.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/AbstractSourceSplit.java new file mode 100644 index 00000000000000..ab8fda340cf65e --- /dev/null +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/AbstractSourceSplit.java @@ -0,0 +1,38 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.cdc.split; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; + +import java.io.Serializable; + +@Getter +@Setter +@AllArgsConstructor +@NoArgsConstructor +public abstract class AbstractSourceSplit implements Serializable { + private static final long serialVersionUID = 1L; + protected String splitId; + + public boolean snapshotSplit() { + return this instanceof SnapshotSplit; + } +} diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/BinlogSplit.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/BinlogSplit.java new file mode 100644 index 00000000000000..169c68e2b4499f --- /dev/null +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/BinlogSplit.java @@ -0,0 +1,53 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.cdc.split; + +import com.google.gson.Gson; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.Setter; + +import java.util.List; +import java.util.Map; + +@Getter +@Setter +@EqualsAndHashCode(callSuper = true) +public class BinlogSplit extends AbstractSourceSplit { + private static final long serialVersionUID = 1L; + public static final String BINLOG_SPLIT_ID = "binlog-split"; + private Map startingOffset; + private Map endingOffset; + // binlog split meta, first binlog split requires + private List finishedSplits; + + public BinlogSplit() { + this.splitId = BINLOG_SPLIT_ID; + } + + public BinlogSplit(Map startingOffset) { + this.splitId = BINLOG_SPLIT_ID; + this.startingOffset = startingOffset; + } + + @Override + public String toString() { + return new Gson().toJson(this); + } + +} diff --git a/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/SnapshotSplit.java b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/SnapshotSplit.java new file mode 100644 index 00000000000000..99a84d1aaeb717 --- /dev/null +++ b/fe/fe-common/src/main/java/org/apache/doris/job/cdc/split/SnapshotSplit.java @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.cdc.split; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; + +import java.util.List; +import java.util.Map; + +@Getter +@Setter +@EqualsAndHashCode(callSuper = true) +@NoArgsConstructor +public class SnapshotSplit extends AbstractSourceSplit { + private static final long serialVersionUID = 1L; + private static ObjectMapper objectMapper = new ObjectMapper(); + private String tableId; + private List splitKey; + private Object[] splitStart; + private Object[] splitEnd; + private Map highWatermark; + + public SnapshotSplit( + String splitId, + String tableId, + List splitKey, + Object[] splitStart, + Object[] splitEnd, + Map highWatermark) { + super(splitId); + this.tableId = tableId; + this.splitKey = splitKey; + this.splitStart = splitStart; + this.splitEnd = splitEnd; + this.highWatermark = highWatermark; + } + + @Override + public String toString() { + return new Gson().toJson(this); + } +} diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index 97edf706782be6..290a3c0dcb5164 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -98,8 +98,14 @@ materializedViewStatement | CANCEL MATERIALIZED VIEW TASK taskId=INTEGER_VALUE ON mvName=multipartIdentifier #cancelMTMVTask | SHOW CREATE MATERIALIZED VIEW mvName=multipartIdentifier #showCreateMTMV ; + +jobFromToClause + : FROM sourceType=identifier LEFT_PAREN sourceProperties=propertyItemList RIGHT_PAREN + TO DATABASE targetDb=identifier (LEFT_PAREN targetProperties=propertyItemList RIGHT_PAREN)? + ; + supportedJobStatement - : CREATE JOB label=multipartIdentifier propertyClause? + : CREATE JOB label=multipartIdentifier jobProperties=propertyClause? ON (STREAMING | SCHEDULE( (EVERY timeInterval=INTEGER_VALUE timeUnit=identifier (STARTS (startTime=STRING_LITERAL | CURRENT_TIMESTAMP))? @@ -108,10 +114,12 @@ supportedJobStatement (AT (atTime=STRING_LITERAL | CURRENT_TIMESTAMP)) ) ) - commentSpec? - DO supportedDmlStatement #createScheduledJob + commentSpec? + (jobFromToClause | DO supportedDmlStatement ) #createScheduledJob | PAUSE JOB WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #pauseJob - | ALTER JOB (jobName=multipartIdentifier) (propertyClause | supportedDmlStatement | propertyClause supportedDmlStatement) #alterJob + | ALTER JOB (jobName=multipartIdentifier) + (propertyClause | supportedDmlStatement | propertyClause supportedDmlStatement + | jobFromToClause | propertyClause jobFromToClause) #alterJob | DROP JOB (IF EXISTS)? WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #dropJob | RESUME JOB WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) #resumeJob | CANCEL TASK WHERE (jobNameKey=identifier) EQ (jobNameValue=STRING_LITERAL) AND (taskIdKey=identifier) EQ (taskIdValue=INTEGER_VALUE) #cancelJobTask diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java index ec89fc9d2b1305..0b2b81f4d0dbf9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcClient.java @@ -406,6 +406,30 @@ public List getColumnsFromJdbc(String remoteDbName, String remoteTableNa return dorisTableSchema; } + /** + * get primary keys of one table + */ + public List getPrimaryKeys(String remoteDbName, String remoteTableName) { + Connection conn = getConnection(); + ResultSet rs = null; + List primaryKeys = Lists.newArrayList(); + try { + DatabaseMetaData databaseMetaData = conn.getMetaData(); + String catalogName = getCatalogName(conn); + rs = databaseMetaData.getPrimaryKeys(catalogName, remoteDbName, remoteTableName); + while (rs.next()) { + String fieldName = rs.getString("COLUMN_NAME"); + primaryKeys.add(fieldName); + } + } catch (SQLException e) { + throw new JdbcClientException("failed to get jdbc primary key info for remote table `%s.%s`: %s", + remoteDbName, remoteTableName, Util.getRootCauseMessage(e)); + } finally { + close(rs, conn); + } + return primaryKeys; + } + // protected methods, for subclass to override protected String getCatalogName(Connection conn) throws SQLException { return conn.getCatalog(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java index 129c2e04a3a6bc..09208fd4c2157c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/jdbc/client/JdbcMySQLClient.java @@ -185,6 +185,27 @@ public List getJdbcColumnsInfo(String remoteDbName, String remo return tableSchema; } + @Override + public List getPrimaryKeys(String remoteDbName, String remoteTableName) { + Connection conn = getConnection(); + ResultSet rs = null; + List primaryKeys = Lists.newArrayList(); + try { + DatabaseMetaData databaseMetaData = conn.getMetaData(); + rs = databaseMetaData.getPrimaryKeys(remoteDbName, null, remoteTableName); + while (rs.next()) { + String fieldName = rs.getString("COLUMN_NAME"); + primaryKeys.add(fieldName); + } + } catch (SQLException e) { + throw new JdbcClientException("failed to get jdbc primary key info for remote table `%s.%s`: %s", + remoteDbName, remoteTableName, Util.getRootCauseMessage(e)); + } finally { + close(rs, conn); + } + return primaryKeys; + } + protected String getCatalogName(Connection conn) throws SQLException { return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java index 4fcd432051f7ee..571a6dc50dee9d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java @@ -29,7 +29,6 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.LoadException; import org.apache.doris.common.Pair; -import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugPointUtil; import org.apache.doris.httpv2.entity.ResponseEntityBuilder; import org.apache.doris.httpv2.entity.RestBaseResult; @@ -591,18 +590,6 @@ private Pair splitHostAndPort(String hostPort) throws AnalysisE return Pair.of(pair[0], port); } - // NOTE: This function can only be used for AuditlogPlugin stream load for now. - // AuditlogPlugin should be re-disigned carefully, and blow method focuses on - // temporarily addressing the users' needs for audit logs. - // So this function is not widely tested under general scenario - private boolean checkClusterToken(String token) { - try { - return Env.getCurrentEnv().getTokenManager().checkAuthToken(token); - } catch (UserException e) { - throw new UnauthorizedException(e.getMessage()); - } - } - // NOTE: This function can only be used for AuditlogPlugin stream load for now. // AuditlogPlugin should be re-disigned carefully, and blow method focuses on // temporarily addressing the users' needs for audit logs. diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RestBaseController.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RestBaseController.java index 25ab947c19fc2e..6a87d9d8c975b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RestBaseController.java +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/RestBaseController.java @@ -22,6 +22,7 @@ import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.Config; import org.apache.doris.common.FeConstants; +import org.apache.doris.common.UserException; import org.apache.doris.common.util.NetUtils; import org.apache.doris.httpv2.controller.BaseController; import org.apache.doris.httpv2.entity.ResponseEntityBuilder; @@ -232,6 +233,18 @@ public boolean checkForwardToMaster(HttpServletRequest request) { return !Env.getCurrentEnv().isMaster(); } + // NOTE: This function can only be used for AuditlogPlugin stream load for now. + // AuditlogPlugin should be re-disigned carefully, and blow method focuses on + // temporarily addressing the users' needs for audit logs. + // So this function is not widely tested under general scenario + protected boolean checkClusterToken(String token) { + try { + return Env.getCurrentEnv().getTokenManager().checkAuthToken(token); + } catch (UserException e) { + throw new UnauthorizedException(e.getMessage()); + } + } + private String getRequestBody(HttpServletRequest request) throws IOException { BufferedReader reader = request.getReader(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StreamingJobAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StreamingJobAction.java new file mode 100644 index 00000000000000..53610142f12bc2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/StreamingJobAction.java @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.httpv2.rest; + +import org.apache.doris.catalog.Env; +import org.apache.doris.httpv2.entity.ResponseEntityBuilder; +import org.apache.doris.httpv2.exception.UnauthorizedException; +import org.apache.doris.job.base.AbstractJob; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; + +import com.google.common.base.Strings; +import jakarta.servlet.http.HttpServletRequest; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.RestController; + +@RestController +public class StreamingJobAction extends RestBaseController { + private static final Logger LOG = LogManager.getLogger(StreamingJobAction.class); + + @RequestMapping(path = "/api/streaming/commit_offset", method = RequestMethod.PUT) + public Object commitOffset(@RequestBody CommitOffsetRequest offsetRequest, HttpServletRequest request) { + String authToken = request.getHeader("token"); + // if auth token is not null, check it first + if (!Strings.isNullOrEmpty(authToken)) { + if (!checkClusterToken(authToken)) { + throw new UnauthorizedException("Invalid token: " + authToken); + } + return updateOffset(offsetRequest); + } else { + // only use for token + throw new UnauthorizedException("Miss token"); + } + } + + private Object updateOffset(CommitOffsetRequest offsetRequest) { + AbstractJob job = Env.getCurrentEnv().getJobManager().getJob(offsetRequest.getJobId()); + if (job == null) { + String errMsg = "Job " + offsetRequest.getJobId() + " not found"; + return ResponseEntityBuilder.okWithCommonError(errMsg); + } + if (!(job instanceof StreamingInsertJob)) { + return ResponseEntityBuilder + .okWithCommonError("Job " + offsetRequest.getJobId() + " is not a streaming job"); + } + + StreamingInsertJob streamingJob = (StreamingInsertJob) job; + try { + streamingJob.commitOffset(offsetRequest); + return ResponseEntityBuilder.ok("Offset committed successfully"); + } catch (Exception e) { + LOG.warn("Failed to commit offset for job {}, offset {}: {}", offsetRequest.getJobId(), + offsetRequest.getOffset(), e.getMessage()); + return ResponseEntityBuilder.okWithCommonError(e.getMessage()); + } + } + + @Getter + @Setter + @NoArgsConstructor + public static class CommitOffsetRequest { + public long jobId; + public long taskId; + public String offset; + public long scannedRows; + public long scannedBytes; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/common/DataSourceType.java b/fe/fe-core/src/main/java/org/apache/doris/job/common/DataSourceType.java new file mode 100644 index 00000000000000..4ba5670fbd3410 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/common/DataSourceType.java @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.common; + +public enum DataSourceType { + MYSQL +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java new file mode 100644 index 00000000000000..2618d8a122cc19 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/AbstractStreamingTask.java @@ -0,0 +1,166 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.extensions.insert.streaming; + +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.FeConstants; +import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.job.base.Job; +import org.apache.doris.job.common.TaskStatus; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.offset.Offset; +import org.apache.doris.thrift.TCell; +import org.apache.doris.thrift.TRow; + +import lombok.Getter; +import lombok.Setter; +import lombok.extern.log4j.Log4j2; +import org.apache.commons.lang3.StringUtils; + +import java.util.concurrent.atomic.AtomicBoolean; + +@Log4j2 +@Getter +public abstract class AbstractStreamingTask { + private static final int MAX_RETRY = 3; + private static final String LABEL_SPLITTER = "_"; + private int retryCount = 0; + protected String labelName; + protected Offset runningOffset; + protected UserIdentity userIdentity; + @Setter + protected volatile TaskStatus status; + @Setter + protected String errMsg; + protected long jobId; + protected long taskId; + protected Long createTimeMs; + protected Long startTimeMs; + protected Long finishTimeMs; + @Getter + private AtomicBoolean isCanceled = new AtomicBoolean(false); + + public AbstractStreamingTask(long jobId, long taskId, UserIdentity userIdentity) { + this.jobId = jobId; + this.taskId = taskId; + this.userIdentity = userIdentity; + this.labelName = getJobId() + LABEL_SPLITTER + getTaskId(); + this.createTimeMs = System.currentTimeMillis(); + } + + public abstract void before() throws Exception; + + public abstract void run() throws JobException; + + public abstract boolean onSuccess() throws JobException; + + public abstract void closeOrReleaseResources(); + + public void execute() throws JobException { + while (retryCount <= MAX_RETRY) { + try { + before(); + run(); + onSuccess(); + return; + } catch (Exception e) { + if (TaskStatus.CANCELED.equals(status)) { + return; + } + this.errMsg = e.getMessage(); + retryCount++; + if (retryCount > MAX_RETRY) { + log.error("Task execution failed after {} retries.", MAX_RETRY, e); + onFail(e.getMessage()); + return; + } + log.warn("execute streaming task error, job id is {}, task id is {}, retrying {}/{}: {}", + jobId, taskId, retryCount, MAX_RETRY, e.getMessage()); + } finally { + // The cancel logic will call the closeOrReleased Resources method by itself. + // If it is also called here, + // it may result in the inability to obtain relevant information when canceling the task + if (!TaskStatus.CANCELED.equals(status)) { + closeOrReleaseResources(); + } + } + } + } + + protected void onFail(String errMsg) throws JobException { + if (getIsCanceled().get()) { + return; + } + this.errMsg = errMsg; + this.status = TaskStatus.FAILED; + this.finishTimeMs = System.currentTimeMillis(); + if (!isCallable()) { + return; + } + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + StreamingInsertJob streamingInsertJob = (StreamingInsertJob) job; + streamingInsertJob.onStreamTaskFail(this); + } + + protected boolean isCallable() { + if (status.equals(TaskStatus.CANCELED)) { + return false; + } + if (null != Env.getCurrentEnv().getJobManager().getJob(jobId)) { + return true; + } + return false; + } + + public void cancel(boolean needWaitCancelComplete) { + if (TaskStatus.SUCCESS.equals(status) || TaskStatus.FAILED.equals(status) + || TaskStatus.CANCELED.equals(status)) { + return; + } + status = TaskStatus.CANCELED; + if (getIsCanceled().get()) { + return; + } + getIsCanceled().getAndSet(true); + this.errMsg = "task cancelled"; + } + + /** + * show streaming insert task info detail + */ + public TRow getTvfInfo(String jobName) { + TRow trow = new TRow(); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(this.getTaskId()))); + trow.addToColumnValue(new TCell().setStringVal(String.valueOf(this.getJobId()))); + trow.addToColumnValue(new TCell().setStringVal(jobName)); + trow.addToColumnValue(new TCell().setStringVal(this.getLabelName())); + trow.addToColumnValue(new TCell().setStringVal(this.getStatus().name())); + // err msg + trow.addToColumnValue(new TCell().setStringVal(StringUtils.isNotBlank(errMsg) + ? errMsg : FeConstants.null_string)); + + // create time + trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(this.getCreateTimeMs()))); + trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? FeConstants.null_string + : TimeUtils.longToTimeString(this.getStartTimeMs()))); + // load end time + trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(this.getFinishTimeMs()))); + return trow; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java new file mode 100644 index 00000000000000..f8850dd9d70f19 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java @@ -0,0 +1,79 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.extensions.insert.streaming; + +import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.util.StreamingJobUtils; + +import com.google.common.collect.Sets; + +import java.util.Map; +import java.util.Set; + +public class DataSourceConfigValidator { + private static final Set ALLOW_SOURCE_KEYS = Sets.newHashSet( + DataSourceConfigKeys.JDBC_URL, + DataSourceConfigKeys.USER, + DataSourceConfigKeys.PASSWORD, + DataSourceConfigKeys.OFFSET, + DataSourceConfigKeys.DRIVER_URL, + DataSourceConfigKeys.DRIVER_CLASS, + DataSourceConfigKeys.DATABASE, + DataSourceConfigKeys.INCLUDE_TABLES, + DataSourceConfigKeys.EXCLUDE_TABLES + ); + + public static void validateSource(Map input) throws IllegalArgumentException { + for (Map.Entry entry : input.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + + if (!ALLOW_SOURCE_KEYS.contains(key)) { + throw new IllegalArgumentException("Unexpected key: '" + key + "'"); + } + + if (!isValidValue(key, value)) { + throw new IllegalArgumentException("Invalid value for key '" + key + "': " + value); + } + } + } + + public static void validateTarget(Map input) throws IllegalArgumentException { + for (Map.Entry entry : input.entrySet()) { + String key = entry.getKey(); + if (!key.startsWith(StreamingJobUtils.TABLE_PROPS_PREFIX)) { + throw new IllegalArgumentException("Only support target properties with prefix " + + StreamingJobUtils.TABLE_PROPS_PREFIX); + } + } + } + + private static boolean isValidValue(String key, String value) { + if (value == null || value.isEmpty()) { + return false; + } + + if (key.equals(DataSourceConfigKeys.OFFSET) + && !(value.equals(DataSourceConfigKeys.OFFSET_INITIAL) + || value.equals(DataSourceConfigKeys.OFFSET_LATEST))) { + return false; + } + return true; + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index dd6bea8084074f..94f45461c07ffc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -18,6 +18,7 @@ package org.apache.doris.job.extensions.insert.streaming; import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; import org.apache.doris.cloud.proto.Cloud; import org.apache.doris.cloud.rpc.MetaServiceProxy; @@ -32,9 +33,12 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.httpv2.rest.StreamingJobAction.CommitOffsetRequest; import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.base.TimerDefinition; +import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.common.FailureReason; import org.apache.doris.job.common.IntervalUnit; import org.apache.doris.job.common.JobStatus; @@ -47,6 +51,8 @@ import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; import org.apache.doris.job.offset.SourceOffsetProviderFactory; +import org.apache.doris.job.offset.jdbc.JdbcSourceOffsetProvider; +import org.apache.doris.job.util.StreamingJobUtils; import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.loadv2.LoadStatistic; import org.apache.doris.mysql.privilege.PrivPredicate; @@ -54,6 +60,7 @@ import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.trees.plans.commands.AlterJobCommand; +import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; import org.apache.doris.nereids.trees.plans.commands.info.BaseViewInfo; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; import org.apache.doris.nereids.trees.plans.commands.insert.InsertUtils; @@ -82,6 +89,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -93,7 +101,13 @@ public class StreamingInsertJob extends AbstractJob> implements TxnStateChangeCallback, GsonPostProcessable { private long dbId; + // Streaming job statistics, all persisted in txn attachment private StreamingJobStatistic jobStatistic = new StreamingJobStatistic(); + // Non-txn persisted statistics, used for streaming multi task + @Getter + @Setter + @SerializedName("ntjs") + private StreamingJobStatistic nonTxnJobStatistic = new StreamingJobStatistic(); @Getter @Setter @SerializedName("fr") @@ -113,18 +127,36 @@ public class StreamingInsertJob extends AbstractJob originTvfProps; @Getter - StreamingInsertTask runningStreamTask; + AbstractStreamingTask runningStreamTask; SourceOffsetProvider offsetProvider; + @Getter + @Setter + @SerializedName("opp") + // The value to be persisted in offsetProvider + private String offsetProviderPersist; @Setter @Getter private long lastScheduleTaskTimestamp = -1L; private InsertIntoTableCommand baseCommand; private ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); - private ConcurrentLinkedQueue streamInsertTaskQueue = new ConcurrentLinkedQueue<>(); + private ConcurrentLinkedQueue streamInsertTaskQueue = new ConcurrentLinkedQueue<>(); @Setter @Getter private String jobRuntimeMsg = ""; + @Getter + @SerializedName("tdb") + private String targetDb; + @Getter + @SerializedName("ds") + private DataSourceType dataSourceType; + @Getter + @SerializedName("sprops") + private Map sourceProperties; + @Getter + @SerializedName("tprops") + private Map targetProperties; + public StreamingInsertJob(String jobName, JobStatus jobStatus, String dbName, @@ -137,7 +169,89 @@ public StreamingInsertJob(String jobName, super(Env.getCurrentEnv().getNextId(), jobName, jobStatus, dbName, comment, createUser, jobConfig, createTimeMs, executeSql); this.properties = properties; - init(); + initInsertJob(); + } + + // for streaming job from...to database + public StreamingInsertJob(String jobName, + JobStatus jobStatus, + String dbName, + String comment, + UserIdentity createUser, + JobExecutionConfiguration jobConfig, + Long createTimeMs, + String executeSql, + Map properties, + String targetDb, + DataSourceType dataSourceType, + Map sourceProperties, + Map targetProperties) { + super(Env.getCurrentEnv().getNextId(), jobName, jobStatus, dbName, comment, createUser, + jobConfig, createTimeMs, executeSql); + this.properties = properties; + this.targetDb = targetDb; + this.dataSourceType = dataSourceType; + this.sourceProperties = sourceProperties; + this.targetProperties = targetProperties; + initSourceJob(); + } + + /** + * Initialize job from source to database, like multi table mysql to doris. + * 1. get mysql connection info from sourceProperties + * 2. fetch table list from mysql + * 3. create doris table if not exists + * 4. check whether need full data sync + * 5. need => fetch split and write to system table + */ + private void initSourceJob() { + try { + init(); + checkRequiredSourceProperties(); + List createTbls = createTableIfNotExists(); + this.offsetProvider = new JdbcSourceOffsetProvider(getJobId(), dataSourceType, sourceProperties); + JdbcSourceOffsetProvider rdsOffsetProvider = (JdbcSourceOffsetProvider) this.offsetProvider; + rdsOffsetProvider.splitChunks(createTbls); + } catch (Exception ex) { + log.warn("init streaming job for {} failed", dataSourceType, ex); + throw new RuntimeException(ex.getMessage()); + } + } + + private void checkRequiredSourceProperties() { + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.JDBC_URL) != null, + "jdbc_url is required property"); + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.DRIVER_URL) != null, + "driver_url is required property"); + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.DRIVER_CLASS) != null, + "driver_class is required property"); + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.USER) != null, + "user is required property"); + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.PASSWORD) != null, + "password is required property"); + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.DATABASE) != null, + "database is required property"); + Preconditions.checkArgument(sourceProperties.get(DataSourceConfigKeys.INCLUDE_TABLES) != null + || sourceProperties.get(DataSourceConfigKeys.EXCLUDE_TABLES) != null, + "Either include_tables or exclude_tables must be specified"); + if (!sourceProperties.containsKey(DataSourceConfigKeys.OFFSET)) { + sourceProperties.put(DataSourceConfigKeys.OFFSET, DataSourceConfigKeys.OFFSET_LATEST); + } + } + + private List createTableIfNotExists() throws Exception { + List syncTbls = new ArrayList<>(); + List createTblCmds = StreamingJobUtils.generateCreateTableCmds(targetDb, + dataSourceType, sourceProperties, targetProperties); + Database db = Env.getCurrentEnv().getInternalCatalog().getDbNullable(targetDb); + Preconditions.checkNotNull(db, "target database %s does not exist", targetDb); + for (CreateTableCommand createTblCmd : createTblCmds) { + if (!db.isTableExist(createTblCmd.getCreateTableInfo().getTableName())) { + createTblCmd.run(ConnectContext.get(), null); + } + syncTbls.add(createTblCmd.getCreateTableInfo().getTableName()); + } + return syncTbls; } private void init() { @@ -151,7 +265,15 @@ private void init() { timerDefinition.setIntervalUnit(IntervalUnit.SECOND); timerDefinition.setStartTimeMs(execConfig.getTimerDefinition().getStartTimeMs()); execConfig.setTimerDefinition(timerDefinition); + } catch (AnalysisException ae) { + log.warn("parse streaming insert job failed, props: {}", properties, ae); + throw new RuntimeException(ae.getMessage()); + } + } + private void initInsertJob() { + try { + init(); UnboundTVFRelation currentTvf = getCurrentTvf(); this.tvfType = currentTvf.getFunctionName(); this.originTvfProps = currentTvf.getProperties().getMap(); @@ -161,9 +283,6 @@ private void init() { Offset offset = validateOffset(jobProperties.getOffsetProperty()); this.offsetProvider.updateOffset(offset); } - } catch (AnalysisException ae) { - log.warn("parse streaming insert job failed, props: {}", properties, ae); - throw new RuntimeException(ae.getMessage()); } catch (Exception ex) { log.warn("init streaming insert job failed, sql: {}", getExecuteSql(), ex); throw new RuntimeException(ex.getMessage()); @@ -243,6 +362,18 @@ public void alterJob(AlterJobCommand alterJobCommand) throws AnalysisException, modifyPropertiesInternal(alterJobCommand.getProperties()); logParts.add("properties: " + alterJobCommand.getProperties()); } + + // update source properties + if (!alterJobCommand.getSourceProperties().isEmpty()) { + this.sourceProperties.putAll(alterJobCommand.getSourceProperties()); + logParts.add("source properties: " + alterJobCommand.getSourceProperties()); + } + + // update target properties + if (!alterJobCommand.getTargetProperties().isEmpty()) { + this.sourceProperties.putAll(alterJobCommand.getTargetProperties()); + logParts.add("target properties: " + alterJobCommand.getTargetProperties()); + } log.info("Alter streaming job {}, {}", getJobId(), String.join(", ", logParts)); } @@ -315,12 +446,12 @@ public List createTasks(TaskType taskType, Map queryAllStreamTasks() { + public List queryAllStreamTasks() { if (CollectionUtils.isEmpty(streamInsertTaskQueue)) { return new ArrayList<>(); } - List tasks = new ArrayList<>(streamInsertTaskQueue); - Comparator taskComparator = - Comparator.comparingLong(StreamingInsertTask::getCreateTimeMs).reversed(); + List tasks = new ArrayList<>(streamInsertTaskQueue); + Comparator taskComparator = + Comparator.comparingLong(AbstractStreamingTask::getCreateTimeMs).reversed(); tasks.sort(taskComparator); return tasks; } protected void fetchMeta() { try { - if (originTvfProps == null) { - this.originTvfProps = getCurrentTvf().getProperties().getMap(); + if (tvfType != null) { + if (originTvfProps == null) { + this.originTvfProps = getCurrentTvf().getProperties().getMap(); + } + offsetProvider.fetchRemoteMeta(originTvfProps); + } else { + offsetProvider.fetchRemoteMeta(new HashMap<>()); } - offsetProvider.fetchRemoteMeta(originTvfProps); } catch (Exception ex) { log.warn("fetch remote meta failed, job id: {}", getJobId(), ex); failureReason = new FailureReason(InternalErrorCode.GET_REMOTE_DATA_ERROR, @@ -405,7 +558,7 @@ public void onTaskSuccess(StreamingJobSchedulerTask task) throws JobException { getRunningTasks().remove(task); } - public void onStreamTaskFail(StreamingInsertTask task) throws JobException { + public void onStreamTaskFail(AbstractStreamingTask task) throws JobException { try { failedTaskCount.incrementAndGet(); Env.getCurrentEnv().getJobManager().getStreamingTaskManager().removeRunningTask(task); @@ -416,11 +569,11 @@ public void onStreamTaskFail(StreamingInsertTask task) throws JobException { updateJobStatus(JobStatus.PAUSED); } - public void onStreamTaskSuccess(StreamingInsertTask task) { + public void onStreamTaskSuccess(AbstractStreamingTask task) { try { succeedTaskCount.incrementAndGet(); Env.getCurrentEnv().getJobManager().getStreamingTaskManager().removeRunningTask(task); - StreamingInsertTask nextTask = createStreamingInsertTask(); + AbstractStreamingTask nextTask = createStreamingTask(); this.runningStreamTask = nextTask; log.info("Streaming insert job {} create next streaming insert task {} after task {} success", getJobId(), nextTask.getTaskId(), task.getTaskId()); @@ -451,6 +604,25 @@ private void updateCloudJobStatisticAndOffset(StreamingTaskTxnCommitAttachment a offsetProvider.updateOffset(offsetProvider.deserializeOffset(attachment.getOffset())); } + private void updateJobStatisticAndOffset(CommitOffsetRequest offsetRequest) { + if (this.jobStatistic == null) { + this.jobStatistic = new StreamingJobStatistic(); + } + this.jobStatistic.setScannedRows(this.jobStatistic.getScannedRows() + offsetRequest.getScannedRows()); + this.jobStatistic.setLoadBytes(this.jobStatistic.getLoadBytes() + offsetRequest.getScannedBytes()); + offsetProvider.updateOffset(offsetProvider.deserializeOffset(offsetRequest.getOffset())); + } + + private void updateNoTxnJobStatisticAndOffset(CommitOffsetRequest offsetRequest) { + if (this.nonTxnJobStatistic == null) { + this.nonTxnJobStatistic = new StreamingJobStatistic(); + } + this.nonTxnJobStatistic + .setScannedRows(this.nonTxnJobStatistic.getScannedRows() + offsetRequest.getScannedRows()); + this.nonTxnJobStatistic.setLoadBytes(this.nonTxnJobStatistic.getLoadBytes() + offsetRequest.getScannedBytes()); + offsetProvider.updateOffset(offsetProvider.deserializeOffset(offsetRequest.getOffset())); + } + @Override public void onRegister() throws JobException { Env.getCurrentGlobalTransactionMgr().getCallbackFactory().addCallback(this); @@ -462,13 +634,19 @@ public void onReplayCreate() throws JobException { super.onReplayCreate(); } + /** * Because the offset statistics of the streamingInsertJob are all stored in txn, * only some fields are replayed here. * @param replayJob */ public void replayOnUpdated(StreamingInsertJob replayJob) { - setJobStatus(replayJob.getJobStatus()); + if (!JobStatus.RUNNING.equals(replayJob.getJobStatus())) { + // No need to restore in the running state, as scheduling relies on pending states. + // insert TVF does not persist the running state. + // streaming multi task persists the running state when commitOffset() is called. + setJobStatus(replayJob.getJobStatus()); + } try { modifyPropertiesInternal(replayJob.getProperties()); // When the pause state is restarted, it also needs to be updated @@ -479,6 +657,12 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { // should not happen log.error("replay modify streaming insert job properties failed, job id: {}", getJobId(), e); } + if (replayJob.getOffsetProviderPersist() != null) { + setOffsetProviderPersist(replayJob.getOffsetProviderPersist()); + } + if (replayJob.getNonTxnJobStatistic() != null) { + setNonTxnJobStatistic(replayJob.getNonTxnJobStatistic()); + } setExecuteSql(replayJob.getExecuteSql()); setSucceedTaskCount(replayJob.getSucceedTaskCount()); setFailedTaskCount(replayJob.getFailedTaskCount()); @@ -490,7 +674,7 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { */ private void modifyPropertiesInternal(Map inputProperties) throws AnalysisException, JobException { StreamingJobProperties inputStreamProps = new StreamingJobProperties(inputProperties); - if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty())) { + if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty()) && this.tvfType != null) { Offset offset = validateOffset(inputStreamProps.getOffsetProperty()); this.offsetProvider.updateOffset(offset); if (Config.isCloudMode()) { @@ -545,9 +729,7 @@ public TRow getTvfInfo() { trow.addToColumnValue(new TCell().setStringVal(getJobConfig().getExecuteType().name())); trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); trow.addToColumnValue(new TCell().setStringVal(getJobStatus().name())); - - trow.addToColumnValue(new TCell().setStringVal(StringUtils.isNotEmpty(getEncryptedSql()) - ? getEncryptedSql() : generateEncryptedSql())); + trow.addToColumnValue(new TCell().setStringVal(getShowSQL())); trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(getCreateTimeMs()))); trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getSucceedTaskCount().get()))); trow.addToColumnValue(new TCell().setStringVal(String.valueOf(getFailedTaskCount().get()))); @@ -567,9 +749,13 @@ public TRow getTvfInfo() { } else { trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); } - - trow.addToColumnValue(new TCell().setStringVal( - jobStatistic == null ? FeConstants.null_string : jobStatistic.toJson())); + if (tvfType != null) { + trow.addToColumnValue(new TCell().setStringVal( + jobStatistic == null ? FeConstants.null_string : jobStatistic.toJson())); + } else { + trow.addToColumnValue(new TCell().setStringVal( + nonTxnJobStatistic == null ? FeConstants.null_string : nonTxnJobStatistic.toJson())); + } trow.addToColumnValue(new TCell().setStringVal(failureReason == null ? FeConstants.null_string : failureReason.getMsg())); trow.addToColumnValue(new TCell().setStringVal(jobRuntimeMsg == null @@ -577,7 +763,35 @@ public TRow getTvfInfo() { return trow; } - private static boolean checkPrivilege(ConnectContext ctx, LogicalPlan logicalPlan) throws AnalysisException { + private String getShowSQL() { + if (StringUtils.isNotEmpty(getExecuteSql())) { + return StringUtils.isNotEmpty(getEncryptedSql()) + ? getEncryptedSql() : generateEncryptedSql(); + } else { + StringBuilder sb = new StringBuilder(); + sb.append("FROM ").append(dataSourceType.name()); + sb.append("("); + for (Map.Entry entry : sourceProperties.entrySet()) { + sb.append("'").append(entry.getKey()) + .append("'='").append(entry.getValue()).append("',"); + } + sb.deleteCharAt(sb.length() - 1); + sb.append(" ) TO DATABSE ").append(targetDb); + if (!targetProperties.isEmpty()) { + sb.append(" ("); + for (Map.Entry entry : targetProperties.entrySet()) { + sb.append("'").append(entry.getKey()) + .append("'='").append(entry.getValue()).append("',"); + } + sb.deleteCharAt(sb.length() - 1); + sb.append(")"); + } + return sb.toString(); + } + } + + private static boolean checkPrivilege(ConnectContext ctx, String sql) throws AnalysisException { + LogicalPlan logicalPlan = new NereidsParser().parseSingle(sql); if (!(logicalPlan instanceof InsertIntoTableCommand)) { throw new AnalysisException("Only support insert command"); } @@ -598,33 +812,67 @@ private static boolean checkPrivilege(ConnectContext ctx, LogicalPlan logicalPla return true; } - public boolean checkPrivilege(ConnectContext ctx) throws AnalysisException { - LogicalPlan logicalPlan = new NereidsParser().parseSingle(getExecuteSql()); - return checkPrivilege(ctx, logicalPlan); + public static boolean checkPrivilege(ConnectContext ctx, String sql, String targetDb) throws AnalysisException { + if (StringUtils.isNotEmpty(sql)) { + return checkPrivilege(ctx, sql); + } else if (StringUtils.isNotEmpty(targetDb)) { + return checkHasSourceJobPriv(ctx, targetDb); + } else { + log.info("insert sql and target db are both empty"); + return false; + } } - public static boolean checkPrivilege(ConnectContext ctx, String sql) throws AnalysisException { - LogicalPlan logicalPlan = new NereidsParser().parseSingle(sql); - return checkPrivilege(ctx, logicalPlan); + public boolean checkPrivilege(ConnectContext ctx) throws AnalysisException { + if (StringUtils.isNotEmpty(getExecuteSql())) { + return checkPrivilege(ctx, getExecuteSql()); + } else if (StringUtils.isNotEmpty(getTargetDb())) { + return checkHasSourceJobPriv(ctx, targetDb); + } else { + log.info("insert sql and target db are both empty"); + return false; + } } public boolean hasPrivilege(UserIdentity userIdentity) { ConnectContext ctx = InsertTask.makeConnectContext(userIdentity, getCurrentDbName()); try { - LogicalPlan logicalPlan = new NereidsParser().parseSingle(getExecuteSql()); - LogicalPlan logicalQuery = ((InsertIntoTableCommand) logicalPlan).getLogicalQuery(); - List targetTable = InsertUtils.getTargetTableQualified(logicalQuery, ctx); - Preconditions.checkArgument(targetTable.size() == 3, "target table name is invalid"); - return Env.getCurrentEnv().getAccessManager().checkTblPriv(userIdentity, - InternalCatalog.INTERNAL_CATALOG_NAME, - targetTable.get(1), - targetTable.get(2), - PrivPredicate.LOAD); + if (StringUtils.isNotEmpty(getExecuteSql())) { + LogicalPlan logicalPlan = new NereidsParser().parseSingle(getExecuteSql()); + LogicalPlan logicalQuery = ((InsertIntoTableCommand) logicalPlan).getLogicalQuery(); + List targetTable = InsertUtils.getTargetTableQualified(logicalQuery, ctx); + Preconditions.checkArgument(targetTable.size() == 3, "target table name is invalid"); + return Env.getCurrentEnv().getAccessManager().checkTblPriv(userIdentity, + InternalCatalog.INTERNAL_CATALOG_NAME, + targetTable.get(1), + targetTable.get(2), + PrivPredicate.LOAD); + } else if (StringUtils.isNotEmpty(getTargetDb())) { + return Env.getCurrentEnv().getAccessManager().checkDbPriv(ctx, + InternalCatalog.INTERNAL_CATALOG_NAME, + targetDb, + PrivPredicate.LOAD); + } else { + log.info("insert sql and target db are both empty"); + return false; + } } finally { ctx.cleanup(); } } + private static boolean checkHasSourceJobPriv(ConnectContext ctx, String targetDb) throws AnalysisException { + if (!Env.getCurrentEnv().getAccessManager().checkDbPriv(ctx, + InternalCatalog.INTERNAL_CATALOG_NAME, + targetDb, + PrivPredicate.LOAD)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED_ERROR, "LOAD", + PrivPredicate.LOAD.getPrivs().toString(), + targetDb); + } + return true; + } + private String generateEncryptedSql() { makeConnectContext(); TreeMap, String> indexInSqlToString = new TreeMap<>(new Pair.PairComparator<>()); @@ -766,12 +1014,10 @@ public void afterAborted(TransactionState txnState, boolean txnOperated, String @Override public void replayOnAborted(TransactionState txnState) { - } @Override public void afterVisible(TransactionState txnState, boolean txnOperated) { - } @Override @@ -780,8 +1026,12 @@ public void replayOnVisible(TransactionState txnState) { @Override public void gsonPostProcess() throws IOException { - if (offsetProvider == null && tvfType != null) { - offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(tvfType); + if (offsetProvider == null) { + if (tvfType != null) { + offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(tvfType); + } else { + offsetProvider = new JdbcSourceOffsetProvider(getJobId(), dataSourceType, sourceProperties); + } } if (jobProperties == null && properties != null) { @@ -806,4 +1056,104 @@ public void gsonPostProcess() throws IOException { this.lock = new ReentrantReadWriteLock(true); } } + + /** + * The current streamingTask times out; create a new streamingTask. + * Only applies to StreamingMultiTask. + */ + public void processTimeoutTasks() { + if (!(runningStreamTask instanceof StreamingMultiTblTask)) { + return; + } + writeLock(); + try { + StreamingMultiTblTask runningMultiTask = (StreamingMultiTblTask) this.runningStreamTask; + if (TaskStatus.RUNNING.equals(runningMultiTask.getStatus()) + && runningMultiTask.isTimeout()) { + runningMultiTask.cancel(false); + runningMultiTask.setErrMsg("task cancelled cause timeout"); + + // renew streaming multi task + this.runningStreamTask = createStreamingMultiTblTask(); + Env.getCurrentEnv().getJobManager().getStreamingTaskManager().registerTask(runningStreamTask); + this.runningStreamTask.setStatus(TaskStatus.PENDING); + log.info("create new streaming multi tasks due to timeout, for job {}, task {} ", + getJobId(), runningStreamTask.getTaskId()); + recordTasks(runningStreamTask); + } + } finally { + writeUnlock(); + } + } + + public void commitOffset(CommitOffsetRequest offsetRequest) throws JobException { + if (!(offsetProvider instanceof JdbcSourceOffsetProvider)) { + throw new JobException("Unsupported commit offset for offset provider type: " + + offsetProvider.getClass().getSimpleName()); + } + writeLock(); + try { + if (offsetRequest.getScannedRows() == 0 && offsetRequest.getScannedBytes() == 0) { + JdbcSourceOffsetProvider op = (JdbcSourceOffsetProvider) offsetProvider; + op.setHasMoreData(false); + } + updateNoTxnJobStatisticAndOffset(offsetRequest); + if (this.runningStreamTask != null + && this.runningStreamTask instanceof StreamingMultiTblTask) { + if (this.runningStreamTask.getTaskId() != offsetRequest.getTaskId()) { + throw new JobException("Task id mismatch when commit offset. expected: " + + this.runningStreamTask.getTaskId() + ", actual: " + offsetRequest.getTaskId()); + } + persistOffsetProviderIfNeed(); + ((StreamingMultiTblTask) this.runningStreamTask).successCallback(offsetRequest); + } + } finally { + writeUnlock(); + } + } + + private void persistOffsetProviderIfNeed() { + // only for jdbc + this.offsetProviderPersist = offsetProvider.getPersistInfo(); + if (this.offsetProviderPersist != null) { + logUpdateOperation(); + } + } + + public void replayOffsetProviderIfNeed() throws JobException { + if (this.offsetProviderPersist != null && offsetProvider != null) { + offsetProvider.replayIfNeed(this); + } + } + + /** + * 1. Clean offset info in ms (s3 tvf) + * 2. Clean chunk info in meta table (jdbc) + */ + public void cleanup() throws JobException { + // s3 tvf clean offset + if (tvfType != null && Config.isCloudMode()) { + Cloud.DeleteStreamingJobResponse resp = null; + try { + Cloud.DeleteStreamingJobRequest req = Cloud.DeleteStreamingJobRequest.newBuilder() + .setCloudUniqueId(Config.cloud_unique_id) + .setDbId(getDbId()) + .setJobId(getJobId()) + .build(); + resp = MetaServiceProxy.getInstance().deleteStreamingJob(req); + if (resp.getStatus().getCode() != Cloud.MetaServiceCode.OK) { + log.warn("failed to delete streaming job, response: {}", resp); + throw new JobException("deleteJobKey failed for jobId=%s, dbId=%s, status=%s", + getJobId(), getJobId(), resp.getStatus()); + } + } catch (RpcException e) { + log.warn("failed to delete streaming job {}", resp, e); + } + } + + if (this.offsetProvider instanceof JdbcSourceOffsetProvider) { + // jdbc clean chunk meta table + ((JdbcSourceOffsetProvider) this.offsetProvider).cleanMeta(getJobId()); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java index f25ebe794a0788..635c639256c276 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java @@ -21,13 +21,11 @@ import org.apache.doris.catalog.Env; import org.apache.doris.common.FeConstants; import org.apache.doris.common.Status; -import org.apache.doris.common.util.TimeUtils; import org.apache.doris.common.util.Util; import org.apache.doris.job.base.Job; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.InsertTask; -import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.nereids.StatementContext; @@ -42,43 +40,24 @@ import org.apache.doris.thrift.TStatusCode; import lombok.Getter; -import lombok.Setter; import lombok.extern.log4j.Log4j2; -import org.apache.commons.lang3.StringUtils; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.atomic.AtomicBoolean; @Log4j2 @Getter -public class StreamingInsertTask { - private static final String LABEL_SPLITTER = "_"; - private static final int MAX_RETRY = 3; - private long jobId; - private long taskId; - private String labelName; - @Setter - private volatile TaskStatus status; - private String errMsg; - private Long createTimeMs; - private Long startTimeMs; - private Long finishTimeMs; +public class StreamingInsertTask extends AbstractStreamingTask { private String sql; private StmtExecutor stmtExecutor; private InsertIntoTableCommand taskCommand; private String currentDb; - private UserIdentity userIdentity; private ConnectContext ctx; - private Offset runningOffset; - @Getter - private AtomicBoolean isCanceled = new AtomicBoolean(false); private StreamingJobProperties jobProperties; private Map originTvfProps; SourceOffsetProvider offsetProvider; - private int retryCount = 0; public StreamingInsertTask(long jobId, long taskId, @@ -88,51 +67,17 @@ public StreamingInsertTask(long jobId, StreamingJobProperties jobProperties, Map originTvfProps, UserIdentity userIdentity) { - this.jobId = jobId; - this.taskId = taskId; + super(jobId, taskId, userIdentity); this.sql = sql; - this.userIdentity = userIdentity; this.currentDb = currentDb; this.offsetProvider = offsetProvider; this.jobProperties = jobProperties; this.originTvfProps = originTvfProps; - this.labelName = getJobId() + LABEL_SPLITTER + getTaskId(); - this.createTimeMs = System.currentTimeMillis(); } - public void execute() throws JobException { - while (retryCount <= MAX_RETRY) { - try { - before(); - run(); - onSuccess(); - return; - } catch (Exception e) { - if (TaskStatus.CANCELED.equals(status)) { - return; - } - this.errMsg = e.getMessage(); - retryCount++; - if (retryCount > MAX_RETRY) { - log.error("Task execution failed after {} retries.", MAX_RETRY, e); - onFail(e.getMessage()); - return; - } - log.warn("execute streaming task error, job id is {}, task id is {}, retrying {}/{}: {}", - jobId, taskId, retryCount, MAX_RETRY, e.getMessage()); - } finally { - // The cancel logic will call the closeOrReleased Resources method by itself. - // If it is also called here, - // it may result in the inability to obtain relevant information when canceling the task - if (!TaskStatus.CANCELED.equals(status)) { - closeOrReleaseResources(); - } - } - } - } - - private void before() throws Exception { - if (isCanceled.get()) { + @Override + public void before() throws Exception { + if (getIsCanceled().get()) { log.info("streaming insert task has been canceled, task id is {}", getTaskId()); return; } @@ -158,8 +103,9 @@ private void before() throws Exception { this.stmtExecutor = new StmtExecutor(ctx, new LogicalPlanAdapter(taskCommand, ctx.getStatementContext())); } - private void run() throws JobException { - if (isCanceled.get()) { + @Override + public void run() throws JobException { + if (getIsCanceled().get()) { log.info("task has been canceled, task id is {}", getTaskId()); return; } @@ -180,8 +126,9 @@ private void run() throws JobException { } } + @Override public boolean onSuccess() throws JobException { - if (isCanceled.get()) { + if (getIsCanceled().get()) { return false; } this.status = TaskStatus.SUCCESS; @@ -200,32 +147,14 @@ public boolean onSuccess() throws JobException { return true; } - public void onFail(String errMsg) throws JobException { - if (isCanceled.get()) { - return; - } - this.errMsg = errMsg; - this.status = TaskStatus.FAILED; - this.finishTimeMs = System.currentTimeMillis(); - if (!isCallable()) { - return; - } - Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); - StreamingInsertJob streamingInsertJob = (StreamingInsertJob) job; - streamingInsertJob.onStreamTaskFail(this); + @Override + protected void onFail(String errMsg) throws JobException { + super.onFail(errMsg); } + @Override public void cancel(boolean needWaitCancelComplete) { - if (TaskStatus.SUCCESS.equals(status) || TaskStatus.FAILED.equals(status) - || TaskStatus.CANCELED.equals(status)) { - return; - } - status = TaskStatus.CANCELED; - if (isCanceled.get()) { - return; - } - isCanceled.getAndSet(true); - this.errMsg = "task cancelled"; + super.cancel(needWaitCancelComplete); if (null != stmtExecutor) { log.info("cancelling streaming insert task, job id is {}, task id is {}", getJobId(), getTaskId()); @@ -234,6 +163,7 @@ public void cancel(boolean needWaitCancelComplete) { } } + @Override public void closeOrReleaseResources() { if (null != stmtExecutor) { stmtExecutor = null; @@ -246,37 +176,9 @@ public void closeOrReleaseResources() { } } - private boolean isCallable() { - if (status.equals(TaskStatus.CANCELED)) { - return false; - } - if (null != Env.getCurrentEnv().getJobManager().getJob(jobId)) { - return true; - } - return false; - } - - /** - * show streaming insert task info detail - */ + @Override public TRow getTvfInfo(String jobName) { - TRow trow = new TRow(); - trow.addToColumnValue(new TCell().setStringVal(String.valueOf(this.getTaskId()))); - trow.addToColumnValue(new TCell().setStringVal(String.valueOf(this.getJobId()))); - trow.addToColumnValue(new TCell().setStringVal(jobName)); - trow.addToColumnValue(new TCell().setStringVal(this.getLabelName())); - trow.addToColumnValue(new TCell().setStringVal(this.getStatus().name())); - // err msg - trow.addToColumnValue(new TCell().setStringVal(StringUtils.isNotBlank(errMsg) - ? errMsg : FeConstants.null_string)); - - // create time - trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(this.getCreateTimeMs()))); - trow.addToColumnValue(new TCell().setStringVal(null == getStartTimeMs() ? FeConstants.null_string - : TimeUtils.longToTimeString(this.getStartTimeMs()))); - // load end time - trow.addToColumnValue(new TCell().setStringVal(TimeUtils.longToTimeString(this.getFinishTimeMs()))); - + TRow trow = super.getTvfInfo(jobName); List loadJobs = Env.getCurrentEnv().getLoadManager() .queryLoadJobsByJobIds(Arrays.asList(this.getTaskId())); if (!loadJobs.isEmpty()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java index c3655e6697eb9f..d0034de4b4a266 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -44,7 +44,7 @@ public void run() throws JobException { handlePendingState(); break; case RUNNING: - streamingInsertJob.fetchMeta(); + handleRunningState(); break; case PAUSED: autoResumeHandler(); @@ -65,11 +65,17 @@ private void handlePendingState() throws JobException { return; } } - streamingInsertJob.createStreamingInsertTask(); + streamingInsertJob.replayOffsetProviderIfNeed(); + streamingInsertJob.createStreamingTask(); streamingInsertJob.updateJobStatus(JobStatus.RUNNING); streamingInsertJob.setAutoResumeCount(0); } + private void handleRunningState() { + streamingInsertJob.processTimeoutTasks(); + streamingInsertJob.fetchMeta(); + } + private void autoResumeHandler() throws JobException { final FailureReason failureReason = streamingInsertJob.getFailureReason(); final long latestAutoResumeTimestamp = streamingInsertJob.getLatestAutoResumeTimestamp(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java index 8b209cd6e81bf4..71bf9e6f065307 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobStatistic.java @@ -19,15 +19,18 @@ import com.google.gson.Gson; +import com.google.gson.annotations.SerializedName; import lombok.Getter; import lombok.Setter; public class StreamingJobStatistic { @Getter @Setter + @SerializedName("scannedRows") private long scannedRows; @Getter @Setter + @SerializedName("loadBytes") private long loadBytes; @Getter @Setter diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java new file mode 100644 index 00000000000000..7c610b406886e9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingMultiTblTask.java @@ -0,0 +1,282 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.extensions.insert.streaming; + +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; +import org.apache.doris.common.FeConstants; +import org.apache.doris.httpv2.entity.ResponseBody; +import org.apache.doris.httpv2.rest.RestApiStatusCode; +import org.apache.doris.httpv2.rest.StreamingJobAction.CommitOffsetRequest; +import org.apache.doris.job.base.Job; +import org.apache.doris.job.cdc.request.WriteRecordRequest; +import org.apache.doris.job.cdc.split.BinlogSplit; +import org.apache.doris.job.cdc.split.SnapshotSplit; +import org.apache.doris.job.common.DataSourceType; +import org.apache.doris.job.common.TaskStatus; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.job.offset.jdbc.JdbcOffset; +import org.apache.doris.job.offset.jdbc.JdbcSourceOffsetProvider; +import org.apache.doris.job.util.StreamingJobUtils; +import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.InternalService.PRequestCdcClientResult; +import org.apache.doris.rpc.BackendServiceProxy; +import org.apache.doris.system.Backend; +import org.apache.doris.thrift.TCell; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TRow; +import org.apache.doris.thrift.TStatusCode; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import lombok.Getter; +import lombok.extern.log4j.Log4j2; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +@Log4j2 +@Getter +public class StreamingMultiTblTask extends AbstractStreamingTask { + private static final ObjectMapper objectMapper = new ObjectMapper(); + private DataSourceType dataSourceType; + private SourceOffsetProvider offsetProvider; + private Map sourceProperties; + private Map targetProperties; + private String targetDb; + private StreamingJobProperties jobProperties; + private long scannedRows = 0L; + private long scannedBytes = 0L; + private long timeoutMs; + + public StreamingMultiTblTask(Long jobId, + long taskId, + DataSourceType dataSourceType, + SourceOffsetProvider offsetProvider, + Map sourceProperties, + String targetDb, + Map targetProperties, + StreamingJobProperties jobProperties, + UserIdentity userIdentity) { + super(jobId, taskId, userIdentity); + this.dataSourceType = dataSourceType; + this.offsetProvider = offsetProvider; + this.sourceProperties = sourceProperties; + this.targetProperties = targetProperties; + this.jobProperties = jobProperties; + this.targetDb = targetDb; + this.timeoutMs = Config.streaming_task_timeout_multiplier * jobProperties.getMaxIntervalSecond() * 1000L; + } + + @Override + public void before() throws Exception { + if (getIsCanceled().get()) { + log.info("streaming multi task has been canceled, task id is {}", getTaskId()); + return; + } + this.status = TaskStatus.RUNNING; + this.startTimeMs = System.currentTimeMillis(); + this.runningOffset = offsetProvider.getNextOffset(null, sourceProperties); + log.info("streaming multi task {} get running offset: {}", taskId, runningOffset.toString()); + } + + @Override + public void run() throws JobException { + if (getIsCanceled().get()) { + log.info("task has been canceled, task id is {}", getTaskId()); + return; + } + log.info("start to run streaming multi task, offset is {}", runningOffset.toString()); + sendWriteRequest(); + } + + private void sendWriteRequest() throws JobException { + Backend backend = StreamingJobUtils.selectBackend(jobId); + WriteRecordRequest params = buildRequestParams(); + InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() + .setApi("/api/writeRecords") + .setParams(new Gson().toJson(params)).build(); + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); + InternalService.PRequestCdcClientResult result = null; + try { + Future future = + BackendServiceProxy.getInstance().requestCdcClient(address, request); + result = future.get(); + TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + log.error("Failed to get split from backend, {}", result.getStatus().getErrorMsgs(0)); + throw new JobException( + "Failed to get split from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + + result.getResponse()); + } + String response = result.getResponse(); + try { + ResponseBody responseObj = objectMapper.readValue( + response, + new TypeReference>() { + } + ); + if (responseObj.getCode() == RestApiStatusCode.OK.code) { + log.info("Send write records request successfully, response: {}", responseObj.getData()); + return; + } + } catch (JsonProcessingException e) { + log.error("Failed to parse write records response: {}", response, e); + throw new JobException("Failed to parse write records response: " + response); + } + throw new JobException("Failed to send write records request , error message: " + response); + } catch (ExecutionException | InterruptedException ex) { + log.error("Send write request failed: ", ex); + throw new JobException(ex); + } + } + + private String getToken() throws JobException { + String token = ""; + try { + // Acquire token from master + token = Env.getCurrentEnv().getTokenManager().acquireToken(); + } catch (Exception e) { + log.warn("Failed to get auth token:", e); + throw new JobException(e.getMessage()); + } + return token; + } + + private WriteRecordRequest buildRequestParams() throws JobException { + JdbcOffset offset = (JdbcOffset) runningOffset; + WriteRecordRequest request = new WriteRecordRequest(); + request.setJobId(getJobId()); + request.setConfig(sourceProperties); + request.setDataSource(dataSourceType.name()); + + request.setTaskId(getTaskId() + ""); + request.setToken(getToken()); + request.setTargetDb(targetDb); + Map splitMeta = objectMapper.convertValue(offset.getSplit(), + new TypeReference>() { + }); + request.setMeta(splitMeta); + String feAddr = Env.getCurrentEnv().getMasterHost() + ":" + Env.getCurrentEnv().getMasterHttpPort(); + request.setFrontendAddress(feAddr); + request.setMaxInterval(jobProperties.getMaxIntervalSecond()); + return request; + } + + @Override + public boolean onSuccess() throws JobException { + if (getIsCanceled().get()) { + return false; + } + log.info("streaming multi task {} send write request run successfully.", getTaskId()); + return false; + } + + /** + * Callback function for offset commit success. + */ + public void successCallback(CommitOffsetRequest offsetRequest) { + if (getIsCanceled().get()) { + return; + } + this.status = TaskStatus.SUCCESS; + this.finishTimeMs = System.currentTimeMillis(); + JdbcOffset runOffset = (JdbcOffset) this.runningOffset; + if (!isCallable()) { + return; + } + // set end offset to running offset + Map offsetMeta; + try { + offsetMeta = objectMapper.readValue(offsetRequest.getOffset(), new TypeReference>() { + }); + } catch (JsonProcessingException e) { + log.warn("Failed to parse offset meta from request: {}", offsetRequest.getOffset(), e); + throw new RuntimeException(e); + } + String splitId = offsetMeta.remove(JdbcSourceOffsetProvider.SPLIT_ID); + if (runOffset.getSplit().snapshotSplit() + && !BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)) { + SnapshotSplit split = (SnapshotSplit) runOffset.getSplit(); + split.setHighWatermark(offsetMeta); + } else if (!runOffset.getSplit().snapshotSplit() + && BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)) { + BinlogSplit split = (BinlogSplit) runOffset.getSplit(); + split.setEndingOffset(offsetMeta); + } else { + log.warn("Split id is not consistent, task running split id {}," + + " offset commit request split id {}", runOffset.getSplit().getSplitId(), splitId); + throw new RuntimeException("Split id is not consistent"); + } + this.scannedRows = offsetRequest.getScannedRows(); + this.scannedBytes = offsetRequest.getScannedBytes(); + Job job = Env.getCurrentEnv().getJobManager().getJob(getJobId()); + if (null == job) { + log.info("job is null, job id is {}", jobId); + return; + } + StreamingInsertJob streamingInsertJob = (StreamingInsertJob) job; + streamingInsertJob.onStreamTaskSuccess(this); + } + + @Override + protected void onFail(String errMsg) throws JobException { + super.onFail(errMsg); + } + + @Override + public void cancel(boolean needWaitCancelComplete) { + // No manual cancellation is required; the task ID will be checked for consistency in the beforeCommit function. + super.cancel(needWaitCancelComplete); + } + + @Override + public void closeOrReleaseResources() { + // no need + } + + public boolean isTimeout() { + return (System.currentTimeMillis() - createTimeMs) > timeoutMs; + } + + @Override + public TRow getTvfInfo(String jobName) { + TRow trow = super.getTvfInfo(jobName); + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + Map statistic = new HashMap<>(); + statistic.put("scannedRows", scannedRows); + statistic.put("loadBytes", scannedBytes); + trow.addToColumnValue(new TCell().setStringVal(new Gson().toJson(statistic))); + + if (this.getUserIdentity() == null) { + trow.addToColumnValue(new TCell().setStringVal(FeConstants.null_string)); + } else { + trow.addToColumnValue(new TCell().setStringVal(this.getUserIdentity().getQualifiedUser())); + } + trow.addToColumnValue(new TCell().setStringVal("")); + trow.addToColumnValue(new TCell().setStringVal(runningOffset == null + ? FeConstants.null_string : runningOffset.showRange())); + return trow; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java index 4274809febd0aa..45e940ebc3182b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/JobManager.java @@ -20,11 +20,8 @@ import org.apache.doris.analysis.CompoundPredicate; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; -import org.apache.doris.cloud.proto.Cloud; -import org.apache.doris.cloud.rpc.MetaServiceProxy; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.CaseSensibility; -import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; @@ -52,7 +49,6 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.plans.commands.AlterJobCommand; import org.apache.doris.qe.ConnectContext; -import org.apache.doris.rpc.RpcException; import com.google.common.collect.Lists; import lombok.Getter; @@ -228,7 +224,7 @@ public void dropJobInternal(T job, boolean isReplay) throws JobException { } writeLock(); try { - deleteStremingJob(job); + deleteStreamingJob(job); jobMap.remove(job.getJobId()); if (isReplay) { job.onReplayEnd(job); @@ -242,27 +238,12 @@ public void dropJobInternal(T job, boolean isReplay) throws JobException { } } - private void deleteStremingJob(AbstractJob job) throws JobException { - if (!(Config.isCloudMode() && job instanceof StreamingInsertJob)) { + private void deleteStreamingJob(AbstractJob job) throws JobException { + if (!(job instanceof StreamingInsertJob)) { return; } StreamingInsertJob streamingJob = (StreamingInsertJob) job; - Cloud.DeleteStreamingJobResponse resp = null; - try { - Cloud.DeleteStreamingJobRequest req = Cloud.DeleteStreamingJobRequest.newBuilder() - .setCloudUniqueId(Config.cloud_unique_id) - .setDbId(streamingJob.getDbId()) - .setJobId(job.getJobId()) - .build(); - resp = MetaServiceProxy.getInstance().deleteStreamingJob(req); - if (resp.getStatus().getCode() != Cloud.MetaServiceCode.OK) { - log.warn("failed to delete streaming job, response: {}", resp); - throw new JobException("deleteJobKey failed for jobId=%s, dbId=%s, status=%s", - job.getJobId(), job.getJobId(), resp.getStatus()); - } - } catch (RpcException e) { - log.warn("failed to delete streaming job {}", resp, e); - } + streamingJob.cleanup(); } public void alterJobStatus(Long jobId, JobStatus status) throws JobException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java b/fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java index 7c6fecee14a179..afe4e62baaadc8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/manager/StreamingTaskManager.java @@ -17,6 +17,7 @@ package org.apache.doris.job.manager; +import org.apache.doris.job.extensions.insert.streaming.AbstractStreamingTask; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; import lombok.Getter; @@ -28,28 +29,29 @@ public class StreamingTaskManager { @Getter - private final LinkedBlockingDeque needScheduleTasksQueue = new LinkedBlockingDeque<>(); + private final LinkedBlockingDeque needScheduleTasksQueue = new LinkedBlockingDeque<>(); @Getter - private List runningTasks = Collections.synchronizedList(new ArrayList<>()); + private List runningTasks = Collections.synchronizedList(new ArrayList<>()); - public void registerTask(StreamingInsertTask task) { + public void registerTask(AbstractStreamingTask task) { needScheduleTasksQueue.add(task); } public StreamingInsertTask getStreamingInsertTaskById(long taskId) { synchronized (runningTasks) { - return runningTasks.stream() + return (StreamingInsertTask) runningTasks.stream() .filter(task -> task.getTaskId() == taskId) + .filter(task -> task instanceof StreamingInsertTask) .findFirst() .orElse(null); } } - public void addRunningTask(StreamingInsertTask task) { + public void addRunningTask(AbstractStreamingTask task) { runningTasks.add(task); } - public void removeRunningTask(StreamingInsertTask task) { + public void removeRunningTask(AbstractStreamingTask task) { runningTasks.remove(task); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index c1460017d4c288..892231444e37a9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -17,6 +17,8 @@ package org.apache.doris.job.offset; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; @@ -29,30 +31,35 @@ public interface SourceOffsetProvider { /** * Get source type, e.g. s3, kafka + * * @return */ String getSourceType(); /** * Get next offset to consume + * * @return */ Offset getNextOffset(StreamingJobProperties jobProps, Map properties); /** * Get current offset to show + * * @return */ String getShowCurrentOffset(); /** * Get remote datasource max offset to show + * * @return */ String getShowMaxOffset(); /** * Rewrite the TVF parameters in the SQL based on the current offset. + * * @param nextOffset * @return rewritten InsertIntoTableCommand */ @@ -60,6 +67,7 @@ public interface SourceOffsetProvider { /** * Update the offset of the source. + * * @param offset */ void updateOffset(Offset offset); @@ -71,21 +79,36 @@ public interface SourceOffsetProvider { /** * Whether there is more data to consume + * * @return */ boolean hasMoreDataToConsume(); /** * Deserialize string offset to Offset + * * @return */ Offset deserializeOffset(String offset); /** * Deserialize offset property to Offset + * * @return */ Offset deserializeOffsetProperty(String offset); + /** + * Replaying OffsetProvider is currently only required by JDBC. + * + * @return + */ + default void replayIfNeed(StreamingInsertJob job) throws JobException { + } + + default String getPersistInfo() { + return null; + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java new file mode 100644 index 00000000000000..83ad4314d8b85b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.offset.jdbc; + +import org.apache.doris.job.cdc.split.AbstractSourceSplit; +import org.apache.doris.job.cdc.split.BinlogSplit; +import org.apache.doris.job.offset.Offset; + +import com.google.gson.Gson; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.Setter; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.collections4.MapUtils; + +import java.util.HashMap; + +@Getter +@Setter +@AllArgsConstructor +@NoArgsConstructor +public class JdbcOffset implements Offset { + + private AbstractSourceSplit split; + + @Override + public String toSerializedJson() { + return null; + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public boolean isValidOffset() { + return false; + } + + @Override + public String showRange() { + if (split.snapshotSplit()) { + // need to show hw + return new Gson().toJson(split); + } else { + BinlogSplit binlogSplit = (BinlogSplit) split; + HashMap showMap = new HashMap<>(); + showMap.put(JdbcSourceOffsetProvider.SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); + if (binlogSplit.getStartingOffset() != null) { + showMap.put("startOffset", binlogSplit.getStartingOffset()); + } else if (CollectionUtils.isNotEmpty(binlogSplit.getFinishedSplits())) { + showMap.put("finishedSplitSize", binlogSplit.getFinishedSplits().size()); + } + if (MapUtils.isNotEmpty(binlogSplit.getEndingOffset())) { + showMap.put("endOffset", binlogSplit.getEndingOffset()); + } + return new Gson().toJson(showMap); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java new file mode 100644 index 00000000000000..d7195abac0cee4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -0,0 +1,494 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.offset.jdbc; + +import org.apache.doris.httpv2.entity.ResponseBody; +import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.cdc.request.CompareOffsetRequest; +import org.apache.doris.job.cdc.request.FetchTableSplitsRequest; +import org.apache.doris.job.cdc.request.JobBaseConfig; +import org.apache.doris.job.cdc.split.AbstractSourceSplit; +import org.apache.doris.job.cdc.split.BinlogSplit; +import org.apache.doris.job.cdc.split.SnapshotSplit; +import org.apache.doris.job.common.DataSourceType; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; +import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; +import org.apache.doris.job.offset.Offset; +import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.job.util.StreamingJobUtils; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.proto.InternalService; +import org.apache.doris.proto.InternalService.PRequestCdcClientResult; +import org.apache.doris.rpc.BackendServiceProxy; +import org.apache.doris.system.Backend; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TStatusCode; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; +import com.google.gson.annotations.SerializedName; +import lombok.Getter; +import lombok.Setter; +import lombok.extern.log4j.Log4j2; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.collections4.MapUtils; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.stream.Collectors; + +@Getter +@Setter +@Log4j2 +public class JdbcSourceOffsetProvider implements SourceOffsetProvider { + public static final String SPLIT_ID = "splitId"; + private static final ObjectMapper objectMapper = new ObjectMapper(); + private Long jobId; + private DataSourceType sourceType; + private Map sourceProperties = new HashMap<>(); + + List remainingSplits = new ArrayList<>(); + List finishedSplits = new ArrayList<>(); + + JdbcOffset currentOffset; + Map endBinlogOffset; + + @SerializedName("chw") + // tableID -> splitId -> chunk of highWatermark + Map>> chunkHighWatermarkMap; + @SerializedName("bop") + Map binlogOffsetPersist; + + boolean hasMoreData = true; + + public JdbcSourceOffsetProvider(Long jobId, DataSourceType sourceType, Map sourceProperties) { + this.jobId = jobId; + this.sourceType = sourceType; + this.sourceProperties = sourceProperties; + this.chunkHighWatermarkMap = new HashMap<>(); + } + + @Override + public String getSourceType() { + return "jdbc"; + } + + @Override + public Offset getNextOffset(StreamingJobProperties jobProps, Map properties) { + JdbcOffset nextOffset = new JdbcOffset(); + if (!remainingSplits.isEmpty()) { + // snapshot read + SnapshotSplit snapshotSplit = remainingSplits.get(0); + nextOffset.setSplit(snapshotSplit); + return nextOffset; + } else if (currentOffset != null && currentOffset.getSplit().snapshotSplit()) { + // snapshot to binlog + BinlogSplit binlogSplit = new BinlogSplit(); + binlogSplit.setFinishedSplits(finishedSplits); + nextOffset.setSplit(binlogSplit); + return nextOffset; + } else { + // only binlog + return currentOffset == null ? new JdbcOffset(new BinlogSplit()) : currentOffset; + } + } + + @Override + public String getShowCurrentOffset() { + if (this.currentOffset != null) { + AbstractSourceSplit split = this.currentOffset.getSplit(); + if (split.snapshotSplit()) { + return new Gson().toJson(split); + } else { + BinlogSplit binlogSplit = (BinlogSplit) split; + HashMap showMap = new HashMap<>(); + showMap.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); + if (binlogSplit.getStartingOffset() != null) { + showMap.putAll(binlogSplit.getStartingOffset()); + } + return new Gson().toJson(showMap); + } + } + return null; + } + + @Override + public String getShowMaxOffset() { + if (endBinlogOffset != null) { + return new Gson().toJson(endBinlogOffset); + } + return null; + } + + @Override + public InsertIntoTableCommand rewriteTvfParams(InsertIntoTableCommand originCommand, Offset nextOffset) { + // todo: only for cdc tvf + return null; + } + + @Override + public void updateOffset(Offset offset) { + this.currentOffset = (JdbcOffset) offset; + AbstractSourceSplit split = currentOffset.getSplit(); + if (split.snapshotSplit()) { + SnapshotSplit snapshotSplit = (SnapshotSplit) split; + String splitId = split.getSplitId(); + remainingSplits.removeIf(v -> { + if (v.getSplitId().equals(splitId)) { + snapshotSplit.setTableId(v.getTableId()); + snapshotSplit.setSplitKey(v.getSplitKey()); + snapshotSplit.setSplitStart(v.getSplitStart()); + snapshotSplit.setSplitEnd(v.getSplitEnd()); + return true; + } + return false; + }); + finishedSplits.add(snapshotSplit); + chunkHighWatermarkMap.computeIfAbsent(snapshotSplit.getTableId(), k -> new HashMap<>()) + .put(snapshotSplit.getSplitId(), snapshotSplit.getHighWatermark()); + } else { + BinlogSplit binlogSplit = (BinlogSplit) split; + binlogOffsetPersist = new HashMap<>(binlogSplit.getStartingOffset()); + binlogOffsetPersist.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID); + } + } + + @Override + public void fetchRemoteMeta(Map properties) throws Exception { + Backend backend = StreamingJobUtils.selectBackend(jobId); + JobBaseConfig requestParams = new JobBaseConfig(getJobId(), sourceType.name(), sourceProperties); + InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() + .setApi("/api/fetchEndOffset") + .setParams(new Gson().toJson(requestParams)).build(); + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); + InternalService.PRequestCdcClientResult result = null; + try { + Future future = + BackendServiceProxy.getInstance().requestCdcClient(address, request); + result = future.get(); + TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + log.error("Failed to get end offset from backend, {}", result.getStatus().getErrorMsgs(0)); + throw new JobException( + "Failed to get end offset from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + + result.getResponse()); + } + String response = result.getResponse(); + try { + ResponseBody> responseObj = objectMapper.readValue( + response, + new TypeReference>>() { + } + ); + if (endBinlogOffset != null + && !endBinlogOffset.equals(responseObj.getData())) { + hasMoreData = true; + } + endBinlogOffset = responseObj.getData(); + } catch (JsonProcessingException e) { + log.error("Failed to parse end offset response: {}", response, e); + throw new JobException(response); + } + } catch (ExecutionException | InterruptedException ex) { + log.error("Get end offset error: ", ex); + throw new JobException(ex); + } + } + + @Override + public boolean hasMoreDataToConsume() { + if (!hasMoreData) { + return false; + } + + if (currentOffset == null) { + return true; + } + + if (CollectionUtils.isNotEmpty(remainingSplits)) { + return true; + } + if (MapUtils.isEmpty(endBinlogOffset)) { + return false; + } + try { + if (!currentOffset.getSplit().snapshotSplit()) { + BinlogSplit binlogSplit = (BinlogSplit) currentOffset.getSplit(); + return compareOffset(endBinlogOffset, new HashMap<>(binlogSplit.getStartingOffset())); + } else { + SnapshotSplit snapshotSplit = (SnapshotSplit) currentOffset.getSplit(); + if (MapUtils.isNotEmpty(snapshotSplit.getHighWatermark())) { + return compareOffset(endBinlogOffset, new HashMap<>(snapshotSplit.getHighWatermark())); + } + } + } catch (Exception ex) { + log.info("Compare offset error: ", ex); + return false; + } + return false; + } + + private boolean compareOffset(Map offsetFirst, Map offsetSecond) + throws JobException { + Backend backend = StreamingJobUtils.selectBackend(jobId); + CompareOffsetRequest requestParams = + new CompareOffsetRequest(getJobId(), sourceType.name(), sourceProperties, offsetFirst, offsetSecond); + InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() + .setApi("/api/compareOffset") + .setParams(new Gson().toJson(requestParams)).build(); + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); + InternalService.PRequestCdcClientResult result = null; + try { + Future future = + BackendServiceProxy.getInstance().requestCdcClient(address, request); + result = future.get(); + TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + log.error("Failed to compare offset , {}", result.getStatus().getErrorMsgs(0)); + throw new JobException( + "Failed to compare offset ," + result.getStatus().getErrorMsgs(0) + ", response: " + + result.getResponse()); + } + String response = result.getResponse(); + try { + ResponseBody responseObj = objectMapper.readValue( + response, + new TypeReference>() { + } + ); + return responseObj.getData() > 0; + } catch (JsonProcessingException e) { + log.error("Failed to parse compare offset response: {}", response, e); + throw new JobException("Failed to parse compare offset response: " + response); + } + } catch (ExecutionException | InterruptedException ex) { + log.error("Compare offset error: ", ex); + throw new JobException(ex); + } + } + + @Override + public Offset deserializeOffset(String offset) { + try { + // chunk is highWatermark, binlog is offset map + Map offsetMeta = objectMapper.readValue(offset, new TypeReference>() { + }); + String splitId = offsetMeta.remove(SPLIT_ID); + if (BinlogSplit.BINLOG_SPLIT_ID.equals(splitId)) { + BinlogSplit binlogSplit = new BinlogSplit(); + binlogSplit.setSplitId(splitId); + binlogSplit.setStartingOffset(offsetMeta); + return new JdbcOffset(binlogSplit); + } else { + SnapshotSplit snapshotSplit = new SnapshotSplit(); + snapshotSplit.setSplitId(splitId); + snapshotSplit.setHighWatermark(offsetMeta); + return new JdbcOffset(snapshotSplit); + } + } catch (JsonProcessingException e) { + log.warn("Failed to deserialize offset: {}", offset, e); + throw new RuntimeException(e); + } + } + + @Override + public Offset deserializeOffsetProperty(String offset) { + // no need + return null; + } + + /** + * Replay snapshot splits if needed + */ + @Override + public void replayIfNeed(StreamingInsertJob job) throws JobException { + String offsetProviderPersist = job.getOffsetProviderPersist(); + if (job.getOffsetProviderPersist() == null) { + return; + } + JdbcSourceOffsetProvider replayFromPersist = GsonUtils.GSON.fromJson(offsetProviderPersist, + JdbcSourceOffsetProvider.class); + this.binlogOffsetPersist = replayFromPersist.getBinlogOffsetPersist(); + this.chunkHighWatermarkMap = replayFromPersist.getChunkHighWatermarkMap(); + + if (MapUtils.isNotEmpty(binlogOffsetPersist)) { + currentOffset = new JdbcOffset(); + currentOffset.setSplit(new BinlogSplit(binlogOffsetPersist)); + } else { + try { + Map> snapshotSplits = StreamingJobUtils.restoreSplitsToJob(job.getJobId()); + if (MapUtils.isNotEmpty(chunkHighWatermarkMap) && MapUtils.isNotEmpty(snapshotSplits)) { + SnapshotSplit lastSnapshotSplit = recalculateRemainingSplits(chunkHighWatermarkMap, snapshotSplits); + if (this.remainingSplits.isEmpty()) { + currentOffset = new JdbcOffset(); + currentOffset.setSplit(lastSnapshotSplit); + } + } + } catch (Exception ex) { + log.warn("Replay snapshot splits error with job {} ", job.getJobId(), ex); + throw new JobException(ex); + } + } + } + + /** + * Assign the HW value to the synchronized Split, + * and remove the Split from remainSplit and place it in finishedSplit. + */ + private SnapshotSplit recalculateRemainingSplits( + Map>> chunkHighWatermarkMap, + Map> snapshotSplits) { + if (this.finishedSplits == null) { + this.finishedSplits = new ArrayList<>(); + } + SnapshotSplit lastSnapshotSplit = null; + for (Map.Entry>> entry : chunkHighWatermarkMap.entrySet()) { + String tableId = entry.getKey(); + Map> splitIdToHighWatermark = entry.getValue(); + if (MapUtils.isEmpty(splitIdToHighWatermark)) { + continue; + } + // db.schema.table + String tableName = getTableName(tableId); + if (tableName == null) { + continue; + } + List tableSplits = snapshotSplits.get(tableName); + if (CollectionUtils.isEmpty(tableSplits)) { + continue; + } + lastSnapshotSplit = tableSplits.get(tableSplits.size() - 1); + tableSplits.removeIf(split -> { + String splitId = split.getSplitId(); + Map highWatermark = splitIdToHighWatermark.get(splitId); + if (highWatermark != null) { + split.setHighWatermark(highWatermark); + finishedSplits.add(split); + return true; + } + return false; + }); + } + + this.remainingSplits = snapshotSplits.values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); + return lastSnapshotSplit; + } + + private String getTableName(String tableId) { + if (tableId == null) { + return null; + } + String[] split = tableId.split("\\."); + return split[split.length - 1]; + } + + @Override + public String getPersistInfo() { + return GsonUtils.GSON.toJson(this); + } + + public void splitChunks(List createTbls) throws JobException { + // todo: When splitting takes a long time, it needs to be changed to asynchronous. + if (checkNeedSplitChunks(sourceProperties)) { + Map> tableSplits = new LinkedHashMap<>(); + for (String tbl : createTbls) { + List snapshotSplits = requestTableSplits(tbl); + tableSplits.put(tbl, snapshotSplits); + } + // save chunk list to system table + saveChunkMeta(tableSplits); + this.remainingSplits = tableSplits.values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); + } + } + + private void saveChunkMeta(Map> tableSplits) throws JobException { + try { + StreamingJobUtils.createMetaTableIfNotExist(); + StreamingJobUtils.insertSplitsToMeta(getJobId(), tableSplits); + } catch (Exception e) { + log.warn("save chunk meta error: ", e); + throw new JobException(e.getMessage()); + } + } + + private List requestTableSplits(String table) throws JobException { + Backend backend = StreamingJobUtils.selectBackend(jobId); + FetchTableSplitsRequest requestParams = + new FetchTableSplitsRequest(getJobId(), sourceType.name(), sourceProperties, table); + InternalService.PRequestCdcClientRequest request = InternalService.PRequestCdcClientRequest.newBuilder() + .setApi("/api/fetchSplits") + .setParams(new Gson().toJson(requestParams)).build(); + TNetworkAddress address = new TNetworkAddress(backend.getHost(), backend.getBrpcPort()); + InternalService.PRequestCdcClientResult result = null; + try { + Future future = + BackendServiceProxy.getInstance().requestCdcClient(address, request); + result = future.get(); + TStatusCode code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + log.error("Failed to get split from backend, {}", result.getStatus().getErrorMsgs(0)); + throw new JobException( + "Failed to get split from backend," + result.getStatus().getErrorMsgs(0) + ", response: " + + result.getResponse()); + } + String response = result.getResponse(); + try { + ResponseBody> responseObj = objectMapper.readValue( + response, + new TypeReference>>() { + } + ); + List splits = responseObj.getData(); + return splits; + } catch (JsonProcessingException e) { + log.error("Failed to parse split response: {}", response, e); + throw new JobException("Failed to parse split response: " + response); + } + } catch (ExecutionException | InterruptedException ex) { + log.error("Get splits error: ", ex); + throw new JobException(ex); + } + } + + private boolean checkNeedSplitChunks(Map sourceProperties) { + String startMode = sourceProperties.get(DataSourceConfigKeys.OFFSET); + if (startMode == null) { + return false; + } + return DataSourceConfigKeys.OFFSET_INITIAL.equalsIgnoreCase(startMode); + } + + public void cleanMeta(Long jobId) { + // clean meta table + StreamingJobUtils.deleteJobMeta(jobId); + // todo: close cdc client source + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java index ea1cf7825ec56b..91d5fb1a658737 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/scheduler/StreamingTaskScheduler.java @@ -25,8 +25,8 @@ import org.apache.doris.job.common.FailureReason; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.AbstractStreamingTask; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; -import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; import lombok.extern.log4j.Log4j2; @@ -74,16 +74,16 @@ protected void runAfterCatalogReady() { } private void process() throws InterruptedException { - List tasks = new ArrayList<>(); - LinkedBlockingDeque needScheduleTasksQueue = + List tasks = new ArrayList<>(); + LinkedBlockingDeque needScheduleTasksQueue = Env.getCurrentEnv().getJobManager().getStreamingTaskManager().getNeedScheduleTasksQueue(); tasks.add(needScheduleTasksQueue.take()); needScheduleTasksQueue.drainTo(tasks); scheduleTasks(tasks); } - private void scheduleTasks(List tasks) { - for (StreamingInsertTask task : tasks) { + private void scheduleTasks(List tasks) { + for (AbstractStreamingTask task : tasks) { threadPool.execute(() -> { try { scheduleOneTask(task); @@ -104,7 +104,7 @@ private void scheduleTasks(List tasks) { } } - private void scheduleOneTask(StreamingInsertTask task) { + private void scheduleOneTask(AbstractStreamingTask task) { if (DebugPointUtil.isEnable("StreamingJob.scheduleTask.exception")) { throw new RuntimeException("debug point StreamingJob.scheduleTask.exception"); } @@ -143,7 +143,7 @@ private void scheduleOneTask(StreamingInsertTask task) { } } - private void scheduleTaskWithDelay(StreamingInsertTask task, long delayMs) { + private void scheduleTaskWithDelay(AbstractStreamingTask task, long delayMs) { delayScheduler.schedule(() -> { Env.getCurrentEnv().getJobManager().getStreamingTaskManager().registerTask(task); }, delayMs, TimeUnit.MILLISECONDS); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java new file mode 100644 index 00000000000000..436ef0aea35991 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/util/StreamingJobUtils.java @@ -0,0 +1,337 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.job.util; + +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.FeConstants; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.datasource.jdbc.client.JdbcClient; +import org.apache.doris.datasource.jdbc.client.JdbcClientConfig; +import org.apache.doris.datasource.jdbc.client.JdbcMySQLClient; +import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.cdc.split.SnapshotSplit; +import org.apache.doris.job.common.DataSourceType; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand; +import org.apache.doris.nereids.trees.plans.commands.info.ColumnDefinition; +import org.apache.doris.nereids.trees.plans.commands.info.CreateTableInfo; +import org.apache.doris.nereids.trees.plans.commands.info.DistributionDescriptor; +import org.apache.doris.nereids.trees.plans.commands.info.PartitionTableInfo; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.qe.AutoCloseConnectContext; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.StmtExecutor; +import org.apache.doris.statistics.ResultRow; +import org.apache.doris.system.Backend; +import org.apache.doris.system.BeSelectionPolicy; +import org.apache.doris.system.SystemInfoService; +import org.apache.doris.thrift.TUniqueId; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import lombok.extern.log4j.Log4j2; +import org.apache.commons.text.StringSubstitutor; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.stream.Collectors; + +@Log4j2 +public class StreamingJobUtils { + public static final String TABLE_PROPS_PREFIX = "table.create.properties."; + public static final String INTERNAL_STREAMING_JOB_META_TABLE_NAME = "streaming_job_meta"; + public static final String FULL_QUALIFIED_META_TBL_NAME = InternalCatalog.INTERNAL_CATALOG_NAME + + "." + FeConstants.INTERNAL_DB_NAME + "." + INTERNAL_STREAMING_JOB_META_TABLE_NAME; + private static final String CREATE_META_TABLE = "CREATE TABLE IF NOT EXISTS %s(\n" + + "id int,\n" + + "job_id bigint,\n" + + "table_name string,\n" + + "chunk_list json\n" + + ")\n" + + "UNIQUE KEY(id, job_id)\n" + + "DISTRIBUTED BY HASH(job_id)\n" + + "BUCKETS 2\n" + + "PROPERTIES ('replication_num' = '1')"; // todo: modify replication num like statistic sys tbl + private static final String BATCH_INSERT_INTO_META_TABLE_TEMPLATE = + "INSERT INTO " + FULL_QUALIFIED_META_TBL_NAME + " values"; + + private static final String INSERT_INTO_META_TABLE_TEMPLATE = + "('${id}', '${job_id}', '${table_name}', '${chunk_list}')"; + + private static final String SELECT_SPLITS_TABLE_TEMPLATE = + "SELECT table_name, chunk_list FROM " + FULL_QUALIFIED_META_TBL_NAME + " WHERE job_id='%s' ORDER BY id ASC"; + + private static final String DELETE_JOB_META_TEMPLATE = + "DELETE FROM " + FULL_QUALIFIED_META_TBL_NAME + " WHERE job_id='%s'"; + + private static final ObjectMapper objectMapper = new ObjectMapper(); + + public static void createMetaTableIfNotExist() throws Exception { + Optional optionalDatabase = + Env.getCurrentEnv().getInternalCatalog() + .getDb(FeConstants.INTERNAL_DB_NAME); + if (!optionalDatabase.isPresent()) { + // should not happen + throw new JobException("Internal database does not exist"); + } + Database database = optionalDatabase.get(); + Table t = database.getTableNullable(INTERNAL_STREAMING_JOB_META_TABLE_NAME); + if (t == null) { + execute(String.format(CREATE_META_TABLE, FULL_QUALIFIED_META_TBL_NAME)); + // double check + t = database.getTableNullable(INTERNAL_STREAMING_JOB_META_TABLE_NAME); + if (t == null) { + throw new JobException(String.format("Table %s doesn't exist", FULL_QUALIFIED_META_TBL_NAME)); + } + } + } + + public static Map> restoreSplitsToJob(Long jobId) throws IOException { + List resultRows; + String sql = String.format(SELECT_SPLITS_TABLE_TEMPLATE, jobId); + try (AutoCloseConnectContext context + = new AutoCloseConnectContext(buildConnectContext())) { + StmtExecutor stmtExecutor = new StmtExecutor(context.connectContext, sql); + resultRows = stmtExecutor.executeInternalQuery(); + } + + Map> tableSplits = new LinkedHashMap<>(); + for (ResultRow row : resultRows) { + String tableName = row.get(0); + String chunkListStr = row.get(1); + List splits = + new ArrayList<>(Arrays.asList(objectMapper.readValue(chunkListStr, SnapshotSplit[].class))); + tableSplits.put(tableName, splits); + } + return tableSplits; + } + + public static void deleteJobMeta(Long jobId) { + String sql = String.format(DELETE_JOB_META_TEMPLATE, jobId); + try { + execute(sql); + } catch (Exception e) { + log.info("Failed to delete job meta for job id {}: {}", + jobId, e.getMessage(), e); + } + } + + public static void insertSplitsToMeta(Long jobId, Map> tableSplits) throws Exception { + List values = new ArrayList<>(); + int index = 1; + for (Map.Entry> entry : tableSplits.entrySet()) { + Map params = new HashMap<>(); + params.put("id", index + ""); + params.put("job_id", jobId + ""); + params.put("table_name", entry.getKey()); + params.put("chunk_list", objectMapper.writeValueAsString(entry.getValue())); + StringSubstitutor stringSubstitutor = new StringSubstitutor(params); + String sql = stringSubstitutor.replace(INSERT_INTO_META_TABLE_TEMPLATE); + values.add(sql); + index++; + } + batchInsert(values); + } + + private static void batchInsert(List values) throws Exception { + if (values.isEmpty()) { + return; + } + StringBuilder insertSQL = new StringBuilder(BATCH_INSERT_INTO_META_TABLE_TEMPLATE); + for (int i = 0; i < values.size(); i++) { + insertSQL.append(values.get(i)); + if (i + 1 != values.size()) { + insertSQL.append(","); + } else { + insertSQL.append(";"); + } + } + execute(insertSQL.toString()); + } + + private static void execute(String sql) throws Exception { + try (AutoCloseConnectContext context + = new AutoCloseConnectContext(buildConnectContext())) { + StmtExecutor stmtExecutor = new StmtExecutor(context.connectContext, sql); + stmtExecutor.execute(); + } + } + + private static ConnectContext buildConnectContext() { + ConnectContext ctx = new ConnectContext(); + ctx.setEnv(Env.getCurrentEnv()); + ctx.setCurrentUserIdentity(UserIdentity.ADMIN); + ctx.getState().reset(); + ctx.getState().setInternal(true); + ctx.getState().setNereids(true); + ctx.setThreadLocalInfo(); + UUID uuid = UUID.randomUUID(); + TUniqueId queryId = new TUniqueId(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits()); + ctx.setQueryId(queryId); + return ctx; + } + + private static JdbcClient getJdbcClient(DataSourceType sourceType, Map properties) + throws JobException { + JdbcClientConfig config = new JdbcClientConfig(); + config.setCatalog(sourceType.name()); + config.setUser(properties.get(DataSourceConfigKeys.USER)); + config.setPassword(properties.get(DataSourceConfigKeys.PASSWORD)); + config.setDriverClass(properties.get(DataSourceConfigKeys.DRIVER_CLASS)); + config.setDriverUrl(properties.get(DataSourceConfigKeys.DRIVER_URL)); + config.setJdbcUrl(properties.get(DataSourceConfigKeys.JDBC_URL)); + switch (sourceType) { + case MYSQL: + JdbcClient client = JdbcMySQLClient.createJdbcClient(config); + return client; + default: + throw new JobException("Unsupported source type " + sourceType); + } + } + + public static Backend selectBackend(Long jobId) throws JobException { + Backend backend = null; + BeSelectionPolicy policy = null; + + policy = new BeSelectionPolicy.Builder() + .setEnableRoundRobin(true) + .needLoadAvailable().build(); + List backendIds; + backendIds = Env.getCurrentSystemInfo().selectBackendIdsByPolicy(policy, -1); + if (backendIds.isEmpty()) { + throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); + } + // jobid % backendSize + long index = backendIds.get(jobId.intValue() % backendIds.size()); + backend = Env.getCurrentSystemInfo().getBackend(index); + if (backend == null) { + throw new JobException(SystemInfoService.NO_BACKEND_LOAD_AVAILABLE_MSG + ", policy: " + policy); + } + return backend; + } + + public static List generateCreateTableCmds(String targetDb, DataSourceType sourceType, + Map properties, Map targetProperties) + throws JobException { + List createtblCmds = new ArrayList<>(); + String includeTables = properties.get(DataSourceConfigKeys.INCLUDE_TABLES); + String excludeTables = properties.get(DataSourceConfigKeys.EXCLUDE_TABLES); + List includeTablesList = new ArrayList<>(); + if (includeTables != null) { + includeTablesList = Arrays.asList(includeTables.split(",")); + } + + String database = properties.get(DataSourceConfigKeys.DATABASE); + JdbcClient jdbcClient = getJdbcClient(sourceType, properties); + List tablesNameList = jdbcClient.getTablesNameList(database); + if (tablesNameList.isEmpty()) { + throw new JobException("No tables found in database " + database); + } + Map tableCreateProperties = getTableCreateProperties(targetProperties); + List noPrimaryKeyTables = new ArrayList<>(); + for (String table : tablesNameList) { + if (!includeTablesList.isEmpty() && !includeTablesList.contains(table)) { + log.info("Skip table {} in database {} as it does not in include_tables {}", table, database, + includeTables); + continue; + } + + if (excludeTables != null && excludeTables.contains(table)) { + log.info("Skip table {} in database {} as it in exclude_tables {}", table, database, + excludeTables); + continue; + } + + List columns = jdbcClient.getColumnsFromJdbc(database, table); + List primaryKeys = jdbcClient.getPrimaryKeys(database, table); + if (primaryKeys.isEmpty()) { + noPrimaryKeyTables.add(table); + } + // Convert Column to ColumnDefinition + List columnDefinitions = columns.stream().map(col -> { + DataType dataType = DataType.fromCatalogType(col.getType()); + return new ColumnDefinition(col.getName(), dataType, col.isAllowNull(), col.getComment()); + }).collect(Collectors.toList()); + + // Create DistributionDescriptor + DistributionDescriptor distribution = new DistributionDescriptor( + true, // isHash + true, // isAutoBucket + FeConstants.default_bucket_num, + primaryKeys + ); + + // Create CreateTableInfo + CreateTableInfo createtblInfo = new CreateTableInfo( + true, // ifNotExists + false, // isExternal + false, // isTemp + InternalCatalog.INTERNAL_CATALOG_NAME, // ctlName + targetDb, // dbName + table, // tableName + columnDefinitions, // columns + ImmutableList.of(), // indexes + "olap", // engineName + KeysType.UNIQUE_KEYS, // keysType + primaryKeys, // keys + "", // comment + PartitionTableInfo.EMPTY, // partitionTableInfo + distribution, // distribution + ImmutableList.of(), // rollups + new HashMap<>(tableCreateProperties), // properties + ImmutableMap.of(), // extProperties + ImmutableList.of() // clusterKeyColumnNames + ); + CreateTableCommand createtblCmd = new CreateTableCommand(Optional.empty(), createtblInfo); + createtblCmds.add(createtblCmd); + } + if (createtblCmds.isEmpty()) { + throw new JobException("Can not found match table in database " + database); + } + + if (!noPrimaryKeyTables.isEmpty()) { + throw new JobException("The following tables do not have primary key defined: " + + String.join(", ", noPrimaryKeyTables)); + } + return createtblCmds; + } + + private static Map getTableCreateProperties(Map properties) { + final Map tableCreateProps = new HashMap<>(); + for (Map.Entry entry : properties.entrySet()) { + if (entry.getKey().startsWith(TABLE_PROPS_PREFIX)) { + String subKey = entry.getKey().substring(TABLE_PROPS_PREFIX.length()); + tableCreateProps.put(subKey, entry.getValue()); + } + } + return tableCreateProps; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 8721f11038f215..f769c0aace8823 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -248,6 +248,7 @@ import org.apache.doris.nereids.DorisParser.IntervalContext; import org.apache.doris.nereids.DorisParser.Is_not_null_predContext; import org.apache.doris.nereids.DorisParser.IsnullContext; +import org.apache.doris.nereids.DorisParser.JobFromToClauseContext; import org.apache.doris.nereids.DorisParser.JoinCriteriaContext; import org.apache.doris.nereids.DorisParser.JoinRelationContext; import org.apache.doris.nereids.DorisParser.KillQueryContext; @@ -1194,13 +1195,26 @@ public LogicalPlan visitCreateScheduledJob(DorisParser.CreateScheduledJobContext Optional interval = ctx.timeInterval == null ? Optional.empty() : Optional.of(Long.valueOf(ctx.timeInterval.getText())); Optional intervalUnit = ctx.timeUnit == null ? Optional.empty() : Optional.of(ctx.timeUnit.getText()); - Map properties = ctx.propertyClause() != null - ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); + Map jobProperties = ctx.jobProperties != null + ? Maps.newHashMap(visitPropertyClause(ctx.jobProperties)) : Maps.newHashMap(); String comment = visitCommentSpec(ctx.commentSpec()); - String executeSql = getOriginSql(ctx.supportedDmlStatement()); + String executeSql = ctx.supportedDmlStatement() == null ? "" : getOriginSql(ctx.supportedDmlStatement()); + JobFromToClauseContext jobFromToClauseCtx = ctx.jobFromToClause(); + String sourceType = null; + String targetDb = null; + Map sourceProperties = Maps.newHashMap(); + Map targetProperties = Maps.newHashMap(); + if (jobFromToClauseCtx != null) { + sourceType = jobFromToClauseCtx.sourceType.getText(); + targetDb = jobFromToClauseCtx.targetDb == null ? "" : jobFromToClauseCtx.targetDb.getText(); + sourceProperties = Maps.newHashMap(visitPropertyItemList(jobFromToClauseCtx.sourceProperties)); + targetProperties = jobFromToClauseCtx.targetProperties != null + ? Maps.newHashMap(visitPropertyItemList(jobFromToClauseCtx.targetProperties)) : Maps.newHashMap(); + } CreateJobInfo createJobInfo = new CreateJobInfo(label, atTime, interval, intervalUnit, startTime, - endsTime, immediateStartOptional, comment, executeSql, ctx.STREAMING() != null, properties); + endsTime, immediateStartOptional, comment, executeSql, ctx.STREAMING() != null, + jobProperties, sourceType, targetDb, sourceProperties, targetProperties); return new CreateJobCommand(createJobInfo); } @@ -1215,7 +1229,20 @@ public LogicalPlan visitAlterJob(DorisParser.AlterJobContext ctx) { Map properties = ctx.propertyClause() != null ? Maps.newHashMap(visitPropertyClause(ctx.propertyClause())) : Maps.newHashMap(); String executeSql = ctx.supportedDmlStatement() != null ? getOriginSql(ctx.supportedDmlStatement()) : ""; - return new AlterJobCommand(ctx.jobName.getText(), properties, executeSql); + String sourceType = null; + String targetDb = null; + Map sourceProperties = Maps.newHashMap(); + Map targetProperties = Maps.newHashMap(); + if (ctx.jobFromToClause() != null) { + sourceType = ctx.jobFromToClause().sourceType.getText(); + targetDb = ctx.jobFromToClause().targetDb == null ? "" : ctx.jobFromToClause().targetDb.getText(); + sourceProperties = Maps.newHashMap(visitPropertyItemList(ctx.jobFromToClause().sourceProperties)); + targetProperties = ctx.jobFromToClause().targetProperties != null + ? Maps.newHashMap(visitPropertyItemList(ctx.jobFromToClause().targetProperties)) + : Maps.newHashMap(); + } + return new AlterJobCommand(ctx.jobName.getText(), properties, + executeSql, sourceType, targetDb, sourceProperties, targetProperties); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index 6cbd17ced6c20a..584ee0848db2a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -23,7 +23,9 @@ import org.apache.doris.common.Pair; import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.base.JobExecuteType; +import org.apache.doris.job.cdc.DataSourceConfigKeys; import org.apache.doris.job.common.JobStatus; +import org.apache.doris.job.extensions.insert.streaming.DataSourceConfigValidator; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.nereids.analyzer.UnboundTVFRelation; @@ -37,6 +39,7 @@ import org.apache.doris.qe.StmtExecutor; import com.google.common.base.Preconditions; +import org.apache.commons.lang3.StringUtils; import java.util.List; import java.util.Map; @@ -51,12 +54,29 @@ public class AlterJobCommand extends AlterCommand implements ForwardWithSync, Ne private final String jobName; private final Map properties; private final String sql; + private final String sourceType; + private final String targetDb; + private final Map sourceProperties; + private final Map targetProperties; - public AlterJobCommand(String jobName, Map properties, String sql) { + /** + * AlterJobCommand constructor. + */ + public AlterJobCommand(String jobName, + Map properties, + String sql, + String sourceType, + String targetDb, + Map sourceProperties, + Map targetProperties) { super(PlanType.ALTER_JOB_COMMAND); this.jobName = jobName; this.properties = properties; this.sql = sql; + this.sourceType = sourceType; + this.targetDb = targetDb; + this.sourceProperties = sourceProperties; + this.targetProperties = targetProperties; } public String getJobName() { @@ -71,6 +91,22 @@ public String getSql() { return sql; } + public String getSourceType() { + return sourceType; + } + + public String getTargetDb() { + return targetDb; + } + + public Map getSourceProperties() { + return sourceProperties; + } + + public Map getTargetProperties() { + return targetProperties; + } + @Override public StmtType stmtType() { return StmtType.ALTER; @@ -100,26 +136,90 @@ private void validate() throws Exception { StreamingInsertJob streamingJob = (StreamingInsertJob) job; streamingJob.checkPrivilege(ConnectContext.get()); - boolean propModified = isPropertiesModified(streamingJob.getProperties()); - if (propModified) { - validateProps(streamingJob); - } - boolean sqlModified = isSqlModified(streamingJob.getExecuteSql()); - if (sqlModified) { - checkUnmodifiableProperties(streamingJob.getExecuteSql()); - } - if (!propModified && !sqlModified) { - throw new AnalysisException("No properties or sql changed in ALTER JOB"); + if (sourceType == null) { + boolean propModified = + isPropertiesModified(streamingJob.getProperties(), this.getProperties()); + if (propModified) { + validateProps(streamingJob); + } + boolean sqlModified = isSqlModified(streamingJob.getExecuteSql()); + if (sqlModified) { + checkUnmodifiableProperties(streamingJob.getExecuteSql()); + } + if (!propModified && !sqlModified) { + throw new AnalysisException("No properties or sql changed in ALTER JOB"); + } + } else { + if (!sourceType.toUpperCase().equals(streamingJob.getDataSourceType().name())) { + throw new AnalysisException("source type can't be modified in ALTER JOB"); + } + + if (StringUtils.isNotEmpty(targetDb) && !targetDb.equals(streamingJob.getTargetDb())) { + throw new AnalysisException("target database can't be modified in ALTER JOB"); + } + + boolean propModified = isPropertiesModified(streamingJob.getProperties(), this.getProperties()); + if (propModified) { + validateProps(streamingJob); + } + + boolean sourcePropModified = + isPropertiesModified(streamingJob.getSourceProperties(), this.getSourceProperties()); + if (sourcePropModified) { + DataSourceConfigValidator.validateSource(this.getSourceProperties()); + checkUnmodifiableSourceProperties(streamingJob.getSourceProperties()); + } + + boolean targetPropModified = + isPropertiesModified(streamingJob.getTargetProperties(), this.getTargetProperties()); + if (targetPropModified) { + DataSourceConfigValidator.validateTarget(this.getTargetProperties()); + } + if (!propModified && !targetPropModified && !sourcePropModified) { + throw new AnalysisException("No properties or source or target properties changed in ALTER JOB"); + } } } else { throw new AnalysisException("Unsupported job type for ALTER:" + job.getJobType()); } } + private void checkUnmodifiableSourceProperties(Map originSourceProperties) { + if (sourceProperties.containsKey(DataSourceConfigKeys.JDBC_URL)) { + Preconditions.checkArgument(Objects.equals( + originSourceProperties.get(DataSourceConfigKeys.JDBC_URL), + sourceProperties.get(DataSourceConfigKeys.JDBC_URL)), + "The jdbc_url property cannot be modified in ALTER JOB"); + } + + if (sourceProperties.containsKey(DataSourceConfigKeys.DATABASE)) { + Preconditions.checkArgument(Objects.equals( + originSourceProperties.get(DataSourceConfigKeys.DATABASE), + sourceProperties.get(DataSourceConfigKeys.DATABASE)), + "The database property cannot be modified in ALTER JOB"); + } + + if (sourceProperties.containsKey(DataSourceConfigKeys.INCLUDE_TABLES)) { + Preconditions.checkArgument(Objects.equals( + originSourceProperties.get(DataSourceConfigKeys.INCLUDE_TABLES), + sourceProperties.get(DataSourceConfigKeys.INCLUDE_TABLES)), + "The include_tables property cannot be modified in ALTER JOB"); + } + + if (sourceProperties.containsKey(DataSourceConfigKeys.EXCLUDE_TABLES)) { + Preconditions.checkArgument(Objects.equals( + originSourceProperties.get(DataSourceConfigKeys.EXCLUDE_TABLES), + sourceProperties.get(DataSourceConfigKeys.EXCLUDE_TABLES)), + "The exclude_tables property cannot be modified in ALTER JOB"); + } + } + private void validateProps(StreamingInsertJob streamingJob) throws AnalysisException { StreamingJobProperties jobProperties = new StreamingJobProperties(properties); jobProperties.validate(); - if (jobProperties.getOffsetProperty() != null) { + // from to job no need valiate offset in job properties + if (streamingJob.getDataSourceType() == null + && jobProperties.getOffsetProperty() != null) { streamingJob.validateOffset(jobProperties.getOffsetProperty()); } } @@ -165,11 +265,11 @@ private Pair, UnboundTVFRelation> getTargetTableAndTvf(String sql) return Pair.of(targetTable, unboundTVFRelation); } - private boolean isPropertiesModified(Map originProps) { - if (this.properties == null || this.properties.isEmpty()) { + private boolean isPropertiesModified(Map originProps, Map modifiedProps) { + if (modifiedProps == null || modifiedProps.isEmpty()) { return false; } - if (!Objects.equals(this.properties, originProps)) { + if (!Objects.equals(modifiedProps, originProps)) { return true; } return false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java index 03a423e1b0b850..1640143d277f01 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java @@ -22,12 +22,15 @@ import org.apache.doris.common.Config; import org.apache.doris.job.base.AbstractJob; import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.DataSourceConfigValidator; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.commands.info.CreateJobInfo; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.StmtExecutor; +import org.apache.commons.lang3.StringUtils; + /** * syntax: * CREATE @@ -69,6 +72,11 @@ private void validate() throws JobException { if (streamingJobCnt >= Config.max_streaming_job_num) { throw new JobException("Exceed max streaming job num limit in fe.conf:" + Config.max_streaming_job_num); } + + if (StringUtils.isNotEmpty(createJobInfo.getSourceType())) { + DataSourceConfigValidator.validateSource(createJobInfo.getSourceProperties()); + DataSourceConfigValidator.validateTarget(createJobInfo.getTargetProperties()); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java index 361be24ee07be4..544e649059bf70 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.trees.plans.commands.info; +import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; @@ -28,6 +29,7 @@ import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.base.TimerDefinition; +import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.common.IntervalUnit; import org.apache.doris.job.common.JobStatus; import org.apache.doris.job.extensions.insert.InsertJob; @@ -39,6 +41,7 @@ import org.apache.doris.qe.ConnectContext; import com.google.common.base.Strings; +import org.apache.commons.lang3.StringUtils; import java.util.Map; import java.util.Optional; @@ -70,6 +73,10 @@ public class CreateJobInfo { private final String executeSql; private final boolean streamingJob; private final Map jobProperties; + private final String sourceType; + private final String targetDb; + private final Map sourceProperties; + private final Map targetProperties; /** * Constructor for CreateJobInfo. @@ -88,7 +95,8 @@ public CreateJobInfo(Optional labelNameOptional, Optional onceJo Optional intervalOptional, Optional intervalTimeUnitOptional, Optional startsTimeStampOptional, Optional endsTimeStampOptional, Optional immediateStartOptional, String comment, String executeSql, - boolean streamingJob, Map jobProperties) { + boolean streamingJob, Map jobProperties, String sourceType, + String targetDb, Map sourceProperties, Map targetProperties) { this.labelNameOptional = labelNameOptional; this.onceJobStartTimestampOptional = onceJobStartTimestampOptional; this.intervalOptional = intervalOptional; @@ -100,6 +108,10 @@ public CreateJobInfo(Optional labelNameOptional, Optional onceJo this.executeSql = executeSql; this.streamingJob = streamingJob; this.jobProperties = jobProperties; + this.sourceType = sourceType; + this.targetDb = targetDb; + this.sourceProperties = sourceProperties; + this.targetProperties = targetProperties; } /** @@ -137,8 +149,23 @@ public AbstractJob analyzeAndBuildJobInfo(ConnectContext ctx) throws UserExcepti } else { buildRecurringJob(timerDefinition, jobExecutionConfiguration); } + jobExecutionConfiguration.setTimerDefinition(timerDefinition); - return analyzeAndCreateJob(executeSql, dbName, jobExecutionConfiguration, jobProperties); + // set source type + if (streamingJob) { + if (StringUtils.isNotEmpty(sourceType)) { + DataSourceType dataSourceType = DataSourceType.valueOf(sourceType.toUpperCase()); + return analyzeAndCreateFromSourceJob(dbName, jobExecutionConfiguration, + jobProperties, targetDb, dataSourceType, sourceProperties, targetProperties); + } else { + return analyzeAndCreateStreamingInsertJob(executeSql, dbName, jobExecutionConfiguration, jobProperties); + } + } else { + if (sourceType != null) { + throw new AnalysisException("From..To Database is only supported in streaming job"); + } + return analyzeAndCreateInsertJob(executeSql, dbName, jobExecutionConfiguration); + } } private void buildStreamingJob(TimerDefinition timerDefinition) { @@ -213,7 +240,7 @@ private void buildRecurringJob(TimerDefinition timerDefinition, protected void checkAuth() throws AnalysisException { if (streamingJob) { - StreamingInsertJob.checkPrivilege(ConnectContext.get(), executeSql); + StreamingInsertJob.checkPrivilege(ConnectContext.get(), executeSql, targetDb); return; } @@ -222,6 +249,34 @@ protected void checkAuth() throws AnalysisException { } } + /** + * Analyzes From Source To Database. + * + * @return an instance of AbstractJob corresponding to the SQL statement + * @throws UserException if there is an error during SQL analysis or job creation + */ + private AbstractJob analyzeAndCreateFromSourceJob(String currentDbName, + JobExecutionConfiguration jobExecutionConfiguration, + Map jobProperties, + String targetDb, + DataSourceType dataSourceType, + Map sourceProperties, + Map targetProperties) throws UserException { + Optional db = Env.getCurrentEnv().getInternalCatalog().getDb(targetDb); + if (!db.isPresent()) { + throw new AnalysisException("Target database " + targetDb + " does not exist"); + } + return new StreamingInsertJob(labelNameOptional.get(), + JobStatus.PENDING, + currentDbName, + comment, + ConnectContext.get().getCurrentUserIdentity(), + jobExecutionConfiguration, + System.currentTimeMillis(), + "", + jobProperties, targetDb, dataSourceType, sourceProperties, targetProperties); + } + /** * Analyzes the provided SQL statement and creates an appropriate job based on the parsed logical plan. * Currently, only "InsertIntoTableCommand" is supported for job creation. @@ -232,16 +287,6 @@ protected void checkAuth() throws AnalysisException { * @return an instance of AbstractJob corresponding to the SQL statement * @throws UserException if there is an error during SQL analysis or job creation */ - private AbstractJob analyzeAndCreateJob(String sql, String currentDbName, - JobExecutionConfiguration jobExecutionConfiguration, - Map properties) throws UserException { - if (jobExecutionConfiguration.getExecuteType().equals(JobExecuteType.STREAMING)) { - return analyzeAndCreateStreamingInsertJob(sql, currentDbName, jobExecutionConfiguration, properties); - } else { - return analyzeAndCreateInsertJob(sql, currentDbName, jobExecutionConfiguration); - } - } - private AbstractJob analyzeAndCreateInsertJob(String sql, String currentDbName, JobExecutionConfiguration jobExecutionConfiguration) throws UserException { NereidsParser parser = new NereidsParser(); @@ -267,6 +312,16 @@ private AbstractJob analyzeAndCreateInsertJob(String sql, String currentDbName, } } + /** + * Analyzes the provided SQL statement and creates an appropriate job based on the parsed logical plan. + * Currently, only "InsertIntoTableCommand" is supported for job creation. + * + * @param sql the SQL statement to be analyzed + * @param currentDbName the current database name where the SQL statement will be executed + * @param jobExecutionConfiguration the configuration for job execution + * @return an instance of AbstractJob corresponding to the SQL statement + * @throws UserException if there is an error during SQL analysis or job creation + */ private AbstractJob analyzeAndCreateStreamingInsertJob(String sql, String currentDbName, JobExecutionConfiguration jobExecutionConfiguration, Map properties) throws UserException { NereidsParser parser = new NereidsParser(); @@ -323,4 +378,20 @@ public static Long stripQuotesAndParseTimestamp(String str) { public boolean streamingJob() { return streamingJob; } + + public String getSourceType() { + return sourceType; + } + + public String getTargetDb() { + return targetDb; + } + + public Map getSourceProperties() { + return sourceProperties; + } + + public Map getTargetProperties() { + return targetProperties; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java index 0cd79e1ba5301c..eff777c98d1157 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceClient.java @@ -209,6 +209,11 @@ public Future abortRefreshDicti return stub.withDeadlineAfter(timeoutSec, TimeUnit.SECONDS).abortRefreshDictionary(request); } + public Future requestCdcClient( + InternalService.PRequestCdcClientRequest request) { + return stub.requestCdcClient(request); + } + public void shutdown() { ConnectivityState state = channel.getState(false); LOG.warn("shut down backend service client: {}, channel state: {}", address, state); diff --git a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java index 7e15a0503935f4..68fa927f0f2fe5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java +++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java @@ -587,4 +587,15 @@ public Future abortDictionaryAs } return null; } + + public Future requestCdcClient(TNetworkAddress address, + InternalService.PRequestCdcClientRequest request) { + try { + final BackendServiceClient client = getProxy(address); + return client.requestCdcClient(request); + } catch (Throwable e) { + LOG.warn("request cdc client failed, address={}:{}", address.getHostname(), address.getPort(), e); + } + return null; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java index fb4f5d6ad17992..c59c31be328c39 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java @@ -67,8 +67,8 @@ import org.apache.doris.datasource.maxcompute.MaxComputeExternalCatalog; import org.apache.doris.datasource.mvcc.MvccUtil; import org.apache.doris.job.common.JobType; +import org.apache.doris.job.extensions.insert.streaming.AbstractStreamingTask; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; -import org.apache.doris.job.extensions.insert.streaming.StreamingInsertTask; import org.apache.doris.job.extensions.mtmv.MTMVJob; import org.apache.doris.job.task.AbstractTask; import org.apache.doris.mtmv.MTMVPartitionUtil; @@ -1231,8 +1231,8 @@ private static TFetchSchemaTableDataResult taskMetadataResult(TMetadataTableRequ if (job instanceof StreamingInsertJob) { StreamingInsertJob streamingJob = (StreamingInsertJob) job; - List streamingInsertTasks = streamingJob.queryAllStreamTasks(); - for (StreamingInsertTask task : streamingInsertTasks) { + List streamingInsertTasks = streamingJob.queryAllStreamTasks(); + for (AbstractStreamingTask task : streamingInsertTasks) { TRow tvfInfo = task.getTvfInfo(job.getJobName()); if (tvfInfo != null) { dataBatch.add(tvfInfo); diff --git a/fs_brokers/cdc_client/build.sh b/fs_brokers/cdc_client/build.sh new file mode 100755 index 00000000000000..52d9683baf0ee3 --- /dev/null +++ b/fs_brokers/cdc_client/build.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +set -eo pipefail + +ROOT="$(cd "$(dirname "${BASH_SOURCE[0]}")" &>/dev/null && pwd)" + +export DORIS_HOME="${ROOT}/../.." +. "${DORIS_HOME}/env.sh" + +export CDC_CLIENT_HOME="${ROOT}" + +cd "${DORIS_HOME}/fe" +"${MVN_CMD}" install -pl fe-common -Dskip.doc=true -DskipTests + + +echo "Install cdc client..." +cd "${CDC_CLIENT_HOME}" +"${MVN_CMD}" package -DskipTests + +echo "Finished" diff --git a/fs_brokers/cdc_client/pom.xml b/fs_brokers/cdc_client/pom.xml new file mode 100644 index 00000000000000..1f5723f9b9cacf --- /dev/null +++ b/fs_brokers/cdc_client/pom.xml @@ -0,0 +1,261 @@ + + + + 4.0.0 + + org.apache + apache + 29 + + cdc_client + 1.0.0 + Doris Cdc Client + https://doris.apache.org/ + + + Apache 2.0 License + https://www.apache.org/licenses/LICENSE-2.0.html + repo + + + + scm:git:https://git@github.com/apache/doris.git + scm:git:https://git@github.com/apache/doris.git + scm:git:https://git@github.com/apache/doris.git + HEAD + + + GitHub + https://github.com/apache/doris/issues + + + + Dev Mailing List + dev@doris.apache.org + dev-subscribe@doris.apache.org + dev-unsubscribe@doris.apache.org + + + Commits Mailing List + commits@doris.apache.org + commits-subscribe@doris.apache.org + commits-unsubscribe@doris.apache.org + + + + 3.5.7 + 1.20.3 + 1.7.36 + 1.2-SNAPSHOT + 2.43.0 + 8.14 + 2.17 + 4.5.13 + 1.18.24 + 1.15 + + + + + org.apache.doris + fe-common + ${doris.version} + + + * + * + + + + + org.springframework.boot + spring-boot-starter-web + ${spring.version} + + + org.springframework.boot + spring-boot-starter-tomcat + + + org.springframework.boot + spring-boot-starter-logging + + + + + org.springframework.boot + spring-boot-starter-jetty + ${spring.version} + + + org.springframework.boot + spring-boot-starter-actuator + ${spring.version} + + + org.springframework.boot + spring-boot-starter-log4j2 + ${spring.version} + + + org.apache.flink + flink-connector-mysql-cdc + 3.5.0 + + + org.apache.flink + flink-clients + ${flink.version} + + + org.apache.flink + flink-connector-base + ${flink.version} + + + + org.apache.flink + flink-table-api-java-bridge + ${flink.version} + + + + org.apache.httpcomponents + httpclient + ${httpcomponents.version} + + + commons-codec + commons-codec + + + + + + commons-codec + commons-codec + ${commons-codec.version} + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + org.slf4j + slf4j-log4j12 + ${slf4j.version} + + + + org.projectlombok + lombok + ${lombok.version} + provided + + + + + + cdc-client + + + org.springframework.boot + spring-boot-maven-plugin + 2.0.3.RELEASE + + + + repackage + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 17 + 17 + + + + com.diffplug.spotless + spotless-maven-plugin + ${spotless.version} + + + + 1.17.0 + + + + + org.apache.doris,org.apache,javax,java,scala,\# + + + + + + + + spotless-check + validate + + check + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + ${maven-checkstyle-plugin.version} + + + com.puppycrawl.tools + checkstyle + + ${checkstyle.version} + + + + + validate + validate + + check + + + + + true + ../../fe/check/checkstyle/suppressions.xml + true + ../../fe/check/checkstyle/checkstyle.xml + true + true + + + + + \ No newline at end of file diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java new file mode 100644 index 00000000000000..0547dea6741246 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/CdcClientApplication.java @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient; + +import java.util.Arrays; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.boot.SpringApplication; +import org.springframework.boot.autoconfigure.SpringBootApplication; +import org.springframework.boot.context.properties.EnableConfigurationProperties; +import org.springframework.boot.web.servlet.ServletComponentScan; +import org.springframework.boot.web.servlet.support.SpringBootServletInitializer; + +@SpringBootApplication +@EnableConfigurationProperties +@ServletComponentScan +public class CdcClientApplication extends SpringBootServletInitializer { + private static final Logger LOG = LoggerFactory.getLogger(CdcClientApplication.class); + + public static void main(String[] args) { + LOG.info("args: {}, log.path: {}", Arrays.asList(args), System.getProperty("log.path")); + SpringApplication.run(CdcClientApplication.class, args); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java new file mode 100644 index 00000000000000..f7a49bdaa08c01 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Constants.java @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.common; + +public class Constants { + public static final String DORIS_DELETE_SIGN = "__DORIS_DELETE_SIGN__"; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java new file mode 100644 index 00000000000000..a37222158650b2 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/common/Env.java @@ -0,0 +1,126 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.common; + +import org.apache.doris.cdcclient.source.factory.DataSource; +import org.apache.doris.cdcclient.source.factory.SourceReaderFactory; +import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.job.cdc.request.JobBaseConfig; + +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +import lombok.Setter; + +public class Env { + private static volatile Env INSTANCE; + private final Map jobContexts; + @Setter private int backendHttpPort; + + private Env() { + this.jobContexts = new ConcurrentHashMap<>(); + } + + public String getBackendHostPort() { + return "127.0.0.1:" + backendHttpPort; + } + + public static Env getCurrentEnv() { + if (INSTANCE == null) { + synchronized (Env.class) { + if (INSTANCE == null) { + INSTANCE = new Env(); + } + } + } + return INSTANCE; + } + + public SourceReader getReader(JobBaseConfig jobConfig) { + DataSource ds = resolveDataSource(jobConfig.getDataSource()); + Env manager = Env.getCurrentEnv(); + return manager.getOrCreateReader(jobConfig.getJobId(), ds, jobConfig.getConfig()); + } + + private DataSource resolveDataSource(String source) { + if (source == null || source.trim().isEmpty()) { + throw new IllegalArgumentException("Missing dataSource"); + } + try { + return DataSource.valueOf(source.trim().toUpperCase(Locale.ROOT)); + } catch (IllegalArgumentException ex) { + throw new IllegalArgumentException("Unsupported dataSource: " + source, ex); + } + } + + private SourceReader getOrCreateReader( + Long jobId, DataSource dataSource, Map config) { + JobContext context = getOrCreateContext(jobId, dataSource, config); + return context.getOrCreateReader(dataSource); + } + + public void close(Long jobId) { + JobContext context = jobContexts.remove(jobId); + if (context != null) { + context.close(); + } + } + + private JobContext getOrCreateContext( + Long jobId, DataSource dataSource, Map config) { + Objects.requireNonNull(jobId, "jobId"); + Objects.requireNonNull(dataSource, "dataSource"); + return jobContexts.computeIfAbsent(jobId, id -> new JobContext(id, dataSource, config)); + } + + private static final class JobContext { + private final long jobId; + private volatile SourceReader reader; + private volatile Map config; + private volatile DataSource dataSource; + + private JobContext(long jobId, DataSource dataSource, Map config) { + this.jobId = jobId; + this.dataSource = dataSource; + this.config = config; + } + + private synchronized SourceReader getOrCreateReader(DataSource source) { + if (reader == null) { + reader = SourceReaderFactory.createSourceReader(source); + reader.initialize(config); + dataSource = source; + } else if (dataSource != source) { + throw new IllegalStateException( + String.format( + "Job %d already bound to datasource %s, cannot switch to %s", + jobId, dataSource, source)); + } + return reader; + } + + private void close() { + if (reader != null) { + reader.close(jobId); + reader = null; + } + } + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java new file mode 100644 index 00000000000000..c19771345305a6 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/BackendPortHolder.java @@ -0,0 +1,36 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.config; + +import org.apache.doris.cdcclient.common.Env; + +import jakarta.annotation.PostConstruct; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.stereotype.Component; + +@Component +public class BackendPortHolder { + + @Value("${backend.http.port}") + private int port; + + @PostConstruct + public void init() { + Env.getCurrentEnv().setBackendHttpPort(port); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java new file mode 100644 index 00000000000000..8b4883b6203183 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java @@ -0,0 +1,38 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.config; + +import org.apache.doris.cdcclient.model.rest.RestResponse; + +import jakarta.servlet.http.HttpServletRequest; +import lombok.extern.slf4j.Slf4j; +import org.springframework.web.bind.annotation.ControllerAdvice; +import org.springframework.web.bind.annotation.ExceptionHandler; +import org.springframework.web.bind.annotation.ResponseBody; + +@Slf4j +@ControllerAdvice +public class GlobalExceptionHandler { + + @ExceptionHandler(Exception.class) + @ResponseBody + public Object exceptionHandler(HttpServletRequest request, Exception e) { + log.error("Unexpected exception", e); + return RestResponse.internalError(e.getMessage()); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java new file mode 100644 index 00000000000000..916b461e72dedf --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/controller/ClientController.java @@ -0,0 +1,106 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.controller; + +import org.apache.doris.cdcclient.common.Env; +import org.apache.doris.cdcclient.model.rest.RestResponse; +import org.apache.doris.cdcclient.service.PipelineCoordinator; +import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.job.cdc.request.CompareOffsetRequest; +import org.apache.doris.job.cdc.request.FetchRecordRequest; +import org.apache.doris.job.cdc.request.FetchTableSplitsRequest; +import org.apache.doris.job.cdc.request.JobBaseConfig; +import org.apache.doris.job.cdc.request.WriteRecordRequest; + +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.web.bind.annotation.PathVariable; +import org.springframework.web.bind.annotation.RequestBody; +import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; +import org.springframework.web.bind.annotation.RestController; + +@RestController +public class ClientController { + private static final Logger LOG = LoggerFactory.getLogger(ClientController.class); + + @Autowired private PipelineCoordinator pipelineCoordinator; + + /** Fetch source splits for snapshot */ + @RequestMapping(path = "/api/fetchSplits", method = RequestMethod.POST) + public Object fetchSplits(@RequestBody FetchTableSplitsRequest ftsReq) { + try { + SourceReader reader = Env.getCurrentEnv().getReader(ftsReq); + List splits = reader.getSourceSplits(ftsReq); + return RestResponse.success(splits); + } catch (IllegalArgumentException ex) { + LOG.error("Failed to fetch splits, jobId={}", ftsReq.getJobId(), ex); + return RestResponse.internalError(ex.getMessage()); + } + } + + /** Fetch records from source reader */ + @RequestMapping(path = "/api/fetchRecords", method = RequestMethod.POST) + public Object fetchRecords(@RequestBody FetchRecordRequest recordReq) { + try { + SourceReader reader = Env.getCurrentEnv().getReader(recordReq); + return RestResponse.success(reader.read(recordReq)); + } catch (Exception ex) { + LOG.error("Failed fetch record, jobId={}", recordReq.getJobId(), ex); + return RestResponse.internalError(ex.getMessage()); + } + } + + /** Fetch records from source reader and Write records to backend */ + @RequestMapping(path = "/api/writeRecords", method = RequestMethod.POST) + public Object writeRecord(@RequestBody WriteRecordRequest recordReq) { + LOG.info( + "Received write record request for jobId={}, taskId={}, meta={}", + recordReq.getJobId(), + recordReq.getTaskId(), + recordReq.getMeta()); + pipelineCoordinator.writeRecordsAsync(recordReq); + return RestResponse.success("Request accepted, processing asynchronously"); + } + + /** Fetch lastest end meta */ + @RequestMapping(path = "/api/fetchEndOffset", method = RequestMethod.POST) + public Object fetchEndOffset(@RequestBody JobBaseConfig jobConfig) { + SourceReader reader = Env.getCurrentEnv().getReader(jobConfig); + return RestResponse.success(reader.getEndOffset(jobConfig)); + } + + /** compare datasource Binlog Offset */ + @RequestMapping(path = "/api/compareOffset", method = RequestMethod.POST) + public Object compareOffset(@RequestBody CompareOffsetRequest compareOffsetRequest) { + SourceReader reader = Env.getCurrentEnv().getReader(compareOffsetRequest); + return RestResponse.success(reader.compareOffset(compareOffsetRequest)); + } + + /** Close job */ + @RequestMapping(path = "/api/close/{jobId}", method = RequestMethod.POST) + public Object close(@PathVariable long jobId) { + Env env = Env.getCurrentEnv(); + env.close(jobId); + pipelineCoordinator.closeJob(jobId); + return RestResponse.success(true); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/CdcClientException.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/CdcClientException.java new file mode 100644 index 00000000000000..574cf4c73b7bcc --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/CdcClientException.java @@ -0,0 +1,29 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.exception; + +public class CdcClientException extends RuntimeException { + + public CdcClientException(String message) { + super(message); + } + + public CdcClientException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/StreamLoadException.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/StreamLoadException.java new file mode 100644 index 00000000000000..4c344e19799dab --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/exception/StreamLoadException.java @@ -0,0 +1,44 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.exception; + +public class StreamLoadException extends RuntimeException { + public StreamLoadException() { + super(); + } + + public StreamLoadException(String message) { + super(message); + } + + public StreamLoadException(String message, Throwable cause) { + super(message, cause); + } + + public StreamLoadException(Throwable cause) { + super(cause); + } + + protected StreamLoadException( + String message, + Throwable cause, + boolean enableSuppression, + boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java new file mode 100644 index 00000000000000..4c6d0e4ee7fe00 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/response/RecordWithMeta.java @@ -0,0 +1,36 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model.response; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import lombok.Data; + +@Data +public class RecordWithMeta { + private Map meta; + private List records; + + public RecordWithMeta() { + this.meta = new HashMap<>(); + this.records = new ArrayList<>(); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestResponse.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestResponse.java new file mode 100644 index 00000000000000..5126f138c2db48 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/model/rest/RestResponse.java @@ -0,0 +1,54 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.model.rest; + +import java.io.Serializable; + +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@NoArgsConstructor +public class RestResponse implements Serializable { + private static final long serialVersionUID = 1L; + public static final int SUCCESS = 0; + public static final int FAIL = 1; + + private static final String DEFAULT_SUCCESS_MESSAGE = "Success"; + private static final String DEFAULT_SYSTEM_ERROR_MESSAGE = "Internal Error"; + + private String msg; + private int code; + private T data; + + public static RestResponse success(T data) { + RestResponse response = new RestResponse<>(); + response.setCode(SUCCESS); + response.setMsg(DEFAULT_SUCCESS_MESSAGE); + response.setData(data); + return response; + } + + public static RestResponse internalError(T data) { + RestResponse response = new RestResponse<>(); + response.setCode(FAIL); + response.setMsg(DEFAULT_SYSTEM_ERROR_MESSAGE); + response.setData(data); + return response; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java new file mode 100644 index 00000000000000..40d66e7fa1e5e9 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java @@ -0,0 +1,224 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.service; + +import org.apache.doris.cdcclient.common.Env; +import org.apache.doris.cdcclient.exception.StreamLoadException; +import org.apache.doris.cdcclient.sink.DorisBatchStreamLoad; +import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.cdcclient.source.reader.SplitReadResult; +import org.apache.doris.job.cdc.request.WriteRecordRequest; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import com.google.common.base.Preconditions; +import io.debezium.data.Envelope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Component; + +/** Pipeline coordinator. */ +@Component +public class PipelineCoordinator { + private static final Logger LOG = LoggerFactory.getLogger(PipelineCoordinator.class); + private static final String SPLIT_ID = "splitId"; + // jobId + private final Map batchStreamLoadMap = new ConcurrentHashMap<>(); + private final ThreadPoolExecutor executor; + private static final int MAX_CONCURRENT_TASKS = 10; + private static final int QUEUE_CAPACITY = 128; + + public PipelineCoordinator() { + this.executor = + new ThreadPoolExecutor( + MAX_CONCURRENT_TASKS, + MAX_CONCURRENT_TASKS, + 60L, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(QUEUE_CAPACITY), + r -> { + Thread t = + new Thread( + r, "async-write-record-" + System.currentTimeMillis()); + t.setDaemon(false); + return t; + }, + new ThreadPoolExecutor.AbortPolicy()); + } + + public CompletableFuture writeRecordsAsync(WriteRecordRequest writeRecordRequest) { + Preconditions.checkNotNull(writeRecordRequest.getToken(), "token must not be null"); + Preconditions.checkNotNull(writeRecordRequest.getTaskId(), "taskId must not be null"); + Preconditions.checkNotNull(writeRecordRequest.getTargetDb(), "targetDb must not be null"); + return CompletableFuture.runAsync( + () -> { + try { + LOG.info( + "Start processing async write record, jobId={} taskId={}", + writeRecordRequest.getJobId(), + writeRecordRequest.getTaskId()); + writeRecords(writeRecordRequest); + LOG.info( + "Successfully processed async write record, jobId={} taskId={}", + writeRecordRequest.getJobId(), + writeRecordRequest.getTaskId()); + } catch (Exception ex) { + closeJob(writeRecordRequest.getJobId()); + LOG.error( + "Failed to process async write record, jobId={} taskId={}", + writeRecordRequest.getJobId(), + writeRecordRequest.getTaskId(), + ex); + } + }, + executor); + } + + /** Read data from SourceReader and write it to Doris, while returning meta information. */ + public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception { + SourceReader sourceReader = Env.getCurrentEnv().getReader(writeRecordRequest); + DorisBatchStreamLoad batchStreamLoad = null; + Map metaResponse = new HashMap<>(); + boolean hasData = false; + long scannedRows = 0L; + long scannedBytes = 0L; + SplitReadResult readResult = null; + try { + readResult = sourceReader.readSplitRecords(writeRecordRequest); + batchStreamLoad = + getOrCreateBatchStreamLoad( + writeRecordRequest.getJobId(), writeRecordRequest.getTargetDb()); + batchStreamLoad.setCurrentTaskId(writeRecordRequest.getTaskId()); + batchStreamLoad.setFrontendAddress(writeRecordRequest.getFrontendAddress()); + batchStreamLoad.setToken(writeRecordRequest.getToken()); + + // Record start time for maxInterval check + long startTime = System.currentTimeMillis(); + long maxIntervalMillis = writeRecordRequest.getMaxInterval() * 1000; + + // Use iterators to read and write. + Iterator iterator = readResult.getRecordIterator(); + while (iterator != null && iterator.hasNext()) { + SourceRecord element = iterator.next(); + List serializedRecords = + sourceReader.deserialize(writeRecordRequest.getConfig(), element); + + if (!CollectionUtils.isEmpty(serializedRecords)) { + String database = writeRecordRequest.getTargetDb(); + String table = extractTable(element); + hasData = true; + for (String record : serializedRecords) { + scannedRows++; + byte[] dataBytes = record.getBytes(); + scannedBytes += dataBytes.length; + batchStreamLoad.writeRecord(database, table, dataBytes); + } + + Map lastMeta = + RecordUtils.getBinlogPosition(element).getOffset(); + if (sourceReader.isBinlogSplit(readResult.getSplit()) + && readResult.getSplit() != null) { + lastMeta.put(SPLIT_ID, readResult.getSplit().splitId()); + } + metaResponse = lastMeta; + } + // Check if maxInterval has been exceeded + long elapsedTime = System.currentTimeMillis() - startTime; + if (maxIntervalMillis > 0 && elapsedTime >= maxIntervalMillis) { + LOG.info( + "Max interval {} seconds reached, stopping data reading", + writeRecordRequest.getMaxInterval()); + break; + } + } + } finally { + sourceReader.finishSplitRecords(); + } + + try { + if (!hasData) { + // todo: need return the lastest heartbeat offset, means the maximum offset that the + // current job can recover. + if (sourceReader.isBinlogSplit(readResult.getSplit())) { + Map offsetRes = + sourceReader.extractBinlogOffset(readResult.getSplit()); + batchStreamLoad.commitOffset(offsetRes, scannedRows, scannedBytes); + return; + } else { + throw new RuntimeException("should not happen"); + } + } + + // wait all stream load finish + batchStreamLoad.forceFlush(); + // update offset meta + if (!sourceReader.isBinlogSplit(readResult.getSplit())) { + Map offsetRes = + sourceReader.extractSnapshotOffset( + readResult.getSplit(), readResult.getSplitState()); + if (offsetRes == null) { + // should not happen + throw new StreamLoadException( + "Chunk data cannot be obtained from highWatermark."); + } + metaResponse = offsetRes; + } + // request fe api + batchStreamLoad.commitOffset(metaResponse, scannedRows, scannedBytes); + + } finally { + batchStreamLoad.resetTaskId(); + } + } + + private DorisBatchStreamLoad getOrCreateBatchStreamLoad(Long jobId, String targetDb) { + return batchStreamLoadMap.computeIfAbsent( + jobId, + k -> { + LOG.info("Create DorisBatchStreamLoad for jobId={}", jobId); + return new DorisBatchStreamLoad(jobId, targetDb); + }); + } + + public void closeJob(Long jobId) { + DorisBatchStreamLoad batchStreamLoad = batchStreamLoadMap.remove(jobId); + if (batchStreamLoad != null) { + LOG.info("Close DorisBatchStreamLoad for jobId={}", jobId); + batchStreamLoad.close(); + batchStreamLoad = null; + } + } + + private String extractTable(SourceRecord record) { + Struct value = (Struct) record.value(); + return value.getStruct(Envelope.FieldName.SOURCE).getString("table"); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferHttpEntity.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferHttpEntity.java new file mode 100644 index 00000000000000..04c7a058727f6c --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferHttpEntity.java @@ -0,0 +1,77 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import org.apache.http.entity.AbstractHttpEntity; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BatchBufferHttpEntity extends AbstractHttpEntity { + + private static final Logger LOG = LoggerFactory.getLogger(BatchBufferHttpEntity.class); + protected static final int OUTPUT_BUFFER_SIZE = 4096; + private final List buffer; + private final long contentLength; + + public BatchBufferHttpEntity(BatchRecordBuffer recordBuffer) { + this.buffer = recordBuffer.getBuffer(); + this.contentLength = recordBuffer.getBufferSizeBytes(); + } + + @Override + public boolean isRepeatable() { + return true; + } + + @Override + public boolean isChunked() { + return false; + } + + @Override + public long getContentLength() { + return contentLength; + } + + @Override + public InputStream getContent() { + return new BatchBufferStream(buffer); + } + + @Override + public void writeTo(OutputStream outStream) throws IOException { + try (InputStream inStream = new BatchBufferStream(buffer)) { + final byte[] buffer = new byte[OUTPUT_BUFFER_SIZE]; + int readLen; + while ((readLen = inStream.read(buffer)) != -1) { + outStream.write(buffer, 0, readLen); + } + } + } + + @Override + public boolean isStreaming() { + return false; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferStream.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferStream.java new file mode 100644 index 00000000000000..bf83d44a846de3 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchBufferStream.java @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Iterator; +import java.util.List; + +public class BatchBufferStream extends InputStream { + private final Iterator iterator; + private byte[] currentRow; + private int currentPos; + + public BatchBufferStream(List buffer) { + this.iterator = buffer.iterator(); + } + + @Override + public int read() throws IOException { + return 0; + } + + @Override + public int read(byte[] buf) throws IOException { + return read(buf, 0, buf.length); + } + + @Override + public int read(byte[] buf, int off, int len) throws IOException { + if (!iterator.hasNext() && currentRow == null) { + return -1; + } + + byte[] item = currentRow; + int pos = currentPos; + int readBytes = 0; + while (readBytes < len && (item != null || iterator.hasNext())) { + if (item == null) { + item = iterator.next(); + pos = 0; + } + + int size = Math.min(len - readBytes, item.length - pos); + System.arraycopy(item, pos, buf, off + readBytes, size); + readBytes += size; + pos += size; + + if (pos == item.length) { + item = null; + pos = 0; + } + } + currentRow = item; + currentPos = pos; + return readBytes; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java new file mode 100644 index 00000000000000..a81e91a07ee418 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/BatchRecordBuffer.java @@ -0,0 +1,146 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import java.util.LinkedList; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** buffer to queue. */ +public class BatchRecordBuffer { + private static final Logger LOG = LoggerFactory.getLogger(BatchRecordBuffer.class); + public static final String LINE_SEPARATOR = "\n"; + private String labelName; + private LinkedList buffer; + private byte[] lineDelimiter; + private int numOfRecords = 0; + private long bufferSizeBytes = 0; + private boolean loadBatchFirstRecord = true; + private String database; + private String table; + + public BatchRecordBuffer() { + this.buffer = new LinkedList<>(); + } + + public BatchRecordBuffer(String database, String table, byte[] lineDelimiter) { + super(); + this.database = database; + this.table = table; + this.lineDelimiter = lineDelimiter; + this.buffer = new LinkedList<>(); + } + + public int insert(byte[] record) { + int recordSize = record.length; + if (loadBatchFirstRecord) { + loadBatchFirstRecord = false; + } else if (lineDelimiter != null) { + this.buffer.add(this.lineDelimiter); + setBufferSizeBytes(this.bufferSizeBytes + this.lineDelimiter.length); + recordSize += this.lineDelimiter.length; + } + this.buffer.add(record); + setNumOfRecords(this.numOfRecords + 1); + setBufferSizeBytes(this.bufferSizeBytes + record.length); + return recordSize; + } + + public String getLabelName() { + return labelName; + } + + public void setLabelName(String labelName) { + this.labelName = labelName; + } + + /** + * @return true if buffer is empty + */ + public boolean isEmpty() { + return numOfRecords == 0; + } + + public void clear() { + this.buffer.clear(); + this.numOfRecords = 0; + this.bufferSizeBytes = 0; + this.labelName = null; + this.loadBatchFirstRecord = true; + } + + public LinkedList getBuffer() { + return buffer; + } + + /** + * @return Number of records in this buffer + */ + public int getNumOfRecords() { + return numOfRecords; + } + + /** + * @return Buffer size in bytes + */ + public long getBufferSizeBytes() { + return bufferSizeBytes; + } + + /** + * @param numOfRecords Updates number of records (Usually by 1) + */ + public void setNumOfRecords(int numOfRecords) { + this.numOfRecords = numOfRecords; + } + + /** + * @param bufferSizeBytes Updates sum of size of records present in this buffer (Bytes) + */ + public void setBufferSizeBytes(long bufferSizeBytes) { + this.bufferSizeBytes = bufferSizeBytes; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getTable() { + return table; + } + + public void setTable(String table) { + this.table = table; + } + + public String getTableIdentifier() { + if (database != null && table != null) { + return database + "." + table; + } + return null; + } + + public byte[] getLineDelimiter() { + return lineDelimiter; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java new file mode 100644 index 00000000000000..ed100dde3830bf --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/DorisBatchStreamLoad.java @@ -0,0 +1,543 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import org.apache.doris.cdcclient.common.Env; +import org.apache.doris.cdcclient.exception.StreamLoadException; +import org.apache.doris.cdcclient.utils.HttpUtil; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.entity.StringEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.util.EntityUtils; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.Setter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** async stream load. */ +public class DorisBatchStreamLoad implements Serializable { + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(DorisBatchStreamLoad.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final List DORIS_SUCCESS_STATUS = + new ArrayList<>(Arrays.asList("Success")); + private final int FLUSH_QUEUE_SIZE = 1; + private final long STREAM_LOAD_MAX_BYTES = 500 * 1024 * 1024L; // 500MB + private final int RETRY = 3; + private final byte[] lineDelimiter = "\n".getBytes(); + private static final String LOAD_URL_PATTERN = "http://%s/api/%s/%s/_stream_load"; + private static final String COMMIT_URL_PATTERN = "http://%s/api/streaming/commit_offset"; + private String hostPort; + @Setter private String frontendAddress; + private Map bufferMap = new ConcurrentHashMap<>(); + private ExecutorService loadExecutorService; + private LoadAsyncExecutor loadAsyncExecutor; + private BlockingQueue flushQueue; + private final AtomicBoolean started; + private volatile boolean loadThreadAlive = false; + private AtomicReference exception = new AtomicReference<>(null); + private long maxBlockedBytes; + private final AtomicLong currentCacheBytes = new AtomicLong(0L); + private final Lock lock = new ReentrantLock(); + private final Condition block = lock.newCondition(); + private final Map bufferMapLock = new ConcurrentHashMap<>(); + @Setter private String currentTaskId; + private String targetDb; + private long jobId; + @Setter private String token; + + public DorisBatchStreamLoad(long jobId, String targetDb) { + this.hostPort = Env.getCurrentEnv().getBackendHostPort(); + this.flushQueue = new LinkedBlockingDeque<>(1); + // maxBlockedBytes is two times of FLUSH_MAX_BYTE_SIZE + this.maxBlockedBytes = STREAM_LOAD_MAX_BYTES * 2; + this.loadAsyncExecutor = new LoadAsyncExecutor(FLUSH_QUEUE_SIZE, jobId); + this.loadExecutorService = + new ThreadPoolExecutor( + 1, + 1, + 0L, + TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(1), + new DefaultThreadFactory("stream-load-executor-" + jobId), + new ThreadPoolExecutor.AbortPolicy()); + this.started = new AtomicBoolean(true); + this.loadExecutorService.execute(loadAsyncExecutor); + this.targetDb = targetDb; + this.jobId = jobId; + } + + /** + * write record into cache. + * + * @param record + * @throws IOException + */ + public void writeRecord(String database, String table, byte[] record) { + checkFlushException(); + String bufferKey = getTableIdentifier(database, table); + + getLock(bufferKey).readLock().lock(); + BatchRecordBuffer buffer = + bufferMap.computeIfAbsent( + bufferKey, k -> new BatchRecordBuffer(database, table, this.lineDelimiter)); + + int bytes = buffer.insert(record); + currentCacheBytes.addAndGet(bytes); + getLock(bufferKey).readLock().unlock(); + + if (currentCacheBytes.get() > maxBlockedBytes) { + cacheFullFlush(); + lock.lock(); + try { + while (currentCacheBytes.get() >= maxBlockedBytes) { + checkFlushException(); + LOG.info( + "Cache full, waiting for flush, currentBytes: {}, maxBlockedBytes: {}", + currentCacheBytes.get(), + maxBlockedBytes); + block.await(1, TimeUnit.SECONDS); + } + } catch (InterruptedException e) { + this.exception.set(e); + throw new RuntimeException(e); + } finally { + lock.unlock(); + } + } + } + + public boolean cacheFullFlush() { + return doFlush(true, true); + } + + public boolean forceFlush() { + return doFlush(true, false); + } + + private synchronized boolean doFlush(boolean waitUtilDone, boolean cacheFull) { + checkFlushException(); + if (waitUtilDone || cacheFull) { + return flush(waitUtilDone); + } + return false; + } + + private synchronized boolean flush(boolean waitUtilDone) { + if (!waitUtilDone && bufferMap.isEmpty()) { + // bufferMap may have been flushed by other threads + LOG.info("bufferMap is empty, no need to flush"); + return false; + } + for (String key : bufferMap.keySet()) { + if (waitUtilDone) { + // Ensure that the interval satisfies intervalMS + flushBuffer(key); + } + } + if (!waitUtilDone) { + return false; + } else { + waitAsyncLoadFinish(); + } + return true; + } + + private synchronized void flushBuffer(String bufferKey) { + BatchRecordBuffer buffer; + try { + getLock(bufferKey).writeLock().lock(); + buffer = bufferMap.remove(bufferKey); + } finally { + getLock(bufferKey).writeLock().unlock(); + } + if (buffer == null) { + LOG.info("buffer key is not exist {}, skipped", bufferKey); + return; + } + buffer.setLabelName(UUID.randomUUID().toString().replace("-", "")); + LOG.debug("flush buffer for key {} with label {}", bufferKey, buffer.getLabelName()); + putRecordToFlushQueue(buffer); + } + + private void putRecordToFlushQueue(BatchRecordBuffer buffer) { + checkFlushException(); + if (!loadThreadAlive) { + throw new RuntimeException("load thread already exit, write was interrupted"); + } + try { + flushQueue.put(buffer); + } catch (InterruptedException e) { + throw new RuntimeException("Failed to put record buffer to flush queue"); + } + // When the load thread reports an error, the flushQueue will be cleared, + // and need to force a check for the exception. + checkFlushException(); + } + + private void checkFlushException() { + if (exception.get() != null) { + throw new StreamLoadException(exception.get()); + } + } + + private void waitAsyncLoadFinish() { + // Because the queue will have a drainTo operation, it needs to be multiplied by 2 + for (int i = 0; i < FLUSH_QUEUE_SIZE * 2 + 1; i++) { + // eof buffer + BatchRecordBuffer empty = new BatchRecordBuffer(); + putRecordToFlushQueue(empty); + } + } + + private String getTableIdentifier(String database, String table) { + return database + "." + table; + } + + public void close() { + // close async executor + this.loadExecutorService.shutdown(); + this.started.set(false); + } + + @VisibleForTesting + public boolean mergeBuffer(List recordList, BatchRecordBuffer buffer) { + boolean merge = false; + if (recordList.size() > 1) { + boolean sameTable = + recordList.stream() + .map(BatchRecordBuffer::getTableIdentifier) + .distinct() + .count() + == 1; + // Buffers can be merged only if they belong to the same table. + if (sameTable) { + for (BatchRecordBuffer recordBuffer : recordList) { + if (recordBuffer != null + && recordBuffer.getLabelName() != null + && !buffer.getLabelName().equals(recordBuffer.getLabelName()) + && !recordBuffer.getBuffer().isEmpty()) { + merge(buffer, recordBuffer); + merge = true; + } + } + LOG.info( + "merge {} buffer to one stream load, result bufferBytes {}", + recordList.size(), + buffer.getBufferSizeBytes()); + } + } + return merge; + } + + private boolean merge(BatchRecordBuffer mergeBuffer, BatchRecordBuffer buffer) { + if (buffer.getBuffer().isEmpty()) { + return false; + } + if (!mergeBuffer.getBuffer().isEmpty()) { + mergeBuffer.getBuffer().add(mergeBuffer.getLineDelimiter()); + mergeBuffer.setBufferSizeBytes( + mergeBuffer.getBufferSizeBytes() + mergeBuffer.getLineDelimiter().length); + currentCacheBytes.addAndGet(buffer.getLineDelimiter().length); + } + mergeBuffer.getBuffer().addAll(buffer.getBuffer()); + mergeBuffer.setNumOfRecords(mergeBuffer.getNumOfRecords() + buffer.getNumOfRecords()); + mergeBuffer.setBufferSizeBytes( + mergeBuffer.getBufferSizeBytes() + buffer.getBufferSizeBytes()); + return true; + } + + private ReadWriteLock getLock(String bufferKey) { + return bufferMapLock.computeIfAbsent(bufferKey, k -> new ReentrantReadWriteLock()); + } + + class LoadAsyncExecutor implements Runnable { + + private int flushQueueSize; + private long jobId; + + public LoadAsyncExecutor(int flushQueueSize, long jobId) { + this.flushQueueSize = flushQueueSize; + this.jobId = jobId; + } + + @Override + public void run() { + LOG.info("LoadAsyncExecutor start for jobId {}", jobId); + loadThreadAlive = true; + List recordList = new ArrayList<>(flushQueueSize); + while (started.get()) { + recordList.clear(); + try { + BatchRecordBuffer buffer = flushQueue.poll(2000L, TimeUnit.MILLISECONDS); + if (buffer == null) { + continue; + } + if (buffer.getLabelName() == null) { + // When the label is empty, it is the eof buffer for checkpoint flush. + continue; + } + + recordList.add(buffer); + boolean merge = false; + if (!flushQueue.isEmpty()) { + flushQueue.drainTo(recordList, flushQueueSize - 1); + if (mergeBuffer(recordList, buffer)) { + load(buffer.getLabelName(), buffer); + merge = true; + } + } + + if (!merge) { + for (BatchRecordBuffer bf : recordList) { + if (bf == null || bf.getLabelName() == null) { + // When the label is empty, it's eof buffer for checkpointFlush. + continue; + } + load(bf.getLabelName(), bf); + } + } + } catch (Exception e) { + LOG.error("worker running error", e); + exception.set(e); + // clear queue to avoid writer thread blocking + flushQueue.clear(); + break; + } + } + LOG.info("LoadAsyncExecutor stop for jobId {}", jobId); + loadThreadAlive = false; + } + + /** execute stream load. */ + public void load(String label, BatchRecordBuffer buffer) throws IOException { + BatchBufferHttpEntity entity = new BatchBufferHttpEntity(buffer); + HttpPutBuilder putBuilder = new HttpPutBuilder(); + + String loadUrl = String.format(LOAD_URL_PATTERN, hostPort, targetDb, buffer.getTable()); + String finalLabel = String.format("%s_%s_%s", jobId, currentTaskId, label); + putBuilder + .setUrl(loadUrl) + .addTokenAuth(token) + .setLabel(finalLabel) + .formatJson() + .addCommonHeader() + .setEntity(entity) + .addHiddenColumns(true) + .setEntity(entity); + + Throwable resEx = new Throwable(); + int retry = 0; + while (retry <= RETRY) { + LOG.info("stream load started for {} on host {}", putBuilder.getLabel(), hostPort); + try (CloseableHttpClient httpClient = HttpUtil.getHttpClient()) { + try (CloseableHttpResponse response = httpClient.execute(putBuilder.build())) { + int statusCode = response.getStatusLine().getStatusCode(); + String reason = response.getStatusLine().toString(); + if (statusCode == 200 && response.getEntity() != null) { + String loadResult = EntityUtils.toString(response.getEntity()); + LOG.info("load Result {}", loadResult); + RespContent respContent = + OBJECT_MAPPER.readValue(loadResult, RespContent.class); + if (DORIS_SUCCESS_STATUS.contains(respContent.getStatus())) { + long cacheByteBeforeFlush = + currentCacheBytes.getAndAdd(-respContent.getLoadBytes()); + LOG.info( + "load success, cacheBeforeFlushBytes: {}, currentCacheBytes : {}", + cacheByteBeforeFlush, + currentCacheBytes.get()); + lock.lock(); + try { + block.signal(); + } finally { + lock.unlock(); + } + return; + } else { + String errMsg = null; + if (StringUtils.isBlank(respContent.getMessage()) + && StringUtils.isBlank(respContent.getErrorURL())) { + // sometimes stream load will not return message + errMsg = + String.format( + "stream load error, response is %s", + loadResult); + throw new StreamLoadException(errMsg); + } else { + errMsg = + String.format( + "stream load error: %s, see more in %s", + respContent.getMessage(), + respContent.getErrorURL()); + } + throw new StreamLoadException(errMsg); + } + } + LOG.error( + "stream load failed with {}, reason {}, to retry", + hostPort, + reason); + if (retry == RETRY) { + resEx = new StreamLoadException("stream load failed with: " + reason); + } + } catch (Exception ex) { + resEx = ex; + LOG.error("stream load error with {}, to retry, cause by", hostPort, ex); + } + } + retry++; + try { + Thread.sleep(retry * 1000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + buffer.clear(); + buffer = null; + + if (retry >= RETRY) { + throw new StreamLoadException("stream load error: " + resEx.getMessage(), resEx); + } + } + } + + static class DefaultThreadFactory implements ThreadFactory { + private static final AtomicInteger poolNumber = new AtomicInteger(1); + private final AtomicInteger threadNumber = new AtomicInteger(1); + private final String namePrefix; + + DefaultThreadFactory(String name) { + namePrefix = "pool-" + poolNumber.getAndIncrement() + "-" + name + "-"; + } + + public Thread newThread(Runnable r) { + Thread t = new Thread(r, namePrefix + threadNumber.getAndIncrement()); + t.setDaemon(false); + return t; + } + } + + public void resetTaskId() { + this.currentTaskId = null; + } + + /** commit offfset to frontends. */ + public void commitOffset(Map meta, long scannedRows, long scannedBytes) { + try { + String url = String.format(COMMIT_URL_PATTERN, frontendAddress, targetDb); + Map commitParams = new HashMap<>(); + commitParams.put("offset", OBJECT_MAPPER.writeValueAsString(meta)); + commitParams.put("jobId", jobId); + commitParams.put("taskId", currentTaskId); + commitParams.put("scannedRows", scannedRows); + commitParams.put("scannedBytes", scannedBytes); + String param = OBJECT_MAPPER.writeValueAsString(commitParams); + + HttpPutBuilder builder = + new HttpPutBuilder() + .addCommonHeader() + .addBodyContentType() + .addTokenAuth(token) + .setUrl(url) + .commit() + .setEntity(new StringEntity(param)); + + LOG.info( + "commit offset for jobId {} taskId {}, params {}", jobId, currentTaskId, param); + Throwable resEx = null; + int retry = 0; + while (retry <= RETRY) { + try (CloseableHttpClient httpClient = HttpUtil.getHttpClient()) { + try (CloseableHttpResponse httpResponse = httpClient.execute(builder.build())) { + int statusCode = httpResponse.getStatusLine().getStatusCode(); + String reason = httpResponse.getStatusLine().toString(); + String responseBody = + httpResponse.getEntity() != null + ? EntityUtils.toString(httpResponse.getEntity()) + : ""; + LOG.info("commit result {}", responseBody); + if (statusCode == 200) { + LOG.info("commit offset for jobId {} taskId {}", jobId, currentTaskId); + return; + } + LOG.error( + "commit offset failed with {}, reason {}, to retry", + hostPort, + reason); + if (retry == RETRY) { + resEx = new StreamLoadException("commit offset failed with: " + reason); + } + } catch (Exception ex) { + resEx = ex; + LOG.error("commit offset error with {}, to retry, cause by", hostPort, ex); + } + } + retry++; + if (retry <= RETRY) { + try { + Thread.sleep(retry * 1000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + } + + if (retry > RETRY) { + throw new StreamLoadException( + "commit offset error: " + + (resEx != null ? resEx.getMessage() : "unknown error"), + resEx); + } + } catch (Exception ex) { + LOG.error("Failed to commit offset, jobId={}", jobId, ex); + throw new StreamLoadException("Failed to commit offset", ex); + } + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java new file mode 100644 index 00000000000000..c1594e04ffc30e --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/HttpPutBuilder.java @@ -0,0 +1,135 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import org.apache.doris.cdcclient.common.Constants; + +import org.apache.flink.util.Preconditions; +import org.apache.http.HttpEntity; +import org.apache.http.HttpHeaders; +import org.apache.http.client.methods.HttpPut; +import org.apache.http.entity.StringEntity; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** Builder for HttpPut. */ +public class HttpPutBuilder { + String url; + Map header; + HttpEntity httpEntity; + + public HttpPutBuilder() { + header = new HashMap<>(); + } + + public HttpPutBuilder setUrl(String url) { + this.url = url; + return this; + } + + public HttpPutBuilder addCommonHeader() { + header.put(HttpHeaders.EXPECT, "100-continue"); + return this; + } + + public HttpPutBuilder addBodyContentType() { + header.put(HttpHeaders.CONTENT_TYPE, "application/json;charset=UTF-8"); + return this; + } + + public HttpPutBuilder addHiddenColumns(boolean add) { + if (add) { + header.put("hidden_columns", Constants.DORIS_DELETE_SIGN); + } + return this; + } + + public HttpPutBuilder enable2PC() { + header.put("two_phase_commit", "true"); + return this; + } + + public HttpPutBuilder addTokenAuth(String token) { + header.put(HttpHeaders.AUTHORIZATION, "Basic YWRtaW46"); + header.put("token", token); + return this; + } + + public HttpPutBuilder addTxnId(long txnID) { + header.put("txn_id", String.valueOf(txnID)); + return this; + } + + public HttpPutBuilder formatJson() { + header.put("read_json_by_line", "true"); + header.put("format", "json"); + return this; + } + + public HttpPutBuilder commit() { + header.put("txn_operation", "commit"); + return this; + } + + public HttpPutBuilder abort() { + header.put("txn_operation", "abort"); + return this; + } + + public HttpPutBuilder setEntity(HttpEntity httpEntity) { + this.httpEntity = httpEntity; + return this; + } + + public HttpPutBuilder setEmptyEntity() { + try { + this.httpEntity = new StringEntity(""); + } catch (Exception e) { + throw new IllegalArgumentException(e); + } + return this; + } + + public HttpPutBuilder addProperties(Properties properties) { + // TODO: check duplicate key. + properties.forEach((key, value) -> header.put(String.valueOf(key), String.valueOf(value))); + return this; + } + + public HttpPutBuilder setLabel(String label) { + if (label != null) { + header.put("label", label); + } + return this; + } + + public String getLabel() { + return header.get("label"); + } + + public HttpPut build() { + Preconditions.checkNotNull(url); + Preconditions.checkNotNull(httpEntity); + HttpPut put = new HttpPut(url); + header.forEach(put::setHeader); + put.setEntity(httpEntity); + return put; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/RespContent.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/RespContent.java new file mode 100644 index 00000000000000..35327aa4553cef --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/sink/RespContent.java @@ -0,0 +1,170 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.sink; + +import org.apache.flink.annotation.VisibleForTesting; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class RespContent { + + @JsonProperty(value = "TxnId") + private Long txnId; + + @JsonProperty(value = "Label") + private String label; + + @JsonProperty(value = "Status") + private String status; + + @JsonProperty(value = "TwoPhaseCommit") + private String twoPhaseCommit; + + @JsonProperty(value = "ExistingJobStatus") + private String existingJobStatus; + + @JsonProperty(value = "Message") + private String message; + + @JsonProperty(value = "NumberTotalRows") + private Long numberTotalRows; + + @JsonProperty(value = "NumberLoadedRows") + private Long numberLoadedRows; + + @JsonProperty(value = "NumberFilteredRows") + private Integer numberFilteredRows; + + @JsonProperty(value = "NumberUnselectedRows") + private Integer numberUnselectedRows; + + @JsonProperty(value = "LoadBytes") + private Long loadBytes; + + @JsonProperty(value = "LoadTimeMs") + private Integer loadTimeMs; + + @JsonProperty(value = "BeginTxnTimeMs") + private Integer beginTxnTimeMs; + + @JsonProperty(value = "StreamLoadPutTimeMs") + private Integer streamLoadPutTimeMs; + + @JsonProperty(value = "ReadDataTimeMs") + private Integer readDataTimeMs; + + @JsonProperty(value = "WriteDataTimeMs") + private Integer writeDataTimeMs; + + @JsonProperty(value = "CommitAndPublishTimeMs") + private Integer commitAndPublishTimeMs; + + @JsonProperty(value = "ErrorURL") + private String errorURL; + + public Long getTxnId() { + return txnId; + } + + public String getStatus() { + return status; + } + + public String getTwoPhaseCommit() { + return twoPhaseCommit; + } + + public String getMessage() { + return message; + } + + public String getExistingJobStatus() { + return existingJobStatus; + } + + public Long getNumberTotalRows() { + return numberTotalRows; + } + + public Long getNumberLoadedRows() { + return numberLoadedRows; + } + + public Integer getNumberFilteredRows() { + return numberFilteredRows; + } + + public Integer getNumberUnselectedRows() { + return numberUnselectedRows; + } + + public Long getLoadBytes() { + return loadBytes; + } + + public Integer getLoadTimeMs() { + return loadTimeMs; + } + + public Integer getBeginTxnTimeMs() { + return beginTxnTimeMs; + } + + public Integer getStreamLoadPutTimeMs() { + return streamLoadPutTimeMs; + } + + public Integer getReadDataTimeMs() { + return readDataTimeMs; + } + + public Integer getWriteDataTimeMs() { + return writeDataTimeMs; + } + + public Integer getCommitAndPublishTimeMs() { + return commitAndPublishTimeMs; + } + + public String getLabel() { + return label; + } + + @VisibleForTesting + public void setMessage(String message) { + this.message = message; + } + + @Override + public String toString() { + ObjectMapper mapper = new ObjectMapper(); + try { + return mapper.writeValueAsString(this); + } catch (JsonProcessingException e) { + return ""; + } + } + + public String getErrorURL() { + return errorURL; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java new file mode 100644 index 00000000000000..1bc63dc3f66b50 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/DebeziumJsonDeserializer.java @@ -0,0 +1,310 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.deserialize; + +import org.apache.doris.cdcclient.utils.ConfigUtil; +import org.apache.doris.job.cdc.DataSourceConfigKeys; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; +import org.apache.flink.cdc.debezium.utils.TemporalConversions; +import org.apache.flink.table.data.TimestampData; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.DateTimeException; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.doris.cdcclient.common.Constants.DORIS_DELETE_SIGN; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.debezium.data.Bits; +import io.debezium.data.Envelope; +import io.debezium.data.SpecialValueDecimal; +import io.debezium.data.VariableScaleDecimal; +import io.debezium.time.MicroTime; +import io.debezium.time.MicroTimestamp; +import io.debezium.time.NanoTime; +import io.debezium.time.NanoTimestamp; +import io.debezium.time.Time; +import io.debezium.time.Timestamp; +import io.debezium.time.ZonedTimestamp; +import lombok.Setter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** SourceRecord ==> [{},{}] */ +public class DebeziumJsonDeserializer + implements SourceRecordDeserializer> { + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(DebeziumJsonDeserializer.class); + private static ObjectMapper objectMapper = new ObjectMapper(); + @Setter private ZoneId serverTimeZone = ZoneId.systemDefault(); + + public DebeziumJsonDeserializer() {} + + @Override + public void init(Map props) { + this.serverTimeZone = + ConfigUtil.getServerTimeZone(props.get(DataSourceConfigKeys.JDBC_URL)); + } + + @Override + public List deserialize(Map context, SourceRecord record) + throws IOException { + if (RecordUtils.isDataChangeRecord(record)) { + LOG.trace("Process data change record: {}", record); + return deserializeDataChangeRecord(record); + } else if (RecordUtils.isSchemaChangeEvent(record)) { + return Collections.emptyList(); + } else { + return Collections.emptyList(); + } + } + + private List deserializeDataChangeRecord(SourceRecord record) throws IOException { + List rows = new ArrayList<>(); + Envelope.Operation op = Envelope.operationFor(record); + Struct value = (Struct) record.value(); + Schema valueSchema = record.valueSchema(); + if (Envelope.Operation.DELETE.equals(op)) { + String deleteRow = extractBeforeRow(value, valueSchema); + if (StringUtils.isNotEmpty(deleteRow)) { + rows.add(deleteRow); + } + } else if (Envelope.Operation.READ.equals(op) + || Envelope.Operation.CREATE.equals(op) + || Envelope.Operation.UPDATE.equals(op)) { + String insertRow = extractAfterRow(value, valueSchema); + if (StringUtils.isNotEmpty(insertRow)) { + rows.add(insertRow); + } + } + return rows; + } + + private String extractAfterRow(Struct value, Schema valueSchema) + throws JsonProcessingException { + Map record = new HashMap<>(); + Struct after = value.getStruct(Envelope.FieldName.AFTER); + if (after == null) { + return null; + } + Schema afterSchema = valueSchema.field(Envelope.FieldName.AFTER).schema(); + afterSchema + .fields() + .forEach( + field -> { + Object valueConverted = + convert(field.schema(), after.getWithoutDefault(field.name())); + record.put(field.name(), valueConverted); + }); + record.put(DORIS_DELETE_SIGN, 0); + return objectMapper.writeValueAsString(record); + } + + private String extractBeforeRow(Struct value, Schema valueSchema) + throws JsonProcessingException { + Map record = new HashMap<>(); + Struct before = value.getStruct(Envelope.FieldName.BEFORE); + if (before == null) { + return null; + } + Schema beforeSchema = valueSchema.field(Envelope.FieldName.BEFORE).schema(); + beforeSchema + .fields() + .forEach( + field -> { + Object valueConverted = + convert(field.schema(), before.getWithoutDefault(field.name())); + record.put(field.name(), valueConverted); + }); + record.put(DORIS_DELETE_SIGN, 1); + return objectMapper.writeValueAsString(record); + } + + private Object convert(Schema fieldSchema, Object dbzObj) { + if (dbzObj == null) { + return null; + } + String name = fieldSchema.name(); + Schema.Type type = fieldSchema.type(); + if (StringUtils.isEmpty(name)) { + switch (type) { + case INT8: + case INT16: + case INT32: + case INT64: + return Long.parseLong(dbzObj.toString()); + case FLOAT32: + return Float.parseFloat(dbzObj.toString()); + case FLOAT64: + return Double.parseDouble(dbzObj.toString()); + case BOOLEAN: + return Boolean.parseBoolean(dbzObj.toString()); + case STRING: + case ARRAY: + case MAP: + case STRUCT: + return dbzObj.toString(); + case BYTES: + return convertToBinary(dbzObj, fieldSchema); + default: + LOG.debug("Unsupported type: {}, transform value to string", type); + return dbzObj.toString(); + } + } else { + switch (name) { + case Time.SCHEMA_NAME: + case MicroTime.SCHEMA_NAME: + case NanoTime.SCHEMA_NAME: + return convertToTime(dbzObj, fieldSchema); + case io.debezium.time.Date.SCHEMA_NAME: + return TemporalConversions.toLocalDate(dbzObj).toString(); + case Timestamp.SCHEMA_NAME: + case MicroTimestamp.SCHEMA_NAME: + case NanoTimestamp.SCHEMA_NAME: + return convertTimestamp(name, dbzObj); + case ZonedTimestamp.SCHEMA_NAME: + return convertZoneTimestamp(dbzObj); + case Decimal.LOGICAL_NAME: + return convertDecimal(dbzObj, fieldSchema); + case Bits.LOGICAL_NAME: + return dbzObj; + default: + LOG.debug( + "Unsupported type: {} with name {}, transform value to string", + type, + name); + return dbzObj.toString(); + } + } + } + + private Object convertZoneTimestamp(Object dbzObj) { + if (dbzObj instanceof String) { + String str = (String) dbzObj; + // TIMESTAMP_LTZ type is encoded in string type + Instant instant = Instant.parse(str); + return TimestampData.fromLocalDateTime(LocalDateTime.ofInstant(instant, serverTimeZone)) + .toTimestamp() + .toString(); + } + LOG.warn("Unable to convert to zone timestamp, default {}", dbzObj); + return dbzObj.toString(); + } + + private Object convertTimestamp(String typeName, Object dbzObj) { + if (dbzObj instanceof Long) { + switch (typeName) { + case Timestamp.SCHEMA_NAME: + return TimestampData.fromEpochMillis((Long) dbzObj).toTimestamp().toString(); + case MicroTimestamp.SCHEMA_NAME: + long micro = (long) dbzObj; + return TimestampData.fromEpochMillis(micro / 1000, (int) (micro % 1000 * 1000)) + .toTimestamp() + .toString(); + case NanoTimestamp.SCHEMA_NAME: + long nano = (long) dbzObj; + return TimestampData.fromEpochMillis(nano / 1000_000, (int) (nano % 1000_000)) + .toTimestamp() + .toString(); + } + } + LocalDateTime localDateTime = TemporalConversions.toLocalDateTime(dbzObj, serverTimeZone); + return java.sql.Timestamp.valueOf(localDateTime); + } + + protected Object convertToBinary(Object dbzObj, Schema schema) { + if (dbzObj instanceof byte[]) { + return dbzObj; + } else if (dbzObj instanceof ByteBuffer) { + ByteBuffer byteBuffer = (ByteBuffer) dbzObj; + byte[] bytes = new byte[byteBuffer.remaining()]; + byteBuffer.get(bytes); + return bytes; + } else { + LOG.warn("Unable to convert to binary, default {}", dbzObj); + return dbzObj.toString(); + } + } + + private Object convertDecimal(Object dbzObj, Schema schema) { + BigDecimal bigDecimal; + if (dbzObj instanceof byte[]) { + // decimal.handling.mode=precise + bigDecimal = Decimal.toLogical(schema, (byte[]) dbzObj); + } else if (dbzObj instanceof String) { + // decimal.handling.mode=string + bigDecimal = new BigDecimal((String) dbzObj); + } else if (dbzObj instanceof Double) { + // decimal.handling.mode=double + bigDecimal = BigDecimal.valueOf((Double) dbzObj); + } else { + if (VariableScaleDecimal.LOGICAL_NAME.equals(schema.name())) { + SpecialValueDecimal decimal = VariableScaleDecimal.toLogical((Struct) dbzObj); + bigDecimal = decimal.getDecimalValue().orElse(BigDecimal.ZERO); + } else { + // fallback to string + bigDecimal = new BigDecimal(dbzObj.toString()); + } + } + return bigDecimal; + } + + protected Object convertToTime(Object dbzObj, Schema schema) { + try { + if (dbzObj instanceof Long) { + switch (schema.name()) { + case MicroTime.SCHEMA_NAME: + // micro to nano + return LocalTime.ofNanoOfDay((Long) dbzObj * 1000L).toString(); + case NanoTime.SCHEMA_NAME: + return LocalTime.ofNanoOfDay((Long) dbzObj).toString(); + } + } else if (dbzObj instanceof Integer) { + // millis to nano + return LocalTime.ofNanoOfDay((Integer) dbzObj * 1_000_000L).toString(); + } else if (dbzObj instanceof java.util.Date) { + long millisOfDay = ((Date) dbzObj).getTime() % (24 * 60 * 60 * 1000); + // mills to nano + return LocalTime.ofNanoOfDay(millisOfDay * 1_000_000L).toString(); + } + // get number of milliseconds of the day + return TemporalConversions.toLocalTime(dbzObj).toString(); + } catch (DateTimeException ex) { + LOG.warn("Unable to convert to time, default {}", dbzObj); + return dbzObj.toString(); + } + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/SourceRecordDeserializer.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/SourceRecordDeserializer.java new file mode 100644 index 00000000000000..f93567a230abea --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/deserialize/SourceRecordDeserializer.java @@ -0,0 +1,28 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.deserialize; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Map; + +public interface SourceRecordDeserializer extends Serializable { + void init(Map props); + + C deserialize(Map context, T record) throws IOException; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/DataSource.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/DataSource.java new file mode 100644 index 00000000000000..20f1f94ecf9524 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/DataSource.java @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.factory; + +public enum DataSource { + MYSQL +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java new file mode 100644 index 00000000000000..5a7a91c6f96a56 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/factory/SourceReaderFactory.java @@ -0,0 +1,58 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.factory; + +import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.cdcclient.source.reader.mysql.MySqlSourceReader; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** SourceReader register. */ +public final class SourceReaderFactory { + private static final Logger LOG = LoggerFactory.getLogger(SourceReaderFactory.class); + private static final Map> REGISTRY = + new ConcurrentHashMap<>(); + + static { + register(DataSource.MYSQL, MySqlSourceReader::new); + } + + private SourceReaderFactory() {} + + public static void register(DataSource source, Supplier supplier) { + Objects.requireNonNull(source, "source"); + Objects.requireNonNull(supplier, "supplier"); + REGISTRY.put(source, supplier); + LOG.info("Registered SourceReader provider for {}", source); + } + + public static SourceReader createSourceReader(DataSource source) { + Supplier supplier = REGISTRY.get(source); + if (supplier == null) { + throw new IllegalArgumentException( + "Unsupported SourceReader with datasource : " + source); + } + return supplier.get(); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java new file mode 100644 index 00000000000000..373d10a0ef3f37 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SourceReader.java @@ -0,0 +1,74 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.reader; + +import org.apache.doris.cdcclient.model.response.RecordWithMeta; +import org.apache.doris.job.cdc.request.CompareOffsetRequest; +import org.apache.doris.job.cdc.request.FetchRecordRequest; +import org.apache.doris.job.cdc.request.FetchTableSplitsRequest; +import org.apache.doris.job.cdc.request.JobBaseConfig; +import org.apache.doris.job.cdc.request.JobBaseRecordRequest; +import org.apache.doris.job.cdc.split.AbstractSourceSplit; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.kafka.connect.source.SourceRecord; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** Source Reader Interface */ +public interface SourceReader { + /** Initialization, called when the program starts */ + void initialize(Map config); + + /** Divide the data to be read. For example: split mysql to chunks */ + List getSourceSplits(FetchTableSplitsRequest config); + + /** Reading Data */ + RecordWithMeta read(FetchRecordRequest meta) throws Exception; + + /** Reading Data for split reader */ + SplitReadResult readSplitRecords(JobBaseRecordRequest baseReq) throws Exception; + + /** Extract offset information from snapshot split state. */ + Map extractSnapshotOffset(SourceSplit split, Object splitState); + + /** Extract offset information from binlog split. */ + Map extractBinlogOffset(SourceSplit split); + + /** Is the split a binlog split */ + boolean isBinlogSplit(SourceSplit split); + + /** Is the split a snapshot split */ + boolean isSnapshotSplit(SourceSplit split); + + /** Finish reading all split records */ + void finishSplitRecords(); + + /** Get the end offset for the job */ + Map getEndOffset(JobBaseConfig jobConfig); + + /** Compare the offsets */ + int compareOffset(CompareOffsetRequest compareOffsetRequest); + + /** Called when closing */ + void close(Long jobId); + + List deserialize(Map config, SourceRecord element) throws IOException; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java new file mode 100644 index 00000000000000..39f386f89886c1 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitReadResult.java @@ -0,0 +1,35 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.reader; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.kafka.connect.source.SourceRecord; + +import java.util.Iterator; + +import lombok.Data; + +/** The result of reading a split with iterator. */ +@Data +public class SplitReadResult { + private Iterator recordIterator; + // MySqlSplitState, SourceSplitState + private Object splitState; + // MySqlSplit SourceSplitBase + private SourceSplit split; +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java new file mode 100644 index 00000000000000..d7c712e463d116 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/SplitRecords.java @@ -0,0 +1,60 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.reader; + +import org.apache.flink.cdc.connectors.mysql.source.split.SourceRecords; +import org.apache.kafka.connect.source.SourceRecord; + +import java.util.Iterator; + +public class SplitRecords { + private final String splitId; + private final SourceRecords records; + private Iterator iterator; + + public SplitRecords(String splitId, SourceRecords records) { + this.splitId = splitId; + this.records = records; + this.iterator = records.iterator(); + } + + public String getSplitId() { + return splitId; + } + + public SourceRecords getRecords() { + return records; + } + + public Iterator getIterator() { + return iterator; + } + + public void setIterator(Iterator iterator) { + this.iterator = iterator; + } + + public boolean isEmpty() { + return splitId == null || records == null || !records.iterator().hasNext(); + } + + @Override + public String toString() { + return "SplitRecords{" + "split=" + splitId + ", records=" + records + '}'; + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java new file mode 100644 index 00000000000000..d9ff9ab81061e9 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -0,0 +1,756 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.source.reader.mysql; + +import org.apache.doris.cdcclient.model.response.RecordWithMeta; +import org.apache.doris.cdcclient.source.deserialize.DebeziumJsonDeserializer; +import org.apache.doris.cdcclient.source.deserialize.SourceRecordDeserializer; +import org.apache.doris.cdcclient.source.reader.SourceReader; +import org.apache.doris.cdcclient.source.reader.SplitReadResult; +import org.apache.doris.cdcclient.source.reader.SplitRecords; +import org.apache.doris.cdcclient.utils.ConfigUtil; +import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.cdc.request.CompareOffsetRequest; +import org.apache.doris.job.cdc.request.FetchRecordRequest; +import org.apache.doris.job.cdc.request.FetchTableSplitsRequest; +import org.apache.doris.job.cdc.request.JobBaseConfig; +import org.apache.doris.job.cdc.request.JobBaseRecordRequest; +import org.apache.doris.job.cdc.split.AbstractSourceSplit; +import org.apache.doris.job.cdc.split.BinlogSplit; +import org.apache.doris.job.cdc.split.SnapshotSplit; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.cdc.common.utils.Preconditions; +import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; +import org.apache.flink.cdc.connectors.mysql.debezium.reader.BinlogSplitReader; +import org.apache.flink.cdc.connectors.mysql.debezium.reader.DebeziumReader; +import org.apache.flink.cdc.connectors.mysql.debezium.reader.SnapshotSplitReader; +import org.apache.flink.cdc.connectors.mysql.debezium.task.context.StatefulTaskContext; +import org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlSnapshotSplitAssigner; +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; +import org.apache.flink.cdc.connectors.mysql.source.split.FinishedSnapshotSplitInfo; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlBinlogSplit; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlBinlogSplitState; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSnapshotSplit; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSnapshotSplitState; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplit; +import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplitState; +import org.apache.flink.cdc.connectors.mysql.source.split.SourceRecords; +import org.apache.flink.cdc.connectors.mysql.source.utils.ChunkUtils; +import org.apache.flink.cdc.connectors.mysql.source.utils.RecordUtils; +import org.apache.flink.cdc.connectors.mysql.source.utils.TableDiscoveryUtils; +import org.apache.flink.cdc.connectors.mysql.table.StartupMode; +import org.apache.flink.cdc.debezium.history.FlinkJsonTableChangeSerializer; +import org.apache.flink.table.types.logical.RowType; +import org.apache.kafka.connect.source.SourceRecord; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.flink.cdc.connectors.mysql.source.assigners.MySqlBinlogSplitAssigner.BINLOG_SPLIT_ID; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import io.debezium.connector.mysql.MySqlConnection; +import io.debezium.connector.mysql.MySqlPartition; +import io.debezium.document.Array; +import io.debezium.relational.Column; +import io.debezium.relational.TableId; +import io.debezium.relational.history.HistoryRecord; +import io.debezium.relational.history.TableChanges; +import lombok.Data; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Data +public class MySqlSourceReader implements SourceReader { + private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceReader.class); + private static ObjectMapper objectMapper = new ObjectMapper(); + private static final String SPLIT_ID = "splitId"; + private static final FlinkJsonTableChangeSerializer TABLE_CHANGE_SERIALIZER = + new FlinkJsonTableChangeSerializer(); + private SourceRecordDeserializer> serializer; + private SnapshotSplitReader snapshotReader; + private BinlogSplitReader binlogReader; + private DebeziumReader currentReader; + private Map tableSchemas; + private SplitRecords currentSplitRecords; + private MySqlSplit currentSplit; + + public MySqlSourceReader() { + this.serializer = new DebeziumJsonDeserializer(); + } + + @Override + public void initialize(Map config) { + this.serializer.init(config); + } + + @Override + public List getSourceSplits(FetchTableSplitsRequest ftsReq) { + LOG.info("Get table {} splits for job {}", ftsReq.getSnapshotTable(), ftsReq.getJobId()); + MySqlSourceConfig sourceConfig = getSourceConfig(ftsReq); + StartupMode startupMode = sourceConfig.getStartupOptions().startupMode; + List remainingSnapshotSplits = new ArrayList<>(); + MySqlBinlogSplit remainingBinlogSplit = null; + if (startupMode.equals(StartupMode.INITIAL)) { + remainingSnapshotSplits = + startSplitChunks(sourceConfig, ftsReq.getSnapshotTable(), ftsReq.getConfig()); + } else { + remainingBinlogSplit = + new MySqlBinlogSplit( + BINLOG_SPLIT_ID, + sourceConfig.getStartupOptions().binlogOffset, + BinlogOffset.ofNonStopping(), + new ArrayList<>(), + new HashMap<>(), + 0); + } + List splits = new ArrayList<>(); + if (!remainingSnapshotSplits.isEmpty()) { + for (MySqlSnapshotSplit snapshotSplit : remainingSnapshotSplits) { + String splitId = snapshotSplit.splitId(); + String tableId = snapshotSplit.getTableId().identifier(); + Object[] splitStart = snapshotSplit.getSplitStart(); + Object[] splitEnd = snapshotSplit.getSplitEnd(); + List splitKey = snapshotSplit.getSplitKeyType().getFieldNames(); + SnapshotSplit split = + new SnapshotSplit(splitId, tableId, splitKey, splitStart, splitEnd, null); + splits.add(split); + } + } else { + BinlogOffset startingOffset = remainingBinlogSplit.getStartingOffset(); + BinlogSplit binlogSplit = new BinlogSplit(); + binlogSplit.setSplitId(remainingBinlogSplit.splitId()); + binlogSplit.setStartingOffset(startingOffset.getOffset()); + splits.add(binlogSplit); + } + return splits; + } + + /** + * 1. If the SplitRecords iterator has it, read the iterator directly. 2. If there is a + * binlogreader, poll it. 3. If there is none, resubmit split. 4. If reload is true, need to + * reset binlogSplitReader and submit split. + */ + @Override + public RecordWithMeta read(FetchRecordRequest fetchRecord) throws Exception { + SplitReadResult readResult = readSplitRecords(fetchRecord); + return buildRecordResponse(fetchRecord, readResult); + } + + /** read split records. */ + @Override + public SplitReadResult readSplitRecords(JobBaseRecordRequest baseReq) throws Exception { + Map offsetMeta = baseReq.getMeta(); + if (offsetMeta == null || offsetMeta.isEmpty()) { + throw new RuntimeException("miss meta offset"); + } + LOG.info("Job {} read split records with offset: {}", baseReq.getJobId(), offsetMeta); + + // If there is an active split being consumed, reuse it directly; + // Otherwise, create a new snapshot/binlog split based on offset and start the reader. + MySqlSplit split = null; + SplitRecords currentSplitRecords = this.getCurrentSplitRecords(); + if (currentSplitRecords == null) { + DebeziumReader currentReader = this.getCurrentReader(); + if (currentReader == null || baseReq.isReload()) { + LOG.info( + "No current reader or reload {}, create new split reader", + baseReq.isReload()); + // build split + Tuple2 splitFlag = createMySqlSplit(offsetMeta, baseReq); + split = splitFlag.f0; + // reset binlog reader + closeBinlogReader(); + currentSplitRecords = pollSplitRecordsWithSplit(split, baseReq); + this.setCurrentSplitRecords(currentSplitRecords); + this.setCurrentSplit(split); + } else if (currentReader instanceof BinlogSplitReader) { + LOG.info("Continue poll records with current binlog reader"); + // only for binlog reader + currentSplitRecords = pollSplitRecordsWithCurrentReader(currentReader); + split = this.getCurrentSplit(); + } else { + throw new RuntimeException("Should not happen"); + } + } else { + LOG.info( + "Continue read records with current split records, splitId: {}", + currentSplitRecords.getSplitId()); + } + + // build response with iterator + SplitReadResult result = new SplitReadResult(); + MySqlSplitState currentSplitState = null; + MySqlSplit currentSplit = this.getCurrentSplit(); + if (currentSplit.isSnapshotSplit()) { + currentSplitState = new MySqlSnapshotSplitState(currentSplit.asSnapshotSplit()); + } else { + currentSplitState = new MySqlBinlogSplitState(currentSplit.asBinlogSplit()); + } + + Iterator filteredIterator = + new FilteredRecordIterator(currentSplitRecords, currentSplitState); + + result.setRecordIterator(filteredIterator); + result.setSplitState(currentSplitState); + result.setSplit(split); + return result; + } + + /** build RecordWithMeta */ + private RecordWithMeta buildRecordResponse( + FetchRecordRequest fetchRecord, SplitReadResult readResult) throws Exception { + RecordWithMeta recordResponse = new RecordWithMeta(); + SourceSplit split = readResult.getSplit(); + int count = 0; + try { + // Serialize records and add them to the response (collect from iterator) + Iterator iterator = readResult.getRecordIterator(); + while (iterator != null && iterator.hasNext()) { + SourceRecord element = iterator.next(); + List serializedRecords = + serializer.deserialize(fetchRecord.getConfig(), element); + if (!CollectionUtils.isEmpty(serializedRecords)) { + recordResponse.getRecords().addAll(serializedRecords); + count += serializedRecords.size(); + // update meta + Map lastMeta = + RecordUtils.getBinlogPosition(element).getOffset(); + if (isBinlogSplit(split)) { + lastMeta.put(SPLIT_ID, BINLOG_SPLIT_ID); + recordResponse.setMeta(lastMeta); + } + if (count >= fetchRecord.getFetchSize()) { + return recordResponse; + } + } + } + } finally { + finishSplitRecords(); + } + + // Set meta information + if (isSnapshotSplit(split) && readResult.getSplitState() != null) { + Map offsetRes = + extractSnapshotOffset(split, readResult.getSplitState()); + recordResponse.setMeta(offsetRes); + return recordResponse; + } + if (CollectionUtils.isEmpty(recordResponse.getRecords())) { + if (isBinlogSplit(split)) { + Map offsetRes = extractBinlogOffset(readResult.getSplit()); + recordResponse.setMeta(offsetRes); + } else { + SnapshotSplit snapshotSplit = + objectMapper.convertValue(fetchRecord.getMeta(), SnapshotSplit.class); + Map meta = new HashMap<>(); + meta.put(SPLIT_ID, snapshotSplit.getSplitId()); + // chunk no data + recordResponse.setMeta(meta); + } + } + return recordResponse; + } + + /** + * refresh table changes after schema change + * + * @param element + * @param jobId + * @throws IOException + */ + private void refreshTableChanges(SourceRecord element, Long jobId) throws IOException { + HistoryRecord historyRecord = RecordUtils.getHistoryRecord(element); + Array tableChanges = historyRecord.document().getArray(HistoryRecord.Fields.TABLE_CHANGES); + TableChanges changes = TABLE_CHANGE_SERIALIZER.deserialize(tableChanges, true); + Map tableChangeMap = this.getTableSchemas(); + if (tableChangeMap == null) { + tableChangeMap = new ConcurrentHashMap<>(); + this.setTableSchemas(tableChangeMap); + } + for (TableChanges.TableChange tblChange : changes) { + tableChangeMap.put(tblChange.getTable().id(), tblChange); + } + } + + private Tuple2 createMySqlSplit( + Map offsetMeta, JobBaseConfig jobConfig) + throws JsonProcessingException { + Tuple2 splitRes = null; + String splitId = String.valueOf(offsetMeta.get(SPLIT_ID)); + if (!BINLOG_SPLIT_ID.equals(splitId)) { + MySqlSnapshotSplit split = createSnapshotSplit(offsetMeta, jobConfig); + splitRes = Tuple2.of(split, false); + } else { + splitRes = createBinlogSplit(offsetMeta, jobConfig); + } + return splitRes; + } + + private MySqlSnapshotSplit createSnapshotSplit( + Map offset, JobBaseConfig jobConfig) throws JsonProcessingException { + SnapshotSplit snapshotSplit = objectMapper.convertValue(offset, SnapshotSplit.class); + TableId tableId = TableId.parse(snapshotSplit.getTableId()); + Object[] splitStart = snapshotSplit.getSplitStart(); + Object[] splitEnd = snapshotSplit.getSplitEnd(); + List splitKeys = snapshotSplit.getSplitKey(); + Map tableSchemas = getTableSchemas(jobConfig); + TableChanges.TableChange tableChange = tableSchemas.get(tableId); + Preconditions.checkNotNull( + tableChange, "Can not find table " + tableId + " in job " + jobConfig.getJobId()); + // only support one split key + String splitKey = splitKeys.get(0); + Column splitColumn = tableChange.getTable().columnWithName(splitKey); + RowType splitType = ChunkUtils.getChunkKeyColumnType(splitColumn, false); + MySqlSnapshotSplit split = + new MySqlSnapshotSplit( + tableId, + snapshotSplit.getSplitId(), + splitType, + splitStart, + splitEnd, + null, + tableSchemas); + return split; + } + + private Tuple2 createBinlogSplit( + Map meta, JobBaseConfig config) { + MySqlSourceConfig sourceConfig = getSourceConfig(config); + BinlogOffset offsetConfig = null; + if (sourceConfig.getStartupOptions() != null) { + offsetConfig = sourceConfig.getStartupOptions().binlogOffset; + } + BinlogSplit binlogSplit = objectMapper.convertValue(meta, BinlogSplit.class); + List finishedSnapshotSplitInfos = new ArrayList<>(); + BinlogOffset minOffsetFinishSplits = null; + BinlogOffset maxOffsetFinishSplits = null; + if (CollectionUtils.isNotEmpty(binlogSplit.getFinishedSplits())) { + List splitWithHW = binlogSplit.getFinishedSplits(); + List assignedSplitLists = + splitWithHW.stream() + .sorted(Comparator.comparing(AbstractSourceSplit::getSplitId)) + .toList(); + + for (SnapshotSplit split : assignedSplitLists) { + // find the min binlog offset + Map offsetMap = split.getHighWatermark(); + BinlogOffset binlogOffset = new BinlogOffset(offsetMap); + if (minOffsetFinishSplits == null || binlogOffset.isBefore(minOffsetFinishSplits)) { + minOffsetFinishSplits = binlogOffset; + } + if (maxOffsetFinishSplits == null || binlogOffset.isAfter(maxOffsetFinishSplits)) { + maxOffsetFinishSplits = binlogOffset; + } + finishedSnapshotSplitInfos.add( + new FinishedSnapshotSplitInfo( + TableId.parse(split.getTableId()), + split.getSplitId(), + split.getSplitStart(), + split.getSplitEnd(), + binlogOffset)); + } + } + + BinlogOffset startOffset; + BinlogOffset lastOffset = + new BinlogOffset( + binlogSplit.getStartingOffset() == null + ? new HashMap<>() + : binlogSplit.getStartingOffset()); + if (minOffsetFinishSplits != null && lastOffset.getOffsetKind() == null) { + startOffset = minOffsetFinishSplits; + } else if (lastOffset.getOffsetKind() != null && lastOffset.getFilename() != null) { + startOffset = lastOffset; + } else if (offsetConfig != null) { + startOffset = offsetConfig; + } else { + startOffset = BinlogOffset.ofEarliest(); + } + + boolean pureBinlogPhase = false; + if (maxOffsetFinishSplits == null) { + pureBinlogPhase = true; + } else if (startOffset.isAtOrAfter(maxOffsetFinishSplits)) { + // All the offsets of the current split are smaller than the offset of the binlog, + // indicating that the binlog phase has been fully entered. + pureBinlogPhase = true; + LOG.info( + "The binlog phase has been fully entered, the current split is: {}", + startOffset); + } + + MySqlBinlogSplit split = + new MySqlBinlogSplit( + BINLOG_SPLIT_ID, + startOffset, + BinlogOffset.ofNonStopping(), + finishedSnapshotSplitInfos, + new HashMap<>(), + 0); + // filterTableSchema + MySqlBinlogSplit binlogSplitFinal = + MySqlBinlogSplit.fillTableSchemas(split.asBinlogSplit(), getTableSchemas(config)); + return Tuple2.of(binlogSplitFinal, pureBinlogPhase); + } + + private List startSplitChunks( + MySqlSourceConfig sourceConfig, String snapshotTable, Map config) { + List remainingTables = new ArrayList<>(); + if (snapshotTable != null) { + // need add database name + String database = config.get(DataSourceConfigKeys.DATABASE); + remainingTables.add(TableId.parse(database + "." + snapshotTable)); + } + List remainingSplits = new ArrayList<>(); + MySqlSnapshotSplitAssigner splitAssigner = + new MySqlSnapshotSplitAssigner( + sourceConfig, 1, remainingTables, false, new MockSplitEnumeratorContext(1)); + splitAssigner.open(); + try { + while (true) { + Optional mySqlSplit = splitAssigner.getNext(); + if (mySqlSplit.isPresent()) { + MySqlSnapshotSplit snapshotSplit = mySqlSplit.get().asSnapshotSplit(); + remainingSplits.add(snapshotSplit); + } else { + break; + } + } + } finally { + // splitAssigner.close(); + closeChunkSplitterOnly(splitAssigner); + } + return remainingSplits; + } + + /** + * The JdbcConnectionPools inside MySqlSnapshotSplitAssigner are singletons. Calling + * MySqlSnapshotSplitAssigner.close() closes the entire JdbcConnectionPools, which can cause + * problems under high concurrency. This only closes the connection of the current + * MySqlSnapshotSplitAssigner. + */ + private void closeChunkSplitterOnly(MySqlSnapshotSplitAssigner splitAssigner) { + try { + // call closeExecutorService() + java.lang.reflect.Method closeExecutorMethod = + MySqlSnapshotSplitAssigner.class.getDeclaredMethod("closeExecutorService"); + closeExecutorMethod.setAccessible(true); + closeExecutorMethod.invoke(splitAssigner); + + // call chunkSplitter.close() + java.lang.reflect.Field field = + MySqlSnapshotSplitAssigner.class.getDeclaredField("chunkSplitter"); + field.setAccessible(true); + Object chunkSplitter = field.get(splitAssigner); + + if (chunkSplitter != null) { + java.lang.reflect.Method closeMethod = chunkSplitter.getClass().getMethod("close"); + closeMethod.invoke(chunkSplitter); + LOG.info("Closed chunkSplitter JDBC connection"); + } + } catch (Exception e) { + LOG.warn("Failed to close chunkSplitter via reflection,", e); + } + } + + private SplitRecords pollSplitRecordsWithSplit(MySqlSplit split, JobBaseConfig jobConfig) + throws Exception { + Preconditions.checkState(split != null, "split is null"); + Iterator dataIt = null; + String currentSplitId = null; + DebeziumReader currentReader = null; + LOG.info("Get a split: {}", split.splitId()); + if (split instanceof MySqlSnapshotSplit) { + currentReader = getSnapshotSplitReader(jobConfig); + } else if (split instanceof MySqlBinlogSplit) { + currentReader = getBinlogSplitReader(jobConfig); + } + this.setCurrentReader(currentReader); + currentReader.submitSplit(split); + currentSplitId = split.splitId(); + // make split record available + // todo: Until debezium_heartbeat is consumed + Thread.sleep(1000); + dataIt = currentReader.pollSplitRecords(); + if (currentReader instanceof SnapshotSplitReader) { + closeSnapshotReader(); + } + return dataIt == null ? null : new SplitRecords(currentSplitId, dataIt.next()); + } + + private SplitRecords pollSplitRecordsWithCurrentReader( + DebeziumReader currentReader) throws Exception { + Iterator dataIt = null; + if (currentReader instanceof BinlogSplitReader) { + dataIt = currentReader.pollSplitRecords(); + return dataIt == null ? null : new SplitRecords(BINLOG_SPLIT_ID, dataIt.next()); + } else { + throw new IllegalStateException("Unsupported reader type."); + } + } + + private SnapshotSplitReader getSnapshotSplitReader(JobBaseConfig config) { + MySqlSourceConfig sourceConfig = getSourceConfig(config); + SnapshotSplitReader snapshotReader = this.getSnapshotReader(); + if (snapshotReader == null) { + final MySqlConnection jdbcConnection = + DebeziumUtils.createMySqlConnection(sourceConfig); + final BinaryLogClient binaryLogClient = + DebeziumUtils.createBinaryClient(sourceConfig.getDbzConfiguration()); + final StatefulTaskContext statefulTaskContext = + new StatefulTaskContext(sourceConfig, binaryLogClient, jdbcConnection); + snapshotReader = new SnapshotSplitReader(statefulTaskContext, 0); + this.setSnapshotReader(snapshotReader); + } + return snapshotReader; + } + + private BinlogSplitReader getBinlogSplitReader(JobBaseConfig config) { + MySqlSourceConfig sourceConfig = getSourceConfig(config); + BinlogSplitReader binlogReader = this.getBinlogReader(); + if (binlogReader == null) { + final MySqlConnection jdbcConnection = + DebeziumUtils.createMySqlConnection(sourceConfig); + final BinaryLogClient binaryLogClient = + DebeziumUtils.createBinaryClient(sourceConfig.getDbzConfiguration()); + final StatefulTaskContext statefulTaskContext = + new StatefulTaskContext(sourceConfig, binaryLogClient, jdbcConnection); + binlogReader = new BinlogSplitReader(statefulTaskContext, 0); + this.setBinlogReader(binlogReader); + } + return binlogReader; + } + + private void closeSnapshotReader() { + SnapshotSplitReader reusedSnapshotReader = this.getSnapshotReader(); + if (reusedSnapshotReader != null) { + LOG.info( + "Close snapshot reader {}", reusedSnapshotReader.getClass().getCanonicalName()); + reusedSnapshotReader.close(); + DebeziumReader currentReader = this.getCurrentReader(); + if (reusedSnapshotReader == currentReader) { + this.setCurrentReader(null); + } + this.setSnapshotReader(null); + } + } + + private void closeBinlogReader() { + BinlogSplitReader reusedBinlogReader = this.getBinlogReader(); + if (reusedBinlogReader != null) { + LOG.info("Close binlog reader {}", reusedBinlogReader.getClass().getCanonicalName()); + reusedBinlogReader.close(); + DebeziumReader currentReader = this.getCurrentReader(); + if (reusedBinlogReader == currentReader) { + this.setCurrentReader(null); + } + this.setBinlogReader(null); + } + } + + private MySqlSourceConfig getSourceConfig(JobBaseConfig config) { + return ConfigUtil.generateMySqlConfig(config); + } + + @Override + public Map extractSnapshotOffset(SourceSplit split, Object splitState) { + Preconditions.checkNotNull(split, "split is null"); + Preconditions.checkNotNull(splitState, "splitState is null"); + MySqlSplitState mysqlSplitState = (MySqlSplitState) splitState; + MySqlSplit mysqlSplit = (MySqlSplit) split; + BinlogOffset highWatermark = mysqlSplitState.asSnapshotSplitState().getHighWatermark(); + Map offsetRes = new HashMap<>(highWatermark.getOffset()); + offsetRes.put(SPLIT_ID, mysqlSplit.splitId()); + return offsetRes; + } + + @Override + public Map extractBinlogOffset(SourceSplit split) { + Preconditions.checkNotNull(split, "split is null"); + MySqlSplit mysqlSplit = (MySqlSplit) split; + Map offsetRes = mysqlSplit.asBinlogSplit().getStartingOffset().getOffset(); + offsetRes.put(SPLIT_ID, BINLOG_SPLIT_ID); + return offsetRes; + } + + @Override + public boolean isBinlogSplit(SourceSplit split) { + Preconditions.checkNotNull(split, "split is null"); + MySqlSplit mysqlSplit = (MySqlSplit) split; + return mysqlSplit.isBinlogSplit(); + } + + @Override + public boolean isSnapshotSplit(SourceSplit split) { + Preconditions.checkNotNull(split, "split is null"); + MySqlSplit mysqlSplit = (MySqlSplit) split; + return mysqlSplit.isSnapshotSplit(); + } + + @Override + public void finishSplitRecords() { + this.setCurrentSplitRecords(null); + } + + @Override + public Map getEndOffset(JobBaseConfig jobConfig) { + MySqlSourceConfig sourceConfig = getSourceConfig(jobConfig); + try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) { + BinlogOffset binlogOffset = DebeziumUtils.currentBinlogOffset(jdbc); + return binlogOffset.getOffset(); + } catch (SQLException ex) { + throw new RuntimeException(ex); + } + } + + @Override + public int compareOffset(CompareOffsetRequest compareOffsetRequest) { + Map offsetFirst = compareOffsetRequest.getOffsetFirst(); + Map offsetSecond = compareOffsetRequest.getOffsetSecond(); + // make server id is equals + String serverId1 = offsetFirst.get("server_id"); + String serverId2 = offsetSecond.get("server_id"); + if (serverId1 == null && serverId2 != null) { + offsetFirst.put("server_id", serverId2); + } + if (serverId2 == null && serverId1 != null) { + offsetSecond.put("server_id", serverId1); + } + + BinlogOffset binlogOffset1 = new BinlogOffset(offsetFirst); + BinlogOffset binlogOffset2 = new BinlogOffset(offsetSecond); + return binlogOffset1.compareTo(binlogOffset2); + } + + private Map getTableSchemas(JobBaseConfig config) { + Map schemas = this.getTableSchemas(); + if (schemas == null) { + schemas = discoverTableSchemas(config); + this.setTableSchemas(schemas); + } + return schemas; + } + + private Map discoverTableSchemas(JobBaseConfig config) { + MySqlSourceConfig sourceConfig = getSourceConfig(config); + try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) { + MySqlPartition partition = + new MySqlPartition(sourceConfig.getMySqlConnectorConfig().getLogicalName()); + return TableDiscoveryUtils.discoverSchemaForCapturedTables( + partition, sourceConfig, jdbc); + } catch (SQLException ex) { + throw new RuntimeException(ex); + } + } + + @Override + public void close(Long jobId) { + LOG.info("Close source reader for job {}", jobId); + closeSnapshotReader(); + closeBinlogReader(); + currentReader = null; + currentSplitRecords = null; + tableSchemas.clear(); + tableSchemas = null; + } + + @Override + public List deserialize(Map config, SourceRecord element) + throws IOException { + return serializer.deserialize(config, element); + } + + /** + * Filtered record iterator that only returns data change records, filtering out watermark, + * heartbeat and other events. This is a private static inner class that encapsulates record + * filtering logic, making the main method cleaner. + */ + private static class FilteredRecordIterator implements Iterator { + private final Iterator sourceIterator; + private final MySqlSplitState splitState; + private SourceRecord nextRecord; + + FilteredRecordIterator(SplitRecords currentSplitRecords, MySqlSplitState splitState) { + this.sourceIterator = + currentSplitRecords != null && !currentSplitRecords.isEmpty() + ? currentSplitRecords.getIterator() + : null; + this.splitState = splitState; + } + + @Override + public boolean hasNext() { + if (sourceIterator == null) { + return false; + } + if (nextRecord != null) { + return true; + } + + while (sourceIterator.hasNext()) { + SourceRecord element = sourceIterator.next(); + if (RecordUtils.isWatermarkEvent(element)) { + BinlogOffset watermark = RecordUtils.getWatermark(element); + if (RecordUtils.isHighWatermarkEvent(element) + && splitState.isSnapshotSplitState()) { + splitState.asSnapshotSplitState().setHighWatermark(watermark); + } + } else if (RecordUtils.isHeartbeatEvent(element)) { + LOG.debug("Receive heartbeat event: {}", element); + if (splitState.isBinlogSplitState()) { + BinlogOffset position = RecordUtils.getBinlogPosition(element); + splitState.asBinlogSplitState().setStartingOffset(position); + } + } else if (RecordUtils.isDataChangeRecord(element)) { + nextRecord = element; + return true; + } else { + LOG.debug("Ignore event: {}", element); + } + } + return false; + } + + @Override + public SourceRecord next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + SourceRecord record = nextRecord; + nextRecord = null; + return record; + } + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java new file mode 100644 index 00000000000000..6cf84a5340d709 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/ConfigUtil.java @@ -0,0 +1,201 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.utils; + +import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.cdc.request.JobBaseConfig; + +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; +import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetUtils; +import org.apache.flink.cdc.connectors.mysql.table.StartupOptions; + +import java.sql.SQLException; +import java.time.ZoneId; +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.mysql.cj.conf.ConnectionUrl; +import io.debezium.connector.mysql.MySqlConnection; + +public class ConfigUtil { + private static final ObjectMapper mapper = new ObjectMapper(); + + public static String getServerId(long jobId) { + return String.valueOf(Math.abs(String.valueOf(jobId).hashCode())); + } + + public static MySqlSourceConfig generateMySqlConfig(JobBaseConfig config) { + return generateMySqlConfig(config.getConfig(), getServerId(config.getJobId())); + } + + public static MySqlSourceConfig generateMySqlConfig(Map config) { + return generateMySqlConfig(config, "0"); + } + + public static ZoneId getServerTimeZone(String jdbcUrl) { + Preconditions.checkNotNull(jdbcUrl, "jdbcUrl is null"); + ConnectionUrl cu = ConnectionUrl.getConnectionUrlInstance(jdbcUrl, null); + return getTimeZoneFromProps(cu.getOriginalProperties()); + } + + private static MySqlSourceConfig generateMySqlConfig( + Map cdcConfig, String serverId) { + MySqlSourceConfigFactory configFactory = new MySqlSourceConfigFactory(); + ConnectionUrl cu = + ConnectionUrl.getConnectionUrlInstance( + cdcConfig.get(DataSourceConfigKeys.JDBC_URL), null); + configFactory.hostname(cu.getMainHost().getHost()); + configFactory.port(cu.getMainHost().getPort()); + configFactory.username(cdcConfig.get(DataSourceConfigKeys.USER)); + configFactory.password(cdcConfig.get(DataSourceConfigKeys.PASSWORD)); + String databaseName = cdcConfig.get(DataSourceConfigKeys.DATABASE); + configFactory.databaseList(databaseName); + configFactory.serverId(serverId); + configFactory.serverTimeZone(getTimeZoneFromProps(cu.getOriginalProperties()).toString()); + + configFactory.includeSchemaChanges(false); + + String includingTables = cdcConfig.get(DataSourceConfigKeys.INCLUDE_TABLES); + String[] includingTbls = + Arrays.stream(includingTables.split(",")) + .map(t -> databaseName + "." + t.trim()) + .toArray(String[]::new); + configFactory.tableList(includingTbls); + + String excludingTables = cdcConfig.get(DataSourceConfigKeys.EXCLUDE_TABLES); + if (StringUtils.isNotEmpty(excludingTables)) { + String excludingTbls = + Arrays.stream(excludingTables.split(",")) + .map(t -> databaseName + "." + t.trim()) + .collect(Collectors.joining(",")); + configFactory.excludeTableList(excludingTbls); + } + + // setting startMode + String startupMode = cdcConfig.get(DataSourceConfigKeys.OFFSET); + if (DataSourceConfigKeys.OFFSET_INITIAL.equalsIgnoreCase(startupMode)) { + // do not need set offset when initial + // configFactory.startupOptions(StartupOptions.initial()); + } else if (DataSourceConfigKeys.OFFSET_EARLIEST.equalsIgnoreCase(startupMode)) { + configFactory.startupOptions(StartupOptions.earliest()); + BinlogOffset binlogOffset = + initializeEffectiveOffset( + configFactory, StartupOptions.earliest().binlogOffset); + configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); + } else if (DataSourceConfigKeys.OFFSET_LATEST.equalsIgnoreCase(startupMode)) { + configFactory.startupOptions(StartupOptions.latest()); + BinlogOffset binlogOffset = + initializeEffectiveOffset(configFactory, StartupOptions.latest().binlogOffset); + configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); + } else if (isJson(startupMode)) { + // start from specific offset + Map offsetMap = toStringMap(startupMode); + if (MapUtils.isEmpty(offsetMap)) { + throw new RuntimeException("Incorrect offset " + startupMode); + } + if (offsetMap.containsKey(BinlogOffset.BINLOG_FILENAME_OFFSET_KEY) + && offsetMap.containsKey(BinlogOffset.BINLOG_POSITION_OFFSET_KEY)) { + BinlogOffset binlogOffset = new BinlogOffset(offsetMap); + configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); + } else { + throw new RuntimeException("Incorrect offset " + startupMode); + } + } else if (is13Timestamp(startupMode)) { + // start from timestamp + Long ts = Long.parseLong(startupMode); + BinlogOffset binlogOffset = + initializeEffectiveOffset( + configFactory, StartupOptions.timestamp(ts).binlogOffset); + configFactory.startupOptions(StartupOptions.specificOffset(binlogOffset)); + } else { + throw new RuntimeException("Unknown offset " + startupMode); + } + + Properties jdbcProperteis = new Properties(); + jdbcProperteis.putAll(cu.getOriginalProperties()); + configFactory.jdbcProperties(jdbcProperteis); + + // configFactory.heartbeatInterval(Duration.ofMillis(1)); + if (cdcConfig.containsKey(DataSourceConfigKeys.SPLIT_SIZE)) { + configFactory.splitSize( + Integer.parseInt(cdcConfig.get(DataSourceConfigKeys.SPLIT_SIZE))); + } + + return configFactory.createConfig(0); + } + + private static ZoneId getTimeZoneFromProps(Map originalProperties) { + if (originalProperties != null && originalProperties.containsKey("serverTimezone")) { + String timeZone = originalProperties.get("serverTimezone"); + if (StringUtils.isNotEmpty(timeZone)) { + return ZoneId.of(timeZone); + } + } + return ZoneId.systemDefault(); + } + + private static BinlogOffset initializeEffectiveOffset( + MySqlSourceConfigFactory configFactory, BinlogOffset binlogOffset) { + MySqlSourceConfig config = configFactory.createConfig(0); + try (MySqlConnection connection = DebeziumUtils.createMySqlConnection(config)) { + return BinlogOffsetUtils.initializeEffectiveOffset(binlogOffset, connection, config); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private static boolean is13Timestamp(String s) { + return s != null && s.matches("\\d{13}"); + } + + private static boolean isJson(String str) { + if (str == null || str.trim().isEmpty()) { + return false; + } + try { + JsonNode node = mapper.readTree(str); + return node.isObject(); + } catch (Exception e) { + return false; + } + } + + private static Map toStringMap(String json) { + if (!isJson(json)) { + return null; + } + + try { + return mapper.readValue(json, new TypeReference>() {}); + } catch (JsonProcessingException e) { + return null; + } + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/HttpUtil.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/HttpUtil.java new file mode 100644 index 00000000000000..4d1356003fba60 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/utils/HttpUtil.java @@ -0,0 +1,55 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.cdcclient.utils; + +import org.apache.http.client.config.RequestConfig; +import org.apache.http.impl.NoConnectionReuseStrategy; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.DefaultRedirectStrategy; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.protocol.HttpRequestExecutor; +import org.apache.http.protocol.RequestContent; + +public class HttpUtil { + private static int connectTimeout = 30 * 1000; + private static int waitForContinueTimeout = 60 * 1000; + private static int socketTimeout = 10 * 60 * 1000; // stream load timeout 10 min + + public static CloseableHttpClient getHttpClient() { + return HttpClients.custom() + // default timeout 3s, maybe report 307 error when fe busy + .setRequestExecutor(new HttpRequestExecutor(waitForContinueTimeout)) + .setRedirectStrategy( + new DefaultRedirectStrategy() { + @Override + protected boolean isRedirectable(String method) { + return true; + } + }) + .setRetryHandler((exception, executionCount, context) -> false) + .setConnectionReuseStrategy(NoConnectionReuseStrategy.INSTANCE) + .setDefaultRequestConfig( + RequestConfig.custom() + .setConnectTimeout(connectTimeout) + .setConnectionRequestTimeout(connectTimeout) + .setSocketTimeout(socketTimeout) + .build()) + .addInterceptorLast(new RequestContent(true)) + .build(); + } +} diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/flink/api/connector/source/mocks/MockSplitEnumeratorContext.java b/fs_brokers/cdc_client/src/main/java/org/apache/flink/api/connector/source/mocks/MockSplitEnumeratorContext.java new file mode 100644 index 00000000000000..448e9acce516de --- /dev/null +++ b/fs_brokers/cdc_client/src/main/java/org/apache/flink/api/connector/source/mocks/MockSplitEnumeratorContext.java @@ -0,0 +1,320 @@ +/* +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package org.apache.flink.api.connector.source.mocks; + +import org.apache.flink.api.connector.source.ReaderInfo; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.api.connector.source.SupportsIntermediateNoMoreSplits; +import org.apache.flink.metrics.groups.SplitEnumeratorMetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.util.ThrowableCatchingRunnable; + +import javax.annotation.Nonnull; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; + +/** + * A mock class for {@link SplitEnumeratorContext}. Copy from + * https://github.com/apache/flink/blob/release-1.20.3/flink-core/src/test/java/org/apache/flink/api/connector/source/mocks/MockSplitEnumeratorContext.java + */ +public class MockSplitEnumeratorContext + implements SplitEnumeratorContext, SupportsIntermediateNoMoreSplits, AutoCloseable { + private final Map> sentSourceEvent; + private final ConcurrentMap registeredReaders; + private final List> splitsAssignmentSequence; + private final ExecutorService workerExecutor; + private final ExecutorService mainExecutor; + private final TestingExecutorThreadFactory mainThreadFactory; + private final AtomicReference errorInWorkerThread; + private final AtomicReference errorInMainThread; + private final BlockingQueue>> oneTimeCallables; + private final List>> periodicCallables; + private final AtomicBoolean stoppedAcceptAsyncCalls; + private final boolean[] subtaskHasNoMoreSplits; + + private final int parallelism; + + public MockSplitEnumeratorContext(int parallelism) { + this.sentSourceEvent = new HashMap<>(); + this.registeredReaders = new ConcurrentHashMap<>(); + this.splitsAssignmentSequence = new ArrayList<>(); + this.parallelism = parallelism; + this.errorInWorkerThread = new AtomicReference<>(); + this.errorInMainThread = new AtomicReference<>(); + this.oneTimeCallables = new ArrayBlockingQueue<>(100); + this.periodicCallables = Collections.synchronizedList(new ArrayList<>()); + this.mainThreadFactory = getThreadFactory("SplitEnumerator-main", errorInMainThread); + this.workerExecutor = + getExecutor(getThreadFactory("SplitEnumerator-worker", errorInWorkerThread)); + this.mainExecutor = getExecutor(mainThreadFactory); + this.stoppedAcceptAsyncCalls = new AtomicBoolean(false); + this.subtaskHasNoMoreSplits = new boolean[parallelism]; + } + + @Override + public SplitEnumeratorMetricGroup metricGroup() { + return UnregisteredMetricsGroup.createSplitEnumeratorMetricGroup(); + } + + @Override + public void sendEventToSourceReader(int subtaskId, SourceEvent event) { + try { + if (!mainThreadFactory.isCurrentThreadMainExecutorThread()) { + mainExecutor + .submit( + () -> + sentSourceEvent + .computeIfAbsent(subtaskId, id -> new ArrayList<>()) + .add(event)) + .get(); + } else { + sentSourceEvent.computeIfAbsent(subtaskId, id -> new ArrayList<>()).add(event); + } + } catch (Exception e) { + throw new RuntimeException("Failed to assign splits", e); + } + } + + @Override + public int currentParallelism() { + return parallelism; + } + + @Override + public Map registeredReaders() { + return registeredReaders; + } + + @Override + public void assignSplits(SplitsAssignment newSplitAssignments) { + splitsAssignmentSequence.add(newSplitAssignments); + } + + @Override + public void signalNoMoreSplits(int subtask) { + subtaskHasNoMoreSplits[subtask] = true; + } + + @Override + public void signalIntermediateNoMoreSplits(int subtask) {} + + public void resetNoMoreSplits(int subtask) { + subtaskHasNoMoreSplits[subtask] = false; + } + + @Override + public void callAsync(Callable callable, BiConsumer handler) { + if (stoppedAcceptAsyncCalls.get()) { + return; + } + oneTimeCallables.add( + () -> + workerExecutor.submit( + wrap( + errorInWorkerThread, + () -> { + try { + T result = callable.call(); + mainExecutor + .submit( + wrap( + errorInMainThread, + () -> + handler.accept( + result, + null))) + .get(); + } catch (Throwable t) { + handler.accept(null, t); + } + }))); + } + + @Override + public void callAsync( + Callable callable, + BiConsumer handler, + long initialDelay, + long period) { + if (stoppedAcceptAsyncCalls.get()) { + return; + } + periodicCallables.add( + () -> + workerExecutor.submit( + wrap( + errorInWorkerThread, + () -> { + try { + T result = callable.call(); + mainExecutor + .submit( + wrap( + errorInMainThread, + () -> + handler.accept( + result, + null))) + .get(); + } catch (Throwable t) { + handler.accept(null, t); + } + }))); + } + + @Override + public void runInCoordinatorThread(Runnable runnable) { + mainExecutor.execute(runnable); + } + + @Override + public void setIsProcessingBacklog(boolean isProcessingBacklog) {} + + public void close() throws Exception { + stoppedAcceptAsyncCalls.set(true); + workerExecutor.shutdownNow(); + mainExecutor.shutdownNow(); + } + + // ------------ helper method to manipulate the context ------------- + + public void runNextOneTimeCallable() throws Throwable { + oneTimeCallables.take().call().get(); + checkError(); + } + + public void runPeriodicCallable(int index) throws Throwable { + periodicCallables.get(index).call().get(); + checkError(); + } + + public Map> getSentSourceEvent() throws Exception { + return workerExecutor.submit(() -> new HashMap<>(sentSourceEvent)).get(); + } + + public void registerReader(ReaderInfo readerInfo) { + registeredReaders.put(readerInfo.getSubtaskId(), readerInfo); + } + + public void unregisterReader(int readerId) { + registeredReaders.remove(readerId); + } + + public List>> getPeriodicCallables() { + return periodicCallables; + } + + public BlockingQueue>> getOneTimeCallables() { + return oneTimeCallables; + } + + public List> getSplitsAssignmentSequence() { + return splitsAssignmentSequence; + } + + public boolean hasNoMoreSplits(int subtaskIndex) { + return subtaskHasNoMoreSplits[subtaskIndex]; + } + + // ------------- private helpers ------------- + + private void checkError() throws Throwable { + if (errorInMainThread.get() != null) { + throw errorInMainThread.get(); + } + if (errorInWorkerThread.get() != null) { + throw errorInWorkerThread.get(); + } + } + + private static TestingExecutorThreadFactory getThreadFactory( + String threadName, AtomicReference error) { + return new TestingExecutorThreadFactory(threadName, error); + } + + private static ExecutorService getExecutor(TestingExecutorThreadFactory threadFactory) { + return Executors.newSingleThreadScheduledExecutor(threadFactory); + } + + private static ThrowableCatchingRunnable wrap(AtomicReference error, Runnable r) { + return new ThrowableCatchingRunnable( + t -> { + if (!error.compareAndSet(null, t)) { + error.get().addSuppressed(t); + } + }, + r); + } + + // -------- private class ----------- + + /** A thread factory class that provides some helper methods. */ + public static class TestingExecutorThreadFactory implements ThreadFactory { + private final String coordinatorThreadName; + private final AtomicReference error; + private Thread t; + + TestingExecutorThreadFactory( + String coordinatorThreadName, AtomicReference error) { + this.coordinatorThreadName = coordinatorThreadName; + this.error = error; + this.t = null; + } + + @Override + public Thread newThread(@Nonnull Runnable r) { + if (t != null) { + throw new IllegalStateException( + "Should never happen. This factory should only be used by a " + + "SingleThreadExecutor."); + } + t = new Thread(r, coordinatorThreadName); + t.setUncaughtExceptionHandler( + (t1, e) -> { + if (!error.compareAndSet(null, e)) { + error.get().addSuppressed(e); + } + }); + return t; + } + + boolean isCurrentThreadMainExecutorThread() { + return Thread.currentThread() == t; + } + } +} diff --git a/fs_brokers/cdc_client/src/main/resources/application.properties b/fs_brokers/cdc_client/src/main/resources/application.properties new file mode 100644 index 00000000000000..9a7ee6c3f76ec3 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/resources/application.properties @@ -0,0 +1,20 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +spring.web.resources.add-mappings=false +server.port=9096 +backend.http.port=8040 \ No newline at end of file diff --git a/fs_brokers/cdc_client/src/main/resources/log4j.properties b/fs_brokers/cdc_client/src/main/resources/log4j.properties new file mode 100644 index 00000000000000..1ac680d4ddc5a2 --- /dev/null +++ b/fs_brokers/cdc_client/src/main/resources/log4j.properties @@ -0,0 +1,32 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +log4j.rootLogger=INFO, FILE + +log.path=./logs + +log4j.appender.STDOUT=org.apache.log4j.ConsoleAppender +log4j.appender.STDOUT.layout=org.apache.log4j.PatternLayout +log4j.appender.STDOUT.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.SSS} [%t] %-5p %c - %m%n + +log4j.appender.FILE=org.apache.log4j.RollingFileAppender +log4j.appender.FILE.File=${log.path}/cdc-client.log +log4j.appender.FILE.MaxFileSize=50MB +log4j.appender.FILE.MaxBackupIndex=10 +log4j.appender.FILE.layout=org.apache.log4j.PatternLayout +log4j.appender.FILE.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.SSS} [%t] %-5p %c - %m%n \ No newline at end of file diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index 384a4347c71e5c..9e9b6725401261 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -1116,6 +1116,16 @@ message PFetchPeerDataResponse { repeated CacheBlockPB datas = 2; } +message PRequestCdcClientRequest { + optional string api = 1; + optional string params = 2; +} + +message PRequestCdcClientResult { + optional PStatus status = 1; + optional string response = 2; +} + service PBackendService { // If #fragments of a query is < 3, use exec_plan_fragment directly. // If #fragments of a query is >=3, use exec_plan_fragment_prepare + exec_plan_fragment_start @@ -1173,5 +1183,6 @@ service PBackendService { rpc abort_refresh_dictionary(PAbortRefreshDictionaryRequest) returns (PAbortRefreshDictionaryResponse); rpc get_tablet_rowsets(PGetTabletRowsetsRequest) returns (PGetTabletRowsetsResponse); rpc fetch_peer_data(PFetchPeerDataRequest) returns (PFetchPeerDataResponse); + rpc request_cdc_client(PRequestCdcClientRequest) returns (PRequestCdcClientResult); }; diff --git a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out new file mode 100644 index 00000000000000..aebbb6815e3952 --- /dev/null +++ b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job.out @@ -0,0 +1,18 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_snapshot_table1 -- +A1 1 +B1 2 + +-- !select_snapshot_table2 -- +A2 1 +B2 2 + +-- !select_binlog_table1 -- +B1 10 +Doris 18 + +-- !select_next_binlog_table1 -- +Apache 40 +B1 10 +Doris 18 + diff --git a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_all_type.out b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_all_type.out new file mode 100644 index 00000000000000..691380f5b33ac8 --- /dev/null +++ b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_all_type.out @@ -0,0 +1,54 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !desc_all_types_null -- +tinyint_u smallint No true \N +smallint_u int Yes false \N NONE +mediumint_u int Yes false \N NONE +int_u bigint Yes false \N NONE +bigint_u largeint Yes false \N NONE +decimal1_u decimal(11,0) Yes false \N NONE +decimal2_u decimal(10,2) Yes false \N NONE +decimal3_u decimal(19,5) Yes false \N NONE +decimal4_u text Yes false \N NONE +decimal5_u text Yes false \N NONE +double_u double Yes false \N NONE +float_u float Yes false \N NONE +boolean boolean Yes false \N NONE +tinyint tinyint Yes false \N NONE +smallint smallint Yes false \N NONE +mediumint int Yes false \N NONE +int int Yes false \N NONE +bigint bigint Yes false \N NONE +double double Yes false \N NONE +float float Yes false \N NONE +decimal1 decimal(10,0) Yes false \N NONE +decimal2 decimal(9,2) Yes false \N NONE +decimal3 decimal(18,5) Yes false \N NONE +decimal4 decimal(38,10) Yes false \N NONE +decimal5 text Yes false \N NONE +year smallint Yes false \N NONE +time1 text Yes false \N NONE +time2 text Yes false \N NONE +time3 text Yes false \N NONE +date date Yes false \N NONE +datetime datetime Yes false \N NONE +timestamp1 datetime Yes false \N NONE +timestamp2 datetime(3) Yes false \N NONE +timestamp3 datetime(6) Yes false \N NONE +char char(5) Yes false \N NONE +varchar varchar(10) Yes false \N NONE +text text Yes false \N NONE +blob text Yes false \N NONE +json text Yes false \N NONE +set text Yes false \N NONE +bit text Yes false \N NONE +binary text Yes false \N NONE +varbinary text Yes false \N NONE +enum text Yes false \N NONE + +-- !select_all_types_null -- +1 120 50000 1000000 9000000000 1000 12345.67 987654.12345 123456789.1234567890 99999999.123456789012345678901234567890 123.45 12.34 true -5 -300 -20000 -500000 -8000000000 -123.45 -12.34 -1000 -1234.56 -98765.43210 -123456789.1234567890 -99999999.123456789012345678901234567890 2023 08:30 08:30:00.123 08:30:00.123456 2023-06-15 2023-06-15T08:30 2023-06-15T08:30 2023-06-15T08:30:00.123 2023-06-15T08:30:00.123456 abc user_001 normal text content c2ltcGxlIGJsb2IgZGF0YQ== {"id": 1, "name": "userA"} Option1 Mw== YmluYXJ5X2RhdGEx dmFyYmluXzAx Value1 + +-- !select_all_types_null2 -- +1 120 50000 1000000 9000000000 1000 12345.67 987654.12345 123456789.1234567890 99999999.123456789012345678901234567890 123.45 12.34 true -5 -300 -20000 -500000 -8000000000 -123.45 -12.34 -1000 -1234.56 -98765.43210 -123456789.1234567890 -99999999.123456789012345678901234567890 2023 08:30 08:30:00.123 08:30:00.123456 2023-06-15 2023-06-15T08:30 2023-06-15T08:30 2023-06-15T08:30:00.123 2023-06-15T08:30:00.123456 abc user_001 normal text content c2ltcGxlIGJsb2IgZGF0YQ== {"id": 1, "name": "userA"} Option1 Mw== YmluYXJ5X2RhdGEx dmFyYmluXzAx Value1 +2 100 100000 1000000000 100000000000 12345 12345.67 123456789.12345 12345678901234567890.1234567890 123456789012345678901234567890.123456789012345678901234567890 12345.6789 123.456 true -10 -1000 -100000 -100000000 -1000000000000 -12345.6789 -123.456 -12345 -12345.67 -123456789.12345 -12345678901234567890.1234567890 -123456789012345678901234567890.123456789012345678901234567890 2024 12:34:56 12:34:56.789 12:34:56.789123 2024-01-01 2024-01-01T12:34:56 2024-01-01T12:34:56 2024-01-01T12:34:56.789 2024-01-01T12:34:56.789123 hello hello123 this is a text field dGhpcyBpcyBhIGJsb2I= {"id":10,"name":"mock"} Option2 Kg== YmluX2RhdGFfMTIz dmFyYmluX2RhdGE= Value2 + diff --git a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_exclude.out b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_exclude.out new file mode 100644 index 00000000000000..4a0130d4b60dad --- /dev/null +++ b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_exclude.out @@ -0,0 +1,5 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +A1 1 +B1 2 + diff --git a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_priv.out b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_priv.out new file mode 100644 index 00000000000000..255245c9942034 --- /dev/null +++ b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_priv.out @@ -0,0 +1,6 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select -- +A1 1 +B1 2 +DorisTestPriv 28 + diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy new file mode 100644 index 00000000000000..fa3065849fc000 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job.groovy @@ -0,0 +1,167 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job", "p0,external,mysql,external_docker,external_docker_mysql") { + def jobName = "test_streaming_mysql_job_name" + def currentDb = (sql "select database()")[0][0] + def table1 = "user_info_normal1" + def table2 = "user_info_normal2" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + sql """drop table if exists ${currentDb}.${table2} force""" + + // Pre-create table2 + sql """ + CREATE TABLE IF NOT EXISTS ${currentDb}.${table2} ( + `name` varchar(200) NULL, + `age` int NULL + ) ENGINE=OLAP + UNIQUE KEY(`name`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`name`) BUCKETS AUTO + PROPERTIES ("replication_allocation" = "tag.location.default: 1"); + """ + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // create test + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table2}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('A1', 1);""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('B1', 2);""" + sql """CREATE TABLE ${mysqlDb}.${table2} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB""" + // mock snapshot data + sql """INSERT INTO ${mysqlDb}.${table2} (name, age) VALUES ('A2', 1);""" + sql """INSERT INTO ${mysqlDb}.${table2} (name, age) VALUES ('B2', 2);""" + } + + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1},${table2}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + def showAllTables = sql """ show tables from ${currentDb}""" + log.info("showAllTables: " + showAllTables) + // check table created + def showTables = sql """ show tables from ${currentDb} like '${table1}'; """ + assert showTables.size() == 1 + def showTables2 = sql """ show tables from ${currentDb} like '${table2}'; """ + assert showTables2.size() == 1 + + // check table schema correct + def showTbl1 = sql """show create table ${currentDb}.${table1}""" + def createTalInfo = showTbl1[0][1]; + assert createTalInfo.contains("`name` varchar(200)"); + assert createTalInfo.contains("`age` int"); + assert createTalInfo.contains("UNIQUE KEY(`name`)"); + assert createTalInfo.contains("DISTRIBUTED BY HASH(`name`) BUCKETS AUTO"); + + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + // check job status and succeed task count larger than 2 + jobSuccendCount.size() == 1 && '2' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + // check snapshot data + qt_select_snapshot_table1 """ SELECT * FROM ${table1} order by name asc """ + qt_select_snapshot_table2 """ SELECT * FROM ${table2} order by name asc """ + + // mock mysql incremental into + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """INSERT INTO ${mysqlDb}.${table1} (name,age) VALUES ('Doris',18);""" + sql """UPDATE ${mysqlDb}.${table1} SET age = 10 WHERE name = 'B1';""" + sql """DELETE FROM ${mysqlDb}.${table1} WHERE name = 'A1';""" + } + + sleep(30000); // wait for cdc incremental data + + // check incremental data + qt_select_binlog_table1 """ SELECT * FROM ${table1} order by name asc """ + + def jobInfo = sql """ + select loadStatistic, status from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfo: " + jobInfo) + assert jobInfo.get(0).get(0) == "{\"scannedRows\":7,\"loadBytes\":334,\"fileNumber\":0,\"fileSize\":0}" + assert jobInfo.get(0).get(1) == "RUNNING" + + // mock mysql incremental into again + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """INSERT INTO ${mysqlDb}.${table1} (name,age) VALUES ('Apache',40);""" + } + + sleep(30000); // wait for cdc incremental data + + // check incremental data + qt_select_next_binlog_table1 """ SELECT * FROM ${table1} order by name asc """ + + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } +} diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_all_type.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_all_type.groovy new file mode 100644 index 00000000000000..e24e93b61770db --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_all_type.groovy @@ -0,0 +1,156 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job_all_type", "p0,external,mysql,external_docker,external_docker_mysql") { + def jobName = "test_streaming_mysql_job_all_type_name" + def currentDb = (sql "select database()")[0][0] + def table1 = "streaming_all_types_nullable_with_pk" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // create test + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """ + create table ${mysqlDb}.${table1} ( + `tinyint_u` tinyint unsigned primary key, + `smallint_u` smallint unsigned, + `mediumint_u` mediumint unsigned, + `int_u` int unsigned, + `bigint_u` bigint unsigned, + `decimal1_u` decimal unsigned, + `decimal2_u` decimal(9, 2) unsigned, + `decimal3_u` decimal(18, 5) unsigned, + `decimal4_u` decimal(38, 10) unsigned, + `decimal5_u` decimal(65, 30) unsigned, + `double_u` double unsigned, + `float_u` float unsigned, + `boolean` boolean, + `tinyint` tinyint, + `smallint` smallint, + `mediumint` mediumint, + `int` int, + `bigint` bigint, + `double` double, + `float` float, + `decimal1` decimal, + `decimal2` decimal(9, 2), + `decimal3` decimal(18, 5) , + `decimal4` decimal(38, 10), + `decimal5` decimal(65, 30), + `year` year, + `time1` time, + `time2` time(3), + `time3` time(6), + `date` date, + `datetime` datetime, + `timestamp1` timestamp null, + `timestamp2` timestamp(3) null, + `timestamp3` timestamp(6) null, + `char` char(5), + `varchar` varchar(10), + `text` text, + `blob` blob, + `json` json, + `set` set('Option1', 'Option2', 'Option3'), + `bit` bit(6), + `binary` binary(12), + `varbinary` varbinary(12), + `enum` enum('Value1', 'Value2', 'Value3') + ) engine=innodb charset=utf8; + """ + // mock snapshot data + sql """ + insert into ${mysqlDb}.${table1} values (1,120,50000,1000000,9000000000,1000,12345.67,987654.12345,123456789.1234567890,99999999.123456789012345678901234567890,123.45,12.34,true,-5,-300,-20000,-500000,-8000000000,-123.45,-12.34,-1000,-1234.56,-98765.43210,-123456789.1234567890,-99999999.123456789012345678901234567890,2023,'08:30:00','08:30:00.123','08:30:00.123456','2023-06-15','2023-06-15 08:30:00','2023-06-15 08:30:00','2023-06-15 08:30:00.123','2023-06-15 08:30:00.123456','abc','user_001','normal text content','simple blob data','{"id":1,"name":"userA"}','Option1',b'110011','binary_data1','varbin_01','Value1'); + """ + } + + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}?serverTimezone=UTC", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + // check job status and succeed task count larger than 2 + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + qt_desc_all_types_null """desc ${currentDb}.${table1};""" + qt_select_all_types_null """select * from ${currentDb}.${table1} order by 1;""" + + // mock mysql incremental into + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """insert into ${mysqlDb}.${table1} values (2,100,100000,1000000000,100000000000,12345,12345.67,123456789.12345,12345678901234567890.1234567890,123456789012345678901234567890.123456789012345678901234567890,12345.6789,123.456,true,-10,-1000,-100000,-100000000,-1000000000000,-12345.6789,-123.456,-12345,-12345.67,-123456789.12345,-12345678901234567890.1234567890,-123456789012345678901234567890.123456789012345678901234567890,2024,'12:34:56','12:34:56.789','12:34:56.789123','2024-01-01','2024-01-01 12:34:56','2024-01-01 12:34:56','2024-01-01 12:34:56.789','2024-01-01 12:34:56.789123','hello','hello123','this is a text field','this is a blob','{"id":10,"name":"mock"}','Option2',b'101010','bin_data_123','varbin_data','Value2');""" + } + + sleep(30000); // wait for cdc incremental data + + // check incremental data + qt_select_all_types_null2 """select * from ${currentDb}.${table1} order by 1;""" + + + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } +} diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_create_alter.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_create_alter.groovy new file mode 100644 index 00000000000000..5b210a2fd74295 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_create_alter.groovy @@ -0,0 +1,464 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job_create_alter", "p0,external,mysql,external_docker,external_docker_mysql") { + def jobName = "test_streaming_mysql_job_create_alter" + def currentDb = (sql "select database()")[0][0] + def table1 = "create_alter_user_info" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // unexcepted source properties + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial1" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Invalid value for key 'offset': initial1" + } + + // unexcepted target properties + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties1.replication_num" = "1" + ) + """ + exception "Only support target properties with prefix table.create.properties" + } + + //error jdbc url format + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc1:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Failed to parse db type from jdbcUrl" + } + + // error jdbc url format + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root12345", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Access denied for user" + } + + // no exist db or tables + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root12345", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Access denied for user" + } + + // no exist db or empty db + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "noexistdb", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "No tables found in database" + } + + // no match table + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "noexisttable", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Can not found match table in database" + } + + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('A1', 1);""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('B1', 2);""" + } + + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + // check job status and succeed task count larger than 1 + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + def jobInfo = sql """ + select status from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfo: " + jobInfo) + assert jobInfo.get(0).get(0) == "RUNNING" + + // alter job + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://xxx:xxx", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Only PAUSED job can be altered" + } + + sql "PAUSE JOB where jobname = '${jobName}'"; + def jobInfo2 = sql """ + select status from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfo: " + jobInfo2) + assert jobInfo2.get(0).get(0) == "PAUSED" + + // alter jdbc url + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://127.0.0.1:4456", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "The jdbc_url property cannot be modified in ALTER JOB" + } + + // alter database + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "updatedatabase", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "The database property cannot be modified in ALTER JOB" + } + + // alter include tables + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "changeTable", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "The include_tables property cannot be modified in ALTER JOB" + } + + // alter exclude tables + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "exclude_tables" = "xxxx", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "The exclude_tables property cannot be modified in ALTER JOB" + } + + // unexcept properties + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial", + "xxx"="xxx" + ) + TO DATABASE ${currentDb} + """ + exception "Unexpected key" + } + + // update source type + test { + sql """ALTER JOB ${jobName} + FROM POSTGRES ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} + """ + exception "source type can't be modified" + } + + // unexcept properties + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE update_database + """ + exception "target database can't be modified" + } + + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb}""" + + def jobInfoOrigin = sql """ + select CurrentOffset,LoadStatistic from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfoOrigin: " + jobInfoOrigin) + + // alter job properties + sql """ALTER JOB ${jobName} + PROPERTIES( + "max_interval" = "5" + ) """ + + sql "RESUME JOB where jobname = '${jobName}'"; + + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobStatus = sql """ select status from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobStatus: " + jobStatus) + // check job status and succeed task count larger than 1 + jobStatus.size() == 1 && 'RUNNING' == jobStatus.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + def jobInfoCurrent = sql """ + select CurrentOffset,LoadStatistic,Properties,ExecuteSql from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfoCurrent: " + jobInfoCurrent) + assert jobInfoCurrent.get(0).get(0) == jobInfoOrigin.get(0).get(0) + assert jobInfoCurrent.get(0).get(1) == jobInfoOrigin.get(0).get(1) + assert jobInfoCurrent.get(0).get(2) == "{\"max_interval\":\"5\"}" + assert jobInfoCurrent.get(0).get(3).contains("latest") + + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } +} diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_dup.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_dup.groovy new file mode 100644 index 00000000000000..dba24d884c52d2 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_dup.groovy @@ -0,0 +1,79 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_streaming_mysql_job_dup", "p0,external,mysql,external_docker,external_docker_mysql") { + def jobName = "test_streaming_mysql_job_name" + def currentDb = (sql "select database()")[0][0] + def table1 = "test_streaming_mysql_job_dup" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // create test + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('A1', 1);""" + } + + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "The following tables do not have primary key defined: ${table1}" + } + + def jobInfo = sql """ + select * from jobs("type"="insert") where Name='${jobName}' + """ + assert jobInfo.size()== 0 + + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } +} diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_exclude.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_exclude.groovy new file mode 100644 index 00000000000000..bd6240269d19a0 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_exclude.groovy @@ -0,0 +1,117 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job_exclude", "p0,external,mysql,external_docker,external_docker_mysql") { + def jobName = "test_streaming_mysql_job_exclude_name" + def currentDb = (sql "select database()")[0][0] + def table1 = "user_info_exclude1" + def table2 = "user_info_exclude2" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + sql """drop table if exists ${currentDb}.${table2} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // create test + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table2}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('A1', 1);""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('B1', 2);""" + sql """CREATE TABLE ${mysqlDb}.${table2} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB""" + // mock snapshot data + sql """INSERT INTO ${mysqlDb}.${table2} (name, age) VALUES ('A2', 1);""" + sql """INSERT INTO ${mysqlDb}.${table2} (name, age) VALUES ('B2', 2);""" + } + + // exclude_table + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "exclude_tables" = "${table2}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + // check table created + def showTables = sql """ show tables from ${currentDb} like '${table1}'; """ + assert showTables.size() == 1 + def showTables2 = sql """ show tables from ${currentDb} like '${table2}'; """ + assert showTables2.size() == 0 + + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + // check job status and succeed task count larger than 1 + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + // check snapshot data + qt_select """ SELECT * FROM ${table1} order by name asc """ + + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } +} diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_priv.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_priv.groovy new file mode 100644 index 00000000000000..aae1bc581341d7 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_priv.groovy @@ -0,0 +1,202 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job_priv", "p0,external,mysql,external_docker,external_docker_mysql") { + def tableName = "test_streaming_mysql_job_priv_tbl" + def jobName = "test_streaming_mysql_job_priv_name" + + def currentDb = (sql "select database()")[0][0] + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${tableName} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // create user + def user = "test_streaming_mysql_job_priv_user" + def pwd = '123456' + def dbName = context.config.getDbNameByFile(context.file) + sql """DROP USER IF EXISTS '${user}'""" + sql """CREATE USER '${user}' IDENTIFIED BY '${pwd}'""" + def tokens = context.config.jdbcUrl.split('/') + def url = tokens[0] + "//" + tokens[2] + "/" + dbName + "?" + sql """grant select_priv on ${dbName}.* to ${user}""" + + if (isCloudMode()){ + // Cloud requires USAGE_PRIV to show clusters. + def clusters = sql_return_maparray """show clusters""" + log.info("show cluster res: " + clusters) + assertNotNull(clusters) + + for (item in clusters) { + log.info("cluster item: " + item.is_current + ", " + item.cluster) + if (item.is_current.equalsIgnoreCase("TRUE")) { + sql """GRANT USAGE_PRIV ON CLUSTER `${item.cluster}` TO ${user}"""; + break + } + } + } + + // create job with select priv user + connect(user, "${pwd}", url) { + expectExceptionLike({ + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${tableName}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + }, "you need (at least one of) the (LOAD) privilege") + } + + def jobCount = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + assert jobCount.size() == 0 + + // create mysql test + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${tableName}""" + sql """CREATE TABLE ${mysqlDb}.${tableName} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${tableName} (name, age) VALUES ('A1', 1);""" + sql """INSERT INTO ${mysqlDb}.${tableName} (name, age) VALUES ('B1', 2);""" + } + + // create streaming job by load_priv and create_priv + sql """grant load_priv,create_priv on ${dbName}.* to ${user}""" + connect(user, "${pwd}", url) { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}?allowPublicKeyRetrieval=true&useSSL=false", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${tableName}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + } + + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + // check job status and succeed task count larger than 2 + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex) { + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + def jobResult = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + log.info("show jobResult: " + jobResult) + + // create a new mysql user only has select priv + def newMysqlUser = "mysql_job_priv" + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """DROP USER IF EXISTS '${newMysqlUser}'@'%'""" + sql """CREATE USER '${newMysqlUser}'@'%' IDENTIFIED WITH mysql_native_password BY 'test123'""" + sql """GRANT ALL PRIVILEGES ON ${mysqlDb}.* TO '${newMysqlUser}'@'%'""" + sql """FLUSH PRIVILEGES""" + } + + sql "PAUSE JOB where jobname = '${jobName}'"; + + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}?allowPublicKeyRetrieval=true&useSSL=false", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "${newMysqlUser}", + "password" = "test123", + "database" = "${mysqlDb}", + "include_tables" = "${tableName}", + "offset" = "latest" + ) + TO DATABASE ${currentDb}""" + + sql "RESUME JOB where jobname = '${jobName}'"; + + // mock binlog data to ensure successful modification of user and password. + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """INSERT INTO ${mysqlDb}.${tableName} (name,age) VALUES ('DorisTestPriv',28);""" + } + + sleep(20000) + + def jobErrorMsg = sql """select ErrorMsg from jobs("type"="insert") where Name='${jobName}'""" + log.info("jobErrorMsg: " + jobErrorMsg) + assert jobErrorMsg.get(0).get(0).contains("Failed to fetch meta") + + // grant binlog priv to mysqluser + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """GRANT REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO '${newMysqlUser}'@'%'""" + sql """FLUSH PRIVILEGES""" + } + + sleep(30000) + + // check incremental data + qt_select """ SELECT * FROM ${tableName} order by name asc """ + + + sql """DROP USER IF EXISTS '${user}'""" + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } +} diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_restart_fe.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_restart_fe.groovy new file mode 100644 index 00000000000000..eb0e4866143ed4 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_restart_fe.groovy @@ -0,0 +1,126 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import org.apache.doris.regression.suite.ClusterOptions +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job_restart_fe", "docker,mysql,external_docker,external_docker_mysql") { + def jobName = "test_streaming_mysql_job_restart_fe" + def options = new ClusterOptions() + options.setFeNum(1) + // run in cloud and not cloud + options.cloudMode = null + + docker(options) { + def currentDb = (sql "select database()")[0][0] + def table1 = "restart_user_info" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // create test + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `name` varchar(200) NOT NULL, + `age` int DEFAULT NULL, + PRIMARY KEY (`name`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('A1', 1);""" + sql """INSERT INTO ${mysqlDb}.${table1} (name, age) VALUES ('B1', 2);""" + } + + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + // check table created + def showTables = sql """ show tables from ${currentDb} like '${table1}'; """ + assert showTables.size() == 1 + + // check job running + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + // check job status and succeed task count larger than 2 + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex){ + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + def jobInfoBeforeRestart = sql """ + select loadStatistic, status, currentOffset from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfoBeforeRestart: " + jobInfoBeforeRestart) + assert jobInfoBeforeRestart.get(0).get(0) == "{\"scannedRows\":2,\"loadBytes\":94,\"fileNumber\":0,\"fileSize\":0}" + assert jobInfoBeforeRestart.get(0).get(1) == "RUNNING" + + // Restart FE + cluster.restartFrontends() + sleep(60000) + context.reconnectFe() + + // check is it consistent after restart + def jobAfterRestart = sql """ + select loadStatistic, status, currentOffset from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobAfterRestart: " + jobAfterRestart) + assert jobAfterRestart.get(0).get(0) == "{\"scannedRows\":2,\"loadBytes\":94,\"fileNumber\":0,\"fileSize\":0}" + assert jobAfterRestart.get(0).get(1) == "RUNNING" + assert jobAfterRestart.get(0).get(2) == jobInfoBeforeRestart.get(0).get(2) + + sql """ + DROP JOB IF EXISTS where jobname = '${jobName}' + """ + def jobCountRsp = sql """select count(1) from jobs("type"="insert") where Name ='${jobName}'""" + assert jobCountRsp.get(0).get(0) == 0 + } + } +}