diff --git a/be/src/agent/agent_server.cpp b/be/src/agent/agent_server.cpp index 72aeeb080e5db0..740a6e4de10a94 100644 --- a/be/src/agent/agent_server.cpp +++ b/be/src/agent/agent_server.cpp @@ -70,13 +70,6 @@ AgentServer::AgentServer(ExecEnv* exec_env, } } - // create tmp dir -// boost::filesystem::path tmp_path(config::agent_tmp_dir); -// if (boost::filesystem::exists(tmp_path)) { -// boost::filesystem::remove_all(tmp_path); -// } -// boost::filesystem::create_directories(config::agent_tmp_dir); - // init task worker pool _create_table_workers = new TaskWorkerPool( TaskWorkerPool::TaskWorkerType::CREATE_TABLE, diff --git a/be/src/agent/heartbeat_server.cpp b/be/src/agent/heartbeat_server.cpp index e485426e41f800..af7a0b72f89b97 100644 --- a/be/src/agent/heartbeat_server.cpp +++ b/be/src/agent/heartbeat_server.cpp @@ -143,6 +143,10 @@ Status HeartbeatServer::_heartbeat( } } + if (master_info.__isset.http_port) { + _master_info->__set_http_port(master_info.http_port); + } + if (need_report) { LOG(INFO) << "Master FE is changed or restarted. report tablet and disk info immediately"; _olap_engine->report_notify(true); diff --git a/be/src/common/config.h b/be/src/common/config.h index 4529ad2e8508f9..2ab793c5782c06 100644 --- a/be/src/common/config.h +++ b/be/src/common/config.h @@ -114,14 +114,10 @@ namespace config { CONF_Int32(sleep_one_second, "1"); // sleep time for five seconds CONF_Int32(sleep_five_seconds, "5"); - // trans file tools dir - CONF_String(trans_file_tool_path, "${DORIS_HOME}/tools/trans_file_tool/trans_files.sh"); - // agent tmp dir - CONF_String(agent_tmp_dir, "${DORIS_HOME}/tmp"); // log dir CONF_String(sys_log_dir, "${DORIS_HOME}/log"); - CONF_String(user_function_dir, "${DORIS_HOME}/lib/usr"); + CONF_String(user_function_dir, "${DORIS_HOME}/lib/udf"); // INFO, WARNING, ERROR, FATAL CONF_String(sys_log_level, "INFO"); // TIME-DAY, TIME-HOUR, SIZE-MB-nnn @@ -208,7 +204,7 @@ namespace config { CONF_Int32(file_descriptor_cache_clean_interval, "3600"); CONF_Int32(disk_stat_monitor_interval, "5"); CONF_Int32(unused_index_monitor_interval, "30"); - CONF_String(storage_root_path, "${DORIS_HOME}/storage"); + CONF_String(storage_root_path, "${DORIS_HOME}/data"); CONF_Int32(min_percentage_of_error_disk, "50"); CONF_Int32(default_num_rows_per_data_block, "1024"); CONF_Int32(default_num_rows_per_column_file_block, "1024"); @@ -256,12 +252,8 @@ namespace config { // Port to start debug webserver on CONF_Int32(webserver_port, "8040"); - // Interface to start debug webserver on. If blank, webserver binds to 0.0.0.0 - CONF_String(webserver_interface, ""); - CONF_String(webserver_doc_root, "${DORIS_HOME}"); + // Number of webserver workers CONF_Int32(webserver_num_workers, "5"); - // If true, webserver may serve static files from the webserver_doc_root - CONF_Bool(enable_webserver_doc_root, "true"); // Period to update rate counters and sampling counters in ms. CONF_Int32(periodic_counter_update_period_ms, "500"); @@ -419,6 +411,9 @@ namespace config { // same cache size configuration. // TODO(cmy): use different config to set different client cache if necessary. CONF_Int32(max_client_cache_size_per_host, "10"); + + // Dir to save files downloaded by SmallFileMgr + CONF_String(small_file_dir, "${DORIS_HOME}/lib/small_file/"); } // namespace config } // namespace doris diff --git a/be/src/runtime/CMakeLists.txt b/be/src/runtime/CMakeLists.txt index 22e8c5c0aad556..5ad5e3585b19b4 100644 --- a/be/src/runtime/CMakeLists.txt +++ b/be/src/runtime/CMakeLists.txt @@ -99,6 +99,7 @@ set(RUNTIME_FILES routine_load/data_consumer_group.cpp routine_load/data_consumer_pool.cpp routine_load/routine_load_task_executor.cpp + small_file_mgr.cpp ) if (WITH_MYSQL) diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h index 9d7e1bc301e936..9595a7d4a7c92e 100644 --- a/be/src/runtime/exec_env.h +++ b/be/src/runtime/exec_env.h @@ -52,6 +52,7 @@ class TmpFileMgr; class WebPageHandler; class StreamLoadExecutor; class RoutineLoadTaskExecutor; +class SmallFileMgr; class BackendServiceClient; class FrontendServiceClient; @@ -112,6 +113,7 @@ class ExecEnv { BufferPool* buffer_pool() { return _buffer_pool; } TabletWriterMgr* tablet_writer_mgr() { return _tablet_writer_mgr; } LoadStreamMgr* load_stream_mgr() { return _load_stream_mgr; } + SmallFileMgr* small_file_mgr() { return _small_file_mgr; } const std::vector& store_paths() const { return _store_paths; } void set_store_paths(const std::vector& paths) { _store_paths = paths; } @@ -167,6 +169,7 @@ class ExecEnv { StreamLoadExecutor* _stream_load_executor = nullptr; RoutineLoadTaskExecutor* _routine_load_task_executor = nullptr; + SmallFileMgr* _small_file_mgr = nullptr; }; } diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp index 21e78cd4941a92..9208410c143c15 100644 --- a/be/src/runtime/exec_env_init.cpp +++ b/be/src/runtime/exec_env_init.cpp @@ -47,6 +47,7 @@ #include "runtime/routine_load/routine_load_task_executor.h" #include "runtime/stream_load/load_stream_mgr.h" #include "runtime/stream_load/stream_load_executor.h" +#include "runtime/small_file_mgr.h" #include "util/pretty_printer.h" #include "util/doris_metrics.h" #include "util/brpc_stub_cache.h" @@ -99,6 +100,7 @@ Status ExecEnv::_init(const std::vector& store_paths) { _brpc_stub_cache = new BrpcStubCache(); _stream_load_executor = new StreamLoadExecutor(this); _routine_load_task_executor = new RoutineLoadTaskExecutor(this); + _small_file_mgr = new SmallFileMgr(this, config::small_file_dir); _backend_client_cache->init_metrics(DorisMetrics::metrics(), "backend"); _frontend_client_cache->init_metrics(DorisMetrics::metrics(), "frontend"); @@ -118,6 +120,7 @@ Status ExecEnv::_init(const std::vector& store_paths) { exit(-1); } _broker_mgr->init(); + _small_file_mgr->init(); _init_mem_tracker(); RETURN_IF_ERROR(_tablet_writer_mgr->start_bg_worker()); return Status::OK; diff --git a/be/src/runtime/pull_load_task_mgr.cpp b/be/src/runtime/pull_load_task_mgr.cpp index bfe87389bfded9..4002ecef439d8a 100644 --- a/be/src/runtime/pull_load_task_mgr.cpp +++ b/be/src/runtime/pull_load_task_mgr.cpp @@ -118,13 +118,13 @@ PullLoadTaskMgr::~PullLoadTaskMgr() { Status PullLoadTaskMgr::init() { auto st = load_task_ctxes(); if (!st.ok()) { - LOG(WARNING) << "Load task from directory failed. because " << st.get_error_msg(); _dir_exist = false; } return Status::OK; } Status PullLoadTaskMgr::load_task_ctxes() { + /* // 1. scan all files std::vector files; RETURN_IF_ERROR(FileUtils::scan_dir(_path, &files)); @@ -141,8 +141,9 @@ Status PullLoadTaskMgr::load_task_ctxes() { << ", status:" << status.get_error_msg(); } } + */ - return Status::OK; + return Status("Not implemented"); } Status PullLoadTaskMgr::load_task_ctx(const std::string& file_path) { diff --git a/be/src/runtime/routine_load/data_consumer.cpp b/be/src/runtime/routine_load/data_consumer.cpp index 677c17e14b1f57..14df29c284320f 100644 --- a/be/src/runtime/routine_load/data_consumer.cpp +++ b/be/src/runtime/routine_load/data_consumer.cpp @@ -24,6 +24,7 @@ #include "common/status.h" #include "service/backend_options.h" +#include "runtime/small_file_mgr.h" #include "util/defer_op.h" #include "util/stopwatch.hpp" #include "util/uid_util.h" @@ -52,9 +53,17 @@ Status KafkaDataConsumer::init(StreamLoadContext* ctx) { std::string errstr; auto set_conf = [&conf, &errstr](const std::string& conf_key, const std::string& conf_val) { - if (conf->set(conf_key, conf_val, errstr) != RdKafka::Conf::CONF_OK) { + RdKafka::Conf::ConfResult res = conf->set(conf_key, conf_val, errstr); + if (res == RdKafka::Conf::CONF_UNKNOWN) { + // ignore unknown config + return Status::OK; + } else if (errstr.find("not supported") != std::string::npos) { + // some java-only properties may be passed to here, and librdkafak will return 'xxx' not supported + // ignore it + return Status::OK; + } else if (res != RdKafka::Conf::CONF_OK) { std::stringstream ss; - ss << "failed to set '" << conf_key << "'"; + ss << "PAUSE: failed to set '" << conf_key << "', value: '" << conf_val << "', err: " << errstr; LOG(WARNING) << ss.str(); return Status(ss.str()); } @@ -73,12 +82,31 @@ Status KafkaDataConsumer::init(StreamLoadContext* ctx) { RETURN_IF_ERROR(set_conf("api.version.fallback.ms", "0")); for (auto& item : ctx->kafka_info->properties) { - RETURN_IF_ERROR(set_conf(item.first, item.second)); + if (boost::algorithm::starts_with(item.second, "FILE:")) { + // file property should has format: FILE:file_id:md5 + std::vector parts; + boost::split(parts, item.second, boost::is_any_of(":")); + if (parts.size() != 3) { + return Status("PAUSE: Invalid file property of kafka: " + item.second); + } + int64_t file_id = std::stol(parts[1]); + std::string file_path; + Status st = ctx->exec_env()->small_file_mgr()->get_file(file_id, parts[2], &file_path); + if (!st.ok()) { + std::stringstream ss; + ss << "PAUSE: failed to get file for config: " << item.first << ", error: " << st.get_error_msg(); + return Status(ss.str()); + } + RETURN_IF_ERROR(set_conf(item.first, file_path)); + } else { + RETURN_IF_ERROR(set_conf(item.first, item.second)); + } + _custom_properties.emplace(item.first, item.second); } if (conf->set("event_cb", &_k_event_cb, errstr) != RdKafka::Conf::CONF_OK) { std::stringstream ss; - ss << "failed to set 'event_cb'"; + ss << "PAUSE: failed to set 'event_cb'"; LOG(WARNING) << ss.str(); return Status(ss.str()); } @@ -86,8 +114,8 @@ Status KafkaDataConsumer::init(StreamLoadContext* ctx) { // create consumer _k_consumer = RdKafka::KafkaConsumer::create(conf, errstr); if (!_k_consumer) { - LOG(WARNING) << "failed to create kafka consumer"; - return Status("failed to create kafka consumer"); + LOG(WARNING) << "PAUSE: failed to create kafka consumer: " << errstr; + return Status("PAUSE: failed to create kafka consumer: " + errstr); } VLOG(3) << "finished to init kafka consumer. " << ctx->brief(); @@ -174,7 +202,7 @@ Status KafkaDataConsumer::group_consume( case RdKafka::ERR__TIMED_OUT: // leave the status as OK, because this may happend // if there is no data in kafka. - LOG(WARNING) << "kafka consume timeout: " << _id; + LOG(INFO) << "kafka consume timeout: " << _id; break; default: LOG(WARNING) << "kafka consume failed: " << _id @@ -199,6 +227,66 @@ Status KafkaDataConsumer::group_consume( return st; } +Status KafkaDataConsumer::get_partition_meta(std::vector* partition_ids) { + // create topic conf + RdKafka::Conf *tconf = RdKafka::Conf::create(RdKafka::Conf::CONF_TOPIC); + auto conf_deleter = [tconf] () { delete tconf; }; + DeferOp delete_conf(std::bind(conf_deleter)); + + // create topic + std::string errstr; + RdKafka::Topic *topic = RdKafka::Topic::create(_k_consumer, _topic, tconf, errstr); + if (topic == nullptr) { + std::stringstream ss; + ss << "failed to create topic: " << errstr; + LOG(WARNING) << ss.str(); + return Status(ss.str()); + } + auto topic_deleter = [topic] () { delete topic; }; + DeferOp delete_topic(std::bind(topic_deleter)); + + // get topic metadata + RdKafka::Metadata* metadata = nullptr; + RdKafka::ErrorCode err = _k_consumer->metadata(true/* for this topic */, topic, &metadata, 5000); + if (err != RdKafka::ERR_NO_ERROR) { + std::stringstream ss; + ss << "failed to get partition meta: " << RdKafka::err2str(err); + LOG(WARNING) << ss.str(); + return Status(ss.str()); + } + auto meta_deleter = [metadata] () { delete metadata; }; + DeferOp delete_meta(std::bind(meta_deleter)); + + // get partition ids + RdKafka::Metadata::TopicMetadataIterator it; + for (it = metadata->topics()->begin(); it != metadata->topics()->end(); ++it) { + if ((*it)->topic() != _topic) { + continue; + } + + if ((*it)->err() != RdKafka::ERR_NO_ERROR) { + std::stringstream ss; + ss << "error: " << err2str((*it)->err()); + if ((*it)->err() == RdKafka::ERR_LEADER_NOT_AVAILABLE) { + ss << ", try again"; + } + LOG(WARNING) << ss.str(); + return Status(ss.str()); + } + + RdKafka::TopicMetadata::PartitionMetadataIterator ip; + for (ip = (*it)->partitions()->begin(); ip != (*it)->partitions()->end(); ++ip) { + partition_ids->push_back((*ip)->id()); + } + } + + if (partition_ids->empty()) { + return Status("no partition in this topic"); + } + + return Status::OK; +} + Status KafkaDataConsumer::cancel(StreamLoadContext* ctx) { std::unique_lock l(_lock); if (!_init) { @@ -225,6 +313,15 @@ bool KafkaDataConsumer::match(StreamLoadContext* ctx) { if (_brokers != ctx->kafka_info->brokers || _topic != ctx->kafka_info->topic) { return false; } + // check properties + if (_custom_properties.size() != ctx->kafka_info->properties.size()) { + return false; + } + for (auto& item : ctx->kafka_info->properties) { + if (_custom_properties.find(item.first) == _custom_properties.end()) { + return false; + } + } return true; } diff --git a/be/src/runtime/routine_load/data_consumer.h b/be/src/runtime/routine_load/data_consumer.h index b42d36c89110ba..baf2cf74492f9d 100644 --- a/be/src/runtime/routine_load/data_consumer.h +++ b/be/src/runtime/routine_load/data_consumer.h @@ -19,6 +19,7 @@ #include #include +#include #include "librdkafka/rdkafkacpp.h" @@ -141,9 +142,13 @@ class KafkaDataConsumer : public DataConsumer { // start the consumer and put msgs to queue Status group_consume(BlockingQueue* queue, int64_t max_running_time_ms); + // get the partitions ids of the topic + Status get_partition_meta(std::vector* partition_ids); + private: std::string _brokers; std::string _topic; + std::unordered_map _custom_properties; KafkaEventCb _k_event_cb; RdKafka::KafkaConsumer* _k_consumer = nullptr; diff --git a/be/src/runtime/routine_load/data_consumer_pool.cpp b/be/src/runtime/routine_load/data_consumer_pool.cpp index 0267fee119ce5a..9528bb4968c6d0 100644 --- a/be/src/runtime/routine_load/data_consumer_pool.cpp +++ b/be/src/runtime/routine_load/data_consumer_pool.cpp @@ -50,7 +50,7 @@ Status DataConsumerPool::get_consumer( break; default: std::stringstream ss; - ss << "unknown routine load task type: " << ctx->load_type; + ss << "PAUSE: unknown routine load task type: " << ctx->load_type; return Status(ss.str()); } @@ -66,7 +66,7 @@ Status DataConsumerPool::get_consumer_grp( StreamLoadContext* ctx, std::shared_ptr* ret) { if (ctx->load_src_type != TLoadSourceType::KAFKA) { - return Status("Currently nly support consumer group for Kafka data source"); + return Status("PAUSE: Currently only support consumer group for Kafka data source"); } DCHECK(ctx->kafka_info); diff --git a/be/src/runtime/routine_load/routine_load_task_executor.cpp b/be/src/runtime/routine_load/routine_load_task_executor.cpp index 333768bf1c1180..5875be74310b56 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.cpp +++ b/be/src/runtime/routine_load/routine_load_task_executor.cpp @@ -33,6 +33,40 @@ namespace doris { +Status RoutineLoadTaskExecutor::get_kafka_partition_meta( + const PKafkaMetaProxyRequest& request, std::vector* partition_ids) { + DCHECK(request.has_kafka_info()); + + // This context is meaningless, just for unifing the interface + StreamLoadContext ctx(_exec_env); + ctx.load_type = TLoadType::ROUTINE_LOAD; + ctx.load_src_type = TLoadSourceType::KAFKA; + ctx.label = "NaN"; + + // convert PKafkaInfo to TKafkaLoadInfo + TKafkaLoadInfo t_info; + t_info.brokers = request.kafka_info().brokers(); + t_info.topic = request.kafka_info().topic(); + std::map properties; + for (int i = 0; i < request.kafka_info().properties_size(); ++i) { + const PStringPair& pair = request.kafka_info().properties(i); + properties.emplace(pair.key(), pair.val()); + } + t_info.__set_properties(std::move(properties)); + + ctx.kafka_info = new KafkaLoadInfo(t_info); + ctx.need_rollback = false; + + std::shared_ptr consumer; + RETURN_IF_ERROR(_data_consumer_pool.get_consumer(&ctx, &consumer)); + + Status st = std::static_pointer_cast(consumer)->get_partition_meta(partition_ids); + if (st.ok()) { + _data_consumer_pool.return_consumer(consumer); + } + return st; +} + Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { std::unique_lock l(_lock); if (_task_map.find(task.id) != _task_map.end()) { @@ -41,7 +75,8 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) { return Status::OK; } - if (_thread_pool.get_queue_size() > 100) { + // the max queue size of thread pool is 100, here we use 80 as a very conservative limit + if (_thread_pool.get_queue_size() >= 80) { LOG(INFO) << "too many tasks in queue: " << _thread_pool.get_queue_size() << ", reject task: " << UniqueId(task.id); return Status("too many tasks"); } diff --git a/be/src/runtime/routine_load/routine_load_task_executor.h b/be/src/runtime/routine_load/routine_load_task_executor.h index bbecb13a051c86..adf2dab473ffd1 100644 --- a/be/src/runtime/routine_load/routine_load_task_executor.h +++ b/be/src/runtime/routine_load/routine_load_task_executor.h @@ -25,6 +25,8 @@ #include "util/thread_pool.hpp" #include "util/uid_util.h" +#include "gen_cpp/internal_service.pb.h" + namespace doris { class ExecEnv; @@ -55,6 +57,8 @@ class RoutineLoadTaskExecutor { // submit a routine load task Status submit_task(const TRoutineLoadTask& task); + + Status get_kafka_partition_meta(const PKafkaMetaProxyRequest& request, std::vector* partition_ids); private: // execute the task diff --git a/be/src/runtime/small_file_mgr.cpp b/be/src/runtime/small_file_mgr.cpp new file mode 100644 index 00000000000000..eed5eea22c9eec --- /dev/null +++ b/be/src/runtime/small_file_mgr.cpp @@ -0,0 +1,231 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "runtime/small_file_mgr.h" + +#include +#include +#include + +#include "common/status.h" +#include "http/http_client.h" +#include "runtime/exec_env.h" +#include "util/file_utils.h" +#include "util/md5.h" + +#include // boost::split +#include // boost::algorithm::starts_with +#include // boost::is_any_of + +#include "gen_cpp/HeartbeatService.h" + +namespace doris { + +SmallFileMgr::SmallFileMgr( + ExecEnv* env, + const std::string& local_path) : + _exec_env(env), + _local_path(local_path) { +} + +SmallFileMgr::~SmallFileMgr() { +} + +Status SmallFileMgr::init() { + RETURN_IF_ERROR(_load_local_files()); + return Status::OK; +} + +Status SmallFileMgr::_load_local_files() { + RETURN_IF_ERROR(FileUtils::create_dir(_local_path)); + + auto scan_cb = [this] (const std::string& dir, const std::string& file) { + auto st = _load_single_file(dir, file); + if (!st.ok()) { + LOG(WARNING) << "load small file failed: " << st.get_error_msg(); + } + return true; + }; + + RETURN_IF_ERROR(FileUtils::scan_dir(_local_path, scan_cb)); + return Status::OK; +} + +Status SmallFileMgr::_load_single_file( + const std::string& path, + const std::string& file_name) { + // file name format should be like: + // file_id.md5 + std::vector parts; + boost::split(parts, file_name, boost::is_any_of(".")); + if (parts.size() != 2) { + return Status("Not a valid file name: " + file_name); + } + int64_t file_id = std::stol(parts[0]); + std::string md5 = parts[1]; + + if (_file_cache.find(file_id) != _file_cache.end()) { + return Status("File with same id is already been loaded: " + file_id); + } + + std::string file_md5; + RETURN_IF_ERROR(FileUtils::md5sum(path + "/" + file_name, &file_md5)); + if (file_md5 != md5) { + return Status("Invalid md5 of file: " + file_name); + } + + CacheEntry entry; + entry.path = path + "/" + file_name; + entry.md5 = file_md5; + + _file_cache.emplace(file_id, entry); + return Status::OK; +} + +Status SmallFileMgr::get_file( + int64_t file_id, + const std::string& md5, + std::string* file_path) { + + std::unique_lock l(_lock); + // find in cache + auto it = _file_cache.find(file_id); + if (it != _file_cache.end()) { + // find the cached file, check it + CacheEntry& entry = it->second; + Status st = _check_file(entry, md5); + if (!st.ok()) { + // check file failed, we should remove this cache and download it from FE again + if (remove(entry.path.c_str()) != 0) { + std::stringstream ss; + ss << "failed to remove file: " << file_id << ", err: "<< std::strerror(errno); + return Status(ss.str()); + } + _file_cache.erase(it); + } else { + // check ok, return the path + *file_path = entry.path; + return Status::OK; + } + } + + // file not found in cache. download it from FE + RETURN_IF_ERROR(_download_file(file_id, md5, file_path)); + + return Status::OK; +} + +Status SmallFileMgr::_check_file(const CacheEntry& entry, const std::string& md5) { + if (!FileUtils::check_exist(entry.path)) { + return Status("file not exist"); + } + if (!boost::iequals(md5, entry.md5)) { + return Status("invalid MD5"); + } + return Status::OK; +} + +Status SmallFileMgr::_download_file( + int64_t file_id, + const std::string& md5, + std::string* file_path) { + + std::stringstream ss; + ss << _local_path << "/" << file_id << ".tmp"; + std::string tmp_file = ss.str(); + bool should_delete = true; + auto fp_closer = [&tmp_file, &should_delete] (FILE* fp) { + fclose(fp); + if (should_delete) remove(tmp_file.c_str()); + }; + + std::unique_ptr fp(fopen(tmp_file.c_str(), "w"), fp_closer); + if (fp == nullptr) { + LOG(WARNING) << "fail to open file, file=" << tmp_file; + return Status("fail to open file"); + } + + HttpClient client; + + std::stringstream url_ss; +#ifndef BE_TEST + TMasterInfo* master_info = _exec_env->master_info(); + url_ss << master_info->network_address.hostname << ":" << master_info->http_port << "/api/get_small_file?" + << "file_id=" << file_id << "&token=" << master_info->token; +#else + url_ss << "127.0.0.1:29997/api/get_small_file?file_id=" << file_id; +#endif + + std::string url = url_ss.str(); + + LOG(INFO) << "download file from: " << url; + + RETURN_IF_ERROR(client.init(url)); + Status status; + Md5Digest digest; + auto download_cb = [&status, &tmp_file, &fp, &digest] (const void* data, size_t length) { + digest.update(data, length); + auto res = fwrite(data, length, 1, fp.get()); + if (res != 1) { + LOG(WARNING) << "fail to write data to file, file=" << tmp_file + << ", error=" << ferror(fp.get()); + status = Status("fail to write data when download"); + return false; + } + return true; + }; + RETURN_IF_ERROR(client.execute(download_cb)); + RETURN_IF_ERROR(status); + digest.digest(); + + if (!boost::iequals(digest.hex(), md5)) { + LOG(WARNING) << "file's checksum is not equal, download: " << digest.hex() + << ", expected: " << md5 << ", file: " << file_id; + return Status("download with invalid md5"); + } + + // close this file + should_delete = false; + fp.reset(); + + // rename temporary file to library file + std::stringstream real_ss; + real_ss << _local_path << "/" << file_id << "." << md5; + std::string real_file_path = real_ss.str(); + auto ret = rename(tmp_file.c_str(), real_file_path.c_str()); + if (ret != 0) { + char buf[64]; + LOG(WARNING) << "fail to rename file from=" << tmp_file << ", to=" << real_file_path + << ", errno=" << errno << ", errmsg=" << strerror_r(errno, buf, 64); + remove(tmp_file.c_str()); + remove(real_file_path.c_str()); + return Status("fail to rename file"); + } + + // add to file cache + CacheEntry entry; + entry.path = real_file_path; + entry.md5 = md5; + _file_cache.emplace(file_id, entry); + + *file_path = real_file_path; + + LOG(INFO) << "finished to download file: " << file_path; + return Status::OK; +} + +} // end namespace doris diff --git a/be/src/runtime/small_file_mgr.h b/be/src/runtime/small_file_mgr.h new file mode 100644 index 00000000000000..7607b80dbd5ff2 --- /dev/null +++ b/be/src/runtime/small_file_mgr.h @@ -0,0 +1,88 @@ +// 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. + +#ifndef DORIS_BE_SRC_RUNTIME_SMALL_FILE_MGR_H +#define DORIS_BE_SRC_RUNTIME_SMALL_FILE_MGR_H + +#include + +#include +#include +#include +#include +#include + +#include "gen_cpp/Types_types.h" + +#include "common/status.h" +#include "runtime/client_cache.h" + +namespace doris { + +class ExecEnv; + +struct CacheEntry { + std::string path; // absolute path + std::string md5; +}; + +/* + * SmallFileMgr is used to download small files saved in FE, + * such as certification files, public/private keys + */ +class SmallFileMgr { +public: + SmallFileMgr(ExecEnv* env, const std::string& local_path); + + ~SmallFileMgr(); + + // call init() when BE start up. load all local files + Status init(); + + // get file by specified file_id, return 'file_path' + // if file does not exist, it will be downloaded from FE + Status get_file( + int64_t file_id, + const std::string& md5, + std::string* file_path); + +private: + Status _load_local_files(); + + // load one single local file + Status _load_single_file( + const std::string& path, + const std::string& file_name); + + Status _check_file(const CacheEntry& entry, const std::string& md5); + + Status _download_file( + int64_t file_id, + const std::string& md5, + std::string* file_path); + +private: + std::mutex _lock; + ExecEnv* _exec_env; + std::string _local_path; + // file id -> small file + std::unordered_map _file_cache; +}; + +} // end namespace doris + +#endif // DORIS_BE_SRC_RUNTIME_SMALL_FILE_MGR_H diff --git a/be/src/runtime/stream_load/stream_load_context.h b/be/src/runtime/stream_load/stream_load_context.h index 54a5b6e209e646..55ac6a30c18aad 100644 --- a/be/src/runtime/stream_load/stream_load_context.h +++ b/be/src/runtime/stream_load/stream_load_context.h @@ -172,6 +172,10 @@ class StreamLoadContext { // consumer_id is used for data consumer cache key. // to identified a specified data consumer. int64_t consumer_id; + +public: + ExecEnv* exec_env() { return _exec_env; } + private: ExecEnv* _exec_env; std::atomic _refs; diff --git a/be/src/service/backend_service.h b/be/src/service/backend_service.h index 51f9505a4a6c53..78bf4fc69f63fc 100644 --- a/be/src/service/backend_service.h +++ b/be/src/service/backend_service.h @@ -148,7 +148,6 @@ class BackendService : public BackendServiceIf { virtual void get_tablet_stat(TTabletStatResult& result) override; virtual void submit_routine_load_task(TStatus& t_status, const std::vector& tasks) override; - private: Status start_plan_fragment_execution(const TExecPlanFragmentParams& exec_params); diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp index 07095faaa983d3..fb4baac6cf8f1c 100644 --- a/be/src/service/internal_service.cpp +++ b/be/src/service/internal_service.cpp @@ -28,6 +28,7 @@ #include "util/thrift_util.h" #include "runtime/buffer_control_block.h" #include "runtime/result_buffer_mgr.h" +#include "runtime/routine_load/routine_load_task_executor.h" namespace doris { @@ -183,6 +184,30 @@ void PInternalServiceImpl::trigger_profile_report( st.to_protobuf(result->mutable_status()); } +template +void PInternalServiceImpl::get_info( + google::protobuf::RpcController* controller, + const PProxyRequest* request, + PProxyResult* response, + google::protobuf::Closure* done) { + + brpc::ClosureGuard closure_guard(done); + if (request->has_kafka_meta_request()) { + std::vector partition_ids; + Status st = _exec_env->routine_load_task_executor()->get_kafka_partition_meta(request->kafka_meta_request(), &partition_ids); + if (st.ok()) { + PKafkaMetaProxyResult* kafka_result = response->mutable_kafka_meta_result(); + for (int32_t id : partition_ids) { + kafka_result->add_partition_ids(id); + } + } + st.to_protobuf(response->mutable_status()); + return; + } + Status::OK.to_protobuf(response->mutable_status()); +} + + template class PInternalServiceImpl; template class PInternalServiceImpl; diff --git a/be/src/service/internal_service.h b/be/src/service/internal_service.h index c6406ee5699641..957bb793e2fdf0 100644 --- a/be/src/service/internal_service.h +++ b/be/src/service/internal_service.h @@ -80,6 +80,12 @@ class PInternalServiceImpl : public T { PTriggerProfileReportResult* result, google::protobuf::Closure* done) override; + void get_info( + google::protobuf::RpcController* controller, + const PProxyRequest* request, + PProxyResult* response, + google::protobuf::Closure* done) override; + private: Status _exec_plan_fragment(brpc::Controller* cntl); private: diff --git a/be/test/runtime/CMakeLists.txt b/be/test/runtime/CMakeLists.txt index dbb3068853d548..97d562063e30ed 100644 --- a/be/test/runtime/CMakeLists.txt +++ b/be/test/runtime/CMakeLists.txt @@ -58,3 +58,4 @@ ADD_BE_TEST(snapshot_loader_test) ADD_BE_TEST(user_function_cache_test) ADD_BE_TEST(kafka_consumer_pipe_test) ADD_BE_TEST(routine_load_task_executor_test) +ADD_BE_TEST(small_file_mgr_test) diff --git a/be/test/runtime/small_file_mgr_test.cpp b/be/test/runtime/small_file_mgr_test.cpp new file mode 100644 index 00000000000000..508dd6a5ee1e65 --- /dev/null +++ b/be/test/runtime/small_file_mgr_test.cpp @@ -0,0 +1,134 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "runtime/small_file_mgr.h" + +#include + +#include +#include + +#include "common/logging.h" +#include "http/ev_http_server.h" +#include "http/http_channel.h" +#include "http/http_handler.h" +#include "http/http_request.h" + + +int main(int argc, char* argv[]); + +namespace doris { + +std::string g_download_path = "./be/test/runtime/test_data/small_file/downloaded/"; +std::string g_src_path = "./be/test/runtime/test_data/small_file/source/"; +std::string g_md5_12345 = "5dd39cab1c53c2c77cd352983f9641e1"; +std::string g_md5_67890 = "a06e26ae5511b0acea8273cf180ca7bf"; +int64_t g_file_12345 = 12345L; // ready to be downloaded file +int64_t g_file_67890 = 67890L; // already exist file +std::string src_file = "small_file.txt"; + +class SmallFileMgrTestHandler : public HttpHandler { +public: + void handle(HttpRequest* req) override { + auto it = req->query_params().find("file_id"); + if (it == req->query_params().end()) { + HttpChannel::send_error(req, INTERNAL_SERVER_ERROR); + return; + } + if (std::stol(it->second) != g_file_12345) { + HttpChannel::send_error(req, INTERNAL_SERVER_ERROR); + return; + } + + std::string file_path = g_src_path + "/small_file.txt"; + FILE* fp = fopen(file_path.c_str(), "r"); + if (fp == nullptr) { + HttpChannel::send_error(req, INTERNAL_SERVER_ERROR); + return; + } + std::string response; + char buf[1024]; + while (true) { + auto size = fread(buf, 1, 1024, fp); + response.append(buf, size); + if (size < 1024) { + break; + } + } + HttpChannel::send_reply(req, response); + fclose(fp); + } +}; + +static SmallFileMgrTestHandler s_test_handler = SmallFileMgrTestHandler(); +static EvHttpServer* s_server = nullptr; + +class SmallFileMgrTest: public testing::Test { +public: + SmallFileMgrTest() { } + virtual ~SmallFileMgrTest() { } + + static void SetUpTestCase() { + s_server = new EvHttpServer(29997); + s_server->register_handler(GET, "/api/get_small_file", &s_test_handler); + s_server->start(); + } + + static void TearDownTestCase() { + delete s_server; + std::stringstream ss; + ss << g_download_path << "/" << g_file_12345 << "." << g_md5_12345; + remove(ss.str().c_str()); + } + + void SetUp() override { + } +}; + +TEST_F(SmallFileMgrTest, test_get_file) { + SmallFileMgr mgr(nullptr, g_download_path); + Status st = mgr.init(); + ASSERT_TRUE(st.ok()); + + // get already exist file + std::string file_path; + st = mgr.get_file(g_file_67890, g_md5_67890, &file_path); + ASSERT_TRUE(st.ok()); + std::stringstream expected; + expected << g_download_path << "/" << g_file_67890 << "." << g_md5_67890; + ASSERT_EQ(expected.str(), file_path); + + // get ready to be downloaded file + std::string file_path_2; + st = mgr.get_file(g_file_12345, g_md5_12345, &file_path_2); + ASSERT_TRUE(st.ok()); + std::stringstream expected2; + expected2 << g_download_path << "/" << g_file_12345 << "." << g_md5_12345; + ASSERT_EQ(expected2.str(), file_path_2); + + // get wrong file + std::string file_path_3; + st = mgr.get_file(13579, "xxxx", &file_path_3); + ASSERT_TRUE(!st.ok()); +} + +} + +int main(int argc, char* argv[]) { + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/be/test/runtime/test_data/small_file/downloaded/67890.a06e26ae5511b0acea8273cf180ca7bf b/be/test/runtime/test_data/small_file/downloaded/67890.a06e26ae5511b0acea8273cf180ca7bf new file mode 100644 index 00000000000000..d91496f08b0d3c --- /dev/null +++ b/be/test/runtime/test_data/small_file/downloaded/67890.a06e26ae5511b0acea8273cf180ca7bf @@ -0,0 +1 @@ +already exist file diff --git a/be/test/runtime/test_data/small_file/source/small_file.txt b/be/test/runtime/test_data/small_file/source/small_file.txt new file mode 100644 index 00000000000000..9f4b6d8bfeaf44 --- /dev/null +++ b/be/test/runtime/test_data/small_file/source/small_file.txt @@ -0,0 +1 @@ +This is a test file diff --git a/docs/documentation/cn/administrator-guide/http-actions/fe_get_log_file.md b/docs/documentation/cn/administrator-guide/http-actions/fe_get_log_file.md new file mode 100644 index 00000000000000..64a876ebc3ca8e --- /dev/null +++ b/docs/documentation/cn/administrator-guide/http-actions/fe_get_log_file.md @@ -0,0 +1,52 @@ +# get\_log\_file + +用户可以通过该 HTTP 接口获取 FE 的日志文件。 + +## 日志类型 + +支持获取以下类型的 FE 日志: + +1. fe.audit.log(审计日志) + + 审计日志记录了对应 FE 节点的所有请求语句已经请求的信息。审计日志的文件命名规则如下: + + ``` + fe.audit.log # 当前的最新日志 + fe.audit.log.20190603.1 # 对应日期的审计日志,当对应日期的日志大小超过 1GB 后,会生成序号后缀。序号越小的日志,内容越新。 + fe.audit.log.20190603.2 + fe.audit.log.20190602.1 + ... + ``` + +## 接口示例 + +1. 获取对应类型的日志文件列表 + + 示例: + + `curl -X HEAD -uuser:passwd http://fe_host:http_port/api/get_log_file?type=fe.audit.log` + + 返回结果: + + ``` + HTTP/1.1 200 OK + file_infos: {"fe.audit.log":24759,"fe.audit.log.20190528.1":132934} + content-type: text/html + connection: keep-alive + ``` + + 在返回的 header 中,`file_infos` 字段以 json 格式展示文件列表以及对应文件大小(单位字节) + +2. 下载日志文件 + + 示例: + + `curl -X GET -uuser:passwd http://fe_host:http_port/api/get_log_file?type=fe.audit.log&file=fe.audit.log.20190528.1` + + 返回结果: + + 以文件的形式下载指定的文件。 + +## 接口说明 + +该接口需要 admin 权限。 diff --git a/docs/documentation/cn/administrator-guide/load-data/routine-load-manual.md b/docs/documentation/cn/administrator-guide/load-data/routine-load-manual.md index c7f872c399eb62..98b440ac01ada6 100644 --- a/docs/documentation/cn/administrator-guide/load-data/routine-load-manual.md +++ b/docs/documentation/cn/administrator-guide/load-data/routine-load-manual.md @@ -57,7 +57,7 @@ FE 中的 JobScheduler 根据汇报结果,继续生成后续新的 Task,或 ### 使用限制 -1. 仅支持无认证的 Kafka 访问。 +1. 支持无认证的 Kafka 访问,以及通过 SSL 方式认证的 Kafka 集群。 2. 支持的消息格式为 csv 文本格式。每一个 message 为一行,且行尾**不包含**换行符。 3. 仅支持 Kafka 0.10.0.0(含) 以上版本。 @@ -126,6 +126,41 @@ FE 中的 JobScheduler 根据汇报结果,继续生成后续新的 Task,或 `data_source_properties` 中可以指定消费具体的 Kakfa partition。如果不指定,则默认消费所订阅的 topic 的所有 partition。 注意,当显式的指定了 partition,则导入作业不会再动态的检测 Kafka partition 的变化。如果没有指定,则会根据 kafka partition 的变化,动态调整需要消费的 partition。 + +#### 访问 SSL 认证的 Kafka 集群 + +访问 SSL 认证的 Kafka 集群需要用户提供用于认证 Kafka Broker 公钥的证书文件(ca.pem)。如果 Kafka 集群同时开启了客户端认证,则还需提供客户端的公钥(client.pem)、密钥文件(client.key),以及密钥密码。这里所需的文件需要先通过 `CREAE FILE` 命令上传到 Doris 中,**并且 catalog 名称为 `kafka`**。`CREATE FILE` 命令的具体帮助可以参见 `HELP CREATE FILE;`。这里给出示例: + +1. 上传文件 + + ``` + CREATE FILE "ca.pem" PROPERTIES("url" = "https://example_url/kafka-key/ca.pem", "catalog" = "kafka"); + CREATE FILE "client.key" PROPERTIES("url" = "https://example_urlkafka-key/client.key", "catalog" = "kafka"); + CREATE FILE "client.pem" PROPERTIES("url" = "https://example_url/kafka-key/client.pem", "catalog" = "kafka"); + ``` + +2. 创建例行导入作业 + + ``` + CREATE ROUTINE LOAD db1.job1 on tbl1 + PROPERTIES + ( + "desired_concurrent_number"="1" + ) + FROM KAFKA + ( + "kafka_broker_list"= "broker1:9091,broker2:9091", + "kafka_topic" = "my_topic", + "property.security.protocol" = "ssl", + "property.ssl.ca.location" = "FILE:ca.pem", + "property.ssl.certificate.location" = "FILE:client.pem", + "property.ssl.key.location" = "FILE:client.key", + "property.ssl.key.password" = "abcdefg" + ); + ``` + +> Doris 通过 Kafka 的 C++ API `librdkafka` 来访问 Kafka 集群。`librdkafka` 所支持的参数可以参阅[这里](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md) + ### 查看导入作业状态 diff --git a/docs/documentation/cn/administrator-guide/small_file_mgr.md b/docs/documentation/cn/administrator-guide/small_file_mgr.md new file mode 100644 index 00000000000000..ed1094ade70937 --- /dev/null +++ b/docs/documentation/cn/administrator-guide/small_file_mgr.md @@ -0,0 +1,78 @@ +# 文件管理器 + +Doris 中的一些功能需要使用一些用户自定义的文件。比如用于访问外部数据源的公钥、密钥文件、证书文件等等。文件管理器提供这样一个功能,能够让用户预先上传这些文件并保存在 Doris 系统中,然后可以在其他命令中引用或访问。 + +## 名词解释 + +* FE:Frontend,Doris 的前端节点。负责元数据管理和请求接入。 +* BE:Backend,Doris 的后端节点。负责查询执行和数据存储。 +* BDBJE:Oracle Berkeley DB Java Edition。FE 中用于持久化元数据的分布式嵌入式数据库。 +* SmallFileMgr:文件管理器。负责创建并维护用户的文件。 + +## 基本概念 + +文件是指用户创建并保存在 Doris 中的文件。 + +一个文件由 `数据库名称(database)`、`分类(catalog)` 和 `文件名(file_name)` 共同定位。同时每个文件也有一个全局唯一的 id(file_id),作为系统内的标识。 + +文件的创建和删除只能由拥有 `admin` 权限的用户进行操作。一个文件隶属于一个数据库。对某一数据库拥有访问权限(查询、导入、修改等等)的用户都可以使用该数据库下创建的文件。 + +## 具体操作 + +文件管理主要有三个命令:`CREATE FILE`,`SHOW FILE` 和 `DROP FILE`,分别为创建、查看和删除文件。这三个命令的具体语法可以通过连接到 Doris 后,执行 `HELP cmd;` 的方式查看帮助。 + +1. CREATE FILE + + 在创建文件的命令中,用户必须提供以下信息: + + * file_name:文件名。用户自定义,在一个 catalog 内唯一即可。 + * catalog:文件所属分类。用户自定义,在一个 database 内唯一即可。 + + > Doris 也有一些特殊的分类名称供特定的命令使用。 + + > 1. kafka + + > 当在例行导入命令中指定数据源为 Kafka,并且需要引用到文件时,Doris 会默认从 catalog 名为 "kafka" 的分类中查找文件。 + + * url:文件的下载地址。目前仅支持无认证的 http 下载地址。该下载地址仅用于在执行创建文件命令时,从这个地址下载文件。当文件成功创建并保存在 Doris 中后,该地址将不再被使用。 + * md5:可选项。文件的 MD5 值。如果用户提供该值,将在文件下载后进行 MD5 值的校验。校验失败则文件创建失败。 + + 文件创建成功后,文件相关的信息将持久化在 Doris 中。用户可以通过 `SHOW FILE` 命令查看已经创建成功的文件。 + +2. SHOW FILE + + 该命令可以查看已经创建成功的文件。具体操作见:`HELP SHOW FILE;` + +3. DROP FILE + + 该命令可以删除一个已经创建的文件。具体操作见:`HELP DROP FILE;` + +## 实现细节 + +### 创建和删除文件 + +当用户执行 `CREATE FILE` 命令后,FE 会从给定的 URL 下载文件。并将文件的内容以 Base64 编码的形式直接保存在 FE 的内存中。同时会将文件内容以及文件相关的元信息持久化在 BDBJE 中。所有被创建的文件,其元信息和文件内容都会常驻于 FE 的内存中。如果 FE 宕机重启,也会从 BDBJE 中加载元信息和文件内容到内存中。当文件被删除时,会直接从 FE 内存中删除相关信息,同时也从 BDBJE 中删除持久化的信息。 + +### 文件的使用 + +如果是 FE 端需要使用创建的文件,则 SmallFileMgr 会直接将 FE 内存中的数据保存为本地文件,存储在指定的目录中,并返回本地的文件路径供使用。 + +如果是 BE 端需要使用创建的文件,BE 会通过 FE 的 http 接口 `/api/get_small_file` 将文件内容下载到 BE 上指定的目录中,供使用。同时,BE 也会在内存中记录当前已经下载过的文件的信息。当 BE 请求一个文件时,会先查看本地文件是否存在并校验。如果校验通过,则直接返回本地文件路径。如果校验失败,则会删除本地文件,重新从 FE 下载。当 BE 重启时,会预先加载本地的文件到内存中。 + +## 使用限制 + +因为文件元信息和内容都存储于 FE 的内存中。所以默认仅支持上传大小在 1MB 以内的文件。并且总文件数量限制为 100 个。可以通过下一小节介绍的配置项进行修改。 + +## 相关配置 + +1. FE 配置 + + * `small_file_dir`:用于存放上传文件的路径,默认为 FE 运行目录的 `small_files/` 目录下。 + * `max_small_file_size_bytes`:单个文件大小限制,单位为字节。默认为 1MB。大于该配置的文件创建将会被拒绝。 + * `max_small_file_number`:一个 Doris 集群支持的总文件数量。默认为 100。当创建的文件数超过这个值后,后续的创建将会被拒绝。 + + > 如果需要上传更多文件或提高单个文件的大小限制,可以通过 `ADMIN SET CONFIG` 命令修改 `max_small_file_size_bytes` 和 `max_small_file_number` 参数。但文件数量和大小的增加,会导致 FE 内存使用量的增加。 + +2. BE 配置 + + * `small_file_dir`:用于存放从 FE 下载的文件的路径,默认为 BE 运行目录的 `lib/small_files/` 目录下。 diff --git a/docs/help/Contents/Administration/small_files.md b/docs/help/Contents/Administration/small_files.md new file mode 100644 index 00000000000000..631c125d75769a --- /dev/null +++ b/docs/help/Contents/Administration/small_files.md @@ -0,0 +1,103 @@ +# CREATE FILE +## description + + 该语句用于创建并上传一个文件到 Doris 集群。 + 该功能通常用于管理一些其他命令中需要使用到的文件,如证书、公钥私钥等等。 + + 该命令只用 amdin 权限用户可以执行。 + 某个文件都归属与某一个的 database。对 database 拥有访问权限的用户都可以使用该文件。 + + 单个文件大小限制为 1MB。 + 一个 Doris 集群最多上传 100 个文件。 + + 语法: + + CREATE FILE "file_name" [IN database] + [properties] + + 说明: + file_name: 自定义文件名。 + database: 文件归属于某一个 db,如果没有指定,则使用当前 session 的 db。 + properties 支持以下参数: + + url: 必须。指定一个文件的下载路径。当前仅支持无认证的 http 下载路径。命令之行成功后,文件将被保存在 doris 中,该 url 将不再需要。 + catalog: 必须。对文件的分类名,可以自定义。但在某些命令中,会查找指定 catalog 中的文件。比如例行导入中的,数据源为 kafka 时,会查找 catalog 名为 kafka 下的文件。 + md5: 可选。文件的 md5。如果指定,会在下载文件后进行校验。 + +## example + + 1. 创建文件 ca.pem ,分类为 kafka + + CREATE FILE "ca.pem" + PROPERTIES + ( + "url" = "https://test.bj.bcebos.com/kafka-key/ca.pem", + "catalog" = "kafka" + ); + + 2. 创建文件 client.key,分类为 my_catalog + + CREATE FILE "client.key" + IN my_database + PROPERTIES + ( + "url" = "https://test.bj.bcebos.com/kafka-key/client.key", + "catalog" = "my_catalog", + "md5" = "b5bb901bf10f99205b39a46ac3557dd9" + ); + +## keyword + CREATE,FILE + +# SHOW FILE +## description + + 该语句用于展示一个 database 内创建的文件 + + 语法: + + SHOW FILE [FROM database]; + + 说明: + + FileId: 文件ID,全局唯一 + DbName: 所属数据库名称 + Catalog: 自定义分类 + FileName: 文件名 + FileSize: 文件大小,单位字节 + MD5: 文件的 MD5 + +## example + + 1. 查看数据库 my_database 中已上传的文件 + + SHOW FILE FROM my_database; + +## keyword + SHOW,FILE + +# DROP FILE +## description + + 该语句用于删除一个已上传的文件。 + + 语法: + + DROP FILE "file_name" [FROM database] + [properties] + + 说明: + file_name: 文件名。 + database: 文件归属的某一个 db,如果没有指定,则使用当前 session 的 db。 + properties 支持以下参数: + + catalog: 必须。文件所属分类。 + +## example + + 1. 删除文件 ca.pem + + DROP FILE "ca.pem" properties("catalog" = "kafka"); + +## keyword + DROP,FILE diff --git a/docs/help/Contents/Data Definition/ddl_stmt.md b/docs/help/Contents/Data Definition/ddl_stmt.md index 2bd4d804a1c344..02b1f0ac2b13e4 100644 --- a/docs/help/Contents/Data Definition/ddl_stmt.md +++ b/docs/help/Contents/Data Definition/ddl_stmt.md @@ -595,9 +595,9 @@ ORDER BY (k3,k1,k2,v2,v1) FROM example_rollup_index; 11. 修改表的 bloom filter 列 - ALTER TABLE example_db.my_table set ("bloom_filter_columns"="k1,k2,k3"); + ALTER TABLE example_db.my_table SET ("bloom_filter_columns"="k1,k2,k3"); - 也可以合并到上面的 schema change 操作中 + 也可以合并到上面的 schema change 操作中(注意多子句的语法有少许区别) ALTER TABLE example_db.my_table DROP COLUMN col2 PROPERTIES ("bloom_filter_columns"="k1,k2,k3"); @@ -889,8 +889,8 @@ FROM `example_repo` ON ( - `backup_tbl` PARTITION (`p1`, `p2`) AS `backup_tbl2`, - `backup_tbl2` + `backup_tbl` PARTITION (`p1`, `p2`), + `backup_tbl2` AS `new_tbl` ) PROPERTIES ( diff --git a/docs/help/Contents/Data Manipulation/routine_load.md b/docs/help/Contents/Data Manipulation/routine_load.md index 022d57f75ab6f0..9dcd7d687cc651 100644 --- a/docs/help/Contents/Data Manipulation/routine_load.md +++ b/docs/help/Contents/Data Manipulation/routine_load.md @@ -2,7 +2,7 @@ ## description 例行导入(Routine Load)功能,支持用户提交一个常驻的导入任务,通过不断的从指定的数据源读取数据,将数据导入到 Doris 中。 - 目前仅支持通过无认证的方式,从 Kakfa 导入文本格式(CSV)的数据。 + 目前仅支持通过无认证或者 SSL 认证方式,从 Kakfa 导入文本格式(CSV)的数据。 语法: @@ -159,12 +159,36 @@ "kafka_offsets" = "101,0,OFFSET_BEGINNING,OFFSET_END" 4. property - 指定自定义kafka参数 - 功能等同于kafka shell中 "--property" 参数 + + 指定自定义kafka参数。 + 功能等同于kafka shell中 "--property" 参数。 + 当参数的 value 为一个文件时,需要在 value 前加上关键词:"FILE:"。 + 关于如何创建文件,请参阅 "HELP CREATE FILE;" + 更多支持的自定义参数,请参阅 librdkafka 的官方 CONFIGURATION 文档中,client 端的配置项。 示例: - "property.client.id" = "12345" + "property.client.id" = "12345", + "property.ssl.ca.location" = "FILE:ca.pem" + + 使用 SSL 连接 Kafka 时,需要指定以下参数: + + "property.security.protocol" = "ssl", + "property.ssl.ca.location" = "FILE:ca.pem", + "property.ssl.certificate.location" = "FILE:client.pem", + "property.ssl.key.location" = "FILE:client.key", + "property.ssl.key.password" = "abcdefg" + + 其中: + "property.security.protocol" 和 "property.ssl.ca.location" 为必须,用于指明连接方式为 SSL,以及 CA 证书的位置。 + + 如果 Kafka server 端开启了 client 认证,则还需设置: + + "property.ssl.certificate.location" + "property.ssl.key.location" + "property.ssl.key.password" + 分别用于指定 client 的 public key,private key 以及 private key 的密码。 + 7. 导入数据格式样例 @@ -196,6 +220,30 @@ "kafka_offsets" = "101,0,0,200" ); + 2. 通过 SSL 认证方式,从 Kafka 集群导入数据。同时设置 client.id 参数。 + + CREATE ROUTINE LOAD example_db.test1 ON example_tbl + COLUMNS(k1, k2, k3, v1, v2, v3 = k1 * 100), + WHERE k1 > 100 and k2 like "%doris%" + PROPERTIES + ( + "desired_concurrent_number"="3", + "max_batch_interval" = "20", + "max_batch_rows" = "300000", + "max_batch_size" = "209715200" + ) + FROM KAFKA + ( + "kafka_broker_list" = "broker1:9092,broker2:9092,broker3:9092", + "kafka_topic" = "my_topic", + "property.security.protocol" = "ssl", + "property.ssl.ca.location" = "FILE:ca.pem", + "property.ssl.certificate.location" = "FILE:client.pem", + "property.ssl.key.location" = "FILE:client.key", + "property.ssl.key.password" = "abcdefg", + "property.client.id" = "my_client_id" + ); + ## keyword CREATE,ROUTINE,LOAD diff --git a/fe/pom.xml b/fe/pom.xml index 8d94da3e76e72b..ab67ab1584bf1c 100644 --- a/fe/pom.xml +++ b/fe/pom.xml @@ -36,6 +36,7 @@ under the License. UTF-8 1.8 1.8 + 2.2.11 @@ -242,7 +243,8 @@ under the License. com.baidu jprotobuf - 1.11.0 + ${jprotobuf.version} + jar-with-dependencies @@ -256,7 +258,7 @@ under the License. com.baidu jprotobuf-rpc-core - 3.5.17 + 3.5.21 @@ -358,7 +360,7 @@ under the License. com.google.protobuf protobuf-java - 2.5.0 + 3.5.1 @@ -576,6 +578,7 @@ under the License. + org.codehaus.mojo exec-maven-plugin @@ -596,6 +599,24 @@ under the License. ${skip.plugin} + + gen_proto + generate-sources + + + exec + + + java + + -jar + ${settings.localRepository}/com/baidu/jprotobuf/${jprotobuf.version}/jprotobuf-${jprotobuf.version}-jar-with-dependencies.jar + --java_out=${palo.home}/gensrc/build/java/ + ${palo.home}/gensrc/proto/internal_service.proto + + ${skip.plugin} + + diff --git a/fe/src/main/cup/sql_parser.cup b/fe/src/main/cup/sql_parser.cup index 77c905e9ba3801..bd7824e1284366 100644 --- a/fe/src/main/cup/sql_parser.cup +++ b/fe/src/main/cup/sql_parser.cup @@ -198,10 +198,10 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_A KW_DATA, KW_DATABASE, KW_DATABASES, KW_DATE, KW_DATETIME, KW_DECIMAL, KW_DECOMMISSION, KW_DEFAULT, KW_DESC, KW_DESCRIBE, KW_DELETE, KW_DISTINCT, KW_DISTINCTPC, KW_DISTINCTPCSA, KW_DISTRIBUTED, KW_DISTRIBUTION, KW_BUCKETS, KW_DIV, KW_DOUBLE, KW_DROP, KW_DROPP, KW_DUPLICATE, KW_ELSE, KW_END, KW_ENGINE, KW_ENGINES, KW_ENTER, KW_ERRORS, KW_EVENTS, KW_EXISTS, KW_EXPORT, KW_EXTERNAL, KW_EXTRACT, - KW_FALSE, KW_FOLLOWER, KW_FOLLOWING, KW_FREE, KW_FROM, KW_FIRST, KW_FLOAT, KW_FOR, KW_FORMAT, KW_FRONTEND, KW_FRONTENDS, KW_FULL, KW_FUNCTION, + KW_FALSE, KW_FOLLOWER, KW_FOLLOWING, KW_FREE, KW_FROM, KW_FILE, KW_FIRST, KW_FLOAT, KW_FOR, KW_FORMAT, KW_FRONTEND, KW_FRONTENDS, KW_FULL, KW_FUNCTION, KW_GLOBAL, KW_GRANT, KW_GRANTS, KW_GROUP, KW_HASH, KW_HAVING, KW_HELP,KW_HLL, KW_HLL_UNION, KW_HUB, - KW_IDENTIFIED, KW_IF, KW_IN, KW_INDEX, KW_INDEXES, KW_INFILE, + KW_IDENTIFIED, KW_IF, KW_IN, KW_INDEX, KW_INDEXES, KW_INFILE, KW_INNER, KW_INSERT, KW_INT, KW_INTERMEDIATE, KW_INTERVAL, KW_INTO, KW_IS, KW_ISNULL, KW_ISOLATION, KW_JOIN, KW_KEY, KW_KILL, @@ -905,6 +905,10 @@ create_stmt ::= {: RESULT = new CreateRoleStmt(role); :} + | KW_CREATE KW_FILE STRING_LITERAL:fileName opt_db:db KW_PROPERTIES LPAREN key_value_map:properties RPAREN + {: + RESULT = new CreateFileStmt(fileName, db, properties); + :} ; opt_aggregate ::= @@ -1345,6 +1349,10 @@ drop_stmt ::= {: RESULT = new DropRoleStmt(role); :} + | KW_DROP KW_FILE STRING_LITERAL:fileName opt_db:dbName KW_PROPERTIES LPAREN key_value_map:properties RPAREN + {: + RESULT = new DropFileStmt(fileName, dbName, properties); + :} ; // Recover statement @@ -1906,6 +1914,10 @@ show_param ::= {: RESULT = new ShowFunctionStmt(dbName); :} + | KW_FILE opt_db:dbName + {: + RESULT = new ShowSmallFilesStmt(dbName); + :} ; keys_or_index ::= @@ -3903,6 +3915,8 @@ keyword ::= {: RESULT = id; :} | KW_BUCKETS:id {: RESULT = id; :} + | KW_FILE:id + {: RESULT = id; :} | KW_FIRST:id {: RESULT = id; :} | KW_FORMAT:id diff --git a/fe/src/main/java/org/apache/doris/analysis/CreateFileStmt.java b/fe/src/main/java/org/apache/doris/analysis/CreateFileStmt.java new file mode 100644 index 00000000000000..d17dffefe52d2e --- /dev/null +++ b/fe/src/main/java/org/apache/doris/analysis/CreateFileStmt.java @@ -0,0 +1,170 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.analysis; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.PrintableMap; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableSet; + +import java.util.Map; +import java.util.Optional; + +public class CreateFileStmt extends DdlStmt { + public static final String PROP_CATALOG = "catalog"; + public static final String PROP_URL = "url"; + public static final String PROP_MD5 = "md5"; + public static final String PROP_SAVE_CONTENT = "save_content"; + + private static final ImmutableSet PROPERTIES_SET = new ImmutableSet.Builder() + .add(PROP_CATALOG).add(PROP_URL).add(PROP_MD5).build(); + + private final String fileName; + private String dbName; + private final Map properties; + + // needed item set after analyzed + private String catalogName; + private String downloadUrl; + private String checksum; + // if saveContent is true, the file content will be saved in FE memory, so the file size will be limited + // by the configuration. If it is false, only URL will be saved. + private boolean saveContent = true; + + public CreateFileStmt(String fileName, String dbName, Map properties) { + this.fileName = fileName; + this.dbName = dbName; + this.properties = properties; + } + + public String getFileName() { + return fileName; + } + + public String getDbName() { + return dbName; + } + + public String getCatalogName() { + return catalogName; + } + + public String getDownloadUrl() { + return downloadUrl; + } + + public String getChecksum() { + return checksum; + } + + public boolean isSaveContent() { + return saveContent; + } + + @Override + public void analyze(Analyzer analyzer) throws UserException { + super.analyze(analyzer); + + // check operation privilege + if (!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + if (dbName == null) { + dbName = analyzer.getDefaultDb(); + } else { + if (Strings.isNullOrEmpty(analyzer.getClusterName())) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_CLUSTER_NAME_NULL); + } + dbName = ClusterNamespace.getFullName(analyzer.getClusterName(), dbName); + } + + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + + if (Strings.isNullOrEmpty(fileName)) { + throw new AnalysisException("File name is not specified"); + } + + analyzeProperties(); + } + + private void analyzeProperties() throws AnalysisException { + Optional optional = properties.keySet().stream().filter( + entity -> !PROPERTIES_SET.contains(entity)).findFirst(); + if (optional.isPresent()) { + throw new AnalysisException(optional.get() + " is invalid property"); + } + + catalogName = properties.get(PROP_CATALOG); + if (Strings.isNullOrEmpty(catalogName)) { + throw new AnalysisException("catalog name is missing"); + } + + downloadUrl = properties.get(PROP_URL); + if (Strings.isNullOrEmpty(downloadUrl)) { + throw new AnalysisException("download url is missing"); + } + + if (properties.containsKey(PROP_MD5)) { + checksum = properties.get(PROP_MD5); + } + + if (properties.containsKey(PROP_SAVE_CONTENT)) { + throw new AnalysisException("'save_content' property is not supported yet"); + /* + String val = properties.get(PROP_SAVE_CONTENT); + if (val.equalsIgnoreCase("true")) { + saveContent = true; + } else if (val.equalsIgnoreCase("false")) { + saveContent = false; + } else { + throw new AnalysisException("Invalid 'save_content' property: " + val); + } + */ + } + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("CREATE FILE \"").append(fileName).append("\""); + if (dbName != null) { + sb.append(" IN ").append(dbName); + } + + sb.append(" PROPERTIES("); + PrintableMap map = new PrintableMap<>(properties, ",", true, false); + sb.append(map.toString()); + return sb.toString(); + } + + @Override + public RedirectStatus getRedirectStatus() { + return RedirectStatus.FORWARD_WITH_SYNC; + } +} diff --git a/fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java b/fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java index 0f7a65736fec7b..0ec39039ab14af 100644 --- a/fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/CreateRoutineLoadStmt.java @@ -211,6 +211,7 @@ public List> getKafkaPartitionOffsets() { public Map getCustomKafkaProperties() { return customKafkaProperties; } + @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); @@ -278,7 +279,7 @@ public void checkLoadProperties(Analyzer analyzer) throws UserException { } private void checkJobProperties() throws AnalysisException { - Optional optional = jobProperties.keySet().parallelStream().filter( + Optional optional = jobProperties.keySet().stream().filter( entity -> !PROPERTIES_SET.contains(entity)).findFirst(); if (optional.isPresent()) { throw new AnalysisException(optional.get() + " is invalid property"); @@ -322,7 +323,7 @@ private void checkDataSourceProperties() throws AnalysisException { } private void checkKafkaProperties() throws AnalysisException { - Optional optional = dataSourceProperties.keySet().parallelStream() + Optional optional = dataSourceProperties.keySet().stream() .filter(entity -> !KAFKA_PROPERTIES_SET.contains(entity)) .filter(entity -> !entity.startsWith("property.")).findFirst(); if (optional.isPresent()) { @@ -400,6 +401,7 @@ private void checkKafkaProperties() throws AnalysisException { } } } + // check custom kafka property for (Map.Entry dataSourceProperty : dataSourceProperties.entrySet()) { if (dataSourceProperty.getKey().startsWith("property.")) { diff --git a/fe/src/main/java/org/apache/doris/analysis/DropFileStmt.java b/fe/src/main/java/org/apache/doris/analysis/DropFileStmt.java new file mode 100644 index 00000000000000..2aaf29412e4ec3 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/analysis/DropFileStmt.java @@ -0,0 +1,123 @@ +// 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.analysis; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.common.util.PrintableMap; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Strings; + +import java.util.Map; +import java.util.Optional; + +/* + * Author: Chenmingyu + * Date: May 29, 2019 + */ + +public class DropFileStmt extends DdlStmt { + public static final String PROP_CATALOG = "catalog"; + + private String fileName; + private String dbName; + private Map properties; + + private String catalogName; + + public DropFileStmt(String fileName, String dbName, Map properties) { + this.fileName = fileName; + this.dbName = dbName; + this.properties = properties; + } + + public String getFileName() { + return fileName; + } + + public String getDbName() { + return dbName; + } + + public String getCatalogName() { + return catalogName; + } + + @Override + public void analyze(Analyzer analyzer) throws AnalysisException, UserException { + super.analyze(analyzer); + + // check operation privilege + if (!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN"); + } + + if (dbName == null) { + dbName = analyzer.getDefaultDb(); + } else { + if (Strings.isNullOrEmpty(analyzer.getClusterName())) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_CLUSTER_NAME_NULL); + } + dbName = ClusterNamespace.getFullName(analyzer.getClusterName(), dbName); + } + + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + + if (Strings.isNullOrEmpty(fileName)) { + throw new AnalysisException("File name is not specified"); + } + + Optional optional = properties.keySet().stream().filter( + entity -> !PROP_CATALOG.equals(entity)).findFirst(); + if (optional.isPresent()) { + throw new AnalysisException(optional.get() + " is invalid property"); + } + + catalogName = properties.get(PROP_CATALOG); + if (Strings.isNullOrEmpty(catalogName)) { + throw new AnalysisException("catalog name is missing"); + } + } + + @Override + public String toSql() { + StringBuilder sb = new StringBuilder(); + sb.append("DROP FILE \"").append(fileName).append("\""); + if (dbName != null) { + sb.append(" FROM ").append(dbName); + } + + sb.append(" PROPERTIES("); + PrintableMap map = new PrintableMap<>(properties, ",", true, false); + sb.append(map.toString()); + return sb.toString(); + } + + @Override + public RedirectStatus getRedirectStatus() { + return RedirectStatus.FORWARD_WITH_SYNC; + } +} diff --git a/fe/src/main/java/org/apache/doris/analysis/LabelName.java b/fe/src/main/java/org/apache/doris/analysis/LabelName.java index 8bbcc450ddc370..4e3cd8d70848dd 100644 --- a/fe/src/main/java/org/apache/doris/analysis/LabelName.java +++ b/fe/src/main/java/org/apache/doris/analysis/LabelName.java @@ -20,6 +20,8 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.FeMetaVersion; import org.apache.doris.common.FeNameFormat; import org.apache.doris.common.io.Text; @@ -59,7 +61,7 @@ public String getLabelName() { public void analyze(Analyzer analyzer) throws AnalysisException { if (Strings.isNullOrEmpty(dbName)) { if (Strings.isNullOrEmpty(analyzer.getDefaultDb())) { - throw new AnalysisException("No database when load data."); + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); } dbName = analyzer.getDefaultDb(); } diff --git a/fe/src/main/java/org/apache/doris/analysis/ShowRoutineLoadStmt.java b/fe/src/main/java/org/apache/doris/analysis/ShowRoutineLoadStmt.java index 3a29417c847424..2186e5db7af798 100644 --- a/fe/src/main/java/org/apache/doris/analysis/ShowRoutineLoadStmt.java +++ b/fe/src/main/java/org/apache/doris/analysis/ShowRoutineLoadStmt.java @@ -22,6 +22,8 @@ import org.apache.doris.catalog.ScalarType; import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; import org.apache.doris.qe.ShowResultSetMetaData; @@ -117,8 +119,7 @@ private void checkLabelName(Analyzer analyzer) throws AnalysisException { if (Strings.isNullOrEmpty(dbName)) { dbFullName = analyzer.getContext().getDatabase(); if (Strings.isNullOrEmpty(dbFullName)) { - throw new AnalysisException("please choose a database firstly " - + "such as use db, show routine load db.name etc."); + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); } } else { dbFullName = ClusterNamespace.getFullName(getClusterName(), dbName); diff --git a/fe/src/main/java/org/apache/doris/analysis/ShowSmallFilesStmt.java b/fe/src/main/java/org/apache/doris/analysis/ShowSmallFilesStmt.java new file mode 100644 index 00000000000000..3d6ec14faf55f8 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/analysis/ShowSmallFilesStmt.java @@ -0,0 +1,75 @@ +// 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.analysis; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.cluster.ClusterNamespace; +import org.apache.doris.common.ErrorCode; +import org.apache.doris.common.ErrorReport; +import org.apache.doris.common.UserException; +import org.apache.doris.mysql.privilege.PrivPredicate; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.ShowResultSetMetaData; + +import com.google.common.base.Strings; + +public class ShowSmallFilesStmt extends ShowStmt { + private static final ShowResultSetMetaData META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("Id", ScalarType.createVarchar(32))) + .addColumn(new Column("DbName", ScalarType.createVarchar(256))) + .addColumn(new Column("Catalog", ScalarType.createVarchar(32))) + .addColumn(new Column("FileName", ScalarType.createVarchar(16))) + .addColumn(new Column("FileSize", ScalarType.createVarchar(16))) + .addColumn(new Column("IsContent", ScalarType.createVarchar(16))) + .addColumn(new Column("MD5", ScalarType.createVarchar(16))) + .build(); + + private String dbName; + + public ShowSmallFilesStmt(String dbName) { + this.dbName = dbName; + } + + public String getDbName() { return dbName; } + + @Override + public void analyze(Analyzer analyzer) throws UserException { + super.analyze(analyzer); + if (Strings.isNullOrEmpty(dbName)) { + dbName = analyzer.getDefaultDb(); + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR); + } + } else { + dbName = ClusterNamespace.getFullName(getClusterName(), dbName); + } + + if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), dbName, PrivPredicate.SHOW)) { + ErrorReport.reportAnalysisException( + ErrorCode.ERR_DB_ACCESS_DENIED, ConnectContext.get().getQualifiedUser(), dbName); + } + } + + @Override + public ShowResultSetMetaData getMetaData() { + return META_DATA; + } +} diff --git a/fe/src/main/java/org/apache/doris/backup/Repository.java b/fe/src/main/java/org/apache/doris/backup/Repository.java index 03397916056bf4..6000d2020c052b 100644 --- a/fe/src/main/java/org/apache/doris/backup/Repository.java +++ b/fe/src/main/java/org/apache/doris/backup/Repository.java @@ -346,6 +346,7 @@ public Status getSnapshotMetaFile(String label, List backupMetas, in BackupMeta backupMeta = BackupMeta.fromFile(localMetaFile.getAbsolutePath(), metaVersion); backupMetas.add(backupMeta); } catch (IOException e) { + LOG.warn("failed to read backup meta from file", e); return new Status(ErrCode.COMMON_ERROR, "Failed create backup meta from file: " + localMetaFile.getAbsolutePath() + ", msg: " + e.getMessage()); } finally { diff --git a/fe/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/src/main/java/org/apache/doris/catalog/Catalog.java index 774016e88ff06c..f26e0da59303fb 100644 --- a/fe/src/main/java/org/apache/doris/catalog/Catalog.java +++ b/fe/src/main/java/org/apache/doris/catalog/Catalog.java @@ -108,6 +108,7 @@ import org.apache.doris.common.util.PrintableMap; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.QueryableReentrantLock; +import org.apache.doris.common.util.SmallFileMgr; import org.apache.doris.common.util.Util; import org.apache.doris.consistency.ConsistencyChecker; import org.apache.doris.deploy.DeployManager; @@ -363,6 +364,8 @@ public class Catalog { private RoutineLoadTaskScheduler routineLoadTaskScheduler; + private SmallFileMgr smallFileMgr; + public List getFrontends(FrontendNodeType nodeType) { if (nodeType == null) { // get all @@ -486,6 +489,8 @@ private Catalog() { this.loadManager = new LoadManager(loadJobScheduler); this.routineLoadScheduler = new RoutineLoadScheduler(routineLoadManager); this.routineLoadTaskScheduler = new RoutineLoadTaskScheduler(routineLoadManager); + + this.smallFileMgr = new SmallFileMgr(); } public static void destroyCheckpoint() { @@ -1070,8 +1075,7 @@ private void transferToMaster() throws IOException { LOG.info("checkpointer thread started. thread id is {}", checkpointThreadId); // heartbeat mgr - heartbeatMgr.setMaster( - FrontendOptions.getLocalHostAddress(), Config.rpc_port, clusterId, token, epoch); + heartbeatMgr.setMaster(clusterId, token, epoch); heartbeatMgr.start(); pullLoadJobMgr.start(); @@ -1328,6 +1332,7 @@ public void loadImage(String imageDir) throws IOException, DdlException { checksum = loadColocateTableIndex(dis, checksum); checksum = loadRoutineLoadJobs(dis, checksum); checksum = loadLoadJobsV2(dis, checksum); + checksum = loadSmallFiles(dis, checksum); long remoteChecksum = dis.readLong(); Preconditions.checkState(remoteChecksum == checksum, remoteChecksum + " vs. " + checksum); @@ -1757,6 +1762,13 @@ public long loadLoadJobsV2(DataInputStream in, long checksum) throws IOException return checksum; } + public long loadSmallFiles(DataInputStream in, long checksum) throws IOException { + if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_52) { + smallFileMgr.readFields(in); + } + return checksum; + } + // Only called by checkpoint thread public void saveImage() throws IOException { // Write image.ckpt @@ -1803,6 +1815,7 @@ public void saveImage(File curFile, long replayedJournalId) throws IOException { checksum = saveColocateTableIndex(dos, checksum); checksum = saveRoutineLoadJobs(dos, checksum); checksum = saveLoadJobsV2(dos, checksum); + checksum = saveSmallFiles(dos, checksum); dos.writeLong(checksum); } finally { dos.close(); @@ -2061,6 +2074,11 @@ public long saveLoadJobsV2(DataOutputStream out, long checksum) throws IOExcepti return checksum; } + private long saveSmallFiles(DataOutputStream out, long checksum) throws IOException { + smallFileMgr.write(out); + return checksum; + } + public void createCleaner() { cleaner = new Daemon() { protected void runOneCycle() { @@ -4609,6 +4627,10 @@ public Clone getCloneInstance() { return this.clone; } + public SmallFileMgr getSmallFileMgr() { + return this.smallFileMgr; + } + public long getReplayedJournalId() { return this.replayedJournalId.get(); } diff --git a/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java b/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java index d14ba2fd509989..dbebb35f7c0710 100644 --- a/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java +++ b/fe/src/main/java/org/apache/doris/clone/ColocateTableBalancer.java @@ -17,9 +17,6 @@ package org.apache.doris.clone; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.ColocateTableIndex; import org.apache.doris.catalog.Database; @@ -35,7 +32,10 @@ import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; +import com.google.common.base.Preconditions; import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; @@ -84,7 +84,7 @@ public static ColocateTableBalancer getInstance() { * 4 update colcate backendsPerBucketSeq meta * 5 do real data migration by clone job * 6 delete redundant replicas after all clone job done - * 7 mark colocate group stable in colocate meta and balance done + * 7 mark colocate group stable in colocate meta and balance done */ protected void runOneCycle() { checkAndCloneBalancingGroup(); diff --git a/fe/src/main/java/org/apache/doris/common/Config.java b/fe/src/main/java/org/apache/doris/common/Config.java index a3939ec8acd3d9..40d023212a9ca6 100644 --- a/fe/src/main/java/org/apache/doris/common/Config.java +++ b/fe/src/main/java/org/apache/doris/common/Config.java @@ -798,5 +798,22 @@ public class Config extends ConfigBase { */ @ConfField(mutable = true, masterOnly = true) public static int max_routine_load_task_concurrent_num = 5; + + /* + * The max number of files store in SmallFileMgr + */ + @ConfField(mutable = true, masterOnly = true) + public static int max_small_file_number = 100; + + /* + * The max size of a single file store in SmallFileMgr + */ + @ConfField(mutable = true, masterOnly = true) + public static int max_small_file_size_bytes = 1024 * 1024; // 1MB + + /* + * Save small files + */ + @ConfField public static String small_file_dir = System.getenv("DORIS_HOME") + "/small_files"; } diff --git a/fe/src/main/java/org/apache/doris/common/FeConstants.java b/fe/src/main/java/org/apache/doris/common/FeConstants.java index afad4ecb49693d..74850a52d2c2ed 100644 --- a/fe/src/main/java/org/apache/doris/common/FeConstants.java +++ b/fe/src/main/java/org/apache/doris/common/FeConstants.java @@ -35,5 +35,5 @@ public class FeConstants { // general model // Current meta data version. Use this version to write journals and image - public static int meta_version = FeMetaVersion.VERSION_51; + public static int meta_version = FeMetaVersion.VERSION_52; } diff --git a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java index 2e95c3f12ec587..4b64d79cac2106 100644 --- a/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java +++ b/fe/src/main/java/org/apache/doris/common/FeMetaVersion.java @@ -112,4 +112,6 @@ public final class FeMetaVersion { public static final int VERSION_50 = 50; // kafka custom properties public static final int VERSION_51 = 51; + // small files + public static final int VERSION_52 = 52; } diff --git a/fe/src/main/java/org/apache/doris/common/Status.java b/fe/src/main/java/org/apache/doris/common/Status.java index 65f83f7c03cd13..13107b3233c63c 100644 --- a/fe/src/main/java/org/apache/doris/common/Status.java +++ b/fe/src/main/java/org/apache/doris/common/Status.java @@ -17,7 +17,7 @@ package org.apache.doris.common; -import org.apache.doris.rpc.PStatus; +import org.apache.doris.proto.PStatus; import org.apache.doris.thrift.TStatus; import org.apache.doris.thrift.TStatusCode; @@ -82,9 +82,9 @@ public void setStatus(String msg) { } public void setPstatus(PStatus status) { - this.errorCode = TStatusCode.findByValue(status.code); - if (status.msgs != null && !status.msgs.isEmpty()) { - this.errorMsg = status.msgs.get(0); + this.errorCode = TStatusCode.findByValue(status.status_code); + if (status.error_msgs != null && !status.error_msgs.isEmpty()) { + this.errorMsg = status.error_msgs.get(0); } } diff --git a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java index eeb8811cc149ca..801ea9131811f9 100644 --- a/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java +++ b/fe/src/main/java/org/apache/doris/common/proc/CurrentQueryInfoProvider.java @@ -17,26 +17,31 @@ package org.apache.doris.common.proc; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import org.apache.doris.catalog.Catalog; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Pair; import org.apache.doris.common.util.Counter; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.common.util.RuntimeProfile; +import org.apache.doris.proto.PTriggerProfileReportResult; +import org.apache.doris.proto.PUniqueId; import org.apache.doris.qe.QueryStatisticsItem; -import org.apache.doris.rpc.*; +import org.apache.doris.rpc.BackendServiceProxy; +import org.apache.doris.rpc.PTriggerProfileReportRequest; +import org.apache.doris.rpc.RpcException; import org.apache.doris.system.Backend; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TStatusCode; import org.apache.doris.thrift.TUniqueId; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.util.Collection; -import java.util.Formatter; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -198,7 +203,9 @@ private void triggerProfileReport(Collection items, boolean } // specified query instance which will report. if (!allQuery) { - final PUniqueId pUId = new PUniqueId(instanceInfo.getInstanceId()); + final PUniqueId pUId = new PUniqueId(); + pUId.hi = instanceInfo.getInstanceId().hi; + pUId.lo = instanceInfo.getInstanceId().lo; request.addInstanceId(pUId); } } @@ -230,11 +237,11 @@ private void recvResponse(List try { final PTriggerProfileReportResult result = pair.second.get(2, TimeUnit.SECONDS); - final TStatusCode code = TStatusCode.findByValue(result.status.code); + final TStatusCode code = TStatusCode.findByValue(result.status.status_code); if (code != TStatusCode.OK) { String errMsg = ""; - if (result.status.msgs != null && !result.status.msgs.isEmpty()) { - errMsg = result.status.msgs.get(0); + if (result.status.error_msgs != null && !result.status.error_msgs.isEmpty()) { + errMsg = result.status.error_msgs.get(0); } throw new AnalysisException(reasonPrefix + " backend:" + pair.first.getAddress() + " reason:" + errMsg); diff --git a/fe/src/main/java/org/apache/doris/common/util/DebugUtil.java b/fe/src/main/java/org/apache/doris/common/util/DebugUtil.java index 783c06d88b3068..31febdcf2b2ef1 100644 --- a/fe/src/main/java/org/apache/doris/common/util/DebugUtil.java +++ b/fe/src/main/java/org/apache/doris/common/util/DebugUtil.java @@ -18,7 +18,7 @@ package org.apache.doris.common.util; import org.apache.doris.common.Pair; -import org.apache.doris.rpc.PUniqueId; +import org.apache.doris.proto.PUniqueId; import org.apache.doris.thrift.TUniqueId; import java.io.PrintWriter; diff --git a/fe/src/main/java/org/apache/doris/common/util/KafkaUtil.java b/fe/src/main/java/org/apache/doris/common/util/KafkaUtil.java new file mode 100644 index 00000000000000..6f75f8e5f4695f --- /dev/null +++ b/fe/src/main/java/org/apache/doris/common/util/KafkaUtil.java @@ -0,0 +1,103 @@ +// 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.common.util; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.ClientPool; +import org.apache.doris.common.LoadException; +import org.apache.doris.common.UserException; +import org.apache.doris.proto.PKafkaLoadInfo; +import org.apache.doris.proto.PKafkaMetaProxyRequest; +import org.apache.doris.proto.PProxyRequest; +import org.apache.doris.proto.PProxyResult; +import org.apache.doris.proto.PStringPair; +import org.apache.doris.rpc.BackendServiceProxy; +import org.apache.doris.system.Backend; +import org.apache.doris.thrift.BackendService; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TStatusCode; + +import com.google.common.collect.Lists; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +public class KafkaUtil { + private static final Logger LOG = LogManager.getLogger(KafkaUtil.class); + + public static List getAllKafkaPartitions(String brokerList, String topic, + Map convertedCustomProperties) throws UserException { + BackendService.Client client = null; + TNetworkAddress address = null; + boolean ok = false; + try { + List backendIds = Catalog.getCurrentSystemInfo().getBackendIds(true); + if (backendIds.isEmpty()) { + throw new LoadException("Failed to get all partitions. No alive backends"); + } + Collections.shuffle(backendIds); + Backend be = Catalog.getCurrentSystemInfo().getBackend(backendIds.get(0)); + address = new TNetworkAddress(be.getHost(), be.getBrpcPort()); + + // create request + PKafkaLoadInfo kafkaLoadInfo = new PKafkaLoadInfo(); + kafkaLoadInfo.brokers = brokerList; + kafkaLoadInfo.topic = topic; + for (Map.Entry entry : convertedCustomProperties.entrySet()) { + PStringPair pair = new PStringPair(); + pair.key = entry.getKey(); + pair.val = entry.getValue(); + if (kafkaLoadInfo.properties == null) { + kafkaLoadInfo.properties = Lists.newArrayList(); + } + kafkaLoadInfo.properties.add(pair); + } + PKafkaMetaProxyRequest kafkaRequest = new PKafkaMetaProxyRequest(); + kafkaRequest.kafka_info = kafkaLoadInfo; + PProxyRequest request = new PProxyRequest(); + request.kafka_meta_request = kafkaRequest; + + // get info + Future future = BackendServiceProxy.getInstance().getInfo(address, request); + PProxyResult result = future.get(5, TimeUnit.SECONDS); + TStatusCode code = TStatusCode.findByValue(result.status.status_code); + if (code != TStatusCode.OK) { + throw new UserException("failed to get kafka partition info: " + result.status.error_msgs); + } else { + return result.kafka_meta_result.partition_ids; + } + } catch (Exception e) { + LOG.warn("failed to get partitions.", e); + throw new LoadException( + "Failed to get all partitions of kafka topic: " + topic + ". error: " + e.getMessage()); + } finally { + if (ok) { + ClientPool.backendPool.returnObject(address, client); + } else { + ClientPool.backendPool.invalidateObject(address, client); + } + } + } +} + diff --git a/fe/src/main/java/org/apache/doris/common/util/SmallFileMgr.java b/fe/src/main/java/org/apache/doris/common/util/SmallFileMgr.java new file mode 100644 index 00000000000000..2f6eb8b958a77d --- /dev/null +++ b/fe/src/main/java/org/apache/doris/common/util/SmallFileMgr.java @@ -0,0 +1,529 @@ +// 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.common.util; + +import org.apache.doris.analysis.CreateFileStmt; +import org.apache.doris.analysis.DropFileStmt; +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Database; +import org.apache.doris.common.Config; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.io.Text; +import org.apache.doris.common.io.Writable; + +import com.google.common.base.Strings; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Table; + +import org.apache.commons.codec.binary.Hex; +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.BufferedInputStream; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.net.HttpURLConnection; +import java.net.URL; +import java.net.URLConnection; +import java.nio.file.Paths; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.Base64; +import java.util.List; +import java.util.Map; + +/* + * Author: Chenmingyu + * Date: May 29, 2019 + */ + +/* + * Manage some small files, such as certification file, public/private key used for some operations + */ +public class SmallFileMgr implements Writable { + public static final Logger LOG = LogManager.getLogger(SmallFileMgr.class); + + public static class SmallFile implements Writable { + public long dbId; + public String catalog; + public String name; + public long id; + public String content; + public long size; + public String md5; + public boolean isContent; + + private SmallFile() { + + } + + public SmallFile(Long dbId, String catalogName, String fileName, Long id, String content, long size, + String md5, boolean isContent) { + this.dbId = dbId; + this.catalog = catalogName; + this.name = fileName; + this.id = id; + this.content = content; + this.size = size; + this.md5 = md5.toLowerCase(); + this.isContent = isContent; + } + + public static SmallFile read(DataInput in) throws IOException { + SmallFile smallFile = new SmallFile(); + smallFile.readFields(in); + return smallFile; + } + + public byte[] getContentBytes() { + if (!isContent) { + return null; + } + return Base64.getDecoder().decode(content); + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeLong(dbId); + Text.writeString(out, catalog); + Text.writeString(out, name); + out.writeLong(id); + Text.writeString(out, content); + out.writeLong(size); + Text.writeString(out, md5); + out.writeBoolean(isContent); + } + + @Override + public void readFields(DataInput in) throws IOException { + dbId = in.readLong(); + catalog = Text.readString(in); + name = Text.readString(in); + id = in.readLong(); + content = Text.readString(in); + size = in.readLong(); + md5 = Text.readString(in); + isContent = in.readBoolean(); + } + } + + public static class SmallFiles { + // file name -> file + private Map files = Maps.newHashMap(); + + public SmallFiles() { + + } + + public Map getFiles() { + return files; + } + + public void addFile(String fileName, SmallFile file) throws DdlException { + if (files.containsKey(fileName)) { + throw new DdlException("File " + fileName + " already exist"); + } + this.files.put(fileName, file); + } + + public SmallFile removeFile(String fileName) { + return files.remove(fileName); + } + + public SmallFile getFile(String fileName) { + return files.get(fileName); + } + + public boolean containsFile(String fileName) { + return files.containsKey(fileName); + } + } + + // db id -> catalog -> files + private Table files = HashBasedTable.create(); + private Map idToFiles = Maps.newHashMap(); + + public SmallFileMgr() { + } + + public void createFile(CreateFileStmt stmt) throws DdlException { + String dbName = stmt.getDbName(); + Database db = Catalog.getCurrentCatalog().getDb(dbName); + if (db == null) { + throw new DdlException("Database " + dbName + " does not exist"); + } + downloadAndAddFile(db.getId(), stmt.getCatalogName(), stmt.getFileName(), + stmt.getDownloadUrl(), stmt.getChecksum(), stmt.isSaveContent()); + } + + public void dropFile(DropFileStmt stmt) throws DdlException { + String dbName = stmt.getDbName(); + Database db = Catalog.getCurrentCatalog().getDb(dbName); + if (db == null) { + throw new DdlException("Database " + dbName + " does not exist"); + } + removeFile(db.getId(), stmt.getCatalogName(), stmt.getFileName(), false); + } + + private void downloadAndAddFile(long dbId, String catalog, String fileName, String downloadUrl, String md5sum, + boolean saveContent) throws DdlException { + synchronized (files) { + if (idToFiles.size() >= Config.max_small_file_number) { + throw new DdlException("File number exceeds limit: " + Config.max_small_file_number); + } + } + + SmallFile smallFile = downloadAndCheck(dbId, catalog, fileName, downloadUrl, md5sum, saveContent); + + synchronized (files) { + if (idToFiles.size() >= Config.max_small_file_number) { + throw new DdlException("File number exceeds limit: " + Config.max_small_file_number); + } + + SmallFiles smallFiles = files.get(dbId, catalog); + if (smallFiles == null) { + smallFiles = new SmallFiles(); + files.put(dbId, catalog, smallFiles); + } + + smallFiles.addFile(fileName, smallFile); + idToFiles.put(smallFile.id, smallFile); + + Catalog.getCurrentCatalog().getEditLog().logCreateSmallFile(smallFile); + + LOG.info("finished to add file {} from url {}. current file number: {}", fileName, downloadUrl, + idToFiles.size()); + } + } + + public void replayCreateFile(SmallFile smallFile) { + synchronized (files) { + SmallFiles smallFiles = files.get(smallFile.dbId, smallFile.catalog); + if (smallFiles == null) { + smallFiles = new SmallFiles(); + files.put(smallFile.dbId, smallFile.catalog, smallFiles); + } + + try { + smallFiles.addFile(smallFile.name, smallFile); + idToFiles.put(smallFile.id, smallFile); + } catch (DdlException e) { + LOG.warn("should not happen", e); + } + } + } + + public void removeFile(long dbId, String catalog, String fileName, boolean isReplay) throws DdlException { + synchronized (files) { + SmallFiles smallFiles = files.get(dbId, catalog); + if (smallFiles == null) { + throw new DdlException("No such file in catalog: " + catalog); + } + SmallFile smallFile = smallFiles.removeFile(fileName); + if (smallFile != null) { + idToFiles.remove(smallFile.id); + + if (!isReplay) { + Catalog.getCurrentCatalog().getEditLog().logDropSmallFile(smallFile); + } + + LOG.info("finished to remove file {}. current file number: {}. is replay: {}", + fileName, idToFiles.size(), isReplay); + } else { + throw new DdlException("No such file: " + fileName); + } + } + } + + public void replayRemoveFile(SmallFile smallFile) { + try { + removeFile(smallFile.dbId, smallFile.catalog, smallFile.name, true); + } catch (DdlException e) { + LOG.error("should not happen", e); + } + } + + public boolean containsFile(long dbId, String catalog, String fileName) { + synchronized (files) { + SmallFiles smallFiles = files.get(dbId, catalog); + if (smallFiles == null) { + return false; + } + return smallFiles.containsFile(fileName); + } + } + + public SmallFile getSmallFile(long dbId, String catalog, String fileName, boolean needContent) + throws DdlException { + synchronized (files) { + SmallFiles smallFiles = files.get(dbId, catalog); + if (smallFiles == null) { + throw new DdlException("file does not exist with db: " + dbId + " and catalog: " + catalog); + } + SmallFile smallFile = smallFiles.getFile(fileName); + if (smallFile == null) { + throw new DdlException("File does not exist"); + } else if (needContent && !smallFile.isContent) { + throw new DdlException("File exists but not with content"); + } + return smallFile; + } + } + + public SmallFile getSmallFile(long fileId) { + synchronized (files) { + return idToFiles.get(fileId); + } + } + + private SmallFile downloadAndCheck(long dbId, String catalog, String fileName, + String downloadUrl, String md5sum, boolean saveContent) throws DdlException { + try { + URL url = new URL(downloadUrl); + // get file length + URLConnection urlConnection = url.openConnection(); + if (urlConnection instanceof HttpURLConnection) { + ((HttpURLConnection) urlConnection).setRequestMethod("HEAD"); + } + urlConnection.setReadTimeout(10000); // 10s + urlConnection.getInputStream(); + + int contentLength = urlConnection.getContentLength(); + if (contentLength == -1 || contentLength > Config.max_small_file_size_bytes) { + throw new DdlException("Failed to download file from url: " + url + ", invalid content length: " + contentLength); + } + + int bytesRead = 0; + String base64Content = null; + MessageDigest digest = MessageDigest.getInstance("MD5"); + if (saveContent) { + // download from url, and check file size + bytesRead = 0; + byte buf[] = new byte[contentLength]; + try (BufferedInputStream in = new BufferedInputStream(url.openStream())) { + while (bytesRead < contentLength) { + bytesRead += in.read(buf, bytesRead, contentLength - bytesRead); + } + + // check if there still has data(should not happen) + if (in.read() != -1) { + throw new DdlException("Failed to download file from url: " + url + + ", content length does not equals to actual file length"); + } + } + + if (bytesRead != contentLength) { + throw new DdlException("Failed to download file from url: " + url + + ", invalid read bytes: " + bytesRead + ", expected: " + contentLength); + } + + digest.update(buf, 0, bytesRead); + // encoded to base64 + base64Content = Base64.getEncoder().encodeToString(buf); + } else { + byte[] buf = new byte[4096]; + int tmpSize = 0; + try (BufferedInputStream in = new BufferedInputStream(url.openStream())) { + do { + tmpSize = in.read(buf); + if (tmpSize < 0) { + break; + } + digest.update(buf, 0, tmpSize); + bytesRead += tmpSize; + } while (true); + } + } + + // check md5sum if necessary + String checksum = Hex.encodeHexString(digest.digest()); + if (!Strings.isNullOrEmpty(md5sum)) { + if (!checksum.equalsIgnoreCase(md5sum)) { + throw new DdlException("Invalid md5sum of file in url: " + downloadUrl + ", read: " + checksum + + ", expected: " + checksum); + } + } + + SmallFile smallFile; + long fileId = Catalog.getCurrentCatalog().getNextId(); + if (saveContent) { + smallFile = new SmallFile(dbId, catalog, fileName, fileId, base64Content, bytesRead, + checksum, true /* is content */); + } else { + // only save download url + smallFile = new SmallFile(dbId, catalog, fileName, fileId, downloadUrl, bytesRead, + checksum, false /* not content */); + } + return smallFile; + } catch (IOException | NoSuchAlgorithmException e) { + LOG.warn("failed to get file from url: {}", downloadUrl, e); + String errorMsg = e.getMessage(); + if (e instanceof FileNotFoundException) { + errorMsg = "File not found"; + } + throw new DdlException("Failed to get file from url: " + downloadUrl + ". Error: " + errorMsg); + } + } + + // save the specified file to disk. if file already exist, check it. + // return the absolute file path. + public String saveToFile(long dbId, String catalog, String fileName) throws DdlException { + SmallFile smallFile; + synchronized (files) { + SmallFiles smallFiles = files.get(dbId, catalog); + if (smallFiles == null) { + throw new DdlException("File " + fileName + " does not exist"); + } + + smallFile = smallFiles.getFile(fileName); + if (smallFile == null) { + throw new DdlException("File " + fileName + " does not exist"); + } + + if (!smallFile.isContent) { + throw new DdlException("File does not contain content: " + smallFile.id); + } + } + + // check file + File file = getAbsoluteFile(dbId, catalog, fileName); + if (file.exists()) { + if (!file.isFile()) { + throw new DdlException("File exist but not a file: " + fileName); + } + + if (checkMd5(file, smallFile.md5)) { + return file.getAbsolutePath(); + } + + // file is invalid, delete it and create a new one + file.delete(); + } + + // write to file + try { + if (!file.getParentFile().exists() && !file.getParentFile().mkdirs()) { + throw new IOException("failed to make dir for file: " + fileName); + } + file.createNewFile(); + byte[] decoded = Base64.getDecoder().decode(smallFile.content); + FileOutputStream outputStream = new FileOutputStream(file); + outputStream.write(decoded); + outputStream.flush(); + outputStream.close(); + + if (!checkMd5(file, smallFile.md5)) { + throw new DdlException("write file " + fileName +" failed. md5 is invalid. expected: " + smallFile.md5); + } + } catch (IOException e) { + LOG.warn("failed to write file: {}", fileName, e); + throw new DdlException("failed to write file: " + fileName); + } + + return file.getAbsolutePath(); + } + + private boolean checkMd5(File file, String expectedMd5) throws DdlException { + String md5sum = null; + try { + md5sum = DigestUtils.md5Hex(new FileInputStream(file)); + } catch (FileNotFoundException e) { + throw new DdlException("File " + file.getName() + " does not exist"); + } catch (IOException e) { + LOG.warn("failed to check md5 of file: {}", file.getName(), e); + throw new DdlException("Failed to check md5 of file: " + file.getName()); + } + + return md5sum.equalsIgnoreCase(expectedMd5); + } + + private File getAbsoluteFile(long dbId, String catalog, String fileName) { + return Paths.get(Config.small_file_dir, String.valueOf(dbId), catalog, fileName).normalize().toAbsolutePath().toFile(); + } + + public List> getInfo(String dbName) throws DdlException { + Database db = Catalog.getCurrentCatalog().getDb(dbName); + if (db == null) { + throw new DdlException("Database " + dbName + " does not exist"); + } + + List> infos = Lists.newArrayList(); + synchronized (files) { + if (files.containsRow(db.getId())) { + Map dbFiles = files.row(db.getId()); + for (Map.Entry entry : dbFiles.entrySet()) { + SmallFiles smallFiles = entry.getValue(); + for (Map.Entry entry2 : smallFiles.getFiles().entrySet()) { + List info = Lists.newArrayList(); + info.add(String.valueOf(entry2.getValue().id)); + info.add(dbName); + info.add(entry.getKey()); // catalog + info.add(entry2.getKey()); // file name + info.add(String.valueOf(entry2.getValue().size)); // file size + info.add(String.valueOf(entry2.getValue().isContent)); + info.add(entry2.getValue().md5); + infos.add(info); + } + } + } + } + return infos; + } + + public static SmallFileMgr read(DataInput in) throws IOException { + SmallFileMgr mgr = new SmallFileMgr(); + mgr.readFields(in); + return mgr; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(idToFiles.size()); + for (SmallFile smallFile : idToFiles.values()) { + smallFile.write(out); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + int size = in.readInt(); + for (int i = 0; i < size; i++) { + SmallFile smallFile = SmallFile.read(in); + idToFiles.put(smallFile.id, smallFile); + SmallFiles smallFiles = files.get(smallFile.dbId, smallFile.catalog); + if (smallFiles == null) { + smallFiles = new SmallFiles(); + files.put(smallFile.dbId, smallFile.catalog, smallFiles); + } + try { + smallFiles.addFile(smallFile.name, smallFile); + } catch (DdlException e) { + // should not happen + e.printStackTrace(); + } + } + } +} diff --git a/fe/src/main/java/org/apache/doris/http/BaseAction.java b/fe/src/main/java/org/apache/doris/http/BaseAction.java index 9ff453a4c361a5..be215b4963b77b 100644 --- a/fe/src/main/java/org/apache/doris/http/BaseAction.java +++ b/fe/src/main/java/org/apache/doris/http/BaseAction.java @@ -30,6 +30,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.ByteArrayInputStream; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -63,6 +64,8 @@ import io.netty.handler.codec.http.cookie.ServerCookieEncoder; import io.netty.handler.ssl.SslHandler; import io.netty.handler.stream.ChunkedFile; +import io.netty.handler.stream.ChunkedInput; +import io.netty.handler.stream.ChunkedStream; import io.netty.util.CharsetUtil; public abstract class BaseAction implements IAction { @@ -139,43 +142,64 @@ protected void writeResponse(BaseRequest request, BaseResponse response, HttpRes request.getContext().write(responseObj); } } - - protected void writeFileResponse(BaseRequest request, BaseResponse response, HttpResponseStatus status, - File resFile) { + + // Object only support File or byte[] + protected void writeObjectResponse(BaseRequest request, BaseResponse response, HttpResponseStatus status, + Object obj, String fileName) { + Preconditions.checkState((obj instanceof File) || (obj instanceof byte[])); + HttpResponse responseObj = new DefaultHttpResponse(HttpVersion.HTTP_1_1, status); if (HttpUtil.isKeepAlive(request.getRequest())) { response.updateHeader(HttpHeaderNames.CONNECTION.toString(), HttpHeaderValues.KEEP_ALIVE.toString()); } - + response.updateHeader(HttpHeaderNames.CONTENT_TYPE.toString(), HttpHeaderValues.APPLICATION_OCTET_STREAM.toString()); + response.updateHeader(HttpHeaderNames.CONTENT_DISPOSITION.toString(), + HttpHeaderValues.ATTACHMENT.toString() + "; " + HttpHeaderValues.FILENAME.toString() + "=" + fileName); + ChannelFuture sendFileFuture; ChannelFuture lastContentFuture; - // Read and return file content - RandomAccessFile rafFile; try { - rafFile = new RandomAccessFile(resFile, "r"); - long fileLength = 0; - fileLength = rafFile.length(); - response.updateHeader(HttpHeaderNames.CONTENT_LENGTH.toString(), String.valueOf(fileLength)); + Object writable = null; + long contentLen = 0; + boolean sslEnable = request.getContext().pipeline().get(SslHandler.class) != null; + if (obj instanceof File) { + RandomAccessFile rafFile = new RandomAccessFile((File) obj, "r"); + contentLen = rafFile.length(); + if (!sslEnable) { + // use zero-copy file transfer. + writable = new DefaultFileRegion(rafFile.getChannel(), 0, contentLen); + } else { + // cannot use zero-copy file transfer. + writable = new ChunkedFile(rafFile, 0, contentLen, 8192); + } + } else if (obj instanceof byte[]) { + contentLen = ((byte[]) obj).length; + if (!sslEnable) { + writable = Unpooled.wrappedBuffer((byte[]) obj); + } else { + writable = new ChunkedStream(new ByteArrayInputStream((byte[]) obj)); + } + } + + response.updateHeader(HttpHeaderNames.CONTENT_LENGTH.toString(), String.valueOf(contentLen)); writeCookies(response, responseObj); writeCustomHeaders(response, responseObj); // Write headers request.getContext().write(responseObj); - // Write file - if (request.getContext().pipeline().get(SslHandler.class) == null) { - sendFileFuture = request.getContext().write(new DefaultFileRegion(rafFile.getChannel(), 0, fileLength), - request.getContext().newProgressivePromise()); + // Write object + if (!sslEnable) { + sendFileFuture = request.getContext().write(writable, request.getContext().newProgressivePromise()); // Write the end marker. lastContentFuture = request.getContext().writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); } else { sendFileFuture = request.getContext().writeAndFlush( - new HttpChunkedInput(new ChunkedFile(rafFile, 0, fileLength, 8192)), + new HttpChunkedInput((ChunkedInput) writable), request.getContext().newProgressivePromise()); - // HttpChunkedInput will write the end marker (LastHttpContent) - // for us. + // HttpChunkedInput will write the end marker (LastHttpContent) for us. lastContentFuture = sendFileFuture; } } catch (FileNotFoundException ignore) { diff --git a/fe/src/main/java/org/apache/doris/http/HttpServer.java b/fe/src/main/java/org/apache/doris/http/HttpServer.java index 9a5e212776a28c..d1e56beb8bf05e 100644 --- a/fe/src/main/java/org/apache/doris/http/HttpServer.java +++ b/fe/src/main/java/org/apache/doris/http/HttpServer.java @@ -44,6 +44,7 @@ import org.apache.doris.http.rest.GetDdlStmtAction; import org.apache.doris.http.rest.GetLoadInfoAction; import org.apache.doris.http.rest.GetLogFileAction; +import org.apache.doris.http.rest.GetSmallFileAction; import org.apache.doris.http.rest.GetStreamLoadState; import org.apache.doris.http.rest.HealthAction; import org.apache.doris.http.rest.LoadAction; @@ -139,6 +140,7 @@ private void registerActions() throws IllegalArgException { ShowProcAction.registerAction(controller); ShowRuntimeInfoAction.registerAction(controller); GetLogFileAction.registerAction(controller); + GetSmallFileAction.registerAction(controller); RowCountAction.registerAction(controller); CheckDecommissionAction.registerAction(controller); MetaReplayerCheckAction.registerAction(controller); diff --git a/fe/src/main/java/org/apache/doris/http/action/StaticResourceAction.java b/fe/src/main/java/org/apache/doris/http/action/StaticResourceAction.java index ed9295e6a090a5..9d98f86942dae2 100644 --- a/fe/src/main/java/org/apache/doris/http/action/StaticResourceAction.java +++ b/fe/src/main/java/org/apache/doris/http/action/StaticResourceAction.java @@ -190,7 +190,7 @@ public void executeGet(BaseRequest request, BaseResponse response) { response.updateHeader(HttpHeaders.Names.CONTENT_TYPE, getContentType(resourceAbsolutePath)); setDateAndCacheHeaders(response, resFile); - writeFileResponse(request, response, HttpResponseStatus.OK, resFile); + writeObjectResponse(request, response, HttpResponseStatus.OK, resFile, resFile.getName()); } diff --git a/fe/src/main/java/org/apache/doris/http/meta/MetaBaseAction.java b/fe/src/main/java/org/apache/doris/http/meta/MetaBaseAction.java index 653d19df8d607f..b8c78cfcbac936 100644 --- a/fe/src/main/java/org/apache/doris/http/meta/MetaBaseAction.java +++ b/fe/src/main/java/org/apache/doris/http/meta/MetaBaseAction.java @@ -85,7 +85,7 @@ protected void writeFileResponse(BaseRequest request, BaseResponse response, Fil response.updateHeader(CONTENT_DISPOSITION, "attachment; filename=" + file.getName()); response.updateHeader(MetaHelper.X_IMAGE_SIZE, String.valueOf(file.length())); - writeFileResponse(request, response, HttpResponseStatus.OK, file); + writeObjectResponse(request, response, HttpResponseStatus.OK, file, file.getName()); return; } diff --git a/fe/src/main/java/org/apache/doris/http/rest/GetLogFileAction.java b/fe/src/main/java/org/apache/doris/http/rest/GetLogFileAction.java index 709a88a8db288a..ab3305943d791d 100644 --- a/fe/src/main/java/org/apache/doris/http/rest/GetLogFileAction.java +++ b/fe/src/main/java/org/apache/doris/http/rest/GetLogFileAction.java @@ -24,22 +24,33 @@ import org.apache.doris.http.IllegalArgException; import com.google.common.base.Strings; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; +import org.codehaus.jackson.map.ObjectMapper; import java.io.File; +import java.io.IOException; +import java.util.Map; import java.util.Set; -import io.netty.handler.codec.http.HttpHeaderNames; import io.netty.handler.codec.http.HttpMethod; import io.netty.handler.codec.http.HttpResponseStatus; +/* + * get log file infos: + * curl -I http://fe_host:http_port/api/get_log_file?type=fe.audit.log + * return: + * HTTP/1.1 200 OK + * file_infos: {"fe.audit.log":24759,"fe.audit.log.20190528.1":132934} + * content-type: text/html + * connection: keep-alive + * + * get log file: + * curl -X GET http://fe_host:http_port/api/get_log_file?type=fe.audit.log&file=fe.audit.log.20190528.1 + */ public class GetLogFileAction extends RestBaseAction { - Set logFileTypes = Sets.newHashSet("fe.audit.log"); - - private static final Logger LOG = LogManager.getLogger(GetLogFileAction.class); + private final Set logFileTypes = Sets.newHashSet("fe.audit.log"); public GetLogFileAction(ActionController controller) { super(controller); @@ -51,9 +62,9 @@ public static void registerAction(ActionController controller) throws IllegalArg } @Override - public void execute(BaseRequest request, BaseResponse response) { + public void executeWithoutPassword(AuthorizationInfo authInfo, BaseRequest request, BaseResponse response) { String logType = request.getSingleParameter("type"); - String logDate = request.getSingleParameter("date"); + String logFile = request.getSingleParameter("file"); // check param empty if (Strings.isNullOrEmpty(logType)) { @@ -69,38 +80,53 @@ public void execute(BaseRequest request, BaseResponse response) { return; } - // check log file exist - File logFile = getLogFileName(logType, logDate); - if (logFile == null || !logFile.exists()) { - response.appendContent("Log file not exist."); - writeResponse(request, response, HttpResponseStatus.NOT_FOUND); - return; - } - - // get file or just file size HttpMethod method = request.getRequest().method(); - if (method.equals(HttpMethod.GET)) { - writeFileResponse(request, response, HttpResponseStatus.OK, logFile); - } else if (method.equals(HttpMethod.HEAD)) { - long fileLength = logFile.length(); - response.updateHeader(HttpHeaderNames.CONTENT_LENGTH.toString(), String.valueOf(fileLength)); + if (method.equals(HttpMethod.HEAD)) { + String fileInfos = getFileInfos(logType); + response.updateHeader("file_infos", fileInfos); writeResponse(request, response, HttpResponseStatus.OK); + return; + } else if (method.equals(HttpMethod.GET)) { + File log = getLogFile(logType, logFile); + if (!log.exists() || !log.isFile()) { + response.appendContent("Log file not exist: " + log.getName()); + writeResponse(request, response, HttpResponseStatus.NOT_FOUND); + return; + } + writeObjectResponse(request, response, HttpResponseStatus.OK, log, log.getName()); } else { response.appendContent(new RestBaseResult("HTTP method is not allowed.").toJson()); writeResponse(request, response, HttpResponseStatus.METHOD_NOT_ALLOWED); } } - private File getLogFileName(String type, String date) { - String logPath = ""; - - if ("fe.audit.log".equals(type)) { - logPath = Config.audit_log_dir + "/fe.audit.log"; - if (!Strings.isNullOrEmpty(date)) { - logPath += "." + date; + private String getFileInfos(String logType) { + Map fileInfos = Maps.newTreeMap(); + if (logType.equals("fe.audit.log")) { + File logDir = new File(Config.audit_log_dir); + File[] files = logDir.listFiles(); + for (int i = 0; i < files.length; i++) { + if (files[i].isFile() && files[i].getName().startsWith("fe.audit.log")) { + fileInfos.put(files[i].getName(), files[i].length()); + } } } + String result = ""; + ObjectMapper mapper = new ObjectMapper(); + try { + result = mapper.writeValueAsString(fileInfos); + } catch (Exception e) { + // do nothing + } + return result; + } + + private File getLogFile(String logType, String logFile) { + String logPath = ""; + if ("fe.audit.log".equals(logType)) { + logPath = Config.audit_log_dir + "/" + logFile; + } return new File(logPath); } } diff --git a/fe/src/main/java/org/apache/doris/http/rest/GetSmallFileAction.java b/fe/src/main/java/org/apache/doris/http/rest/GetSmallFileAction.java new file mode 100644 index 00000000000000..6148256e9a9768 --- /dev/null +++ b/fe/src/main/java/org/apache/doris/http/rest/GetSmallFileAction.java @@ -0,0 +1,93 @@ +// 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.http.rest; + +import org.apache.doris.catalog.Catalog; +import org.apache.doris.common.util.SmallFileMgr; +import org.apache.doris.common.util.SmallFileMgr.SmallFile; +import org.apache.doris.http.ActionController; +import org.apache.doris.http.BaseRequest; +import org.apache.doris.http.BaseResponse; +import org.apache.doris.http.IllegalArgException; + +import com.google.common.base.Strings; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.IOException; + +import io.netty.handler.codec.http.HttpMethod; +import io.netty.handler.codec.http.HttpResponseStatus; + +public class GetSmallFileAction extends RestBaseAction { + private static final Logger LOG = LogManager.getLogger(GetSmallFileAction.class); + + public GetSmallFileAction(ActionController controller) { + super(controller); + } + + public static void registerAction(ActionController controller) throws IllegalArgException { + controller.registerHandler(HttpMethod.GET, "/api/get_small_file", new GetSmallFileAction(controller)); + } + + @Override + public void execute(BaseRequest request, BaseResponse response) { + String token = request.getSingleParameter("token"); + String fileIdStr = request.getSingleParameter("file_id"); + + // check param empty + if (Strings.isNullOrEmpty(token) || Strings.isNullOrEmpty(fileIdStr)) { + response.appendContent("Missing parameter"); + writeResponse(request, response, HttpResponseStatus.BAD_REQUEST); + return; + } + + // check token + if (!token.equals(Catalog.getCurrentCatalog().getToken())) { + response.appendContent("Invalid token"); + writeResponse(request, response, HttpResponseStatus.BAD_REQUEST); + return; + } + + long fileId = -1; + try { + fileId = Long.valueOf(fileIdStr); + } catch (NumberFormatException e) { + response.appendContent("Invalid file id format: " + fileIdStr); + writeResponse(request, response, HttpResponseStatus.BAD_REQUEST); + return; + } + + SmallFileMgr fileMgr = Catalog.getCurrentCatalog().getSmallFileMgr(); + SmallFile smallFile = fileMgr.getSmallFile(fileId); + if (smallFile == null || !smallFile.isContent) { + response.appendContent("File not found or is not content"); + writeResponse(request, response, HttpResponseStatus.BAD_REQUEST); + return; + } + + HttpMethod method = request.getRequest().method(); + if (method.equals(HttpMethod.GET)) { + writeObjectResponse(request, response, HttpResponseStatus.OK, smallFile.getContentBytes(), smallFile.name); + } else { + response.appendContent(new RestBaseResult("HTTP method is not allowed.").toJson()); + writeResponse(request, response, HttpResponseStatus.METHOD_NOT_ALLOWED); + } + } +} diff --git a/fe/src/main/java/org/apache/doris/http/rest/RestBaseAction.java b/fe/src/main/java/org/apache/doris/http/rest/RestBaseAction.java index 6da7d1c9be51cc..9e017ccbf1ef5f 100644 --- a/fe/src/main/java/org/apache/doris/http/rest/RestBaseAction.java +++ b/fe/src/main/java/org/apache/doris/http/rest/RestBaseAction.java @@ -32,6 +32,7 @@ import java.net.URI; import java.net.URISyntaxException; +import io.netty.handler.codec.http.HttpHeaderNames; import io.netty.handler.codec.http.HttpHeaders; import io.netty.handler.codec.http.HttpResponseStatus; @@ -49,7 +50,7 @@ public void handleRequest(BaseRequest request) throws Exception { execute(request, response); } catch (DdlException e) { if (e instanceof UnauthorizedException) { - response.updateHeader(HttpHeaders.Names.WWW_AUTHENTICATE, "Basic realm=\"\""); + response.updateHeader(HttpHeaderNames.WWW_AUTHENTICATE.toString(), "Basic realm=\"\""); writeResponse(request, response, HttpResponseStatus.UNAUTHORIZED); } else { sendResult(request, response, new RestBaseResult(e.getMessage())); @@ -65,7 +66,8 @@ public void execute(BaseRequest request, BaseResponse response) throws DdlExcept executeWithoutPassword(authInfo, request, response); } - // all derived classed should implement this method, NOT 'execute' + // If user password should be checked, the derived class should implement this method, NOT 'execute()', + // otherwise, override 'execute()' directly protected void executeWithoutPassword(AuthorizationInfo authInfo, BaseRequest request, BaseResponse response) throws DdlException { throw new DdlException("Not implemented"); diff --git a/fe/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/src/main/java/org/apache/doris/journal/JournalEntity.java index 43ac7b78a5f661..8818e5032ed467 100644 --- a/fe/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -32,6 +32,7 @@ import org.apache.doris.cluster.Cluster; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; +import org.apache.doris.common.util.SmallFileMgr.SmallFile; import org.apache.doris.ha.MasterInfo; import org.apache.doris.journal.bdbje.Timestamp; import org.apache.doris.load.AsyncDeleteJob; @@ -435,6 +436,12 @@ public void readFields(DataInput in) throws IOException { needRead = false; break; } + case OperationType.OP_CREATE_SMALL_FILE: + case OperationType.OP_DROP_SMALL_FILE: { + data = SmallFile.read(in); + needRead = false; + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); diff --git a/fe/src/main/java/org/apache/doris/load/Load.java b/fe/src/main/java/org/apache/doris/load/Load.java index b1be1752583d5c..e60cbd8a98143a 100644 --- a/fe/src/main/java/org/apache/doris/load/Load.java +++ b/fe/src/main/java/org/apache/doris/load/Load.java @@ -17,7 +17,6 @@ package org.apache.doris.load; -import org.apache.doris.catalog.AggregateType; import org.apache.doris.analysis.BinaryPredicate; import org.apache.doris.analysis.CancelLoadStmt; import org.apache.doris.analysis.ColumnSeparator; @@ -31,6 +30,7 @@ import org.apache.doris.analysis.SlotRef; import org.apache.doris.backup.BlobStorage; import org.apache.doris.backup.Status; +import org.apache.doris.catalog.AggregateType; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Database; @@ -1182,19 +1182,24 @@ public boolean cancelLoadJob(CancelLoadStmt stmt) throws DdlException { } // cancel job - if (!cancelLoadJob(job, CancelType.USER_CANCEL, "user cancel")) { - throw new DdlException("Cancel load job fail"); + List failedMsg = Lists.newArrayList(); + if (!cancelLoadJob(job, CancelType.USER_CANCEL, "user cancel", failedMsg)) { + throw new DdlException("Cancel load job fail: " + (failedMsg.isEmpty() ? "Unknown reason" : failedMsg.get(0))); } return true; } public boolean cancelLoadJob(LoadJob job, CancelType cancelType, String msg) { + return cancelLoadJob(job, cancelType, msg, null); + } + + public boolean cancelLoadJob(LoadJob job, CancelType cancelType, String msg, List failedMsg) { // update job to cancelled LOG.info("try to cancel load job: {}", job); JobState srcState = job.getState(); - if (!updateLoadJobState(job, JobState.CANCELLED, cancelType, msg)) { - LOG.warn("cancel load job failed. job: {}", job, new Exception()); + if (!updateLoadJobState(job, JobState.CANCELLED, cancelType, msg, failedMsg)) { + LOG.warn("cancel load job failed. job: {}", job); return false; } @@ -2243,10 +2248,11 @@ public void clearJob(LoadJob job, JobState srcState) { } public boolean updateLoadJobState(LoadJob job, JobState destState) { - return updateLoadJobState(job, destState, CancelType.UNKNOWN, null); + return updateLoadJobState(job, destState, CancelType.UNKNOWN, null, null); } - public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType cancelType, String msg) { + public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType cancelType, String msg, + List failedMsg) { boolean result = true; JobState srcState = null; @@ -2262,7 +2268,7 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca try { // sometimes db is dropped and then cancel the job, the job must have transactionid // transaction state should only be dropped when job is dropped - processCancelled(job, cancelType, errMsg); + processCancelled(job, cancelType, errMsg, failedMsg); } finally { writeUnlock(); } @@ -2307,7 +2313,7 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca Catalog.getInstance().getEditLog().logLoadQuorum(job); } else { errMsg = "process loading finished fail"; - processCancelled(job, cancelType, errMsg); + processCancelled(job, cancelType, errMsg, failedMsg); } break; case FINISHED: @@ -2348,7 +2354,7 @@ public boolean updateLoadJobState(LoadJob job, JobState destState, CancelType ca Catalog.getInstance().getEditLog().logLoadDone(job); break; case CANCELLED: - processCancelled(job, cancelType, errMsg); + processCancelled(job, cancelType, errMsg, failedMsg); break; default: Preconditions.checkState(false, "wrong job state: " + destState.name()); @@ -2452,7 +2458,7 @@ private void updatePartitionVersion(Partition partition, long version, long vers version, versionHash, jobId, partitionId); } - private boolean processCancelled(LoadJob job, CancelType cancelType, String msg) { + private boolean processCancelled(LoadJob job, CancelType cancelType, String msg, List failedMsg) { long jobId = job.getId(); JobState srcState = job.getState(); CancelType tmpCancelType = CancelType.UNKNOWN; @@ -2464,6 +2470,9 @@ private boolean processCancelled(LoadJob job, CancelType cancelType, String msg) job.getFailMsg().toString()); } catch (Exception e) { LOG.info("errors while abort transaction", e); + if (failedMsg != null) { + failedMsg.add("Abort tranaction failed: " + e.getMessage()); + } return false; } switch (srcState) { diff --git a/fe/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java b/fe/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java index 1b9320b97335e2..4ab576e6acccc5 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/KafkaRoutineLoadJob.java @@ -21,6 +21,7 @@ import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.common.Config; +import org.apache.doris.common.DdlException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.FeMetaVersion; @@ -30,8 +31,11 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.io.Text; import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.common.util.KafkaUtil; import org.apache.doris.common.util.LogBuilder; import org.apache.doris.common.util.LogKey; +import org.apache.doris.common.util.SmallFileMgr; +import org.apache.doris.common.util.SmallFileMgr.SmallFile; import org.apache.doris.system.SystemInfoService; import com.google.common.base.Joiner; @@ -40,8 +44,6 @@ import com.google.gson.Gson; import com.google.gson.GsonBuilder; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.PartitionInfo; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -52,7 +54,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.UUID; /** @@ -62,7 +63,7 @@ public class KafkaRoutineLoadJob extends RoutineLoadJob { private static final Logger LOG = LogManager.getLogger(KafkaRoutineLoadJob.class); - private static final int FETCH_PARTITIONS_TIMEOUT_SECOND = 5; + public static final String KAFKA_FILE_CATALOG = "kafka"; private String brokerList; private String topic; @@ -70,11 +71,9 @@ public class KafkaRoutineLoadJob extends RoutineLoadJob { private List customKafkaPartitions = Lists.newArrayList(); // current kafka partitions is the actually partition which will be fetched private List currentKafkaPartitions = Lists.newArrayList(); - //kafka properties ,property prefix will be mapped to kafka custom parameters, which can be extended in the future - private Map customKafkaProperties = Maps.newHashMap(); - - // this is the kafka consumer which is used to fetch the number of partitions - private KafkaConsumer consumer; + // kafka properties ,property prefix will be mapped to kafka custom parameters, which can be extended in the future + private Map customProperties = Maps.newHashMap(); + private Map convertedCustomProperties = Maps.newHashMap(); public KafkaRoutineLoadJob() { // for serialization, id is dummy @@ -87,7 +86,6 @@ public KafkaRoutineLoadJob(Long id, String name, String clusterName, long dbId, this.brokerList = brokerList; this.topic = topic; this.progress = new KafkaProgress(); - setConsumer(); } public String getTopic() { @@ -98,8 +96,38 @@ public String getBrokerList() { return brokerList; } - public Map getCustomKafkaProperties() { - return customKafkaProperties; + public Map getConvertedCustomProperties() { + return convertedCustomProperties; + } + + public void resetConvertedCustomProperties() { + convertedCustomProperties.clear(); + } + + @Override + public void prepare() throws UserException { + super.prepare(); + // should reset converted properties each time the job being prepared. + // because the file info can be changed anytime. + resetConvertedCustomProperties(); + convertCustomProperties(); + } + + private void convertCustomProperties() throws DdlException { + if (!convertedCustomProperties.isEmpty() || customProperties.isEmpty()) { + return; + } + SmallFileMgr smallFileMgr = Catalog.getCurrentCatalog().getSmallFileMgr(); + for (Map.Entry entry : customProperties.entrySet()) { + if (entry.getValue().startsWith("FILE:")) { + // convert FILE:file_name -> FILE:file_id:md5 + String file = entry.getValue().substring(entry.getValue().indexOf(":") + 1); + SmallFile smallFile = smallFileMgr.getSmallFile(dbId, KAFKA_FILE_CATALOG, file, true); + convertedCustomProperties.put(entry.getKey(), "FILE:" + smallFile.id + ":" + smallFile.md5); + } else { + convertedCustomProperties.put(entry.getKey(), entry.getValue()); + } + } } @Override @@ -276,17 +304,9 @@ protected String getStatistic() { return gson.toJson(summary); } - private List getAllKafkaPartitions() throws LoadException { - List result = new ArrayList<>(); - try { - List partitionList = consumer.partitionsFor(topic); - for (PartitionInfo partitionInfo : partitionList) { - result.add(partitionInfo.partition()); - } - } catch (Exception e) { - throw new LoadException("failed to get partitions for topic: " + topic + ". " + e.getMessage()); - } - return result; + private List getAllKafkaPartitions() throws UserException { + convertCustomProperties(); + return KafkaUtil.getAllKafkaPartitions(brokerList, topic, convertedCustomProperties); } public static KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) throws UserException { @@ -305,17 +325,44 @@ public static KafkaRoutineLoadJob fromCreateStmt(CreateRoutineLoadStmt stmt) thr db.readUnlock(); } - // TODO(ml): check partition - // init kafka routine load job long id = Catalog.getInstance().getNextId(); KafkaRoutineLoadJob kafkaRoutineLoadJob = new KafkaRoutineLoadJob(id, stmt.getName(), db.getClusterName(), db.getId(), tableId, stmt.getKafkaBrokerList(), stmt.getKafkaTopic()); kafkaRoutineLoadJob.setOptional(stmt); + kafkaRoutineLoadJob.checkCustomProperties(); + kafkaRoutineLoadJob.checkCustomPartition(); return kafkaRoutineLoadJob; } + private void checkCustomPartition() throws UserException { + if (customKafkaPartitions.isEmpty()) { + return; + } + List allKafkaPartitions = getAllKafkaPartitions(); + for (Integer customPartition : customKafkaPartitions) { + if (!allKafkaPartitions.contains(customPartition)) { + throw new LoadException("there is a custom kafka partition " + customPartition + + " which is invalid for topic " + topic); + } + } + } + + private void checkCustomProperties() throws DdlException { + SmallFileMgr smallFileMgr = Catalog.getCurrentCatalog().getSmallFileMgr(); + for (Map.Entry entry : customProperties.entrySet()) { + if (entry.getValue().startsWith("FILE:")) { + String file = entry.getValue().substring(entry.getValue().indexOf(":") + 1); + // check file + if (!smallFileMgr.containsFile(dbId, KAFKA_FILE_CATALOG, file)) { + throw new DdlException("File " + file + " does not exist in db " + + dbId + " with catalog: " + KAFKA_FILE_CATALOG); + } + } + } + } + private void updateNewPartitionProgress() { // update the progress of new partitions for (Integer kafkaPartition : currentKafkaPartitions) { @@ -332,15 +379,6 @@ private void updateNewPartitionProgress() { } } - private void setConsumer() { - Properties props = new Properties(); - props.put("bootstrap.servers", this.brokerList); - props.put("group.id", UUID.randomUUID().toString()); - props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); - props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); - consumer = new KafkaConsumer<>(props); - } - @Override protected void setOptional(CreateRoutineLoadStmt stmt) throws UserException { super.setOptional(stmt); @@ -355,20 +393,14 @@ protected void setOptional(CreateRoutineLoadStmt stmt) throws UserException { // this is a unprotected method which is called in the initialization function private void setCustomKafkaPartitions(List> kafkaPartitionOffsets) throws LoadException { - // check if custom kafka partition is valid - List allKafkaPartitions = getAllKafkaPartitions(); for (Pair partitionOffset : kafkaPartitionOffsets) { - if (!allKafkaPartitions.contains(partitionOffset.first)) { - throw new LoadException("there is a custom kafka partition " + partitionOffset.first - + " which is invalid for topic " + topic); - } this.customKafkaPartitions.add(partitionOffset.first); ((KafkaProgress) progress).addPartitionOffset(partitionOffset); } } private void setCustomKafkaProperties(Map kafkaProperties) { - this.customKafkaProperties = kafkaProperties; + this.customProperties = kafkaProperties; } @Override @@ -386,7 +418,7 @@ protected String dataSourcePropertiesJsonToString() { @Override protected String customPropertiesJsonToString() { Gson gson = new GsonBuilder().disableHtmlEscaping().create(); - return gson.toJson(customKafkaProperties); + return gson.toJson(customProperties); } @Override @@ -400,8 +432,8 @@ public void write(DataOutput out) throws IOException { out.writeInt(partitionId); } - out.writeInt(customKafkaProperties.size()); - for (Map.Entry property : customKafkaProperties.entrySet()) { + out.writeInt(customProperties.size()); + for (Map.Entry property : customProperties.entrySet()) { Text.writeString(out, "property." + property.getKey()); Text.writeString(out, property.getValue()); } @@ -423,11 +455,9 @@ public void readFields(DataInput in) throws IOException { String propertyKey = Text.readString(in); String propertyValue = Text.readString(in); if (propertyKey.startsWith("property.")) { - this.customKafkaProperties.put(propertyKey.substring(propertyKey.indexOf(".") + 1), propertyValue); + this.customProperties.put(propertyKey.substring(propertyKey.indexOf(".") + 1), propertyValue); } } } - - setConsumer(); } } diff --git a/fe/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java b/fe/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java index bca0e9d8090029..a748d0cf3b9892 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/KafkaTaskInfo.java @@ -82,7 +82,7 @@ public TRoutineLoadTask createRoutineLoadTask() throws UserException { tKafkaLoadInfo.setTopic((routineLoadJob).getTopic()); tKafkaLoadInfo.setBrokers((routineLoadJob).getBrokerList()); tKafkaLoadInfo.setPartition_begin_offset(partitionIdToOffset); - tKafkaLoadInfo.setProperties((routineLoadJob).getCustomKafkaProperties()); + tKafkaLoadInfo.setProperties(routineLoadJob.getConvertedCustomProperties()); tRoutineLoadTask.setKafka_load_info(tKafkaLoadInfo); tRoutineLoadTask.setType(TLoadSourceType.KAFKA); tRoutineLoadTask.setParams(updateTExecPlanFragmentParams(routineLoadJob)); diff --git a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java index dcdbde75bbaf90..04890988fdf665 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadJob.java @@ -544,7 +544,13 @@ protected void replayUpdateProgress(RLTaskTxnCommitAttachment attachment) { abstract RoutineLoadTaskInfo unprotectRenewTask(RoutineLoadTaskInfo routineLoadTaskInfo); - public void initPlanner() throws UserException { + // call before first scheduling + // derived class can override this. + public void prepare() throws UserException { + initPlanner(); + } + + private void initPlanner() throws UserException { StreamLoadTask streamLoadTask = StreamLoadTask.fromRoutineLoadJob(this); Database db = Catalog.getCurrentCatalog().getDb(dbId); if (db == null) { @@ -702,8 +708,9 @@ public void afterAborted(TransactionState txnState, boolean txnOperated, String if (txnStatusChangeReason != null) { switch (txnStatusChangeReason) { case OFFSET_OUT_OF_RANGE: + case PAUSE: updateState(JobState.PAUSED, "be " + taskBeId + " abort task " - + "with reason " + txnStatusChangeReason.toString(), false /* not replay */); + + "with reason: " + txnStatusChangeReasonString, false /* not replay */); return; default: break; diff --git a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadScheduler.java b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadScheduler.java index 1aaf19a9d8ecfa..6eec6e91912d54 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadScheduler.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadScheduler.java @@ -75,8 +75,7 @@ private void process() throws UserException { RoutineLoadJob.JobState errorJobState = null; UserException userException = null; try { - // init the stream load planner - routineLoadJob.initPlanner(); + routineLoadJob.prepare(); // judge nums of tasks more then max concurrent tasks of cluster int desiredConcurrentTaskNum = routineLoadJob.calculateCurrentConcurrentTaskNum(); if (desiredConcurrentTaskNum <= 0) { diff --git a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java index 3fa493557ceeed..dcf12003f03848 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskInfo.java @@ -57,6 +57,9 @@ public abstract class RoutineLoadTaskInfo { // the be id of this task protected long beId = -1L; + // last time this task being scheduled by RoutineLoadTaskScheduler + protected long lastScheduledTime = -1; + public RoutineLoadTaskInfo(UUID id, long jobId, String clusterName) { this.id = id; this.jobId = jobId; @@ -113,6 +116,14 @@ public boolean isRunning() { return executeStartTimeMs > 0; } + public long getLastScheduledTime() { + return lastScheduledTime; + } + + public void setLastScheduledTime(long lastScheduledTime) { + this.lastScheduledTime = lastScheduledTime; + } + abstract TRoutineLoadTask createRoutineLoadTask() throws LoadException, UserException; public void beginTxn() throws LabelAlreadyUsedException, BeginTransactionException, AnalysisException { diff --git a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java index 75714dd2f6877f..c41b8d78511ac6 100644 --- a/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java +++ b/fe/src/main/java/org/apache/doris/load/routineload/RoutineLoadTaskScheduler.java @@ -22,7 +22,6 @@ import org.apache.doris.common.ClientPool; import org.apache.doris.common.LabelAlreadyUsedException; import org.apache.doris.common.LoadException; -import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.UserException; import org.apache.doris.common.util.Daemon; import org.apache.doris.common.util.LogBuilder; @@ -60,6 +59,7 @@ public class RoutineLoadTaskScheduler extends Daemon { private static final long BACKEND_SLOT_UPDATE_INTERVAL_MS = 10000; // 10s private static final long SLOT_FULL_SLEEP_MS = 1000; // 1s + private static final long MIN_SCHEDULE_INTERVAL_MS = 1000; // 1s private RoutineLoadManager routineLoadManager; private LinkedBlockingQueue needScheduleTasksQueue = Queues.newLinkedBlockingQueue(); @@ -105,6 +105,11 @@ private void process() throws UserException, InterruptedException { try { // This step will be blocked when queue is empty RoutineLoadTaskInfo routineLoadTaskInfo = needScheduleTasksQueue.take(); + if (System.currentTimeMillis() - routineLoadTaskInfo.getLastScheduledTime() < MIN_SCHEDULE_INTERVAL_MS) { + // delay this schedule, to void too many failure + needScheduleTasksQueue.put(routineLoadTaskInfo); + return; + } scheduleOneTask(routineLoadTaskInfo); } catch (InterruptedException e) { LOG.warn("Taking routine load task from queue has been interrupted", e); @@ -114,6 +119,7 @@ private void process() throws UserException, InterruptedException { private void scheduleOneTask(RoutineLoadTaskInfo routineLoadTaskInfo) throws InterruptedException, UserException { + routineLoadTaskInfo.setLastScheduledTime(System.currentTimeMillis()); // check if task has been abandoned if (!routineLoadManager.checkTaskInJob(routineLoadTaskInfo.getId())) { // task has been abandoned while renew task has been added in queue @@ -212,8 +218,7 @@ private void submitBatchTasksIfNotEmpty(Map> beIdTo // check if previous be has idle slot // true: allocate previous be to task // false: allocate the most idle be to task - private void allocateTaskToBe(RoutineLoadTaskInfo routineLoadTaskInfo) - throws MetaNotFoundException, LoadException { + private void allocateTaskToBe(RoutineLoadTaskInfo routineLoadTaskInfo) throws LoadException { if (routineLoadTaskInfo.getPreviousBeId() != -1L) { if (routineLoadManager.checkBeToTask(routineLoadTaskInfo.getPreviousBeId(), routineLoadTaskInfo.getClusterName())) { if (LOG.isDebugEnabled()) { diff --git a/fe/src/main/java/org/apache/doris/persist/EditLog.java b/fe/src/main/java/org/apache/doris/persist/EditLog.java index 2786080b3e6dfa..e37a069f4dd3e8 100644 --- a/fe/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/src/main/java/org/apache/doris/persist/EditLog.java @@ -37,6 +37,7 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; +import org.apache.doris.common.util.SmallFileMgr.SmallFile; import org.apache.doris.ha.MasterInfo; import org.apache.doris.journal.Journal; import org.apache.doris.journal.JournalCursor; @@ -691,6 +692,16 @@ public static void loadJournal(Catalog catalog, JournalEntity journal) { Catalog.getCurrentCatalog().getLoadManager().replayEndLoadJob(operation); break; } + case OperationType.OP_CREATE_SMALL_FILE: { + SmallFile smallFile = (SmallFile) journal.getData(); + Catalog.getCurrentCatalog().getSmallFileMgr().replayCreateFile(smallFile); + break; + } + case OperationType.OP_DROP_SMALL_FILE: { + SmallFile smallFile = (SmallFile) journal.getData(); + Catalog.getCurrentCatalog().getSmallFileMgr().replayRemoveFile(smallFile); + break; + } default: { IOException e = new IOException(); LOG.error("UNKNOWN Operation Type {}", opCode, e); @@ -1201,4 +1212,12 @@ public void logCreateLoadJob(org.apache.doris.load.loadv2.LoadJob loadJob) { public void logEndLoadJob(LoadJobFinalOperation loadJobFinalOperation) { logEdit(OperationType.OP_END_LOAD_JOB, loadJobFinalOperation); } + + public void logCreateSmallFile(SmallFile info) { + logEdit(OperationType.OP_CREATE_SMALL_FILE, info); + } + + public void logDropSmallFile(SmallFile info) { + logEdit(OperationType.OP_DROP_SMALL_FILE, info); + } } diff --git a/fe/src/main/java/org/apache/doris/persist/OperationType.java b/fe/src/main/java/org/apache/doris/persist/OperationType.java index 5fcc11862683ee..564bb5ebef6b3d 100644 --- a/fe/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/src/main/java/org/apache/doris/persist/OperationType.java @@ -166,4 +166,8 @@ public class OperationType { public static final short OP_CREATE_LOAD_JOB = 230; // this finish op include finished and cancelled public static final short OP_END_LOAD_JOB = 231; + + // small files 251~260 + public static final short OP_CREATE_SMALL_FILE = 251; + public static final short OP_DROP_SMALL_FILE = 252; } diff --git a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java index efc77b8ff6f824..a2c5bd532d43fd 100644 --- a/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -35,7 +35,7 @@ import org.apache.doris.mysql.MysqlPacket; import org.apache.doris.mysql.MysqlProto; import org.apache.doris.mysql.MysqlSerializer; -import org.apache.doris.rpc.PQueryStatistics; +import org.apache.doris.proto.PQueryStatistics; import org.apache.doris.thrift.TMasterOpRequest; import org.apache.doris.thrift.TMasterOpResult; @@ -95,16 +95,15 @@ private void handlePing() { ctx.getState().setOk(); } - private void auditAfterExec(String origStmt, StatementBase parsedStmt, - PQueryStatistics statistics) { + private void auditAfterExec(String origStmt, StatementBase parsedStmt, PQueryStatistics statistics) { // slow query long elapseMs = System.currentTimeMillis() - ctx.getStartTime(); // query state log ctx.getAuditBuilder().put("State", ctx.getState()); ctx.getAuditBuilder().put("Time", elapseMs); Preconditions.checkNotNull(statistics); - ctx.getAuditBuilder().put("ScanBytes", statistics.scanBytes); - ctx.getAuditBuilder().put("ScanRows", statistics.scanRows); + ctx.getAuditBuilder().put("ScanBytes", statistics.scan_bytes); + ctx.getAuditBuilder().put("ScanRows", statistics.scan_rows); ctx.getAuditBuilder().put("ReturnRows", ctx.getReturnRows()); ctx.getAuditBuilder().put("StmtId", ctx.getStmtId()); ctx.getAuditBuilder().put("QueryId", ctx.queryId() == null ? "NaN" : DebugUtil.printId(ctx.queryId())); diff --git a/fe/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/src/main/java/org/apache/doris/qe/Coordinator.java index c16f0e098c1d27..8d289b038d75ee 100644 --- a/fe/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/src/main/java/org/apache/doris/qe/Coordinator.java @@ -44,8 +44,8 @@ import org.apache.doris.planner.ResultSink; import org.apache.doris.planner.ScanNode; import org.apache.doris.planner.UnionNode; +import org.apache.doris.proto.PExecPlanFragmentResult; import org.apache.doris.rpc.BackendServiceProxy; -import org.apache.doris.rpc.PExecPlanFragmentResult; import org.apache.doris.rpc.RpcException; import org.apache.doris.service.FrontendOptions; import org.apache.doris.system.Backend; @@ -441,9 +441,9 @@ public void exec() throws Exception { try { PExecPlanFragmentResult result = pair.second.get(Config.remote_fragment_exec_timeout_ms, TimeUnit.MILLISECONDS); - code = TStatusCode.findByValue(result.status.code); - if (result.status.msgs != null && !result.status.msgs.isEmpty()) { - errMsg = result.status.msgs.get(0); + code = TStatusCode.findByValue(result.status.status_code); + if (result.status.error_msgs != null && !result.status.error_msgs.isEmpty()) { + errMsg = result.status.error_msgs.get(0); } } catch (ExecutionException e) { LOG.warn("catch a execute exception", e); diff --git a/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java b/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java index c4435cfb6bedc7..6f775e6d6cd6b2 100644 --- a/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/DdlExecutor.java @@ -32,6 +32,7 @@ import org.apache.doris.analysis.CancelLoadStmt; import org.apache.doris.analysis.CreateClusterStmt; import org.apache.doris.analysis.CreateDbStmt; +import org.apache.doris.analysis.CreateFileStmt; import org.apache.doris.analysis.CreateFunctionStmt; import org.apache.doris.analysis.CreateRepositoryStmt; import org.apache.doris.analysis.CreateRoleStmt; @@ -43,6 +44,7 @@ import org.apache.doris.analysis.DeleteStmt; import org.apache.doris.analysis.DropClusterStmt; import org.apache.doris.analysis.DropDbStmt; +import org.apache.doris.analysis.DropFileStmt; import org.apache.doris.analysis.DropFunctionStmt; import org.apache.doris.analysis.DropRepositoryStmt; import org.apache.doris.analysis.DropRoleStmt; @@ -192,8 +194,13 @@ public static void execute(Catalog catalog, DdlStmt ddlStmt, String origStmt) th catalog.getTabletChecker().cancelRepairTable((AdminCancelRepairTableStmt) ddlStmt); } else if (ddlStmt instanceof AdminSetConfigStmt) { catalog.setConfig((AdminSetConfigStmt) ddlStmt); + } else if (ddlStmt instanceof CreateFileStmt) { + catalog.getSmallFileMgr().createFile((CreateFileStmt) ddlStmt); + } else if (ddlStmt instanceof DropFileStmt) { + catalog.getSmallFileMgr().dropFile((DropFileStmt) ddlStmt); } else { throw new DdlException("Unknown statement."); + } } } diff --git a/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java b/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java index ed8a0323f7852a..c30941480953f2 100644 --- a/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java +++ b/fe/src/main/java/org/apache/doris/qe/ResultReceiver.java @@ -17,13 +17,11 @@ package org.apache.doris.qe; -import org.apache.doris.common.ClientPool; -import org.apache.doris.common.UserException; import org.apache.doris.common.Status; +import org.apache.doris.proto.PFetchDataResult; +import org.apache.doris.proto.PUniqueId; import org.apache.doris.rpc.BackendServiceProxy; import org.apache.doris.rpc.PFetchDataRequest; -import org.apache.doris.rpc.PFetchDataResult; -import org.apache.doris.rpc.PUniqueId; import org.apache.doris.rpc.RpcException; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TResultBatch; @@ -52,7 +50,9 @@ public class ResultReceiver { private Thread currentThread; public ResultReceiver(TUniqueId tid, Long backendId, TNetworkAddress address, int timeoutMs) { - this.finstId = new PUniqueId(tid); + this.finstId = new PUniqueId(); + this.finstId.hi = tid.hi; + this.finstId.lo = tid.lo; this.backendId = backendId; this.address = address; this.timeoutTs = System.currentTimeMillis() + timeoutMs; @@ -86,16 +86,16 @@ public RowBatch getNext(Status status) throws TException { } } } - TStatusCode code = TStatusCode.findByValue(pResult.status.code); + TStatusCode code = TStatusCode.findByValue(pResult.status.status_code); if (code != TStatusCode.OK) { status.setPstatus(pResult.status); return null; } - rowBatch.setQueryStatistics(pResult.statistics); + rowBatch.setQueryStatistics(pResult.query_statistics); - if (packetIdx != pResult.packetSeq) { - LOG.warn("receive packet failed, expect={}, receive={}", packetIdx, pResult.packetSeq); + if (packetIdx != pResult.packet_seq) { + LOG.warn("receive packet failed, expect={}, receive={}", packetIdx, pResult.packet_seq); status.setRpcStatus("receive error packet"); return null; } diff --git a/fe/src/main/java/org/apache/doris/qe/RowBatch.java b/fe/src/main/java/org/apache/doris/qe/RowBatch.java index 98bdfcf10888ab..b6dfce96ecead0 100644 --- a/fe/src/main/java/org/apache/doris/qe/RowBatch.java +++ b/fe/src/main/java/org/apache/doris/qe/RowBatch.java @@ -17,7 +17,7 @@ package org.apache.doris.qe; -import org.apache.doris.rpc.PQueryStatistics; +import org.apache.doris.proto.PQueryStatistics; import org.apache.doris.thrift.TResultBatch; public final class RowBatch { diff --git a/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java index c72cb17ff6bec7..61984cf46e3ed6 100644 --- a/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -17,7 +17,6 @@ package org.apache.doris.qe; -import com.google.common.base.Strings; import org.apache.doris.analysis.AdminShowConfigStmt; import org.apache.doris.analysis.AdminShowReplicaDistributionStmt; import org.apache.doris.analysis.AdminShowReplicaStatusStmt; @@ -53,6 +52,7 @@ import org.apache.doris.analysis.ShowRollupStmt; import org.apache.doris.analysis.ShowRoutineLoadStmt; import org.apache.doris.analysis.ShowRoutineLoadTaskStmt; +import org.apache.doris.analysis.ShowSmallFilesStmt; import org.apache.doris.analysis.ShowSnapshotStmt; import org.apache.doris.analysis.ShowStmt; import org.apache.doris.analysis.ShowTableStatusStmt; @@ -112,6 +112,7 @@ import org.apache.doris.system.SystemInfoService; 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.common.collect.Sets; @@ -230,6 +231,8 @@ public ShowResultSet execute() throws AnalysisException { handleAdminShowTabletDistribution(); } else if (stmt instanceof AdminShowConfigStmt) { handleAdminShowConfig(); + } else if (stmt instanceof ShowSmallFilesStmt) { + handleShowSmallFiles(); } else { handleEmtpy(); } @@ -1325,6 +1328,17 @@ private void handleAdminShowConfig() throws AnalysisException { resultSet = new ShowResultSet(showStmt.getMetaData(), results); } + private void handleShowSmallFiles() throws AnalysisException { + ShowSmallFilesStmt showStmt = (ShowSmallFilesStmt) stmt; + List> results; + try { + results = Catalog.getCurrentCatalog().getSmallFileMgr().getInfo(showStmt.getDbName()); + } catch (DdlException e) { + throw new AnalysisException(e.getMessage()); + } + resultSet = new ShowResultSet(showStmt.getMetaData(), results); + } + } diff --git a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java index d4ad71d607c5df..1e08f3c86b905d 100644 --- a/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -61,8 +61,8 @@ import org.apache.doris.mysql.MysqlSerializer; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.planner.Planner; +import org.apache.doris.proto.PQueryStatistics; import org.apache.doris.rewrite.ExprRewriter; -import org.apache.doris.rpc.PQueryStatistics; import org.apache.doris.rpc.RpcException; import org.apache.doris.task.LoadEtlTask; import org.apache.doris.thrift.TExplainLevel; @@ -802,6 +802,12 @@ public PQueryStatistics getQueryStatisticsForAuditLog() { if (statisticsForAuditLog == null) { statisticsForAuditLog = new PQueryStatistics(); } + if (statisticsForAuditLog.scan_bytes == null) { + statisticsForAuditLog.scan_bytes = 0L; + } + if (statisticsForAuditLog.scan_rows == null) { + statisticsForAuditLog.scan_rows = 0L; + } return statisticsForAuditLog; } } diff --git a/fe/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java b/fe/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java index 76da014b300e8d..e19e35fac0d5e6 100644 --- a/fe/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java +++ b/fe/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java @@ -17,14 +17,22 @@ package org.apache.doris.rpc; -import com.baidu.jprotobuf.pbrpc.client.ProtobufRpcProxy; -import com.baidu.jprotobuf.pbrpc.transport.RpcClient; -import com.baidu.jprotobuf.pbrpc.transport.RpcClientOptions; import org.apache.doris.common.Config; +import org.apache.doris.proto.PCancelPlanFragmentRequest; +import org.apache.doris.proto.PCancelPlanFragmentResult; +import org.apache.doris.proto.PExecPlanFragmentResult; +import org.apache.doris.proto.PFetchDataResult; +import org.apache.doris.proto.PProxyRequest; +import org.apache.doris.proto.PProxyResult; +import org.apache.doris.proto.PTriggerProfileReportResult; +import org.apache.doris.proto.PUniqueId; import org.apache.doris.thrift.TExecPlanFragmentParams; import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TUniqueId; +import com.baidu.jprotobuf.pbrpc.client.ProtobufRpcProxy; +import com.baidu.jprotobuf.pbrpc.transport.RpcClient; +import com.baidu.jprotobuf.pbrpc.transport.RpcClientOptions; import com.google.common.collect.Maps; import org.apache.logging.log4j.LogManager; @@ -104,7 +112,11 @@ public Future execPlanFragmentAsync( public Future cancelPlanFragmentAsync( TNetworkAddress address, TUniqueId finstId) throws RpcException { - final PCancelPlanFragmentRequest pRequest = new PCancelPlanFragmentRequest(new PUniqueId(finstId));; + final PCancelPlanFragmentRequest pRequest = new PCancelPlanFragmentRequest(); + PUniqueId uid = new PUniqueId(); + uid.hi = finstId.hi; + uid.lo = finstId.lo; + pRequest.finst_id = uid; try { final PBackendService service = getProxy(address); return service.cancelPlanFragmentAsync(pRequest); @@ -153,4 +165,15 @@ public Future triggerProfileReportAsync( throw new RpcException(e.getMessage()); } } + + public Future getInfo( + TNetworkAddress address, PProxyRequest request) throws RpcException { + try { + final PBackendService service = getProxy(address); + return service.getInfo(request); + } catch (Throwable e) { + LOG.warn("failed to get info, address={}:{}", address.getHostname(), address.getPort(), e); + throw new RpcException(e.getMessage()); + } + } } diff --git a/fe/src/main/java/org/apache/doris/rpc/PBackendService.java b/fe/src/main/java/org/apache/doris/rpc/PBackendService.java index 5050debdf1ea7e..38bc2e701262e7 100644 --- a/fe/src/main/java/org/apache/doris/rpc/PBackendService.java +++ b/fe/src/main/java/org/apache/doris/rpc/PBackendService.java @@ -17,6 +17,14 @@ package org.apache.doris.rpc; +import org.apache.doris.proto.PCancelPlanFragmentRequest; +import org.apache.doris.proto.PCancelPlanFragmentResult; +import org.apache.doris.proto.PExecPlanFragmentResult; +import org.apache.doris.proto.PFetchDataResult; +import org.apache.doris.proto.PProxyRequest; +import org.apache.doris.proto.PProxyResult; +import org.apache.doris.proto.PTriggerProfileReportResult; + import com.baidu.jprotobuf.pbrpc.ProtobufRPC; import java.util.concurrent.Future; @@ -38,4 +46,8 @@ public interface PBackendService { @ProtobufRPC(serviceName = "PBackendService", methodName = "trigger_profile_report", attachmentHandler = ThriftClientAttachmentHandler.class, onceTalkTimeout = 10000) Future triggerProfileReport(PTriggerProfileReportRequest request); + + @ProtobufRPC(serviceName = "PBackendService", methodName = "get_info", onceTalkTimeout = 10000) + Future getInfo(PProxyRequest request); } + diff --git a/fe/src/main/java/org/apache/doris/rpc/PCancelPlanFragmentRequest.java b/fe/src/main/java/org/apache/doris/rpc/PCancelPlanFragmentRequest.java deleted file mode 100644 index 266ffa254298f9..00000000000000 --- a/fe/src/main/java/org/apache/doris/rpc/PCancelPlanFragmentRequest.java +++ /dev/null @@ -1,32 +0,0 @@ -// 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.rpc; - -import com.baidu.bjf.remoting.protobuf.annotation.Protobuf; -import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; - -@ProtobufClass -public class PCancelPlanFragmentRequest { - public PCancelPlanFragmentRequest() { } - public PCancelPlanFragmentRequest(PUniqueId finstId) { - this.finstId = finstId; - } - - @Protobuf(order = 1, required = true) - public PUniqueId finstId; -} diff --git a/fe/src/main/java/org/apache/doris/rpc/PCancelPlanFragmentResult.java b/fe/src/main/java/org/apache/doris/rpc/PCancelPlanFragmentResult.java deleted file mode 100644 index 5f31c631266292..00000000000000 --- a/fe/src/main/java/org/apache/doris/rpc/PCancelPlanFragmentResult.java +++ /dev/null @@ -1,27 +0,0 @@ -// 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.rpc; - -import com.baidu.bjf.remoting.protobuf.annotation.Protobuf; -import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; - -@ProtobufClass -public class PCancelPlanFragmentResult { - @Protobuf(order = 1, required = true) - public PStatus status; -} diff --git a/fe/src/main/java/org/apache/doris/rpc/PExecPlanFragmentResult.java b/fe/src/main/java/org/apache/doris/rpc/PExecPlanFragmentResult.java deleted file mode 100644 index b2026f225e4e4f..00000000000000 --- a/fe/src/main/java/org/apache/doris/rpc/PExecPlanFragmentResult.java +++ /dev/null @@ -1,27 +0,0 @@ -// 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.rpc; - -import com.baidu.bjf.remoting.protobuf.annotation.Protobuf; -import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; - -@ProtobufClass -public class PExecPlanFragmentResult { - @Protobuf(order = 1, required = true) - public PStatus status; -} diff --git a/fe/src/main/java/org/apache/doris/rpc/PFetchDataRequest.java b/fe/src/main/java/org/apache/doris/rpc/PFetchDataRequest.java index cf26e71359fd8b..7cad9d44182559 100644 --- a/fe/src/main/java/org/apache/doris/rpc/PFetchDataRequest.java +++ b/fe/src/main/java/org/apache/doris/rpc/PFetchDataRequest.java @@ -17,6 +17,8 @@ package org.apache.doris.rpc; +import org.apache.doris.proto.PUniqueId; + import com.baidu.bjf.remoting.protobuf.annotation.Protobuf; import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; diff --git a/fe/src/main/java/org/apache/doris/rpc/PFetchDataResult.java b/fe/src/main/java/org/apache/doris/rpc/PFetchDataResult.java deleted file mode 100644 index 389327ea62c766..00000000000000 --- a/fe/src/main/java/org/apache/doris/rpc/PFetchDataResult.java +++ /dev/null @@ -1,33 +0,0 @@ -// 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.rpc; - -import com.baidu.bjf.remoting.protobuf.annotation.Protobuf; -import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; - -@ProtobufClass -public class PFetchDataResult { - @Protobuf(order = 1, required = true) - public PStatus status; - @Protobuf(order = 2, required = false) - public long packetSeq; - @Protobuf(order = 3, required = false) - public boolean eos; - @Protobuf(order = 4, required = false) - public PQueryStatistics statistics; -} diff --git a/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java b/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java deleted file mode 100644 index 333ded38c820ca..00000000000000 --- a/fe/src/main/java/org/apache/doris/rpc/PQueryStatistics.java +++ /dev/null @@ -1,35 +0,0 @@ -// 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.rpc; - -import com.baidu.bjf.remoting.protobuf.annotation.Protobuf; -import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; - -@ProtobufClass -public class PQueryStatistics { - @Protobuf(order = 1, required = false) - public long scanRows; - @Protobuf(order = 2, required = false) - public long scanBytes; - - public PQueryStatistics() { - scanRows = 0; - scanBytes = 0; - } - -} diff --git a/fe/src/main/java/org/apache/doris/rpc/PStatus.java b/fe/src/main/java/org/apache/doris/rpc/PStatus.java deleted file mode 100644 index 920c5bf511e773..00000000000000 --- a/fe/src/main/java/org/apache/doris/rpc/PStatus.java +++ /dev/null @@ -1,31 +0,0 @@ -// 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.rpc; - -import com.baidu.bjf.remoting.protobuf.annotation.Protobuf; -import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; - -import java.util.List; - -@ProtobufClass -public class PStatus { - @Protobuf(order = 1, required = true) - public int code; - @Protobuf(order = 2) - public List msgs; -} diff --git a/fe/src/main/java/org/apache/doris/rpc/PTriggerProfileReportRequest.java b/fe/src/main/java/org/apache/doris/rpc/PTriggerProfileReportRequest.java index 3c948a6346efdf..2bfce900273f83 100644 --- a/fe/src/main/java/org/apache/doris/rpc/PTriggerProfileReportRequest.java +++ b/fe/src/main/java/org/apache/doris/rpc/PTriggerProfileReportRequest.java @@ -17,6 +17,8 @@ package org.apache.doris.rpc; +import org.apache.doris.proto.PUniqueId; + import com.baidu.bjf.remoting.protobuf.FieldType; import com.baidu.bjf.remoting.protobuf.annotation.Protobuf; import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; diff --git a/fe/src/main/java/org/apache/doris/rpc/PTriggerProfileReportResult.java b/fe/src/main/java/org/apache/doris/rpc/PTriggerProfileReportResult.java deleted file mode 100644 index 80e8c6e13f6c55..00000000000000 --- a/fe/src/main/java/org/apache/doris/rpc/PTriggerProfileReportResult.java +++ /dev/null @@ -1,30 +0,0 @@ -// 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.rpc; - -import com.baidu.bjf.remoting.protobuf.annotation.Protobuf; -import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; - -@ProtobufClass -public class PTriggerProfileReportResult { - @Protobuf(order = 1, required = true) - public PStatus status; - - public PTriggerProfileReportResult() { - } -} diff --git a/fe/src/main/java/org/apache/doris/rpc/PUniqueId.java b/fe/src/main/java/org/apache/doris/rpc/PUniqueId.java deleted file mode 100644 index 1eea967eddcaf4..00000000000000 --- a/fe/src/main/java/org/apache/doris/rpc/PUniqueId.java +++ /dev/null @@ -1,68 +0,0 @@ -// 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.rpc; - -import com.baidu.bjf.remoting.protobuf.annotation.Protobuf; -import com.baidu.bjf.remoting.protobuf.annotation.ProtobufClass; -import org.apache.doris.common.util.DebugUtil; -import org.apache.doris.thrift.TUniqueId; - -@ProtobufClass -public class PUniqueId { - - public PUniqueId() {} - public PUniqueId(TUniqueId tid) { - hi = tid.getHi(); - lo = tid.getLo(); - } - - @Protobuf(order = 1, required = true) - public long hi; - @Protobuf(order = 2, required = true) - public long lo; - - @Override - public int hashCode() { - int result = 16; - result = 31 * result + (int)(hi ^ (hi >>> 32)); - result = 31 * result + (int)(lo ^ (lo >>> 32)); - return result; - } - - @Override - public String toString() { - return DebugUtil.printId(this); - } - - @Override - public boolean equals(Object obj) { - if (obj == this) { - return true; - } - - if (!(obj instanceof PUniqueId) || obj == null) { - return false; - } - - final PUniqueId other = (PUniqueId)obj; - if (hi != other.hi || lo != other.lo) { - return false; - } - return true; - } -} diff --git a/fe/src/main/java/org/apache/doris/system/HeartbeatMgr.java b/fe/src/main/java/org/apache/doris/system/HeartbeatMgr.java index 381f38d075a991..e95551c7a28aaf 100644 --- a/fe/src/main/java/org/apache/doris/system/HeartbeatMgr.java +++ b/fe/src/main/java/org/apache/doris/system/HeartbeatMgr.java @@ -26,6 +26,7 @@ import org.apache.doris.common.util.Util; import org.apache.doris.http.rest.BootstrapFinishAction; import org.apache.doris.persist.HbPackage; +import org.apache.doris.service.FrontendOptions; import org.apache.doris.system.HeartbeatResponse.HbStatus; import org.apache.doris.thrift.HeartbeatService; import org.apache.doris.thrift.TBackendInfo; @@ -74,9 +75,11 @@ public HeartbeatMgr(SystemInfoService nodeMgr) { this.executor = Executors.newCachedThreadPool(); } - public void setMaster(String masterHost, int masterPort, int clusterId, String token, long epoch) { - TMasterInfo tMasterInfo = new TMasterInfo(new TNetworkAddress(masterHost, masterPort), clusterId, epoch); + public void setMaster(int clusterId, String token, long epoch) { + TMasterInfo tMasterInfo = new TMasterInfo( + new TNetworkAddress(FrontendOptions.getLocalHostAddress(), Config.rpc_port), clusterId, epoch); tMasterInfo.setToken(token); + tMasterInfo.setHttp_port(Config.http_port); masterInfo.set(tMasterInfo); } @@ -218,7 +221,6 @@ public HeartbeatResponse call() { TBackendInfo tBackendInfo = result.getBackend_info(); int bePort = tBackendInfo.getBe_port(); int httpPort = tBackendInfo.getHttp_port(); - int beRpcPort = tBackendInfo.getBe_rpc_port(); int brpcPort = -1; if (tBackendInfo.isSetBrpc_port()) { brpcPort = tBackendInfo.getBrpc_port(); diff --git a/fe/src/main/java/org/apache/doris/transaction/TransactionState.java b/fe/src/main/java/org/apache/doris/transaction/TransactionState.java index 7fef992c30d36d..41e1832bf239cf 100644 --- a/fe/src/main/java/org/apache/doris/transaction/TransactionState.java +++ b/fe/src/main/java/org/apache/doris/transaction/TransactionState.java @@ -79,7 +79,8 @@ public static LoadJobSourceType valueOf(int flag) { public enum TxnStatusChangeReason { DB_DROPPED, TIMEOUT, - OFFSET_OUT_OF_RANGE; + OFFSET_OUT_OF_RANGE, + PAUSE; public static TxnStatusChangeReason fromString(String reasonString) { for (TxnStatusChangeReason txnStatusChangeReason : TxnStatusChangeReason.values()) { diff --git a/fe/src/main/jflex/sql_scanner.flex b/fe/src/main/jflex/sql_scanner.flex index 7e217cbb15c724..07d1fb61999444 100644 --- a/fe/src/main/jflex/sql_scanner.flex +++ b/fe/src/main/jflex/sql_scanner.flex @@ -148,6 +148,7 @@ import org.apache.doris.common.util.SqlUtils; keywordMap.put("external", new Integer(SqlParserSymbols.KW_EXTERNAL)); keywordMap.put("extract", new Integer(SqlParserSymbols.KW_EXTRACT)); keywordMap.put("false", new Integer(SqlParserSymbols.KW_FALSE)); + keywordMap.put("file", new Integer(SqlParserSymbols.KW_FILE)); keywordMap.put("first", new Integer(SqlParserSymbols.KW_FIRST)); keywordMap.put("float", new Integer(SqlParserSymbols.KW_FLOAT)); keywordMap.put("follower", new Integer(SqlParserSymbols.KW_FOLLOWER)); diff --git a/fe/src/test/java/org/apache/doris/common/GenericPoolTest.java b/fe/src/test/java/org/apache/doris/common/GenericPoolTest.java index a7d04e3c0be6e6..c4a45867e3475c 100644 --- a/fe/src/test/java/org/apache/doris/common/GenericPoolTest.java +++ b/fe/src/test/java/org/apache/doris/common/GenericPoolTest.java @@ -259,7 +259,6 @@ public void testSetMaxPerKey() throws Exception { flag = true; // pass } catch (Exception e) { - // can't get here Assert.fail(); } Assert.assertTrue(flag); diff --git a/fe/src/test/java/org/apache/doris/common/util/SmallFileMgrTest.java b/fe/src/test/java/org/apache/doris/common/util/SmallFileMgrTest.java new file mode 100644 index 00000000000000..85dfe96e013fd7 --- /dev/null +++ b/fe/src/test/java/org/apache/doris/common/util/SmallFileMgrTest.java @@ -0,0 +1,131 @@ +// 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.common.util; + +import org.apache.doris.analysis.CreateFileStmt; +import org.apache.doris.catalog.Catalog; +import org.apache.doris.catalog.Database; +import org.apache.doris.common.Config; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.util.SmallFileMgr.SmallFile; +import org.apache.doris.persist.EditLog; + +import org.junit.Assert; +import org.junit.Test; + +import mockit.Deencapsulation; +import mockit.Expectations; +import mockit.Injectable; +import mockit.Mocked; + +public class SmallFileMgrTest { + + @Mocked + Catalog catalog; + @Mocked + EditLog editLog; + @Mocked + Database db; + + @Test + public void test(@Injectable CreateFileStmt stmt1, @Injectable CreateFileStmt stmt2) throws DdlException { + new Expectations() { + { + db.getId(); + result = 1L; + catalog.getDb(anyString); + result = db; + stmt1.getDbName(); + result = "db1"; + stmt1.getFileName(); + result = "file1"; + stmt1.getCatalogName(); + result = "kafka"; + stmt1.getDownloadUrl(); + result = "http://127.0.0.1:8001/file1"; + + stmt2.getDbName(); + result = "db1"; + stmt2.getFileName(); + result = "file2"; + stmt2.getCatalogName(); + result = "kafka"; + stmt2.getDownloadUrl(); + result = "http://127.0.0.1:8001/file2"; + } + }; + + SmallFile smallFile = new SmallFile(1L, "kafka", "file1", 10001L, "ABCD", 12, "12345", true); + final SmallFileMgr smallFileMgr = new SmallFileMgr(); + new Expectations(SmallFileMgr.class) { + { + Deencapsulation.invoke(smallFileMgr, "downloadAndCheck", anyLong, anyString, anyString, anyString, + anyString, anyBoolean); + result = smallFile; + } + }; + + // 1. test create + try { + smallFileMgr.createFile(stmt1); + } catch (DdlException e) { + e.printStackTrace(); + Assert.fail(e.getMessage()); + } + + Assert.assertTrue(smallFileMgr.containsFile(1L, "kafka", "file1")); + SmallFile gotFile = smallFileMgr.getSmallFile(1L, "kafka", "file1", true); + Assert.assertEquals(10001L, gotFile.id); + gotFile = smallFileMgr.getSmallFile(10001L); + Assert.assertEquals(10001L, gotFile.id); + + // 2. test file num limit + Config.max_small_file_number = 1; + boolean fail = false; + try { + smallFileMgr.createFile(stmt2); + } catch (DdlException e) { + fail = true; + Assert.assertTrue(e.getMessage().contains("File number exceeds limit")); + } + Assert.assertTrue(fail); + + // 3. test remove + try { + smallFileMgr.removeFile(2L, "kafka", "file1", true); + } catch (DdlException e) { + // this is expected + } + gotFile = smallFileMgr.getSmallFile(10001L); + Assert.assertEquals(10001L, gotFile.id); + smallFileMgr.removeFile(1L, "kafka", "file1", true); + gotFile = smallFileMgr.getSmallFile(10001L); + Assert.assertNull(gotFile); + + // 4. test file limit again + try { + smallFileMgr.createFile(stmt1); + } catch (DdlException e) { + e.printStackTrace(); + Assert.fail(e.getMessage()); + } + gotFile = smallFileMgr.getSmallFile(10001L); + Assert.assertEquals(10001L, gotFile.id); + } + +} diff --git a/fe/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java b/fe/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java index b64ae82ad0f98c..5f1027bd5ae690 100644 --- a/fe/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java +++ b/fe/src/test/java/org/apache/doris/load/routineload/KafkaRoutineLoadJobTest.java @@ -22,7 +22,6 @@ import org.apache.doris.analysis.LabelName; import org.apache.doris.analysis.ParseNode; import org.apache.doris.analysis.PartitionNames; -import org.apache.doris.analysis.TableName; import org.apache.doris.catalog.Catalog; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.OlapTable; @@ -33,6 +32,7 @@ import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.KafkaUtil; import org.apache.doris.load.RoutineLoadDesc; import org.apache.doris.qe.ConnectContext; import org.apache.doris.system.SystemInfoService; @@ -44,7 +44,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -61,15 +60,14 @@ import mockit.Deencapsulation; import mockit.Expectations; import mockit.Injectable; +import mockit.Mock; +import mockit.MockUp; import mockit.Mocked; import mockit.Verifications; public class KafkaRoutineLoadJobTest { - private static final Logger LOG = LogManager.getLogger(KafkaRoutineLoadJobTest.class); - private static final int DEFAULT_TASK_TIMEOUT_SECONDS = 10; - private String jobName = "job1"; private String dbName = "db1"; private LabelName labelName = new LabelName(dbName, jobName); @@ -86,8 +84,6 @@ public class KafkaRoutineLoadJobTest { ConnectContext connectContext; @Mocked TResourceInfo tResourceInfo; - @Mocked - KafkaConsumer kafkaConsumer; @Before public void init() { @@ -97,8 +93,7 @@ public void init() { } @Test - public void testBeNumMin(@Mocked KafkaConsumer kafkaConsumer, - @Injectable PartitionInfo partitionInfo1, + public void testBeNumMin(@Injectable PartitionInfo partitionInfo1, @Injectable PartitionInfo partitionInfo2, @Mocked Catalog catalog, @Mocked SystemInfoService systemInfoService, @@ -130,36 +125,31 @@ public void testBeNumMin(@Mocked KafkaConsumer kafkaConsumer, RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(1L, "kafka_routine_load_job", clusterName1, 1L, 1L, "127.0.0.1:9020", "topic1"); - Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); Deencapsulation.setField(routineLoadJob, "currentKafkaPartitions", partitionList1); Assert.assertEquals(1, routineLoadJob.calculateCurrentConcurrentTaskNum()); // 3 partitions, 4 be routineLoadJob = new KafkaRoutineLoadJob(1L, "kafka_routine_load_job", clusterName2, 1L, 1L, "127.0.0.1:9020", "topic1"); - Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); Deencapsulation.setField(routineLoadJob, "currentKafkaPartitions", partitionList2); Assert.assertEquals(3, routineLoadJob.calculateCurrentConcurrentTaskNum()); // 4 partitions, 4 be routineLoadJob = new KafkaRoutineLoadJob(1L, "kafka_routine_load_job", clusterName2, 1L, 1L, "127.0.0.1:9020", "topic1"); - Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); Deencapsulation.setField(routineLoadJob, "currentKafkaPartitions", partitionList3); Assert.assertEquals(4, routineLoadJob.calculateCurrentConcurrentTaskNum()); // 7 partitions, 4 be routineLoadJob = new KafkaRoutineLoadJob(1L, "kafka_routine_load_job", clusterName2, 1L, 1L, "127.0.0.1:9020", "topic1"); - Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); Deencapsulation.setField(routineLoadJob, "currentKafkaPartitions", partitionList4); Assert.assertEquals(4, routineLoadJob.calculateCurrentConcurrentTaskNum()); } @Test - public void testDivideRoutineLoadJob(@Mocked KafkaConsumer kafkaConsumer, - @Injectable GlobalTransactionMgr globalTransactionMgr, + public void testDivideRoutineLoadJob(@Injectable GlobalTransactionMgr globalTransactionMgr, @Mocked Catalog catalog, @Injectable RoutineLoadManager routineLoadManager, @Injectable RoutineLoadTaskScheduler routineLoadTaskScheduler, @@ -180,7 +170,6 @@ public void testDivideRoutineLoadJob(@Mocked KafkaConsumer kafkaConsumer, }; Deencapsulation.setField(routineLoadJob, "currentKafkaPartitions", Arrays.asList(1, 4, 6)); - Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); routineLoadJob.divideRoutineLoadJob(2); @@ -293,8 +282,14 @@ public void testFromCreateStmt(@Mocked Catalog catalog, result = tableId; table.getType(); result = Table.TableType.OLAP; - kafkaConsumer.partitionsFor(anyString); - result = kafkaPartitionInfoList; + } + }; + + new MockUp() { + @Mock + public List getAllKafkaPartitions(String brokerList, String topic, + Map convertedCustomProperties) throws UserException { + return Lists.newArrayList(1, 2, 3); } }; @@ -309,7 +304,6 @@ public void testFromCreateStmt(@Mocked Catalog catalog, } private CreateRoutineLoadStmt initCreateRoutineLoadStmt() { - TableName tableName = new TableName(dbName, tableNameString); List loadPropertyList = new ArrayList<>(); loadPropertyList.add(columnSeparator); loadPropertyList.add(partitionNames); diff --git a/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java index 667e4ec017f500..7de3c51d3e7e67 100644 --- a/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java +++ b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadJobTest.java @@ -27,6 +27,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Table; import org.apache.doris.common.UserException; +import org.apache.doris.common.util.KafkaUtil; import org.apache.doris.persist.EditLog; import org.apache.doris.transaction.TransactionException; import org.apache.doris.transaction.TransactionState; @@ -34,7 +35,6 @@ import com.google.common.base.Strings; import com.google.common.collect.Lists; -import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.junit.Assert; import org.junit.Test; @@ -47,6 +47,8 @@ import mockit.Deencapsulation; import mockit.Expectations; import mockit.Injectable; +import mockit.Mock; +import mockit.MockUp; import mockit.Mocked; public class RoutineLoadJobTest { @@ -61,8 +63,6 @@ public class RoutineLoadJobTest { CreateRoutineLoadStmt createRoutineLoadStmt; @Mocked Symbol symbol; - @Mocked - KafkaConsumer kafkaConsumer; @Test public void testAfterAbortedReasonOffsetOutOfRange(@Injectable TransactionState transactionState, @@ -212,14 +212,18 @@ public void testUpdateWhilePartitionChanged(@Injectable Database database, result = database; database.getTable(anyLong); result = table; - kafkaConsumer.partitionsFor(anyString); - result = partitionInfoList; - partitionInfo.partition(); - result = 1; } }; + + new MockUp() { + @Mock + public List getAllKafkaPartitions(String brokerList, String topic, + Map convertedCustomProperties) throws UserException { + return Lists.newArrayList(1, 2, 3); + } + }; + RoutineLoadJob routineLoadJob = new KafkaRoutineLoadJob(); - Deencapsulation.setField(routineLoadJob, "consumer", kafkaConsumer); Deencapsulation.setField(routineLoadJob, "state", RoutineLoadJob.JobState.RUNNING); Deencapsulation.setField(routineLoadJob, "progress", kafkaProgress); routineLoadJob.update(); diff --git a/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java index e14f2383597a46..21e9965e732322 100644 --- a/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java +++ b/fe/src/test/java/org/apache/doris/load/routineload/RoutineLoadSchedulerTest.java @@ -31,7 +31,6 @@ import com.google.common.collect.Lists; -import org.apache.kafka.clients.consumer.KafkaConsumer; import org.junit.Assert; import org.junit.Test; @@ -53,8 +52,7 @@ public class RoutineLoadSchedulerTest { TResourceInfo tResourceInfo; @Test - public void testNormalRunOneCycle(@Mocked KafkaConsumer consumer, - @Mocked Catalog catalog, + public void testNormalRunOneCycle(@Mocked Catalog catalog, @Injectable RoutineLoadManager routineLoadManager, @Injectable SystemInfoService systemInfoService, @Injectable Database database, @@ -81,7 +79,6 @@ public void testNormalRunOneCycle(@Mocked KafkaConsumer consumer, Deencapsulation.setField(kafkaRoutineLoadJob, "customKafkaPartitions", partitions); Deencapsulation.setField(kafkaRoutineLoadJob, "desireTaskConcurrentNum", 3); - Deencapsulation.setField(kafkaRoutineLoadJob, "consumer", consumer); new Expectations() { { diff --git a/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java b/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java index d1085ef261fdfa..c029bac849a042 100644 --- a/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java +++ b/fe/src/test/java/org/apache/doris/qe/ConnectProcessorTest.java @@ -26,7 +26,7 @@ import org.apache.doris.mysql.MysqlErrPacket; import org.apache.doris.mysql.MysqlOkPacket; import org.apache.doris.mysql.MysqlSerializer; -import org.apache.doris.rpc.PQueryStatistics; +import org.apache.doris.proto.PQueryStatistics; import org.apache.doris.thrift.TUniqueId; import org.easymock.EasyMock; @@ -56,6 +56,8 @@ public class ConnectProcessorTest { private static AuditBuilder auditBuilder = new AuditBuilder(); ConnectContext myContext; + private static PQueryStatistics statistics = new PQueryStatistics(); + @BeforeClass public static void setUpClass() { // Init Database packet @@ -97,6 +99,9 @@ public static void setUpClass() { fieldListPacket = serializer.toByteBuffer(); } + statistics.scan_bytes = 0L; + statistics.scan_rows = 0L; + MetricRepo.init(); } @@ -233,7 +238,7 @@ public void testQuery() throws Exception { // Mock statement executor StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); - EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new PQueryStatistics()); + EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(statistics); EasyMock.expectLastCall().anyTimes(); EasyMock.replay(qe); PowerMock.expectNew( @@ -257,7 +262,7 @@ public void testQueryFail() throws Exception { StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); EasyMock.expectLastCall().andThrow(new IOException("Fail")).anyTimes(); - EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new PQueryStatistics()); + EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(statistics); EasyMock.replay(qe); PowerMock.expectNew(StmtExecutor.class, EasyMock.isA(ConnectContext.class), EasyMock.isA(String.class)) .andReturn(qe).anyTimes(); @@ -275,7 +280,7 @@ public void testQueryFail2() throws Exception { // Mock statement executor StmtExecutor qe = EasyMock.createNiceMock(StmtExecutor.class); qe.execute(); - EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(new PQueryStatistics()); + EasyMock.expect(qe.getQueryStatisticsForAuditLog()).andReturn(statistics); EasyMock.expectLastCall().andThrow(new NullPointerException("Fail")).anyTimes(); EasyMock.replay(qe); PowerMock.expectNew(StmtExecutor.class, EasyMock.isA(ConnectContext.class), EasyMock.isA(String.class)) diff --git a/gensrc/proto/Makefile b/gensrc/proto/Makefile index c8f93d1a804ffe..7104e960f01dff 100644 --- a/gensrc/proto/Makefile +++ b/gensrc/proto/Makefile @@ -25,11 +25,20 @@ SOURCES = $(shell find ${CURDIR} -name "*.proto") OBJECTS = $(patsubst ${CURDIR}/%.proto, ${BUILD_DIR}/gen_cpp/%.pb.cc, ${SOURCES}) HEADERS = $(patsubst ${CURDIR}/%.proto, ${BUILD_DIR}/gen_cpp/%.pb.h, ${SOURCES}) +#JAVA_OBJECTS = $(patsubst ${CURDIR}/%.proto, ${BUILD_DIR}/java/org/apache/doris/proto/%.java, ${SOURCES}) + +#all: ${JAVA_OBJECTS} ${OBJECTS} ${HEADERS} all: ${OBJECTS} ${HEADERS} .PHONY: all ${BUILD_DIR}/gen_cpp/%.pb.h ${BUILD_DIR}/gen_cpp/%.pb.cc: ${CURDIR}/%.proto | ${BUILD_DIR}/gen_cpp ${PROTOC} --proto_path=${CURDIR} --cpp_out=${BUILD_DIR}/gen_cpp $< +#${BUILD_DIR}/java/org/apache/doris/proto/%.java: ${CURDIR}/%.proto | ${BUILD_DIR}/java +# ${PROTOC} --proto_path=${CURDIR} --java_out=${BUILD_DIR}/java/ $< + ${BUILD_DIR}/gen_cpp: mkdir -p $@ + +#${BUILD_DIR}/java: +# mkdir -p $@ diff --git a/gensrc/proto/column_data_file.proto b/gensrc/proto/column_data_file.proto index e6e35835546add..8bffbcc9b78a9f 100644 --- a/gensrc/proto/column_data_file.proto +++ b/gensrc/proto/column_data_file.proto @@ -15,7 +15,10 @@ // specific language governing permissions and limitations // under the License. +syntax="proto2"; + package doris; +option java_package = "org.apache.doris.proto"; import "olap_common.proto"; diff --git a/gensrc/proto/data.proto b/gensrc/proto/data.proto index d690fb6c58ddd4..ef13a405988870 100644 --- a/gensrc/proto/data.proto +++ b/gensrc/proto/data.proto @@ -18,6 +18,7 @@ syntax="proto2"; package doris; +option java_package = "org.apache.doris.proto"; message PQueryStatistics { optional int64 scan_rows = 1; diff --git a/gensrc/proto/descriptors.proto b/gensrc/proto/descriptors.proto index a2c94acd4e7718..3dab3f0beec82b 100644 --- a/gensrc/proto/descriptors.proto +++ b/gensrc/proto/descriptors.proto @@ -18,6 +18,7 @@ syntax="proto2"; package doris; +option java_package = "org.apache.doris.proto"; import "types.proto"; diff --git a/gensrc/proto/internal_service.proto b/gensrc/proto/internal_service.proto index 61ea6f3f6ca241..4b57fb9b8e9f2e 100644 --- a/gensrc/proto/internal_service.proto +++ b/gensrc/proto/internal_service.proto @@ -18,6 +18,7 @@ syntax="proto2"; package doris; +option java_package = "org.apache.doris.proto"; import "data.proto"; import "descriptors.proto"; @@ -135,12 +136,42 @@ message PFetchDataResult { message PTriggerProfileReportRequest { repeated PUniqueId instance_ids = 1; -} +}; message PTriggerProfileReportResult { required PStatus status = 1; -} +}; + +message PStringPair { + required string key = 1; + required string val = 2; +}; + +message PKafkaLoadInfo { + required string brokers = 1; + required string topic = 2; + repeated PStringPair properties = 3; +}; + +message PKafkaMetaProxyRequest { + optional PKafkaLoadInfo kafka_info = 1; +}; + +message PProxyRequest { + optional PKafkaMetaProxyRequest kafka_meta_request = 1; +}; + +message PKafkaMetaProxyResult { + repeated int32 partition_ids = 1; +}; + +message PProxyResult { + required PStatus status = 1; + optional PKafkaMetaProxyResult kafka_meta_result = 2; +}; +// NOTE(zc): If you want to add new method here, +// you MUST add same method to palo_internal_service.proto service PBackendService { rpc transmit_data(PTransmitDataParams) returns (PTransmitDataResult); rpc exec_plan_fragment(PExecPlanFragmentRequest) returns (PExecPlanFragmentResult); @@ -150,7 +181,6 @@ service PBackendService { rpc tablet_writer_add_batch(PTabletWriterAddBatchRequest) returns (PTabletWriterAddBatchResult); rpc tablet_writer_cancel(PTabletWriterCancelRequest) returns (PTabletWriterCancelResult); rpc trigger_profile_report(PTriggerProfileReportRequest) returns (PTriggerProfileReportResult); - // NOTE(zc): If you want to add new method here, - // you MUST add same method to palo_internal_service.proto + rpc get_info(PProxyRequest) returns (PProxyResult); }; diff --git a/gensrc/proto/olap_common.proto b/gensrc/proto/olap_common.proto index e86820c910552d..d37baf557bb333 100644 --- a/gensrc/proto/olap_common.proto +++ b/gensrc/proto/olap_common.proto @@ -16,7 +16,10 @@ // under the License. // Define common messages shared by other proto files. +syntax="proto2"; + package doris; +option java_package = "org.apache.doris.proto"; message ColumnMessage { required string name = 1; diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index f58c00ff002958..3cfe66843f5eb0 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -16,7 +16,10 @@ // under the License. // Define file format struct, like data header, index header. +syntax="proto2"; + package doris; +option java_package = "org.apache.doris.proto"; import "olap_common.proto"; import "types.proto"; diff --git a/gensrc/proto/palo_internal_service.proto b/gensrc/proto/palo_internal_service.proto index da9faf694e1f7a..3adc1ec34d1178 100644 --- a/gensrc/proto/palo_internal_service.proto +++ b/gensrc/proto/palo_internal_service.proto @@ -22,6 +22,7 @@ syntax="proto2"; import "internal_service.proto"; package palo; +option java_package = "org.apache.doris.proto"; option cc_generic_services = true; @@ -34,4 +35,5 @@ service PInternalService { rpc tablet_writer_add_batch(doris.PTabletWriterAddBatchRequest) returns (doris.PTabletWriterAddBatchResult); rpc tablet_writer_cancel(doris.PTabletWriterCancelRequest) returns (doris.PTabletWriterCancelResult); rpc trigger_profile_report(doris.PTriggerProfileReportRequest) returns (doris.PTriggerProfileReportResult); + rpc get_info(doris.PProxyRequest) returns (doris.PProxyResult); }; diff --git a/gensrc/proto/status.proto b/gensrc/proto/status.proto index bfa78d1cedb4c8..d1e9e7dda086c5 100644 --- a/gensrc/proto/status.proto +++ b/gensrc/proto/status.proto @@ -18,6 +18,7 @@ syntax="proto2"; package doris; +option java_package = "org.apache.doris.proto"; message PStatus { required int32 status_code = 1; diff --git a/gensrc/proto/types.proto b/gensrc/proto/types.proto index 21f9fda8ed3ce2..762229df1e3441 100644 --- a/gensrc/proto/types.proto +++ b/gensrc/proto/types.proto @@ -18,6 +18,7 @@ syntax="proto2"; package doris; +option java_package = "org.apache.doris.proto"; message PScalarType { // TPrimitiveType, use int32 to avoid redefine Enum diff --git a/gensrc/thrift/BackendService.thrift b/gensrc/thrift/BackendService.thrift index b60bd21264347a..6293ed4b535e36 100644 --- a/gensrc/thrift/BackendService.thrift +++ b/gensrc/thrift/BackendService.thrift @@ -86,6 +86,23 @@ struct TRoutineLoadTask { 13: optional PaloInternalService.TExecPlanFragmentParams params } +struct TKafkaMetaProxyRequest { + 1: optional TKafkaLoadInfo kafka_info +} + +struct TKafkaMetaProxyResult { + 1: optional list partition_ids +} + +struct TProxyRequest { + 1: optional TKafkaMetaProxyRequest kafka_meta_request; +} + +struct TProxyResult { + 1: required Status.TStatus status; + 2: optional TKafkaMetaProxyResult kafka_meta_result; +} + service BackendService { // Called by coord to start asynchronous execution of plan fragment in backend. // Returns as soon as all incoming data streams have been set up. diff --git a/gensrc/thrift/HeartbeatService.thrift b/gensrc/thrift/HeartbeatService.thrift index a978503bc618ad..268052d0509190 100644 --- a/gensrc/thrift/HeartbeatService.thrift +++ b/gensrc/thrift/HeartbeatService.thrift @@ -24,6 +24,7 @@ struct TMasterInfo { 3: required Types.TEpoch epoch 4: optional string token 5: optional string backend_ip + 6: optional Types.TPort http_port } struct TBackendInfo { diff --git a/run-ut.sh b/run-ut.sh index 8f32a4a071aa9a..15029175395cc9 100755 --- a/run-ut.sh +++ b/run-ut.sh @@ -191,7 +191,8 @@ ${DORIS_TEST_BINARY_DIR}/runtime/stream_load_pipe_test ${DORIS_TEST_BINARY_DIR}/runtime/tablet_writer_mgr_test ${DORIS_TEST_BINARY_DIR}/runtime/snapshot_loader_test ${DORIS_TEST_BINARY_DIR}/runtime/user_function_cache_test -## Running expr Unittest +${DORIS_TEST_BINARY_DIR}/runtime/small_file_mgr_test +# Running expr Unittest # Running http ${DORIS_TEST_BINARY_DIR}/http/metrics_action_test