diff --git a/be/cmake/thirdparty.cmake b/be/cmake/thirdparty.cmake index 227f81411f1322..1dba907307deca 100644 --- a/be/cmake/thirdparty.cmake +++ b/be/cmake/thirdparty.cmake @@ -140,6 +140,7 @@ add_thirdparty(aws-c-s3 LIB64) add_thirdparty(aws-c-sdkutils LIB64) add_thirdparty(aws-cpp-sdk-identity-management LIB64) add_thirdparty(aws-cpp-sdk-sts LIB64) +add_thirdparty(aws-cpp-sdk-kinesis LIB64) if (NOT OS_MACOSX) add_thirdparty(aws-s2n LIBNAME "lib/libs2n.a") endif() diff --git a/be/src/common/metrics/doris_metrics.cpp b/be/src/common/metrics/doris_metrics.cpp index 81f3469dc2ce03..0c8716ec1f65ae 100644 --- a/be/src/common/metrics/doris_metrics.cpp +++ b/be/src/common/metrics/doris_metrics.cpp @@ -167,6 +167,13 @@ DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(routine_load_get_msg_latency, MetricUnit::M DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(routine_load_get_msg_count, MetricUnit::NOUNIT); DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(routine_load_consume_rows, MetricUnit::ROWS); DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(routine_load_consume_bytes, MetricUnit::BYTES); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(routine_load_kinesis_get_records_latency, + MetricUnit::MILLISECONDS); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(routine_load_kinesis_get_records_count, MetricUnit::NOUNIT); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(routine_load_kinesis_throttle_count, MetricUnit::NOUNIT); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(routine_load_kinesis_retriable_error_count, + MetricUnit::NOUNIT); +DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(routine_load_kinesis_closed_shard_count, MetricUnit::NOUNIT); DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(memtable_flush_total, MetricUnit::OPERATIONS); DEFINE_COUNTER_METRIC_PROTOTYPE_2ARG(memtable_flush_duration_us, MetricUnit::MICROSECONDS); @@ -353,6 +360,11 @@ DorisMetrics::DorisMetrics() : _metric_registry(_s_registry_name) { INT_COUNTER_METRIC_REGISTER(_server_metric_entity, routine_load_get_msg_count); INT_COUNTER_METRIC_REGISTER(_server_metric_entity, routine_load_consume_bytes); INT_COUNTER_METRIC_REGISTER(_server_metric_entity, routine_load_consume_rows); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, routine_load_kinesis_get_records_latency); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, routine_load_kinesis_get_records_count); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, routine_load_kinesis_throttle_count); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, routine_load_kinesis_retriable_error_count); + INT_COUNTER_METRIC_REGISTER(_server_metric_entity, routine_load_kinesis_closed_shard_count); INT_COUNTER_METRIC_REGISTER(_server_metric_entity, memtable_flush_total); INT_COUNTER_METRIC_REGISTER(_server_metric_entity, memtable_flush_duration_us); diff --git a/be/src/common/metrics/doris_metrics.h b/be/src/common/metrics/doris_metrics.h index faa4830c2b1b81..a91fb92feed94b 100644 --- a/be/src/common/metrics/doris_metrics.h +++ b/be/src/common/metrics/doris_metrics.h @@ -139,6 +139,11 @@ class DorisMetrics { IntCounter* routine_load_get_msg_count = nullptr; IntCounter* routine_load_consume_bytes = nullptr; IntCounter* routine_load_consume_rows = nullptr; + IntCounter* routine_load_kinesis_get_records_latency = nullptr; + IntCounter* routine_load_kinesis_get_records_count = nullptr; + IntCounter* routine_load_kinesis_throttle_count = nullptr; + IntCounter* routine_load_kinesis_retriable_error_count = nullptr; + IntCounter* routine_load_kinesis_closed_shard_count = nullptr; IntCounter* memtable_flush_total = nullptr; IntCounter* memtable_flush_duration_us = nullptr; diff --git a/be/src/io/fs/kafka_consumer_pipe.h b/be/src/io/fs/kafka_consumer_pipe.h index d13004a46b9e40..40373fa6be7e61 100644 --- a/be/src/io/fs/kafka_consumer_pipe.h +++ b/be/src/io/fs/kafka_consumer_pipe.h @@ -27,21 +27,6 @@ class KafkaConsumerPipe : public StreamLoadPipe { : StreamLoadPipe(max_buffered_bytes, min_chunk_size) {} ~KafkaConsumerPipe() override = default; - - virtual Status append_with_line_delimiter(const char* data, size_t size) { - Status st = append(data, size); - if (!st.ok()) { - return st; - } - - // append the line delimiter - st = append("\n", 1); - return st; - } - - virtual Status append_json(const char* data, size_t size) { - return append_and_flush(data, size); - } }; } // namespace io } // end namespace doris diff --git a/be/src/io/fs/kinesis_consumer_pipe.h b/be/src/io/fs/kinesis_consumer_pipe.h new file mode 100644 index 00000000000000..8fc21550e76cc0 --- /dev/null +++ b/be/src/io/fs/kinesis_consumer_pipe.h @@ -0,0 +1,34 @@ +// 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 "io/fs/stream_load_pipe.h" + +namespace doris { +namespace io { + +class KinesisConsumerPipe : public StreamLoadPipe { +public: + KinesisConsumerPipe(size_t max_buffered_bytes = 1024 * 1024, size_t min_chunk_size = 64 * 1024) + : StreamLoadPipe(max_buffered_bytes, min_chunk_size) {} + + ~KinesisConsumerPipe() override = default; +}; + +} // namespace io +} // namespace doris diff --git a/be/src/io/fs/stream_load_pipe.h b/be/src/io/fs/stream_load_pipe.h index c5456a85357abb..fee6186a8da1e2 100644 --- a/be/src/io/fs/stream_load_pipe.h +++ b/be/src/io/fs/stream_load_pipe.h @@ -53,6 +53,15 @@ class StreamLoadPipe : public MessageBodySink, public FileReader { Status append(const char* data, size_t size) override; Status append(const ByteBufferPtr& buf) override; + virtual Status append_with_line_delimiter(const char* data, size_t size) { + RETURN_IF_ERROR(append(data, size)); + return append("\n", 1); + } + + virtual Status append_json(const char* data, size_t size) { + return append_and_flush(data, size); + } + const Path& path() const override { return _path; } size_t size() const override { return 0; } diff --git a/be/src/load/routine_load/consumer_group_helpers.h b/be/src/load/routine_load/consumer_group_helpers.h new file mode 100644 index 00000000000000..685f306bd28973 --- /dev/null +++ b/be/src/load/routine_load/consumer_group_helpers.h @@ -0,0 +1,96 @@ +// 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 + +namespace doris { + +// Helper class for partitioning work items (partitions/shards) across consumers +template +class WorkPartitioner { +public: + // Divide work items equally across N consumers using round-robin + static std::vector> partition_round_robin( + const std::map& work_items, int consumer_count) { + std::vector> result(consumer_count); + int i = 0; + for (const auto& [key, value] : work_items) { + int idx = i % consumer_count; + result[idx].emplace(key, value); + i++; + } + return result; + } +}; + +// Helper class for managing format-based append operations +class FormatAppender { +public: + // Get the appropriate append function pointer based on format type + template + static auto get_append_function(TFileFormatType::type format) + -> Status (PipeType::*)(const char*, size_t) { + return (format == TFileFormatType::FORMAT_JSON) ? &PipeType::append_json + : &PipeType::append_with_line_delimiter; + } +}; + +// Helper class for tracking consumption progress +class ConsumptionProgress { +public: + ConsumptionProgress(int64_t max_time_ms, int64_t max_rows, int64_t max_bytes) + : _initial_time(max_time_ms), + _initial_rows(max_rows), + _initial_bytes(max_bytes), + _left_time(max_time_ms), + _left_rows(max_rows), + _left_bytes(max_bytes) {} + + // Check if any limit is reached + bool is_limit_reached() const { return _left_time <= 0 || _left_rows <= 0 || _left_bytes <= 0; } + + // Update progress after consuming one item + void consume_item(int64_t bytes) { + _left_rows--; + _left_bytes -= bytes; + } + + // Update time progress + void update_time(int64_t elapsed_us) { _left_time = _initial_time - elapsed_us / 1000; } + + // Getters + int64_t left_time() const { return _left_time; } + int64_t left_rows() const { return _left_rows; } + int64_t left_bytes() const { return _left_bytes; } + int64_t consumed_rows() const { return _initial_rows - _left_rows; } + int64_t consumed_bytes() const { return _initial_bytes - _left_bytes; } + int64_t consumed_time() const { return _initial_time - _left_time; } + +private: + int64_t _initial_time; + int64_t _initial_rows; + int64_t _initial_bytes; + int64_t _left_time; + int64_t _left_rows; + int64_t _left_bytes; +}; + +} // namespace doris diff --git a/be/src/load/routine_load/consumer_helpers.cpp b/be/src/load/routine_load/consumer_helpers.cpp new file mode 100644 index 00000000000000..23d3d6912745fa --- /dev/null +++ b/be/src/load/routine_load/consumer_helpers.cpp @@ -0,0 +1,61 @@ +// 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 "load/routine_load/consumer_helpers.h" + +#include +#include + +#include "common/metrics/doris_metrics.h" + +namespace doris { + +// ConsumerMetrics implementation +void ConsumerMetrics::track_get_msg(int64_t latency_us) { + _get_msg_count++; + DorisMetrics::instance()->routine_load_get_msg_count->increment(1); + DorisMetrics::instance()->routine_load_get_msg_latency->increment(latency_us / 1000); +} + +void ConsumerMetrics::track_consume_bytes(int64_t bytes) { + _consume_bytes += bytes; + DorisMetrics::instance()->routine_load_consume_bytes->increment(bytes); +} + +void ConsumerMetrics::track_consume_rows(int64_t rows) { + _consume_rows += rows; + DorisMetrics::instance()->routine_load_consume_rows->increment(rows); +} + +// RetryPolicy implementation +void RetryPolicy::retry_with_backoff() { + _retry_count++; + if (_retry_count <= _max_retries) { + std::this_thread::sleep_for(std::chrono::milliseconds(_backoff_ms)); + } +} + +// ThrottleBackoff implementation +void ThrottleBackoff::backoff_and_sleep() { + _throttle_count++; + // Exponential backoff: initial_ms * 2^(count-1), capped at max_ms + int backoff_ms = std::min(_initial_backoff_ms * (1 << std::min(_throttle_count - 1, 3)), + _max_backoff_ms); + std::this_thread::sleep_for(std::chrono::milliseconds(backoff_ms)); +} + +} // namespace doris diff --git a/be/src/load/routine_load/consumer_helpers.h b/be/src/load/routine_load/consumer_helpers.h new file mode 100644 index 00000000000000..0da8ed51c2f7b5 --- /dev/null +++ b/be/src/load/routine_load/consumer_helpers.h @@ -0,0 +1,119 @@ +// 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 + +namespace doris { + +// Helper class for tracking metrics in data consumers +class ConsumerMetrics { +public: + ConsumerMetrics() = default; + + // Track a single message fetch operation + void track_get_msg(int64_t latency_us); + + // Track consumed bytes + void track_consume_bytes(int64_t bytes); + + // Track consumed rows + void track_consume_rows(int64_t rows); + + // Get accumulated metrics + int64_t get_msg_count() const { return _get_msg_count; } + int64_t consume_bytes() const { return _consume_bytes; } + int64_t consume_rows() const { return _consume_rows; } + +private: + int64_t _get_msg_count = 0; + int64_t _consume_bytes = 0; + int64_t _consume_rows = 0; +}; + +// Helper class for retry logic with configurable backoff +class RetryPolicy { +public: + explicit RetryPolicy(int max_retries = 3, int backoff_ms = 200) + : _max_retries(max_retries), _backoff_ms(backoff_ms), _retry_count(0) {} + + // Check if should retry + bool should_retry() const { return _retry_count < _max_retries; } + + // Increment retry count and sleep with backoff + void retry_with_backoff(); + + // Reset retry counter (call on success) + void reset() { _retry_count = 0; } + + // Get current retry count + int retry_count() const { return _retry_count; } + +private: + int _max_retries; + int _backoff_ms; + int _retry_count; +}; + +// Helper class for exponential backoff (used for throttling) +class ThrottleBackoff { +public: + explicit ThrottleBackoff(int initial_backoff_ms = 1000, int max_backoff_ms = 10000) + : _initial_backoff_ms(initial_backoff_ms), + _max_backoff_ms(max_backoff_ms), + _throttle_count(0) {} + + // Increment throttle count and sleep with exponential backoff + void backoff_and_sleep(); + + // Reset throttle counter (call on success) + void reset() { _throttle_count = 0; } + + // Get current throttle count + int throttle_count() const { return _throttle_count; } + +private: + int _initial_backoff_ms; + int _max_backoff_ms; + int _throttle_count; +}; + +// Helper class for comparing custom properties between consumers +class PropertyMatcher { +public: + // Check if two property maps match exactly + template + static bool properties_match(const MapType1& props1, const MapType2& props2) { + if (props1.size() != props2.size()) { + return false; + } + + for (const auto& [key, value] : props1) { + auto it = props2.find(key); + if (it == props2.end() || it->second != value) { + return false; + } + } + + return true; + } +}; + +} // namespace doris diff --git a/be/src/load/routine_load/data_consumer.cpp b/be/src/load/routine_load/data_consumer.cpp index a8a3808335d6c6..516649f0f35b8e 100644 --- a/be/src/load/routine_load/data_consumer.cpp +++ b/be/src/load/routine_load/data_consumer.cpp @@ -22,6 +22,19 @@ #include #include +// AWS Kinesis SDK includes +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + #include // IWYU pragma: no_include #include // IWYU pragma: keep @@ -33,6 +46,8 @@ #include "common/config.h" #include "common/metrics/doris_metrics.h" #include "common/status.h" +#include "load/routine_load/consumer_helpers.h" +#include "load/routine_load/kinesis_conf.h" #include "runtime/aws_msk_iam_auth.h" #include "runtime/exec_env.h" #include "runtime/small_file_mgr.h" @@ -40,6 +55,7 @@ #include "util/blocking_queue.hpp" #include "util/debug_points.h" #include "util/defer_op.h" +#include "util/s3_util.h" #include "util/stopwatch.hpp" #include "util/string_util.h" #include "util/uid_util.h" @@ -230,18 +246,18 @@ Status KafkaDataConsumer::assign_topic_partitions( Status KafkaDataConsumer::group_consume(BlockingQueue* queue, int64_t max_running_time_ms) { - static constexpr int MAX_RETRY_TIMES_FOR_TRANSPORT_FAILURE = 3; int64_t left_time = max_running_time_ms; LOG(INFO) << "start kafka consumer: " << _id << ", grp: " << _grp_id << ", max running time(ms): " << left_time; int64_t received_rows = 0; int64_t put_rows = 0; - int32_t retry_times = 0; + RetryPolicy retry_policy(3, 200); Status st = Status::OK(); MonotonicStopWatch consumer_watch; MonotonicStopWatch watch; watch.start(); + while (true) { { std::unique_lock l(_lock); @@ -259,9 +275,11 @@ Status KafkaDataConsumer::group_consume(BlockingQueue* queue, consumer_watch.start(); std::unique_ptr msg(_k_consumer->consume(1000 /* timeout, ms */)); consumer_watch.stop(); + DorisMetrics::instance()->routine_load_get_msg_count->increment(1); DorisMetrics::instance()->routine_load_get_msg_latency->increment( consumer_watch.elapsed_time() / 1000 / 1000); + DBUG_EXECUTE_IF("KafkaDataConsumer.group_consume.out_of_range", { done = true; std::stringstream ss; @@ -272,8 +290,10 @@ Status KafkaDataConsumer::group_consume(BlockingQueue* queue, st = Status::InternalError(ss.str()); break; }); + switch (msg->err()) { case RdKafka::ERR_NO_ERROR: + retry_policy.reset(); if (_consuming_partition_ids.count(msg->partition()) <= 0) { _consuming_partition_ids.insert(msg->partition()); } @@ -300,9 +320,9 @@ Status KafkaDataConsumer::group_consume(BlockingQueue* queue, break; case RdKafka::ERR__TRANSPORT: LOG(INFO) << "kafka consume Disconnected: " << _id - << ", retry times: " << retry_times++; - if (retry_times <= MAX_RETRY_TIMES_FOR_TRANSPORT_FAILURE) { - std::this_thread::sleep_for(std::chrono::milliseconds(200)); + << ", retry times: " << retry_policy.retry_count(); + if (retry_policy.should_retry()) { + retry_policy.retry_with_backoff(); break; } [[fallthrough]]; @@ -582,21 +602,551 @@ bool KafkaDataConsumer::match(std::shared_ptr ctx) { return false; } // check properties - if (_custom_properties.size() != ctx->kafka_info->properties.size()) { + return PropertyMatcher::properties_match(_custom_properties, ctx->kafka_info->properties); +} + +// ==================== AWS Kinesis Data Consumer Implementation ==================== + +KinesisDataConsumer::KinesisDataConsumer(std::shared_ptr ctx) + : _region(ctx->kinesis_info->region), + _stream(ctx->kinesis_info->stream), + _endpoint(ctx->kinesis_info->endpoint) { + VLOG_NOTICE << "construct Kinesis consumer: stream=" << _stream << ", region=" << _region; +} + +KinesisDataConsumer::~KinesisDataConsumer() { + VLOG_NOTICE << "destruct Kinesis consumer: stream=" << _stream; + // AWS SDK client managed by shared_ptr, will be automatically cleaned up +} + +Status KinesisDataConsumer::init(std::shared_ptr ctx) { + std::unique_lock l(_lock); + if (_init) { + return Status::OK(); // Already initialized (idempotent) + } + + // Store custom properties (AWS credentials, etc.) + _custom_properties.insert(ctx->kinesis_info->properties.begin(), + ctx->kinesis_info->properties.end()); + + // Create KinesisConf and configure it + _kinesis_conf = std::make_unique(); + std::string errstr; + + // Parse and categorize aws.kinesis.* properties into three types + for (auto& item : _custom_properties) { + if (starts_with(item.first, "aws.kinesis.")) { + std::string conf_key = item.first.substr(12); // Remove "aws.kinesis." prefix + + // Type 2: Frequently-used parameters (explicit members) + if (conf_key == "shards") { + std::vector parts = + absl::StrSplit(item.second, ",", absl::SkipWhitespace()); + _explicit_shards = std::move(parts); + VLOG_NOTICE << "Set explicit shards: " << item.second; + } else if (conf_key == "default.pos") { + _default_position = item.second; + VLOG_NOTICE << "Set default position: " << item.second; + } else if (starts_with(conf_key, "shards.pos.")) { + std::string shard_id = conf_key.substr(11); // Remove "shards.pos." prefix + _shard_positions[shard_id] = item.second; + VLOG_NOTICE << "Set shard position: " << shard_id << " = " << item.second; + } + // Type 3: Less-frequently-used API parameters (KinesisConf determines which API) + else { + KinesisConf::ConfResult res = _kinesis_conf->set(conf_key, item.second, errstr); + if (res == KinesisConf::CONF_INVALID) { + return Status::InternalError("Failed to set '{}': {}", conf_key, errstr); + } + // CONF_UNKNOWN is acceptable (parameter will be ignored) + } + } + } + + // Create AWS Kinesis client + RETURN_IF_ERROR(_create_kinesis_client(ctx)); + + VLOG_NOTICE << "finished to init Kinesis consumer. stream=" << _stream << ", region=" << _region + << ", " << ctx->brief(); + _init = true; + return Status::OK(); +} + +Status KinesisDataConsumer::_create_kinesis_client(std::shared_ptr ctx) { + // Reuse S3ClientFactory's credential provider logic + // This supports all AWS authentication methods: + // - Simple AK/SK + // - IAM instance profile (EC2) + // - STS assume role + // - Session tokens + // - Environment variables + // - Default credential chain + + S3ClientConf s3_conf; + s3_conf.region = _region; + s3_conf.endpoint = _endpoint; + + auto get_property = [this](const char* key) -> std::string { + auto it = _custom_properties.find(key); + if (it != _custom_properties.end() && !it->second.empty()) { + return it->second; + } + return ""; + }; + + // Keep one naming convention aligned with FE-side Kinesis properties. + s3_conf.ak = get_property("aws.access_key"); + s3_conf.sk = get_property("aws.secret_key"); + s3_conf.token = get_property("aws.session_key"); + s3_conf.role_arn = get_property("aws.role_arn"); + s3_conf.external_id = get_property("aws.external.id"); + + const std::string provider = get_property("aws.credentials.provider"); + if (!provider.empty()) { + // Map provider type string to enum + if (provider == "instance_profile") { + s3_conf.cred_provider_type = CredProviderType::InstanceProfile; + } else if (provider == "env") { + s3_conf.cred_provider_type = CredProviderType::Env; + } else if (provider == "simple") { + s3_conf.cred_provider_type = CredProviderType::Simple; + } + } + + // Create AWS ClientConfiguration + Aws::Client::ClientConfiguration aws_config = S3ClientFactory::getClientConfiguration(); + aws_config.region = _region; + + if (!_endpoint.empty()) { + aws_config.endpointOverride = _endpoint; + } + + std::string ca_cert_file_path = + get_valid_ca_cert_path(doris::split(config::ca_cert_file_paths, ";")); + if (!ca_cert_file_path.empty()) { + aws_config.caFile = ca_cert_file_path; + } + + auto parse_timeout_ms = [](const std::string& timeout_value, const std::string& property_name, + long* timeout_ms) -> Status { + try { + *timeout_ms = std::stol(timeout_value); + } catch (const std::exception&) { + return Status::InternalError("Invalid value for {}: {}", property_name, timeout_value); + } + return Status::OK(); + }; + + // Set timeouts from properties or use defaults + auto it_request_timeout = _custom_properties.find("aws.request.timeout.ms"); + if (it_request_timeout != _custom_properties.end()) { + RETURN_IF_ERROR(parse_timeout_ms(it_request_timeout->second, "aws.request.timeout.ms", + &aws_config.requestTimeoutMs)); + } else { + aws_config.requestTimeoutMs = 30000; // 30s default + } + + auto it_conn_timeout = _custom_properties.find("aws.connection.timeout.ms"); + if (it_conn_timeout != _custom_properties.end()) { + RETURN_IF_ERROR(parse_timeout_ms(it_conn_timeout->second, "aws.connection.timeout.ms", + &aws_config.connectTimeoutMs)); + } + + // Get credentials provider (reuses S3 infrastructure) + auto credentials_provider = S3ClientFactory::instance().get_aws_credentials_provider(s3_conf); + + // Create Kinesis client + _kinesis_client = + std::make_shared(credentials_provider, aws_config); + + if (!_kinesis_client) { + return Status::InternalError( + "Failed to create AWS Kinesis client for stream: {}, region: {}", _stream, _region); + } + + LOG(INFO) << "Created Kinesis client for stream: " << _stream << ", region: " << _region; + return Status::OK(); +} + +Status KinesisDataConsumer::assign_shards( + const std::map& shard_sequence_numbers, + const std::string& stream_name, std::shared_ptr ctx) { + DORIS_CHECK(_kinesis_client); + + std::stringstream ss; + ss << "Assigning shards to Kinesis consumer " << _id << ": "; + + for (auto& entry : shard_sequence_numbers) { + const std::string& shard_id = entry.first; + const std::string& sequence_number = entry.second; + + // Get shard iterator for this shard + std::string iterator; + RETURN_IF_ERROR(_get_shard_iterator(shard_id, sequence_number, &iterator)); + + _shard_iterators[shard_id] = iterator; + _consuming_shard_ids.insert(shard_id); + + ss << "[" << shard_id << ": " << sequence_number << "] "; + } + + LOG(INFO) << ss.str(); + return Status::OK(); +} + +Status KinesisDataConsumer::_get_shard_iterator(const std::string& shard_id, + const std::string& sequence_number, + std::string* iterator) { + Aws::Kinesis::Model::GetShardIteratorRequest request; + + // Apply all configurations through KinesisConf + DCHECK(_kinesis_conf != nullptr); + Status st = _kinesis_conf->apply_to_get_shard_iterator_request(request, _stream, shard_id, + sequence_number); + if (!st.ok()) { + return Status::InternalError( + "Failed to apply Kinesis config to GetShardIteratorRequest: {}", st.to_string()); + } + + auto outcome = _kinesis_client->GetShardIterator(request); + if (!outcome.IsSuccess()) { + auto& error = outcome.GetError(); + return Status::InternalError("Failed to get shard iterator for shard {}: {} ({})", shard_id, + error.GetMessage(), static_cast(error.GetErrorType())); + } + + *iterator = outcome.GetResult().GetShardIterator(); + VLOG_NOTICE << "Got shard iterator for shard: " << shard_id; + return Status::OK(); +} + +Status KinesisDataConsumer::group_consume( + BlockingQueue>* queue, + int64_t max_running_time_ms) { + static constexpr int INTER_SHARD_SLEEP_MS = 10; // Small sleep between shards + static constexpr int MIN_INTERVAL_BETWEEN_ROUNDS_MS = 200; // Min 200ms between rounds + + int64_t left_time = max_running_time_ms; + LOG(INFO) << "start Kinesis consumer: " << _id << ", grp: " << _grp_id + << ", stream: " << _stream << ", max running time(ms): " << left_time; + + int64_t received_rows = 0; + int64_t put_rows = 0; + RetryPolicy retry_policy(3, 200); + ThrottleBackoff throttle_backoff(1000, 10000); + Status st = Status::OK(); + bool done = false; + + MonotonicStopWatch consumer_watch; + MonotonicStopWatch watch; + watch.start(); + + while (true) { + // Check cancellation flag + { + std::unique_lock l(_lock); + if (_cancelled) { + break; + } + } + + if (left_time <= 0) { + break; + } + + // Round-robin through all active shards + for (auto it = _consuming_shard_ids.begin(); it != _consuming_shard_ids.end() && !done;) { + const std::string& shard_id = *it; + auto iter_it = _shard_iterators.find(shard_id); + + if (iter_it == _shard_iterators.end() || iter_it->second.empty()) { + // Shard exhausted (closed due to split/merge), remove from active set + LOG(INFO) << "Shard exhausted: " << shard_id; + it = _consuming_shard_ids.erase(it); + continue; + } + + consumer_watch.start(); + + Aws::Kinesis::Model::GetRecordsRequest request; + + DCHECK(_kinesis_conf != nullptr); + st = _kinesis_conf->apply_to_get_records_request(request, iter_it->second); + if (!st.ok()) { + LOG(WARNING) << "Failed to apply Kinesis config to GetRecordsRequest: " << st; + done = true; + break; + } + + auto outcome = _kinesis_client->GetRecords(request); + consumer_watch.stop(); + + // Track generic routine load metrics and Kinesis-specific metrics. + DorisMetrics::instance()->routine_load_get_msg_count->increment(1); + DorisMetrics::instance()->routine_load_get_msg_latency->increment( + consumer_watch.elapsed_time() / 1000 / 1000); + DorisMetrics::instance()->routine_load_kinesis_get_records_count->increment(1); + DorisMetrics::instance()->routine_load_kinesis_get_records_latency->increment( + consumer_watch.elapsed_time() / 1000 / 1000); + + if (!outcome.IsSuccess()) { + auto& error = outcome.GetError(); + + // Handle throttling (ProvisionedThroughputExceededException) + if (error.GetErrorType() == + Aws::Kinesis::KinesisErrors::PROVISIONED_THROUGHPUT_EXCEEDED) { + DorisMetrics::instance()->routine_load_kinesis_throttle_count->increment(1); + LOG(INFO) << "Kinesis rate limit exceeded for shard: " << shard_id + << ", throttle_count: " << throttle_backoff.throttle_count() + << ", backing off"; + throttle_backoff.backoff_and_sleep(); + ++it; // Move to next shard, will retry this one next round + continue; + } + + // Handle retriable errors + if (_is_retriable_error(error)) { + DorisMetrics::instance()->routine_load_kinesis_retriable_error_count->increment( + 1); + LOG(INFO) << "Kinesis retriable error for shard " << shard_id << ": " + << error.GetMessage() + << ", retry times: " << retry_policy.retry_count(); + if (retry_policy.should_retry()) { + retry_policy.retry_with_backoff(); + continue; + } + } + + // Fatal error + LOG(WARNING) << "Kinesis consume failed for shard " << shard_id << ": " + << error.GetMessage() << " (" << static_cast(error.GetErrorType()) + << ")"; + st = Status::InternalError("Kinesis GetRecords failed for shard {}: {}", shard_id, + error.GetMessage()); + done = true; + break; + } + + // Reset retry counter on success + retry_policy.reset(); + throttle_backoff.reset(); + + // Process records - move result to allow moving individual records + auto result = outcome.GetResultWithOwnership(); + auto millis_behind = result.GetMillisBehindLatest(); + std::string next_iterator = result.GetNextShardIterator(); + size_t record_count = result.GetRecords().size(); + RETURN_IF_ERROR(_process_records(shard_id, std::move(result), queue, &received_rows, + &put_rows)); + + // Track MillisBehindLatest for this shard (used by FE for lag monitoring & scheduling) + _millis_behind_latest[shard_id] = millis_behind; + + // Update shard iterator for next call + if (next_iterator.empty()) { + // Shard is closed (split/merge), mark as closed and remove from active set + LOG(INFO) << "Shard closed: " << shard_id << " (split/merge detected)"; + DorisMetrics::instance()->routine_load_kinesis_closed_shard_count->increment(1); + _closed_shard_ids.insert(shard_id); + _shard_iterators.erase(shard_id); + it = _consuming_shard_ids.erase(it); + } else { + // Update iterator for next consumption + _shard_iterators[shard_id] = next_iterator; + + if (record_count == 0) { + // No records in this batch - shard has caught up with latest data + // Remove from active set for this round (similar to Kafka PARTITION_EOF) + // but keep iterator and progress for next task execution + LOG(INFO) << "Shard has no new data: " << shard_id + << " (MillisBehindLatest=" << millis_behind << ")"; + it = _consuming_shard_ids.erase(it); + } else { + ++it; + } + } + + // Check if all shards are exhausted + if (_consuming_shard_ids.empty()) { + LOG(INFO) << "All shards exhausted for consumer: " << _id; + done = true; + break; + } + + // Small sleep to avoid tight loop + std::this_thread::sleep_for(std::chrono::milliseconds(INTER_SHARD_SLEEP_MS)); + } + + // Ensure minimum interval between rounds to respect Kinesis rate limits (5 GetRecords/sec per shard) + std::this_thread::sleep_for(std::chrono::milliseconds(MIN_INTERVAL_BETWEEN_ROUNDS_MS)); + + left_time = max_running_time_ms - watch.elapsed_time() / 1000 / 1000; + if (done) { + break; + } + } + + LOG(INFO) << "Kinesis consumer done: " << _id << ", grp: " << _grp_id + << ". cancelled: " << _cancelled << ", left time(ms): " << left_time + << ", total cost(ms): " << watch.elapsed_time() / 1000 / 1000 + << ", consume cost(ms): " << consumer_watch.elapsed_time() / 1000 / 1000 + << ", received rows: " << received_rows << ", put rows: " << put_rows; + + return st; +} + +Status KinesisDataConsumer::_process_records( + const std::string& shard_id, Aws::Kinesis::Model::GetRecordsResult result, + BlockingQueue>* queue, int64_t* received_rows, + int64_t* put_rows) { + // result is owned by value, safe to get mutable access to its records + auto records = + std::move(const_cast&>(result.GetRecords())); + for (auto& record : records) { + DorisMetrics::instance()->routine_load_consume_bytes->increment( + record.GetData().GetLength()); + + if (record.GetData().GetLength() == 0) { + // Skip empty records + continue; + } + + // Track the last sequence number for this shard + _committed_sequence_numbers[shard_id] = record.GetSequenceNumber(); + + // Move record into shared_ptr to avoid expensive copy + auto record_ptr = std::make_shared(std::move(record)); + + if (!queue->controlled_blocking_put(record_ptr, + config::blocking_queue_cv_wait_timeout_ms)) { + // Queue shutdown + return Status::InternalError("Queue shutdown during record processing"); + } + + (*put_rows)++; + (*received_rows)++; + DorisMetrics::instance()->routine_load_consume_rows->increment(1); + } + + return Status::OK(); +} + +bool KinesisDataConsumer::_is_retriable_error( + const Aws::Client::AWSError& error) { + auto error_type = error.GetErrorType(); + + return error_type == Aws::Kinesis::KinesisErrors::PROVISIONED_THROUGHPUT_EXCEEDED || + error_type == Aws::Kinesis::KinesisErrors::SERVICE_UNAVAILABLE || + error_type == Aws::Kinesis::KinesisErrors::INTERNAL_FAILURE || + error_type == Aws::Kinesis::KinesisErrors::NETWORK_CONNECTION || error.ShouldRetry(); +} + +Status KinesisDataConsumer::reset() { + std::unique_lock l(_lock); + _cancelled = false; + _consuming_shard_ids.clear(); + _shard_iterators.clear(); + _millis_behind_latest.clear(); + _committed_sequence_numbers.clear(); + _closed_shard_ids.clear(); + _last_visit_time = time(nullptr); + LOG(INFO) << "Kinesis consumer reset: " << _id; + return Status::OK(); +} + +Status KinesisDataConsumer::cancel(std::shared_ptr ctx) { + std::unique_lock l(_lock); + if (!_init) { + return Status::InternalError("Kinesis consumer is not initialized"); + } + _cancelled = true; + LOG(INFO) << "Kinesis consumer cancelled: " << _id << ", " << ctx->brief(); + return Status::OK(); +} + +bool KinesisDataConsumer::match(std::shared_ptr ctx) { + if (ctx->load_src_type != TLoadSourceType::KINESIS) { return false; } - for (auto& item : ctx->kafka_info->properties) { - std::unordered_map::const_iterator itr = - _custom_properties.find(item.first); - if (itr == _custom_properties.end()) { - return false; + + if (_region != ctx->kinesis_info->region || _stream != ctx->kinesis_info->stream || + _endpoint != ctx->kinesis_info->endpoint) { + return false; + } + + // Check that properties match + return PropertyMatcher::properties_match(_custom_properties, ctx->kinesis_info->properties); +} + +Status KinesisDataConsumer::get_shard_list(std::vector* shard_ids) { + DORIS_CHECK(_kinesis_client); + + // If user specified explicit shards, return those + if (!_explicit_shards.empty()) { + *shard_ids = _explicit_shards; + LOG(INFO) << "Using " << shard_ids->size() << " explicit shards for stream: " << _stream; + return Status::OK(); + } + + // Discover all shards + Aws::Kinesis::Model::ListShardsRequest request; + + DCHECK(_kinesis_conf != nullptr); + Status st = _kinesis_conf->apply_to_list_shards_request(request, _stream); + if (!st.ok()) { + return Status::InternalError("Failed to apply Kinesis config to ListShardsRequest: {}", + st.to_string()); + } + + // Only return OPEN shards here. FE will keep recently retired parent shards in its + // closed list until they are fully drained, then remove them permanently. Returning + // CLOSED shards from ListShards would make already-drained parents look newly discovered + // and cause them to restart from TRIM_HORIZON. + std::vector discovered_shard_ids; + bool saw_any_shard = false; + while (true) { + auto outcome = _kinesis_client->ListShards(request); + if (!outcome.IsSuccess()) { + auto& error = outcome.GetError(); + return Status::InternalError("Failed to list shards for stream {}: {} ({})", _stream, + error.GetMessage(), + static_cast(error.GetErrorType())); } - if (itr->second != item.second) { - return false; + const auto& result = outcome.GetResult(); + if (!result.GetShards().empty()) { + saw_any_shard = true; } + for (const auto& shard : result.GetShards()) { + const auto& ending_sequence_number = + shard.GetSequenceNumberRange().GetEndingSequenceNumber(); + if (!ending_sequence_number.empty()) { + continue; + } + discovered_shard_ids.emplace_back(shard.GetShardId()); + } + + const Aws::String& next_token = result.GetNextToken(); + if (next_token.empty()) { + break; + } + + Aws::Kinesis::Model::ListShardsRequest next_request; + // AWS requires paginated ListShards requests to use NextToken instead of StreamName. + next_request.SetNextToken(next_token); + if (request.MaxResultsHasBeenSet()) { + next_request.SetMaxResults(request.GetMaxResults()); + } + request = std::move(next_request); + } + + if (discovered_shard_ids.empty() && !saw_any_shard) { + return Status::InternalError("No shards found in Kinesis stream: {}", _stream); } - return true; + + *shard_ids = std::move(discovered_shard_ids); + LOG(INFO) << "Found " << shard_ids->size() << " open shards in stream: " << _stream; + return Status::OK(); } } // end namespace doris diff --git a/be/src/load/routine_load/data_consumer.h b/be/src/load/routine_load/data_consumer.h index 2be8e3d23ec2f7..67ccc0e1bda5d6 100644 --- a/be/src/load/routine_load/data_consumer.h +++ b/be/src/load/routine_load/data_consumer.h @@ -17,6 +17,12 @@ #pragma once +#include +#include +#include +#include +#include +#include #include #include @@ -31,6 +37,7 @@ #include "common/logging.h" #include "common/status.h" #include "librdkafka/rdkafkacpp.h" +#include "load/routine_load/kinesis_conf.h" #include "load/stream_load/stream_load_context.h" #include "runtime/aws_msk_iam_auth.h" #include "util/uid_util.h" @@ -173,4 +180,111 @@ class KafkaDataConsumer : public DataConsumer { std::unique_ptr _aws_msk_oauth_callback; }; +// AWS Kinesis Data Consumer +// Consumes data from AWS Kinesis Data Streams for routine load jobs. +// Kinesis is similar to Kafka but uses shards instead of partitions +// and sequence numbers (strings) instead of offsets (integers). +class KinesisDataConsumer : public DataConsumer { +public: + KinesisDataConsumer(std::shared_ptr ctx); + virtual ~KinesisDataConsumer(); + + // DataConsumer interface implementation + Status init(std::shared_ptr ctx) override; + Status consume(std::shared_ptr ctx) override { return Status::OK(); } + Status cancel(std::shared_ptr ctx) override; + Status reset() override; + bool match(std::shared_ptr ctx) override; + + // Kinesis-specific methods + // Assign shards with their starting sequence numbers + Status assign_shards(const std::map& shard_sequence_numbers, + const std::string& stream_name, std::shared_ptr ctx); + + // Main consumption loop - pulls records from all assigned shards + Status group_consume(BlockingQueue>* queue, + int64_t max_running_time_ms); + + // Get list of shard IDs + Status get_shard_list(std::vector* shard_ids); + +private: + // Configuration - Basic AWS settings + std::string _region; + std::string _stream; + std::string _endpoint; // Optional custom endpoint (e.g., LocalStack) + + // Type 1: Doris-internal parameters (not passed to AWS SDK) + std::unordered_map _doris_internal_properties; + + // Type 2: Frequently-used AWS parameters (explicit members for performance) + // These are parsed from aws.kinesis.* properties during init() + std::vector _explicit_shards; // aws.kinesis.shards (comma-separated) + std::string _default_position; // aws.kinesis.default.pos (LATEST/TRIM_HORIZON) + std::map + _shard_positions; // aws.kinesis.shards.pos (shard_id:position) + + // Type 3: Less-frequently-used AWS API parameters (wrapped in KinesisConf) + std::unique_ptr _kinesis_conf; + + // AWS credentials and other properties + std::unordered_map _custom_properties; + + // Active shards being consumed + std::set _consuming_shard_ids; + + // AWS Kinesis client + std::shared_ptr _kinesis_client; + + // Shard iterator management + // Kinesis requires shard iterators to consume records + // shard_id -> current shard iterator + std::map _shard_iterators; + + // Tracks the MillisBehindLatest value per shard from the last GetRecords call. + // Updated during group_consume; read by the task executor to populate ctx after consumption. + std::map _millis_behind_latest; + + // Tracks the last consumed sequence number per shard. + // Updated during group_consume via _process_records; read by the consumer group + // to populate ctx->kinesis_info->cmt_sequence_number after consumption. + std::map _committed_sequence_numbers; + + // Tracks shards that have been closed (split/merge) during consumption. + // FE should remove these shards from its tracking to avoid reassigning them. + std::set _closed_shard_ids; + +public: + // Returns the MillisBehindLatest snapshot collected during group_consume. + const std::map& get_millis_behind_latest() const { + return _millis_behind_latest; + } + + // Returns the committed sequence numbers per shard collected during group_consume. + const std::map& get_committed_sequence_numbers() const { + return _committed_sequence_numbers; + } + + // Returns the set of closed shard IDs detected during group_consume. + const std::set& get_closed_shard_ids() const { return _closed_shard_ids; } + +private: + // Helper methods + // Create and configure AWS Kinesis client with credentials + Status _create_kinesis_client(std::shared_ptr ctx); + + // Get shard iterator for a shard at a specific sequence number position + Status _get_shard_iterator(const std::string& shard_id, const std::string& sequence_number, + std::string* iterator); + + // Process records from GetRecords result and add to queue + Status _process_records(const std::string& shard_id, + Aws::Kinesis::Model::GetRecordsResult result, + BlockingQueue>* queue, + int64_t* received_rows, int64_t* put_rows); + + // Check if an AWS error is retriable (throttling, network, etc.) + bool _is_retriable_error(const Aws::Client::AWSError& error); +}; + } // end namespace doris diff --git a/be/src/load/routine_load/data_consumer_group.cpp b/be/src/load/routine_load/data_consumer_group.cpp index b6b79bfc2e028c..2a08c6bc3f62a5 100644 --- a/be/src/load/routine_load/data_consumer_group.cpp +++ b/be/src/load/routine_load/data_consumer_group.cpp @@ -21,11 +21,13 @@ #include #include -#include #include #include "common/logging.h" +#include "io/fs/kafka_consumer_pipe.h" +#include "io/fs/kinesis_consumer_pipe.h" #include "librdkafka/rdkafkacpp.h" +#include "load/routine_load/consumer_group_helpers.h" #include "load/routine_load/data_consumer.h" #include "load/stream_load/stream_load_context.h" #include "util/stopwatch.hpp" @@ -33,19 +35,85 @@ namespace doris { #include "common/compile_check_begin.h" +bool DataConsumerGroup::_submit_all_consumers( + std::function, ConsumeFinishCallback)> consume_fn, + std::function shutdown_fn, Status& result_st) { + for (auto& consumer : _consumers) { + auto cb = [this, shutdown_fn, &result_st](const Status& st) { + std::unique_lock lock(_mutex); + if (--_counter == 0) { + shutdown_fn(); + LOG(INFO) << "all consumers finished, shutdown queue. grp: " << _grp_id; + } + if (result_st.ok() && !st.ok()) { + result_st = st; + } + }; + if (!_thread_pool.offer([consume_fn, consumer, cb] { consume_fn(consumer, cb); })) { + LOG(WARNING) << "failed to submit consumer: " << consumer->id() << ", grp: " << _grp_id; + return false; + } + VLOG_CRITICAL << "submit consumer: " << consumer->id() << ", grp: " << _grp_id; + } + return true; +} + +Status DataConsumerGroup::_run_consume_loop(std::shared_ptr ctx, + std::shared_ptr pipe, + Status& result_st) { + int64_t left_time = ctx->max_interval_s * 1000; + int64_t left_rows = ctx->max_batch_rows; + int64_t left_bytes = ctx->max_batch_size; + + LOG(INFO) << "start consumer group: " << _grp_id << ". max time(ms): " << left_time + << ", batch rows: " << left_rows << ", batch size: " << left_bytes << ". " + << ctx->brief(); + + MonotonicStopWatch watch; + watch.start(); + bool eos = false; + while (true) { + if (eos || left_time <= 0 || left_rows <= 0 || left_bytes <= 0) { + LOG(INFO) << "consumer group done: " << _grp_id + << ". consume time(ms)=" << ctx->max_interval_s * 1000 - left_time + << ", received rows=" << ctx->max_batch_rows - left_rows + << ", received bytes=" << ctx->max_batch_size - left_bytes << ", eos: " << eos + << ", left_time: " << left_time << ", left_rows: " << left_rows + << ", left_bytes: " << left_bytes + << ", blocking get time(us): " << pipe->get_queue_size() << ", " + << ctx->brief(); + + _shutdown_queue(); + for (auto& consumer : _consumers) { + static_cast(consumer->cancel(ctx)); + } + _thread_pool.shutdown(); + _thread_pool.join(); + if (!result_st.ok()) { + pipe->cancel(result_st.to_string()); + return result_st; + } + RETURN_IF_ERROR(pipe->finish()); + _on_finish(ctx); + ctx->receive_bytes = ctx->max_batch_size - left_bytes; + return Status::OK(); + } + + if (!_dequeue_and_process(pipe.get(), left_rows, left_bytes, result_st)) { + eos = true; + } + left_time = ctx->max_interval_s * 1000 - watch.elapsed_time() / 1000 / 1000; + } +} + Status KafkaDataConsumerGroup::assign_topic_partitions(std::shared_ptr ctx) { DCHECK(ctx->kafka_info); DCHECK(_consumers.size() >= 1); - // divide partitions + // divide partitions using round-robin partitioner int consumer_size = doris::cast_set(_consumers.size()); - std::vector> divide_parts(consumer_size); - int i = 0; - for (auto& kv : ctx->kafka_info->begin_offset) { - int idx = i % consumer_size; - divide_parts[idx].emplace(kv.first, kv.second); - i++; - } + auto divide_parts = WorkPartitioner::partition_round_robin( + ctx->kafka_info->begin_offset, consumer_size); // assign partitions to consumers equally for (int j = 0; j < consumer_size; ++j) { @@ -73,141 +141,174 @@ KafkaDataConsumerGroup::~KafkaDataConsumerGroup() { } Status KafkaDataConsumerGroup::start_all(std::shared_ptr ctx, - std::shared_ptr kafka_pipe) { + std::shared_ptr pipe) { + DORIS_CHECK(std::dynamic_pointer_cast(pipe) != nullptr); Status result_st = Status::OK(); - // start all consumers - for (auto& consumer : _consumers) { - if (!_thread_pool.offer(std::bind( - &KafkaDataConsumerGroup::actual_consume, this, consumer, &_queue, - ctx->max_interval_s * 1000, [this, &result_st](const Status& st) { - std::unique_lock lock(_mutex); - _counter--; - VLOG_CRITICAL << "group counter is: " << _counter << ", grp: " << _grp_id; - if (_counter == 0) { - _queue.shutdown(); - LOG(INFO) << "all consumers are finished. shutdown queue. group id: " - << _grp_id; - } - if (result_st.ok() && !st.ok()) { - result_st = st; - } - }))) { - LOG(WARNING) << "failed to submit data consumer: " << consumer->id() - << ", group id: " << _grp_id; - return Status::InternalError("failed to submit data consumer"); - } else { - VLOG_CRITICAL << "submit a data consumer: " << consumer->id() - << ", group id: " << _grp_id; + _cmt_offset = ctx->kafka_info->cmt_offset; + _format = ctx->format; + + if (!_submit_all_consumers( + [this, max_time = ctx->max_interval_s * 1000](std::shared_ptr c, + ConsumeFinishCallback cb) { + actual_consume(c, &_queue, max_time, cb); + }, + [this] { _queue.shutdown(); }, result_st)) { + return Status::InternalError("failed to submit data consumer"); + } + RETURN_IF_ERROR(_run_consume_loop(ctx, pipe, result_st)); + ctx->kafka_info->cmt_offset = std::move(_cmt_offset); + return Status::OK(); +} + +bool KafkaDataConsumerGroup::_dequeue_and_process(io::StreamLoadPipe* pipe, int64_t& left_rows, + int64_t& left_bytes, Status& result_st) { + RdKafka::Message* msg = nullptr; + if (!_queue.controlled_blocking_get(&msg, config::blocking_queue_cv_wait_timeout_ms)) { + return false; + } + Defer delete_msg {[msg] { delete msg; }}; + VLOG_NOTICE << "get kafka message, partition: " << msg->partition() + << ", offset: " << msg->offset() << ", len: " << msg->len(); + + if (msg->err() == RdKafka::ERR__PARTITION_EOF) { + if (msg->offset() > 0) { + _cmt_offset[msg->partition()] = msg->offset() - 1; } + return true; } - // consuming from queue and put data to stream load pipe - int64_t left_time = ctx->max_interval_s * 1000; - int64_t left_rows = ctx->max_batch_rows; - int64_t left_bytes = ctx->max_batch_size; + auto append_fn = FormatAppender::get_append_function(_format); + Status st = (pipe->*append_fn)(static_cast(msg->payload()), + static_cast(msg->len())); + if (st.ok()) { + left_rows--; + left_bytes -= msg->len(); + _cmt_offset[msg->partition()] = msg->offset(); + VLOG_NOTICE << "consume partition[" << msg->partition() << " - " << msg->offset() << "]"; + } else { + LOG(WARNING) << "failed to append msg to pipe. grp: " << _grp_id; + std::unique_lock lock(_mutex); + if (result_st.ok()) { + result_st = st; + } + } + return true; +} - LOG(INFO) << "start consumer group: " << _grp_id << ". max time(ms): " << left_time - << ", batch rows: " << left_rows << ", batch size: " << left_bytes << ". " - << ctx->brief(); +void KafkaDataConsumerGroup::_on_finish(std::shared_ptr ctx) { + // cmt_offset is moved back in start_all after _run_consume_loop returns +} - // copy one - std::map cmt_offset = ctx->kafka_info->cmt_offset; +void KafkaDataConsumerGroup::actual_consume(std::shared_ptr consumer, + BlockingQueue* queue, + int64_t max_running_time_ms, ConsumeFinishCallback cb) { + Status st = std::static_pointer_cast(consumer)->group_consume( + queue, max_running_time_ms); + cb(st); +} - //improve performance - Status (io::KafkaConsumerPipe::*append_data)(const char* data, size_t size); - if (ctx->format == TFileFormatType::FORMAT_JSON) { - append_data = &io::KafkaConsumerPipe::append_json; - } else { - append_data = &io::KafkaConsumerPipe::append_with_line_delimiter; +Status KinesisDataConsumerGroup::assign_stream_shards(std::shared_ptr ctx) { + DCHECK(ctx->kinesis_info); + DCHECK(_consumers.size() >= 1); + + // divide shards using round-robin partitioner + int consumer_size = doris::cast_set(_consumers.size()); + auto divide_shards = WorkPartitioner::partition_round_robin( + ctx->kinesis_info->begin_sequence_number, consumer_size); + + // assign shards to consumers equally + for (int j = 0; j < consumer_size; ++j) { + RETURN_IF_ERROR(std::static_pointer_cast(_consumers[j]) + ->assign_shards(divide_shards[j], ctx->kinesis_info->stream, ctx)); } - MonotonicStopWatch watch; - watch.start(); - bool eos = false; + return Status::OK(); +} + +KinesisDataConsumerGroup::~KinesisDataConsumerGroup() { + _queue.shutdown(); while (true) { - if (eos || left_time <= 0 || left_rows <= 0 || left_bytes <= 0) { - LOG(INFO) << "consumer group done: " << _grp_id - << ". consume time(ms)=" << ctx->max_interval_s * 1000 - left_time - << ", received rows=" << ctx->max_batch_rows - left_rows - << ", received bytes=" << ctx->max_batch_size - left_bytes << ", eos: " << eos - << ", left_time: " << left_time << ", left_rows: " << left_rows - << ", left_bytes: " << left_bytes - << ", blocking get time(us): " << _queue.total_get_wait_time() / 1000 - << ", blocking put time(us): " << _queue.total_put_wait_time() / 1000 << ", " - << ctx->brief(); + std::shared_ptr record; + if (_queue.blocking_get(&record)) { + record.reset(); + } else { + break; + } + } + DCHECK(_queue.get_size() == 0); +} - // shutdown queue - _queue.shutdown(); - // cancel all consumers - for (auto& consumer : _consumers) { - static_cast(consumer->cancel(ctx)); - } +Status KinesisDataConsumerGroup::start_all(std::shared_ptr ctx, + std::shared_ptr pipe) { + DORIS_CHECK(std::dynamic_pointer_cast(pipe) != nullptr); + Status result_st = Status::OK(); + _format = ctx->format; - // waiting all threads finished - _thread_pool.shutdown(); - _thread_pool.join(); - if (!result_st.ok()) { - kafka_pipe->cancel(result_st.to_string()); - return result_st; - } - static_cast(kafka_pipe->finish()); - ctx->kafka_info->cmt_offset = std::move(cmt_offset); - ctx->receive_bytes = ctx->max_batch_size - left_bytes; - return Status::OK(); + if (!_submit_all_consumers( + [this, max_time = ctx->max_interval_s * 1000](std::shared_ptr c, + ConsumeFinishCallback cb) { + actual_consume(c, &_queue, max_time, cb); + }, + [this] { _queue.shutdown(); }, result_st)) { + return Status::InternalError("failed to submit kinesis data consumer"); + } + return _run_consume_loop(ctx, pipe, result_st); +} + +bool KinesisDataConsumerGroup::_dequeue_and_process(io::StreamLoadPipe* pipe, int64_t& left_rows, + int64_t& left_bytes, Status& result_st) { + std::shared_ptr record; + if (!_queue.controlled_blocking_get(&record, config::blocking_queue_cv_wait_timeout_ms)) { + return false; + } + auto& data = record->GetData(); + const char* payload = reinterpret_cast(data.GetUnderlyingData()); + size_t len = data.GetLength(); + VLOG_NOTICE << "get kinesis record, seq: " << record->GetSequenceNumber() << ", len: " << len; + + auto append_fn = FormatAppender::get_append_function(_format); + Status st = (pipe->*append_fn)(payload, len); + if (st.ok()) { + left_rows--; + left_bytes -= len; + VLOG_NOTICE << "consume kinesis record [seq=" << record->GetSequenceNumber() << "]"; + } else { + LOG(WARNING) << "failed to append kinesis record to pipe. grp: " << _grp_id; + std::unique_lock lock(_mutex); + if (result_st.ok()) { + result_st = st; } + } + return true; +} - RdKafka::Message* msg; - bool res = _queue.controlled_blocking_get(&msg, config::blocking_queue_cv_wait_timeout_ms); - if (res) { - // conf has to be deleted finally - Defer delete_msg {[msg]() { delete msg; }}; - VLOG_NOTICE << "get kafka message" - << ", partition: " << msg->partition() << ", offset: " << msg->offset() - << ", len: " << msg->len(); - - if (msg->err() == RdKafka::ERR__PARTITION_EOF) { - if (msg->offset() > 0) { - cmt_offset[msg->partition()] = msg->offset() - 1; - } - } else { - Status st = (kafka_pipe.get()->*append_data)( - static_cast(msg->payload()), static_cast(msg->len())); - if (st.ok()) { - left_rows--; - left_bytes -= msg->len(); - cmt_offset[msg->partition()] = msg->offset(); - VLOG_NOTICE << "consume partition[" << msg->partition() << " - " - << msg->offset() << "]"; - } else { - // failed to append this msg, we must stop - LOG(WARNING) << "failed to append msg to pipe. grp: " << _grp_id; - eos = true; - { - std::unique_lock lock(_mutex); - if (result_st.ok()) { - result_st = st; - } - } - } +void KinesisDataConsumerGroup::_on_finish(std::shared_ptr ctx) { + for (auto& consumer : _consumers) { + auto kinesis_consumer = std::static_pointer_cast(consumer); + for (auto& [shard_id, seq_num] : kinesis_consumer->get_committed_sequence_numbers()) { + ctx->kinesis_info->cmt_sequence_number[shard_id] = seq_num; + } + for (auto& [shard_id, millis] : kinesis_consumer->get_millis_behind_latest()) { + auto [it, inserted] = ctx->kinesis_info->millis_behind_latest.emplace(shard_id, millis); + if (!inserted && it->second < millis) { + it->second = millis; } - } else { - // queue is empty and shutdown - eos = true; } - - left_time = ctx->max_interval_s * 1000 - watch.elapsed_time() / 1000 / 1000; + for (auto& shard_id : kinesis_consumer->get_closed_shard_ids()) { + ctx->kinesis_info->closed_shard_ids.insert(shard_id); + } } - - return Status::OK(); } -void KafkaDataConsumerGroup::actual_consume(std::shared_ptr consumer, - BlockingQueue* queue, - int64_t max_running_time_ms, ConsumeFinishCallback cb) { - Status st = std::static_pointer_cast(consumer)->group_consume( +void KinesisDataConsumerGroup::actual_consume( + std::shared_ptr consumer, + BlockingQueue>* queue, + int64_t max_running_time_ms, ConsumeFinishCallback cb) { + Status st = std::static_pointer_cast(consumer)->group_consume( queue, max_running_time_ms); cb(st); } + #include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/load/routine_load/data_consumer_group.h b/be/src/load/routine_load/data_consumer_group.h index bf59a6584b21ec..41a23b157bde70 100644 --- a/be/src/load/routine_load/data_consumer_group.h +++ b/be/src/load/routine_load/data_consumer_group.h @@ -26,7 +26,6 @@ #include "common/cast_set.h" #include "common/status.h" -#include "io/fs/kafka_consumer_pipe.h" #include "load/routine_load/data_consumer.h" #include "util/blocking_queue.hpp" #include "util/uid_util.h" @@ -67,11 +66,34 @@ class DataConsumerGroup { // start all consumers virtual Status start_all(std::shared_ptr ctx, - std::shared_ptr kafka_pipe) { + std::shared_ptr pipe) { return Status::OK(); } protected: + // Submit all consumers to thread pool. + // consume_fn: wraps actual_consume per consumer. + // shutdown_fn: called when last consumer finishes (shuts down queue). + // Returns false if any submission fails. + bool _submit_all_consumers( + std::function, ConsumeFinishCallback)> consume_fn, + std::function shutdown_fn, Status& result_st); + + // Shared consumption loop skeleton. Calls _dequeue_and_process per iteration. + Status _run_consume_loop(std::shared_ptr ctx, + std::shared_ptr pipe, Status& result_st); + + // Dequeue one item and append to pipe. Update left_rows/left_bytes. + // Returns false → queue empty/shutdown (eos). Returns true → continue. + virtual bool _dequeue_and_process(io::StreamLoadPipe* pipe, int64_t& left_rows, + int64_t& left_bytes, Status& result_st) = 0; + + // Shutdown the subclass queue. Called at loop exit. + virtual void _shutdown_queue() = 0; + + // Called after successful finish. Override to collect post-consume state. + virtual void _on_finish(std::shared_ptr ctx) {} + UniqueId _grp_id; std::vector> _consumers; // thread pool to run each consumer in multi thread @@ -89,23 +111,55 @@ class KafkaDataConsumerGroup : public DataConsumerGroup { public: KafkaDataConsumerGroup(size_t consumer_num) : DataConsumerGroup(consumer_num), _queue(500) {} - virtual ~KafkaDataConsumerGroup(); + ~KafkaDataConsumerGroup() override; Status start_all(std::shared_ptr ctx, - std::shared_ptr kafka_pipe) override; + std::shared_ptr pipe) override; // assign topic partitions to all consumers equally Status assign_topic_partitions(std::shared_ptr ctx); -private: // start a single consumer void actual_consume(std::shared_ptr consumer, BlockingQueue* queue, int64_t max_running_time_ms, ConsumeFinishCallback cb); private: - // blocking queue to receive msgs from all consumers + bool _dequeue_and_process(io::StreamLoadPipe* pipe, int64_t& left_rows, int64_t& left_bytes, + Status& result_st) override; + void _shutdown_queue() override { _queue.shutdown(); } + void _on_finish(std::shared_ptr ctx) override; + BlockingQueue _queue; + std::map _cmt_offset; + TFileFormatType::type _format; +}; + +// for kinesis +class KinesisDataConsumerGroup : public DataConsumerGroup { +public: + KinesisDataConsumerGroup(size_t consumer_num) : DataConsumerGroup(consumer_num), _queue(500) {} + + ~KinesisDataConsumerGroup() override; + + Status start_all(std::shared_ptr ctx, + std::shared_ptr pipe) override; + + Status assign_stream_shards(std::shared_ptr ctx); + +private: + void actual_consume(std::shared_ptr consumer, + BlockingQueue>* queue, + int64_t max_running_time_ms, ConsumeFinishCallback cb); + + bool _dequeue_and_process(io::StreamLoadPipe* pipe, int64_t& left_rows, int64_t& left_bytes, + Status& result_st) override; + void _shutdown_queue() override { _queue.shutdown(); } + void _on_finish(std::shared_ptr ctx) override; + + BlockingQueue> _queue; + TFileFormatType::type _format; }; + #include "common/compile_check_end.h" } // end namespace doris diff --git a/be/src/load/routine_load/data_consumer_pool.cpp b/be/src/load/routine_load/data_consumer_pool.cpp index f08d56f8489949..3093467165542c 100644 --- a/be/src/load/routine_load/data_consumer_pool.cpp +++ b/be/src/load/routine_load/data_consumer_pool.cpp @@ -63,6 +63,9 @@ Status DataConsumerPool::get_consumer(std::shared_ptr ctx, case TLoadSourceType::KAFKA: consumer = std::make_shared(ctx); break; + case TLoadSourceType::KINESIS: + consumer = std::make_shared(ctx); + break; default: return Status::InternalError("PAUSE: unknown routine load task type: {}", ctx->load_type); } @@ -77,22 +80,32 @@ Status DataConsumerPool::get_consumer(std::shared_ptr ctx, Status DataConsumerPool::get_consumer_grp(std::shared_ptr ctx, std::shared_ptr* ret) { - if (ctx->load_src_type != TLoadSourceType::KAFKA) { - return Status::InternalError( - "PAUSE: Currently only support consumer group for Kafka data source"); + size_t consumer_num = config::max_consumer_num_per_group; + std::shared_ptr grp; + switch (ctx->load_src_type) { + case TLoadSourceType::KAFKA: { + DCHECK(ctx->kafka_info); + if (ctx->kafka_info->begin_offset.size() == 0) { + return Status::InternalError( + "PAUSE: The size of begin_offset of task should not be 0."); + } + consumer_num = std::min(consumer_num, ctx->kafka_info->begin_offset.size()); + grp = std::make_shared(consumer_num); + break; } - DCHECK(ctx->kafka_info); - - if (ctx->kafka_info->begin_offset.size() == 0) { - return Status::InternalError("PAUSE: The size of begin_offset of task should not be 0."); + case TLoadSourceType::KINESIS: { + DCHECK(ctx->kinesis_info); + if (ctx->kinesis_info->begin_sequence_number.size() == 0) { + return Status::InternalError( + "PAUSE: The size of begin_sequence_number of task should not be 0."); + } + consumer_num = std::min(consumer_num, ctx->kinesis_info->begin_sequence_number.size()); + grp = std::make_shared(consumer_num); + break; + } + default: + return Status::Cancelled("unknown routine load task type: {}", ctx->load_type); } - - // one data consumer group contains at least one data consumers. - int max_consumer_num = config::max_consumer_num_per_group; - size_t consumer_num = std::min((size_t)max_consumer_num, ctx->kafka_info->begin_offset.size()); - - std::shared_ptr grp = - std::make_shared(consumer_num); for (int i = 0; i < consumer_num; ++i) { std::shared_ptr consumer; diff --git a/be/src/load/routine_load/kinesis_conf.cpp b/be/src/load/routine_load/kinesis_conf.cpp new file mode 100644 index 00000000000000..d11dcfc215a29d --- /dev/null +++ b/be/src/load/routine_load/kinesis_conf.cpp @@ -0,0 +1,169 @@ +// 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 "load/routine_load/kinesis_conf.h" + +#include + +namespace doris { + +KinesisConf::ConfResult KinesisConf::set(const std::string& name, const std::string& value, + std::string& errstr) { + // Determine which API(s) this parameter belongs to based on its semantic meaning + // All parameters come in as simple names (e.g., "max_records", "stream_arn") + // after "aws.kinesis." prefix is removed in data_consumer.cpp + + if (name == "max_records") { + // GetRecords API parameter + int max_records; + if (!parse_int(value, max_records, errstr)) { + return CONF_INVALID; + } + if (max_records < 1 || max_records > 10000) { + errstr = "max_records must be between 1 and 10000"; + return CONF_INVALID; + } + _get_records_params[name] = value; + return CONF_OK; + } else if (name == "stream_arn") { + // Used by all three APIs + _get_records_params[name] = value; + _get_shard_iterator_params[name] = value; + _list_shards_params[name] = value; + return CONF_OK; + } else if (name == "timestamp") { + // GetShardIterator API parameter (for AT_TIMESTAMP iterator type) + long timestamp; + if (!parse_long(value, timestamp, errstr)) { + return CONF_INVALID; + } + _get_shard_iterator_params[name] = value; + return CONF_OK; + } else if (name == "max_results") { + // ListShards API parameter + int max_results; + if (!parse_int(value, max_results, errstr)) { + return CONF_INVALID; + } + if (max_results < 1 || max_results > 10000) { + errstr = "max_results must be between 1 and 10000"; + return CONF_INVALID; + } + _list_shards_params[name] = value; + return CONF_OK; + } + + VLOG_NOTICE << "Unknown Kinesis configuration: " << name; + return CONF_UNKNOWN; +} + +Status KinesisConf::apply_to_get_records_request(Aws::Kinesis::Model::GetRecordsRequest& request, + const std::string& shard_iterator) const { + request.SetShardIterator(shard_iterator); + + auto it = _get_records_params.find("max_records"); + if (it != _get_records_params.end()) { + try { + request.SetLimit(std::stoi(it->second)); + } catch (const std::exception&) { + return Status::InternalError("Failed to apply get_records.max_records: {}", it->second); + } + } + + it = _get_records_params.find("stream_arn"); + if (it != _get_records_params.end() && !it->second.empty()) { + request.SetStreamARN(it->second); + } + + return Status::OK(); +} + +Status KinesisConf::apply_to_get_shard_iterator_request( + Aws::Kinesis::Model::GetShardIteratorRequest& request, const std::string& stream_name, + const std::string& shard_id, const std::string& sequence_number) const { + request.SetStreamName(stream_name); + request.SetShardId(shard_id); + + if (sequence_number.empty() || sequence_number == "TRIM_HORIZON" || sequence_number == "-2") { + request.SetShardIteratorType(Aws::Kinesis::Model::ShardIteratorType::TRIM_HORIZON); + } else if (sequence_number == "LATEST" || sequence_number == "-1") { + request.SetShardIteratorType(Aws::Kinesis::Model::ShardIteratorType::LATEST); + } else { + request.SetShardIteratorType(Aws::Kinesis::Model::ShardIteratorType::AFTER_SEQUENCE_NUMBER); + request.SetStartingSequenceNumber(sequence_number); + } + + auto it = _get_shard_iterator_params.find("stream_arn"); + if (it != _get_shard_iterator_params.end() && !it->second.empty()) { + request.SetStreamARN(it->second); + } + + it = _get_shard_iterator_params.find("timestamp"); + if (it != _get_shard_iterator_params.end()) { + try { + request.SetTimestamp(Aws::Utils::DateTime(std::stol(it->second))); + } catch (const std::exception&) { + return Status::InternalError("Failed to apply get_shard_iterator.timestamp: {}", + it->second); + } + } + + return Status::OK(); +} + +Status KinesisConf::apply_to_list_shards_request(Aws::Kinesis::Model::ListShardsRequest& request, + const std::string& stream_name) const { + request.SetStreamName(stream_name); + + auto it = _list_shards_params.find("stream_arn"); + if (it != _list_shards_params.end() && !it->second.empty()) { + request.SetStreamARN(it->second); + } + + it = _list_shards_params.find("max_results"); + if (it != _list_shards_params.end()) { + try { + request.SetMaxResults(std::stoi(it->second)); + } catch (const std::exception&) { + return Status::InternalError("Failed to apply list_shards.max_results: {}", it->second); + } + } + + return Status::OK(); +} + +bool KinesisConf::parse_int(const std::string& value, int& result, std::string& errstr) const { + try { + result = std::stoi(value); + return true; + } catch (const std::exception&) { + errstr = "Invalid integer value: " + value; + return false; + } +} + +bool KinesisConf::parse_long(const std::string& value, long& result, std::string& errstr) const { + try { + result = std::stol(value); + return true; + } catch (const std::exception&) { + errstr = "Invalid long value: " + value; + return false; + } +} + +} // namespace doris diff --git a/be/src/load/routine_load/kinesis_conf.h b/be/src/load/routine_load/kinesis_conf.h new file mode 100644 index 00000000000000..abf001e490d8c7 --- /dev/null +++ b/be/src/load/routine_load/kinesis_conf.h @@ -0,0 +1,87 @@ +// 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 +#include + +namespace doris { + +/** + * KinesisConf - Configuration management for AWS Kinesis API parameters + * + * Manages less-frequently-used Kinesis API parameters, organized by API type. + * Parameters are categorized into three maps based on which API they apply to: + * - GetRecords API parameters (max_records, stream_arn) + * - GetShardIterator API parameters (stream_arn, timestamp) + * - ListShards API parameters (stream_arn, max_results) + * + * Note: Frequently-used parameters (stream, shards, positions) are stored + * as explicit members in KinesisDataConsumer for better performance. + */ +class KinesisConf { +public: + enum ConfResult { CONF_OK = 0, CONF_INVALID = 1, CONF_UNKNOWN = 2 }; + + KinesisConf() = default; + ~KinesisConf() = default; + + /** + * Set a configuration property for Kinesis APIs + * Automatically categorizes parameters by API type: + * + * GetRecords API: + * - get_records.max_records: Max records per call (1-10000) + * - get_records.stream_arn: StreamARN for enhanced fan-out + * + * GetShardIterator API: + * - get_shard_iterator.stream_arn: StreamARN for enhanced fan-out + * - get_shard_iterator.timestamp: Timestamp for AT_TIMESTAMP iterator type + * + * ListShards API: + * - list_shards.stream_arn: StreamARN for enhanced fan-out + * - list_shards.max_results: Max shards per call (1-10000) + */ + ConfResult set(const std::string& name, const std::string& value, std::string& errstr); + + Status apply_to_get_records_request(Aws::Kinesis::Model::GetRecordsRequest& request, + const std::string& shard_iterator) const; + + Status apply_to_get_shard_iterator_request( + Aws::Kinesis::Model::GetShardIteratorRequest& request, const std::string& stream_name, + const std::string& shard_id, const std::string& sequence_number) const; + + Status apply_to_list_shards_request(Aws::Kinesis::Model::ListShardsRequest& request, + const std::string& stream_name) const; + +private: + // Separate parameter maps for each API + std::map _get_records_params; + std::map _get_shard_iterator_params; + std::map _list_shards_params; + + bool parse_int(const std::string& value, int& result, std::string& errstr) const; + bool parse_long(const std::string& value, long& result, std::string& errstr) const; +}; + +} // namespace doris diff --git a/be/src/load/routine_load/routine_load_task_executor.cpp b/be/src/load/routine_load/routine_load_task_executor.cpp index 8364de65e037b7..1c1937673ea89b 100644 --- a/be/src/load/routine_load/routine_load_task_executor.cpp +++ b/be/src/load/routine_load/routine_load_task_executor.cpp @@ -41,6 +41,7 @@ #include "common/status.h" #include "common/utils.h" #include "io/fs/kafka_consumer_pipe.h" +#include "io/fs/kinesis_consumer_pipe.h" #include "io/fs/multi_table_pipe.h" #include "io/fs/stream_load_pipe.h" #include "load/message_body_sink.h" @@ -139,6 +140,48 @@ Status RoutineLoadTaskExecutor::get_kafka_partition_meta(const PKafkaMetaProxyRe return st; } +Status RoutineLoadTaskExecutor::_prepare_ctx(const PKinesisMetaProxyRequest& request, + std::shared_ptr ctx) { + ctx->load_type = TLoadType::ROUTINE_LOAD; + ctx->load_src_type = TLoadSourceType::KINESIS; + ctx->label = "NaN"; + + // convert PKinesisLoadInfo to TKinesisLoadInfo + TKinesisLoadInfo t_info; + t_info.region = request.kinesis_info().region(); + t_info.stream = request.kinesis_info().stream(); + if (request.kinesis_info().has_endpoint()) { + t_info.__set_endpoint(request.kinesis_info().endpoint()); + } + std::map properties; + for (int i = 0; i < request.kinesis_info().properties_size(); ++i) { + const PStringPair& pair = request.kinesis_info().properties(i); + properties.emplace(pair.key(), pair.val()); + } + t_info.__set_properties(std::move(properties)); + + ctx->kinesis_info.reset(new KinesisLoadInfo(t_info)); + ctx->need_rollback = false; + return Status::OK(); +} + +Status RoutineLoadTaskExecutor::get_kinesis_shard_meta(const PKinesisMetaProxyRequest& request, + std::vector* shard_ids) { + CHECK(request.has_kinesis_info()); + + std::shared_ptr ctx = std::make_shared(_exec_env); + RETURN_IF_ERROR(_prepare_ctx(request, ctx)); + + std::shared_ptr consumer; + RETURN_IF_ERROR(_data_consumer_pool.get_consumer(ctx, &consumer)); + + Status st = std::static_pointer_cast(consumer)->get_shard_list(shard_ids); + if (st.ok()) { + _data_consumer_pool.return_consumer(consumer); + } + return st; +} + Status RoutineLoadTaskExecutor::get_kafka_partition_offsets_for_times( const PKafkaMetaProxyRequest& request, std::vector* partition_offsets, int timeout) { @@ -315,6 +358,9 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { case TLoadSourceType::KAFKA: ctx->kafka_info.reset(new KafkaLoadInfo(task.kafka_load_info)); break; + case TLoadSourceType::KINESIS: + ctx->kinesis_info.reset(new KinesisLoadInfo(task.kinesis_load_info)); + break; default: LOG(WARNING) << "unknown load source type: " << task.type; return Status::InternalError("unknown load source type"); @@ -414,6 +460,25 @@ void RoutineLoadTaskExecutor::exec_task(std::shared_ptr ctx, } break; } + case TLoadSourceType::KINESIS: { + if (ctx->is_multi_table) { + err_handler(ctx, Status::Cancelled("Cancelled"), + "Kinesis doesn't support multi-table yet"); + cb(ctx); + return; + } else { + pipe = std::make_shared(); + } + Status st = std::static_pointer_cast(consumer_grp) + ->assign_stream_shards(ctx); + + if (!st.ok()) { + err_handler(ctx, st, st.to_string()); + cb(ctx); + return; + } + break; + } default: { std::stringstream ss; ss << "unknown routine load task type: " << ctx->load_type; @@ -441,15 +506,15 @@ void RoutineLoadTaskExecutor::exec_task(std::shared_ptr ctx, #endif } - std::shared_ptr kafka_pipe = - std::static_pointer_cast(ctx->body_sink); + pipe = std::static_pointer_cast(ctx->body_sink); + // Multi-table currently only supported for Kafka if (ctx->is_multi_table) { Status st; // plan the rest of unplanned data auto multi_table_pipe = std::static_pointer_cast(ctx->body_sink); // start to consume, this may block a while - st = consumer_grp->start_all(ctx, kafka_pipe); + st = consumer_grp->start_all(ctx, pipe); if (!st.ok()) { multi_table_pipe->handle_consume_finished(); HANDLE_MULTI_TABLE_ERROR(st, "consuming failed"); @@ -461,10 +526,10 @@ void RoutineLoadTaskExecutor::exec_task(std::shared_ptr ctx, } // need memory order multi_table_pipe->handle_consume_finished(); - HANDLE_MULTI_TABLE_ERROR(kafka_pipe->finish(), "finish multi table task failed"); + HANDLE_MULTI_TABLE_ERROR(pipe->finish(), "finish multi table task failed"); } else { // start to consume, this may block a while - HANDLE_ERROR(consumer_grp->start_all(ctx, kafka_pipe), "consuming failed"); + HANDLE_ERROR(consumer_grp->start_all(ctx, pipe), "consuming failed"); } // wait for all consumers finished @@ -516,6 +581,13 @@ void RoutineLoadTaskExecutor::exec_task(std::shared_ptr ctx, }}; break; } + case TLoadSourceType::KINESIS: { + // kinesis metrics/progress are aggregated in KinesisDataConsumerGroup::start_all() + LOG(INFO) << "Kinesis routine load task completed. Committed sequence numbers for " + << ctx->kinesis_info->cmt_sequence_number.size() + << " shards. Task: " << ctx->brief(); + break; + } default: break; } diff --git a/be/src/load/routine_load/routine_load_task_executor.h b/be/src/load/routine_load/routine_load_task_executor.h index 45fe1587bede22..a66bbe9dffef56 100644 --- a/be/src/load/routine_load/routine_load_task_executor.h +++ b/be/src/load/routine_load/routine_load_task_executor.h @@ -38,6 +38,7 @@ class StreamLoadContext; class TRoutineLoadTask; class PIntegerPair; class PKafkaMetaProxyRequest; +class PKinesisMetaProxyRequest; // A routine load task executor will receive routine load // tasks from FE, put it to a fixed thread pool. @@ -73,6 +74,9 @@ class RoutineLoadTaskExecutor { std::vector* partition_offsets, int timeout); + Status get_kinesis_shard_meta(const PKinesisMetaProxyRequest& request, + std::vector* shard_ids); + ThreadPool& get_thread_pool() { return *_thread_pool; } private: @@ -88,6 +92,10 @@ class RoutineLoadTaskExecutor { // create a dummy StreamLoadContext for PKafkaMetaProxyRequest Status _prepare_ctx(const PKafkaMetaProxyRequest& request, std::shared_ptr ctx); + + // create a dummy StreamLoadContext for PKinesisMetaProxyRequest + Status _prepare_ctx(const PKinesisMetaProxyRequest& request, + std::shared_ptr ctx); bool _reach_memory_limit(std::string& reason); private: diff --git a/be/src/load/stream_load/stream_load_context.h b/be/src/load/stream_load/stream_load_context.h index 4e257c86e7d9b4..2eab22dc7d4513 100644 --- a/be/src/load/stream_load/stream_load_context.h +++ b/be/src/load/stream_load/stream_load_context.h @@ -89,6 +89,42 @@ class KafkaLoadInfo { std::map properties; }; +// AWS Kinesis related info +class KinesisLoadInfo { +public: + KinesisLoadInfo(const TKinesisLoadInfo& t_info) + : region(t_info.region), + stream(t_info.stream), + endpoint(t_info.__isset.endpoint ? t_info.endpoint : ""), + begin_sequence_number(t_info.shard_begin_sequence_number), + properties(t_info.__isset.properties ? t_info.properties + : std::map()) {} + + void reset_sequence_numbers() { cmt_sequence_number.clear(); } + + std::string region; + std::string stream; + std::string endpoint; // Optional custom endpoint (for LocalStack/VPC) + + // The following members control the max progress of a consuming process + // If any of them reach, the consuming will finish. + int64_t max_interval_s = 5; + int64_t max_batch_rows = 1024; + int64_t max_batch_size = 100 * 1024 * 1024; // 100MB + + // shard_id -> begin sequence number, inclusive + std::map begin_sequence_number; + // shard_id -> committed sequence number (last consumed). + // This starts empty and is populated only after records are successfully consumed. + std::map cmt_sequence_number; + // shard_id -> millisBehindLatest from the last GetRecords call + std::map millis_behind_latest; + // Set of shard IDs that have been closed (split/merge) during consumption + std::set closed_shard_ids; + // AWS credentials and custom Kinesis properties + std::map properties; +}; + class MessageBodySink; class StreamLoadContext { @@ -235,6 +271,7 @@ class StreamLoadContext { std::string existing_job_status = ""; std::unique_ptr kafka_info; + std::unique_ptr kinesis_info; // consumer_id is used for data consumer cache key. // to identified a specified data consumer. diff --git a/be/src/load/stream_load/stream_load_executor.cpp b/be/src/load/stream_load/stream_load_executor.cpp index 08fc3bb34b15a9..478ed22755df9e 100644 --- a/be/src/load/stream_load/stream_load_executor.cpp +++ b/be/src/load/stream_load/stream_load_executor.cpp @@ -129,6 +129,9 @@ Status StreamLoadExecutor::execute_plan_fragment( case TLoadSourceType::KAFKA: ctx->kafka_info->reset_offset(); break; + case TLoadSourceType::KINESIS: + ctx->kinesis_info->reset_sequence_numbers(); + break; default: break; } @@ -450,6 +453,26 @@ bool StreamLoadExecutor::collect_load_stat(StreamLoadContext* ctx, TTxnCommitAtt } return true; } + case TLoadSourceType::KINESIS: { + TRLTaskTxnCommitAttachment& rl_attach = attach->rlTaskTxnCommitAttachment; + rl_attach.loadSourceType = TLoadSourceType::KINESIS; + + TKinesisRLTaskProgress kinesis_progress; + kinesis_progress.shardCmtSeqNum = ctx->kinesis_info->cmt_sequence_number; + if (!ctx->kinesis_info->millis_behind_latest.empty()) { + kinesis_progress.__set_shardMillsBehindLatest(ctx->kinesis_info->millis_behind_latest); + } + if (!ctx->kinesis_info->closed_shard_ids.empty()) { + kinesis_progress.__set_closedShardIds(ctx->kinesis_info->closed_shard_ids); + } + + rl_attach.kinesisRLTaskProgress = kinesis_progress; + rl_attach.__isset.kinesisRLTaskProgress = true; + if (!ctx->error_url.empty()) { + rl_attach.__set_errorLogUrl(ctx->error_url); + } + return true; + } default: return true; } diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index e4ac5b79bca7db..073e6d672389ab 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -1417,6 +1417,19 @@ void PInternalService::get_info(google::protobuf::RpcController* controller, return; } } + if (request->has_kinesis_meta_request()) { + std::vector shard_ids; + Status st = _exec_env->routine_load_task_executor()->get_kinesis_shard_meta( + request->kinesis_meta_request(), &shard_ids); + if (st.ok()) { + PKinesisMetaProxyResult* kinesis_result = response->mutable_kinesis_meta_result(); + for (const auto& shard_id : shard_ids) { + kinesis_result->add_shard_ids(shard_id); + } + } + st.to_protobuf(response->mutable_status()); + return; + } Status::OK().to_protobuf(response->mutable_status()); }); if (!ret) { diff --git a/be/src/util/s3_util.h b/be/src/util/s3_util.h index 5cedee997eeff4..139f424877bc2f 100644 --- a/be/src/util/s3_util.h +++ b/be/src/util/s3_util.h @@ -156,6 +156,9 @@ class S3ClientFactory { S3RateLimiterHolder* rate_limiter(S3RateLimitType type); + std::shared_ptr get_aws_credentials_provider( + const S3ClientConf& s3_conf); + #ifdef BE_TEST void set_client_creator_for_test( std::function(const S3ClientConf&)> creator); @@ -172,8 +175,6 @@ class S3ClientFactory { const S3ClientConf& s3_conf); std::shared_ptr _create_credentials_provider( CredProviderType type); - std::shared_ptr get_aws_credentials_provider( - const S3ClientConf& s3_conf); S3ClientFactory(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java index b947ef0649e501..97fb570b64763c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/cloud/transaction/TxnUtil.java @@ -34,8 +34,8 @@ import org.apache.doris.load.FailMsg; import org.apache.doris.load.loadv2.JobState; import org.apache.doris.load.loadv2.LoadJobFinalOperation; -import org.apache.doris.load.routineload.KafkaProgress; import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; +import org.apache.doris.load.routineload.kafka.KafkaProgress; import org.apache.doris.thrift.TEtlState; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.TransactionState; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/kinesis/KinesisUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/kinesis/KinesisUtil.java new file mode 100644 index 00000000000000..0e58633370ec83 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/kinesis/KinesisUtil.java @@ -0,0 +1,165 @@ +// 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.datasource.kinesis; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; +import org.apache.doris.common.LoadException; +import org.apache.doris.metric.MetricRepo; +import org.apache.doris.proto.InternalService; +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 org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class KinesisUtil { + private static final Logger LOG = LogManager.getLogger(KinesisUtil.class); + + /** + * Get all shard IDs for the given Kinesis stream. + * This delegates to a BE node which uses the AWS SDK to call ListShards API. + * + * @param region AWS region of the stream + * @param stream Kinesis stream name + * @param endpoint optional custom endpoint (e.g. LocalStack), empty string for default + * @param convertedCustomProperties AWS credentials and other properties + * @return list of shard IDs + */ + public static List getAllKinesisShards(String region, String stream, String endpoint, + Map convertedCustomProperties) throws LoadException { + try { + InternalService.PKinesisLoadInfo.Builder kinesisInfoBuilder = + InternalService.PKinesisLoadInfo.newBuilder() + .setRegion(region) + .setStream(stream) + .addAllProperties(convertedCustomProperties.entrySet().stream() + .map(e -> InternalService.PStringPair.newBuilder() + .setKey(e.getKey()) + .setVal(e.getValue()) + .build()) + .collect(Collectors.toList())); + if (endpoint != null && !endpoint.isEmpty()) { + kinesisInfoBuilder.setEndpoint(endpoint); + } + + InternalService.PProxyRequest request = InternalService.PProxyRequest.newBuilder() + .setKinesisMetaRequest( + InternalService.PKinesisMetaProxyRequest.newBuilder() + .setKinesisInfo(kinesisInfoBuilder)) + .setTimeoutSecs(Config.max_get_kafka_meta_timeout_second) + .build(); + + return getInfoRequest(request, Config.max_get_kafka_meta_timeout_second) + .getKinesisMetaResult().getShardIdsList(); + } catch (Exception e) { + throw new LoadException( + "Failed to get shards of Kinesis stream: " + stream + ". error: " + e.getMessage()); + } + } + + private static InternalService.PProxyResult getInfoRequest(InternalService.PProxyRequest request, + int timeout) throws LoadException { + long startTime = System.currentTimeMillis(); + int retryTimes = 0; + TNetworkAddress address = null; + Future future = null; + InternalService.PProxyResult result = null; + Set failedBeIds = new HashSet<>(); + TStatusCode code = null; + String errorMsg = null; + + try { + while (retryTimes < 3) { + List backendIds = new ArrayList<>(); + for (Long beId : Env.getCurrentSystemInfo().getAllBackendIds(true)) { + Backend backend = Env.getCurrentSystemInfo().getBackend(beId); + if (backend != null && backend.isLoadAvailable() + && !backend.isDecommissioned() + && !failedBeIds.contains(beId) + && !Env.getCurrentEnv().getRoutineLoadManager().isInBlacklist(beId)) { + backendIds.add(beId); + } + } + if (backendIds.isEmpty()) { + for (Long beId : Env.getCurrentEnv().getRoutineLoadManager().getBlacklist().keySet()) { + backendIds.add(beId); + } + } + if (backendIds.isEmpty()) { + MetricRepo.COUNTER_ROUTINE_LOAD_GET_META_FAIL_COUNT.increase(1L); + if (failedBeIds.isEmpty()) { + errorMsg = "no alive backends"; + } + throw new LoadException("failed to get info: " + errorMsg + ","); + } + Collections.shuffle(backendIds); + Backend be = Env.getCurrentSystemInfo().getBackend(backendIds.get(0)); + address = new TNetworkAddress(be.getHost(), be.getBrpcPort()); + long beId = be.getId(); + + try { + future = BackendServiceProxy.getInstance().getInfo(address, request); + result = future.get(timeout, TimeUnit.SECONDS); + } catch (Exception e) { + errorMsg = e.getMessage(); + LOG.warn("failed to get kinesis info request to {} err {}", address, e.getMessage()); + failedBeIds.add(beId); + retryTimes++; + continue; + } + code = TStatusCode.findByValue(result.getStatus().getStatusCode()); + if (code != TStatusCode.OK) { + errorMsg = result.getStatus().getErrorMsgsList().toString(); + LOG.warn("failed to get kinesis info request to {} err {}", address, + result.getStatus().getErrorMsgsList()); + failedBeIds.add(beId); + retryTimes++; + } else { + return result; + } + } + + MetricRepo.COUNTER_ROUTINE_LOAD_GET_META_FAIL_COUNT.increase(1L); + throw new LoadException("failed to get kinesis info: " + errorMsg + ","); + } finally { + if (code != null && code == TStatusCode.OK && !failedBeIds.isEmpty()) { + for (Long beId : failedBeIds) { + Env.getCurrentEnv().getRoutineLoadManager().addToBlacklist(beId); + LOG.info("add beId {} to blacklist, blacklist: {}", beId, + Env.getCurrentEnv().getRoutineLoadManager().getBlacklist()); + } + } + long endTime = System.currentTimeMillis(); + MetricRepo.COUNTER_ROUTINE_LOAD_GET_META_LANTENCY.increase(endTime - startTime); + MetricRepo.COUNTER_ROUTINE_LOAD_GET_META_COUNT.increase(1L); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/LoadDataSourceType.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/LoadDataSourceType.java index 0135f176b480ad..a31f997745e4e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/LoadDataSourceType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/LoadDataSourceType.java @@ -18,5 +18,6 @@ package org.apache.doris.load.routineload; public enum LoadDataSourceType { - KAFKA + KAFKA, + KINESIS } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java index 7b4f681cc42c1b..dd9c4ed85ce908 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RLTaskTxnCommitAttachment.java @@ -18,6 +18,8 @@ package org.apache.doris.load.routineload; import org.apache.doris.cloud.proto.Cloud.RLTaskTxnCommitAttachmentPB; +import org.apache.doris.load.routineload.kafka.KafkaProgress; +import org.apache.doris.load.routineload.kinesis.KinesisProgress; import org.apache.doris.thrift.TRLTaskTxnCommitAttachment; import org.apache.doris.thrift.TUniqueId; import org.apache.doris.transaction.TransactionState; @@ -63,6 +65,9 @@ public RLTaskTxnCommitAttachment(TRLTaskTxnCommitAttachment rlTaskTxnCommitAttac case KAFKA: this.progress = new KafkaProgress(rlTaskTxnCommitAttachment.getKafkaRLTaskProgress()); break; + case KINESIS: + this.progress = new KinesisProgress(rlTaskTxnCommitAttachment.getKinesisRLTaskProgress()); + break; default: break; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadDataSourcePropertyFactory.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadDataSourcePropertyFactory.java index ec0d3bbb954430..045616af29b48e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadDataSourcePropertyFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadDataSourcePropertyFactory.java @@ -18,6 +18,7 @@ package org.apache.doris.load.routineload; import org.apache.doris.load.routineload.kafka.KafkaDataSourceProperties; +import org.apache.doris.load.routineload.kinesis.KinesisDataSourceProperties; import java.util.Map; @@ -25,8 +26,8 @@ * RoutineLoadDataSourcePropertyFactory is used to create data source properties * for routine load job. *

- * Currently, we only support kafka data source. - * If we want to support more data source, we can add more data source properties here. + * Currently, we support kafka and kinesis data sources. + * If we want to support more data sources, we can add more data source properties here. * And we can add more data source type in LoadDataSourceType. * Then we can use this factory to create data source properties. *

@@ -37,6 +38,8 @@ public static AbstractDataSourceProperties createDataSource(String type, Map parameters) { if (type.equalsIgnoreCase(LoadDataSourceType.KAFKA.name())) { return new KafkaDataSourceProperties(parameters); + } else if (type.equalsIgnoreCase(LoadDataSourceType.KINESIS.name())) { + return new KinesisDataSourceProperties(parameters); } throw new IllegalArgumentException("Unknown routine load data source type: " + type); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index a4f42e36e5c3db..59612588d5d0d6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -49,6 +49,7 @@ import org.apache.doris.load.RoutineLoadDesc; import org.apache.doris.load.loadv2.LoadTask; import org.apache.doris.load.routineload.kafka.KafkaConfiguration; +import org.apache.doris.load.routineload.kafka.KafkaProgress; import org.apache.doris.metric.MetricRepo; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.load.NereidsRoutineLoadTaskInfo; @@ -832,9 +833,9 @@ public void processTimeoutTasks() { } } - abstract void updateCloudProgress() throws UserException; + protected abstract void updateCloudProgress() throws UserException; - abstract void divideRoutineLoadJob(int currentConcurrentTaskNum) throws UserException; + protected abstract void divideRoutineLoadJob(int currentConcurrentTaskNum) throws UserException; public int calculateCurrentConcurrentTaskNum() throws MetaNotFoundException { return 0; @@ -1003,7 +1004,8 @@ public Long totalLag() { return 0L; } - abstract RoutineLoadTaskInfo unprotectRenewTask(RoutineLoadTaskInfo routineLoadTaskInfo, boolean delaySchedule); + protected abstract RoutineLoadTaskInfo unprotectRenewTask( + RoutineLoadTaskInfo routineLoadTaskInfo, boolean delaySchedule); // call before first scheduling // derived class can override this. diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java index 3b1f498bcfcf84..f24b9ad2252dd5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadManager.java @@ -38,6 +38,8 @@ import org.apache.doris.common.util.LogBuilder; import org.apache.doris.common.util.LogKey; import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.load.routineload.kafka.KafkaRoutineLoadJob; +import org.apache.doris.load.routineload.kinesis.KinesisRoutineLoadJob; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.trees.plans.commands.AlterRoutineLoadCommand; import org.apache.doris.nereids.trees.plans.commands.info.CreateRoutineLoadInfo; @@ -192,6 +194,9 @@ public void createRoutineLoadJob(CreateRoutineLoadInfo info, ConnectContext ctx) case KAFKA: routineLoadJob = KafkaRoutineLoadJob.fromCreateInfo(info, ctx); break; + case KINESIS: + routineLoadJob = KinesisRoutineLoadJob.fromCreateInfo(info, ctx); + break; default: throw new UserException("Unknown data source type: " + type); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadProgress.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadProgress.java index 573b70f38f2513..f2eb959a8fd659 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadProgress.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadProgress.java @@ -27,7 +27,7 @@ public RoutineLoadProgress(LoadDataSourceType loadDataSourceType) { this.loadDataSourceType = loadDataSourceType; } - abstract void update(RLTaskTxnCommitAttachment attachment); + public abstract void update(RLTaskTxnCommitAttachment attachment); public abstract String toJsonString(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java index efdfb4586dbafe..872f29e72ed209 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java @@ -195,7 +195,7 @@ private void judgeEof(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment) { } } - abstract TRoutineLoadTask createRoutineLoadTask() throws UserException; + protected abstract TRoutineLoadTask createRoutineLoadTask() throws UserException; public void updateAdaptiveTimeout(RoutineLoadJob routineLoadJob) { } @@ -259,9 +259,9 @@ public List getTaskShowInfo() { return row; } - abstract String getTaskDataSourceProperties(); + protected abstract String getTaskDataSourceProperties(); - abstract boolean hasMoreDataToConsume() throws UserException; + protected abstract boolean hasMoreDataToConsume() throws UserException; @Override public boolean equals(Object obj) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaDataSourceProperties.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaDataSourceProperties.java index a5b1afb2cd3f2f..9cab113563da61 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaDataSourceProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaDataSourceProperties.java @@ -22,7 +22,6 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.load.routineload.AbstractDataSourceProperties; -import org.apache.doris.load.routineload.KafkaProgress; import org.apache.doris.load.routineload.LoadDataSourceType; import com.google.common.base.Preconditions; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaProgress.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaProgress.java similarity index 97% rename from fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaProgress.java rename to fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaProgress.java index 28cd4f8fb076e9..f1eba60a47d493 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaProgress.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaProgress.java @@ -15,12 +15,15 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.load.routineload; +package org.apache.doris.load.routineload.kafka; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; import org.apache.doris.common.Pair; import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.load.routineload.LoadDataSourceType; +import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; +import org.apache.doris.load.routineload.RoutineLoadProgress; import org.apache.doris.thrift.TKafkaRLTaskProgress; import com.google.common.base.Joiner; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaRoutineLoadJob.java similarity index 99% rename from fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java rename to fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaRoutineLoadJob.java index 4e2827403a4a6a..9464be78d05858 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaRoutineLoadJob.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.load.routineload; +package org.apache.doris.load.routineload.kafka; import org.apache.doris.analysis.ExprToSqlVisitor; import org.apache.doris.analysis.ImportColumnDesc; @@ -40,8 +40,12 @@ import org.apache.doris.common.util.SmallFileMgr.SmallFile; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.kafka.KafkaUtil; -import org.apache.doris.load.routineload.kafka.KafkaConfiguration; -import org.apache.doris.load.routineload.kafka.KafkaDataSourceProperties; +import org.apache.doris.load.routineload.ErrorReason; +import org.apache.doris.load.routineload.LoadDataSourceType; +import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; +import org.apache.doris.load.routineload.RoutineLoadJob; +import org.apache.doris.load.routineload.RoutineLoadTaskInfo; +import org.apache.doris.load.routineload.ScheduleRule; import org.apache.doris.nereids.load.NereidsImportColumnDesc; import org.apache.doris.nereids.load.NereidsLoadTaskInfo; import org.apache.doris.nereids.load.NereidsLoadUtils; diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaTaskInfo.java similarity index 97% rename from fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java rename to fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaTaskInfo.java index 5e5c7158b71bea..4364c106529131 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kafka/KafkaTaskInfo.java @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.load.routineload; +package org.apache.doris.load.routineload.kafka; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; @@ -24,6 +24,9 @@ import org.apache.doris.common.Config; import org.apache.doris.common.UserException; import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.load.routineload.RoutineLoadJob; +import org.apache.doris.load.routineload.RoutineLoadManager; +import org.apache.doris.load.routineload.RoutineLoadTaskInfo; import org.apache.doris.nereids.load.NereidsLoadTaskInfo; import org.apache.doris.nereids.load.NereidsStreamLoadPlanner; import org.apache.doris.qe.ConnectContext; @@ -154,7 +157,7 @@ protected String getTaskDataSourceProperties() { } @Override - boolean hasMoreDataToConsume() throws UserException { + protected boolean hasMoreDataToConsume() throws UserException { KafkaRoutineLoadJob routineLoadJob = (KafkaRoutineLoadJob) routineLoadManager.getJob(jobId); return routineLoadJob.hasMoreDataToConsume(id, partitionIdToOffset); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisConfiguration.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisConfiguration.java new file mode 100644 index 00000000000000..dc1d38561e3dbe --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisConfiguration.java @@ -0,0 +1,169 @@ +// 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.load.routineload.kinesis; + +import com.google.common.base.Splitter; + +import java.util.Arrays; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Configuration enum for AWS Kinesis data source properties. + * + * Parameters are divided into two categories: + * 1. AWS Client parameters (aws.*): region, endpoint, credentials, timeouts + * 2. Kinesis-specific parameters (aws.kinesis.*): stream, shards, positions, API settings + */ +public enum KinesisConfiguration { + + /** + * AWS region where the Kinesis stream is located. + * Required property. + */ + KINESIS_REGION("aws.region", null, value -> value.trim()), + + /** + * Optional custom endpoint URL for Kinesis service. + * Useful for LocalStack or VPC endpoints. + */ + KINESIS_ENDPOINT("aws.endpoint", null, value -> value.trim()), + + /** + * Name of the Kinesis stream to consume from. + * Required property. + */ + KINESIS_STREAM("kinesis_stream", null, value -> value.trim()), + + /** + * Comma-separated list of shard IDs to consume from. + * If not specified, all shards will be consumed. + */ + KINESIS_SHARDS("kinesis_shards", null, shardsString -> + Arrays.stream(shardsString.replace(" ", "").split(",")) + .collect(Collectors.toList())), + + /** + * Shard iterator positions (sequence numbers) for each shard. + * Format: position1,position2,... corresponding to shards order. + */ + KINESIS_POSITIONS("kinesis_shards_pos", null, + positionsString -> Splitter.on(",").trimResults().splitToList(positionsString)), + + /** + * Default starting position for new shards. + * Valid values: TRIM_HORIZON, LATEST, or sequence number + */ + KINESIS_DEFAULT_POSITION("property.kinesis_default_pos", "LATEST", position -> position.trim()), + + /** + * AWS Access Key ID for authentication. + */ + KINESIS_ACCESS_KEY("aws.access_key", null, value -> value), + + /** + * AWS Secret Access Key for authentication. + */ + KINESIS_SECRET_KEY("aws.secret_key", null, value -> value), + + /** + * AWS Session Token for temporary credentials. + */ + KINESIS_SESSION_TOKEN("aws.session_key", null, value -> value), + + /** + * IAM Role ARN to assume for accessing Kinesis. + */ + KINESIS_ROLE_ARN("aws.role_arn", null, value -> value.trim()), + + /** + * External ID for IAM role assumption. + */ + KINESIS_EXTERNAL_ID("aws.external.id", null, value -> value.trim()), + + /** + * AWS Profile name to use from credentials file. + */ + KINESIS_PROFILE_NAME("aws.profile.name", null, value -> value.trim()), + + /** + * Consumer name for enhanced fan-out (EFO). + */ + KINESIS_CONSUMER_NAME("aws.kinesis.consumer.name", null, value -> value.trim()), + + /** + * Maximum records per GetRecords call. + */ + KINESIS_MAX_RECORDS_PER_FETCH("aws.kinesis.max_records", 10000, Integer::parseInt), + + /** + * Interval between GetRecords calls in milliseconds. + */ + KINESIS_FETCH_INTERVAL_MS("aws.kinesis.fetch_interval_ms", 200, Integer::parseInt), + + /** + * Connection timeout in milliseconds. + */ + KINESIS_CONNECTION_TIMEOUT_MS("aws.connection.timeout.ms", 10000, Integer::parseInt), + + /** + * Request timeout in milliseconds. + */ + KINESIS_REQUEST_TIMEOUT_MS("aws.request.timeout.ms", 10000, Integer::parseInt), + + /** + * Max number of retry attempts for Kinesis API calls. + */ + KINESIS_MAX_RETRIES("aws.kinesis.max_retries", 3, Integer::parseInt), + + /** + * Use HTTPS for Kinesis endpoint. + */ + KINESIS_USE_HTTPS("aws.kinesis.use_https", true, Boolean::parseBoolean); + + private final String name; + private final Object defaultValue; + private final Function converter; + + KinesisConfiguration(String name, T defaultValue, Function converter) { + this.name = name; + this.defaultValue = defaultValue; + this.converter = (Function) converter; + } + + public String getName() { + return name; + } + + public Object getDefaultValue() { + return defaultValue; + } + + public static KinesisConfiguration getByName(String name) { + return Arrays.stream(KinesisConfiguration.values()) + .filter(config -> config.getName().equals(name)) + .findFirst() + .orElseThrow(() -> new IllegalArgumentException("Unknown Kinesis configuration: " + name)); + } + + @SuppressWarnings("unchecked") + public T getParameterValue(String param) { + Object value = param != null ? converter.apply(param) : defaultValue; + return (T) value; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisDataSourceProperties.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisDataSourceProperties.java new file mode 100644 index 00000000000000..edc6cd891fd739 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisDataSourceProperties.java @@ -0,0 +1,411 @@ +// 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.load.routineload.kinesis; + +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Pair; +import org.apache.doris.common.UserException; +import org.apache.doris.load.routineload.AbstractDataSourceProperties; +import org.apache.doris.load.routineload.LoadDataSourceType; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.gson.annotations.SerializedName; +import lombok.Getter; +import lombok.Setter; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * AWS Kinesis data source properties for Routine Load. + * + * Parameters: + * - kinesis_stream: Name of the Kinesis stream (required) + * - kinesis_shards: Comma-separated list of shard IDs (optional) + * - kinesis_shards_pos: Comma-separated list of positions for each shard (optional) + * - aws.region: AWS region (required) + * - aws.endpoint: Custom Kinesis endpoint URL (optional, e.g. LocalStack) + * - aws.access_key: AWS access key (optional) + * - aws.secret_key: AWS secret key (optional) + * - aws.session_key: AWS session token (optional) + * - aws.role_arn: IAM role ARN (optional) + * - property.kinesis_default_pos: Default position for new shards (optional) + * - property.*: Other pass-through parameters for AWS SDK configuration + * + * Example usage in SQL: + * CREATE ROUTINE LOAD my_job ON my_table + * FROM KINESIS ( + * "aws.region" = "us-east-1", + * "aws.access_key" = "AKIAIOSFODNN7EXAMPLE", + * "aws.secret_key" = "wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY", + * "kinesis_stream" = "my-stream", + * "property.kinesis_default_pos" = "TRIM_HORIZON" + * ); + */ +public class KinesisDataSourceProperties extends AbstractDataSourceProperties { + + /** + * List of shard IDs with their starting sequence numbers. + * Pair + * SequenceNumber can be: + * - Actual sequence number string + * - TRIM_HORIZON_VAL (-2) for oldest record + * - LATEST_VAL (-1) for newest record + */ + @Getter + @Setter + @SerializedName(value = "kinesisShardPositions") + private List> kinesisShardPositions = Lists.newArrayList(); + + /** + * Custom Kinesis properties for advanced configuration. + * Includes AWS credentials and client configuration. + */ + @Getter + @SerializedName(value = "customKinesisProperties") + private Map customKinesisProperties; + + /** + * Whether positions are specified as timestamps. + */ + @Getter + @SerializedName(value = "isPositionsForTimes") + private boolean isPositionsForTimes = false; + + /** + * AWS region for the Kinesis stream. + */ + @Getter + @SerializedName(value = "region") + private String region; + + /** + * Name of the Kinesis stream. + */ + @Getter + @SerializedName(value = "stream") + private String stream; + + /** + * Optional endpoint URL for custom endpoints. + */ + @Getter + @SerializedName(value = "endpoint") + private String endpoint; + + // Standard position constants (similar to Kafka's OFFSET_BEGINNING/OFFSET_END) + public static final String POSITION_TRIM_HORIZON = "TRIM_HORIZON"; + public static final String POSITION_LATEST = "LATEST"; + + // Configurable data source properties that can be set by user + // Keep compatibility with existing ALTER/SHOW output key "kinesis_endpoint". + private static final String LEGACY_KINESIS_ENDPOINT_KEY = "kinesis_endpoint"; + + private static final ImmutableSet CONFIGURABLE_DATA_SOURCE_PROPERTIES_SET = + new ImmutableSet.Builder() + .add(KinesisConfiguration.KINESIS_REGION.getName()) + .add(KinesisConfiguration.KINESIS_ENDPOINT.getName()) + .add(LEGACY_KINESIS_ENDPOINT_KEY) + .add(KinesisConfiguration.KINESIS_STREAM.getName()) + .add(KinesisConfiguration.KINESIS_SHARDS.getName()) + .add(KinesisConfiguration.KINESIS_POSITIONS.getName()) + .add(KinesisConfiguration.KINESIS_DEFAULT_POSITION.getName()) + .add(KinesisConfiguration.KINESIS_ACCESS_KEY.getName()) + .add(KinesisConfiguration.KINESIS_SECRET_KEY.getName()) + .add(KinesisConfiguration.KINESIS_SESSION_TOKEN.getName()) + .add(KinesisConfiguration.KINESIS_ROLE_ARN.getName()) + .build(); + + public KinesisDataSourceProperties(Map dataSourceProperties, boolean multiLoad) { + super(dataSourceProperties, multiLoad); + } + + public KinesisDataSourceProperties(Map originalDataSourceProperties) { + super(originalDataSourceProperties); + } + + @Override + protected String getDataSourceType() { + return LoadDataSourceType.KINESIS.name(); + } + + @Override + protected List getRequiredProperties() { + return Arrays.asList( + KinesisConfiguration.KINESIS_REGION.getName(), + KinesisConfiguration.KINESIS_STREAM.getName() + ); + } + + @Override + public void convertAndCheckDataSourceProperties() throws UserException { + // Check for invalid properties - accept property.* parameters as pass-through + Optional invalidProperty = originalDataSourceProperties.keySet().stream() + .filter(key -> !CONFIGURABLE_DATA_SOURCE_PROPERTIES_SET.contains(key)) + .filter(key -> !key.startsWith("property.")) + .findFirst(); + if (invalidProperty.isPresent()) { + throw new AnalysisException(invalidProperty.get() + " is invalid Kinesis property or cannot be set"); + } + + // Parse region (required) + this.region = KinesisConfiguration.KINESIS_REGION.getParameterValue( + originalDataSourceProperties.get(KinesisConfiguration.KINESIS_REGION.getName())); + if (!isAlter() && StringUtils.isBlank(region)) { + throw new AnalysisException(KinesisConfiguration.KINESIS_REGION.getName() + " is a required property"); + } + if (StringUtils.isNotBlank(region)) { + validateRegion(region); + } + + // Parse custom endpoint (optional) + this.endpoint = KinesisConfiguration.KINESIS_ENDPOINT.getParameterValue( + originalDataSourceProperties.containsKey(KinesisConfiguration.KINESIS_ENDPOINT.getName()) + ? originalDataSourceProperties.get(KinesisConfiguration.KINESIS_ENDPOINT.getName()) + : originalDataSourceProperties.get(LEGACY_KINESIS_ENDPOINT_KEY)); + + // Parse stream name (required) + this.stream = KinesisConfiguration.KINESIS_STREAM.getParameterValue( + originalDataSourceProperties.get(KinesisConfiguration.KINESIS_STREAM.getName())); + if (!isAlter() && StringUtils.isBlank(stream)) { + throw new AnalysisException(KinesisConfiguration.KINESIS_STREAM.getName() + " is a required property"); + } + + // Parse custom properties (property.* parameters) + analyzeCustomProperties(); + + // Parse AWS credentials from direct parameters + parseAwsCredentials(); + + // Validate AWS authentication configuration + validateAwsAuthConfig(); + + // Parse shards + List shards = KinesisConfiguration.KINESIS_SHARDS.getParameterValue( + originalDataSourceProperties.get(KinesisConfiguration.KINESIS_SHARDS.getName())); + if (CollectionUtils.isNotEmpty(shards)) { + analyzeKinesisShardProperty(shards); + } + + // Parse positions + List positions = KinesisConfiguration.KINESIS_POSITIONS.getParameterValue( + originalDataSourceProperties.get(KinesisConfiguration.KINESIS_POSITIONS.getName())); + // Get default position from customKinesisProperties (already parsed from "property." prefix) + String defaultPositionString = customKinesisProperties.get("kinesis_default_pos"); + + // Validate that positions and default_position are not both set + if (CollectionUtils.isNotEmpty(positions) && StringUtils.isNotBlank(defaultPositionString)) { + throw new AnalysisException("Only one of " + KinesisConfiguration.KINESIS_POSITIONS.getName() + + " and property.kinesis_default_pos can be set."); + } + + // For alter operation, shards and positions must be set together + if (isAlter() && CollectionUtils.isNotEmpty(shards) && CollectionUtils.isEmpty(positions) + && StringUtils.isBlank(defaultPositionString)) { + throw new AnalysisException("Must set position or default position with shard property"); + } + + // Process positions + if (CollectionUtils.isNotEmpty(positions)) { + this.isPositionsForTimes = analyzeKinesisPositionProperty(positions); + return; + } + this.isPositionsForTimes = analyzeKinesisDefaultPositionProperty(); + if (CollectionUtils.isNotEmpty(kinesisShardPositions)) { + setDefaultPositionForShards(this.kinesisShardPositions, defaultPositionString, this.isPositionsForTimes); + } + } + + /** + * Validate AWS region format. + */ + private void validateRegion(String region) throws AnalysisException { + // AWS regions follow patterns like: us-east-1, eu-west-2, ap-southeast-1, cn-north-1 + if (!region.matches("^[a-z]{2}(-[a-z]+)?-[a-z]+-\\d$")) { + throw new AnalysisException("Invalid AWS region format: " + region + + ". Expected format like: us-east-1, eu-west-2, cn-north-1"); + } + } + + /** + * Parse and store custom Kinesis properties. + * All property.* parameters are passed through to BE. + */ + private void analyzeCustomProperties() throws AnalysisException { + this.customKinesisProperties = new HashMap<>(); + + // Store all property.* parameters (strip the "property." prefix for BE) + for (Map.Entry entry : originalDataSourceProperties.entrySet()) { + String key = entry.getKey(); + if (key.startsWith("property.")) { + // Strip "property." prefix and pass through to BE + String actualKey = key.substring("property.".length()); + customKinesisProperties.put(actualKey, entry.getValue()); + } + } + } + + /** + * Parse AWS credentials from direct parameters and add to customKinesisProperties. + */ + private void parseAwsCredentials() { + String accessKey = originalDataSourceProperties.get(KinesisConfiguration.KINESIS_ACCESS_KEY.getName()); + if (StringUtils.isNotBlank(accessKey)) { + customKinesisProperties.put("aws.access_key", accessKey); + } + + String secretKey = originalDataSourceProperties.get(KinesisConfiguration.KINESIS_SECRET_KEY.getName()); + if (StringUtils.isNotBlank(secretKey)) { + customKinesisProperties.put("aws.secret_key", secretKey); + } + + String sessionToken = originalDataSourceProperties.get(KinesisConfiguration.KINESIS_SESSION_TOKEN.getName()); + if (StringUtils.isNotBlank(sessionToken)) { + customKinesisProperties.put("aws.session_key", sessionToken); + } + + String roleArn = originalDataSourceProperties.get(KinesisConfiguration.KINESIS_ROLE_ARN.getName()); + if (StringUtils.isNotBlank(roleArn)) { + customKinesisProperties.put("aws.role_arn", roleArn); + } + } + + /** + * Validate AWS authentication configuration. + * At least one authentication method must be provided: + * 1. Access key + Secret key + * 2. IAM Role ARN + * 3. AWS Profile name + * 4. Default credential chain (EC2 instance profile, environment variables, etc.) + */ + private void validateAwsAuthConfig() throws AnalysisException { + String accessKey = customKinesisProperties.get("aws.access_key"); + String secretKey = customKinesisProperties.get("aws.secret_key"); + String roleArn = customKinesisProperties.get("aws.role_arn"); + + // If access key is provided, secret key must also be provided + if (StringUtils.isNotBlank(accessKey) && StringUtils.isBlank(secretKey)) { + throw new AnalysisException("When property.aws.access_key is set, property.aws.secret_key " + + "must also be set"); + } + if (StringUtils.isNotBlank(secretKey) && StringUtils.isBlank(accessKey)) { + throw new AnalysisException("When property.aws.secret_key is set, property.aws.access_key " + + "must also be set"); + } + + // If external ID is provided, role ARN must be provided + String externalId = customKinesisProperties.get("aws.external.id"); + if (StringUtils.isNotBlank(externalId) && StringUtils.isBlank(roleArn)) { + throw new AnalysisException("When property.aws.external.id is set, property.aws.role_arn must also be set"); + } + + // Note: We don't require any authentication config because the default credential chain + // can be used in EC2/EKS environments with instance profiles or service accounts + } + + /** + * Initialize shard positions with default values. + */ + private void analyzeKinesisShardProperty(List shards) { + shards.forEach(shardId -> this.kinesisShardPositions.add(Pair.of(shardId, POSITION_LATEST))); + } + + /** + * Parse position property and set positions for each shard. + * All positions are interpreted as sequence-number semantics: + * TRIM_HORIZON, LATEST, or explicit sequence number. + */ + private boolean analyzeKinesisPositionProperty(List positions) throws UserException { + if (positions.size() != kinesisShardPositions.size()) { + throw new AnalysisException("Number of shards must equal number of positions"); + } + + for (int i = 0; i < positions.size(); i++) { + String position = positions.get(i); + validatePosition(position); + kinesisShardPositions.get(i).second = position; + } + return false; + } + + /** + * Validate position value. + */ + private void validatePosition(String position) throws AnalysisException { + if (!position.equalsIgnoreCase(POSITION_TRIM_HORIZON) + && !position.equalsIgnoreCase(POSITION_LATEST) + && !isValidSequenceNumber(position)) { + throw new AnalysisException(KinesisConfiguration.KINESIS_POSITIONS.getName() + + " must be TRIM_HORIZON, LATEST, or a valid sequence number. Got: " + position); + } + } + + /** + * Check if the string is a valid Kinesis sequence number. + * Kinesis sequence numbers are numeric strings. + */ + private boolean isValidSequenceNumber(String position) { + try { + // Kinesis sequence numbers are large numeric strings + new java.math.BigInteger(position); + return true; + } catch (NumberFormatException e) { + return false; + } + } + + /** + * Analyze default position property. + * Default position uses sequence-number semantics: + * TRIM_HORIZON, LATEST, or explicit sequence number. + */ + private boolean analyzeKinesisDefaultPositionProperty() throws AnalysisException { + customKinesisProperties.putIfAbsent("kinesis_default_pos", POSITION_LATEST); + String defaultPosition = customKinesisProperties.get("kinesis_default_pos"); + + if (!defaultPosition.equalsIgnoreCase(POSITION_TRIM_HORIZON) + && !defaultPosition.equalsIgnoreCase(POSITION_LATEST) + && !isValidSequenceNumber(defaultPosition)) { + throw new AnalysisException("property.kinesis_default_pos can only be set to TRIM_HORIZON, " + + "LATEST, or a valid sequence number. Got: " + defaultPosition); + } + return false; + } + + /** + * Set default position for all shards. + */ + private static void setDefaultPositionForShards(List> shardPositions, + String defaultPosition, boolean isForTimes) { + if (isForTimes) { + for (Pair pair : shardPositions) { + pair.second = defaultPosition; + } + } else { + for (Pair pair : shardPositions) { + pair.second = defaultPosition != null ? defaultPosition : POSITION_LATEST; + } + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisProgress.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisProgress.java new file mode 100644 index 00000000000000..016e212759f181 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisProgress.java @@ -0,0 +1,334 @@ +// 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.load.routineload.kinesis; + +import org.apache.doris.common.DdlException; +import org.apache.doris.common.Pair; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.load.routineload.LoadDataSourceType; +import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; +import org.apache.doris.load.routineload.RoutineLoadProgress; +import org.apache.doris.thrift.TKinesisRLTaskProgress; + +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import com.google.gson.annotations.SerializedName; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Progress tracking for Kinesis Routine Load jobs. + * + * Kinesis uses sequence numbers instead of offsets like Kafka. + * A sequence number is a unique identifier for each record within a shard. + * Sequence numbers are string representations of 128-bit integers. + * + * Special position values: + * - TRIM_HORIZON: Start from the oldest record in the shard + * - LATEST: Start from the newest record (records arriving after the iterator is created) + * - Specific sequence number: Start from or after a specific sequence number + */ +public class KinesisProgress extends RoutineLoadProgress { + private static final Logger LOG = LogManager.getLogger(KinesisProgress.class); + + // Special position constants + public static final String POSITION_TRIM_HORIZON = "TRIM_HORIZON"; + public static final String POSITION_LATEST = "LATEST"; + + // Internal representation for special positions + // Using negative values since sequence numbers are always positive + public static final String TRIM_HORIZON_VAL = "-2"; + public static final String LATEST_VAL = "-1"; + + /** + * Map from shard ID to sequence number. + * The sequence number saved here is the next sequence number to be consumed. + * + * Note: Unlike Kafka partitions which are integers, Kinesis shard IDs are strings + * like "shardId-000000000000". + */ + @SerializedName(value = "shardToSeqNum") + private ConcurrentMap shardIdToSequenceNumber = Maps.newConcurrentMap(); + + // MillisBehindLatest per shard, reported by BE from GetRecords response. + // Not persisted — refreshed every task commit. Used only for lag display and scheduling. + private ConcurrentMap shardIdToMillsBehindLatest = Maps.newConcurrentMap(); + + // Set of shard IDs that have been closed (split/merge) during consumption. + // Not persisted — only used during task commit to remove closed shards from tracking. + private java.util.Set closedShardIds = new java.util.HashSet<>(); + + private transient ReentrantLock lock = new ReentrantLock(true); + + public KinesisProgress() { + super(LoadDataSourceType.KINESIS); + } + + public KinesisProgress(TKinesisRLTaskProgress tKinesisRLTaskProgress) { + super(LoadDataSourceType.KINESIS); + this.shardIdToSequenceNumber = new ConcurrentHashMap<>(); + if (tKinesisRLTaskProgress.getShardCmtSeqNum() != null) { + shardIdToSequenceNumber.putAll(tKinesisRLTaskProgress.getShardCmtSeqNum()); + } + if (tKinesisRLTaskProgress.isSetShardMillsBehindLatest()) { + this.shardIdToMillsBehindLatest = new ConcurrentHashMap<>( + tKinesisRLTaskProgress.getShardMillsBehindLatest()); + } + if (tKinesisRLTaskProgress.isSetClosedShardIds()) { + this.closedShardIds = new java.util.HashSet<>(tKinesisRLTaskProgress.getClosedShardIds()); + } + } + + public KinesisProgress(Map shardIdToSequenceNumber) { + super(LoadDataSourceType.KINESIS); + this.shardIdToSequenceNumber = new ConcurrentHashMap<>(); + this.shardIdToSequenceNumber.putAll(shardIdToSequenceNumber); + } + + public KinesisProgress(ConcurrentMap shardIdToSequenceNumber) { + super(LoadDataSourceType.KINESIS); + this.shardIdToSequenceNumber = shardIdToSequenceNumber; + } + + /** + * Get sequence numbers for specified shard IDs. + */ + public ConcurrentMap getShardIdToSequenceNumber(List shardIds) { + ConcurrentMap result = Maps.newConcurrentMap(); + for (Map.Entry entry : shardIdToSequenceNumber.entrySet()) { + for (String shardId : shardIds) { + if (entry.getKey().equals(shardId)) { + result.put(shardId, entry.getValue()); + } + } + } + return result; + } + + /** + * Add a shard with its starting position. + */ + public void addShardPosition(Pair shardPosition) { + ReentrantLock progressLock = getLock(); + progressLock.lock(); + try { + shardIdToSequenceNumber.put(shardPosition.first, shardPosition.second); + } finally { + progressLock.unlock(); + } + } + + /** + * Get the sequence number for a specific shard. + */ + public String getSequenceNumberByShard(String shardId) { + return shardIdToSequenceNumber.get(shardId); + } + + /** + * Get all shard to sequence number mappings. + */ + public ConcurrentMap getShardIdToSequenceNumber() { + return shardIdToSequenceNumber; + } + + /** + * Get all shard to MillisBehindLatest mappings (from the last task commit). + */ + public ConcurrentMap getShardIdToMillsBehindLatest() { + return shardIdToMillsBehindLatest; + } + + /** + * Get the set of closed shard IDs. + */ + public java.util.Set getClosedShardIds() { + return closedShardIds; + } + + /** + * Check if the progress contains a specific shard. + */ + public boolean containsShard(String shardId) { + return shardIdToSequenceNumber.containsKey(shardId); + } + + /** + * Check if any shards are being tracked. + */ + public boolean hasShards() { + return !shardIdToSequenceNumber.isEmpty(); + } + + /** + * Get human-readable progress information. + */ + private void getReadableProgress(ConcurrentMap showShardIdToPosition) { + for (Map.Entry entry : shardIdToSequenceNumber.entrySet()) { + String position = entry.getValue(); + if (TRIM_HORIZON_VAL.equals(position)) { + showShardIdToPosition.put(entry.getKey(), POSITION_TRIM_HORIZON); + } else if (LATEST_VAL.equals(position)) { + showShardIdToPosition.put(entry.getKey(), POSITION_LATEST); + } else { + // For actual sequence numbers, show the last consumed sequence number + showShardIdToPosition.put(entry.getKey(), position); + } + } + } + + /** + * Check that all specified shards exist in the progress. + */ + public void checkShards(List> kinesisShardPositions) throws DdlException { + for (Pair pair : kinesisShardPositions) { + if (!shardIdToSequenceNumber.containsKey(pair.first)) { + throw new DdlException("The specified shard " + pair.first + " is not in the consumed shards"); + } + } + } + + /** + * Modify the position for specific shards. + */ + public void modifyPosition(List> kinesisShardPositions) { + ReentrantLock progressLock = getLock(); + progressLock.lock(); + try { + for (Pair pair : kinesisShardPositions) { + shardIdToSequenceNumber.put(pair.first, pair.second); + } + } finally { + progressLock.unlock(); + } + } + + /** + * Get shard ID and position pairs. + */ + public List> getShardPositionPairs(boolean alreadyConsumed) { + List> pairs = Lists.newArrayList(); + for (Map.Entry entry : shardIdToSequenceNumber.entrySet()) { + String position = entry.getValue(); + if (TRIM_HORIZON_VAL.equals(position)) { + pairs.add(Pair.of(entry.getKey(), POSITION_TRIM_HORIZON)); + } else if (LATEST_VAL.equals(position)) { + pairs.add(Pair.of(entry.getKey(), POSITION_LATEST)); + } else { + // For actual sequence numbers + pairs.add(Pair.of(entry.getKey(), position)); + } + } + return pairs; + } + + /** + * Calculate lag for each shard. + * Note: Kinesis lag calculation is more complex than Kafka because: + * 1. Sequence numbers are strings, not comparable integers + * 2. GetRecords API returns millisBehindLatest which is more useful + * + * This method returns -1 for shards where lag cannot be calculated. + */ + public Map getLag(Map shardIdWithMillsBehindLatest) { + Map lagMap = Maps.newHashMap(); + for (String shardId : shardIdToSequenceNumber.keySet()) { + Long lag = shardIdWithMillsBehindLatest.get(shardId); + lagMap.put(shardId, lag != null ? lag : -1L); + } + return lagMap; + } + + @Override + public String toString() { + ConcurrentMap showShardIdToPosition = Maps.newConcurrentMap(); + getReadableProgress(showShardIdToPosition); + return "KinesisProgress [shardIdToSequenceNumber=" + + Joiner.on("|").withKeyValueSeparator("_").join(showShardIdToPosition) + "]"; + } + + @Override + public String toJsonString() { + ConcurrentMap showShardIdToPosition = Maps.newConcurrentMap(); + getReadableProgress(showShardIdToPosition); + Gson gson = new Gson(); + return gson.toJson(showShardIdToPosition); + } + + @Override + public void update(RLTaskTxnCommitAttachment attachment) { + KinesisProgress newProgress = (KinesisProgress) attachment.getProgress(); + + ReentrantLock progressLock = getLock(); + progressLock.lock(); + try { + // Update sequence numbers for each shard + newProgress.shardIdToSequenceNumber.forEach((shardId, newSeqNum) -> { + this.shardIdToSequenceNumber.put(shardId, newSeqNum); + }); + + // Update MillisBehindLatest + if (newProgress.getShardIdToMillsBehindLatest() != null) { + this.shardIdToMillsBehindLatest.putAll(newProgress.getShardIdToMillsBehindLatest()); + } + + // Remove closed shards from tracking + if (newProgress.getClosedShardIds() != null && !newProgress.getClosedShardIds().isEmpty()) { + for (String closedShardId : newProgress.getClosedShardIds()) { + this.shardIdToSequenceNumber.remove(closedShardId); + this.shardIdToMillsBehindLatest.remove(closedShardId); + LOG.info("Removed closed shard from progress: {}", closedShardId); + } + // Store closed shard IDs for Job to clean up consumingClosedShards + this.closedShardIds.addAll(newProgress.getClosedShardIds()); + } + } finally { + progressLock.unlock(); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("update kinesis progress: {}, task: {}, job: {}", + newProgress.toJsonString(), DebugUtil.printId(attachment.getTaskId()), attachment.getJobId()); + } + } + + /** + * Get total progress as the count of tracked shards. + * Unlike Kafka where we sum offsets, Kinesis sequence numbers are not additive. + */ + public Long totalProgress() { + // Return the number of shards being tracked + // Actual progress is better represented by millisBehindLatest + return (long) shardIdToSequenceNumber.size(); + } + + private ReentrantLock getLock() { + if (lock == null) { + lock = new ReentrantLock(true); + } + return lock; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisRoutineLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisRoutineLoadJob.java new file mode 100644 index 00000000000000..0c05889929924e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisRoutineLoadJob.java @@ -0,0 +1,893 @@ +// 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.load.routineload.kinesis; + +import org.apache.doris.analysis.ExprToSqlVisitor; +import org.apache.doris.analysis.ImportColumnDesc; +import org.apache.doris.analysis.ToSqlParams; +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.RandomDistributionInfo; +import org.apache.doris.common.Config; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.InternalErrorCode; +import org.apache.doris.common.LoadException; +import org.apache.doris.common.Pair; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.common.util.LogBuilder; +import org.apache.doris.common.util.LogKey; +import org.apache.doris.datasource.kinesis.KinesisUtil; +import org.apache.doris.load.routineload.ErrorReason; +import org.apache.doris.load.routineload.LoadDataSourceType; +import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; +import org.apache.doris.load.routineload.RoutineLoadJob; +import org.apache.doris.load.routineload.RoutineLoadTaskInfo; +import org.apache.doris.load.routineload.ScheduleRule; +import org.apache.doris.nereids.load.NereidsImportColumnDesc; +import org.apache.doris.nereids.load.NereidsLoadTaskInfo; +import org.apache.doris.nereids.load.NereidsLoadUtils; +import org.apache.doris.nereids.load.NereidsRoutineLoadTaskInfo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.commands.AlterRoutineLoadCommand; +import org.apache.doris.nereids.trees.plans.commands.info.CreateRoutineLoadInfo; +import org.apache.doris.persist.AlterRoutineLoadJobOperationLog; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.thrift.TFileCompressType; +import org.apache.doris.thrift.TPartialUpdateNewRowPolicy; +import org.apache.doris.transaction.TransactionState; +import org.apache.doris.transaction.TransactionStatus; + +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import com.google.gson.GsonBuilder; +import com.google.gson.annotations.SerializedName; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.collections4.MapUtils; +import org.apache.commons.lang3.BooleanUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +/** + * KinesisRoutineLoadJob is a RoutineLoadJob that fetches data from AWS Kinesis streams. + * + * Key concepts: + * - Stream: Named collection of data records (similar to Kafka topic) + * - Shard: Sequence of data records in a stream (similar to Kafka partition) + * - Sequence Number: Unique identifier for each record within a shard (similar to Kafka offset) + * - Consumer: Application that reads from a stream + * + * The progress tracks sequence numbers for each shard, represented as: + * {"shardId-000000000000": "49590338271490256608559692538361571095921575989136588802", ...} + */ +public class KinesisRoutineLoadJob extends RoutineLoadJob { + private static final Logger LOG = LogManager.getLogger(KinesisRoutineLoadJob.class); + + public static final String KINESIS_FILE_CATALOG = "kinesis"; + + @SerializedName("rg") + private String region; + @SerializedName("stm") + private String stream; + @SerializedName("ep") + private String endpoint; + + // optional, user want to load shards(Kafka's cskp). + @SerializedName("csks") + private List customKinesisShards = Lists.newArrayList(); + + // OPEN shards - actively receiving new data + @SerializedName("opks") + private List openKinesisShards = Lists.newArrayList(); + + // CLOSED shards with unconsumed data - no longer receiving new data but still have data to consume + @SerializedName("clks") + private List closedKinesisShards = Lists.newArrayList(); + + // Default starting position for new shards. + // Values: TRIM_HORIZON, LATEST, or a timestamp string. + private String kinesisDefaultPosition = ""; + + // custom Kinesis properties including AWS credentials and client settings. + @SerializedName("prop") + private Map customProperties = Maps.newHashMap(); + private Map convertedCustomProperties = Maps.newHashMap(); + + // The latest offset of each partition fetched from kinesis server. + // Will be updated periodically by calling hasMoreDataToConsume() + private Map cachedShardWithMillsBehindLatest = Maps.newConcurrentMap(); + + // newly discovered shards from Kinesis. + private List newCurrentKinesisShards = Lists.newArrayList(); + + public KinesisRoutineLoadJob() { + // For serialization + super(-1, LoadDataSourceType.KINESIS); + } + + public KinesisRoutineLoadJob(Long id, String name, long dbId, long tableId, + String region, String stream, UserIdentity userIdentity) { + super(id, name, dbId, tableId, LoadDataSourceType.KINESIS, userIdentity); + this.region = region; + this.stream = stream; + this.progress = new KinesisProgress(); + } + + public KinesisRoutineLoadJob(Long id, String name, long dbId, + String region, String stream, + UserIdentity userIdentity, boolean isMultiTable) { + super(id, name, dbId, LoadDataSourceType.KINESIS, userIdentity); + this.region = region; + this.stream = stream; + this.progress = new KinesisProgress(); + setMultiTable(isMultiTable); + } + + public String getRegion() { + return region; + } + + public String getStream() { + return stream; + } + + public String getEndpoint() { + return endpoint; + } + + public Map getConvertedCustomProperties() { + return convertedCustomProperties; + } + + @Override + public void prepare() throws UserException { + // should reset converted properties each time the job being prepared. + // because the file info can be changed anytime. + convertCustomProperties(true); + } + + private void convertCustomProperties(boolean rebuild) throws DdlException { + if (customProperties.isEmpty()) { + return; + } + + if (!rebuild && !convertedCustomProperties.isEmpty()) { + return; + } + + if (rebuild) { + convertedCustomProperties.clear(); + } + + for (Map.Entry entry : customProperties.entrySet()) { + convertedCustomProperties.put(entry.getKey(), entry.getValue()); + } + + // Handle default position + if (convertedCustomProperties.containsKey("kinesis_default_pos")) { + kinesisDefaultPosition = convertedCustomProperties.get("kinesis_default_pos"); + // Keep it in convertedCustomProperties so BE can use it + } + } + + private String convertedDefaultPosition() { + if (this.kinesisDefaultPosition.isEmpty()) { + return KinesisProgress.POSITION_LATEST; + } + return this.kinesisDefaultPosition; + } + + @Override + public void divideRoutineLoadJob(int currentConcurrentTaskNum) throws UserException { + List result = new ArrayList<>(); + writeLock(); + try { + if (state == JobState.NEED_SCHEDULE) { + // Combine open and closed shards for task assignment + List allShards = Lists.newArrayList(); + allShards.addAll(openKinesisShards); + allShards.addAll(closedKinesisShards); + + // Divide shards into tasks + for (int i = 0; i < currentConcurrentTaskNum; i++) { + Map taskKinesisProgress = Maps.newHashMap(); + for (int j = i; j < allShards.size(); j = j + currentConcurrentTaskNum) { + String shardId = allShards.get(j); + taskKinesisProgress.put(shardId, + ((KinesisProgress) progress).getSequenceNumberByShard(shardId)); + } + KinesisTaskInfo kinesisTaskInfo = new KinesisTaskInfo(UUID.randomUUID(), id, + getTimeout() * 1000, taskKinesisProgress, isMultiTable(), -1, false); + routineLoadTaskInfoList.add(kinesisTaskInfo); + result.add(kinesisTaskInfo); + } + // Change job state to running + if (!result.isEmpty()) { + unprotectUpdateState(JobState.RUNNING, null, false); + } + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignore to divide routine load job while job state {}", state); + } + } + // Save task into queue of needScheduleTasks + Env.getCurrentEnv().getRoutineLoadTaskScheduler().addTasksInQueue(result); + } finally { + writeUnlock(); + } + } + + @Override + public int calculateCurrentConcurrentTaskNum() { + int shardNum = openKinesisShards.size() + closedKinesisShards.size(); + if (desireTaskConcurrentNum == 0) { + desireTaskConcurrentNum = Config.max_routine_load_task_concurrent_num; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("current concurrent task number is min" + + "(shard num: {}, desire task concurrent num: {}, config: {})", + shardNum, desireTaskConcurrentNum, Config.max_routine_load_task_concurrent_num); + } + currentTaskConcurrentNum = Math.min(shardNum, Math.min(desireTaskConcurrentNum, + Config.max_routine_load_task_concurrent_num)); + return currentTaskConcurrentNum; + } + + @Override + protected boolean checkCommitInfo(RLTaskTxnCommitAttachment rlTaskTxnCommitAttachment, + TransactionState txnState, + TransactionState.TxnStatusChangeReason txnStatusChangeReason) { + if (txnState.getTransactionStatus() == TransactionStatus.COMMITTED + || txnState.getTransactionStatus() == TransactionStatus.VISIBLE) { + return true; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("no need to update the progress of kinesis routine load. txn status: {}, " + + "txnStatusChangeReason: {}, task: {}, job: {}", + txnState.getTransactionStatus(), txnStatusChangeReason, + DebugUtil.printId(rlTaskTxnCommitAttachment.getTaskId()), id); + } + return false; + } + + private void updateProgressAndOffsetsCache(RLTaskTxnCommitAttachment attachment) { + KinesisProgress taskProgress = (KinesisProgress) attachment.getProgress(); + + // Keep the latest observed MillisBehindLatest per shard instead of the historical max. + taskProgress.getShardIdToMillsBehindLatest().forEach(cachedShardWithMillsBehindLatest::put); + + // Handle closed shards: move from open to closed list + if (taskProgress.getClosedShardIds() != null && !taskProgress.getClosedShardIds().isEmpty()) { + for (String closedShardId : taskProgress.getClosedShardIds()) { + if (openKinesisShards.remove(closedShardId)) { + if (!closedKinesisShards.contains(closedShardId)) { + closedKinesisShards.add(closedShardId); + LOG.info("Moved shard from open to closed: {}, job: {}", closedShardId, id); + } + } + } + } + + // Update progress (this will remove fully consumed shards from progress) + this.progress.update(attachment); + + if (taskProgress.getClosedShardIds() != null && !taskProgress.getClosedShardIds().isEmpty()) { + taskProgress.getClosedShardIds().forEach(cachedShardWithMillsBehindLatest::remove); + } + + // Remove fully consumed shards from closed list + closedKinesisShards.removeIf(shardId -> !((KinesisProgress) progress).containsShard(shardId)); + } + + @Override + protected void updateProgress(RLTaskTxnCommitAttachment attachment) throws UserException { + updateProgressAndOffsetsCache(attachment); + super.updateProgress(attachment); + } + + @Override + protected void replayUpdateProgress(RLTaskTxnCommitAttachment attachment) { + super.replayUpdateProgress(attachment); + updateProgressAndOffsetsCache(attachment); + } + + @Override + protected RoutineLoadTaskInfo unprotectRenewTask(RoutineLoadTaskInfo routineLoadTaskInfo, boolean delaySchedule) { + KinesisTaskInfo oldKinesisTaskInfo = (KinesisTaskInfo) routineLoadTaskInfo; + // Add new task + KinesisTaskInfo kinesisTaskInfo = new KinesisTaskInfo(oldKinesisTaskInfo, + ((KinesisProgress) progress).getShardIdToSequenceNumber(oldKinesisTaskInfo.getShards()), + isMultiTable()); + kinesisTaskInfo.setDelaySchedule(delaySchedule); + // Remove old task + routineLoadTaskInfoList.remove(routineLoadTaskInfo); + // Add new task + routineLoadTaskInfoList.add(kinesisTaskInfo); + return kinesisTaskInfo; + } + + @Override + protected void unprotectUpdateProgress() throws UserException { + updateNewShardProgress(); + } + + @Override + protected boolean refreshKafkaPartitions(boolean needAutoResume) throws UserException { + // For Kinesis, we refresh shards instead of Kafka partitions + if (this.state == JobState.RUNNING || this.state == JobState.NEED_SCHEDULE || needAutoResume) { + if (customKinesisShards != null && !customKinesisShards.isEmpty()) { + return true; + } + return updateKinesisShards(); + } + return true; + } + + private boolean updateKinesisShards() throws UserException { + try { + this.newCurrentKinesisShards = getAllKinesisShards(); + } catch (Exception e) { + String msg = e.getMessage() + + " may be Kinesis properties set in job is error" + + " or no shard in this stream that should check Kinesis"; + LOG.warn(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("error_msg", msg) + .build(), e); + if (this.state == JobState.NEED_SCHEDULE) { + unprotectUpdateState(JobState.PAUSED, + new ErrorReason(InternalErrorCode.PARTITIONS_ERR, msg), + false /* not replay */); + } + return false; + } + return true; + } + + @Override + protected boolean unprotectNeedReschedule() throws UserException { + if (this.state == JobState.RUNNING || this.state == JobState.NEED_SCHEDULE) { + return isKinesisShardsChanged(); + } + return false; + } + + private boolean isKinesisShardsChanged() throws UserException { + if (CollectionUtils.isNotEmpty(customKinesisShards)) { + if (Config.isCloudMode() && (openKinesisShards.isEmpty() && closedKinesisShards.isEmpty())) { + updateCloudProgress(); + } + openKinesisShards = customKinesisShards; + closedKinesisShards.clear(); + return false; + } + + Preconditions.checkNotNull(this.newCurrentKinesisShards); + + // newCurrentKinesisShards contains only OPEN shards. When an existing shard disappears + // from this list but still has progress, it has become a retired parent shard after + // split/merge and must continue draining from closedKinesisShards. + Set newShards = new HashSet<>(this.newCurrentKinesisShards); + if (syncShardTrackingFromLatestOpenShards(newShards)) { + return true; + } + + // Check if progress is consistent for all tracked shards (open + closed) + Set allTrackedShards = new HashSet<>(newShards); + allTrackedShards.addAll(closedKinesisShards); + for (String shardId : allTrackedShards) { + if (!((KinesisProgress) progress).containsShard(shardId)) { + return true; + } + } + return false; + } + + private boolean syncShardTrackingFromLatestOpenShards(Set latestOpenShards) { + Set currentOpenShards = new HashSet<>(openKinesisShards); + Set currentClosedShards = new HashSet<>(closedKinesisShards); + Set updatedClosedShards = new HashSet<>(currentClosedShards); + + for (String shardId : currentOpenShards) { + if (!latestOpenShards.contains(shardId) && ((KinesisProgress) progress).containsShard(shardId)) { + if (updatedClosedShards.add(shardId)) { + LOG.info("Moved shard from open to closed after shard refresh: {}, job: {}", shardId, id); + } + } + } + + boolean openChanged = !latestOpenShards.equals(currentOpenShards); + boolean closedChanged = !updatedClosedShards.equals(currentClosedShards); + if (!openChanged && !closedChanged) { + return false; + } + + openKinesisShards = new ArrayList<>(latestOpenShards); + closedKinesisShards = new ArrayList<>(updatedClosedShards); + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("open_kinesis_shards", Joiner.on(",").join(openKinesisShards)) + .add("closed_kinesis_shards", Joiner.on(",").join(closedKinesisShards)) + .add("msg", "kinesis shards changed") + .build()); + } + return true; + } + + @Override + protected boolean needAutoResume() { + writeLock(); + try { + if (this.state == JobState.PAUSED) { + return ScheduleRule.isNeedAutoSchedule(this); + } + return false; + } finally { + writeUnlock(); + } + } + + @Override + public String getStatistic() { + Map summary = this.jobStatistic.summary(); + readLock(); + try { + summary.put("openShardNum", openKinesisShards.size()); + summary.put("closedShardNum", closedKinesisShards.size()); + summary.put("trackedShardNum", ((KinesisProgress) progress).getShardIdToSequenceNumber().size()); + summary.put("cachedMillisBehindLatestShardNum", cachedShardWithMillsBehindLatest.size()); + summary.put("totalMillisBehindLatest", totalLag()); + long maxMillisBehindLatest = cachedShardWithMillsBehindLatest.values().stream() + .filter(lag -> lag >= 0) + .mapToLong(v -> v) + .max() + .orElse(-1L); + summary.put("maxMillisBehindLatest", maxMillisBehindLatest); + } finally { + readUnlock(); + } + Gson gson = new GsonBuilder().disableHtmlEscaping().create(); + return gson.toJson(summary); + } + + /** + * Get all shards from the Kinesis stream. + * Delegates to a BE node via gRPC, which calls AWS ListShards API using the SDK. + */ + private List getAllKinesisShards() throws UserException { + convertCustomProperties(false); + if (!customKinesisShards.isEmpty()) { + return customKinesisShards; + } + return KinesisUtil.getAllKinesisShards(region, stream, endpoint, convertedCustomProperties); + } + + /** + * Create a KinesisRoutineLoadJob from CreateRoutineLoadInfo. + */ + public static KinesisRoutineLoadJob fromCreateInfo(CreateRoutineLoadInfo info, ConnectContext ctx) + throws UserException { + if (Config.isCloudMode()) { + throw new DdlException("Kinesis routine load does not support cloud mode"); + } + Database db = Env.getCurrentInternalCatalog().getDbOrDdlException(info.getDBName()); + + long id = Env.getCurrentEnv().getNextId(); + KinesisDataSourceProperties kinesisProperties = + (KinesisDataSourceProperties) info.getDataSourceProperties(); + KinesisRoutineLoadJob kinesisRoutineLoadJob; + + if (kinesisProperties.isMultiTable()) { + kinesisRoutineLoadJob = new KinesisRoutineLoadJob(id, info.getName(), + db.getId(), + kinesisProperties.getRegion(), kinesisProperties.getStream(), + ctx.getCurrentUserIdentity(), true); + } else { + OlapTable olapTable = db.getOlapTableOrDdlException(info.getTableName()); + checkMeta(olapTable, info.getRoutineLoadDesc()); + // Check load_to_single_tablet compatibility + if (info.isLoadToSingleTablet() + && !(olapTable.getDefaultDistributionInfo() instanceof RandomDistributionInfo)) { + throw new DdlException( + "if load_to_single_tablet set to true, the olap table must be with random distribution"); + } + long tableId = olapTable.getId(); + kinesisRoutineLoadJob = new KinesisRoutineLoadJob(id, info.getName(), + db.getId(), tableId, + kinesisProperties.getRegion(), kinesisProperties.getStream(), + ctx.getCurrentUserIdentity()); + } + + kinesisRoutineLoadJob.setOptional(info); + kinesisRoutineLoadJob.checkCustomProperties(); + + return kinesisRoutineLoadJob; + } + + private void checkCustomProperties() throws DdlException { + // Validate custom properties if needed + } + + private void updateNewShardProgress() throws UserException { + // Check if this is initial setup (no shards in progress yet) + boolean isInitialSetup = !((KinesisProgress) progress).hasShards(); + + // Combine open and closed shards + List allShards = Lists.newArrayList(); + allShards.addAll(openKinesisShards); + allShards.addAll(closedKinesisShards); + + for (String shardId : allShards) { + if (!((KinesisProgress) progress).containsShard(shardId)) { + String startPosition; + + if (isInitialSetup) { + // Initial shards: use user-configured default position + startPosition = convertedDefaultPosition(); + } else { + // New shards discovered later: always use TRIM_HORIZON to avoid data loss + startPosition = KinesisProgress.TRIM_HORIZON_VAL; + LOG.info("New shard detected: {}, starting from TRIM_HORIZON to avoid data loss", + shardId); + } + + ((KinesisProgress) progress).addShardPosition(Pair.of(shardId, startPosition)); + if (LOG.isDebugEnabled()) { + LOG.debug(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id) + .add("kinesis_shard_id", shardId) + .add("begin_position", startPosition) + .add("is_initial_setup", isInitialSetup) + .add("msg", "The new shard has been added in job")); + } + } + } + } + + private List> getNewShardPositionsFromDefault(List newShards) + throws UserException { + List> shardPositions = Lists.newArrayList(); + String defaultPosition = convertedDefaultPosition(); + for (String shardId : newShards) { + shardPositions.add(Pair.of(shardId, defaultPosition)); + } + return shardPositions; + } + + protected void setOptional(CreateRoutineLoadInfo info) throws UserException { + super.setOptional(info); + KinesisDataSourceProperties kinesisDataSourceProperties = + (KinesisDataSourceProperties) info.getDataSourceProperties(); + + // Set endpoint if provided + if (kinesisDataSourceProperties.getEndpoint() != null) { + this.endpoint = kinesisDataSourceProperties.getEndpoint(); + } + + // Set custom shards and positions + if (CollectionUtils.isNotEmpty(kinesisDataSourceProperties.getKinesisShardPositions())) { + setCustomKinesisShards(kinesisDataSourceProperties); + } + + // Set custom properties + if (MapUtils.isNotEmpty(kinesisDataSourceProperties.getCustomKinesisProperties())) { + setCustomKinesisProperties(kinesisDataSourceProperties.getCustomKinesisProperties()); + } + } + + private void setCustomKinesisShards(KinesisDataSourceProperties kinesisDataSourceProperties) throws LoadException { + List> shardPositions = kinesisDataSourceProperties.getKinesisShardPositions(); + for (Pair shardPosition : shardPositions) { + this.customKinesisShards.add(shardPosition.first); + ((KinesisProgress) progress).addShardPosition(shardPosition); + } + } + + private void setCustomKinesisProperties(Map kinesisProperties) { + this.customProperties = kinesisProperties; + } + + @Override + public String dataSourcePropertiesJsonToString() { + Map dataSourceProperties = Maps.newHashMap(); + dataSourceProperties.put("region", region); + dataSourceProperties.put("stream", stream); + if (endpoint != null) { + dataSourceProperties.put("endpoint", endpoint); + } + List sortedOpenShards = Lists.newArrayList(openKinesisShards); + Collections.sort(sortedOpenShards); + dataSourceProperties.put("openKinesisShards", Joiner.on(",").join(sortedOpenShards)); + + List sortedClosedShards = Lists.newArrayList(closedKinesisShards); + Collections.sort(sortedClosedShards); + dataSourceProperties.put("closedKinesisShards", Joiner.on(",").join(sortedClosedShards)); + + Gson gson = new GsonBuilder().disableHtmlEscaping().create(); + return gson.toJson(dataSourceProperties); + } + + @Override + public String customPropertiesJsonToString() { + Gson gson = new GsonBuilder().disableHtmlEscaping().create(); + // Mask sensitive information + Map maskedProperties = new HashMap<>(customProperties); + if (maskedProperties.containsKey("aws.secret_key")) { + maskedProperties.put("aws.secret_key", "******"); + } + if (maskedProperties.containsKey("aws.session_key")) { + maskedProperties.put("aws.session_key", "******"); + } + return gson.toJson(maskedProperties); + } + + @Override + public Map getDataSourceProperties() { + Map dataSourceProperties = Maps.newHashMap(); + dataSourceProperties.put(KinesisConfiguration.KINESIS_REGION.getName(), region); + dataSourceProperties.put("kinesis_stream", stream); + if (endpoint != null) { + dataSourceProperties.put("kinesis_endpoint", endpoint); + } + return dataSourceProperties; + } + + @Override + public Map getCustomProperties() { + Map ret = new HashMap<>(); + customProperties.forEach((k, v) -> { + // Mask sensitive values + if (k.equals("aws.secret_key") || k.equals("aws.session_key")) { + ret.put("property." + k, "******"); + } else { + ret.put("property." + k, v); + } + }); + return ret; + } + + @Override + public void modifyProperties(AlterRoutineLoadCommand command) throws UserException { + Map jobProperties = command.getAnalyzedJobProperties(); + KinesisDataSourceProperties dataSourceProperties = + (KinesisDataSourceProperties) command.getDataSourceProperties(); + + writeLock(); + try { + if (getState() != JobState.PAUSED) { + throw new DdlException("Only supports modification of PAUSED jobs"); + } + + modifyPropertiesInternal(jobProperties, dataSourceProperties); + + AlterRoutineLoadJobOperationLog log = new AlterRoutineLoadJobOperationLog(this.id, + jobProperties, dataSourceProperties); + Env.getCurrentEnv().getEditLog().logAlterRoutineLoadJob(log); + } finally { + writeUnlock(); + } + } + + private void modifyPropertiesInternal(Map jobProperties, + KinesisDataSourceProperties dataSourceProperties) + throws UserException { + if (dataSourceProperties != null) { + List> shardPositions = Lists.newArrayList(); + Map customKinesisProperties = Maps.newHashMap(); + boolean resetProgress = false; + boolean hasExplicitShardPositions = false; + + if (MapUtils.isNotEmpty(dataSourceProperties.getOriginalDataSourceProperties())) { + shardPositions = dataSourceProperties.getKinesisShardPositions(); + customKinesisProperties = dataSourceProperties.getCustomKinesisProperties(); + hasExplicitShardPositions = !shardPositions.isEmpty(); + } + + // Update custom properties + if (!customKinesisProperties.isEmpty()) { + this.customProperties.putAll(customKinesisProperties); + convertCustomProperties(true); + } + + // Modify stream if provided + if (!Strings.isNullOrEmpty(dataSourceProperties.getStream())) { + this.stream = dataSourceProperties.getStream(); + resetProgress = true; + } + + // Modify region if provided + if (!Strings.isNullOrEmpty(dataSourceProperties.getRegion())) { + this.region = dataSourceProperties.getRegion(); + } + + // Modify endpoint if provided + if (!Strings.isNullOrEmpty(dataSourceProperties.getEndpoint())) { + this.endpoint = dataSourceProperties.getEndpoint(); + } + + if (resetProgress) { + this.progress = new KinesisProgress(); + this.openKinesisShards.clear(); + this.closedKinesisShards.clear(); + this.cachedShardWithMillsBehindLatest.clear(); + } + + if (hasExplicitShardPositions) { + this.customKinesisShards.clear(); + for (Pair shardPosition : shardPositions) { + this.customKinesisShards.add(shardPosition.first); + } + } else if (resetProgress) { + // Stream change without explicit shards should fall back to dynamic shard discovery. + this.customKinesisShards.clear(); + } + + if (!shardPositions.isEmpty()) { + if (!resetProgress) { + ((KinesisProgress) progress).checkShards(shardPositions); + } + ((KinesisProgress) progress).modifyPosition(shardPositions); + } + } + + if (!jobProperties.isEmpty()) { + Map copiedJobProperties = Maps.newHashMap(jobProperties); + modifyCommonJobProperties(copiedJobProperties); + this.jobProperties.putAll(copiedJobProperties); + if (jobProperties.containsKey(CreateRoutineLoadInfo.PARTIAL_COLUMNS)) { + this.isPartialUpdate = BooleanUtils.toBoolean(jobProperties.get(CreateRoutineLoadInfo.PARTIAL_COLUMNS)); + } + if (jobProperties.containsKey(CreateRoutineLoadInfo.PARTIAL_UPDATE_NEW_KEY_POLICY)) { + String policy = jobProperties.get(CreateRoutineLoadInfo.PARTIAL_UPDATE_NEW_KEY_POLICY); + if ("ERROR".equalsIgnoreCase(policy)) { + this.partialUpdateNewKeyPolicy = TPartialUpdateNewRowPolicy.ERROR; + } else { + this.partialUpdateNewKeyPolicy = TPartialUpdateNewRowPolicy.APPEND; + } + } + } + LOG.info("modify the properties of kinesis routine load job: {}, jobProperties: {}, datasource properties: {}", + this.id, jobProperties, dataSourceProperties); + } + + @Override + public void replayModifyProperties(AlterRoutineLoadJobOperationLog log) { + try { + modifyPropertiesInternal(log.getJobProperties(), + (KinesisDataSourceProperties) log.getDataSourceProperties()); + } catch (UserException e) { + LOG.error("failed to replay modify kinesis routine load job: {}", id, e); + } + } + + @Override + public String getLag() { + Map shardIdToLag = ((KinesisProgress) progress).getLag(cachedShardWithMillsBehindLatest); + Gson gson = new Gson(); + return gson.toJson(shardIdToLag); + } + + @Override + public TFileCompressType getCompressType() { + return TFileCompressType.PLAIN; + } + + @Override + public double getMaxFilterRatio() { + return maxFilterRatio; + } + + @Override + public Long totalProgress() { + return ((KinesisProgress) progress).totalProgress(); + } + + @Override + public Long totalLag() { + Map shardIdToLag = ((KinesisProgress) progress).getLag(cachedShardWithMillsBehindLatest); + return shardIdToLag.values().stream() + .filter(lag -> lag >= 0) + .mapToLong(v -> v) + .sum(); + } + + /** + * Check if there is more data to consume from Kinesis shards. + * + * Kinesis does not provide a cheap FE-side API equivalent to Kafka's latest offset query. + * So FE cannot rely on cached lag to block scheduling, otherwise a task can get stuck after + * catching up once and never probe for newly arrived records. Keep polling and let BE's + * GetRecords result decide whether this round has data. + */ + public boolean hasMoreDataToConsume(UUID taskId, Map shardIdToSequenceNumber) + throws UserException { + if (LOG.isDebugEnabled() && !cachedShardWithMillsBehindLatest.isEmpty()) { + boolean allCaughtUp = true; + for (String shardId : shardIdToSequenceNumber.keySet()) { + Long millis = cachedShardWithMillsBehindLatest.get(shardId); + if (millis == null || millis > 0) { + allCaughtUp = false; + break; + } + } + if (allCaughtUp) { + LOG.debug("All shards are caught up by cached MillisBehindLatest, but keep polling. job {}, task {}", + id, taskId); + } + } + return true; + } + + @Override + public NereidsRoutineLoadTaskInfo toNereidsRoutineLoadTaskInfo() throws UserException { + Expression deleteCondition = getDeleteCondition() != null + ? NereidsLoadUtils.parseExpressionSeq( + getDeleteCondition().accept(ExprToSqlVisitor.INSTANCE, + ToSqlParams.WITHOUT_TABLE)).get(0) + : null; + Expression precedingFilter = getPrecedingFilter() != null + ? NereidsLoadUtils.parseExpressionSeq( + getPrecedingFilter().accept(ExprToSqlVisitor.INSTANCE, + ToSqlParams.WITHOUT_TABLE)).get(0) + : null; + Expression whereExpr = getWhereExpr() != null + ? NereidsLoadUtils.parseExpressionSeq(getWhereExpr().accept( + ExprToSqlVisitor.INSTANCE, ToSqlParams.WITHOUT_TABLE)).get(0) + : null; + NereidsLoadTaskInfo.NereidsImportColumnDescs importColumnDescs = null; + if (columnDescs != null) { + importColumnDescs = new NereidsLoadTaskInfo.NereidsImportColumnDescs(); + for (ImportColumnDesc desc : columnDescs.descs) { + Expression expression = desc.getExpr() != null + ? NereidsLoadUtils.parseExpressionSeq(desc.getExpr().accept( + ExprToSqlVisitor.INSTANCE, ToSqlParams.WITHOUT_TABLE)).get(0) + : null; + importColumnDescs.descs.add(new NereidsImportColumnDesc(desc.getColumnName(), expression)); + } + } + return new NereidsRoutineLoadTaskInfo(execMemLimit, new HashMap<>(jobProperties), maxBatchIntervalS, + partitionNamesInfo, mergeType, deleteCondition, sequenceCol, maxFilterRatio, importColumnDescs, + precedingFilter, whereExpr, columnSeparator, lineDelimiter, enclose, escape, sendBatchParallelism, + loadToSingleTablet, uniqueKeyUpdateMode, partialUpdateNewKeyPolicy, memtableOnSinkNode); + } + + @Override + public void updateCloudProgress() throws UserException { + throw new UserException("Kinesis routine load does not support cloud mode"); + } + + @Override + protected void updateCloudProgress(RLTaskTxnCommitAttachment attachment) { + throw new IllegalStateException("Kinesis routine load does not support cloud mode"); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisTaskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisTaskInfo.java new file mode 100644 index 00000000000000..9c225a51250ec4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/load/routineload/kinesis/KinesisTaskInfo.java @@ -0,0 +1,260 @@ +// 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.load.routineload.kinesis; + +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.Config; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.load.routineload.RoutineLoadJob; +import org.apache.doris.load.routineload.RoutineLoadManager; +import org.apache.doris.load.routineload.RoutineLoadTaskInfo; +import org.apache.doris.nereids.load.NereidsLoadTaskInfo; +import org.apache.doris.nereids.load.NereidsStreamLoadPlanner; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.thrift.TFileFormatType; +import org.apache.doris.thrift.TKinesisLoadInfo; +import org.apache.doris.thrift.TLoadSourceType; +import org.apache.doris.thrift.TPipelineFragmentParams; +import org.apache.doris.thrift.TPipelineWorkloadGroup; +import org.apache.doris.thrift.TPlanFragment; +import org.apache.doris.thrift.TRoutineLoadTask; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.base.Joiner; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import org.apache.commons.lang3.StringUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; + +/** + * Task info for Kinesis Routine Load. + * + * Each task is responsible for consuming data from one or more Kinesis shards. + * The task tracks the sequence number for each shard and reports progress back + * to the FE after successful consumption. + */ +public class KinesisTaskInfo extends RoutineLoadTaskInfo { + private static final Logger LOG = LogManager.getLogger(KinesisTaskInfo.class); + + private RoutineLoadManager getRoutineLoadManager() { + return Env.getCurrentEnv().getRoutineLoadManager(); + } + + /** + * Map from shard ID to starting sequence number for this task. + */ + private Map shardIdToSequenceNumber = Maps.newHashMap(); + + /** + * Create a new KinesisTaskInfo. + * + * @param id Task ID + * @param jobId Job ID + * @param timeoutMs Timeout in milliseconds + * @param shardIdToSequenceNumber Initial shard positions + * @param isMultiTable Whether this is a multi-table job + */ + public KinesisTaskInfo(UUID id, long jobId, long timeoutMs, + Map shardIdToSequenceNumber, + boolean isMultiTable, long taskSubmitTimeMs, boolean isEof) { + super(id, jobId, timeoutMs, isMultiTable, taskSubmitTimeMs, isEof); + this.shardIdToSequenceNumber.putAll(shardIdToSequenceNumber); + } + + /** + * Create a new task from an old task with updated positions. + */ + public KinesisTaskInfo(KinesisTaskInfo oldTask, ConcurrentMap shardIdToSequenceNumber, + boolean isMultiTable) { + super(UUID.randomUUID(), oldTask.getJobId(), oldTask.getTimeoutMs(), + oldTask.getBeId(), isMultiTable, oldTask.getLastScheduledTime(), oldTask.getIsEof()); + this.shardIdToSequenceNumber.putAll(shardIdToSequenceNumber); + } + + /** + * Get the list of shard IDs this task is responsible for. + */ + public List getShards() { + return new ArrayList<>(shardIdToSequenceNumber.keySet()); + } + + /** + * Get the shard to sequence number mapping. + */ + public Map getShardIdToSequenceNumber() { + return shardIdToSequenceNumber; + } + + @Override + public TRoutineLoadTask createRoutineLoadTask() throws UserException { + KinesisRoutineLoadJob routineLoadJob = + (KinesisRoutineLoadJob) getRoutineLoadManager().getJob(jobId); + // Create Thrift task object + TRoutineLoadTask tRoutineLoadTask = new TRoutineLoadTask(); + TUniqueId queryId = new TUniqueId(id.getMostSignificantBits(), id.getLeastSignificantBits()); + tRoutineLoadTask.setId(queryId); + tRoutineLoadTask.setJobId(jobId); + tRoutineLoadTask.setTxnId(txnId); + + Database database = Env.getCurrentInternalCatalog().getDbOrMetaException(routineLoadJob.getDbId()); + tRoutineLoadTask.setDb(database.getFullName()); + + // label = job_name+job_id+task_id+txn_id + String label = Joiner.on("-").join(routineLoadJob.getName(), + routineLoadJob.getId(), DebugUtil.printId(id), txnId); + tRoutineLoadTask.setLabel(label); + tRoutineLoadTask.setAuthCode(routineLoadJob.getAuthCode()); + + // Set Kinesis-specific load info + TKinesisLoadInfo tKinesisLoadInfo = new TKinesisLoadInfo(); + tKinesisLoadInfo.setRegion(routineLoadJob.getRegion()); + tKinesisLoadInfo.setStream(routineLoadJob.getStream()); + if (routineLoadJob.getEndpoint() != null) { + tKinesisLoadInfo.setEndpoint(routineLoadJob.getEndpoint()); + } + tKinesisLoadInfo.setShardBeginSequenceNumber(shardIdToSequenceNumber); + tKinesisLoadInfo.setProperties(routineLoadJob.getConvertedCustomProperties()); + + tRoutineLoadTask.setKinesisLoadInfo(tKinesisLoadInfo); + tRoutineLoadTask.setType(TLoadSourceType.KINESIS); + tRoutineLoadTask.setIsMultiTable(isMultiTable); + + // Set batch parameters + adaptiveBatchParam(tRoutineLoadTask, routineLoadJob); + + if (!isMultiTable) { + Table tbl = database.getTableOrMetaException(routineLoadJob.getTableId()); + tRoutineLoadTask.setTbl(tbl.getName()); + tRoutineLoadTask.setPipelineParams(rePlan(routineLoadJob)); + } else { + Env.getCurrentEnv().getRoutineLoadManager().addMultiLoadTaskTxnIdToRoutineLoadJobId(txnId, jobId); + } + + // Set format + if (!routineLoadJob.getFormat().isEmpty() && routineLoadJob.getFormat().equalsIgnoreCase("json")) { + tRoutineLoadTask.setFormat(TFileFormatType.FORMAT_JSON); + } else { + tRoutineLoadTask.setFormat(TFileFormatType.FORMAT_CSV_PLAIN); + } + + tRoutineLoadTask.setMemtableOnSinkNode(routineLoadJob.isMemtableOnSinkNode()); + tRoutineLoadTask.setQualifiedUser(routineLoadJob.getUserIdentity().getQualifiedUser()); + tRoutineLoadTask.setCloudCluster(routineLoadJob.getCloudCluster()); + + return tRoutineLoadTask; + } + + /** + * Set adaptive batch parameters based on the routine load job configuration. + */ + private void adaptiveBatchParam(TRoutineLoadTask tRoutineLoadTask, RoutineLoadJob routineLoadJob) { + long maxBatchIntervalS = routineLoadJob.getMaxBatchIntervalS(); + long maxBatchRows = routineLoadJob.getMaxBatchRows(); + long maxBatchSize = routineLoadJob.getMaxBatchSizeBytes(); + if (!isEof) { + maxBatchIntervalS = Math.max(maxBatchIntervalS, Config.routine_load_adaptive_min_batch_interval_sec); + maxBatchRows = Math.max(maxBatchRows, RoutineLoadJob.DEFAULT_MAX_BATCH_ROWS); + maxBatchSize = Math.max(maxBatchSize, RoutineLoadJob.DEFAULT_MAX_BATCH_SIZE); + this.timeoutMs = maxBatchIntervalS * Config.routine_load_task_timeout_multiplier * 1000; + } else { + this.timeoutMs = routineLoadJob.getTimeout() * 1000; + } + tRoutineLoadTask.setMaxIntervalS(maxBatchIntervalS); + tRoutineLoadTask.setMaxBatchRows(maxBatchRows); + tRoutineLoadTask.setMaxBatchSize(maxBatchSize); + } + + private TPipelineFragmentParams rePlan(RoutineLoadJob routineLoadJob) throws UserException { + TUniqueId loadId = new TUniqueId(id.getMostSignificantBits(), id.getLeastSignificantBits()); + // plan for each task, in case table has change(rollup or schema change) + Database db = Env.getCurrentInternalCatalog().getDbOrMetaException(routineLoadJob.getDbId()); + NereidsLoadTaskInfo taskInfo = routineLoadJob.toNereidsRoutineLoadTaskInfo(); + taskInfo.setTimeout((int) (this.timeoutMs / 1000)); + NereidsStreamLoadPlanner planner = new NereidsStreamLoadPlanner(db, + (OlapTable) db.getTableOrMetaException(routineLoadJob.getTableId(), + Table.TableType.OLAP), taskInfo); + TPipelineFragmentParams tExecPlanFragmentParams = routineLoadJob.plan(planner, loadId, txnId); + TPlanFragment tPlanFragment = tExecPlanFragmentParams.getFragment(); + tPlanFragment.getOutputSink().getOlapTableSink().setTxnId(txnId); + + if (Config.enable_workload_group) { + try { + List tWgList = new ArrayList<>(); + + ConnectContext tmpContext = new ConnectContext(); + if (Config.isCloudMode()) { + tmpContext.setCloudCluster(routineLoadJob.getCloudCluster()); + } + tmpContext.setCurrentUserIdentity(routineLoadJob.getUserIdentity()); + + String wgName = routineLoadJob.getWorkloadGroup(); + if (!StringUtils.isEmpty(wgName)) { + tmpContext.getSessionVariable().setWorkloadGroup(wgName); + } + + tWgList = Env.getCurrentEnv().getWorkloadGroupMgr().getWorkloadGroup(tmpContext) + .stream() + .map(e -> e.toThrift()) + .collect(Collectors.toList()); + + if (tWgList.size() != 0) { + tExecPlanFragmentParams.setWorkloadGroups(tWgList); + } + } catch (Throwable t) { + LOG.info("Get workload group failed when replan kinesis, job id:{} , ", routineLoadJob.getTxnId(), t); + throw t; + } + } + + return tExecPlanFragmentParams; + } + + @Override + protected String getTaskDataSourceProperties() { + Gson gson = new Gson(); + return gson.toJson(shardIdToSequenceNumber); + } + + @Override + protected boolean hasMoreDataToConsume() throws UserException { + KinesisRoutineLoadJob routineLoadJob = (KinesisRoutineLoadJob) getRoutineLoadManager().getJob(jobId); + return routineLoadJob.hasMoreDataToConsume(id, shardIdToSequenceNumber); + } + + @Override + public String toString() { + return "KinesisTaskInfo{" + + "id=" + id + + ", jobId=" + jobId + + ", txnId=" + txnId + + ", shardIdToSequenceNumber=" + shardIdToSequenceNumber + + '}'; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java index 7a4fd2ae5e4a24..60f2e5b9e3c2c2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/gson/GsonUtils.java @@ -194,12 +194,15 @@ import org.apache.doris.load.loadv2.SparkLoadJob; import org.apache.doris.load.loadv2.SparkLoadJob.SparkLoadJobStateUpdateInfo; import org.apache.doris.load.routineload.AbstractDataSourceProperties; -import org.apache.doris.load.routineload.KafkaProgress; -import org.apache.doris.load.routineload.KafkaRoutineLoadJob; import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; import org.apache.doris.load.routineload.RoutineLoadJob; import org.apache.doris.load.routineload.RoutineLoadProgress; import org.apache.doris.load.routineload.kafka.KafkaDataSourceProperties; +import org.apache.doris.load.routineload.kafka.KafkaProgress; +import org.apache.doris.load.routineload.kafka.KafkaRoutineLoadJob; +import org.apache.doris.load.routineload.kinesis.KinesisDataSourceProperties; +import org.apache.doris.load.routineload.kinesis.KinesisProgress; +import org.apache.doris.load.routineload.kinesis.KinesisRoutineLoadJob; import org.apache.doris.mtmv.MTMVMaxTimestampSnapshot; import org.apache.doris.mtmv.MTMVSnapshotIdSnapshot; import org.apache.doris.mtmv.MTMVSnapshotIf; @@ -440,7 +443,10 @@ public class GsonUtils { private static RuntimeTypeAdapterFactory rdsTypeAdapterFactory = RuntimeTypeAdapterFactory.of( AbstractDataSourceProperties.class, "clazz") - .registerSubtype(KafkaDataSourceProperties.class, KafkaDataSourceProperties.class.getSimpleName()); + .registerSubtype(KafkaDataSourceProperties.class, + KafkaDataSourceProperties.class.getSimpleName()) + .registerSubtype(KinesisDataSourceProperties.class, + KinesisDataSourceProperties.class.getSimpleName()); private static RuntimeTypeAdapterFactory jobExecutorRuntimeTypeAdapterFactory = RuntimeTypeAdapterFactory.of(org.apache.doris.job.base.AbstractJob.class, "clazz") @@ -566,12 +572,14 @@ public class GsonUtils { private static RuntimeTypeAdapterFactory routineLoadTypeAdapterFactory = RuntimeTypeAdapterFactory.of(RoutineLoadProgress.class, "clazz") .registerDefaultSubtype(RoutineLoadProgress.class) - .registerSubtype(KafkaProgress.class, KafkaProgress.class.getSimpleName()); + .registerSubtype(KafkaProgress.class, KafkaProgress.class.getSimpleName()) + .registerSubtype(KinesisProgress.class, KinesisProgress.class.getSimpleName()); private static RuntimeTypeAdapterFactory routineLoadJobTypeAdapterFactory = RuntimeTypeAdapterFactory.of(RoutineLoadJob.class, "clazz") .registerDefaultSubtype(RoutineLoadJob.class) - .registerSubtype(KafkaRoutineLoadJob.class, KafkaRoutineLoadJob.class.getSimpleName()); + .registerSubtype(KafkaRoutineLoadJob.class, KafkaRoutineLoadJob.class.getSimpleName()) + .registerSubtype(KinesisRoutineLoadJob.class, KinesisRoutineLoadJob.class.getSimpleName()); private static RuntimeTypeAdapterFactory jobBackupTypeAdapterFactory diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java index e9fdb5f923bc00..e08853a73a2b68 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java @@ -38,6 +38,8 @@ import org.apache.doris.load.loadv2.LoadTask; import org.apache.doris.load.routineload.kafka.KafkaConfiguration; import org.apache.doris.load.routineload.kafka.KafkaDataSourceProperties; +import org.apache.doris.load.routineload.kafka.KafkaRoutineLoadJob; +import org.apache.doris.load.routineload.kafka.KafkaTaskInfo; import org.apache.doris.mysql.privilege.MockedAuth; import org.apache.doris.nereids.trees.plans.commands.info.CreateRoutineLoadInfo; import org.apache.doris.nereids.trees.plans.commands.info.LabelNameInfo; diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KinesisRoutineLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KinesisRoutineLoadJobTest.java new file mode 100644 index 00000000000000..aa1dc052605f2b --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/KinesisRoutineLoadJobTest.java @@ -0,0 +1,339 @@ +// 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.load.routineload; + +import org.apache.doris.analysis.UserIdentity; +import org.apache.doris.common.Config; +import org.apache.doris.common.jmockit.Deencapsulation; +import org.apache.doris.load.routineload.kinesis.KinesisConfiguration; +import org.apache.doris.load.routineload.kinesis.KinesisDataSourceProperties; +import org.apache.doris.load.routineload.kinesis.KinesisProgress; +import org.apache.doris.load.routineload.kinesis.KinesisRoutineLoadJob; +import org.apache.doris.load.routineload.kinesis.KinesisTaskInfo; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.gson.Gson; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +public class KinesisRoutineLoadJobTest { + + @Test + public void testRoutineLoadTaskConcurrentNum() { + int oldMaxConcurrent = Config.max_routine_load_task_concurrent_num; + try { + Config.max_routine_load_task_concurrent_num = 6; + KinesisRoutineLoadJob routineLoadJob = + new KinesisRoutineLoadJob(1L, "kinesis_routine_load_job", 1L, + 1L, "ap-southeast-1", "stream-1", UserIdentity.ADMIN); + Deencapsulation.setField(routineLoadJob, "openKinesisShards", + Lists.newArrayList("shard-0", "shard-1")); + Deencapsulation.setField(routineLoadJob, "closedKinesisShards", + Lists.newArrayList("shard-2")); + + Assert.assertEquals(3, routineLoadJob.calculateCurrentConcurrentTaskNum()); + + Deencapsulation.setField(routineLoadJob, "desireTaskConcurrentNum", 2); + Assert.assertEquals(2, routineLoadJob.calculateCurrentConcurrentTaskNum()); + + Config.max_routine_load_task_concurrent_num = 1; + Assert.assertEquals(1, routineLoadJob.calculateCurrentConcurrentTaskNum()); + } finally { + Config.max_routine_load_task_concurrent_num = oldMaxConcurrent; + } + } + + @Test + public void testGetStatisticContainsKinesisFields() { + KinesisRoutineLoadJob routineLoadJob = + new KinesisRoutineLoadJob(1L, "kinesis_routine_load_job", 1L, + 1L, "ap-southeast-1", "stream-1", UserIdentity.ADMIN); + + Deencapsulation.setField(routineLoadJob, "openKinesisShards", + Lists.newArrayList("shard-0", "shard-1")); + Deencapsulation.setField(routineLoadJob, "closedKinesisShards", + Lists.newArrayList("shard-2")); + Map shardToSeqNum = new HashMap<>(); + shardToSeqNum.put("shard-0", "100"); + shardToSeqNum.put("shard-1", "200"); + shardToSeqNum.put("shard-2", "300"); + shardToSeqNum.put("shard-3", "400"); + Deencapsulation.setField(routineLoadJob, "progress", + new KinesisProgress(shardToSeqNum)); + Map shardToMillisBehindLatest = new HashMap<>(); + shardToMillisBehindLatest.put("shard-0", 100L); + shardToMillisBehindLatest.put("shard-1", 0L); + shardToMillisBehindLatest.put("shard-2", -1L); + Deencapsulation.setField(routineLoadJob, "cachedShardWithMillsBehindLatest", + shardToMillisBehindLatest); + + Gson gson = new Gson(); + Map statistic = gson.fromJson(routineLoadJob.getStatistic(), Map.class); + + Assert.assertEquals(2L, ((Number) statistic.get("openShardNum")).longValue()); + Assert.assertEquals(1L, ((Number) statistic.get("closedShardNum")).longValue()); + Assert.assertEquals(4L, ((Number) statistic.get("trackedShardNum")).longValue()); + Assert.assertEquals(3L, ((Number) statistic.get("cachedMillisBehindLatestShardNum")).longValue()); + Assert.assertEquals(100L, ((Number) statistic.get("totalMillisBehindLatest")).longValue()); + Assert.assertEquals(100L, ((Number) statistic.get("maxMillisBehindLatest")).longValue()); + } + + @Test + public void testHasMoreDataToConsumeShouldKeepPollingWhenLagCacheIsZero() throws Exception { + KinesisRoutineLoadJob routineLoadJob = + new KinesisRoutineLoadJob(1L, "kinesis_routine_load_job", 1L, + 1L, "ap-southeast-1", "stream-1", UserIdentity.ADMIN); + + Map shardToMillisBehindLatest = new HashMap<>(); + shardToMillisBehindLatest.put("shard-0", 0L); + Deencapsulation.setField(routineLoadJob, "cachedShardWithMillsBehindLatest", + shardToMillisBehindLatest); + + Map shardToSeqNum = new HashMap<>(); + shardToSeqNum.put("shard-0", "100"); + Assert.assertTrue(routineLoadJob.hasMoreDataToConsume(UUID.randomUUID(), shardToSeqNum)); + } + + @Test + public void testLagCacheShouldUseLatestReportInsteadOfHistoricalMax() throws Exception { + KinesisRoutineLoadJob routineLoadJob = + new KinesisRoutineLoadJob(1L, "kinesis_routine_load_job", 1L, + 1L, "ap-southeast-1", "stream-1", UserIdentity.ADMIN); + + Map shardToSeqNum = new HashMap<>(); + shardToSeqNum.put("shard-0", "100"); + Deencapsulation.setField(routineLoadJob, "progress", new KinesisProgress(shardToSeqNum)); + + Map cachedLag = new HashMap<>(); + cachedLag.put("shard-0", 60_000L); + Deencapsulation.setField(routineLoadJob, "cachedShardWithMillsBehindLatest", cachedLag); + + Map updatedSeqNum = new HashMap<>(); + updatedSeqNum.put("shard-0", "101"); + Map latestLag = new HashMap<>(); + latestLag.put("shard-0", 100L); + RLTaskTxnCommitAttachment attachment = + createCommitAttachment(createProgress(updatedSeqNum, latestLag)); + Deencapsulation.invoke(routineLoadJob, "updateProgressAndOffsetsCache", attachment); + + Map updatedLagCache = Deencapsulation.getField(routineLoadJob, "cachedShardWithMillsBehindLatest"); + Assert.assertEquals(100L, updatedLagCache.get("shard-0").longValue()); + + Gson gson = new Gson(); + Map statistic = gson.fromJson(routineLoadJob.getStatistic(), Map.class); + Assert.assertEquals(100L, ((Number) statistic.get("totalMillisBehindLatest")).longValue()); + Assert.assertEquals(100L, ((Number) statistic.get("maxMillisBehindLatest")).longValue()); + + Map lag = gson.fromJson(routineLoadJob.getLag(), Map.class); + Assert.assertEquals(100L, ((Number) lag.get("shard-0")).longValue()); + } + + @Test + public void testModifyPropertiesShouldClearStaleCustomShardsWhenStreamChanges() throws Exception { + KinesisRoutineLoadJob routineLoadJob = + new KinesisRoutineLoadJob(1L, "kinesis_routine_load_job", 1L, + 1L, "ap-southeast-1", "stream-1", UserIdentity.ADMIN); + + Deencapsulation.setField(routineLoadJob, "customKinesisShards", + Lists.newArrayList("shard-old-0", "shard-old-1")); + Deencapsulation.setField(routineLoadJob, "openKinesisShards", + Lists.newArrayList("shard-old-0")); + Deencapsulation.setField(routineLoadJob, "closedKinesisShards", + Lists.newArrayList("shard-old-1")); + Map oldProgress = new HashMap<>(); + oldProgress.put("shard-old-0", "100"); + oldProgress.put("shard-old-1", "200"); + Deencapsulation.setField(routineLoadJob, "progress", new KinesisProgress(oldProgress)); + Map oldLag = new HashMap<>(); + oldLag.put("shard-old-0", 10L); + Deencapsulation.setField(routineLoadJob, "cachedShardWithMillsBehindLatest", oldLag); + + Map alterProps = new HashMap<>(); + alterProps.put(KinesisConfiguration.KINESIS_STREAM.getName(), "stream-2"); + KinesisDataSourceProperties dataSourceProperties = new KinesisDataSourceProperties(alterProps); + dataSourceProperties.setAlter(true); + dataSourceProperties.setTimezone("Asia/Shanghai"); + dataSourceProperties.analyze(); + + Deencapsulation.invoke(routineLoadJob, "modifyPropertiesInternal", + new HashMap(), dataSourceProperties); + + Assert.assertEquals("stream-2", Deencapsulation.getField(routineLoadJob, "stream")); + + List customKinesisShards = Deencapsulation.getField(routineLoadJob, "customKinesisShards"); + Assert.assertTrue(customKinesisShards.isEmpty()); + List openKinesisShards = Deencapsulation.getField(routineLoadJob, "openKinesisShards"); + Assert.assertTrue(openKinesisShards.isEmpty()); + List closedKinesisShards = Deencapsulation.getField(routineLoadJob, "closedKinesisShards"); + Assert.assertTrue(closedKinesisShards.isEmpty()); + + KinesisProgress progress = Deencapsulation.getField(routineLoadJob, "progress"); + Assert.assertFalse(progress.hasShards()); + Map cachedLag = Deencapsulation.getField(routineLoadJob, "cachedShardWithMillsBehindLatest"); + Assert.assertTrue(cachedLag.isEmpty()); + } + + @Test + public void testModifyPropertiesShouldReplaceCustomShardsWhenExplicitShardsProvided() throws Exception { + KinesisRoutineLoadJob routineLoadJob = + new KinesisRoutineLoadJob(1L, "kinesis_routine_load_job", 1L, + 1L, "ap-southeast-1", "stream-1", UserIdentity.ADMIN); + + Deencapsulation.setField(routineLoadJob, "customKinesisShards", + Lists.newArrayList("shard-0")); + Map oldProgress = new HashMap<>(); + oldProgress.put("shard-0", "10"); + oldProgress.put("shard-1", "20"); + oldProgress.put("shard-2", "30"); + Deencapsulation.setField(routineLoadJob, "progress", new KinesisProgress(oldProgress)); + + Map alterProps = new HashMap<>(); + alterProps.put(KinesisConfiguration.KINESIS_SHARDS.getName(), "shard-1,shard-2"); + alterProps.put(KinesisConfiguration.KINESIS_POSITIONS.getName(), "101,202"); + KinesisDataSourceProperties dataSourceProperties = new KinesisDataSourceProperties(alterProps); + dataSourceProperties.setAlter(true); + dataSourceProperties.setTimezone("Asia/Shanghai"); + dataSourceProperties.analyze(); + + Deencapsulation.invoke(routineLoadJob, "modifyPropertiesInternal", + new HashMap(), dataSourceProperties); + + List customKinesisShards = Deencapsulation.getField(routineLoadJob, "customKinesisShards"); + Assert.assertEquals(Lists.newArrayList("shard-1", "shard-2"), customKinesisShards); + + KinesisProgress progress = Deencapsulation.getField(routineLoadJob, "progress"); + Assert.assertEquals("101", progress.getSequenceNumberByShard("shard-1")); + Assert.assertEquals("202", progress.getSequenceNumberByShard("shard-2")); + } + + @Test + public void testShardRefreshShouldMoveRetiredParentToClosedUntilConsumed() throws Exception { + KinesisRoutineLoadJob routineLoadJob = + new KinesisRoutineLoadJob(1L, "kinesis_routine_load_job", 1L, + 1L, "ap-southeast-1", "stream-1", UserIdentity.ADMIN); + + Map shardToSeqNum = new HashMap<>(); + shardToSeqNum.put("shard-parent", "100"); + Deencapsulation.setField(routineLoadJob, "progress", new KinesisProgress(shardToSeqNum)); + Deencapsulation.setField(routineLoadJob, "openKinesisShards", Lists.newArrayList("shard-parent")); + Deencapsulation.setField(routineLoadJob, "closedKinesisShards", Lists.newArrayList()); + Deencapsulation.setField(routineLoadJob, "newCurrentKinesisShards", + Lists.newArrayList("shard-child-0", "shard-child-1")); + + Assert.assertTrue((Boolean) Deencapsulation.invoke(routineLoadJob, "isKinesisShardsChanged")); + List openKinesisShards = Deencapsulation.getField(routineLoadJob, "openKinesisShards"); + List closedKinesisShards = Deencapsulation.getField(routineLoadJob, "closedKinesisShards"); + Assert.assertEquals(new HashSet<>(Lists.newArrayList("shard-child-0", "shard-child-1")), + new HashSet<>(openKinesisShards)); + Assert.assertEquals(new HashSet<>(Lists.newArrayList("shard-parent")), + new HashSet<>(closedKinesisShards)); + + Deencapsulation.invoke(routineLoadJob, "updateNewShardProgress"); + KinesisProgress progress = Deencapsulation.getField(routineLoadJob, "progress"); + Assert.assertTrue(progress.containsShard("shard-parent")); + Assert.assertTrue(progress.containsShard("shard-child-0")); + Assert.assertTrue(progress.containsShard("shard-child-1")); + + Deencapsulation.setField(routineLoadJob, "state", RoutineLoadJob.JobState.NEED_SCHEDULE); + routineLoadJob.divideRoutineLoadJob(2); + Assert.assertEquals(new HashSet<>(Lists.newArrayList("shard-parent", "shard-child-0", "shard-child-1")), + collectAssignedShards(routineLoadJob)); + } + + @Test + public void testFullyConsumedClosedParentShouldNotReappearOnRefresh() throws Exception { + KinesisRoutineLoadJob routineLoadJob = + new KinesisRoutineLoadJob(1L, "kinesis_routine_load_job", 1L, + 1L, "ap-southeast-1", "stream-1", UserIdentity.ADMIN); + + Map parentProgress = new HashMap<>(); + parentProgress.put("shard-parent", "100"); + Deencapsulation.setField(routineLoadJob, "progress", new KinesisProgress(parentProgress)); + Deencapsulation.setField(routineLoadJob, "openKinesisShards", Lists.newArrayList("shard-parent")); + Deencapsulation.setField(routineLoadJob, "closedKinesisShards", Lists.newArrayList()); + Deencapsulation.setField(routineLoadJob, "newCurrentKinesisShards", + Lists.newArrayList("shard-child-0", "shard-child-1")); + + Assert.assertTrue((Boolean) Deencapsulation.invoke(routineLoadJob, "isKinesisShardsChanged")); + Deencapsulation.invoke(routineLoadJob, "updateNewShardProgress"); + + Map childProgress = new HashMap<>(); + childProgress.put("shard-child-0", "200"); + childProgress.put("shard-child-1", "300"); + Map oldLag = new HashMap<>(); + oldLag.put("shard-parent", 60_000L); + Deencapsulation.setField(routineLoadJob, "cachedShardWithMillsBehindLatest", oldLag); + Map childLag = new HashMap<>(); + childLag.put("shard-child-0", 0L); + childLag.put("shard-child-1", 100L); + RLTaskTxnCommitAttachment attachment = + createCommitAttachment(createProgress(childProgress, childLag, "shard-parent")); + Deencapsulation.invoke(routineLoadJob, "updateProgressAndOffsetsCache", attachment); + + KinesisProgress progress = Deencapsulation.getField(routineLoadJob, "progress"); + Assert.assertFalse(progress.containsShard("shard-parent")); + Assert.assertTrue(progress.containsShard("shard-child-0")); + Assert.assertTrue(progress.containsShard("shard-child-1")); + List openKinesisShards = Deencapsulation.getField(routineLoadJob, "openKinesisShards"); + Assert.assertEquals(new HashSet<>(Lists.newArrayList("shard-child-0", "shard-child-1")), + new HashSet<>(openKinesisShards)); + Assert.assertTrue(((List) Deencapsulation.getField(routineLoadJob, "closedKinesisShards")).isEmpty()); + Map cachedLag = Deencapsulation.getField(routineLoadJob, "cachedShardWithMillsBehindLatest"); + Assert.assertFalse(cachedLag.containsKey("shard-parent")); + Assert.assertEquals(0L, cachedLag.get("shard-child-0").longValue()); + Assert.assertEquals(100L, cachedLag.get("shard-child-1").longValue()); + + Deencapsulation.setField(routineLoadJob, "newCurrentKinesisShards", + Lists.newArrayList("shard-child-0", "shard-child-1")); + Assert.assertFalse((Boolean) Deencapsulation.invoke(routineLoadJob, "isKinesisShardsChanged")); + } + + private Set collectAssignedShards(KinesisRoutineLoadJob routineLoadJob) { + List routineLoadTaskInfoList = + Deencapsulation.getField(routineLoadJob, "routineLoadTaskInfoList"); + Set assignedShards = new HashSet<>(); + for (RoutineLoadTaskInfo taskInfo : routineLoadTaskInfoList) { + assignedShards.addAll(((KinesisTaskInfo) taskInfo).getShards()); + } + return assignedShards; + } + + private RLTaskTxnCommitAttachment createCommitAttachment(KinesisProgress progress) { + RLTaskTxnCommitAttachment attachment = new RLTaskTxnCommitAttachment(); + Deencapsulation.setField(attachment, "progress", progress); + return attachment; + } + + private KinesisProgress createProgress(Map shardToSeqNum, Map lagMap, + String... closedShards) { + KinesisProgress progress = new KinesisProgress(shardToSeqNum); + Map shardIdToMillsBehindLatest = Maps.newConcurrentMap(); + shardIdToMillsBehindLatest.putAll(lagMap); + Deencapsulation.setField(progress, "shardIdToMillsBehindLatest", shardIdToMillsBehindLatest); + Deencapsulation.setField(progress, "closedShardIds", new HashSet<>(Lists.newArrayList(closedShards))); + return progress; + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java index d22464064282c4..fee5b84539f564 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java @@ -28,6 +28,9 @@ import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.datasource.kafka.KafkaUtil; +import org.apache.doris.load.routineload.kafka.KafkaProgress; +import org.apache.doris.load.routineload.kafka.KafkaRoutineLoadJob; +import org.apache.doris.load.routineload.kafka.KafkaTaskInfo; import org.apache.doris.nereids.trees.plans.commands.info.CreateRoutineLoadInfo; import org.apache.doris.thrift.TKafkaRLTaskProgress; import org.apache.doris.thrift.TUniqueKeyUpdateMode; diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java index e9a55b1e4566d2..3aeac182b208ab 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadManagerTest.java @@ -34,6 +34,7 @@ import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.load.loadv2.LoadTask; import org.apache.doris.load.routineload.kafka.KafkaConfiguration; +import org.apache.doris.load.routineload.kafka.KafkaRoutineLoadJob; import org.apache.doris.mysql.privilege.AccessControllerManager; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.trees.plans.commands.info.CreateRoutineLoadInfo; diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java index 93587883456850..153d9f008c4a64 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java @@ -28,6 +28,8 @@ import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.load.RoutineLoadDesc; +import org.apache.doris.load.routineload.kafka.KafkaRoutineLoadJob; +import org.apache.doris.load.routineload.kafka.KafkaTaskInfo; import org.apache.doris.qe.ConnectContext; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TResourceInfo; diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java index d45e56eea570d9..4caa85dc7a1e9c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/RoutineLoadTaskSchedulerTest.java @@ -25,6 +25,9 @@ import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.load.RoutineLoadDesc; +import org.apache.doris.load.routineload.kafka.KafkaProgress; +import org.apache.doris.load.routineload.kafka.KafkaRoutineLoadJob; +import org.apache.doris.load.routineload.kafka.KafkaTaskInfo; import org.apache.doris.task.AgentTaskExecutor; import org.apache.doris.thrift.BackendService; import org.apache.doris.transaction.BeginTransactionException; diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/routineload/kinesis/KinesisDataSourcePropertiesTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/kinesis/KinesisDataSourcePropertiesTest.java new file mode 100644 index 00000000000000..6d81c5f987d7bb --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/load/routineload/kinesis/KinesisDataSourcePropertiesTest.java @@ -0,0 +1,83 @@ +// 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.load.routineload.kinesis; + +import org.apache.doris.common.AnalysisException; + +import com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +public class KinesisDataSourcePropertiesTest { + + @Test + public void testConvertAndCheckDataSourcePropertiesWithAwsEndpoint() throws Exception { + Map dataSourceProperties = Maps.newHashMap(); + dataSourceProperties.put(KinesisConfiguration.KINESIS_REGION.getName(), "us-east-1"); + dataSourceProperties.put(KinesisConfiguration.KINESIS_STREAM.getName(), "test_stream"); + dataSourceProperties.put(KinesisConfiguration.KINESIS_ENDPOINT.getName(), "http://localhost:4566"); + + KinesisDataSourceProperties properties = new KinesisDataSourceProperties(dataSourceProperties); + properties.convertAndCheckDataSourceProperties(); + + Assert.assertEquals("http://localhost:4566", properties.getEndpoint()); + } + + @Test + public void testConvertAndCheckDataSourcePropertiesWithLegacyEndpoint() throws Exception { + Map dataSourceProperties = Maps.newHashMap(); + dataSourceProperties.put(KinesisConfiguration.KINESIS_REGION.getName(), "us-east-1"); + dataSourceProperties.put(KinesisConfiguration.KINESIS_STREAM.getName(), "test_stream"); + dataSourceProperties.put("kinesis_endpoint", "http://localhost:4566"); + + KinesisDataSourceProperties properties = new KinesisDataSourceProperties(dataSourceProperties); + properties.convertAndCheckDataSourceProperties(); + + Assert.assertEquals("http://localhost:4566", properties.getEndpoint()); + } + + @Test + public void testPositionsShouldRejectDatetimeString() { + Map dataSourceProperties = Maps.newHashMap(); + dataSourceProperties.put(KinesisConfiguration.KINESIS_REGION.getName(), "us-east-1"); + dataSourceProperties.put(KinesisConfiguration.KINESIS_STREAM.getName(), "test_stream"); + dataSourceProperties.put(KinesisConfiguration.KINESIS_SHARDS.getName(), "shard-000"); + dataSourceProperties.put(KinesisConfiguration.KINESIS_POSITIONS.getName(), "2026-04-08 00:00:00"); + + KinesisDataSourceProperties properties = new KinesisDataSourceProperties(dataSourceProperties); + AnalysisException e = Assert.assertThrows(AnalysisException.class, + properties::convertAndCheckDataSourceProperties); + Assert.assertTrue(e.getMessage().contains("must be TRIM_HORIZON, LATEST, or a valid sequence number")); + } + + @Test + public void testDefaultPositionShouldRejectDatetimeString() { + Map dataSourceProperties = Maps.newHashMap(); + dataSourceProperties.put(KinesisConfiguration.KINESIS_REGION.getName(), "us-east-1"); + dataSourceProperties.put(KinesisConfiguration.KINESIS_STREAM.getName(), "test_stream"); + dataSourceProperties.put(KinesisConfiguration.KINESIS_DEFAULT_POSITION.getName(), + "2026-04-08 00:00:00"); + + KinesisDataSourceProperties properties = new KinesisDataSourceProperties(dataSourceProperties); + AnalysisException e = Assert.assertThrows(AnalysisException.class, + properties::convertAndCheckDataSourceProperties); + Assert.assertTrue(e.getMessage().contains("TRIM_HORIZON, LATEST, or a valid sequence number")); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/resource/ComputeGroupTest.java b/fe/fe-core/src/test/java/org/apache/doris/resource/ComputeGroupTest.java index 7a7e03e3783d96..d40e167614337e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/resource/ComputeGroupTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/resource/ComputeGroupTest.java @@ -28,9 +28,9 @@ import org.apache.doris.common.UserException; import org.apache.doris.datasource.FederationBackendPolicy; import org.apache.doris.load.loadv2.BrokerLoadJob; -import org.apache.doris.load.routineload.KafkaRoutineLoadJob; import org.apache.doris.load.routineload.RoutineLoadJob; import org.apache.doris.load.routineload.RoutineLoadManager; +import org.apache.doris.load.routineload.kafka.KafkaRoutineLoadJob; import org.apache.doris.mysql.privilege.AccessControllerManager; import org.apache.doris.mysql.privilege.Auth; import org.apache.doris.mysql.privilege.PrivPredicate; diff --git a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java index 10e446348aff00..daa4033a88677b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/transaction/GlobalTransactionMgrTest.java @@ -36,14 +36,14 @@ import org.apache.doris.common.QuotaExceedException; import org.apache.doris.common.UserException; import org.apache.doris.common.jmockit.Deencapsulation; -import org.apache.doris.load.routineload.KafkaProgress; -import org.apache.doris.load.routineload.KafkaRoutineLoadJob; -import org.apache.doris.load.routineload.KafkaTaskInfo; import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; import org.apache.doris.load.routineload.RoutineLoadJob; import org.apache.doris.load.routineload.RoutineLoadManager; import org.apache.doris.load.routineload.RoutineLoadStatistic; import org.apache.doris.load.routineload.RoutineLoadTaskInfo; +import org.apache.doris.load.routineload.kafka.KafkaProgress; +import org.apache.doris.load.routineload.kafka.KafkaRoutineLoadJob; +import org.apache.doris.load.routineload.kafka.KafkaTaskInfo; import org.apache.doris.meta.MetaContext; import org.apache.doris.persist.EditLog; import org.apache.doris.rpc.RpcException; @@ -1231,4 +1231,3 @@ private void checkReplicaVersion(Replica replica, long version, long lastSuccess Assert.assertEquals(lastFailedVersion, replica.getLastFailedVersion()); } } - diff --git a/fe/fe-core/src/test/java/org/apache/doris/transaction/TransactionStateTest.java b/fe/fe-core/src/test/java/org/apache/doris/transaction/TransactionStateTest.java index 55faca2a789fa2..f6b72d84af8ac5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/transaction/TransactionStateTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/transaction/TransactionStateTest.java @@ -24,8 +24,8 @@ import org.apache.doris.load.FailMsg.CancelType; import org.apache.doris.load.loadv2.JobState; import org.apache.doris.load.loadv2.LoadJobFinalOperation; -import org.apache.doris.load.routineload.KafkaProgress; import org.apache.doris.load.routineload.RLTaskTxnCommitAttachment; +import org.apache.doris.load.routineload.kafka.KafkaProgress; import org.apache.doris.meta.MetaContext; import org.apache.doris.thrift.TEtlState; import org.apache.doris.thrift.TKafkaRLTaskProgress; diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index 50274c176274fa..0c38238f642cda 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -489,9 +489,32 @@ message PKafkaMetaProxyRequest { repeated PIntegerPair offset_flags = 5; }; +message PKinesisLoadInfo { + required string region = 1; + required string stream = 2; + optional string endpoint = 3; + repeated PStringPair properties = 4; +}; + +message PKinesisMetaProxyRequest { + optional PKinesisLoadInfo kinesis_info = 1; +}; + +message PShardInfo { + required string shard_id = 1; + optional string parent_shard_id = 2; + optional string adjacent_parent_shard_id = 3; +}; + +message PKinesisMetaProxyResult { + repeated string shard_ids = 1; // Deprecated, use shard_infos instead + repeated PShardInfo shard_infos = 2; +}; + message PProxyRequest { optional PKafkaMetaProxyRequest kafka_meta_request = 1; optional int64 timeout_secs = 2; + optional PKinesisMetaProxyRequest kinesis_meta_request = 3; }; message PKafkaMetaProxyResult { @@ -506,6 +529,7 @@ message PProxyResult { required PStatus status = 1; optional PKafkaMetaProxyResult kafka_meta_result = 2; optional PKafkaPartitionOffsets partition_offsets = 3; + optional PKinesisMetaProxyResult kinesis_meta_result = 4; }; message PDataColumn { diff --git a/gensrc/thrift/BackendService.thrift b/gensrc/thrift/BackendService.thrift index 44c1e7cd94c619..56890f7dda65eb 100644 --- a/gensrc/thrift/BackendService.thrift +++ b/gensrc/thrift/BackendService.thrift @@ -54,6 +54,17 @@ struct TKafkaLoadInfo { 4: optional map properties; } +// Kinesis load info for routine load from AWS Kinesis +struct TKinesisLoadInfo { + 1: required string region; + 2: required string stream; + 3: optional string endpoint; + // Map from shard ID to starting sequence number + 4: required map shard_begin_sequence_number; + // AWS credentials and other properties + 5: optional map properties; +} + struct TRoutineLoadTask { 1: required Types.TLoadSourceType type 2: required i64 job_id @@ -74,6 +85,7 @@ struct TRoutineLoadTask { 17: optional bool memtable_on_sink_node; 18: optional string qualified_user 19: optional string cloud_cluster + 20: optional TKinesisLoadInfo kinesis_load_info } struct TKafkaMetaProxyRequest { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 34f56fe2510878..8aa7206f6968f8 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -624,6 +624,19 @@ struct TKafkaRLTaskProgress { 1: required map partitionCmtOffset } +// Kinesis routine load task progress +// Maps shard ID to the last committed sequence number +struct TKinesisRLTaskProgress { + 1: required map shardCmtSeqNum + // MillisBehindLatest per shard, returned by GetRecords API. + // Indicates how far behind the consumer is from the tip of the stream. + // 0 means the consumer has caught up; absent means unknown. + 2: optional map shardMillsBehindLatest + // Set of shard IDs that have been closed (split/merge) during consumption. + // FE should remove these shards from tracking. + 3: optional set closedShardIds +} + struct TRLTaskTxnCommitAttachment { 1: required Types.TLoadSourceType loadSourceType 2: required Types.TUniqueId id @@ -636,6 +649,7 @@ struct TRLTaskTxnCommitAttachment { 9: optional i64 loadCostMs 10: optional TKafkaRLTaskProgress kafkaRLTaskProgress 11: optional string errorLogUrl + 12: optional TKinesisRLTaskProgress kinesisRLTaskProgress } struct TTxnCommitAttachment { diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index 66ad6fd3f549ad..fa560140170eba 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -727,6 +727,7 @@ enum TLoadSourceType { RAW = 0, KAFKA = 1, MULTI_TABLE = 2, + KINESIS = 3, } enum TMergeType { diff --git a/regression-test/framework/pom.xml b/regression-test/framework/pom.xml index d25a5d0d98eb0d..91d97ec1071987 100644 --- a/regression-test/framework/pom.xml +++ b/regression-test/framework/pom.xml @@ -77,6 +77,7 @@ under the License. 4.9.3 2.8.0 1.11.95 + 2.13.5 17.0.0 @@ -224,6 +225,27 @@ under the License. gson 2.10.1 + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + com.fasterxml.jackson.dataformat + jackson-dataformat-cbor + ${jackson.version} + org.apache.groovy groovy-all @@ -349,6 +371,11 @@ under the License. aws-java-sdk-s3 ${aws-java-sdk-s3.version} + + com.amazonaws + aws-java-sdk-kinesis + ${aws-java-sdk-s3.version} + com.aliyun diff --git a/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load.groovy b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load.groovy new file mode 100644 index 00000000000000..743d167f7a405e --- /dev/null +++ b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load.groovy @@ -0,0 +1,307 @@ +// 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 com.amazonaws.auth.AWSStaticCredentialsProvider +import com.amazonaws.auth.BasicAWSCredentials +import com.amazonaws.services.kinesis.AmazonKinesis +import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder +import com.amazonaws.services.kinesis.model.* +import java.math.BigInteger +import java.nio.ByteBuffer + +suite("test_kinesis_routine_load") { + def ak = context.config.awsAccessKey ?: context.config.otherConfigs.get("awsAccessKey") + def sk = context.config.awsSecretKey ?: context.config.otherConfigs.get("awsSecretKey") + def region = context.config.awsRegion ?: context.config.otherConfigs.get("awsRegion") + + if (!region || !ak || !sk) { + logger.info("Skip ${name} case, AWS config not provided") + return + } + + def suffix = UUID.randomUUID().toString().substring(0, 8) + def sharedStreamName = "doris-test-kinesis-shared-${suffix}" + def sharedTableName = "kinesis_test_table_shared" + def sharedJobName = "kinesis_routine_load_job_shared_${suffix}" + def multiStreamName = "doris-test-kinesis-multi-${suffix}" + def multiTableName = "kinesis_test_table_multi" + def multiJobName = "kinesis_routine_load_job_multi_${suffix}" + + def credentials = new BasicAWSCredentials(ak, sk) + def kinesisClient = AmazonKinesisClientBuilder.standard() + .withRegion(region) + .withCredentials(new AWSStaticCredentialsProvider(credentials)) + .build() + + def waitForStreamReady = { String streamName, int timeoutSec -> + def describeRequest = new DescribeStreamRequest().withStreamName(streamName) + for (int i = 0; i < timeoutSec; i++) { + try { + def result = kinesisClient.describeStream(describeRequest) + def streamDesc = result.getStreamDescription() + if (streamDesc.getStreamStatus() == "ACTIVE" && !streamDesc.getShards().isEmpty()) { + return streamDesc + } + } catch (ResourceNotFoundException e) { + // Stream metadata may not be visible immediately after create. + } + Thread.sleep(1000) + } + assertTrue(false, "Stream ${streamName} failed to become ready") + } + + def putRecordWithRetry = { String streamName, String partitionKey, String jsonData, String explicitHashKey -> + def request = new PutRecordRequest() + .withStreamName(streamName) + .withPartitionKey(partitionKey) + .withData(ByteBuffer.wrap(jsonData.getBytes("UTF-8"))) + if (explicitHashKey != null) { + request.withExplicitHashKey(explicitHashKey) + } + for (int retry = 0; retry < 20; retry++) { + try { + kinesisClient.putRecord(request) + return + } catch (ResourceNotFoundException e) { + if (retry == 19) { + throw e + } + Thread.sleep(500) + } + } + } + + def toLongValue = { Object value -> + if (value instanceof Number) { + return ((Number) value).longValue() + } + return Long.parseLong(value.toString().trim()) + } + + def toIntValue = { Object value -> + if (value instanceof Number) { + return ((Number) value).intValue() + } + return Integer.parseInt(value.toString().trim()) + } + + def waitForCountAtLeast = { String tableName, long expectedCount, int timeoutSec -> + long lastCount = -1 + for (int i = 0; i < timeoutSec; i++) { + def result = sql "SELECT COUNT(*) FROM ${tableName}" + lastCount = toLongValue(result[0][0]) + if (lastCount >= expectedCount) { + logger.info("Table ${tableName} row count reached ${lastCount} (expected >= ${expectedCount})") + return lastCount + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting table ${tableName} count >= ${expectedCount}, last count=${lastCount}") + } + + def sharedStreamCreated = false + def sharedJobCreated = false + def test1PreparedForReuse = false + def multiStreamCreated = false + def multiJobCreated = false + + try { + // test 导入stream中已有数据 + try { + logger.info("Test1 create stream: ${sharedStreamName}") + kinesisClient.createStream(new CreateStreamRequest().withStreamName(sharedStreamName).withShardCount(1)) + sharedStreamCreated = true + waitForStreamReady(sharedStreamName, 120) + + // stream 已有数据(先写后建 job) + def firstData = "{\"id\": 1, \"name\": \"user_1\", \"age\": 21}" + putRecordWithRetry(sharedStreamName, "shared_key_1", firstData, null) + + sql "DROP TABLE IF EXISTS ${sharedTableName}" + sql """ + CREATE TABLE ${sharedTableName} ( + id INT, + name VARCHAR(100), + age INT + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ("replication_num" = "1") + """ + + sql """ + CREATE ROUTINE LOAD ${sharedJobName} ON ${sharedTableName} + PROPERTIES ("format" = "json", "desired_concurrent_number" = "1") + FROM KINESIS ( + "aws.region" = "${region}", + "aws.access_key" = "${ak}", + "aws.secret_key" = "${sk}", + "kinesis_stream" = "${sharedStreamName}", + "property.kinesis_default_pos" = "TRIM_HORIZON" + ) + """ + sharedJobCreated = true + + waitForCountAtLeast(sharedTableName, 1, 180) + test1PreparedForReuse = true + } finally { + // test2 要复用 test1 的 stream/table/job,仅在 test1 失败时兜底清理 + if (!test1PreparedForReuse) { + if (sharedJobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${sharedJobName}" + } catch (Exception e) { + logger.warn("Failed to stop shared routine load ${sharedJobName}: ${e.message}") + } + sharedJobCreated = false + } + if (sharedStreamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(sharedStreamName)) + } catch (Exception e) { + logger.warn("Failed to delete shared stream ${sharedStreamName}: ${e.message}") + } + sharedStreamCreated = false + } + sql "DROP TABLE IF EXISTS ${sharedTableName}" + } + } + + // test 导入stream中新写入数据 + try { + assertTrue(test1PreparedForReuse, "Test2 requires test1 to prepare shared stream/table/job") + def secondData = "{\"id\": 2, \"name\": \"user_2\", \"age\": 22}" + putRecordWithRetry(sharedStreamName, "shared_key_2", secondData, null) + waitForCountAtLeast(sharedTableName, 2, 180) + + def dedupResult = sql "SELECT COUNT(*), COUNT(DISTINCT id), MIN(id), MAX(id) FROM ${sharedTableName}" + assertEquals(2L, toLongValue(dedupResult[0][0])) + assertEquals(2L, toLongValue(dedupResult[0][1])) + assertEquals(1, toIntValue(dedupResult[0][2])) + assertEquals(2, toIntValue(dedupResult[0][3])) + } finally { + if (sharedJobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${sharedJobName}" + } catch (Exception e) { + logger.warn("Failed to stop shared routine load ${sharedJobName}: ${e.message}") + } + sharedJobCreated = false + } + if (sharedStreamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(sharedStreamName)) + logger.info("Deleted shared stream: ${sharedStreamName}") + } catch (Exception e) { + logger.warn("Failed to delete shared stream ${sharedStreamName}: ${e.message}") + } + sharedStreamCreated = false + } + sql "DROP TABLE IF EXISTS ${sharedTableName}" + } + + // test 并行消费多分片的数据 + try { + logger.info("Test3 create multi-shard stream: ${multiStreamName}") + kinesisClient.createStream(new CreateStreamRequest().withStreamName(multiStreamName).withShardCount(2)) + multiStreamCreated = true + def streamDesc = waitForStreamReady(multiStreamName, 120) + def openShards = streamDesc.getShards().findAll { + it.getSequenceNumberRange().getEndingSequenceNumber() == null + } + assertTrue(openShards.size() >= 2, "Expected at least 2 open shards, actual=${openShards.size()}") + + int totalRows = 0 + for (int shardIdx = 0; shardIdx < openShards.size(); shardIdx++) { + def shard = openShards[shardIdx] + def start = new BigInteger(shard.getHashKeyRange().getStartingHashKey()) + def end = new BigInteger(shard.getHashKeyRange().getEndingHashKey()) + def mid = start.add(end).divide(BigInteger.valueOf(2)).toString() + for (int i = 1; i <= 25; i++) { + def id = shardIdx * 1000 + i + def data = "{\"id\": ${id}, \"name\": \"multi_${shardIdx}_${i}\", \"age\": ${30 + i % 10}}" + putRecordWithRetry(multiStreamName, "multi_key_${shardIdx}_${i}", data, mid) + totalRows++ + } + } + + sql "DROP TABLE IF EXISTS ${multiTableName}" + sql """ + CREATE TABLE ${multiTableName} ( + id INT, + name VARCHAR(100), + age INT + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ("replication_num" = "1") + """ + + sql """ + CREATE ROUTINE LOAD ${multiJobName} ON ${multiTableName} + PROPERTIES ("format" = "json", "desired_concurrent_number" = "2") + FROM KINESIS ( + "aws.region" = "${region}", + "aws.access_key" = "${ak}", + "aws.secret_key" = "${sk}", + "kinesis_stream" = "${multiStreamName}", + "property.kinesis_default_pos" = "TRIM_HORIZON" + ) + """ + multiJobCreated = true + + waitForCountAtLeast(multiTableName, totalRows, 240) + + boolean seenParallelTask = false + for (int i = 0; i < 120; i++) { + def jobInfo = sql "SHOW ROUTINE LOAD FOR ${multiJobName}" + assertTrue(jobInfo.size() > 0) + int currentTaskNum = toIntValue(jobInfo[0][10]) + if (currentTaskNum >= 2) { + seenParallelTask = true + break + } + def taskRows = sql "SHOW ROUTINE LOAD TASK WHERE JobName = \"${multiJobName}\"" + if (taskRows.size() >= 2) { + seenParallelTask = true + break + } + Thread.sleep(1000) + } + assertTrue(seenParallelTask, "Failed to observe parallel routine load task scheduling") + } finally { + if (multiJobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${multiJobName}" + } catch (Exception e) { + logger.warn("Failed to stop multi routine load ${multiJobName}: ${e.message}") + } + } + if (multiStreamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(multiStreamName)) + logger.info("Deleted multi stream: ${multiStreamName}") + } catch (Exception e) { + logger.warn("Failed to delete multi stream ${multiStreamName}: ${e.message}") + } + } + sql "DROP TABLE IF EXISTS ${multiTableName}" + } + } finally { + kinesisClient.shutdown() + } +} diff --git a/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_be_restart.groovy b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_be_restart.groovy new file mode 100644 index 00000000000000..1e1eb18c54cad2 --- /dev/null +++ b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_be_restart.groovy @@ -0,0 +1,255 @@ +// 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 com.amazonaws.auth.AWSStaticCredentialsProvider +import com.amazonaws.auth.BasicAWSCredentials +import com.amazonaws.services.kinesis.AmazonKinesis +import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder +import com.amazonaws.services.kinesis.model.* +import org.apache.doris.regression.suite.ClusterOptions +import java.nio.ByteBuffer + +suite("test_kinesis_routine_load_be_restart", "docker") { + def region = context.config.awsRegion ?: context.config.otherConfigs.get("awsRegion") + def ak = context.config.awsAccessKey ?: context.config.otherConfigs.get("awsAccessKey") + def sk = context.config.awsSecretKey ?: context.config.otherConfigs.get("awsSecretKey") + + if (!region || !ak || !sk) { + logger.info("Skip ${name} case, missing AWS config: region=${region}, ak=${ak != null}, sk=${sk != null}") + return + } + + def options = new ClusterOptions() + options.setFeNum(1) + options.setBeNum(1) + // Override image-baked JAVA_HOME that may point to a host-only path. + options.feConfigs.add("JAVA_HOME=") + docker(options) { + def suffix = UUID.randomUUID().toString().substring(0, 8) + def streamName = "doris-be-restart-${suffix}" + def tableName = "test_kinesis_be_restart" + def jobName = "test_kinesis_be_restart_${suffix}" + + def credentials = new BasicAWSCredentials(ak, sk) + def kinesisClient = AmazonKinesisClientBuilder.standard() + .withRegion(region) + .withCredentials(new AWSStaticCredentialsProvider(credentials)) + .build() + def streamCreated = false + def tableCreated = false + def jobCreated = false + def backendsStopped = false + + def waitForStreamReady = { int timeoutSec -> + logger.info("Waiting for stream ${streamName} to become active") + def describeRequest = new DescribeStreamRequest().withStreamName(streamName) + for (int i = 0; i < timeoutSec; i++) { + try { + def result = kinesisClient.describeStream(describeRequest) + def description = result.getStreamDescription() + if (description.getStreamStatus() == "ACTIVE" && !description.getShards().isEmpty()) { + return + } + } catch (ResourceNotFoundException e) { + // Metadata may not be visible immediately after create. + } + Thread.sleep(1000) + } + assertTrue(false, "Stream ${streamName} failed to become active") + } + + def writeRange = { int startId, int endId -> + logger.info("Writing records ${startId}-${endId} to stream ${streamName}") + for (int i = startId; i <= endId; i++) { + def data = "{\"id\": ${i}, \"name\": \"user_${i}\"}" + def putRequest = new PutRecordRequest() + .withStreamName(streamName) + .withPartitionKey("key_${i}") + .withData(ByteBuffer.wrap(data.getBytes("UTF-8"))) + for (int retry = 0; retry < 20; retry++) { + try { + kinesisClient.putRecord(putRequest) + break + } catch (ResourceNotFoundException e) { + if (retry == 19) { + throw e + } + Thread.sleep(500) + } + } + } + } + + def queryCount = { + def result = sql "SELECT COUNT(*) FROM ${tableName}" + return ((Number) result[0][0]).longValue() + } + + def waitForCountAtLeast = { long expectedCount, int timeoutSec -> + long lastCount = -1 + for (int i = 0; i < timeoutSec; i++) { + lastCount = queryCount() + if (lastCount >= expectedCount) { + logger.info("Table ${tableName} row count reached ${lastCount} (expected >= ${expectedCount})") + return lastCount + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting row count >= ${expectedCount}, last count=${lastCount}") + } + + def waitForBackendsState = { boolean alive, int timeoutSec -> + for (int i = 0; i < timeoutSec; i++) { + def backends = sql_return_maparray("SHOW BACKENDS") + boolean allMatched = true + for (def backend : backends) { + if (backend.Alive.toString().toBoolean() != alive) { + allMatched = false + break + } + } + if (allMatched) { + logger.info("All backends are ${alive ? "alive" : "down"}") + return + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting all backends to be ${alive ? "alive" : "down"}") + } + + def getJobState = { + def result = sql "SHOW ROUTINE LOAD FOR ${jobName}" + assertTrue(result.size() > 0, "SHOW ROUTINE LOAD returned empty result for ${jobName}") + return result[0][8].toString() + } + + def waitForJobStateIn = { Set expectedStates, int timeoutSec -> + def lastState = "UNKNOWN" + for (int i = 0; i < timeoutSec; i++) { + lastState = getJobState() + if (expectedStates.contains(lastState)) { + logger.info("Routine load job ${jobName} reached state ${lastState}") + return lastState + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting job ${jobName} to reach states ${expectedStates}, last state=${lastState}") + } + + try { + logger.info("Creating Kinesis stream: ${streamName}") + kinesisClient.createStream(new CreateStreamRequest() + .withStreamName(streamName) + .withShardCount(1)) + streamCreated = true + waitForStreamReady(120) + + sql "DROP TABLE IF EXISTS ${tableName}" + sql """ + CREATE TABLE ${tableName} ( + id INT, + name VARCHAR(100) + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ("replication_num" = "1") + """ + tableCreated = true + + sql """ + CREATE ROUTINE LOAD ${jobName} ON ${tableName} + PROPERTIES ( + "format" = "json", + "desired_concurrent_number" = "1" + ) + FROM KINESIS ( + "aws.region" = "${region}", + "aws.access_key" = "${ak}", + "aws.secret_key" = "${sk}", + "kinesis_stream" = "${streamName}", + "property.kinesis_default_pos" = "TRIM_HORIZON" + ) + """ + jobCreated = true + waitForJobStateIn(["RUNNING", "NEED_SCHEDULE"] as Set, 120) + + writeRange(1, 50) + long beforeRestartCount = waitForCountAtLeast(50, 120) + logger.info("Loaded rows before BE restart: ${beforeRestartCount}") + assertEquals(50L, beforeRestartCount) + + logger.info("Stopping all backends") + cluster.stopBackends() + backendsStopped = true + waitForBackendsState(false, 120) + + def stateAfterStop = waitForJobStateIn(["PAUSED", "RUNNING", "NEED_SCHEDULE"] as Set, 120) + logger.info("Routine load state after BE stop: ${stateAfterStop}") + assertNotEquals("CANCELLED", stateAfterStop) + + writeRange(51, 100) + + logger.info("Starting all backends") + cluster.startBackends() + backendsStopped = false + waitForBackendsState(true, 120) + + def stateAfterRestart = waitForJobStateIn(["RUNNING", "NEED_SCHEDULE", "PAUSED"] as Set, 120) + logger.info("Routine load state after BE restart: ${stateAfterRestart}") + assertNotEquals("CANCELLED", stateAfterRestart) + + long finalCount = waitForCountAtLeast(100, 180) + logger.info("Loaded rows after BE restart: ${finalCount}") + assertEquals(100L, finalCount) + def result = sql "SELECT COUNT(*), COUNT(DISTINCT id), MIN(id), MAX(id) FROM ${tableName}" + assertEquals(100L, ((Number) result[0][0]).longValue()) + assertEquals(100L, ((Number) result[0][1]).longValue()) + assertEquals(1, ((Number) result[0][2]).intValue()) + assertEquals(100, ((Number) result[0][3]).intValue()) + + } finally { + if (backendsStopped) { + try { + cluster.startBackends() + waitForBackendsState(true, 120) + backendsStopped = false + } catch (Exception e) { + logger.warn("Failed to restart backends in cleanup: ${e.message}") + } + } + if (jobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${jobName}" + } catch (Exception e) { + logger.warn("Failed to stop routine load ${jobName}: ${e.message}") + } + } + if (streamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(streamName)) + logger.info("Deleted stream: ${streamName}") + } catch (Exception e) { + logger.warn("Failed to delete stream ${streamName}: ${e.message}") + } + } + kinesisClient.shutdown() + if (tableCreated) { + sql "DROP TABLE IF EXISTS ${tableName}" + } + } + } +} diff --git a/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_data_quality_error.groovy b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_data_quality_error.groovy new file mode 100644 index 00000000000000..71a68d48fd5872 --- /dev/null +++ b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_data_quality_error.groovy @@ -0,0 +1,400 @@ +// 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 com.amazonaws.auth.AWSStaticCredentialsProvider +import com.amazonaws.auth.BasicAWSCredentials +import com.amazonaws.services.kinesis.AmazonKinesis +import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder +import com.amazonaws.services.kinesis.model.* +import java.nio.ByteBuffer + +suite("test_kinesis_routine_load_data_quality_error") { + def region = context.config.awsRegion ?: context.config.otherConfigs.get("awsRegion") + def ak = context.config.awsAccessKey ?: context.config.otherConfigs.get("awsAccessKey") + def sk = context.config.awsSecretKey ?: context.config.otherConfigs.get("awsSecretKey") + + if (!region || !ak || !sk) { + logger.info("Skip ${name} case, missing AWS config: region=${region}, ak=${ak != null}, sk=${sk != null}") + return + } + + def suffix = UUID.randomUUID().toString().substring(0, 8) + def streamName = "doris-quality-${suffix}" + def jobName = "test_kinesis_quality_${suffix}" + + def credentials = new BasicAWSCredentials(ak, sk) + def kinesisClient = AmazonKinesisClientBuilder.standard() + .withRegion(region) + .withCredentials(new AWSStaticCredentialsProvider(credentials)) + .build() + + def toLongValue = { Object value -> + if (value instanceof Number) { + return ((Number) value).longValue() + } + return Long.parseLong(value.toString().trim()) + } + + def toIntValue = { Object value -> + if (value instanceof Number) { + return ((Number) value).intValue() + } + return Integer.parseInt(value.toString().trim()) + } + + def waitForStreamReady = { int timeoutSec -> + def describeRequest = new DescribeStreamRequest().withStreamName(streamName) + for (int i = 0; i < timeoutSec; i++) { + try { + def result = kinesisClient.describeStream(describeRequest) + def description = result.getStreamDescription() + if (description.getStreamStatus() == "ACTIVE" && !description.getShards().isEmpty()) { + return + } + } catch (ResourceNotFoundException e) { + // Metadata may not be visible immediately after create. + } + Thread.sleep(1000) + } + assertTrue(false, "Stream ${streamName} failed to become active") + } + + def putRecordWithRetry = { String partitionKey, String data -> + def putRequest = new PutRecordRequest() + .withStreamName(streamName) + .withPartitionKey(partitionKey) + .withData(ByteBuffer.wrap(data.getBytes("UTF-8"))) + for (int retry = 0; retry < 20; retry++) { + try { + kinesisClient.putRecord(putRequest) + return + } catch (ResourceNotFoundException e) { + if (retry == 19) { + throw e + } + Thread.sleep(500) + } + } + } + + def writeMixedJsonRecords = { int startId, int endId, String partitionKeyPrefix -> + int written = 0 + int bad = 0 + logger.info("Writing mixed JSON records ${startId}-${endId} to stream ${streamName}") + for (int i = startId; i <= endId; i++) { + boolean isBadRecord = (i % 4 == 0) + def data + if (isBadRecord) { + if (i % 8 == 0) { + data = "{\"id\": \"bad_id_${i}\", \"age\": ${20 + i}}" + } else { + data = "{\"id\": ${i}, \"age\": \"bad_age_${i}\"}" + } + bad++ + } else { + data = "{\"id\": ${i}, \"age\": ${20 + i}}" + } + putRecordWithRetry("${partitionKeyPrefix}_${i}", data) + written++ + } + return [written: written, bad: bad, good: written - bad] + } + + def writeNotNullViolationRecords = { int startId, int endId, String partitionKeyPrefix, String targetStream -> + int written = 0 + int bad = 0 + logger.info("Writing NOT NULL violation records ${startId}-${endId} to stream ${targetStream}") + for (int i = startId; i <= endId; i++) { + boolean isBadRecord = (i % 4 == 0) + def data + if (isBadRecord) { + data = "{\"id\": null, \"age\": ${20 + i}}" + bad++ + } else { + data = "{\"id\": ${i}, \"age\": ${20 + i}}" + } + def putRequest = new PutRecordRequest() + .withStreamName(targetStream) + .withPartitionKey("${partitionKeyPrefix}_${i}") + .withData(ByteBuffer.wrap(data.getBytes("UTF-8"))) + for (int retry = 0; retry < 20; retry++) { + try { + kinesisClient.putRecord(putRequest) + break + } catch (ResourceNotFoundException e) { + if (retry == 19) { + throw e + } + Thread.sleep(500) + } + } + written++ + } + return [written: written, bad: bad, good: written - bad] + } + + def queryCount = { + def result = sql "SELECT COUNT(*) FROM test_kinesis_quality" + return toLongValue(result[0][0]) + } + + def waitForCountAtLeast = { long expectedCount, int timeoutSec -> + long lastCount = -1 + for (int i = 0; i < timeoutSec; i++) { + lastCount = queryCount() + if (lastCount >= expectedCount) { + logger.info("Table test_kinesis_quality row count reached ${lastCount} (expected >= ${expectedCount})") + return + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting row count >= ${expectedCount}, last count=${lastCount}") + } + + def getJobState = { + def result = sql "SHOW ROUTINE LOAD FOR ${jobName}" + assertTrue(result.size() > 0, "SHOW ROUTINE LOAD returned empty result for ${jobName}") + return result[0][8].toString() + } + + def streamCreated = false + def tableCreated = false + def jobCreated = false + def test1PreparedForReuse = false + + def stopRoutineLoadIfCreated = { + if (jobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${jobName}" + } catch (Exception e) { + logger.warn("Failed to stop routine load ${jobName}: ${e.message}") + } + jobCreated = false + } + } + + def deleteStreamIfCreated = { + if (streamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(streamName)) + logger.info("Deleted stream: ${streamName}") + } catch (Exception e) { + logger.warn("Failed to delete stream ${streamName}: ${e.message}") + } + streamCreated = false + } + } + + def dropTableIfCreated = { + if (tableCreated) { + sql "DROP TABLE IF EXISTS test_kinesis_quality" + tableCreated = false + } + } + + try { + // test1 : load mixed-quality records and verify bad records are tolerated under max_filter_ratio + try { + logger.info("Creating Kinesis stream: ${streamName}") + kinesisClient.createStream(new CreateStreamRequest() + .withStreamName(streamName) + .withShardCount(1)) + streamCreated = true + waitForStreamReady(120) + + def firstBatch = writeMixedJsonRecords(1, 40, "first") + assertEquals(40, firstBatch.written) + assertEquals(10, firstBatch.bad) + assertEquals(30, firstBatch.good) + + sql "DROP TABLE IF EXISTS test_kinesis_quality" + sql """ + CREATE TABLE test_kinesis_quality ( + id INT, + age INT + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ("replication_num" = "1") + """ + tableCreated = true + + sql """ + CREATE ROUTINE LOAD ${jobName} ON test_kinesis_quality + PROPERTIES ( + "format" = "json", + "max_filter_ratio" = "0.3" + ) + FROM KINESIS ( + "aws.region" = "${region}", + "aws.access_key" = "${ak}", + "aws.secret_key" = "${sk}", + "kinesis_stream" = "${streamName}", + "property.kinesis_default_pos" = "TRIM_HORIZON" + ) + """ + jobCreated = true + + waitForCountAtLeast(30L, 180) + + def stateAfterFirstBatch = getJobState() + assertNotEquals("CANCELLED", stateAfterFirstBatch) + test1PreparedForReuse = true + } finally { + if (!test1PreparedForReuse) { + stopRoutineLoadIfCreated() + deleteStreamIfCreated() + dropTableIfCreated() + } + } + + // test2 : continue loading mixed-quality records and verify job keeps ingesting valid records + try { + assertTrue(test1PreparedForReuse, "Test2 requires test1 to prepare stream/table/job") + + def secondBatch = writeMixedJsonRecords(41, 60, "second") + assertEquals(20, secondBatch.written) + assertEquals(5, secondBatch.bad) + assertEquals(15, secondBatch.good) + + waitForCountAtLeast(45L, 360) + + def finalResult = sql "SELECT COUNT(*), COUNT(DISTINCT id), MIN(id), MAX(id) FROM test_kinesis_quality" + assertTrue(toLongValue(finalResult[0][0]) >= 45L) + assertTrue(toLongValue(finalResult[0][1]) >= 45L) + assertEquals(1, toIntValue(finalResult[0][2])) + assertTrue(toIntValue(finalResult[0][3]) >= 59) + + def finalState = getJobState() + assertNotEquals("CANCELLED", finalState) + + // test2.1 : verify NOT NULL violation records are filtered under max_filter_ratio + def notNullStream = "doris-quality-not-null-${suffix}" + def notNullTable = "test_kinesis_quality_not_null" + def notNullJob = "${jobName}_not_null" + def notNullStreamCreated = false + def notNullTableCreated = false + def notNullJobCreated = false + try { + kinesisClient.createStream(new CreateStreamRequest() + .withStreamName(notNullStream) + .withShardCount(1)) + notNullStreamCreated = true + + def notNullStreamActive = false + for (int i = 0; i < 120; i++) { + try { + def describeResult = kinesisClient.describeStream( + new DescribeStreamRequest().withStreamName(notNullStream)) + if (describeResult.getStreamDescription().getStreamStatus() == "ACTIVE" && + !describeResult.getStreamDescription().getShards().isEmpty()) { + notNullStreamActive = true + break + } + } catch (ResourceNotFoundException e) { + // Metadata may not be visible immediately after create. + } + Thread.sleep(1000) + } + assertTrue(notNullStreamActive, "Stream ${notNullStream} failed to become active") + + def thirdBatch = writeNotNullViolationRecords(61, 80, "third_not_null", notNullStream) + assertEquals(20, thirdBatch.written) + assertEquals(5, thirdBatch.bad) + assertEquals(15, thirdBatch.good) + + sql "DROP TABLE IF EXISTS ${notNullTable}" + sql """ + CREATE TABLE ${notNullTable} ( + id INT NOT NULL, + age INT + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ("replication_num" = "1") + """ + notNullTableCreated = true + + sql """ + CREATE ROUTINE LOAD ${notNullJob} ON ${notNullTable} + PROPERTIES ( + "format" = "json", + "max_filter_ratio" = "0.5" + ) + FROM KINESIS ( + "aws.region" = "${region}", + "aws.access_key" = "${ak}", + "aws.secret_key" = "${sk}", + "kinesis_stream" = "${notNullStream}", + "property.kinesis_default_pos" = "TRIM_HORIZON" + ) + """ + notNullJobCreated = true + + def stateRetry = 0 + while (true) { + Thread.sleep(1000) + def stateResult = sql "SHOW ROUTINE LOAD FOR ${notNullJob}" + def state = stateResult[0][8].toString() + if (state == "RUNNING" || state == "NEED_SCHEDULE") { + break + } + stateRetry++ + if (stateRetry > 120) { + fail("NOT NULL job failed to start") + } + } + + long notNullCount = -1 + for (int i = 0; i < 360; i++) { + def notNullResult = sql "SELECT COUNT(*) FROM ${notNullTable}" + notNullCount = toLongValue(notNullResult[0][0]) + if (notNullCount >= 15L) { + logger.info("Table ${notNullTable} row count reached ${notNullCount} (expected >= 15)") + break + } + Thread.sleep(1000) + } + assertTrue(notNullCount >= 15L, "Timeout waiting row count >= 15, last count=${notNullCount}") + } finally { + if (notNullJobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${notNullJob}" + } catch (Exception e) { + logger.warn("Failed to stop routine load ${notNullJob}: ${e.message}") + } + } + if (notNullStreamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(notNullStream)) + } catch (Exception e) { + logger.warn("Failed to delete stream ${notNullStream}: ${e.message}") + } + } + if (notNullTableCreated) { + sql "DROP TABLE IF EXISTS ${notNullTable}" + } + } + } finally { + stopRoutineLoadIfCreated() + deleteStreamIfCreated() + dropTableIfCreated() + } + } finally { + kinesisClient.shutdown() + } +} diff --git a/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_fe_restart.groovy b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_fe_restart.groovy new file mode 100644 index 00000000000000..6d62c04b1dc87a --- /dev/null +++ b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_fe_restart.groovy @@ -0,0 +1,250 @@ +// 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 com.amazonaws.auth.AWSStaticCredentialsProvider +import com.amazonaws.auth.BasicAWSCredentials +import com.amazonaws.services.kinesis.AmazonKinesis +import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder +import com.amazonaws.services.kinesis.model.* +import org.apache.doris.regression.suite.ClusterOptions +import java.nio.ByteBuffer + +suite("test_kinesis_routine_load_fe_restart", "docker") { + def region = context.config.awsRegion ?: context.config.otherConfigs.get("awsRegion") + def ak = context.config.awsAccessKey ?: context.config.otherConfigs.get("awsAccessKey") + def sk = context.config.awsSecretKey ?: context.config.otherConfigs.get("awsSecretKey") + + if (!region || !ak || !sk) { + logger.info("Skip ${name} case, missing AWS config: region=${region}, ak=${ak != null}, sk=${sk != null}") + return + } + + def options = new ClusterOptions() + options.setFeNum(1) + options.setBeNum(1) + // Override image-baked JAVA_HOME that may point to a host-only path. + options.feConfigs.add("JAVA_HOME=") + docker(options) { + def suffix = UUID.randomUUID().toString().substring(0, 8) + def streamName = "doris-fe-restart-${suffix}" + def tableName = "test_kinesis_fe_restart" + def jobName = "test_kinesis_fe_restart_${suffix}" + + def credentials = new BasicAWSCredentials(ak, sk) + def kinesisClient = AmazonKinesisClientBuilder.standard() + .withRegion(region) + .withCredentials(new AWSStaticCredentialsProvider(credentials)) + .build() + def streamCreated = false + def tableCreated = false + def jobCreated = false + def frontendsStopped = false + def masterFeIndex = -1 + + def waitForStreamReady = { int timeoutSec -> + logger.info("Waiting for stream ${streamName} to become active") + def describeRequest = new DescribeStreamRequest().withStreamName(streamName) + for (int i = 0; i < timeoutSec; i++) { + try { + def result = kinesisClient.describeStream(describeRequest) + def description = result.getStreamDescription() + if (description.getStreamStatus() == "ACTIVE" && !description.getShards().isEmpty()) { + return + } + } catch (ResourceNotFoundException e) { + // Metadata may not be visible immediately after create. + } + Thread.sleep(1000) + } + assertTrue(false, "Stream ${streamName} failed to become active") + } + + def writeRange = { int startId, int endId -> + logger.info("Writing records ${startId}-${endId} to stream ${streamName}") + for (int i = startId; i <= endId; i++) { + def data = "{\"id\": ${i}, \"value\": ${i * 100}}" + def putRequest = new PutRecordRequest() + .withStreamName(streamName) + .withPartitionKey("key_${i}") + .withData(ByteBuffer.wrap(data.getBytes("UTF-8"))) + for (int retry = 0; retry < 20; retry++) { + try { + kinesisClient.putRecord(putRequest) + break + } catch (ResourceNotFoundException e) { + if (retry == 19) { + throw e + } + Thread.sleep(500) + } + } + } + } + + def queryCount = { + def result = sql "SELECT COUNT(*) FROM ${tableName}" + return ((Number) result[0][0]).longValue() + } + + def waitForCountAtLeast = { long expectedCount, int timeoutSec -> + long lastCount = -1 + for (int i = 0; i < timeoutSec; i++) { + lastCount = queryCount() + if (lastCount >= expectedCount) { + logger.info("Table ${tableName} row count reached ${lastCount} (expected >= ${expectedCount})") + return lastCount + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting row count >= ${expectedCount}, last count=${lastCount}") + } + + def getJobState = { + def result = sql "SHOW ROUTINE LOAD FOR ${jobName}" + assertTrue(result.size() > 0, "SHOW ROUTINE LOAD returned empty result for ${jobName}") + return result[0][8].toString() + } + + def waitForJobStateIn = { Set expectedStates, int timeoutSec -> + def lastState = "UNKNOWN" + for (int i = 0; i < timeoutSec; i++) { + lastState = getJobState() + if (expectedStates.contains(lastState)) { + logger.info("Routine load job ${jobName} reached state ${lastState}") + return lastState + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting job ${jobName} to reach states ${expectedStates}, last state=${lastState}") + } + + try { + logger.info("Creating Kinesis stream: ${streamName}") + kinesisClient.createStream(new CreateStreamRequest() + .withStreamName(streamName) + .withShardCount(1)) + streamCreated = true + waitForStreamReady(120) + + sql "DROP TABLE IF EXISTS ${tableName}" + sql """ + CREATE TABLE ${tableName} ( + id INT, + value INT + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ("replication_num" = "1") + """ + tableCreated = true + + sql """ + CREATE ROUTINE LOAD ${jobName} ON ${tableName} + PROPERTIES ( + "format" = "json", + "desired_concurrent_number" = "1" + ) + FROM KINESIS ( + "aws.region" = "${region}", + "aws.access_key" = "${ak}", + "aws.secret_key" = "${sk}", + "kinesis_stream" = "${streamName}", + "property.kinesis_default_pos" = "TRIM_HORIZON" + ) + """ + jobCreated = true + waitForJobStateIn(["RUNNING", "NEED_SCHEDULE"] as Set, 120) + + writeRange(1, 50) + long beforeRestartCount = waitForCountAtLeast(50, 120) + logger.info("Loaded rows before FE restart: ${beforeRestartCount}") + assertEquals(50L, beforeRestartCount) + + masterFeIndex = cluster.getMasterFe().index + logger.info("Stopping master FE index=${masterFeIndex}") + frontendsStopped = true + try { + cluster.stopFrontends(masterFeIndex) + } catch (Exception e) { + // In single-FE docker cluster, stop may timeout while checking FE liveness. + // The FE container can already be stopped at this point, so continue the restart flow. + logger.warn("Stop master FE index=${masterFeIndex} returned exception, continue restart flow: ${e.message}") + } + + writeRange(51, 100) + + logger.info("Starting master FE index=${masterFeIndex}") + cluster.startFrontends(masterFeIndex) + frontendsStopped = false + Thread.sleep(30000) + context.reconnectFe() + + def stateAfterRestart = waitForJobStateIn(["RUNNING", "NEED_SCHEDULE", "PAUSED"] as Set, 120) + logger.info("Routine load state after FE restart: ${stateAfterRestart}") + assertNotEquals("CANCELLED", stateAfterRestart) + + long finalCount = waitForCountAtLeast(100, 180) + logger.info("Loaded rows after FE restart: ${finalCount}") + assertEquals(100L, finalCount) + def result = sql "SELECT COUNT(*), COUNT(DISTINCT id), MIN(id), MAX(id), SUM(value) FROM ${tableName}" + assertEquals(100L, ((Number) result[0][0]).longValue()) + assertEquals(100L, ((Number) result[0][1]).longValue()) + assertEquals(1, ((Number) result[0][2]).intValue()) + assertEquals(100, ((Number) result[0][3]).intValue()) + assertEquals(505000L, ((Number) result[0][4]).longValue()) + + } finally { + if (frontendsStopped) { + try { + if (masterFeIndex > 0) { + cluster.startFrontends(masterFeIndex) + } else { + cluster.startFrontends() + } + Thread.sleep(30000) + context.reconnectFe() + frontendsStopped = false + } catch (Exception e) { + logger.warn("Failed to restart FE in cleanup: ${e.message}") + } + } + if (jobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${jobName}" + } catch (Exception e) { + logger.warn("Failed to stop routine load ${jobName}: ${e.message}") + } + } + if (streamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(streamName)) + logger.info("Deleted stream: ${streamName}") + } catch (Exception e) { + logger.warn("Failed to delete stream ${streamName}: ${e.message}") + } + } + kinesisClient.shutdown() + if (tableCreated) { + try { + sql "DROP TABLE IF EXISTS ${tableName}" + } catch (Exception e) { + logger.warn("Failed to drop table ${tableName}: ${e.message}") + } + } + } + } +} diff --git a/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_pause_resume.groovy b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_pause_resume.groovy new file mode 100644 index 00000000000000..26ac277511f7d3 --- /dev/null +++ b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_pause_resume.groovy @@ -0,0 +1,265 @@ +// 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 com.amazonaws.auth.AWSStaticCredentialsProvider +import com.amazonaws.auth.BasicAWSCredentials +import com.amazonaws.services.kinesis.AmazonKinesis +import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder +import com.amazonaws.services.kinesis.model.* +import java.nio.ByteBuffer + +suite("test_kinesis_routine_load_pause_resume") { + def region = context.config.awsRegion ?: context.config.otherConfigs.get("awsRegion") + def ak = context.config.awsAccessKey ?: context.config.otherConfigs.get("awsAccessKey") + def sk = context.config.awsSecretKey ?: context.config.otherConfigs.get("awsSecretKey") + + if (!region || !ak || !sk) { + logger.info("Skip ${name} case, missing AWS config: region=${region}, ak=${ak != null}, sk=${sk != null}") + return + } + + def suffix = UUID.randomUUID().toString().substring(0, 8) + def streamName = "doris-test-pause-${suffix}" + def jobName = "test_kinesis_pause_resume_${suffix}" + + def credentials = new BasicAWSCredentials(ak, sk) + def kinesisClient = AmazonKinesisClientBuilder.standard() + .withRegion(region) + .withCredentials(new AWSStaticCredentialsProvider(credentials)) + .build() + + def waitForStreamReady = { int timeoutSec -> + logger.info("Waiting for stream ${streamName} to become active") + def describeRequest = new DescribeStreamRequest().withStreamName(streamName) + for (int i = 0; i < timeoutSec; i++) { + try { + def result = kinesisClient.describeStream(describeRequest) + def description = result.getStreamDescription() + if (description.getStreamStatus() == "ACTIVE" && !description.getShards().isEmpty()) { + return + } + } catch (ResourceNotFoundException e) { + // Metadata may not be visible immediately after create. + } + Thread.sleep(1000) + } + assertTrue(false, "Stream ${streamName} failed to become active") + } + + def getJobState = { boolean includeHistory = false -> + def showSql = includeHistory + ? "SHOW ALL ROUTINE LOAD FOR ${jobName}" + : "SHOW ROUTINE LOAD FOR ${jobName}" + def result = sql showSql + assertTrue(result.size() > 0, "${showSql} returned empty result for job ${jobName}") + return result[0][8].toString() + } + + def waitForJobStateIn = { Set expectedStates, int timeoutSec, boolean includeHistory = false -> + def lastState = "UNKNOWN" + for (int i = 0; i < timeoutSec; i++) { + lastState = getJobState(includeHistory) + if (expectedStates.contains(lastState)) { + logger.info("Routine load job ${jobName} reached state ${lastState}") + return lastState + } + Thread.sleep(1000) + } + assertTrue(false, + "Timeout waiting job ${jobName} to reach states ${expectedStates}, " + + "includeHistory=${includeHistory}, last state=${lastState}") + } + + def putRecordWithRetry = { String partitionKey, String data -> + def putRequest = new PutRecordRequest() + .withStreamName(streamName) + .withPartitionKey(partitionKey) + .withData(ByteBuffer.wrap(data.getBytes("UTF-8"))) + for (int retry = 0; retry < 20; retry++) { + try { + kinesisClient.putRecord(putRequest) + return + } catch (ResourceNotFoundException e) { + if (retry == 19) { + throw e + } + Thread.sleep(500) + } + } + } + + def writeRange = { int startId, int endId -> + logger.info("Writing records ${startId}-${endId} to stream ${streamName}") + for (int i = startId; i <= endId; i++) { + def data = "{\"id\": ${i}, \"name\": \"user_${i}\", \"age\": ${20 + i}}" + putRecordWithRetry("key_${i}", data) + } + } + + def queryCount = { + def result = sql "SELECT COUNT(*) FROM test_kinesis_pause_resume" + return ((Number) result[0][0]).longValue() + } + + def waitForCountAtLeast = { long expectedCount, int timeoutSec -> + long lastCount = -1 + for (int i = 0; i < timeoutSec; i++) { + lastCount = queryCount() + if (lastCount >= expectedCount) { + logger.info("Table test_kinesis_pause_resume row count reached ${lastCount} (expected >= ${expectedCount})") + return lastCount + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting row count >= ${expectedCount}, last count=${lastCount}") + } + + def streamCreated = false + def tableCreated = false + def jobCreated = false + def test1PreparedForReuse = false + + try { + // test1 : create stream/table/job, load first batch, then PAUSE and verify newly written data is not consumed + try { + logger.info("Creating Kinesis stream: ${streamName}") + kinesisClient.createStream(new CreateStreamRequest() + .withStreamName(streamName) + .withShardCount(1)) + streamCreated = true + waitForStreamReady(120) + + sql "DROP TABLE IF EXISTS test_kinesis_pause_resume" + sql """ + CREATE TABLE test_kinesis_pause_resume ( + id INT, + name VARCHAR(100), + age INT + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ("replication_num" = "1") + """ + tableCreated = true + + sql """ + CREATE ROUTINE LOAD ${jobName} ON test_kinesis_pause_resume + PROPERTIES ( + "format" = "json", + "desired_concurrent_number" = "1" + ) + FROM KINESIS ( + "aws.region" = "${region}", + "aws.access_key" = "${ak}", + "aws.secret_key" = "${sk}", + "kinesis_stream" = "${streamName}", + "property.kinesis_default_pos" = "TRIM_HORIZON" + ) + """ + jobCreated = true + + waitForJobStateIn(["RUNNING", "NEED_SCHEDULE"] as Set, 120) + + writeRange(1, 30) + long loadedBeforePause = waitForCountAtLeast(30, 180) + assertEquals(30L, loadedBeforePause) + + sql "PAUSE ROUTINE LOAD FOR ${jobName}" + waitForJobStateIn(["PAUSED"] as Set, 60) + + // Continue writing new records to the stream while the job is PAUSED + writeRange(31, 60) + + long pausedCountBeforeWait = queryCount() + Thread.sleep(10000) + long pausedCountAfterWait = queryCount() + logger.info("Row count while paused: before=${pausedCountBeforeWait}, after=${pausedCountAfterWait}") + assertEquals(pausedCountBeforeWait, pausedCountAfterWait) + + test1PreparedForReuse = true + } finally { + if (!test1PreparedForReuse) { + if (jobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${jobName}" + } catch (Exception e) { + logger.warn("Failed to stop routine load ${jobName}: ${e.message}") + } + jobCreated = false + } + if (streamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(streamName)) + logger.info("Deleted stream: ${streamName}") + } catch (Exception e) { + logger.warn("Failed to delete stream ${streamName}: ${e.message}") + } + streamCreated = false + } + if (tableCreated) { + sql "DROP TABLE IF EXISTS test_kinesis_pause_resume" + tableCreated = false + } + } + } + + // test2 : RESUME consumption, verify final integrity, then STOP the job + try { + assertTrue(test1PreparedForReuse, "Test2 requires test1 to prepare paused stream/job") + + sql "RESUME ROUTINE LOAD FOR ${jobName}" + waitForJobStateIn(["RUNNING", "NEED_SCHEDULE"] as Set, 120) + + long finalCount = waitForCountAtLeast(60, 180) + assertEquals(60L, finalCount) + + def result = sql "SELECT COUNT(*), COUNT(DISTINCT id), MIN(id), MAX(id) FROM test_kinesis_pause_resume" + assertEquals(60L, ((Number) result[0][0]).longValue()) + assertEquals(60L, ((Number) result[0][1]).longValue()) + assertEquals(1, ((Number) result[0][2]).intValue()) + assertEquals(60, ((Number) result[0][3]).intValue()) + + sql "STOP ROUTINE LOAD FOR ${jobName}" + jobCreated = false + waitForJobStateIn(["STOPPED", "CANCELLED"] as Set, 60, true) + } finally { + if (jobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${jobName}" + } catch (Exception e) { + logger.warn("Failed to stop routine load ${jobName}: ${e.message}") + } + jobCreated = false + } + if (streamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(streamName)) + logger.info("Deleted stream: ${streamName}") + } catch (Exception e) { + logger.warn("Failed to delete stream ${streamName}: ${e.message}") + } + streamCreated = false + } + if (tableCreated) { + sql "DROP TABLE IF EXISTS test_kinesis_pause_resume" + tableCreated = false + } + } + } finally { + kinesisClient.shutdown() + } +} diff --git a/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_property.groovy b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_property.groovy new file mode 100644 index 00000000000000..2dd6f9e559b777 --- /dev/null +++ b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_property.groovy @@ -0,0 +1,255 @@ +// 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 com.amazonaws.auth.AWSStaticCredentialsProvider +import com.amazonaws.auth.BasicAWSCredentials +import com.amazonaws.services.kinesis.AmazonKinesis +import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder +import com.amazonaws.services.kinesis.model.* +import java.nio.ByteBuffer + +suite("test_kinesis_routine_load_property") { + def region = context.config.awsRegion ?: context.config.otherConfigs.get("awsRegion") + def ak = context.config.awsAccessKey ?: context.config.otherConfigs.get("awsAccessKey") + def sk = context.config.awsSecretKey ?: context.config.otherConfigs.get("awsSecretKey") + + if (!region || !ak || !sk) { + logger.info("Skip ${name} case, missing AWS config: region=${region}, ak=${ak != null}, sk=${sk != null}") + return + } + + def suffix = UUID.randomUUID().toString().substring(0, 8) + def streamName = "doris-test-prop-${suffix}" + def jobName = "test_kinesis_property_${suffix}" + + def credentials = new BasicAWSCredentials(ak, sk) + def kinesisClient = AmazonKinesisClientBuilder.standard() + .withRegion(region) + .withCredentials(new AWSStaticCredentialsProvider(credentials)) + .build() + + def toLongValue = { Object value -> + if (value instanceof Number) { + return ((Number) value).longValue() + } + return Long.parseLong(value.toString().trim()) + } + + def toIntValue = { Object value -> + if (value instanceof Number) { + return ((Number) value).intValue() + } + return Integer.parseInt(value.toString().trim()) + } + + def waitForStreamReady = { int timeoutSec -> + logger.info("Waiting for stream ${streamName} to become active") + def describeRequest = new DescribeStreamRequest().withStreamName(streamName) + for (int i = 0; i < timeoutSec; i++) { + try { + def result = kinesisClient.describeStream(describeRequest) + def description = result.getStreamDescription() + if (description.getStreamStatus() == "ACTIVE" && !description.getShards().isEmpty()) { + return + } + } catch (ResourceNotFoundException e) { + // Metadata may not be visible immediately after create. + } + Thread.sleep(1000) + } + assertTrue(false, "Stream ${streamName} failed to become active") + } + + def putRecordWithRetry = { String partitionKey, String data -> + def putRequest = new PutRecordRequest() + .withStreamName(streamName) + .withPartitionKey(partitionKey) + .withData(ByteBuffer.wrap(data.getBytes("UTF-8"))) + for (int retry = 0; retry < 20; retry++) { + try { + kinesisClient.putRecord(putRequest) + return + } catch (ResourceNotFoundException e) { + if (retry == 19) { + throw e + } + Thread.sleep(500) + } + } + } + + def writeCsvRecords = { int startId, int endId, String partitionKeyPrefix -> + logger.info("Writing CSV records ${startId}-${endId} to stream ${streamName}") + for (int i = startId; i <= endId; i++) { + def data = "\"${i}\",\"name\\\"${i}\",\"2023-08-01\",\"value,${i}\",\"2023-08-01 12:00:00\",\"extra${i}\"" + putRecordWithRetry("${partitionKeyPrefix}_${i}", data) + } + } + + def queryCount = { + def result = sql "SELECT COUNT(*) FROM test_kinesis_routine_load_property" + return toLongValue(result[0][0]) + } + + def waitForCountAtLeast = { long expectedCount, int timeoutSec -> + long lastCount = -1 + for (int i = 0; i < timeoutSec; i++) { + lastCount = queryCount() + if (lastCount >= expectedCount) { + logger.info("Table test_kinesis_routine_load_property row count reached ${lastCount} " + + "(expected >= ${expectedCount})") + return + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting row count >= ${expectedCount}, last count=${lastCount}") + } + + def checkParsedRow = { int id -> + def expectedName = "name\"" + id + def expectedValue = "value," + id + def parsedRow = sql """ + SELECT k1, k2, v2 + FROM test_kinesis_routine_load_property + WHERE k1 = ${id} + ORDER BY v3 + LIMIT 1 + """ + assertTrue(parsedRow.size() > 0, "Expected at least one row for k1=${id}") + assertEquals(id, toIntValue(parsedRow[0][0])) + assertEquals(expectedName, parsedRow[0][1].toString()) + assertEquals(expectedValue, parsedRow[0][2].toString()) + } + + def streamCreated = false + def tableCreated = false + def jobCreated = false + def test1PreparedForReuse = false + + def stopRoutineLoadIfCreated = { + if (jobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${jobName}" + } catch (Exception e) { + logger.warn("Failed to stop routine load ${jobName}: ${e.message}") + } + jobCreated = false + } + } + + def deleteStreamIfCreated = { + if (streamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(streamName)) + logger.info("Deleted stream: ${streamName}") + } catch (Exception e) { + logger.warn("Failed to delete stream ${streamName}: ${e.message}") + } + streamCreated = false + } + } + + def dropTableIfCreated = { + if (tableCreated) { + sql "DROP TABLE IF EXISTS test_kinesis_routine_load_property" + tableCreated = false + } + } + + try { + // test1 : create routine load with CSV properties and verify TRIM_HORIZON consumes existing records + try { + logger.info("Creating Kinesis stream: ${streamName}") + kinesisClient.createStream(new CreateStreamRequest() + .withStreamName(streamName) + .withShardCount(1)) + streamCreated = true + waitForStreamReady(120) + + writeCsvRecords(1, 10, "before_create") + + sql "DROP TABLE IF EXISTS test_kinesis_routine_load_property" + sql """ + CREATE TABLE IF NOT EXISTS test_kinesis_routine_load_property ( + k1 INT NULL, + k2 STRING NULL, + v1 DATE NULL, + v2 STRING NULL, + v3 DATETIME NULL, + v4 STRING NULL + ) + ENGINE=OLAP + DUPLICATE KEY(k1) + DISTRIBUTED BY HASH(k1) BUCKETS 3 + PROPERTIES ("replication_num" = "1") + """ + tableCreated = true + + sql """ + CREATE ROUTINE LOAD ${jobName} ON test_kinesis_routine_load_property + COLUMNS TERMINATED BY "," + PROPERTIES ( + "enclose" = "\\"", + "escape" = "\\\\", + "max_batch_interval" = "5", + "max_batch_rows" = "200000", + "max_batch_size" = "209715200" + ) + FROM KINESIS ( + "aws.region" = "${region}", + "aws.access_key" = "${ak}", + "aws.secret_key" = "${sk}", + "kinesis_stream" = "${streamName}", + "property.kinesis_default_pos" = "TRIM_HORIZON" + ) + """ + jobCreated = true + + waitForCountAtLeast(10, 120) + checkParsedRow(1) + test1PreparedForReuse = true + } finally { + if (!test1PreparedForReuse) { + stopRoutineLoadIfCreated() + deleteStreamIfCreated() + dropTableIfCreated() + } + } + + // test2 : verify running job can continue importing newly written records with the same CSV properties + try { + assertTrue(test1PreparedForReuse, "Test2 requires test1 to prepare stream/table/job") + + writeCsvRecords(11, 20, "after_create") + waitForCountAtLeast(20, 120) + checkParsedRow(11) + + def finalResult = sql "SELECT COUNT(*), COUNT(DISTINCT k1), MIN(k1), MAX(k1) " + + "FROM test_kinesis_routine_load_property" + assertEquals(20L, toLongValue(finalResult[0][0])) + assertEquals(20L, toLongValue(finalResult[0][1])) + assertEquals(1, toIntValue(finalResult[0][2])) + assertEquals(20, toIntValue(finalResult[0][3])) + } finally { + stopRoutineLoadIfCreated() + deleteStreamIfCreated() + dropTableIfCreated() + } + } finally { + kinesisClient.shutdown() + } +} diff --git a/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_shard_change.groovy b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_shard_change.groovy new file mode 100644 index 00000000000000..a0d0560be0e62d --- /dev/null +++ b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_routine_load_shard_change.groovy @@ -0,0 +1,407 @@ +// 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 com.amazonaws.auth.AWSStaticCredentialsProvider +import com.amazonaws.auth.BasicAWSCredentials +import com.amazonaws.services.kinesis.AmazonKinesis +import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder +import com.amazonaws.services.kinesis.model.* +import java.math.BigInteger +import java.nio.ByteBuffer + +suite("test_kinesis_routine_load_shard_change") { + def region = context.config.awsRegion ?: context.config.otherConfigs.get("awsRegion") + def ak = context.config.awsAccessKey ?: context.config.otherConfigs.get("awsAccessKey") + def sk = context.config.awsSecretKey ?: context.config.otherConfigs.get("awsSecretKey") + + if (!region || !ak || !sk) { + logger.info("Skip ${name} case, missing AWS config: region=${region}, ak=${ak != null}, sk=${sk != null}") + return + } + + def suffix = UUID.randomUUID().toString().substring(0, 8) + def streamName = "doris-test-shard-change-${suffix}" + def jobName = "test_kinesis_shard_change_${suffix}" + + def credentials = new BasicAWSCredentials(ak, sk) + def kinesisClient = AmazonKinesisClientBuilder.standard() + .withRegion(region) + .withCredentials(new AWSStaticCredentialsProvider(credentials)) + .build() + + def toLongValue = { Object value -> + if (value instanceof Number) { + return ((Number) value).longValue() + } + return Long.parseLong(value.toString().trim()) + } + + def toIntValue = { Object value -> + if (value instanceof Number) { + return ((Number) value).intValue() + } + return Integer.parseInt(value.toString().trim()) + } + + def queryCount = { + def result = sql "SELECT COUNT(*) FROM test_kinesis_shard_change" + return toLongValue(result[0][0]) + } + + def waitForExactCount = { long expectedCount, int timeoutSec -> + long lastCount = -1 + for (int i = 0; i < timeoutSec; i++) { + lastCount = queryCount() + if (lastCount == expectedCount) { + logger.info("Table test_kinesis_shard_change row count reached exact ${expectedCount}") + return + } + if (lastCount > expectedCount) { + assertTrue(false, + "Row count exceeded expected ${expectedCount}, actual=${lastCount}. Possible duplicate consumption") + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting row count == ${expectedCount}, last count=${lastCount}") + } + + def putRecordWithRetry = { String partitionKey, String data, String explicitHashKey -> + def putRequest = new PutRecordRequest() + .withStreamName(streamName) + .withPartitionKey(partitionKey) + .withData(ByteBuffer.wrap(data.getBytes("UTF-8"))) + if (explicitHashKey != null) { + putRequest.withExplicitHashKey(explicitHashKey) + } + for (int retry = 0; retry < 20; retry++) { + try { + kinesisClient.putRecord(putRequest) + return + } catch (ResourceNotFoundException e) { + if (retry == 19) { + throw e + } + Thread.sleep(500) + } + } + } + + def writeRange = { int startId, int endId, String partitionKeyPrefix, String explicitHashKey -> + logger.info("Writing records ${startId}-${endId} to stream ${streamName}, " + + "partitionKeyPrefix=${partitionKeyPrefix}, explicitHashKey=${explicitHashKey}") + for (int i = startId; i <= endId; i++) { + def data = "{\"id\": ${i}, \"name\": \"user_${i}\", \"age\": ${20 + i % 50}}" + putRecordWithRetry("${partitionKeyPrefix}_${i}", data, explicitHashKey) + } + } + + def getShardMiddleHashKey = { shard -> + def startHash = new BigInteger(shard.getHashKeyRange().getStartingHashKey()) + def endHash = new BigInteger(shard.getHashKeyRange().getEndingHashKey()) + return startHash.add(endHash).divide(BigInteger.valueOf(2)).toString() + } + + def waitForOpenShardCount = { int expectedOpenShardNum, int timeoutSec -> + def describeRequest = new DescribeStreamRequest().withStreamName(streamName) + for (int i = 0; i < timeoutSec; i++) { + try { + def result = kinesisClient.describeStream(describeRequest) + def desc = result.getStreamDescription() + if (desc.getStreamStatus() == "ACTIVE" && !desc.getShards().isEmpty()) { + def openShards = desc.getShards().findAll { + it.getSequenceNumberRange().getEndingSequenceNumber() == null + } + if (openShards.size() == expectedOpenShardNum) { + logger.info("Open shard count reached ${expectedOpenShardNum}") + return openShards + } + } + } catch (ResourceNotFoundException e) { + // Metadata may not be visible immediately after create. + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting open shard count = ${expectedOpenShardNum}") + } + + def getJobState = { + def result = sql "SHOW ROUTINE LOAD FOR ${jobName}" + assertTrue(result.size() > 0, "SHOW ROUTINE LOAD returned empty result for ${jobName}") + return result[0][8].toString() + } + + def parseShardSet = { Object value -> + if (value == null) { + return [] as Set + } + def shardText = value.toString().trim() + if (shardText.length() == 0) { + return [] as Set + } + return shardText.split(",") + .collect { it.toString().trim() } + .findAll { it.length() > 0 } + .toSet() + } + + def getCurrentShardView = { + def result = sql "SHOW ROUTINE LOAD FOR ${jobName}" + assertTrue(result.size() > 0, "SHOW ROUTINE LOAD returned empty result for ${jobName}") + def dataSourceProperties = parseJson(result[0][12].toString()) + return [ + open: parseShardSet(dataSourceProperties.get("openKinesisShards")), + closed: parseShardSet(dataSourceProperties.get("closedKinesisShards")) + ] + } + + def waitForShardView = { Set expectedOpenShards, Set expectedClosedShards, Set absentShards, int timeoutSec -> + def lastView = [open: [] as Set, closed: [] as Set] + for (int i = 0; i < timeoutSec; i++) { + lastView = getCurrentShardView() + if (lastView.open == expectedOpenShards + && lastView.closed == expectedClosedShards + && absentShards.every { shardId -> + !lastView.open.contains(shardId) && !lastView.closed.contains(shardId) + }) { + logger.info("Routine load shard view reached expected state. open=${expectedOpenShards}, " + + "closed=${expectedClosedShards}, absent=${absentShards}") + return + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting shard view. expectedOpen=${expectedOpenShards}, " + + "expectedClosed=${expectedClosedShards}, absent=${absentShards}, " + + "lastOpen=${lastView.open}, lastClosed=${lastView.closed}") + } + + def assertShardViewStableAcrossRefresh = { Set expectedOpenShards, Set expectedClosedShards, + Set absentShards, int observeSec -> + for (int i = 0; i < observeSec; i++) { + def shardView = getCurrentShardView() + assertEquals(expectedOpenShards, shardView.open) + assertEquals(expectedClosedShards, shardView.closed) + absentShards.each { shardId -> + assertFalse(shardView.open.contains(shardId), + "Shard ${shardId} unexpectedly reappeared in open shards: ${shardView.open}") + assertFalse(shardView.closed.contains(shardId), + "Shard ${shardId} unexpectedly reappeared in closed shards: ${shardView.closed}") + } + Thread.sleep(1000) + } + } + + def streamCreated = false + def tableCreated = false + def jobCreated = false + def test1PreparedForReuse = false + def test2PreparedForReuse = false + + def parentShard = null + def splitChildShards = null + + def stopRoutineLoadIfCreated = { + if (jobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${jobName}" + } catch (Exception e) { + logger.warn("Failed to stop routine load ${jobName}: ${e.message}") + } + jobCreated = false + } + } + + def deleteStreamIfCreated = { + if (streamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(streamName)) + logger.info("Deleted stream: ${streamName}") + } catch (Exception e) { + logger.warn("Failed to delete stream ${streamName}: ${e.message}") + } + streamCreated = false + } + } + + def dropTableIfCreated = { + if (tableCreated) { + sql "DROP TABLE IF EXISTS test_kinesis_shard_change" + tableCreated = false + } + } + + try { + // test1 : create stream/table/job and verify baseline consumption before shard change + try { + logger.info("Creating Kinesis stream: ${streamName}") + kinesisClient.createStream(new CreateStreamRequest() + .withStreamName(streamName) + .withShardCount(1)) + streamCreated = true + + def initialOpenShards = waitForOpenShardCount(1, 120) + assertEquals(1, initialOpenShards.size()) + parentShard = initialOpenShards[0] + + sql "DROP TABLE IF EXISTS test_kinesis_shard_change" + sql """ + CREATE TABLE test_kinesis_shard_change ( + id INT, + name VARCHAR(100), + age INT + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ("replication_num" = "1") + """ + tableCreated = true + + sql """ + CREATE ROUTINE LOAD ${jobName} ON test_kinesis_shard_change + PROPERTIES ( + "format" = "json", + "desired_concurrent_number" = "2" + ) + FROM KINESIS ( + "aws.region" = "${region}", + "aws.access_key" = "${ak}", + "aws.secret_key" = "${sk}", + "kinesis_stream" = "${streamName}", + "property.kinesis_default_pos" = "TRIM_HORIZON" + ) + """ + jobCreated = true + + // Step 1: write first batch before split. + writeRange(1, 50, "before_split", null) + waitForExactCount(50, 180) + test1PreparedForReuse = true + } finally { + if (!test1PreparedForReuse) { + stopRoutineLoadIfCreated() + deleteStreamIfCreated() + dropTableIfCreated() + } + } + + // test2 : split shard and verify parent/child shard progress inheritance + try { + assertTrue(test1PreparedForReuse, "Test2 requires test1 to prepare stream/table/job") + assertTrue(parentShard != null, "Parent shard should be available before split") + + def parentShardId = parentShard.getShardId() + def splitHash = getShardMiddleHashKey(parentShard) + + // Step 2: split parent shard. + logger.info("Splitting shard ${parentShardId} at hash ${splitHash}") + kinesisClient.splitShard(new SplitShardRequest() + .withStreamName(streamName) + .withShardToSplit(parentShardId) + .withNewStartingHashKey(splitHash.toString())) + + def openShardsAfterSplit = waitForOpenShardCount(2, 180) + splitChildShards = openShardsAfterSplit.findAll { + it.getParentShardId() == parentShardId + } + assertEquals(2, splitChildShards.size()) + def childShardIds = splitChildShards.collect { it.getShardId() } + logger.info("Split produced child shards: ${childShardIds} from parent ${parentShardId}") + + def sortedChildren = splitChildShards.sort { + new BigInteger(it.getHashKeyRange().getStartingHashKey()) + } + def leftChild = sortedChildren[0] + def rightChild = sortedChildren[1] + def leftChildHash = getShardMiddleHashKey(leftChild) + def rightChildHash = getShardMiddleHashKey(rightChild) + + // Step 3: write to both child shards explicitly. + writeRange(51, 75, "split_left", leftChildHash) + writeRange(76, 100, "split_right", rightChildHash) + // Step 4: verify post-split data is imported. + waitForExactCount(100, 240) + + def expectedOpenChildShards = [leftChild.getShardId(), rightChild.getShardId()] as Set + // Wait until the retired parent shard is fully drained and removed from FE tracking. + waitForShardView(expectedOpenChildShards, [] as Set, [parentShardId] as Set, 180) + // Cross at least two FE scheduler cycles so the parent cannot be rediscovered from ListShards. + assertShardViewStableAcrossRefresh(expectedOpenChildShards, [] as Set, [parentShardId] as Set, 25) + test2PreparedForReuse = true + } finally { + if (!test2PreparedForReuse) { + stopRoutineLoadIfCreated() + deleteStreamIfCreated() + dropTableIfCreated() + } + } + + // test3 : merge split child shards and verify no data loss or duplicate consumption + try { + assertTrue(test2PreparedForReuse, "Test3 requires test2 to complete split stage") + assertTrue(splitChildShards != null && splitChildShards.size() == 2, + "Expected two child shards after split before merge") + + def sortedChildren = splitChildShards.sort { + new BigInteger(it.getHashKeyRange().getStartingHashKey()) + } + def leftChild = sortedChildren[0] + def rightChild = sortedChildren[1] + + // Step 5: merge split child shards. + logger.info("Merging adjacent shards: ${leftChild.getShardId()} + ${rightChild.getShardId()}") + kinesisClient.mergeShards(new MergeShardsRequest() + .withStreamName(streamName) + .withShardToMerge(leftChild.getShardId()) + .withAdjacentShardToMerge(rightChild.getShardId())) + + def openShardsAfterMerge = waitForOpenShardCount(1, 180) + assertEquals(1, openShardsAfterMerge.size()) + def mergedShard = openShardsAfterMerge[0] + def mergedParentIds = [mergedShard.getParentShardId(), mergedShard.getAdjacentParentShardId()].findAll { + it != null + } + assertEquals(2, mergedParentIds.size()) + assertTrue(mergedParentIds.contains(leftChild.getShardId())) + assertTrue(mergedParentIds.contains(rightChild.getShardId())) + logger.info("Merged shard ${mergedShard.getShardId()} has parents ${mergedParentIds}") + + def mergedHash = getShardMiddleHashKey(mergedShard) + // Step 6: write to merged shard explicitly. + writeRange(101, 150, "merged", mergedHash) + // Step 7: verify post-merge data is imported. + waitForExactCount(150, 240) + + waitForShardView([mergedShard.getShardId()] as Set, [] as Set, + [leftChild.getShardId(), rightChild.getShardId()] as Set, 180) + + def finalResult = sql "SELECT COUNT(*), COUNT(DISTINCT id), MIN(id), MAX(id) FROM test_kinesis_shard_change" + assertEquals(150L, toLongValue(finalResult[0][0])) + assertEquals(150L, toLongValue(finalResult[0][1])) + assertEquals(1, toIntValue(finalResult[0][2])) + assertEquals(150, toIntValue(finalResult[0][3])) + + def finalState = getJobState() + assertNotEquals("CANCELLED", finalState) + logger.info("Final routine load state after shard split+merge: ${finalState}") + } finally { + stopRoutineLoadIfCreated() + deleteStreamIfCreated() + dropTableIfCreated() + } + } finally { + kinesisClient.shutdown() + } +} diff --git a/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_show_routine_load.groovy b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_show_routine_load.groovy new file mode 100644 index 00000000000000..3c6cd17da367c0 --- /dev/null +++ b/regression-test/suites/load_p0/kinesis_routine_load/test_kinesis_show_routine_load.groovy @@ -0,0 +1,318 @@ +// 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 com.amazonaws.auth.AWSStaticCredentialsProvider +import com.amazonaws.auth.BasicAWSCredentials +import com.amazonaws.services.kinesis.AmazonKinesis +import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder +import com.amazonaws.services.kinesis.model.* +import java.nio.ByteBuffer + +suite("test_kinesis_show_routine_load") { + + def awsRegion = context.config.awsRegion ?: context.config.otherConfigs.get("awsRegion") + def awsAccessKey = context.config.awsAccessKey ?: context.config.otherConfigs.get("awsAccessKey") + def awsSecretKey = context.config.awsSecretKey ?: context.config.otherConfigs.get("awsSecretKey") + + if (!awsRegion || !awsAccessKey || !awsSecretKey) { + logger.info("Skip ${name} case, AWS config not provided") + return + } + + def suffix = UUID.randomUUID().toString().substring(0, 8) + String streamName = "doris-test-show-${suffix}" + String tableName = "test_kinesis_show_${suffix}" + String jobName = "test_kinesis_show_${suffix}" + + def credentials = new BasicAWSCredentials(awsAccessKey, awsSecretKey) + def kinesisClient = AmazonKinesisClientBuilder.standard() + .withRegion(awsRegion) + .withCredentials(new AWSStaticCredentialsProvider(credentials)) + .build() + + def toLongValue = { Object value -> + if (value instanceof Number) { + return ((Number) value).longValue() + } + return Long.parseLong(value.toString().trim()) + } + + def toIntValue = { Object value -> + if (value instanceof Number) { + return ((Number) value).intValue() + } + return Integer.parseInt(value.toString().trim()) + } + + def waitForStreamReady = { int timeoutSec -> + def describeRequest = new DescribeStreamRequest().withStreamName(streamName) + for (int i = 0; i < timeoutSec; i++) { + try { + def result = kinesisClient.describeStream(describeRequest) + def streamDesc = result.getStreamDescription() + if (streamDesc.getStreamStatus() == "ACTIVE" && !streamDesc.getShards().isEmpty()) { + return + } + } catch (ResourceNotFoundException e) { + // Stream metadata may not be visible immediately after create. + } + Thread.sleep(1000) + } + assertTrue(false, "Stream ${streamName} failed to become active") + } + + def putRecordWithRetry = { String partitionKey, String data -> + def putRequest = new PutRecordRequest() + .withStreamName(streamName) + .withPartitionKey(partitionKey) + .withData(ByteBuffer.wrap(data.getBytes("UTF-8"))) + for (int retry = 0; retry < 20; retry++) { + try { + kinesisClient.putRecord(putRequest) + return + } catch (ResourceNotFoundException e) { + if (retry == 19) { + throw e + } + Thread.sleep(500) + } + } + } + + def writeCsvRecords = { int startId, int endId, String partitionKeyPrefix -> + for (int i = startId; i <= endId; i++) { + def data = "${i},name_${i},2023-08-01,value_${i},2023-08-01 12:00:00,extra_${i}" + putRecordWithRetry("${partitionKeyPrefix}_${i}", data) + } + } + + def waitForCountAtLeast = { long expectedCount, int timeoutSec -> + long lastCount = -1 + for (int i = 0; i < timeoutSec; i++) { + def result = sql "SELECT COUNT(*) FROM ${tableName}" + lastCount = toLongValue(result[0][0]) + if (lastCount >= expectedCount) { + logger.info("Table ${tableName} row count reached ${lastCount} (expected >= ${expectedCount})") + return lastCount + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting table ${tableName} count >= ${expectedCount}, last count=${lastCount}") + } + + def waitForJobStateIn = { Set expectedStates, int timeoutSec, boolean includeHistory = false -> + def showSql = includeHistory + ? "SHOW ALL ROUTINE LOAD FOR ${jobName}" + : "SHOW ROUTINE LOAD FOR ${jobName}" + def lastState = "UNKNOWN" + for (int i = 0; i < timeoutSec; i++) { + def result = sql showSql + if (result.size() > 0) { + lastState = result[0][8].toString() + if (expectedStates.contains(lastState)) { + logger.info("Routine load job ${jobName} reached state ${lastState}") + return result[0] + } + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting job ${jobName} to reach states ${expectedStates}, last state=${lastState}") + } + + def waitForShowRowWithOpenShards = { int timeoutSec -> + for (int i = 0; i < timeoutSec; i++) { + def result = sql "SHOW ROUTINE LOAD FOR ${jobName}" + if (result.size() > 0) { + def row = result[0] + def dataSourceProperties = parseJson(row[12].toString()) + if (dataSourceProperties.containsKey("openKinesisShards") + && dataSourceProperties.openKinesisShards.toString().length() > 0) { + return row + } + } + Thread.sleep(1000) + } + assertTrue(false, "Timeout waiting job ${jobName} to expose openKinesisShards") + } + + def streamCreated = false + def tableCreated = false + def jobCreated = false + def test1PreparedForReuse = false + + try { + // test1 : verify SHOW ROUTINE LOAD core fields and kinesis-specific display content + try { + logger.info("Creating Kinesis stream: ${streamName}") + kinesisClient.createStream(new CreateStreamRequest() + .withStreamName(streamName) + .withShardCount(1)) + streamCreated = true + waitForStreamReady(120) + + writeCsvRecords(1, 10, "before_create") + + sql "DROP TABLE IF EXISTS ${tableName}" + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + k1 int(20) NULL, + k2 string NULL, + v1 date NULL, + v2 string NULL, + v3 datetime NULL, + v4 string NULL + ) ENGINE=OLAP + DUPLICATE KEY(k1) + DISTRIBUTED BY HASH(k1) BUCKETS 3 + PROPERTIES ("replication_num" = "1") + """ + tableCreated = true + + sql """ + CREATE ROUTINE LOAD ${jobName} ON ${tableName} + COLUMNS TERMINATED BY "," + PROPERTIES ( + "max_batch_interval" = "5", + "max_batch_rows" = "300000", + "max_batch_size" = "209715200" + ) + FROM KINESIS ( + "aws.region" = "${awsRegion}", + "aws.access_key" = "${awsAccessKey}", + "aws.secret_key" = "${awsSecretKey}", + "kinesis_stream" = "${streamName}", + "property.kinesis_default_pos" = "TRIM_HORIZON" + ) + """ + jobCreated = true + + waitForJobStateIn(["RUNNING", "NEED_SCHEDULE"] as Set, 120) + def showRow = waitForShowRowWithOpenShards(120) + + assertTrue(showRow.size() >= 23, "SHOW ROUTINE LOAD column count should be >= 23") + assertEquals(jobName, showRow[1].toString()) + assertEquals(tableName, showRow[6].toString()) + assertEquals("false", showRow[7].toString().toLowerCase()) + assertEquals("KINESIS", showRow[9].toString()) + assertTrue(toIntValue(showRow[10]) >= 0) + + def jobProperties = showRow[11].toString() + assertTrue(jobProperties.contains("max_batch_interval")) + assertTrue(jobProperties.contains("max_batch_rows")) + assertTrue(jobProperties.contains("max_batch_size")) + + def dataSourceProperties = parseJson(showRow[12].toString()) + assertEquals(awsRegion, dataSourceProperties.region.toString()) + assertEquals(streamName, dataSourceProperties.stream.toString()) + assertTrue(dataSourceProperties.containsKey("openKinesisShards")) + assertTrue(dataSourceProperties.openKinesisShards.toString().length() > 0) + assertTrue(dataSourceProperties.containsKey("closedKinesisShards")) + + def customProperties = parseJson(showRow[13].toString()) + assertEquals("******", customProperties["aws.secret_key"].toString()) + assertTrue(customProperties["aws.access_key"].toString().length() > 0) + + // Verify additional SHOW ROUTINE LOAD paths. + String dbName = context.config.getDbNameByFile(context.file) + def qualifiedResult = sql "SHOW ROUTINE LOAD FOR ${dbName}.${jobName}" + assertTrue(qualifiedResult.size() == 1) + assertEquals(showRow[0].toString(), qualifiedResult[0][0].toString()) + + def likeResult = sql "SHOW ROUTINE LOAD LIKE \"%${suffix}%\"" + assertTrue(likeResult.size() >= 1) + + waitForCountAtLeast(10L, 180) + + def showAfterLoad = sql "SHOW ROUTINE LOAD FOR ${jobName}" + assertTrue(showAfterLoad.size() > 0) + def loadedRow = showAfterLoad[0] + + def statistic = parseJson(loadedRow[14].toString()) + assertTrue(statistic.containsKey("openShardNum")) + assertTrue(toLongValue(statistic.openShardNum) >= 1L) + assertTrue(statistic.containsKey("trackedShardNum")) + + def progress = parseJson(loadedRow[15].toString()) + assertTrue(progress.size() >= 1, "Progress should contain at least one shard") + + def lag = parseJson(loadedRow[16].toString()) + assertTrue(lag.size() >= 1, "Lag should contain at least one shard") + lag.each { shardId, lagMs -> + assertTrue(toLongValue(lagMs) >= -1L, "Invalid lag value for shard ${shardId}: ${lagMs}") + } + test1PreparedForReuse = true + } finally { + if (!test1PreparedForReuse) { + if (jobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${jobName}" + } catch (Exception e) { + logger.warn("Failed to stop routine load ${jobName}: ${e.message}") + } + jobCreated = false + } + if (streamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(streamName)) + } catch (Exception e) { + logger.warn("Failed to delete stream: ${e.message}") + } + streamCreated = false + } + if (tableCreated) { + sql "DROP TABLE IF EXISTS ${tableName}" + tableCreated = false + } + } + } + + // test2 : stop job and verify SHOW ALL ROUTINE LOAD includes final state + try { + assertTrue(test1PreparedForReuse, "Test2 requires test1 to prepare stream/table/job") + + sql "STOP ROUTINE LOAD FOR ${jobName}" + jobCreated = false + + def historyRow = waitForJobStateIn(["STOPPED", "CANCELLED"] as Set, 120, true) + assertEquals(jobName, historyRow[1].toString()) + assertEquals("KINESIS", historyRow[9].toString()) + } finally { + if (jobCreated) { + try { + sql "STOP ROUTINE LOAD FOR ${jobName}" + } catch (Exception e) { + logger.warn("Failed to stop routine load ${jobName}: ${e.message}") + } + jobCreated = false + } + if (streamCreated) { + try { + kinesisClient.deleteStream(new DeleteStreamRequest().withStreamName(streamName)) + } catch (Exception e) { + logger.warn("Failed to delete stream: ${e.message}") + } + streamCreated = false + } + if (tableCreated) { + sql "DROP TABLE IF EXISTS ${tableName}" + tableCreated = false + } + } + } finally { + kinesisClient.shutdown() + } +} diff --git a/thirdparty/CHANGELOG.md b/thirdparty/CHANGELOG.md index eda6d280f5e0e3..5459a81ea658d6 100644 --- a/thirdparty/CHANGELOG.md +++ b/thirdparty/CHANGELOG.md @@ -2,6 +2,10 @@ This file contains version of the third-party dependency libraries in the build-env image. The docker build-env image is apache/doris, and the tag is `build-env-${version}` +## 20260407 + +- Added: aws-cpp-sdk-kinesis (from aws-sdk-cpp 1.11.219) for Kinesis routine load + ## 20260206 - Modified: jindofs 6.8.2 -> 6.10.4 diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh index 793ff84cf38f50..71a3b38afb805f 100755 --- a/thirdparty/build-thirdparty.sh +++ b/thirdparty/build-thirdparty.sh @@ -1428,7 +1428,7 @@ build_aws_sdk() { -DCMAKE_POLICY_VERSION_MINIMUM=3.5 \ -DCMAKE_PREFIX_PATH="${TP_INSTALL_DIR}" -DBUILD_SHARED_LIBS=OFF -DENABLE_TESTING=OFF \ -DCURL_LIBRARY_RELEASE="${TP_INSTALL_DIR}/lib/libcurl.a" -DZLIB_LIBRARY_RELEASE="${TP_INSTALL_DIR}/lib/libz.a" \ - -DBUILD_ONLY="core;s3;s3-crt;transfer;identity-management;sts" \ + -DBUILD_ONLY="core;s3;s3-crt;transfer;identity-management;sts;kinesis" \ -DCMAKE_CXX_FLAGS="-Wno-nonnull -Wno-deprecated-literal-operator ${warning_deprecated_literal_operator} -Wno-deprecated-declarations ${warning_dangling_reference}" -DCPP_STANDARD=17 cd "${BUILD_DIR}" @@ -1453,6 +1453,7 @@ build_aws_sdk() { strip_lib libaws-c-compression.a strip_lib libaws-cpp-sdk-identity-management.a strip_lib libaws-cpp-sdk-sts.a + strip_lib libaws-cpp-sdk-kinesis.a } # lzma diff --git a/thirdparty/download-thirdparty.sh b/thirdparty/download-thirdparty.sh index 75ba6313529c2b..37c1e308b3b846 100755 --- a/thirdparty/download-thirdparty.sh +++ b/thirdparty/download-thirdparty.sh @@ -494,8 +494,11 @@ if [[ " ${TP_ARCHIVES[*]} " =~ " AWS_SDK " ]]; then fi patch -p1 <"${TP_PATCH_DIR}/aws-sdk-cpp-1.11.119.patch" patch -p1 <"${TP_PATCH_DIR}/aws-sdk-cpp-1.11.119-cmake.patch" - else + elif [[ "${AWS_SDK_SOURCE}" == "aws-sdk-cpp-1.11.219" ]]; then bash ./prefetch_crt_dependency.sh + else + echo "Unsupported AWS SDK source ${AWS_SDK_SOURCE}" + exit 1 fi touch "${PATCHED_MARK}" fi diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh index 4fe49dd51687cf..2a2b9cbbc3c39e 100644 --- a/thirdparty/vars.sh +++ b/thirdparty/vars.sh @@ -354,7 +354,7 @@ BOOTSTRAP_TABLE_CSS_NAME="bootstrap-table.min.css" BOOTSTRAP_TABLE_CSS_FILE="bootstrap-table.min.css" BOOTSTRAP_TABLE_CSS_MD5SUM="23389d4456da412e36bae30c469a766a" -# aws sdk +# aws sdk (includes kinesis client for routine load) AWS_SDK_DOWNLOAD="https://github.com/aws/aws-sdk-cpp/archive/refs/tags/1.11.219.tar.gz" AWS_SDK_NAME="aws-sdk-cpp-1.11.219.tar.gz" AWS_SDK_SOURCE="aws-sdk-cpp-1.11.219"