From c1729f46a87bed71fa88aab79b91f1751f639513 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Fri, 19 Jul 2024 01:00:06 +0800 Subject: [PATCH 01/49] feat: add compaction server supporting remote compaction service --- src/analytic_engine/src/lib.rs | 2 +- .../src/grpc/compaction_service/error.rs | 18 +++++ src/server/src/grpc/compaction_service/mod.rs | 65 +++++++++++++++++++ 3 files changed, 84 insertions(+), 1 deletion(-) create mode 100644 src/server/src/grpc/compaction_service/error.rs create mode 100644 src/server/src/grpc/compaction_service/mod.rs diff --git a/src/analytic_engine/src/lib.rs b/src/analytic_engine/src/lib.rs index c1308d88a8..f2cc51ad15 100644 --- a/src/analytic_engine/src/lib.rs +++ b/src/analytic_engine/src/lib.rs @@ -19,7 +19,7 @@ #![feature(option_get_or_insert_default)] -mod compaction; +pub mod compaction; mod context; mod engine; pub mod error; diff --git a/src/server/src/grpc/compaction_service/error.rs b/src/server/src/grpc/compaction_service/error.rs new file mode 100644 index 0000000000..a965996707 --- /dev/null +++ b/src/server/src/grpc/compaction_service/error.rs @@ -0,0 +1,18 @@ +// 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. + +//! Error definitions for compaction service. diff --git a/src/server/src/grpc/compaction_service/mod.rs b/src/server/src/grpc/compaction_service/mod.rs new file mode 100644 index 0000000000..958bcdd798 --- /dev/null +++ b/src/server/src/grpc/compaction_service/mod.rs @@ -0,0 +1,65 @@ +// 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. + +// Compaction rpc service implementation. + +use std::{intrinsics::mir::UnwindResume, sync::Arc}; + +use analytic_engine::compaction::runner::{local_runner::LocalCompactionRunner, CompactionRunner, CompactionRunnerResult, CompactionRunnerTask}; +use async_trait::async_trait; +use common_types::request_id::RequestId; +use horaedbproto::compaction_service::{compaction_service_server::CompactionService, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; +use runtime::Runtime; + +mod error; + +#[derive(Clone)] +pub struct CompactionServiceImpl { + runtime: Arc, + runner: LocalCompactionRunner, +} + +#[async_trait] +impl CompactionService for CompactionServiceImpl { + async fn execute_compaction_task( + &self, + request: tonic::Request, + ) -> Result< + tonic::Response, + tonic::Status, + > { + let execution_response = { + let compaction_task = generate_compaction_task(request.get_ref()); + let execution_result = self.runner.run(compaction_task).await?; + generate_execution_response(&execution_result) + }; + + unimplemented!() + } +} + +// Transform request into compaction task +fn generate_compaction_task(request: &ExecuteCompactionTaskRequest) + -> CompactionRunnerTask { + unimplemented!() +} + +// Transform compaction result into response +fn generate_execution_response(result: &CompactionRunnerResult) + -> ExecuteCompactionTaskResponse { + unimplemented!() +} From 41f166be09e1b732b445702391a8a6cbde909939 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Sun, 21 Jul 2024 23:39:08 +0800 Subject: [PATCH 02/49] fix style. --- src/table_engine/src/predicate.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/table_engine/src/predicate.rs b/src/table_engine/src/predicate.rs index b316b99e24..f444cea534 100644 --- a/src/table_engine/src/predicate.rs +++ b/src/table_engine/src/predicate.rs @@ -112,7 +112,7 @@ impl Predicate { impl TryFrom<&Predicate> for horaedbproto::remote_engine::Predicate { type Error = Error; - fn try_from(predicate: &Predicate) -> std::result::Result { + fn try_from(predicate: &Predicate) -> Result { let time_range = predicate.time_range; let mut exprs = Vec::with_capacity(predicate.exprs.len()); for expr in &predicate.exprs { @@ -137,7 +137,7 @@ impl TryFrom for Predicate { fn try_from( pb: horaedbproto::remote_engine::Predicate, - ) -> std::result::Result { + ) -> Result { let time_range = pb.time_range.context(EmptyTimeRange)?; let mut exprs = Vec::with_capacity(pb.exprs.len()); for pb_expr in pb.exprs { From 51666ca5c1c6b239f237ffd870cc7dae0cfd7853 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Mon, 22 Jul 2024 16:54:51 +0800 Subject: [PATCH 03/49] fix style. --- src/analytic_engine/src/compaction/mod.rs | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/src/analytic_engine/src/compaction/mod.rs b/src/analytic_engine/src/compaction/mod.rs index 34048d6b35..fcf6af6a87 100644 --- a/src/analytic_engine/src/compaction/mod.rs +++ b/src/analytic_engine/src/compaction/mod.rs @@ -20,6 +20,7 @@ use std::{collections::HashMap, fmt, str::FromStr, sync::Arc}; use common_types::COMPACTION_STRATEGY; +use macros::define_result; use serde::{Deserialize, Serialize}; use size_ext::ReadableSize; use snafu::{ensure, Backtrace, GenerateBacktrace, ResultExt, Snafu}; @@ -74,6 +75,8 @@ pub enum Error { InvalidOption { error: String, backtrace: Backtrace }, } +define_result!(Error); + #[derive(Debug, Clone, Copy, Deserialize, Default, PartialEq, Serialize)] pub enum CompactionStrategy { #[default] @@ -145,7 +148,7 @@ impl CompactionStrategy { pub(crate) fn parse_from( value: &str, options: &HashMap, - ) -> Result { + ) -> Result { match value.trim().to_lowercase().as_str() { DEFAULT_STRATEGY => Ok(CompactionStrategy::Default), STC_STRATEGY => Ok(CompactionStrategy::SizeTiered( @@ -182,7 +185,7 @@ impl CompactionStrategy { } impl SizeTieredCompactionOptions { - pub(crate) fn validate(&self) -> Result<(), Error> { + pub(crate) fn validate(&self) -> Result<()> { ensure!( self.bucket_high > self.bucket_low, InvalidOption { @@ -215,7 +218,7 @@ impl SizeTieredCompactionOptions { pub(crate) fn parse_from( options: &HashMap, - ) -> Result { + ) -> Result { let mut opts = SizeTieredCompactionOptions::default(); if let Some(v) = options.get(BUCKET_LOW_KEY) { opts.bucket_low = v.parse().context(ParseFloat { @@ -278,7 +281,7 @@ impl TimeWindowCompactionOptions { ); } - pub(crate) fn validate(&self) -> Result<(), Error> { + pub(crate) fn validate(&self) -> Result<()> { if !Self::valid_timestamp_unit(self.timestamp_resolution) { return InvalidOption { error: format!( @@ -294,7 +297,7 @@ impl TimeWindowCompactionOptions { pub(crate) fn parse_from( options: &HashMap, - ) -> Result { + ) -> Result { let mut opts = TimeWindowCompactionOptions { size_tiered: SizeTieredCompactionOptions::parse_from(options)?, ..Default::default() From 3d007e11ae4115156811c96022264b909dd911ca Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 23 Jul 2024 15:32:15 +0800 Subject: [PATCH 04/49] define error for compaction service. --- .../src/grpc/compaction_service/error.rs | 81 +++++++++++++++++++ 1 file changed, 81 insertions(+) diff --git a/src/server/src/grpc/compaction_service/error.rs b/src/server/src/grpc/compaction_service/error.rs index a965996707..7d1463cf6d 100644 --- a/src/server/src/grpc/compaction_service/error.rs +++ b/src/server/src/grpc/compaction_service/error.rs @@ -16,3 +16,84 @@ // under the License. //! Error definitions for compaction service. + +#![allow(dead_code)] + +use generic_error::GenericError; +use horaedbproto::common::ResponseHeader; +use macros::define_result; +use snafu::Snafu; + +use crate::error_util; + +define_result!(Error); + +#[derive(Snafu, Debug)] +#[snafu(visibility(pub))] +pub enum Error { + #[snafu(display("Server error, code:{:?}, message:{}", code, msg))] + ErrNoCause { code: StatusCode, msg: String }, + + #[snafu(display("Server error, code:{:?}, message:{}, cause:{}", code, msg, source))] + ErrWithCause { + code: StatusCode, + msg: String, + source: GenericError, + }, +} + +impl Error { + pub fn code(&self) -> StatusCode { + match *self { + Error::ErrNoCause { code, .. } => code, + Error::ErrWithCause { code, .. } => code, + } + } + + /// Get the error message returned to the user. + pub fn error_message(&self) -> String { + match self { + Error::ErrNoCause { msg, .. } => msg.clone(), + + Error::ErrWithCause { msg, source, .. } => { + let err_string = source.to_string(); + let first_line = error_util::remove_backtrace_from_err(&err_string); + format!("{msg}. Caused by: {first_line}") + } + } + } +} + +/// A set of codes for compaction service. +/// +/// Note that such a set of codes is different with the codes (alias to http +/// status code) used by storage service. +#[derive(Debug, Default, Clone, Copy, PartialEq, Eq, PartialOrd, Ord)] +pub enum StatusCode { + #[default] + Ok = 0, + BadRequest = 401, + NotFound = 404, + Internal = 500, +} + +impl StatusCode { + #[inline] + pub fn as_u32(self) -> u32 { + self as u32 + } +} + +pub fn build_err_header(err: Error) -> ResponseHeader { + ResponseHeader { + code: err.code().as_u32(), + error: err.error_message(), + } +} + +pub fn build_ok_header() -> ResponseHeader { + ResponseHeader { + code: StatusCode::Ok.as_u32(), + ..Default::default() + } +} From 453b22c86270aad63763e8e75a98583eebcda759 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 23 Jul 2024 15:34:54 +0800 Subject: [PATCH 05/49] enable conversation from request to task. --- src/analytic_engine/src/compaction/mod.rs | 58 +++++++- .../src/compaction/runner/mod.rs | 138 +++++++++++++++++- src/analytic_engine/src/sst/factory.rs | 49 ++++++- src/analytic_engine/src/sst/file.rs | 68 ++++++++- src/analytic_engine/src/table_options.rs | 37 +++++ 5 files changed, 342 insertions(+), 8 deletions(-) diff --git a/src/analytic_engine/src/compaction/mod.rs b/src/analytic_engine/src/compaction/mod.rs index fcf6af6a87..9a8f4dd60c 100644 --- a/src/analytic_engine/src/compaction/mod.rs +++ b/src/analytic_engine/src/compaction/mod.rs @@ -20,16 +20,17 @@ use std::{collections::HashMap, fmt, str::FromStr, sync::Arc}; use common_types::COMPACTION_STRATEGY; +use generic_error::{BoxError, GenericError}; use macros::define_result; use serde::{Deserialize, Serialize}; use size_ext::ReadableSize; -use snafu::{ensure, Backtrace, GenerateBacktrace, ResultExt, Snafu}; +use snafu::{ensure, Backtrace, GenerateBacktrace, ResultExt, OptionExt, Snafu}; use time_ext::TimeUnit; use tokio::sync::oneshot; use crate::{ compaction::picker::{CommonCompactionPicker, CompactionPickerRef}, - sst::file::{FileHandle, Level}, + sst::file::{FileHandle, Level, FileMeta, FilePurgeQueue}, table::data::TableDataRef, }; @@ -73,6 +74,21 @@ pub enum Error { }, #[snafu(display("Invalid compaction option value, err: {}", error))] InvalidOption { error: String, backtrace: Backtrace }, + + #[snafu(display("Empty file meta.\nBacktrace:\n{}", backtrace))] + EmptyFileMeta { backtrace: Backtrace }, + + #[snafu(display("Failed to convert file meta, err:{}", source))] + ConvertFileMeta { source: GenericError }, + + #[snafu(display("Empty purge queue.\nBacktrace:\n{}", backtrace))] + EmptyPurgeQueue { backtrace: Backtrace }, + + #[snafu(display("Failed to convert purge queue, err:{}", source))] + ConvertPurgeQueue { source: GenericError }, + + #[snafu(display("Failed to convert level, err:{}", source))] + ConvertLevel { source: GenericError }, } define_result!(Error); @@ -329,6 +345,44 @@ pub struct CompactionInputFiles { pub output_level: Level, } +impl TryFrom for CompactionInputFiles { + type Error = Error; + + fn try_from(value: horaedbproto::compaction_service::CompactionInputFiles) -> Result { + let level: Level = value.level.try_into().box_err().context(ConvertLevel)?; + let output_level: Level = value.output_level.try_into().box_err().context(ConvertLevel)?; + + let mut files: Vec = Vec::with_capacity(value.files.len()); + for file in value.files { + let meta: FileMeta = file + .meta + .context(EmptyFileMeta)? + .try_into() + .box_err() + .context(ConvertFileMeta)?; + + let purge_queue: FilePurgeQueue = file + .purge_queue + .context(EmptyPurgeQueue)? + .try_into() + .box_err() + .context(ConvertPurgeQueue)?; + + files.push({ + let handle = FileHandle::new(meta, purge_queue); + handle.set_being_compacted(file.being_compacted); + handle + }); + } + + Ok(CompactionInputFiles { + level, + files, + output_level, + }) + } +} + #[derive(Debug, Default, Clone)] pub struct ExpiredFiles { /// Level of the expired files. diff --git a/src/analytic_engine/src/compaction/runner/mod.rs b/src/analytic_engine/src/compaction/runner/mod.rs index 12f333eac3..d8fd69738f 100644 --- a/src/analytic_engine/src/compaction/runner/mod.rs +++ b/src/analytic_engine/src/compaction/runner/mod.rs @@ -21,30 +21,70 @@ use std::sync::Arc; use async_trait::async_trait; use common_types::{request_id::RequestId, schema::Schema, SequenceNumber}; +use generic_error::{BoxError, GenericError}; +use macros::define_result; use object_store::Path; +use snafu::{Backtrace, Snafu, OptionExt, ResultExt}; use table_engine::table::TableId; use crate::{ compaction::CompactionInputFiles, - instance::flush_compaction::Result, + instance::flush_compaction, row_iter::IterOptions, - space::SpaceId, + space::SpaceId, sst::{ factory::SstWriteOptions, writer::{MetaData, SstInfo}, - }, + }, table::data::TableData, }; /// Compaction runner #[async_trait] pub trait CompactionRunner: Send + Sync + 'static { - async fn run(&self, task: CompactionRunnerTask) -> Result; + async fn run(&self, task: CompactionRunnerTask) -> + flush_compaction::Result; } pub type CompactionRunnerPtr = Box; pub type CompactionRunnerRef = Arc; +// TODO: Extend info(msg) for error handling. +#[derive(Debug, Snafu)] +pub enum Error { + #[snafu(display("Empty table schema.\nBacktrace:\n{}", backtrace))] + EmptyTableSchema { backtrace: Backtrace }, + + #[snafu(display("Empty input context.\nBacktrace:\n{}", backtrace))] + EmptyInputContext { backtrace: Backtrace }, + + #[snafu(display("Empty ouput context.\nBacktrace:\n{}", backtrace))] + EmptyOuputContext { backtrace: Backtrace }, + + #[snafu(display("Empty compaction input files.\nBacktrace:\n{}", backtrace))] + EmptyCompactionInputFiles { backtrace: Backtrace }, + + #[snafu(display("Empty write options.\nBacktrace:\n{}", backtrace))] + EmptySstWriteOptions { backtrace: Backtrace }, + + #[snafu(display("Failed to convert table schema, err:{}", source))] + ConvertTableSchema { source: GenericError }, + + #[snafu(display("Failed to convert input context, err:{}", source))] + ConvertInputContext { source: GenericError }, + + #[snafu(display("Failed to convert ouput context, err:{}", source))] + ConvertOuputContext { source: GenericError }, + + #[snafu(display("Failed to convert compaction input files, err:{}", source))] + ConvertCompactionInputFiles { source: GenericError }, + + #[snafu(display("Failed to convert write options, err:{}", source))] + ConvertSstWriteOptions { source: GenericError }, +} + +define_result!(Error); + /// Compaction runner task #[derive(Debug, Clone)] pub struct CompactionRunnerTask { @@ -113,6 +153,51 @@ impl CompactionRunnerTask { } } +impl TryFrom for CompactionRunnerTask { + type Error = Error; + + fn try_from(request: horaedbproto::compaction_service::ExecuteCompactionTaskRequest) -> Result { + let task_key = request.task_key; + let request_id: RequestId = request.request_id.into(); + + let schema: Schema = request + .schema + .context(EmptyTableSchema)? + .try_into() + .box_err() + .context(ConvertTableSchema)?; + + let space_id: SpaceId = request.space_id.into(); + let table_id: TableId = request.table_id.into(); + let sequence: SequenceNumber = request.sequence.into(); + + let input_ctx: InputContext = request + .input_ctx + .context(EmptyInputContext)? + .try_into() + .box_err() + .context(ConvertInputContext)?; + + let output_ctx: OutputContext = request + .output_ctx + .context(EmptyOuputContext)? + .try_into() + .box_err() + .context(ConvertOuputContext)?; + + Ok(Self { + task_key, + request_id, + schema, + space_id, + table_id, + sequence, + input_ctx, + output_ctx, + }) + } +} + pub struct CompactionRunnerResult { pub output_file_path: Path, pub sst_info: SstInfo, @@ -128,6 +213,32 @@ pub struct InputContext { pub need_dedup: bool, } +impl TryFrom for InputContext { + type Error = Error; + + fn try_from(value: horaedbproto::compaction_service::InputContext) -> Result { + let num_rows_per_row_group: usize = value.num_rows_per_row_group as usize; + let merge_iter_options = IterOptions { + batch_size: value.merge_iter_options as usize + }; + let need_dedup = value.need_dedup; + + let files: CompactionInputFiles = value + .files + .context(EmptyCompactionInputFiles)? + .try_into() + .box_err() + .context(ConvertCompactionInputFiles)?; + + Ok(InputContext { + files, + num_rows_per_row_group, + merge_iter_options, + need_dedup, + }) + } +} + #[derive(Debug, Clone)] pub struct OutputContext { /// Output sst file path @@ -135,3 +246,22 @@ pub struct OutputContext { /// Output sst write context pub write_options: SstWriteOptions, } + +impl TryFrom for OutputContext { + type Error = Error; + + fn try_from(value: horaedbproto::compaction_service::OutputContext) -> Result { + let file_path: Path = value.file_path.into(); + let write_options: SstWriteOptions = value + .write_options + .context(EmptySstWriteOptions)? + .try_into() + .box_err() + .context(ConvertSstWriteOptions)?; + + Ok(OutputContext { + file_path, + write_options, + }) + } +} diff --git a/src/analytic_engine/src/sst/factory.rs b/src/analytic_engine/src/sst/factory.rs index 2ddeb24668..3b6832dcdc 100644 --- a/src/analytic_engine/src/sst/factory.rs +++ b/src/analytic_engine/src/sst/factory.rs @@ -21,10 +21,11 @@ use std::{collections::HashMap, fmt::Debug, sync::Arc}; use async_trait::async_trait; use common_types::projected_schema::RowProjectorBuilder; +use generic_error::{BoxError, GenericError}; use macros::define_result; use object_store::{ObjectStoreRef, Path}; use runtime::Runtime; -use snafu::{ResultExt, Snafu}; +use snafu::{Backtrace, ResultExt, Snafu, OptionExt}; use table_engine::predicate::PredicateRef; use trace_metric::MetricsCollector; @@ -50,6 +51,15 @@ use crate::{ pub enum Error { #[snafu(display("Failed to parse sst header, err:{}", source,))] ParseHeader { source: header::Error }, + + #[snafu(display("Empty storage format hint.\nBacktrace:\n{}", backtrace))] + EmptyStorageFormatHint { backtrace: Backtrace }, + + #[snafu(display("Failed to convert storage format hint, err:{}", source))] + ConvertStorageFormatHint { source: GenericError }, + + #[snafu(display("Failed to convert compression, err:{}", source))] + ConvertCompression { source: GenericError }, } define_result!(Error); @@ -164,6 +174,43 @@ pub struct SstWriteOptions { pub column_stats: HashMap, } +impl TryFrom for SstWriteOptions { + type Error = Error; + + fn try_from(value: horaedbproto::compaction_service::SstWriteOptions) -> Result { + let storage_format_hint: StorageFormatHint = value + .storage_format_hint + .context(EmptyStorageFormatHint)? + .try_into() + .box_err() + .context(ConvertStorageFormatHint)?; + + let num_rows_per_row_group = value.num_rows_per_row_group as usize; + let compression: Compression = value + .compression + .try_into() + .box_err() + .context(ConvertCompression)?; + let max_buffer_size = value.max_buffer_size as usize; + + let column_stats: HashMap = value + .column_stats + .into_iter() + .map(|(k,v)| (k.into(), ColumnStats { + low_cardinality: v + })) + .collect(); + + Ok(SstWriteOptions { + storage_format_hint, + num_rows_per_row_group, + compression, + max_buffer_size, + column_stats, + }) + } +} + impl From<&ColumnStats> for ColumnEncoding { fn from(value: &ColumnStats) -> Self { ColumnEncoding { diff --git a/src/analytic_engine/src/sst/file.rs b/src/analytic_engine/src/sst/file.rs index 39cdc7c7d1..a97f181ebf 100644 --- a/src/analytic_engine/src/sst/file.rs +++ b/src/analytic_engine/src/sst/file.rs @@ -35,12 +35,13 @@ use common_types::{ SequenceNumber, }; use future_ext::{retry_async, BackoffConfig, RetryConfig}; +use generic_error::{BoxError, GenericError}; use logger::{error, info, trace, warn}; use macros::define_result; use metric_ext::Meter; use object_store::{ObjectStoreRef, Path}; use runtime::{JoinHandle, Runtime}; -use snafu::{ResultExt, Snafu}; +use snafu::{Backtrace, ResultExt, OptionExt, Snafu}; use table_engine::table::TableId; use tokio::sync::{ mpsc::{self, UnboundedReceiver, UnboundedSender}, @@ -54,6 +55,18 @@ use crate::{space::SpaceId, sst::manager::FileId, table::sst_util, table_options pub enum Error { #[snafu(display("Failed to join purger, err:{}", source))] StopPurger { source: runtime::Error }, + + #[snafu(display("Empty time range.\nBacktrace:\n{}", backtrace))] + EmptyTimeRange { backtrace: Backtrace }, + + #[snafu(display("Failed to convert time range, err:{}", source))] + ConvertTimeRange { source: GenericError }, + + #[snafu(display("Failed to convert storage format, err:{}", source))] + ConvertStorageFormat { source: GenericError }, + + #[snafu(display("Converted overflow, err:{}", source))] + ConvertOverflow { source: GenericError }, } define_result!(Error); @@ -95,6 +108,18 @@ impl From for Level { } } +impl TryFrom for Level { + type Error = Error; + + fn try_from(value: u32) -> Result { + let value: u16 = value + .try_into() + .box_err() + .context(ConvertOverflow)?; + Ok(value.into()) + } +} + impl fmt::Display for Level { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { write!(f, "{}", self.0) @@ -460,6 +485,39 @@ impl FileMeta { } } +impl TryFrom for FileMeta { + type Error = Error; + + fn try_from(value: horaedbproto::compaction_service::FileMeta) -> Result { + let time_range: TimeRange = value + .time_range + .context(EmptyTimeRange)? + .try_into() + .box_err() + .context(ConvertTimeRange)?; + + let storage_format: StorageFormat = value + .storage_format + .try_into() + .box_err() + .context(ConvertStorageFormat)?; + let mut associated_files: Vec = Vec::with_capacity(value.associated_files.len()); + for file in value.associated_files { + associated_files.push(file); + } + + Ok(FileMeta { + id: value.file_id, + size: value.size, + row_num: value.row_num, + time_range, + max_seq: value.max_seq, + storage_format, + associated_files, + }) + } +} + // Queue to store files to be deleted for a table. #[derive(Clone)] pub struct FilePurgeQueue { @@ -510,6 +568,14 @@ impl FilePurgeQueue { } } +impl TryFrom for FilePurgeQueue { + type Error = Error; + + fn try_from(_value: horaedbproto::compaction_service::FilePurgeQueue) -> Result { + unimplemented!() + } +} + struct FilePurgeQueueInner { space_id: SpaceId, table_id: TableId, diff --git a/src/analytic_engine/src/table_options.rs b/src/analytic_engine/src/table_options.rs index c5651618e2..297703ed46 100644 --- a/src/analytic_engine/src/table_options.rs +++ b/src/analytic_engine/src/table_options.rs @@ -130,6 +130,15 @@ pub enum Error { ))] UnknownStorageFormatHint { value: String, backtrace: Backtrace }, + #[snafu(display( + "Unknown compression type. value:{:?}.\nBacktrace:\n{}", + value, + backtrace + ))] + UnknownCompressionType { value: i32, backtrace: Backtrace }, + + + #[snafu(display("Storage format hint is missing.\nBacktrace:\n{}", backtrace))] MissingStorageFormatHint { backtrace: Backtrace }, @@ -237,6 +246,22 @@ impl From for Compression { } } +impl TryFrom for Compression { + type Error = Error; + + fn try_from(compression: i32) -> Result { + let compression = match compression { + 0 => Compression::Uncompressed, + 1 => Compression::Lz4, + 2 => Compression::Snappy, + 4 => Compression::Zstd, + _ => return UnknownCompressionType { value: compression }.fail(), + }; + + Ok(compression) + } +} + impl From for ParquetCompression { fn from(compression: Compression) -> Self { match compression { @@ -366,6 +391,18 @@ impl TryFrom<&str> for StorageFormat { } } +impl TryFrom for StorageFormat { + type Error = Error; + + fn try_from(value: i32) -> Result { + let format = match value { + 0 => Self::Columnar, + _ => return UnknownStorageFormatType { value }.fail(), + }; + Ok(format) + } +} + impl ToString for StorageFormat { fn to_string(&self) -> String { match self { From b549119ae9e76ee30e003aa34e0008ab0fff367b Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 23 Jul 2024 15:36:57 +0800 Subject: [PATCH 06/49] update remote compact execution. --- .../src/compaction/runner/local_runner.rs | 1 + src/server/src/grpc/compaction_service/mod.rs | 53 ++++++++++--------- src/server/src/grpc/mod.rs | 1 + 3 files changed, 30 insertions(+), 25 deletions(-) diff --git a/src/analytic_engine/src/compaction/runner/local_runner.rs b/src/analytic_engine/src/compaction/runner/local_runner.rs index fc34b2bfa6..e379d78544 100644 --- a/src/analytic_engine/src/compaction/runner/local_runner.rs +++ b/src/analytic_engine/src/compaction/runner/local_runner.rs @@ -45,6 +45,7 @@ use crate::{ const MAX_RECORD_BATCHES_IN_FLIGHT_WHEN_COMPACTION_READ: usize = 64; /// Executor carrying for actual compaction work +#[derive(Clone)] pub struct LocalCompactionRunner { runtime: Arc, scan_options: ScanOptions, diff --git a/src/server/src/grpc/compaction_service/mod.rs b/src/server/src/grpc/compaction_service/mod.rs index 958bcdd798..d975d43e41 100644 --- a/src/server/src/grpc/compaction_service/mod.rs +++ b/src/server/src/grpc/compaction_service/mod.rs @@ -17,13 +17,18 @@ // Compaction rpc service implementation. -use std::{intrinsics::mir::UnwindResume, sync::Arc}; +#![allow(dead_code)] -use analytic_engine::compaction::runner::{local_runner::LocalCompactionRunner, CompactionRunner, CompactionRunnerResult, CompactionRunnerTask}; +use std::sync::Arc; + +use analytic_engine::compaction::runner::{local_runner::LocalCompactionRunner, CompactionRunner, CompactionRunnerTask}; use async_trait::async_trait; -use common_types::request_id::RequestId; +use generic_error::BoxError; use horaedbproto::compaction_service::{compaction_service_server::CompactionService, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; use runtime::Runtime; +use snafu::ResultExt; +use tonic::{Request, Response, Status}; +use error::{ErrWithCause, StatusCode}; mod error; @@ -37,29 +42,27 @@ pub struct CompactionServiceImpl { impl CompactionService for CompactionServiceImpl { async fn execute_compaction_task( &self, - request: tonic::Request, - ) -> Result< - tonic::Response, - tonic::Status, - > { - let execution_response = { - let compaction_task = generate_compaction_task(request.get_ref()); - let execution_result = self.runner.run(compaction_task).await?; - generate_execution_response(&execution_result) - }; + request: Request, + ) -> Result, Status> { + let request: Result = request.into_inner().try_into().box_err().context(ErrWithCause { + code: StatusCode::BadRequest, + msg: "fail to convert the execute compaction task request", + }); + + match request { + Ok(request) => { + let request_id = request.request_id.clone(); + let _res = self.runner.run(request).await + .box_err().with_context(|| { + ErrWithCause { + code: StatusCode::Internal, + msg: format!("fail to compact task, request:{request_id}") + } + }); + }, + Err(_e) => {} + } unimplemented!() } } - -// Transform request into compaction task -fn generate_compaction_task(request: &ExecuteCompactionTaskRequest) - -> CompactionRunnerTask { - unimplemented!() -} - -// Transform compaction result into response -fn generate_execution_response(result: &CompactionRunnerResult) - -> ExecuteCompactionTaskResponse { - unimplemented!() -} diff --git a/src/server/src/grpc/mod.rs b/src/server/src/grpc/mod.rs index 7b02a3a2a2..78f5a223c9 100644 --- a/src/server/src/grpc/mod.rs +++ b/src/server/src/grpc/mod.rs @@ -60,6 +60,7 @@ use crate::{ }, }; +mod compaction_service; mod meta_event_service; mod metrics; mod remote_engine_service; From ec60fdc93e7dd52102d5d04f648e4524c9249e10 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Thu, 25 Jul 2024 12:37:17 +0800 Subject: [PATCH 07/49] enable conversation from task result to response. --- src/analytic_engine/src/sst/writer.rs | 24 ++++++++++++++++ src/analytic_engine/src/table_options.rs | 8 ++++++ src/server/src/grpc/compaction_service/mod.rs | 28 +++++++++++++++---- 3 files changed, 55 insertions(+), 5 deletions(-) diff --git a/src/analytic_engine/src/sst/writer.rs b/src/analytic_engine/src/sst/writer.rs index e424e8af48..f8bb9226ab 100644 --- a/src/analytic_engine/src/sst/writer.rs +++ b/src/analytic_engine/src/sst/writer.rs @@ -117,6 +117,18 @@ pub struct SstInfo { pub time_range: TimeRange, } +impl From for horaedbproto::compaction_service::SstInfo { + fn from(value: SstInfo) -> Self { + Self { + file_size: value.file_size as u64, + row_num: value.row_num as u64, + storage_format: value.storage_format.into(), + meta_path: value.meta_path, + time_range: Some(value.time_range.into()), + } + } +} + #[derive(Debug, Clone)] pub struct MetaData { /// Min key of the sst. @@ -131,6 +143,18 @@ pub struct MetaData { pub schema: Schema, } +impl From for horaedbproto::compaction_service::MetaData { + fn from(meta: MetaData) -> Self { + Self { + min_key: meta.min_key.to_vec(), + max_key: meta.max_key.to_vec(), + max_sequence: meta.max_sequence, + time_range: Some(meta.time_range.into()), + schema: Some((&meta.schema).into()), + } + } +} + /// The writer for sst. /// /// The caller provides a stream of [RecordBatch] and the writer takes diff --git a/src/analytic_engine/src/table_options.rs b/src/analytic_engine/src/table_options.rs index 297703ed46..e32b948775 100644 --- a/src/analytic_engine/src/table_options.rs +++ b/src/analytic_engine/src/table_options.rs @@ -368,6 +368,14 @@ impl From for manifest_pb::StorageFormat { } } +impl From for i32 { + fn from(value: StorageFormat) -> Self { + match value { + StorageFormat::Columnar => 0, + } + } +} + impl TryFrom for StorageFormat { type Error = Error; diff --git a/src/server/src/grpc/compaction_service/mod.rs b/src/server/src/grpc/compaction_service/mod.rs index d975d43e41..6a67af6f95 100644 --- a/src/server/src/grpc/compaction_service/mod.rs +++ b/src/server/src/grpc/compaction_service/mod.rs @@ -24,11 +24,11 @@ use std::sync::Arc; use analytic_engine::compaction::runner::{local_runner::LocalCompactionRunner, CompactionRunner, CompactionRunnerTask}; use async_trait::async_trait; use generic_error::BoxError; -use horaedbproto::compaction_service::{compaction_service_server::CompactionService, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; +use horaedbproto::compaction_service::{compaction_service_server::CompactionService, ExecResult, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; use runtime::Runtime; use snafu::ResultExt; use tonic::{Request, Response, Status}; -use error::{ErrWithCause, StatusCode}; +use error::{build_err_header, build_ok_header, ErrWithCause, StatusCode}; mod error; @@ -49,20 +49,38 @@ impl CompactionService for CompactionServiceImpl { msg: "fail to convert the execute compaction task request", }); + let mut resp: ExecuteCompactionTaskResponse = ExecuteCompactionTaskResponse::default(); match request { Ok(request) => { let request_id = request.request_id.clone(); - let _res = self.runner.run(request).await + let res = self.runner.run(request).await .box_err().with_context(|| { ErrWithCause { code: StatusCode::Internal, msg: format!("fail to compact task, request:{request_id}") } }); + + match res { + Ok(res) => { + resp.header = Some(build_ok_header()); + resp.result = Some(ExecResult { + output_file_path: res.output_file_path.into(), + sst_info: Some(res.sst_info.into()), + sst_meta: Some(res.sst_meta.into()), + }); + // TODO: Add status. + } + Err(e) => { + resp.header = Some(build_err_header(e)); + } + } }, - Err(_e) => {} + Err(e) => { + resp.header = Some(build_err_header(e)); + } } - unimplemented!() + Ok(Response::new(resp)) } } From d3b4db27e15852a8a3c30e613422cec71e124835 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Mon, 5 Aug 2024 22:28:41 +0800 Subject: [PATCH 08/49] introduce CompactionCluster for compaction server in distribute mode. --- src/analytic_engine/src/lib.rs | 2 +- src/compaction_cluster/Cargo.toml | 49 +++++++++++++++++ src/compaction_cluster/src/cluster_impl.rs | 50 +++++++++++++++++ src/compaction_cluster/src/lib.rs | 54 +++++++++++++++++++ src/server/Cargo.toml | 1 + src/server/src/grpc/compaction_service/mod.rs | 13 ++--- src/server/src/grpc/mod.rs | 38 +++++++++++-- src/server/src/server.rs | 33 +++++++++++- 8 files changed, 228 insertions(+), 12 deletions(-) create mode 100644 src/compaction_cluster/Cargo.toml create mode 100644 src/compaction_cluster/src/cluster_impl.rs create mode 100644 src/compaction_cluster/src/lib.rs diff --git a/src/analytic_engine/src/lib.rs b/src/analytic_engine/src/lib.rs index f2cc51ad15..91908e61ab 100644 --- a/src/analytic_engine/src/lib.rs +++ b/src/analytic_engine/src/lib.rs @@ -23,7 +23,7 @@ pub mod compaction; mod context; mod engine; pub mod error; -mod instance; +pub mod instance; mod manifest; pub mod memtable; mod payload; diff --git a/src/compaction_cluster/Cargo.toml b/src/compaction_cluster/Cargo.toml new file mode 100644 index 0000000000..a0d5927e4e --- /dev/null +++ b/src/compaction_cluster/Cargo.toml @@ -0,0 +1,49 @@ +# 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] +name = "compaction_cluster" + +[package.license] +workspace = true + +[package.version] +workspace = true + +[package.authors] +workspace = true + +[package.edition] +workspace = true + +[dependencies] +analytic_engine = { workspace = true } +async-trait = { workspace = true } +bytes_ext = { workspace = true } +catalog = { workspace = true } +common_types = { workspace = true } +future_ext = { workspace = true } +generic_error = { workspace = true } +horaedbproto = { workspace = true } +logger = { workspace = true } +macros = { workspace = true } +prost = { workspace = true } +runtime = { workspace = true } +serde = { workspace = true } +serde_json = { workspace = true } +snafu = { workspace = true } +tokio = { workspace = true } diff --git a/src/compaction_cluster/src/cluster_impl.rs b/src/compaction_cluster/src/cluster_impl.rs new file mode 100644 index 0000000000..4ea17b500c --- /dev/null +++ b/src/compaction_cluster/src/cluster_impl.rs @@ -0,0 +1,50 @@ +// 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. + +use std::sync::Arc; +use async_trait::async_trait; + +use analytic_engine::{compaction::runner::{CompactionRunnerRef, CompactionRunnerResult, CompactionRunnerTask}, instance::flush_compaction}; +use runtime::Runtime; + +use crate::{CompactionCluster, Result}; + +/// CompactionClusterImpl is an implementation of [`CompactionCluster`]. +/// +/// Its functions are to: +/// - Handle the action from the HoraeDB; +/// - Handle the heartbeat between CompactionServer and HoraeMeta; +pub struct CompactionClusterImpl { + // Runtime is to be used for processing heartbeat. + _runtime: Arc, + compaction_task_runner: CompactionRunnerRef, +} + +#[async_trait] +impl CompactionCluster for CompactionClusterImpl { + async fn start(&self) -> Result<()> { + unimplemented!() + } + + async fn stop(&self) -> Result<()> { + unimplemented!() + } + + async fn compact(&self, task: CompactionRunnerTask) -> flush_compaction::Result { + self.compaction_task_runner.run(task).await + } +} diff --git a/src/compaction_cluster/src/lib.rs b/src/compaction_cluster/src/lib.rs new file mode 100644 index 0000000000..11b7b0884a --- /dev/null +++ b/src/compaction_cluster/src/lib.rs @@ -0,0 +1,54 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! CompactionCluster sub-crate includes several functionalities for supporting Compaction +//! server to running in the distribute mode. Including: +//! - (todo) Request HoraeMeta for reading topology or configuration. +//! - Accept HoraeDB's commands like compact. +//! +//! The core types are [CompactionCluster] trait and its implementation +//! [CompactionClusterImpl]. + +#![feature(trait_alias)] + +use std::sync::Arc; + +use analytic_engine::{ + instance::flush_compaction, + compaction::runner::{CompactionRunnerResult, CompactionRunnerTask} +}; +use async_trait::async_trait; +use snafu::Snafu; +use macros::define_result; + +pub mod cluster_impl; + +#[derive(Debug, Snafu)] +#[snafu(visibility = "pub")] +pub enum Error {} + +define_result!(Error); + +pub type CompactionClusterRef = Arc; + +#[async_trait] +pub trait CompactionCluster { + async fn start(&self) -> Result<()>; + async fn stop(&self) -> Result<()>; + + async fn compact(&self, task: CompactionRunnerTask) -> flush_compaction::Result; +} diff --git a/src/server/Cargo.toml b/src/server/Cargo.toml index 19b0ddd15e..e117b95ed3 100644 --- a/src/server/Cargo.toml +++ b/src/server/Cargo.toml @@ -41,6 +41,7 @@ catalog = { workspace = true } clru = { workspace = true } cluster = { workspace = true } common_types = { workspace = true } +compaction_cluster = { workspace = true } datafusion = { workspace = true } derive_builder = { workspace = true } df_operator = { workspace = true } diff --git a/src/server/src/grpc/compaction_service/mod.rs b/src/server/src/grpc/compaction_service/mod.rs index 6a67af6f95..9baed0bd85 100644 --- a/src/server/src/grpc/compaction_service/mod.rs +++ b/src/server/src/grpc/compaction_service/mod.rs @@ -21,8 +21,9 @@ use std::sync::Arc; -use analytic_engine::compaction::runner::{local_runner::LocalCompactionRunner, CompactionRunner, CompactionRunnerTask}; +use analytic_engine::compaction::runner::CompactionRunnerTask; use async_trait::async_trait; +use compaction_cluster::CompactionClusterRef; use generic_error::BoxError; use horaedbproto::compaction_service::{compaction_service_server::CompactionService, ExecResult, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; use runtime::Runtime; @@ -34,8 +35,8 @@ mod error; #[derive(Clone)] pub struct CompactionServiceImpl { - runtime: Arc, - runner: LocalCompactionRunner, + pub runtime: Arc, + pub compaction_cluster: CompactionClusterRef, } #[async_trait] @@ -51,9 +52,9 @@ impl CompactionService for CompactionServiceImpl { let mut resp: ExecuteCompactionTaskResponse = ExecuteCompactionTaskResponse::default(); match request { - Ok(request) => { - let request_id = request.request_id.clone(); - let res = self.runner.run(request).await + Ok(task) => { + let request_id = task.request_id.clone(); + let res = self.compaction_cluster.compact(task).await .box_err().with_context(|| { ErrWithCause { code: StatusCode::Internal, diff --git a/src/server/src/grpc/mod.rs b/src/server/src/grpc/mod.rs index 78f5a223c9..7e42ae88c2 100644 --- a/src/server/src/grpc/mod.rs +++ b/src/server/src/grpc/mod.rs @@ -26,12 +26,12 @@ use std::{ use cluster::ClusterRef; use common_types::column_schema; +use compaction_cluster::CompactionClusterRef; +use compaction_service::CompactionServiceImpl; use futures::FutureExt; use generic_error::GenericError; use horaedbproto::{ - meta_event::meta_event_service_server::MetaEventServiceServer, - remote_engine::remote_engine_service_server::RemoteEngineServiceServer, - storage::storage_service_server::StorageServiceServer, + compaction_service::compaction_service_server::CompactionServiceServer, meta_event::meta_event_service_server::MetaEventServiceServer, remote_engine::remote_engine_service_server::RemoteEngineServiceServer, storage::storage_service_server::StorageServiceServer }; use logger::{info, warn}; use macros::define_result; @@ -164,7 +164,9 @@ define_result!(Error); pub struct RpcServices { serve_addr: SocketAddr, rpc_server: InterceptedService, AuthWithFile>, + compaction_rpc_server: Option>, meta_rpc_server: Option>, + // TODO: Consider make remote engine service server optional here. remote_engine_server: RemoteEngineServiceServer, runtime: Arc, stop_tx: Option>, @@ -174,6 +176,7 @@ pub struct RpcServices { impl RpcServices { pub async fn start(&mut self) -> Result<()> { let rpc_server = self.rpc_server.clone(); + let compaction_server = self.compaction_rpc_server.clone(); let meta_rpc_server = self.meta_rpc_server.clone(); let remote_engine_server = self.remote_engine_server.clone(); let serve_addr = self.serve_addr; @@ -183,6 +186,11 @@ impl RpcServices { let mut router = Server::builder().add_service(rpc_server); + if let Some(s) = compaction_server { + info!("Grpc server serves compaction service"); + router = router.add_service(s); + }; + if let Some(s) = meta_rpc_server { info!("Grpc server serves meta rpc service"); router = router.add_service(s); @@ -227,6 +235,7 @@ pub struct Builder { proxy: Option>, query_dedup_config: Option, hotspot_recorder: Option>, + compaction_cluster: Option, } impl Builder { @@ -242,6 +251,7 @@ impl Builder { proxy: None, query_dedup_config: None, hotspot_recorder: None, + compaction_cluster: None, } } @@ -295,6 +305,12 @@ impl Builder { self.query_dedup_config = Some(config); self } + + // CompactionCluster is an optional field for building [RpcServices]. + pub fn compaction_cluster(mut self, compaction_cluster: Option) -> Self { + self.compaction_cluster = compaction_cluster; + self + } } impl Builder { @@ -306,6 +322,21 @@ impl Builder { let proxy = self.proxy.context(MissingProxy)?; let hotspot_recorder = self.hotspot_recorder.context(MissingHotspotRecorder)?; + assert!((self.cluster.clone().is_some() & self.compaction_cluster.clone().is_some()) == false); + + let compaction_rpc_server = { + match self.compaction_cluster { + Some(compaction_cluster) => { + let service = CompactionServiceImpl { + runtime: runtimes.compact_runtime.clone(), + compaction_cluster, + }; + Some(CompactionServiceServer::new(service)) + } + None => None, + } + }; + let meta_rpc_server = self.cluster.map(|v| { let builder = meta_event_service::Builder { cluster: v, @@ -350,6 +381,7 @@ impl Builder { Ok(RpcServices { serve_addr, rpc_server, + compaction_rpc_server, meta_rpc_server, remote_engine_server, runtime, diff --git a/src/server/src/server.rs b/src/server/src/server.rs index f7cd72ec7b..9ed477589c 100644 --- a/src/server/src/server.rs +++ b/src/server/src/server.rs @@ -21,6 +21,7 @@ use std::sync::Arc; use catalog::manager::ManagerRef; use cluster::ClusterRef; +use compaction_cluster::CompactionClusterRef; use datafusion::execution::{runtime_env::RuntimeConfig, FunctionRegistry}; use df_operator::registry::FunctionRegistryRef; use interpreters::table_manipulator::TableManipulatorRef; @@ -131,6 +132,9 @@ pub enum Error { #[snafu(display("Failed to start cluster, err:{}", source))] StartCluster { source: cluster::Error }, + #[snafu(display("Failed to start compaction cluster, err:{}", source))] + StartCompactionCluster { source: compaction_cluster::Error }, + #[snafu(display("Failed to open tables in standalone mode, err:{}", source))] OpenLocalTables { source: local_tables::Error }, @@ -152,6 +156,8 @@ pub struct Server { postgresql_service: postgresql::PostgresqlService, instance: InstanceRef, cluster: Option, + #[allow(dead_code)] + compaction_cluster: Option, local_tables_recoverer: Option, } @@ -162,9 +168,15 @@ impl Server { self.mysql_service.shutdown(); self.postgresql_service.shutdown(); + assert!((self.cluster.clone().is_some() & self.compaction_cluster.clone().is_some()) == false); + if let Some(cluster) = &self.cluster { cluster.stop().await.expect("fail to stop cluster"); } + + if let Some(compaction_cluster) = &self.compaction_cluster { + compaction_cluster.stop().await.expect("fail to stop compaction cluster"); + } } pub async fn start(&mut self) -> Result<()> { @@ -177,12 +189,20 @@ impl Server { .context(OpenLocalTables)?; } - // Run in cluster mode + assert!((self.cluster.clone().is_some() & self.compaction_cluster.clone().is_some()) == false); + + // Run in cluster mode (HoraeDB) if let Some(cluster) = &self.cluster { - info!("Server start, start cluster"); + info!("HoraeDB server start, start cluster"); cluster.start().await.context(StartCluster)?; } + // Run in compaction cluster mode (Compaction Server) + if let Some(compaction_cluster) = &self.compaction_cluster { + info!("Compaction server start, start cluster"); + compaction_cluster.start().await.context(StartCompactionCluster)?; + } + // TODO: Is it necessary to create default schema in cluster mode? info!("Server start, create default schema if not exist"); self.create_default_schema_if_not_exists().await; @@ -251,6 +271,7 @@ pub struct Builder { opened_wals: Option, remote_engine: Option, datatfusion_context: Option, + compaction_cluster: Option, } impl Builder { @@ -274,6 +295,7 @@ impl Builder { opened_wals: None, remote_engine: None, datatfusion_context: None, + compaction_cluster: None, } } @@ -368,6 +390,11 @@ impl Builder { self } + pub fn compaction_cluster(mut self, compaction_cluster: CompactionClusterRef) -> Self { + self.compaction_cluster = Some(compaction_cluster); + self + } + /// Build and run the server pub fn build(self) -> Result { // Build instance @@ -527,6 +554,7 @@ impl Builder { .proxy(proxy) .hotspot_recorder(hotspot_recorder) .query_dedup(self.server_config.query_dedup) + .compaction_cluster(self.compaction_cluster.clone()) .build() .context(BuildGrpcService)?; @@ -537,6 +565,7 @@ impl Builder { postgresql_service, instance, cluster: self.cluster, + compaction_cluster: self.compaction_cluster, local_tables_recoverer: self.local_tables_recoverer, }; Ok(server) From 798dd417df5d18298625769778c537eccb3d1d80 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 6 Aug 2024 23:40:49 +0800 Subject: [PATCH 09/49] enable compaction cluster deployment. --- Cargo.toml | 10 ++-- src/compaction_cluster/Cargo.toml | 1 + src/compaction_cluster/src/cluster_impl.rs | 8 +++- src/compaction_cluster/src/config.rs | 26 +++++++++++ src/compaction_cluster/src/lib.rs | 1 + src/horaedb/Cargo.toml | 53 +++++++++++----------- src/horaedb/bin/horaedb-server.rs | 2 + src/horaedb/src/config.rs | 10 +++- src/horaedb/src/setup.rs | 32 +++++++++++++ src/meta_client/src/lib.rs | 4 +- src/server/src/grpc/mod.rs | 4 +- 11 files changed, 115 insertions(+), 36 deletions(-) create mode 100644 src/compaction_cluster/src/config.rs diff --git a/Cargo.toml b/Cargo.toml index 4563acf5c2..d2df7b3c70 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -24,7 +24,7 @@ license = "Apache-2.0" [workspace] resolver = "2" # In alphabetical order -members = [ +members = [ "horaectl", "integration_tests", "integration_tests/sdk/rust", @@ -33,7 +33,9 @@ members = [ "src/catalog", "src/catalog_impls", "src/cluster", - "src/common_types", + "src/common_types", + "src/compaction_client", + "src/compaction_cluster", "src/components/alloc_tracker", "src/components/arena", "src/components/arrow_ext", @@ -101,12 +103,14 @@ thiserror = "1" bytes_ext = { path = "src/components/bytes_ext" } catalog = { path = "src/catalog" } catalog_impls = { path = "src/catalog_impls" } -horaedbproto = { git = "https://github.com/apache/incubator-horaedb-proto.git", rev = "19ece8f771fc0b3e8e734072cc3d8040de6c74cb" } +horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "9f2a9d1dd65d5fc42dc72fe92846691e1d1975a4" } codec = { path = "src/components/codec" } chrono = "0.4" clap = { version = "4.5.1", features = ["derive"] } clru = "0.6.1" cluster = { path = "src/cluster" } +compaction_cluster = { path = "src/compaction_cluster" } +compaction_client = {path = "src/compaction_client" } criterion = "0.5" horaedb-client = "1.0.2" common_types = { path = "src/common_types" } diff --git a/src/compaction_cluster/Cargo.toml b/src/compaction_cluster/Cargo.toml index a0d5927e4e..46fe4e5829 100644 --- a/src/compaction_cluster/Cargo.toml +++ b/src/compaction_cluster/Cargo.toml @@ -35,6 +35,7 @@ analytic_engine = { workspace = true } async-trait = { workspace = true } bytes_ext = { workspace = true } catalog = { workspace = true } +cluster = { workspace = true } common_types = { workspace = true } future_ext = { workspace = true } generic_error = { workspace = true } diff --git a/src/compaction_cluster/src/cluster_impl.rs b/src/compaction_cluster/src/cluster_impl.rs index 4ea17b500c..79befaa3a7 100644 --- a/src/compaction_cluster/src/cluster_impl.rs +++ b/src/compaction_cluster/src/cluster_impl.rs @@ -27,13 +27,19 @@ use crate::{CompactionCluster, Result}; /// /// Its functions are to: /// - Handle the action from the HoraeDB; -/// - Handle the heartbeat between CompactionServer and HoraeMeta; +/// - (todo) Handle the heartbeat between CompactionServer and HoraeMeta; pub struct CompactionClusterImpl { // Runtime is to be used for processing heartbeat. _runtime: Arc, compaction_task_runner: CompactionRunnerRef, } +impl CompactionClusterImpl { + pub async fn try_new() -> Result { + unimplemented!() + } +} + #[async_trait] impl CompactionCluster for CompactionClusterImpl { async fn start(&self) -> Result<()> { diff --git a/src/compaction_cluster/src/config.rs b/src/compaction_cluster/src/config.rs new file mode 100644 index 0000000000..d9e8d223a6 --- /dev/null +++ b/src/compaction_cluster/src/config.rs @@ -0,0 +1,26 @@ +// 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. + +use cluster::config::EtcdClientConfig; +use serde::{Deserialize, Serialize}; + +#[derive(Default, Clone, Deserialize, Debug, Serialize)] +#[serde(default)] +pub struct CompactionClusterConfig { + pub cmd_channel_buffer_size: usize, + pub etcd_client: EtcdClientConfig, +} diff --git a/src/compaction_cluster/src/lib.rs b/src/compaction_cluster/src/lib.rs index 11b7b0884a..e9689fefd9 100644 --- a/src/compaction_cluster/src/lib.rs +++ b/src/compaction_cluster/src/lib.rs @@ -36,6 +36,7 @@ use snafu::Snafu; use macros::define_result; pub mod cluster_impl; +pub mod config; #[derive(Debug, Snafu)] #[snafu(visibility = "pub")] diff --git a/src/horaedb/Cargo.toml b/src/horaedb/Cargo.toml index 38f224642d..7501cf4d50 100644 --- a/src/horaedb/Cargo.toml +++ b/src/horaedb/Cargo.toml @@ -37,32 +37,33 @@ wal-message-queue = ["wal/wal-message-queue", "analytic_engine/wal-message-queue wal-rocksdb = ["wal/wal-rocksdb", "analytic_engine/wal-rocksdb"] [dependencies] -analytic_engine = { workspace = true } -catalog = { workspace = true } -catalog_impls = { workspace = true } -clap = { workspace = true } -cluster = { workspace = true } -datafusion = { workspace = true } -df_operator = { workspace = true } -etcd-client = { workspace = true } -interpreters = { workspace = true } -logger = { workspace = true } -meta_client = { workspace = true } -moka = { version = "0.10", features = ["future"] } -panic_ext = { workspace = true } -proxy = { workspace = true } -query_engine = { workspace = true } -router = { workspace = true } -runtime = { workspace = true } -serde = { workspace = true } -server = { workspace = true } -signal-hook = "0.3" -size_ext = { workspace = true } -table_engine = { workspace = true } -toml = { workspace = true } -toml_ext = { workspace = true } -tracing_util = { workspace = true } -wal = { workspace = true } +analytic_engine = { workspace = true } +catalog = { workspace = true } +catalog_impls = { workspace = true } +clap = { workspace = true } +cluster = { workspace = true } +compaction_cluster = { workspace = true } +datafusion = { workspace = true } +df_operator = { workspace = true } +etcd-client = { workspace = true } +interpreters = { workspace = true } +logger = { workspace = true } +meta_client = { workspace = true } +moka = { version = "0.10", features = ["future"] } +panic_ext = { workspace = true } +proxy = { workspace = true } +query_engine = { workspace = true } +router = { workspace = true } +runtime = { workspace = true } +serde = { workspace = true } +server = { workspace = true } +signal-hook = "0.3" +size_ext = { workspace = true } +table_engine = { workspace = true } +toml = { workspace = true } +toml_ext = { workspace = true } +tracing_util = { workspace = true } +wal = { workspace = true } [build-dependencies] vergen = { version = "8", default-features = false, features = [ diff --git a/src/horaedb/bin/horaedb-server.rs b/src/horaedb/bin/horaedb-server.rs index bf6662085d..623a5613b3 100644 --- a/src/horaedb/bin/horaedb-server.rs +++ b/src/horaedb/bin/horaedb-server.rs @@ -98,6 +98,8 @@ fn main() { if let Ok(cluster) = env::var(CLUSTER_NAME) { if let Some(ClusterDeployment::WithMeta(v)) = &mut config.cluster_deployment { v.meta_client.cluster_name = cluster; + } else if let Some(ClusterDeployment::CompactionServer(_v)) = &mut config.cluster_deployment { + // TODO: Initialize the CompactionClusterConfig. } } diff --git a/src/horaedb/src/config.rs b/src/horaedb/src/config.rs index b9f8932f19..f1a24d1a95 100644 --- a/src/horaedb/src/config.rs +++ b/src/horaedb/src/config.rs @@ -18,6 +18,7 @@ // Config for horaedb server. use cluster::config::ClusterConfig; +use compaction_cluster::config::CompactionClusterConfig; use proxy::limiter::LimiterConfig; use serde::{Deserialize, Serialize}; use server::config::{ServerConfig, StaticRouteConfig}; @@ -26,8 +27,8 @@ use size_ext::ReadableSize; #[derive(Clone, Debug, Deserialize, Serialize)] #[serde(default)] pub struct NodeInfo { - /// The address of the horaedb node. It can be a domain name or an IP - /// address without port followed. + /// The address of the horaedb (or compaction server) node. It can be a domain + /// name or an IP address without port followed. pub addr: String, pub zone: String, pub idc: String, @@ -99,12 +100,17 @@ impl Config { /// /// [ClusterDeployment::WithMeta] means to start one or multiple HoraeDB /// instance(s) under the control of HoraeMeta. +/// +/// [ClusterDeployment::CompactionServer] means to start one or multiple +/// Compaction Server instance(s) under the control of HoraeMeta. #[derive(Clone, Debug, Deserialize, Serialize)] #[serde(tag = "mode")] #[allow(clippy::large_enum_variant)] pub enum ClusterDeployment { NoMeta(StaticRouteConfig), WithMeta(ClusterConfig), + // TODO: Consider introduce CompactionClusterConfig. + CompactionServer(CompactionClusterConfig), } #[derive(Clone, Debug, Deserialize, Serialize)] diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index 1cc0e2abc9..65bf154421 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -26,6 +26,7 @@ use analytic_engine::{ use catalog::{manager::ManagerRef, schema::OpenOptions, table_operator::TableOperator}; use catalog_impls::{table_based::TableBasedManager, volatile, CatalogManagerImpl}; use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_set::ShardSet}; +use compaction_cluster::cluster_impl::CompactionClusterImpl; use datafusion::execution::runtime_env::RuntimeConfig as DfRuntimeConfig; use df_operator::registry::{FunctionRegistry, FunctionRegistryImpl}; use interpreters::table_manipulator::{catalog_based, meta_based}; @@ -249,6 +250,16 @@ async fn run_server_with_runtimes( ) .await } + Some(ClusterDeployment::CompactionServer(_compaction_cluster_config)) => { + compaction_server_build( + &config, + cluster_config, + builder, + engine_runtimes.clone(), + wal_builder, + ) + .await + } }; // Build and start server @@ -313,6 +324,8 @@ async fn build_with_meta( endpoint, shard_set.clone(), meta_client.clone(), + // FIXME: Introduce optional compaction client here. + None, cluster_config.clone(), runtimes.meta_runtime.clone(), ) @@ -440,6 +453,25 @@ async fn build_without_meta( .local_tables_recoverer(local_tables_recoverer) } +async fn compaction_server_build( + config: &Config, + cluster_config: &ClusterConfig, + builder: Builder, + runtimes: Arc, + wal_opener: T, +) -> Builder { + let compaction_cluster = { + let compaction_cluster_impl = CompactionClusterImpl::try_new() + .await + .unwrap(); + Arc::new(compaction_cluster_impl) + }; + + build_with_meta(config, cluster_config, builder, runtimes, wal_opener) + .await + .compaction_cluster(compaction_cluster) +} + async fn create_static_topology_schema( catalog_mgr: ManagerRef, static_topology_config: StaticTopologyConfig, diff --git a/src/meta_client/src/lib.rs b/src/meta_client/src/lib.rs index a6cb8df6b9..b6da97e0c2 100644 --- a/src/meta_client/src/lib.rs +++ b/src/meta_client/src/lib.rs @@ -94,8 +94,8 @@ pub enum Error { define_result!(Error); -/// MetaClient is the abstraction of client used to communicate with HoraeMeta -/// cluster. +/// MetaClient is the abstraction of client used for HoraeDB to communicate with +/// HoraeMeta cluster. #[async_trait] pub trait MetaClient: Send + Sync { async fn alloc_schema_id(&self, req: AllocSchemaIdRequest) -> Result; diff --git a/src/server/src/grpc/mod.rs b/src/server/src/grpc/mod.rs index 7e42ae88c2..b341089d96 100644 --- a/src/server/src/grpc/mod.rs +++ b/src/server/src/grpc/mod.rs @@ -176,7 +176,7 @@ pub struct RpcServices { impl RpcServices { pub async fn start(&mut self) -> Result<()> { let rpc_server = self.rpc_server.clone(); - let compaction_server = self.compaction_rpc_server.clone(); + let compaction_rpc_server = self.compaction_rpc_server.clone(); let meta_rpc_server = self.meta_rpc_server.clone(); let remote_engine_server = self.remote_engine_server.clone(); let serve_addr = self.serve_addr; @@ -186,7 +186,7 @@ impl RpcServices { let mut router = Server::builder().add_service(rpc_server); - if let Some(s) = compaction_server { + if let Some(s) = compaction_rpc_server { info!("Grpc server serves compaction service"); router = router.add_service(s); }; From cd7c9ae279faf210b92fa38d4a7629638f1c1939 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Wed, 7 Aug 2024 20:50:04 +0800 Subject: [PATCH 10/49] refactor: replace CompactionCluster with ClusterType. --- src/cluster/src/cluster_impl.rs | 8 ++- src/cluster/src/config.rs | 3 + src/cluster/src/lib.rs | 22 +++++- src/horaedb/Cargo.toml | 53 +++++++------- src/horaedb/bin/horaedb-server.rs | 4 +- src/horaedb/src/config.rs | 6 -- src/horaedb/src/setup.rs | 45 +++--------- src/router/src/cluster_based.rs | 8 ++- src/server/Cargo.toml | 1 - src/server/src/grpc/compaction_service/mod.rs | 38 ++++++++-- src/server/src/grpc/mod.rs | 71 +++++++++++-------- src/server/src/server.rs | 36 +++------- 12 files changed, 155 insertions(+), 140 deletions(-) diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index aee54e42b3..1589e119e8 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -45,7 +45,7 @@ use crate::{ shard_lock_manager::{self, ShardLockManager, ShardLockManagerRef}, shard_set::{Shard, ShardRef, ShardSet}, topology::ClusterTopology, - Cluster, ClusterNodesNotFound, ClusterNodesResp, EtcdClientFailureWithCause, + Cluster, ClusterNodesNotFound, ClusterNodesResp, EtcdClientFailureWithCause, ClusterType, InitEtcdClientConfig, InvalidArguments, MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, TableStatus, }; @@ -345,6 +345,8 @@ impl Inner { #[async_trait] impl Cluster for ClusterImpl { + type ClusterType = ClusterType; + async fn start(&self) -> Result<()> { info!("Cluster is starting with config:{:?}", self.config); @@ -376,6 +378,10 @@ impl Cluster for ClusterImpl { Ok(()) } + fn cluster_type(&self) -> ClusterType { + self.config.cluster_type.clone() + } + async fn open_shard(&self, shard_info: &ShardInfo) -> Result { self.inner.open_shard(shard_info).await } diff --git a/src/cluster/src/config.rs b/src/cluster/src/config.rs index 29e0da9719..b59eb0026d 100644 --- a/src/cluster/src/config.rs +++ b/src/cluster/src/config.rs @@ -23,6 +23,8 @@ use serde::{Deserialize, Serialize}; use table_engine::ANALYTIC_ENGINE_TYPE; use time_ext::ReadableDuration; +use crate::ClusterType; + #[derive(Debug, Clone, Deserialize, Serialize)] #[serde(default)] // TODO: move this to table_engine crates @@ -133,6 +135,7 @@ impl Default for TlsConfig { #[serde(default)] pub struct ClusterConfig { pub cmd_channel_buffer_size: usize, + pub cluster_type: ClusterType, pub meta_client: MetaClientConfig, pub etcd_client: EtcdClientConfig, } diff --git a/src/cluster/src/lib.rs b/src/cluster/src/lib.rs index a97c945a0b..ba4f27b2c4 100644 --- a/src/cluster/src/lib.rs +++ b/src/cluster/src/lib.rs @@ -35,6 +35,7 @@ use meta_client::types::{ ClusterNodesRef, RouteTablesRequest, RouteTablesResponse, ShardId, ShardInfo, ShardStatus, ShardVersion, }; +use serde::{Deserialize, Serialize}; use shard_lock_manager::ShardLockManagerRef; use snafu::{Backtrace, Snafu}; @@ -182,7 +183,7 @@ impl From for TableStatus { } } -pub type ClusterRef = Arc; +pub type ClusterRef = Arc + Send + Sync>; #[derive(Clone, Debug)] pub struct ClusterNodesResp { @@ -190,11 +191,28 @@ pub struct ClusterNodesResp { pub cluster_nodes: ClusterNodesRef, } -/// Cluster manages tables and shard infos in cluster mode. +#[derive(Debug, Clone, Deserialize, Serialize)] +pub enum ClusterType { + HoraeDB, + CompactionServer, +} + +impl Default for ClusterType { + fn default() -> Self { + Self::HoraeDB + } +} + +/// Cluster manages tables and shard infos in cluster mode. (HoraeDB) #[async_trait] pub trait Cluster { + type ClusterType: Send + Sync; + async fn start(&self) -> Result<()>; async fn stop(&self) -> Result<()>; + + /// Get cluster type. + fn cluster_type(&self) -> ClusterType; /// Fetch related information and open shard. async fn open_shard(&self, shard_info: &ShardInfo) -> Result; diff --git a/src/horaedb/Cargo.toml b/src/horaedb/Cargo.toml index 7501cf4d50..38f224642d 100644 --- a/src/horaedb/Cargo.toml +++ b/src/horaedb/Cargo.toml @@ -37,33 +37,32 @@ wal-message-queue = ["wal/wal-message-queue", "analytic_engine/wal-message-queue wal-rocksdb = ["wal/wal-rocksdb", "analytic_engine/wal-rocksdb"] [dependencies] -analytic_engine = { workspace = true } -catalog = { workspace = true } -catalog_impls = { workspace = true } -clap = { workspace = true } -cluster = { workspace = true } -compaction_cluster = { workspace = true } -datafusion = { workspace = true } -df_operator = { workspace = true } -etcd-client = { workspace = true } -interpreters = { workspace = true } -logger = { workspace = true } -meta_client = { workspace = true } -moka = { version = "0.10", features = ["future"] } -panic_ext = { workspace = true } -proxy = { workspace = true } -query_engine = { workspace = true } -router = { workspace = true } -runtime = { workspace = true } -serde = { workspace = true } -server = { workspace = true } -signal-hook = "0.3" -size_ext = { workspace = true } -table_engine = { workspace = true } -toml = { workspace = true } -toml_ext = { workspace = true } -tracing_util = { workspace = true } -wal = { workspace = true } +analytic_engine = { workspace = true } +catalog = { workspace = true } +catalog_impls = { workspace = true } +clap = { workspace = true } +cluster = { workspace = true } +datafusion = { workspace = true } +df_operator = { workspace = true } +etcd-client = { workspace = true } +interpreters = { workspace = true } +logger = { workspace = true } +meta_client = { workspace = true } +moka = { version = "0.10", features = ["future"] } +panic_ext = { workspace = true } +proxy = { workspace = true } +query_engine = { workspace = true } +router = { workspace = true } +runtime = { workspace = true } +serde = { workspace = true } +server = { workspace = true } +signal-hook = "0.3" +size_ext = { workspace = true } +table_engine = { workspace = true } +toml = { workspace = true } +toml_ext = { workspace = true } +tracing_util = { workspace = true } +wal = { workspace = true } [build-dependencies] vergen = { version = "8", default-features = false, features = [ diff --git a/src/horaedb/bin/horaedb-server.rs b/src/horaedb/bin/horaedb-server.rs index 623a5613b3..402be90146 100644 --- a/src/horaedb/bin/horaedb-server.rs +++ b/src/horaedb/bin/horaedb-server.rs @@ -98,9 +98,7 @@ fn main() { if let Ok(cluster) = env::var(CLUSTER_NAME) { if let Some(ClusterDeployment::WithMeta(v)) = &mut config.cluster_deployment { v.meta_client.cluster_name = cluster; - } else if let Some(ClusterDeployment::CompactionServer(_v)) = &mut config.cluster_deployment { - // TODO: Initialize the CompactionClusterConfig. - } + } } println!("HoraeDB server tries starting with config:{config:?}"); diff --git a/src/horaedb/src/config.rs b/src/horaedb/src/config.rs index f1a24d1a95..2f15f5269b 100644 --- a/src/horaedb/src/config.rs +++ b/src/horaedb/src/config.rs @@ -18,7 +18,6 @@ // Config for horaedb server. use cluster::config::ClusterConfig; -use compaction_cluster::config::CompactionClusterConfig; use proxy::limiter::LimiterConfig; use serde::{Deserialize, Serialize}; use server::config::{ServerConfig, StaticRouteConfig}; @@ -100,17 +99,12 @@ impl Config { /// /// [ClusterDeployment::WithMeta] means to start one or multiple HoraeDB /// instance(s) under the control of HoraeMeta. -/// -/// [ClusterDeployment::CompactionServer] means to start one or multiple -/// Compaction Server instance(s) under the control of HoraeMeta. #[derive(Clone, Debug, Deserialize, Serialize)] #[serde(tag = "mode")] #[allow(clippy::large_enum_variant)] pub enum ClusterDeployment { NoMeta(StaticRouteConfig), WithMeta(ClusterConfig), - // TODO: Consider introduce CompactionClusterConfig. - CompactionServer(CompactionClusterConfig), } #[derive(Clone, Debug, Deserialize, Serialize)] diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index 65bf154421..0bd6d18c2b 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -25,8 +25,7 @@ use analytic_engine::{ }; use catalog::{manager::ManagerRef, schema::OpenOptions, table_operator::TableOperator}; use catalog_impls::{table_based::TableBasedManager, volatile, CatalogManagerImpl}; -use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_set::ShardSet}; -use compaction_cluster::cluster_impl::CompactionClusterImpl; +use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_set::ShardSet, ClusterType}; use datafusion::execution::runtime_env::RuntimeConfig as DfRuntimeConfig; use df_operator::registry::{FunctionRegistry, FunctionRegistryImpl}; use interpreters::table_manipulator::{catalog_based, meta_based}; @@ -250,16 +249,6 @@ async fn run_server_with_runtimes( ) .await } - Some(ClusterDeployment::CompactionServer(_compaction_cluster_config)) => { - compaction_server_build( - &config, - cluster_config, - builder, - engine_runtimes.clone(), - wal_builder, - ) - .await - } }; // Build and start server @@ -324,8 +313,6 @@ async fn build_with_meta( endpoint, shard_set.clone(), meta_client.clone(), - // FIXME: Introduce optional compaction client here. - None, cluster_config.clone(), runtimes.meta_runtime.clone(), ) @@ -347,7 +334,7 @@ async fn build_with_meta( engine_runtimes: runtimes.clone(), opened_wals: opened_wals.clone(), }; - let TableEngineContext { table_engine, .. } = engine_builder + let TableEngineContext { table_engine, local_compaction_runner } = engine_builder .build() .await .expect("Failed to setup analytic engine"); @@ -365,14 +352,19 @@ async fn build_with_meta( let table_manipulator = Arc::new(meta_based::TableManipulatorImpl::new(meta_client)); let schema_config_provider = Arc::new(ClusterBasedProvider::new(cluster.clone())); - builder + + let mut builder = builder .table_engine(engine_proxy) .catalog_manager(catalog_manager) .table_manipulator(table_manipulator) .cluster(cluster) .opened_wals(opened_wals) .router(router) - .schema_config_provider(schema_config_provider) + .schema_config_provider(schema_config_provider); + if let ClusterType::CompactionServer = cluster_config.cluster_type { + builder = builder.compaction_runner(local_compaction_runner.expect("Empty compaction runner.")); + } + builder } async fn build_without_meta( @@ -453,25 +445,6 @@ async fn build_without_meta( .local_tables_recoverer(local_tables_recoverer) } -async fn compaction_server_build( - config: &Config, - cluster_config: &ClusterConfig, - builder: Builder, - runtimes: Arc, - wal_opener: T, -) -> Builder { - let compaction_cluster = { - let compaction_cluster_impl = CompactionClusterImpl::try_new() - .await - .unwrap(); - Arc::new(compaction_cluster_impl) - }; - - build_with_meta(config, cluster_config, builder, runtimes, wal_opener) - .await - .compaction_cluster(compaction_cluster) -} - async fn create_static_topology_schema( catalog_mgr: ManagerRef, static_topology_config: StaticTopologyConfig, diff --git a/src/router/src/cluster_based.rs b/src/router/src/cluster_based.rs index d929104407..430af768a2 100644 --- a/src/router/src/cluster_based.rs +++ b/src/router/src/cluster_based.rs @@ -203,7 +203,7 @@ mod tests { use cluster::{ shard_lock_manager::ShardLockManagerRef, shard_set::ShardRef, Cluster, ClusterNodesResp, - TableStatus, + ClusterType, TableStatus }; use common_types::table::ShardId; use horaedbproto::storage::{RequestContext, RouteRequest as RouteRequestPb}; @@ -218,6 +218,8 @@ mod tests { #[async_trait] impl Cluster for MockClusterImpl { + type ClusterType = ClusterType; + async fn start(&self) -> cluster::Result<()> { unimplemented!(); } @@ -226,6 +228,10 @@ mod tests { unimplemented!(); } + fn cluster_type(&self) -> ClusterType { + unimplemented!() + } + async fn open_shard(&self, _: &ShardInfo) -> cluster::Result { unimplemented!(); } diff --git a/src/server/Cargo.toml b/src/server/Cargo.toml index e117b95ed3..19b0ddd15e 100644 --- a/src/server/Cargo.toml +++ b/src/server/Cargo.toml @@ -41,7 +41,6 @@ catalog = { workspace = true } clru = { workspace = true } cluster = { workspace = true } common_types = { workspace = true } -compaction_cluster = { workspace = true } datafusion = { workspace = true } derive_builder = { workspace = true } df_operator = { workspace = true } diff --git a/src/server/src/grpc/compaction_service/mod.rs b/src/server/src/grpc/compaction_service/mod.rs index 9baed0bd85..6c776942f6 100644 --- a/src/server/src/grpc/compaction_service/mod.rs +++ b/src/server/src/grpc/compaction_service/mod.rs @@ -21,11 +21,12 @@ use std::sync::Arc; -use analytic_engine::compaction::runner::CompactionRunnerTask; +use analytic_engine::compaction::runner::{CompactionRunnerRef, CompactionRunnerTask}; use async_trait::async_trait; -use compaction_cluster::CompactionClusterRef; +use cluster::ClusterRef; use generic_error::BoxError; use horaedbproto::compaction_service::{compaction_service_server::CompactionService, ExecResult, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; +use proxy::instance::InstanceRef; use runtime::Runtime; use snafu::ResultExt; use tonic::{Request, Response, Status}; @@ -33,10 +34,39 @@ use error::{build_err_header, build_ok_header, ErrWithCause, StatusCode}; mod error; +/// Builder for [CompactionServiceImpl] +pub struct Builder { + pub cluster: ClusterRef, + pub instance: InstanceRef, + pub runtime: Arc, + pub compaction_runner: CompactionRunnerRef, +} + +impl Builder { + pub fn build(self) -> CompactionServiceImpl { + let Self { + cluster, + instance, + runtime, + compaction_runner, + } = self; + + CompactionServiceImpl { + cluster, + instance, + runtime, + compaction_runner, + } + } +} + + #[derive(Clone)] pub struct CompactionServiceImpl { + pub cluster: ClusterRef, + pub instance: InstanceRef, pub runtime: Arc, - pub compaction_cluster: CompactionClusterRef, + pub compaction_runner: CompactionRunnerRef, } #[async_trait] @@ -54,7 +84,7 @@ impl CompactionService for CompactionServiceImpl { match request { Ok(task) => { let request_id = task.request_id.clone(); - let res = self.compaction_cluster.compact(task).await + let res = self.compaction_runner.run(task).await .box_err().with_context(|| { ErrWithCause { code: StatusCode::Internal, diff --git a/src/server/src/grpc/mod.rs b/src/server/src/grpc/mod.rs index b341089d96..823bc199cc 100644 --- a/src/server/src/grpc/mod.rs +++ b/src/server/src/grpc/mod.rs @@ -24,9 +24,9 @@ use std::{ time::Duration, }; +use analytic_engine::compaction::runner::CompactionRunnerRef; use cluster::ClusterRef; use common_types::column_schema; -use compaction_cluster::CompactionClusterRef; use compaction_service::CompactionServiceImpl; use futures::FutureExt; use generic_error::GenericError; @@ -106,6 +106,9 @@ pub enum Error { #[snafu(display("Missing wals.\nBacktrace:\n{}", backtrace))] MissingWals { backtrace: Backtrace }, + #[snafu(display("Missing compaction runner.\nBacktrace:\n{}", backtrace))] + MissingCompactionRunner { backtrace: Backtrace }, + #[snafu(display("Missing timeout.\nBacktrace:\n{}", backtrace))] MissingTimeout { backtrace: Backtrace }, @@ -235,7 +238,7 @@ pub struct Builder { proxy: Option>, query_dedup_config: Option, hotspot_recorder: Option>, - compaction_cluster: Option, + compaction_runner: Option, } impl Builder { @@ -251,7 +254,7 @@ impl Builder { proxy: None, query_dedup_config: None, hotspot_recorder: None, - compaction_cluster: None, + compaction_runner: None, } } @@ -306,9 +309,9 @@ impl Builder { self } - // CompactionCluster is an optional field for building [RpcServices]. - pub fn compaction_cluster(mut self, compaction_cluster: Option) -> Self { - self.compaction_cluster = compaction_cluster; + // Compaction runner is an optional field for building [RpcServices]. + pub fn compaction_runner(mut self, runner: Option) -> Self { + self.compaction_runner = runner; self } } @@ -318,34 +321,40 @@ impl Builder { let auth = self.auth.context(MissingAuth)?; let runtimes = self.runtimes.context(MissingRuntimes)?; let instance = self.instance.context(MissingInstance)?; - let opened_wals = self.opened_wals.context(MissingWals)?; let proxy = self.proxy.context(MissingProxy)?; let hotspot_recorder = self.hotspot_recorder.context(MissingHotspotRecorder)?; - - assert!((self.cluster.clone().is_some() & self.compaction_cluster.clone().is_some()) == false); - - let compaction_rpc_server = { - match self.compaction_cluster { - Some(compaction_cluster) => { - let service = CompactionServiceImpl { - runtime: runtimes.compact_runtime.clone(), - compaction_cluster, - }; - Some(CompactionServiceServer::new(service)) + let mut meta_rpc_server: Option> = None; + let mut compaction_rpc_server: Option> = None; + + self.cluster + .map(|v| { + let result: Result<()> = (|| { + match v.cluster_type() { + cluster::ClusterType::HoraeDB => { + let opened_wals = self.opened_wals.context(MissingWals)?; + let builder = meta_event_service::Builder { + cluster: v, + instance: instance.clone(), + runtime: runtimes.meta_runtime.clone(), + opened_wals, + }; + meta_rpc_server = Some(MetaEventServiceServer::new(builder.build())); + } + cluster::ClusterType::CompactionServer => { + let compaction_runner = self.compaction_runner.context(MissingCompactionRunner)?; + let builder = compaction_service::Builder { + cluster: v, + instance: instance.clone(), + runtime: runtimes.compact_runtime.clone(), + compaction_runner, + }; + compaction_rpc_server = Some(CompactionServiceServer::new(builder.build())); + } } - None => None, - } - }; - - let meta_rpc_server = self.cluster.map(|v| { - let builder = meta_event_service::Builder { - cluster: v, - instance: instance.clone(), - runtime: runtimes.meta_runtime.clone(), - opened_wals, - }; - MetaEventServiceServer::new(builder.build()) - }); + Ok(()) + })(); + result + }).transpose()?; let remote_engine_server = { let query_dedup = self diff --git a/src/server/src/server.rs b/src/server/src/server.rs index 9ed477589c..e49ddcd88b 100644 --- a/src/server/src/server.rs +++ b/src/server/src/server.rs @@ -19,9 +19,9 @@ use std::sync::Arc; +use analytic_engine::compaction::runner::CompactionRunnerRef; use catalog::manager::ManagerRef; use cluster::ClusterRef; -use compaction_cluster::CompactionClusterRef; use datafusion::execution::{runtime_env::RuntimeConfig, FunctionRegistry}; use df_operator::registry::FunctionRegistryRef; use interpreters::table_manipulator::TableManipulatorRef; @@ -132,9 +132,6 @@ pub enum Error { #[snafu(display("Failed to start cluster, err:{}", source))] StartCluster { source: cluster::Error }, - #[snafu(display("Failed to start compaction cluster, err:{}", source))] - StartCompactionCluster { source: compaction_cluster::Error }, - #[snafu(display("Failed to open tables in standalone mode, err:{}", source))] OpenLocalTables { source: local_tables::Error }, @@ -156,8 +153,6 @@ pub struct Server { postgresql_service: postgresql::PostgresqlService, instance: InstanceRef, cluster: Option, - #[allow(dead_code)] - compaction_cluster: Option, local_tables_recoverer: Option, } @@ -168,15 +163,9 @@ impl Server { self.mysql_service.shutdown(); self.postgresql_service.shutdown(); - assert!((self.cluster.clone().is_some() & self.compaction_cluster.clone().is_some()) == false); - if let Some(cluster) = &self.cluster { cluster.stop().await.expect("fail to stop cluster"); } - - if let Some(compaction_cluster) = &self.compaction_cluster { - compaction_cluster.stop().await.expect("fail to stop compaction cluster"); - } } pub async fn start(&mut self) -> Result<()> { @@ -189,20 +178,12 @@ impl Server { .context(OpenLocalTables)?; } - assert!((self.cluster.clone().is_some() & self.compaction_cluster.clone().is_some()) == false); - - // Run in cluster mode (HoraeDB) + // Run in cluster mode if let Some(cluster) = &self.cluster { - info!("HoraeDB server start, start cluster"); + info!("Server start, start cluster"); cluster.start().await.context(StartCluster)?; } - // Run in compaction cluster mode (Compaction Server) - if let Some(compaction_cluster) = &self.compaction_cluster { - info!("Compaction server start, start cluster"); - compaction_cluster.start().await.context(StartCompactionCluster)?; - } - // TODO: Is it necessary to create default schema in cluster mode? info!("Server start, create default schema if not exist"); self.create_default_schema_if_not_exists().await; @@ -271,7 +252,7 @@ pub struct Builder { opened_wals: Option, remote_engine: Option, datatfusion_context: Option, - compaction_cluster: Option, + compaction_runner: Option, } impl Builder { @@ -295,7 +276,7 @@ impl Builder { opened_wals: None, remote_engine: None, datatfusion_context: None, - compaction_cluster: None, + compaction_runner: None, } } @@ -390,8 +371,8 @@ impl Builder { self } - pub fn compaction_cluster(mut self, compaction_cluster: CompactionClusterRef) -> Self { - self.compaction_cluster = Some(compaction_cluster); + pub fn compaction_runner(mut self, runner: CompactionRunnerRef) -> Self { + self.compaction_runner = Some(runner); self } @@ -554,7 +535,7 @@ impl Builder { .proxy(proxy) .hotspot_recorder(hotspot_recorder) .query_dedup(self.server_config.query_dedup) - .compaction_cluster(self.compaction_cluster.clone()) + .compaction_runner(self.compaction_runner.clone()) .build() .context(BuildGrpcService)?; @@ -565,7 +546,6 @@ impl Builder { postgresql_service, instance, cluster: self.cluster, - compaction_cluster: self.compaction_cluster, local_tables_recoverer: self.local_tables_recoverer, }; Ok(server) From 818d61fa7d066a7a4071e083bd5e9cd4f9e1b851 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Wed, 7 Aug 2024 20:59:23 +0800 Subject: [PATCH 11/49] remove compaction cluster, --- Cargo.toml | 4 -- src/compaction_cluster/Cargo.toml | 50 ------------------- src/compaction_cluster/src/cluster_impl.rs | 56 ---------------------- src/compaction_cluster/src/config.rs | 26 ---------- src/compaction_cluster/src/lib.rs | 55 --------------------- 5 files changed, 191 deletions(-) delete mode 100644 src/compaction_cluster/Cargo.toml delete mode 100644 src/compaction_cluster/src/cluster_impl.rs delete mode 100644 src/compaction_cluster/src/config.rs delete mode 100644 src/compaction_cluster/src/lib.rs diff --git a/Cargo.toml b/Cargo.toml index d2df7b3c70..e0c30373ab 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -34,8 +34,6 @@ members = [ "src/catalog_impls", "src/cluster", "src/common_types", - "src/compaction_client", - "src/compaction_cluster", "src/components/alloc_tracker", "src/components/arena", "src/components/arrow_ext", @@ -109,8 +107,6 @@ chrono = "0.4" clap = { version = "4.5.1", features = ["derive"] } clru = "0.6.1" cluster = { path = "src/cluster" } -compaction_cluster = { path = "src/compaction_cluster" } -compaction_client = {path = "src/compaction_client" } criterion = "0.5" horaedb-client = "1.0.2" common_types = { path = "src/common_types" } diff --git a/src/compaction_cluster/Cargo.toml b/src/compaction_cluster/Cargo.toml deleted file mode 100644 index 46fe4e5829..0000000000 --- a/src/compaction_cluster/Cargo.toml +++ /dev/null @@ -1,50 +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] -name = "compaction_cluster" - -[package.license] -workspace = true - -[package.version] -workspace = true - -[package.authors] -workspace = true - -[package.edition] -workspace = true - -[dependencies] -analytic_engine = { workspace = true } -async-trait = { workspace = true } -bytes_ext = { workspace = true } -catalog = { workspace = true } -cluster = { workspace = true } -common_types = { workspace = true } -future_ext = { workspace = true } -generic_error = { workspace = true } -horaedbproto = { workspace = true } -logger = { workspace = true } -macros = { workspace = true } -prost = { workspace = true } -runtime = { workspace = true } -serde = { workspace = true } -serde_json = { workspace = true } -snafu = { workspace = true } -tokio = { workspace = true } diff --git a/src/compaction_cluster/src/cluster_impl.rs b/src/compaction_cluster/src/cluster_impl.rs deleted file mode 100644 index 79befaa3a7..0000000000 --- a/src/compaction_cluster/src/cluster_impl.rs +++ /dev/null @@ -1,56 +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. - -use std::sync::Arc; -use async_trait::async_trait; - -use analytic_engine::{compaction::runner::{CompactionRunnerRef, CompactionRunnerResult, CompactionRunnerTask}, instance::flush_compaction}; -use runtime::Runtime; - -use crate::{CompactionCluster, Result}; - -/// CompactionClusterImpl is an implementation of [`CompactionCluster`]. -/// -/// Its functions are to: -/// - Handle the action from the HoraeDB; -/// - (todo) Handle the heartbeat between CompactionServer and HoraeMeta; -pub struct CompactionClusterImpl { - // Runtime is to be used for processing heartbeat. - _runtime: Arc, - compaction_task_runner: CompactionRunnerRef, -} - -impl CompactionClusterImpl { - pub async fn try_new() -> Result { - unimplemented!() - } -} - -#[async_trait] -impl CompactionCluster for CompactionClusterImpl { - async fn start(&self) -> Result<()> { - unimplemented!() - } - - async fn stop(&self) -> Result<()> { - unimplemented!() - } - - async fn compact(&self, task: CompactionRunnerTask) -> flush_compaction::Result { - self.compaction_task_runner.run(task).await - } -} diff --git a/src/compaction_cluster/src/config.rs b/src/compaction_cluster/src/config.rs deleted file mode 100644 index d9e8d223a6..0000000000 --- a/src/compaction_cluster/src/config.rs +++ /dev/null @@ -1,26 +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. - -use cluster::config::EtcdClientConfig; -use serde::{Deserialize, Serialize}; - -#[derive(Default, Clone, Deserialize, Debug, Serialize)] -#[serde(default)] -pub struct CompactionClusterConfig { - pub cmd_channel_buffer_size: usize, - pub etcd_client: EtcdClientConfig, -} diff --git a/src/compaction_cluster/src/lib.rs b/src/compaction_cluster/src/lib.rs deleted file mode 100644 index e9689fefd9..0000000000 --- a/src/compaction_cluster/src/lib.rs +++ /dev/null @@ -1,55 +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. - -//! CompactionCluster sub-crate includes several functionalities for supporting Compaction -//! server to running in the distribute mode. Including: -//! - (todo) Request HoraeMeta for reading topology or configuration. -//! - Accept HoraeDB's commands like compact. -//! -//! The core types are [CompactionCluster] trait and its implementation -//! [CompactionClusterImpl]. - -#![feature(trait_alias)] - -use std::sync::Arc; - -use analytic_engine::{ - instance::flush_compaction, - compaction::runner::{CompactionRunnerResult, CompactionRunnerTask} -}; -use async_trait::async_trait; -use snafu::Snafu; -use macros::define_result; - -pub mod cluster_impl; -pub mod config; - -#[derive(Debug, Snafu)] -#[snafu(visibility = "pub")] -pub enum Error {} - -define_result!(Error); - -pub type CompactionClusterRef = Arc; - -#[async_trait] -pub trait CompactionCluster { - async fn start(&self) -> Result<()>; - async fn stop(&self) -> Result<()>; - - async fn compact(&self, task: CompactionRunnerTask) -> flush_compaction::Result; -} From 31a306c1aaed5e8d1a16515ebd334fd89ad5c569 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Wed, 7 Aug 2024 22:26:50 +0800 Subject: [PATCH 12/49] fix style and comment. --- src/analytic_engine/src/table_options.rs | 2 -- src/cluster/src/cluster_impl.rs | 1 + src/meta_client/src/lib.rs | 4 ++-- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/analytic_engine/src/table_options.rs b/src/analytic_engine/src/table_options.rs index e32b948775..3382198546 100644 --- a/src/analytic_engine/src/table_options.rs +++ b/src/analytic_engine/src/table_options.rs @@ -137,8 +137,6 @@ pub enum Error { ))] UnknownCompressionType { value: i32, backtrace: Backtrace }, - - #[snafu(display("Storage format hint is missing.\nBacktrace:\n{}", backtrace))] MissingStorageFormatHint { backtrace: Backtrace }, diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index 1589e119e8..b0b01678dd 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -345,6 +345,7 @@ impl Inner { #[async_trait] impl Cluster for ClusterImpl { + /// Type of the server in cluster mode. type ClusterType = ClusterType; async fn start(&self) -> Result<()> { diff --git a/src/meta_client/src/lib.rs b/src/meta_client/src/lib.rs index b6da97e0c2..9d0bfd4863 100644 --- a/src/meta_client/src/lib.rs +++ b/src/meta_client/src/lib.rs @@ -94,8 +94,8 @@ pub enum Error { define_result!(Error); -/// MetaClient is the abstraction of client used for HoraeDB to communicate with -/// HoraeMeta cluster. +/// MetaClient is the abstraction of client used to communicate with HoraeMeta +// cluster. #[async_trait] pub trait MetaClient: Send + Sync { async fn alloc_schema_id(&self, req: AllocSchemaIdRequest) -> Result; From 29894f8878b44c17f7c227840460d1efb0750243 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Thu, 8 Aug 2024 01:02:02 +0800 Subject: [PATCH 13/49] provide cluster type for communication between horaedb/cs (as client) and horaemeta. --- Cargo.toml | 2 +- src/cluster/src/lib.rs | 15 +-------------- src/common_types/src/cluster.rs | 25 +++++++++++++++++++++++++ src/common_types/src/lib.rs | 1 + src/horaedb/Cargo.toml | 1 + src/horaedb/src/setup.rs | 4 +++- src/meta_client/src/types.rs | 9 +++++++-- src/router/src/cluster_based.rs | 4 ++-- src/server/src/grpc/mod.rs | 6 +++--- 9 files changed, 44 insertions(+), 23 deletions(-) create mode 100644 src/common_types/src/cluster.rs diff --git a/Cargo.toml b/Cargo.toml index e0c30373ab..744cee433f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -101,7 +101,7 @@ thiserror = "1" bytes_ext = { path = "src/components/bytes_ext" } catalog = { path = "src/catalog" } catalog_impls = { path = "src/catalog_impls" } -horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "9f2a9d1dd65d5fc42dc72fe92846691e1d1975a4" } +horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "5de87a20dccfdde364ba5bea0a591f29e6556f91" } codec = { path = "src/components/codec" } chrono = "0.4" clap = { version = "4.5.1", features = ["derive"] } diff --git a/src/cluster/src/lib.rs b/src/cluster/src/lib.rs index ba4f27b2c4..a0f5483350 100644 --- a/src/cluster/src/lib.rs +++ b/src/cluster/src/lib.rs @@ -28,14 +28,13 @@ use std::sync::Arc; use async_trait::async_trait; -use common_types::schema::SchemaName; +use common_types::{cluster::ClusterType, schema::SchemaName}; use generic_error::GenericError; use macros::define_result; use meta_client::types::{ ClusterNodesRef, RouteTablesRequest, RouteTablesResponse, ShardId, ShardInfo, ShardStatus, ShardVersion, }; -use serde::{Deserialize, Serialize}; use shard_lock_manager::ShardLockManagerRef; use snafu::{Backtrace, Snafu}; @@ -191,18 +190,6 @@ pub struct ClusterNodesResp { pub cluster_nodes: ClusterNodesRef, } -#[derive(Debug, Clone, Deserialize, Serialize)] -pub enum ClusterType { - HoraeDB, - CompactionServer, -} - -impl Default for ClusterType { - fn default() -> Self { - Self::HoraeDB - } -} - /// Cluster manages tables and shard infos in cluster mode. (HoraeDB) #[async_trait] pub trait Cluster { diff --git a/src/common_types/src/cluster.rs b/src/common_types/src/cluster.rs new file mode 100644 index 0000000000..106e98c96c --- /dev/null +++ b/src/common_types/src/cluster.rs @@ -0,0 +1,25 @@ +// 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. + +use serde::{Serialize, Deserialize}; + +#[derive(Debug, Default, Clone, Serialize, Deserialize, PartialEq, Eq)] +pub enum ClusterType { + #[default] + HoraeDB, + CompactionServer, +} diff --git a/src/common_types/src/lib.rs b/src/common_types/src/lib.rs index a92f48d3f2..449a800e30 100644 --- a/src/common_types/src/lib.rs +++ b/src/common_types/src/lib.rs @@ -18,6 +18,7 @@ //! Contains common types pub mod bitset; +pub mod cluster; pub mod column; pub mod column_block; pub mod column_schema; diff --git a/src/horaedb/Cargo.toml b/src/horaedb/Cargo.toml index 38f224642d..414c9b1cba 100644 --- a/src/horaedb/Cargo.toml +++ b/src/horaedb/Cargo.toml @@ -42,6 +42,7 @@ catalog = { workspace = true } catalog_impls = { workspace = true } clap = { workspace = true } cluster = { workspace = true } +common_types = { workspace = true } datafusion = { workspace = true } df_operator = { workspace = true } etcd-client = { workspace = true } diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index 0bd6d18c2b..a2935ec323 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -25,7 +25,8 @@ use analytic_engine::{ }; use catalog::{manager::ManagerRef, schema::OpenOptions, table_operator::TableOperator}; use catalog_impls::{table_based::TableBasedManager, volatile, CatalogManagerImpl}; -use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_set::ShardSet, ClusterType}; +use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_set::ShardSet}; +use common_types::cluster::ClusterType; use datafusion::execution::runtime_env::RuntimeConfig as DfRuntimeConfig; use df_operator::registry::{FunctionRegistry, FunctionRegistryImpl}; use interpreters::table_manipulator::{catalog_based, meta_based}; @@ -297,6 +298,7 @@ async fn build_with_meta( zone: config.node.zone.clone(), idc: config.node.idc.clone(), binary_version: config.node.binary_version.clone(), + cluster_type: cluster_config.cluster_type.clone(), }; info!("Build horaedb with node meta info:{node_meta_info:?}"); diff --git a/src/meta_client/src/types.rs b/src/meta_client/src/types.rs index 6a6aba6918..1e619768d6 100644 --- a/src/meta_client/src/types.rs +++ b/src/meta_client/src/types.rs @@ -19,8 +19,7 @@ use std::{collections::HashMap, fmt, sync::Arc}; pub use common_types::table::{ShardId, ShardVersion}; use common_types::{ - schema::{SchemaId, SchemaName}, - table::{TableId, TableName}, + cluster::ClusterType, schema::{SchemaId, SchemaName}, table::{TableId, TableName} }; use generic_error::BoxError; use horaedbproto::{cluster as cluster_pb, meta_service as meta_service_pb}; @@ -163,6 +162,7 @@ pub struct NodeMetaInfo { pub zone: String, pub idc: String, pub binary_version: String, + pub cluster_type: ClusterType, } impl NodeMetaInfo { @@ -281,6 +281,11 @@ impl From for meta_service_pb::NodeInfo { binary_version: node_info.node_meta_info.binary_version, shard_infos, lease: 0, + cluster_type: if node_info.node_meta_info.cluster_type == ClusterType::HoraeDB { + meta_service_pb::node_info::ClusterType::HoraeDb + } else { + meta_service_pb::node_info::ClusterType::CompactionServer + } as i32, } } } diff --git a/src/router/src/cluster_based.rs b/src/router/src/cluster_based.rs index 430af768a2..de50d17297 100644 --- a/src/router/src/cluster_based.rs +++ b/src/router/src/cluster_based.rs @@ -203,9 +203,9 @@ mod tests { use cluster::{ shard_lock_manager::ShardLockManagerRef, shard_set::ShardRef, Cluster, ClusterNodesResp, - ClusterType, TableStatus + TableStatus }; - use common_types::table::ShardId; + use common_types::{cluster::ClusterType, table::ShardId}; use horaedbproto::storage::{RequestContext, RouteRequest as RouteRequestPb}; use meta_client::types::{ NodeShard, RouteEntry, RouteTablesResponse, ShardInfo, ShardRole::Leader, TableInfo, diff --git a/src/server/src/grpc/mod.rs b/src/server/src/grpc/mod.rs index 823bc199cc..f791669dee 100644 --- a/src/server/src/grpc/mod.rs +++ b/src/server/src/grpc/mod.rs @@ -26,7 +26,7 @@ use std::{ use analytic_engine::compaction::runner::CompactionRunnerRef; use cluster::ClusterRef; -use common_types::column_schema; +use common_types::{cluster::ClusterType, column_schema}; use compaction_service::CompactionServiceImpl; use futures::FutureExt; use generic_error::GenericError; @@ -330,7 +330,7 @@ impl Builder { .map(|v| { let result: Result<()> = (|| { match v.cluster_type() { - cluster::ClusterType::HoraeDB => { + ClusterType::HoraeDB => { let opened_wals = self.opened_wals.context(MissingWals)?; let builder = meta_event_service::Builder { cluster: v, @@ -340,7 +340,7 @@ impl Builder { }; meta_rpc_server = Some(MetaEventServiceServer::new(builder.build())); } - cluster::ClusterType::CompactionServer => { + ClusterType::CompactionServer => { let compaction_runner = self.compaction_runner.context(MissingCompactionRunner)?; let builder = compaction_service::Builder { cluster: v, From 12f135ab3a014a15a0a2815021b948efe8f7826c Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Mon, 12 Aug 2024 23:05:03 +0800 Subject: [PATCH 14/49] introduce compaction client for horaedb to access remote compaction node. --- Cargo.toml | 2 + src/cluster/Cargo.toml | 1 + src/cluster/src/cluster_impl.rs | 43 ++++-- src/cluster/src/config.rs | 2 + src/cluster/src/lib.rs | 14 +- src/compaction_client/Cargo.toml | 49 +++++++ src/compaction_client/src/compaction_impl.rs | 130 +++++++++++++++++++ src/compaction_client/src/lib.rs | 73 +++++++++++ src/compaction_client/src/types.rs | 43 ++++++ src/horaedb/Cargo.toml | 55 ++++---- src/horaedb/src/setup.rs | 12 ++ src/router/Cargo.toml | 1 + src/router/src/cluster_based.rs | 5 + 13 files changed, 393 insertions(+), 37 deletions(-) create mode 100644 src/compaction_client/Cargo.toml create mode 100644 src/compaction_client/src/compaction_impl.rs create mode 100644 src/compaction_client/src/lib.rs create mode 100644 src/compaction_client/src/types.rs diff --git a/Cargo.toml b/Cargo.toml index 744cee433f..d17050553b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -34,6 +34,7 @@ members = [ "src/catalog_impls", "src/cluster", "src/common_types", + "src/compaction_client", "src/components/alloc_tracker", "src/components/arena", "src/components/arrow_ext", @@ -110,6 +111,7 @@ cluster = { path = "src/cluster" } criterion = "0.5" horaedb-client = "1.0.2" common_types = { path = "src/common_types" } +compaction_client = { path = "src/compaction_client" } datafusion = { git = "https://github.com/CeresDB/arrow-datafusion.git", rev = "e21b03154" } datafusion-proto = { git = "https://github.com/CeresDB/arrow-datafusion.git", rev = "e21b03154" } derive_builder = "0.12" diff --git a/src/cluster/Cargo.toml b/src/cluster/Cargo.toml index e48fd847c1..bd7025ea3e 100644 --- a/src/cluster/Cargo.toml +++ b/src/cluster/Cargo.toml @@ -35,6 +35,7 @@ async-trait = { workspace = true } bytes_ext = { workspace = true } catalog = { workspace = true } common_types = { workspace = true } +compaction_client = { workspace = true } etcd-client = { workspace = true } future_ext = { workspace = true } generic_error = { workspace = true } diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index b0b01678dd..662bddb634 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -22,6 +22,12 @@ use std::{ use async_trait::async_trait; use common_types::table::ShardId; +use compaction_client::{ + types::{ + ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse + }, + CompactionClientRef, +}; use etcd_client::{Certificate, ConnectOptions, Identity, TlsOptions}; use generic_error::BoxError; use logger::{error, info, warn}; @@ -41,13 +47,13 @@ use tokio::{ }; use crate::{ - config::{ClusterConfig, EtcdClientConfig}, - shard_lock_manager::{self, ShardLockManager, ShardLockManagerRef}, - shard_set::{Shard, ShardRef, ShardSet}, - topology::ClusterTopology, - Cluster, ClusterNodesNotFound, ClusterNodesResp, EtcdClientFailureWithCause, ClusterType, - InitEtcdClientConfig, InvalidArguments, MetaClientFailure, OpenShard, OpenShardWithCause, - Result, ShardNotFound, TableStatus, + config::{ClusterConfig, EtcdClientConfig}, + shard_lock_manager::{self, ShardLockManager, ShardLockManagerRef}, + shard_set::{Shard, ShardRef, ShardSet}, + topology::ClusterTopology, + Cluster, ClusterNodesNotFound, ClusterNodesResp, ClusterType, CompactionClientFailure, + EmptyCompactionClient, EtcdClientFailureWithCause, InitEtcdClientConfig, InvalidArguments, + MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, TableStatus }; /// ClusterImpl is an implementation of [`Cluster`] based [`MetaClient`]. @@ -70,6 +76,7 @@ impl ClusterImpl { node_name: String, shard_set: ShardSet, meta_client: MetaClientRef, + compaction_client: Option, config: ClusterConfig, runtime: Arc, ) -> Result { @@ -102,7 +109,7 @@ impl ClusterImpl { }; let shard_lock_manager = ShardLockManager::new(shard_lock_mgr_config, etcd_client); - let inner = Arc::new(Inner::new(shard_set, meta_client)?); + let inner = Arc::new(Inner::new(shard_set, meta_client, compaction_client)?); Ok(Self { inner, runtime, @@ -181,14 +188,16 @@ impl ClusterImpl { struct Inner { shard_set: ShardSet, meta_client: MetaClientRef, + compaction_client: Option, topology: RwLock, } impl Inner { - fn new(shard_set: ShardSet, meta_client: MetaClientRef) -> Result { + fn new(shard_set: ShardSet, meta_client: MetaClientRef, compaction_client: Option) -> Result { Ok(Self { shard_set, meta_client, + compaction_client, topology: Default::default(), }) } @@ -341,6 +350,18 @@ impl Inner { shards.iter().map(|shard| shard.shard_info()).collect() } + + async fn compact(&self, req: &ExecuteCompactionTaskRequest) -> Result { + let compact_resp = self + .compaction_client + .clone() + .context(EmptyCompactionClient)? + .execute_compaction_task(req.clone()) + .await + .context(CompactionClientFailure)?; + + Ok(compact_resp) + } } #[async_trait] @@ -416,6 +437,10 @@ impl Cluster for ClusterImpl { fn shard_lock_manager(&self) -> ShardLockManagerRef { self.shard_lock_manager.clone() } + + async fn compact(&self, req: &ExecuteCompactionTaskRequest) -> Result { + self.inner.compact(req).await + } } /// Build the connect options for accessing etcd cluster. diff --git a/src/cluster/src/config.rs b/src/cluster/src/config.rs index b59eb0026d..679b5e8a30 100644 --- a/src/cluster/src/config.rs +++ b/src/cluster/src/config.rs @@ -18,6 +18,7 @@ use std::time::Duration; use common_types::schema::TIMESTAMP_COLUMN; +use compaction_client::compaction_impl::CompactionClientConfig; use meta_client::meta_impl::MetaClientConfig; use serde::{Deserialize, Serialize}; use table_engine::ANALYTIC_ENGINE_TYPE; @@ -138,4 +139,5 @@ pub struct ClusterConfig { pub cluster_type: ClusterType, pub meta_client: MetaClientConfig, pub etcd_client: EtcdClientConfig, + pub compaction_client: Option, } diff --git a/src/cluster/src/lib.rs b/src/cluster/src/lib.rs index a0f5483350..61f03e4459 100644 --- a/src/cluster/src/lib.rs +++ b/src/cluster/src/lib.rs @@ -29,6 +29,7 @@ use std::sync::Arc; use async_trait::async_trait; use common_types::{cluster::ClusterType, schema::SchemaName}; +use compaction_client::types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; use generic_error::GenericError; use macros::define_result; use meta_client::types::{ @@ -67,6 +68,12 @@ pub enum Error { #[snafu(display("Meta client execute failed, err:{source}."))] MetaClientFailure { source: meta_client::Error }, + #[snafu(display("Compaction client execute failed, err:{source}."))] + CompactionClientFailure { source: compaction_client::Error }, + + #[snafu(display("Compaction client is empty.\nBacktrace:\n{}", backtrace))] + EmptyCompactionClient { backtrace: Backtrace }, + #[snafu(display("Failed to init etcd client config, err:{source}.\nBacktrace:\n{backtrace}"))] InitEtcdClientConfig { source: std::io::Error, @@ -190,7 +197,9 @@ pub struct ClusterNodesResp { pub cluster_nodes: ClusterNodesRef, } -/// Cluster manages tables and shard infos in cluster mode. (HoraeDB) +/// Cluster has the following functions: +/// + Manages tables and shard infos in cluster mode. +/// + (Optional) Executes compaction task remotely. #[async_trait] pub trait Cluster { type ClusterType: Send + Sync; @@ -223,4 +232,7 @@ pub trait Cluster { async fn route_tables(&self, req: &RouteTablesRequest) -> Result; async fn fetch_nodes(&self) -> Result; fn shard_lock_manager(&self) -> ShardLockManagerRef; + + /// Execute compaction task in remote compaction node. + async fn compact(&self, req: &ExecuteCompactionTaskRequest) -> Result; } diff --git a/src/compaction_client/Cargo.toml b/src/compaction_client/Cargo.toml new file mode 100644 index 0000000000..36374130ff --- /dev/null +++ b/src/compaction_client/Cargo.toml @@ -0,0 +1,49 @@ +# 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] +name = "compaction_client" + +[package.license] +workspace = true + +[package.version] +workspace = true + +[package.authors] +workspace = true + +[package.edition] +workspace = true + +[dependencies] +async-trait = { workspace = true } +common_types = { workspace = true } +futures = { workspace = true } +generic_error = { workspace = true } +horaedbproto = { workspace = true } +logger = { workspace = true } +macros = { workspace = true } +prost = { workspace = true } +reqwest = { workspace = true } +serde = { workspace = true } +serde_json = { workspace = true } +snafu = { workspace = true } +time_ext = { workspace = true } +tokio = { workspace = true } +tonic = { workspace = true } +url = "2.2" diff --git a/src/compaction_client/src/compaction_impl.rs b/src/compaction_client/src/compaction_impl.rs new file mode 100644 index 0000000000..05c07ad054 --- /dev/null +++ b/src/compaction_client/src/compaction_impl.rs @@ -0,0 +1,130 @@ +// 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. + +use std::sync::Arc; + +use async_trait::async_trait; +use generic_error::BoxError; +use horaedbproto::{common::ResponseHeader, compaction_service::{self, compaction_service_client::CompactionServiceClient}}; +use logger::info; +use serde::{Deserialize, Serialize}; +use snafu::{ResultExt, OptionExt}; +use time_ext::ReadableDuration; + +use crate::{BadResponse, CompactionClient, CompactionClientRef, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse, FailConnect, FailExecuteCompactionTask, MissingHeader, Result}; + +type CompactionServiceGrpcClient = CompactionServiceClient; + +#[derive(Debug, Deserialize, Clone, Serialize)] +#[serde(default)] +pub struct CompactionClientConfig { + pub cluster_name: String, + pub compaction_server_addr: String, + pub timeout: ReadableDuration, +} + +impl Default for CompactionClientConfig { + fn default() -> Self { + unimplemented!() + } +} + +/// Default compaction client impl, will interact with the remote compaction node. +pub struct CompactionClientImpl { + client: CompactionServiceGrpcClient, +} + +impl CompactionClientImpl { + pub async fn connect(config: CompactionClientConfig) -> Result { + let client = { + let endpoint = tonic::transport::Endpoint::from_shared(config.compaction_server_addr.to_string()) + .box_err() + .context(FailConnect { + addr: &config.compaction_server_addr, + })? + .timeout(config.timeout.0); + CompactionServiceGrpcClient::connect(endpoint) + .await + .box_err() + .context(FailConnect { + addr: &config.compaction_server_addr, + })? + }; + + Ok(Self { + client, + }) + } + + #[inline] + fn client(&self) -> CompactionServiceGrpcClient { + self.client.clone() + } +} + + +#[async_trait] +impl CompactionClient for CompactionClientImpl { + async fn execute_compaction_task(&self, req: ExecuteCompactionTaskRequest) + -> Result { + let pb_req = compaction_service::ExecuteCompactionTaskRequest::from(req); + + // TODO(leslie): Add request header for ExecuteCompactionTaskRequest. + + info!( + "Compaction client try to execute compaction task in remote compaction node, req:{:?}", + pb_req + ); + + let pb_resp = self + .client() + .execute_compaction_task(pb_req) + .await + .box_err() + .context(FailExecuteCompactionTask)? + .into_inner(); + + info!( + "Compaction client finish executing compaction task in remote compaction node, req:{:?}", + pb_resp + ); + + check_response_header(&pb_resp.header)?; + ExecuteCompactionTaskResponse::try_from(pb_resp) + } +} + +// TODO(leslie): Consider to refactor and reuse the similar function in meta_client. +fn check_response_header(header: &Option) -> Result<()> { + let header = header.as_ref().context(MissingHeader)?; + if header.code == 0 { + Ok(()) + } else { + BadResponse { + code: header.code, + msg: header.error.clone(), + } + .fail() + } +} + +pub async fn build_compaction_client( + config: CompactionClientConfig, +) -> Result { + let compaction_client = CompactionClientImpl::connect(config).await?; + Ok(Arc::new(compaction_client)) +} diff --git a/src/compaction_client/src/lib.rs b/src/compaction_client/src/lib.rs new file mode 100644 index 0000000000..64c5614ad8 --- /dev/null +++ b/src/compaction_client/src/lib.rs @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use async_trait::async_trait; +use generic_error::GenericError; +use macros::define_result; +use types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; +use snafu::{Backtrace, Snafu}; + +pub mod types; +pub mod compaction_impl; + +#[derive(Debug, Snafu)] +#[snafu(visibility = "pub")] +pub enum Error { + #[snafu(display( + "Failed to connect the service endpoint:{}, err:{}\nBacktrace:\n{}", + addr, + source, + backtrace + ))] + FailConnect { + addr: String, + source: GenericError, + backtrace: Backtrace, + }, + + #[snafu(display("Failed to execute compaction task, err:{}", source))] + FailExecuteCompactionTask { source: GenericError }, + + #[snafu(display("Missing header in rpc response.\nBacktrace:\n{}", backtrace))] + MissingHeader { backtrace: Backtrace }, + + #[snafu(display( + "Bad response, resp code:{}, msg:{}.\nBacktrace:\n{}", + code, + msg, + backtrace + ))] + BadResponse { + code: u32, + msg: String, + backtrace: Backtrace, + }, +} + +define_result!(Error); + +/// CompactionClient is the abstraction of client used for HoraeDB to +/// communicate with CompactionServer cluster. +#[async_trait] +pub trait CompactionClient: Send + Sync { + async fn execute_compaction_task(&self, req: ExecuteCompactionTaskRequest) + -> Result; +} + +pub type CompactionClientRef = Arc; diff --git a/src/compaction_client/src/types.rs b/src/compaction_client/src/types.rs new file mode 100644 index 0000000000..a274286613 --- /dev/null +++ b/src/compaction_client/src/types.rs @@ -0,0 +1,43 @@ +// 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. + +use horaedbproto::compaction_service; +use macros::define_result; + +use crate::Error; + +define_result!(Error); + +#[derive(Debug, Clone)] +pub struct ExecuteCompactionTaskRequest {} + +impl From for compaction_service::ExecuteCompactionTaskRequest { + fn from(_value: ExecuteCompactionTaskRequest) -> Self { + unimplemented!() + } +} + +#[derive(Debug, Clone)] +pub struct ExecuteCompactionTaskResponse {} + +impl TryFrom for ExecuteCompactionTaskResponse { + type Error = Error; + + fn try_from(_value: compaction_service::ExecuteCompactionTaskResponse) -> Result { + unimplemented!() + } +} diff --git a/src/horaedb/Cargo.toml b/src/horaedb/Cargo.toml index 414c9b1cba..92ffd4e38a 100644 --- a/src/horaedb/Cargo.toml +++ b/src/horaedb/Cargo.toml @@ -37,33 +37,34 @@ wal-message-queue = ["wal/wal-message-queue", "analytic_engine/wal-message-queue wal-rocksdb = ["wal/wal-rocksdb", "analytic_engine/wal-rocksdb"] [dependencies] -analytic_engine = { workspace = true } -catalog = { workspace = true } -catalog_impls = { workspace = true } -clap = { workspace = true } -cluster = { workspace = true } -common_types = { workspace = true } -datafusion = { workspace = true } -df_operator = { workspace = true } -etcd-client = { workspace = true } -interpreters = { workspace = true } -logger = { workspace = true } -meta_client = { workspace = true } -moka = { version = "0.10", features = ["future"] } -panic_ext = { workspace = true } -proxy = { workspace = true } -query_engine = { workspace = true } -router = { workspace = true } -runtime = { workspace = true } -serde = { workspace = true } -server = { workspace = true } -signal-hook = "0.3" -size_ext = { workspace = true } -table_engine = { workspace = true } -toml = { workspace = true } -toml_ext = { workspace = true } -tracing_util = { workspace = true } -wal = { workspace = true } +analytic_engine = { workspace = true } +catalog = { workspace = true } +catalog_impls = { workspace = true } +clap = { workspace = true } +cluster = { workspace = true } +common_types = { workspace = true } +compaction_client = { workspace = true } +datafusion = { workspace = true } +df_operator = { workspace = true } +etcd-client = { workspace = true } +interpreters = { workspace = true } +logger = { workspace = true } +meta_client = { workspace = true } +moka = { version = "0.10", features = ["future"] } +panic_ext = { workspace = true } +proxy = { workspace = true } +query_engine = { workspace = true } +router = { workspace = true } +runtime = { workspace = true } +serde = { workspace = true } +server = { workspace = true } +signal-hook = "0.3" +size_ext = { workspace = true } +table_engine = { workspace = true } +toml = { workspace = true } +toml_ext = { workspace = true } +tracing_util = { workspace = true } +wal = { workspace = true } [build-dependencies] vergen = { version = "8", default-features = false, features = [ diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index a2935ec323..2f7a2be5cd 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -27,6 +27,7 @@ use catalog::{manager::ManagerRef, schema::OpenOptions, table_operator::TableOpe use catalog_impls::{table_based::TableBasedManager, volatile, CatalogManagerImpl}; use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_set::ShardSet}; use common_types::cluster::ClusterType; +use compaction_client::compaction_impl; use datafusion::execution::runtime_env::RuntimeConfig as DfRuntimeConfig; use df_operator::registry::{FunctionRegistry, FunctionRegistryImpl}; use interpreters::table_manipulator::{catalog_based, meta_based}; @@ -309,12 +310,23 @@ async fn build_with_meta( .await .expect("fail to build meta client"); + let compaction_client = if let Some(compaction_client) = cluster_config.compaction_client.clone() { + Some( + compaction_impl::build_compaction_client(compaction_client) + .await + .expect("fail to build compaction client"), + ) + } else { + None + }; + let shard_set = ShardSet::default(); let cluster = { let cluster_impl = ClusterImpl::try_new( endpoint, shard_set.clone(), meta_client.clone(), + compaction_client.clone(), cluster_config.clone(), runtimes.meta_runtime.clone(), ) diff --git a/src/router/Cargo.toml b/src/router/Cargo.toml index 7f3d8c23ac..7183c270e5 100644 --- a/src/router/Cargo.toml +++ b/src/router/Cargo.toml @@ -34,6 +34,7 @@ workspace = true async-trait = { workspace = true } cluster = { workspace = true } common_types = { workspace = true } +compaction_client = { workspace = true } generic_error = { workspace = true } horaedbproto = { workspace = true } logger = { workspace = true } diff --git a/src/router/src/cluster_based.rs b/src/router/src/cluster_based.rs index de50d17297..4734df983a 100644 --- a/src/router/src/cluster_based.rs +++ b/src/router/src/cluster_based.rs @@ -206,6 +206,7 @@ mod tests { TableStatus }; use common_types::{cluster::ClusterType, table::ShardId}; + use compaction_client::types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; use horaedbproto::storage::{RequestContext, RouteRequest as RouteRequestPb}; use meta_client::types::{ NodeShard, RouteEntry, RouteTablesResponse, ShardInfo, ShardRole::Leader, TableInfo, @@ -294,6 +295,10 @@ mod tests { fn shard_lock_manager(&self) -> ShardLockManagerRef { unimplemented!(); } + + async fn compact(&self, _req: &ExecuteCompactionTaskRequest) -> cluster::Result { + unimplemented!() + } } #[tokio::test] From a999f4151a3eb9d75da259809b147d433cf151d1 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Wed, 14 Aug 2024 15:49:25 +0800 Subject: [PATCH 15/49] impl compact in Cluster trait. --- src/cluster/src/cluster_impl.rs | 46 +++++++++++++++++++++------------ src/cluster/src/lib.rs | 3 --- src/horaedb/src/setup.rs | 12 --------- 3 files changed, 30 insertions(+), 31 deletions(-) diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index 662bddb634..3b7fd0387b 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -23,10 +23,11 @@ use std::{ use async_trait::async_trait; use common_types::table::ShardId; use compaction_client::{ + compaction_impl::{build_compaction_client, CompactionClientConfig}, types::{ ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse }, - CompactionClientRef, + CompactionClientRef }; use etcd_client::{Certificate, ConnectOptions, Identity, TlsOptions}; use generic_error::BoxError; @@ -47,13 +48,13 @@ use tokio::{ }; use crate::{ - config::{ClusterConfig, EtcdClientConfig}, - shard_lock_manager::{self, ShardLockManager, ShardLockManagerRef}, - shard_set::{Shard, ShardRef, ShardSet}, - topology::ClusterTopology, - Cluster, ClusterNodesNotFound, ClusterNodesResp, ClusterType, CompactionClientFailure, - EmptyCompactionClient, EtcdClientFailureWithCause, InitEtcdClientConfig, InvalidArguments, - MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, TableStatus + config::{ClusterConfig, EtcdClientConfig}, + shard_lock_manager::{self, ShardLockManager, ShardLockManagerRef}, + shard_set::{Shard, ShardRef, ShardSet}, + topology::ClusterTopology, + Cluster, ClusterNodesNotFound, ClusterNodesResp, ClusterType, CompactionClientFailure, + EtcdClientFailureWithCause, InitEtcdClientConfig, InvalidArguments,MetaClientFailure, + OpenShard, OpenShardWithCause, Result, ShardNotFound, TableStatus }; /// ClusterImpl is an implementation of [`Cluster`] based [`MetaClient`]. @@ -76,7 +77,6 @@ impl ClusterImpl { node_name: String, shard_set: ShardSet, meta_client: MetaClientRef, - compaction_client: Option, config: ClusterConfig, runtime: Arc, ) -> Result { @@ -109,7 +109,7 @@ impl ClusterImpl { }; let shard_lock_manager = ShardLockManager::new(shard_lock_mgr_config, etcd_client); - let inner = Arc::new(Inner::new(shard_set, meta_client, compaction_client)?); + let inner = Arc::new(Inner::new(shard_set, meta_client)?); Ok(Self { inner, runtime, @@ -188,16 +188,14 @@ impl ClusterImpl { struct Inner { shard_set: ShardSet, meta_client: MetaClientRef, - compaction_client: Option, topology: RwLock, } impl Inner { - fn new(shard_set: ShardSet, meta_client: MetaClientRef, compaction_client: Option) -> Result { + fn new(shard_set: ShardSet, meta_client: MetaClientRef) -> Result { Ok(Self { shard_set, meta_client, - compaction_client, topology: Default::default(), }) } @@ -351,11 +349,27 @@ impl Inner { shards.iter().map(|shard| shard.shard_info()).collect() } + /// Get the proper remote compaction node for compaction offload. + async fn get_compaction_node(&self) -> Result { + unimplemented!() + } + + async fn compaction_client(&self) -> CompactionClientRef { + // TODO(leslie): impl better error handling with snafu. + let config = self + .get_compaction_node() + .await + .expect("fail to get remote compaction node"); + + build_compaction_client(config) + .await + .expect("fail to build compaction client") + } + async fn compact(&self, req: &ExecuteCompactionTaskRequest) -> Result { let compact_resp = self - .compaction_client - .clone() - .context(EmptyCompactionClient)? + .compaction_client() + .await .execute_compaction_task(req.clone()) .await .context(CompactionClientFailure)?; diff --git a/src/cluster/src/lib.rs b/src/cluster/src/lib.rs index 61f03e4459..0a2388fbfb 100644 --- a/src/cluster/src/lib.rs +++ b/src/cluster/src/lib.rs @@ -71,9 +71,6 @@ pub enum Error { #[snafu(display("Compaction client execute failed, err:{source}."))] CompactionClientFailure { source: compaction_client::Error }, - #[snafu(display("Compaction client is empty.\nBacktrace:\n{}", backtrace))] - EmptyCompactionClient { backtrace: Backtrace }, - #[snafu(display("Failed to init etcd client config, err:{source}.\nBacktrace:\n{backtrace}"))] InitEtcdClientConfig { source: std::io::Error, diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index 2f7a2be5cd..c074eeddfe 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -27,7 +27,6 @@ use catalog::{manager::ManagerRef, schema::OpenOptions, table_operator::TableOpe use catalog_impls::{table_based::TableBasedManager, volatile, CatalogManagerImpl}; use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_set::ShardSet}; use common_types::cluster::ClusterType; -use compaction_client::compaction_impl; use datafusion::execution::runtime_env::RuntimeConfig as DfRuntimeConfig; use df_operator::registry::{FunctionRegistry, FunctionRegistryImpl}; use interpreters::table_manipulator::{catalog_based, meta_based}; @@ -309,16 +308,6 @@ async fn build_with_meta( meta_impl::build_meta_client(cluster_config.meta_client.clone(), node_meta_info) .await .expect("fail to build meta client"); - - let compaction_client = if let Some(compaction_client) = cluster_config.compaction_client.clone() { - Some( - compaction_impl::build_compaction_client(compaction_client) - .await - .expect("fail to build compaction client"), - ) - } else { - None - }; let shard_set = ShardSet::default(); let cluster = { @@ -326,7 +315,6 @@ async fn build_with_meta( endpoint, shard_set.clone(), meta_client.clone(), - compaction_client.clone(), cluster_config.clone(), runtimes.meta_runtime.clone(), ) From 165c18d8f9fd7568e3640e73495f156e4b65ef65 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Wed, 14 Aug 2024 17:45:39 +0800 Subject: [PATCH 16/49] fix style and add comment. --- src/analytic_engine/src/compaction/runner/mod.rs | 1 - src/analytic_engine/src/lib.rs | 2 +- src/cluster/src/cluster_impl.rs | 13 ++++++++++--- src/cluster/src/config.rs | 2 -- src/cluster/src/lib.rs | 8 ++++++++ src/common_types/src/cluster.rs | 1 + src/compaction_client/src/compaction_impl.rs | 10 ++++++++-- src/meta_client/src/lib.rs | 2 +- src/meta_client/src/types.rs | 4 +++- src/server/src/grpc/compaction_service/mod.rs | 10 +++++----- src/server/src/grpc/mod.rs | 12 +++++++++--- 11 files changed, 46 insertions(+), 19 deletions(-) diff --git a/src/analytic_engine/src/compaction/runner/mod.rs b/src/analytic_engine/src/compaction/runner/mod.rs index d8fd69738f..ed354e89a2 100644 --- a/src/analytic_engine/src/compaction/runner/mod.rs +++ b/src/analytic_engine/src/compaction/runner/mod.rs @@ -49,7 +49,6 @@ pub trait CompactionRunner: Send + Sync + 'static { pub type CompactionRunnerPtr = Box; pub type CompactionRunnerRef = Arc; -// TODO: Extend info(msg) for error handling. #[derive(Debug, Snafu)] pub enum Error { #[snafu(display("Empty table schema.\nBacktrace:\n{}", backtrace))] diff --git a/src/analytic_engine/src/lib.rs b/src/analytic_engine/src/lib.rs index 91908e61ab..f2cc51ad15 100644 --- a/src/analytic_engine/src/lib.rs +++ b/src/analytic_engine/src/lib.rs @@ -23,7 +23,7 @@ pub mod compaction; mod context; mod engine; pub mod error; -pub mod instance; +mod instance; mod manifest; pub mod memtable; mod payload; diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index 3b7fd0387b..322cc06fe6 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -53,8 +53,8 @@ use crate::{ shard_set::{Shard, ShardRef, ShardSet}, topology::ClusterTopology, Cluster, ClusterNodesNotFound, ClusterNodesResp, ClusterType, CompactionClientFailure, - EtcdClientFailureWithCause, InitEtcdClientConfig, InvalidArguments,MetaClientFailure, - OpenShard, OpenShardWithCause, Result, ShardNotFound, TableStatus + CompactionOffloadNotAllowed, EtcdClientFailureWithCause, InitEtcdClientConfig, InvalidArguments, + MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, TableStatus }; /// ClusterImpl is an implementation of [`Cluster`] based [`MetaClient`]. @@ -349,11 +349,12 @@ impl Inner { shards.iter().map(|shard| shard.shard_info()).collect() } - /// Get the proper remote compaction node for compaction offload. + /// Get proper remote compaction node for compaction offload with meta client. async fn get_compaction_node(&self) -> Result { unimplemented!() } + /// Return a new compaction client. async fn compaction_client(&self) -> CompactionClientRef { // TODO(leslie): impl better error handling with snafu. let config = self @@ -453,6 +454,12 @@ impl Cluster for ClusterImpl { } async fn compact(&self, req: &ExecuteCompactionTaskRequest) -> Result { + ensure!( + self.cluster_type() == ClusterType::HoraeDB, + CompactionOffloadNotAllowed { + cluster_type: self.cluster_type() + } + ); self.inner.compact(req).await } } diff --git a/src/cluster/src/config.rs b/src/cluster/src/config.rs index 679b5e8a30..b59eb0026d 100644 --- a/src/cluster/src/config.rs +++ b/src/cluster/src/config.rs @@ -18,7 +18,6 @@ use std::time::Duration; use common_types::schema::TIMESTAMP_COLUMN; -use compaction_client::compaction_impl::CompactionClientConfig; use meta_client::meta_impl::MetaClientConfig; use serde::{Deserialize, Serialize}; use table_engine::ANALYTIC_ENGINE_TYPE; @@ -139,5 +138,4 @@ pub struct ClusterConfig { pub cluster_type: ClusterType, pub meta_client: MetaClientConfig, pub etcd_client: EtcdClientConfig, - pub compaction_client: Option, } diff --git a/src/cluster/src/lib.rs b/src/cluster/src/lib.rs index 0a2388fbfb..4b4dfc5eff 100644 --- a/src/cluster/src/lib.rs +++ b/src/cluster/src/lib.rs @@ -165,6 +165,14 @@ pub enum Error { "Cluster nodes are not found in the topology, version:{version}.\nBacktrace:\n{backtrace}", ))] ClusterNodesNotFound { version: u64, backtrace: Backtrace }, + + #[snafu(display( + "Not allowed to execute compaction offload in cluster_type:{cluster_type:?}.\nBacktrace:\n{backtrace:?}" + ))] + CompactionOffloadNotAllowed { + cluster_type: ClusterType, + backtrace: Backtrace, + }, } define_result!(Error); diff --git a/src/common_types/src/cluster.rs b/src/common_types/src/cluster.rs index 106e98c96c..7c790587a1 100644 --- a/src/common_types/src/cluster.rs +++ b/src/common_types/src/cluster.rs @@ -17,6 +17,7 @@ use serde::{Serialize, Deserialize}; +/// Type to distinguish different server type in cluster mode. #[derive(Debug, Default, Clone, Serialize, Deserialize, PartialEq, Eq)] pub enum ClusterType { #[default] diff --git a/src/compaction_client/src/compaction_impl.rs b/src/compaction_client/src/compaction_impl.rs index 05c07ad054..5af2d6c45b 100644 --- a/src/compaction_client/src/compaction_impl.rs +++ b/src/compaction_client/src/compaction_impl.rs @@ -19,13 +19,19 @@ use std::sync::Arc; use async_trait::async_trait; use generic_error::BoxError; -use horaedbproto::{common::ResponseHeader, compaction_service::{self, compaction_service_client::CompactionServiceClient}}; +use horaedbproto::{ + common::ResponseHeader, + compaction_service::{self, compaction_service_client::CompactionServiceClient} +}; use logger::info; use serde::{Deserialize, Serialize}; use snafu::{ResultExt, OptionExt}; use time_ext::ReadableDuration; -use crate::{BadResponse, CompactionClient, CompactionClientRef, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse, FailConnect, FailExecuteCompactionTask, MissingHeader, Result}; +use crate::{ + BadResponse, CompactionClient, CompactionClientRef, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse, + FailConnect, FailExecuteCompactionTask, MissingHeader, Result, +}; type CompactionServiceGrpcClient = CompactionServiceClient; diff --git a/src/meta_client/src/lib.rs b/src/meta_client/src/lib.rs index 9d0bfd4863..19e204182d 100644 --- a/src/meta_client/src/lib.rs +++ b/src/meta_client/src/lib.rs @@ -95,7 +95,7 @@ pub enum Error { define_result!(Error); /// MetaClient is the abstraction of client used to communicate with HoraeMeta -// cluster. +/// cluster. #[async_trait] pub trait MetaClient: Send + Sync { async fn alloc_schema_id(&self, req: AllocSchemaIdRequest) -> Result; diff --git a/src/meta_client/src/types.rs b/src/meta_client/src/types.rs index 1e619768d6..97c25a1008 100644 --- a/src/meta_client/src/types.rs +++ b/src/meta_client/src/types.rs @@ -19,7 +19,9 @@ use std::{collections::HashMap, fmt, sync::Arc}; pub use common_types::table::{ShardId, ShardVersion}; use common_types::{ - cluster::ClusterType, schema::{SchemaId, SchemaName}, table::{TableId, TableName} + cluster::ClusterType, + schema::{SchemaId, SchemaName}, + table::{TableId, TableName}, }; use generic_error::BoxError; use horaedbproto::{cluster as cluster_pb, meta_service as meta_service_pb}; diff --git a/src/server/src/grpc/compaction_service/mod.rs b/src/server/src/grpc/compaction_service/mod.rs index 6c776942f6..1d6d266e31 100644 --- a/src/server/src/grpc/compaction_service/mod.rs +++ b/src/server/src/grpc/compaction_service/mod.rs @@ -17,15 +17,16 @@ // Compaction rpc service implementation. -#![allow(dead_code)] - use std::sync::Arc; use analytic_engine::compaction::runner::{CompactionRunnerRef, CompactionRunnerTask}; use async_trait::async_trait; use cluster::ClusterRef; use generic_error::BoxError; -use horaedbproto::compaction_service::{compaction_service_server::CompactionService, ExecResult, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; +use horaedbproto::compaction_service::{ + compaction_service_server::CompactionService, ExecResult, ExecuteCompactionTaskRequest, + ExecuteCompactionTaskResponse, +}; use proxy::instance::InstanceRef; use runtime::Runtime; use snafu::ResultExt; @@ -60,7 +61,6 @@ impl Builder { } } - #[derive(Clone)] pub struct CompactionServiceImpl { pub cluster: ClusterRef, @@ -100,7 +100,7 @@ impl CompactionService for CompactionServiceImpl { sst_info: Some(res.sst_info.into()), sst_meta: Some(res.sst_meta.into()), }); - // TODO: Add status. + // TODO(leslie): Add status. } Err(e) => { resp.header = Some(build_err_header(e)); diff --git a/src/server/src/grpc/mod.rs b/src/server/src/grpc/mod.rs index f791669dee..fb4cdf87e9 100644 --- a/src/server/src/grpc/mod.rs +++ b/src/server/src/grpc/mod.rs @@ -31,7 +31,10 @@ use compaction_service::CompactionServiceImpl; use futures::FutureExt; use generic_error::GenericError; use horaedbproto::{ - compaction_service::compaction_service_server::CompactionServiceServer, meta_event::meta_event_service_server::MetaEventServiceServer, remote_engine::remote_engine_service_server::RemoteEngineServiceServer, storage::storage_service_server::StorageServiceServer + compaction_service::compaction_service_server::CompactionServiceServer, + meta_event::meta_event_service_server::MetaEventServiceServer, + remote_engine::remote_engine_service_server::RemoteEngineServiceServer, + storage::storage_service_server::StorageServiceServer, }; use logger::{info, warn}; use macros::define_result; @@ -169,7 +172,6 @@ pub struct RpcServices { rpc_server: InterceptedService, AuthWithFile>, compaction_rpc_server: Option>, meta_rpc_server: Option>, - // TODO: Consider make remote engine service server optional here. remote_engine_server: RemoteEngineServiceServer, runtime: Arc, stop_tx: Option>, @@ -331,6 +333,7 @@ impl Builder { let result: Result<()> = (|| { match v.cluster_type() { ClusterType::HoraeDB => { + // Support meta rpc service. let opened_wals = self.opened_wals.context(MissingWals)?; let builder = meta_event_service::Builder { cluster: v, @@ -341,7 +344,10 @@ impl Builder { meta_rpc_server = Some(MetaEventServiceServer::new(builder.build())); } ClusterType::CompactionServer => { - let compaction_runner = self.compaction_runner.context(MissingCompactionRunner)?; + // Support remote rpc service. + let compaction_runner = self + .compaction_runner + .context(MissingCompactionRunner)?; let builder = compaction_service::Builder { cluster: v, instance: instance.clone(), From f306c1430ca928833bdbcb19fee8e860c36524f5 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Fri, 16 Aug 2024 12:03:59 +0800 Subject: [PATCH 17/49] impl type conversation. --- Cargo.toml | 2 +- src/analytic_engine/src/compaction/mod.rs | 7 +------ src/analytic_engine/src/sst/file.rs | 13 ++++++++----- src/cluster/src/cluster_impl.rs | 1 + 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index d17050553b..81d2b0050b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -102,7 +102,7 @@ thiserror = "1" bytes_ext = { path = "src/components/bytes_ext" } catalog = { path = "src/catalog" } catalog_impls = { path = "src/catalog_impls" } -horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "5de87a20dccfdde364ba5bea0a591f29e6556f91" } +horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "eae28015f26f2736227b05cd50e3cb0ab2561fef" } codec = { path = "src/components/codec" } chrono = "0.4" clap = { version = "4.5.1", features = ["derive"] } diff --git a/src/analytic_engine/src/compaction/mod.rs b/src/analytic_engine/src/compaction/mod.rs index 9a8f4dd60c..54e3318e4a 100644 --- a/src/analytic_engine/src/compaction/mod.rs +++ b/src/analytic_engine/src/compaction/mod.rs @@ -84,9 +84,6 @@ pub enum Error { #[snafu(display("Empty purge queue.\nBacktrace:\n{}", backtrace))] EmptyPurgeQueue { backtrace: Backtrace }, - #[snafu(display("Failed to convert purge queue, err:{}", source))] - ConvertPurgeQueue { source: GenericError }, - #[snafu(display("Failed to convert level, err:{}", source))] ConvertLevel { source: GenericError }, } @@ -364,9 +361,7 @@ impl TryFrom for Compact let purge_queue: FilePurgeQueue = file .purge_queue .context(EmptyPurgeQueue)? - .try_into() - .box_err() - .context(ConvertPurgeQueue)?; + .into(); files.push({ let handle = FileHandle::new(meta, purge_queue); diff --git a/src/analytic_engine/src/sst/file.rs b/src/analytic_engine/src/sst/file.rs index a97f181ebf..b97c6b7cc7 100644 --- a/src/analytic_engine/src/sst/file.rs +++ b/src/analytic_engine/src/sst/file.rs @@ -568,11 +568,14 @@ impl FilePurgeQueue { } } -impl TryFrom for FilePurgeQueue { - type Error = Error; - - fn try_from(_value: horaedbproto::compaction_service::FilePurgeQueue) -> Result { - unimplemented!() +impl From for FilePurgeQueue { + fn from(value: horaedbproto::compaction_service::FilePurgeQueue) -> Self { + let (tx, _rx) = mpsc::unbounded_channel(); + FilePurgeQueue::new( + value.space_id.into(), + value.table_id.into(), + tx + ) } } diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index 322cc06fe6..11afabab36 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -368,6 +368,7 @@ impl Inner { } async fn compact(&self, req: &ExecuteCompactionTaskRequest) -> Result { + // TODO(leslie): Execute the compaction task locally when fails to build compaction client. let compact_resp = self .compaction_client() .await From c01ef00a6b01d2d37e42c7cf718912e27ce829f6 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Fri, 16 Aug 2024 12:18:48 +0800 Subject: [PATCH 18/49] remove dead code. --- src/server/src/grpc/compaction_service/error.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/server/src/grpc/compaction_service/error.rs b/src/server/src/grpc/compaction_service/error.rs index 7d1463cf6d..ac10996439 100644 --- a/src/server/src/grpc/compaction_service/error.rs +++ b/src/server/src/grpc/compaction_service/error.rs @@ -17,8 +17,6 @@ //! Error definitions for compaction service. -#![allow(dead_code)] - use generic_error::GenericError; use horaedbproto::common::ResponseHeader; use macros::define_result; @@ -73,7 +71,6 @@ pub enum StatusCode { #[default] Ok = 0, BadRequest = 401, - NotFound = 404, Internal = 500, } From f2c0e38aa319a9c260c017d7f7f47ad85261df49 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Fri, 16 Aug 2024 16:20:52 +0800 Subject: [PATCH 19/49] remove cluster type in meta client. --- Cargo.toml | 2 +- src/horaedb/src/setup.rs | 1 - src/meta_client/src/types.rs | 7 ------- 3 files changed, 1 insertion(+), 9 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 81d2b0050b..f75bc35691 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -102,7 +102,7 @@ thiserror = "1" bytes_ext = { path = "src/components/bytes_ext" } catalog = { path = "src/catalog" } catalog_impls = { path = "src/catalog_impls" } -horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "eae28015f26f2736227b05cd50e3cb0ab2561fef" } +horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "f5acee2e718d541ba04cd682b612a9fc7c93b25e" } codec = { path = "src/components/codec" } chrono = "0.4" clap = { version = "4.5.1", features = ["derive"] } diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index c074eeddfe..f7c94ae709 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -298,7 +298,6 @@ async fn build_with_meta( zone: config.node.zone.clone(), idc: config.node.idc.clone(), binary_version: config.node.binary_version.clone(), - cluster_type: cluster_config.cluster_type.clone(), }; info!("Build horaedb with node meta info:{node_meta_info:?}"); diff --git a/src/meta_client/src/types.rs b/src/meta_client/src/types.rs index 97c25a1008..6a6aba6918 100644 --- a/src/meta_client/src/types.rs +++ b/src/meta_client/src/types.rs @@ -19,7 +19,6 @@ use std::{collections::HashMap, fmt, sync::Arc}; pub use common_types::table::{ShardId, ShardVersion}; use common_types::{ - cluster::ClusterType, schema::{SchemaId, SchemaName}, table::{TableId, TableName}, }; @@ -164,7 +163,6 @@ pub struct NodeMetaInfo { pub zone: String, pub idc: String, pub binary_version: String, - pub cluster_type: ClusterType, } impl NodeMetaInfo { @@ -283,11 +281,6 @@ impl From for meta_service_pb::NodeInfo { binary_version: node_info.node_meta_info.binary_version, shard_infos, lease: 0, - cluster_type: if node_info.node_meta_info.cluster_type == ClusterType::HoraeDB { - meta_service_pb::node_info::ClusterType::HoraeDb - } else { - meta_service_pb::node_info::ClusterType::CompactionServer - } as i32, } } } From c6521a4c2ead6f059c9014d05a2d3d870b7abd37 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Fri, 16 Aug 2024 22:08:06 +0800 Subject: [PATCH 20/49] fix bug --- Cargo.toml | 1 + src/analytic_engine/src/memtable/mod.rs | 5 +++++ 2 files changed, 6 insertions(+) diff --git a/Cargo.toml b/Cargo.toml index f75bc35691..3102ce6fbb 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -102,6 +102,7 @@ thiserror = "1" bytes_ext = { path = "src/components/bytes_ext" } catalog = { path = "src/catalog" } catalog_impls = { path = "src/catalog_impls" } +# TODO(leslie): modify it when the related pr in incubator-horaedb-proto is merged. horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "f5acee2e718d541ba04cd682b612a9fc7c93b25e" } codec = { path = "src/components/codec" } chrono = "0.4" diff --git a/src/analytic_engine/src/memtable/mod.rs b/src/analytic_engine/src/memtable/mod.rs index f53bff149f..ed55f43371 100644 --- a/src/analytic_engine/src/memtable/mod.rs +++ b/src/analytic_engine/src/memtable/mod.rs @@ -119,6 +119,11 @@ impl From for manifest::LayeredMemtableOptions { fn from(value: LayeredMemtableOptions) -> Self { Self { mutable_segment_switch_threshold: value.mutable_segment_switch_threshold.0, + disable: if value.mutable_segment_switch_threshold.0 == 0 { + true + } else { + false + }, } } } From 723a533beceda70152182d53c32b72afde2093dd Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Fri, 16 Aug 2024 22:09:01 +0800 Subject: [PATCH 21/49] update Cargo.lock --- Cargo.lock | 28 +++++++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index c904497a31..1a7200313e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1460,6 +1460,7 @@ dependencies = [ "bytes_ext", "catalog", "common_types", + "compaction_client", "etcd-client", "future_ext", "generic_error", @@ -1559,6 +1560,28 @@ dependencies = [ "uuid", ] +[[package]] +name = "compaction_client" +version = "2.0.0" +dependencies = [ + "async-trait", + "common_types", + "futures 0.3.28", + "generic_error", + "horaedbproto 2.0.0", + "logger", + "macros", + "prost 0.11.8", + "reqwest", + "serde", + "serde_json", + "snafu 0.6.10", + "time_ext", + "tokio", + "tonic 0.8.3", + "url", +] + [[package]] name = "concurrent-queue" version = "2.1.0" @@ -3062,6 +3085,8 @@ dependencies = [ "catalog_impls", "clap", "cluster", + "common_types", + "compaction_client", "datafusion", "df_operator", "etcd-client", @@ -3135,7 +3160,7 @@ dependencies = [ [[package]] name = "horaedbproto" version = "2.0.0" -source = "git+https://github.com/apache/incubator-horaedb-proto.git?rev=19ece8f771fc0b3e8e734072cc3d8040de6c74cb#19ece8f771fc0b3e8e734072cc3d8040de6c74cb" +source = "git+https://github.com/LeslieKid/incubator-horaedb-proto.git?rev=f5acee2e718d541ba04cd682b612a9fc7c93b25e#f5acee2e718d541ba04cd682b612a9fc7c93b25e" dependencies = [ "prost 0.11.8", "protoc-bin-vendored", @@ -6005,6 +6030,7 @@ dependencies = [ "async-trait", "cluster", "common_types", + "compaction_client", "generic_error", "horaedbproto 2.0.0", "logger", From 3da8fe5baf761e1abdf5a1c806bce27367cd13a0 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Wed, 21 Aug 2024 02:58:14 +0800 Subject: [PATCH 22/49] fix style. --- src/analytic_engine/src/compaction/mod.rs | 23 +++---- .../src/compaction/runner/mod.rs | 30 +++++---- src/analytic_engine/src/memtable/mod.rs | 6 +- src/analytic_engine/src/sst/factory.rs | 8 +-- src/analytic_engine/src/sst/file.rs | 13 +--- src/analytic_engine/src/sst/writer.rs | 2 +- src/analytic_engine/src/table_options.rs | 4 +- src/cluster/src/cluster_impl.rs | 29 ++++---- src/cluster/src/lib.rs | 7 +- src/common_types/src/cluster.rs | 2 +- src/compaction_client/src/compaction_impl.rs | 44 +++++++------ src/compaction_client/src/lib.rs | 12 ++-- src/compaction_client/src/types.rs | 2 +- src/horaedb/bin/horaedb-server.rs | 2 +- src/horaedb/src/config.rs | 4 +- src/horaedb/src/setup.rs | 10 ++- src/meta_client/src/lib.rs | 2 +- src/router/src/cluster_based.rs | 7 +- .../src/grpc/compaction_service/error.rs | 6 +- src/server/src/grpc/compaction_service/mod.rs | 36 +++++----- src/server/src/grpc/mod.rs | 66 ++++++++++--------- src/server/src/server.rs | 2 +- src/table_engine/src/predicate.rs | 4 +- 23 files changed, 171 insertions(+), 150 deletions(-) diff --git a/src/analytic_engine/src/compaction/mod.rs b/src/analytic_engine/src/compaction/mod.rs index 54e3318e4a..dce46d3e9c 100644 --- a/src/analytic_engine/src/compaction/mod.rs +++ b/src/analytic_engine/src/compaction/mod.rs @@ -24,13 +24,13 @@ use generic_error::{BoxError, GenericError}; use macros::define_result; use serde::{Deserialize, Serialize}; use size_ext::ReadableSize; -use snafu::{ensure, Backtrace, GenerateBacktrace, ResultExt, OptionExt, Snafu}; +use snafu::{ensure, Backtrace, GenerateBacktrace, OptionExt, ResultExt, Snafu}; use time_ext::TimeUnit; use tokio::sync::oneshot; use crate::{ compaction::picker::{CommonCompactionPicker, CompactionPickerRef}, - sst::file::{FileHandle, Level, FileMeta, FilePurgeQueue}, + sst::file::{FileHandle, FileMeta, FilePurgeQueue, Level}, table::data::TableDataRef, }; @@ -75,12 +75,12 @@ pub enum Error { #[snafu(display("Invalid compaction option value, err: {}", error))] InvalidOption { error: String, backtrace: Backtrace }, - #[snafu(display("Empty file meta.\nBacktrace:\n{}", backtrace))] + #[snafu(display("Empty file meta.\nBacktrace:\n{}", backtrace))] EmptyFileMeta { backtrace: Backtrace }, #[snafu(display("Failed to convert file meta, err:{}", source))] ConvertFileMeta { source: GenericError }, - + #[snafu(display("Empty purge queue.\nBacktrace:\n{}", backtrace))] EmptyPurgeQueue { backtrace: Backtrace }, @@ -347,21 +347,22 @@ impl TryFrom for Compact fn try_from(value: horaedbproto::compaction_service::CompactionInputFiles) -> Result { let level: Level = value.level.try_into().box_err().context(ConvertLevel)?; - let output_level: Level = value.output_level.try_into().box_err().context(ConvertLevel)?; - + let output_level: Level = value + .output_level + .try_into() + .box_err() + .context(ConvertLevel)?; + let mut files: Vec = Vec::with_capacity(value.files.len()); for file in value.files { - let meta: FileMeta = file + let meta: FileMeta = file .meta .context(EmptyFileMeta)? .try_into() .box_err() .context(ConvertFileMeta)?; - let purge_queue: FilePurgeQueue = file - .purge_queue - .context(EmptyPurgeQueue)? - .into(); + let purge_queue: FilePurgeQueue = file.purge_queue.context(EmptyPurgeQueue)?.into(); files.push({ let handle = FileHandle::new(meta, purge_queue); diff --git a/src/analytic_engine/src/compaction/runner/mod.rs b/src/analytic_engine/src/compaction/runner/mod.rs index ed354e89a2..39ef356f36 100644 --- a/src/analytic_engine/src/compaction/runner/mod.rs +++ b/src/analytic_engine/src/compaction/runner/mod.rs @@ -24,26 +24,28 @@ use common_types::{request_id::RequestId, schema::Schema, SequenceNumber}; use generic_error::{BoxError, GenericError}; use macros::define_result; use object_store::Path; -use snafu::{Backtrace, Snafu, OptionExt, ResultExt}; +use snafu::{Backtrace, OptionExt, ResultExt, Snafu}; use table_engine::table::TableId; use crate::{ compaction::CompactionInputFiles, - instance::flush_compaction, + instance::flush_compaction, row_iter::IterOptions, - space::SpaceId, + space::SpaceId, sst::{ factory::SstWriteOptions, writer::{MetaData, SstInfo}, - }, + }, table::data::TableData, }; /// Compaction runner #[async_trait] pub trait CompactionRunner: Send + Sync + 'static { - async fn run(&self, task: CompactionRunnerTask) -> - flush_compaction::Result; + async fn run( + &self, + task: CompactionRunnerTask, + ) -> flush_compaction::Result; } pub type CompactionRunnerPtr = Box; @@ -71,7 +73,7 @@ pub enum Error { #[snafu(display("Failed to convert input context, err:{}", source))] ConvertInputContext { source: GenericError }, - + #[snafu(display("Failed to convert ouput context, err:{}", source))] ConvertOuputContext { source: GenericError }, @@ -152,10 +154,14 @@ impl CompactionRunnerTask { } } -impl TryFrom for CompactionRunnerTask { +impl TryFrom + for CompactionRunnerTask +{ type Error = Error; - fn try_from(request: horaedbproto::compaction_service::ExecuteCompactionTaskRequest) -> Result { + fn try_from( + request: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, + ) -> Result { let task_key = request.task_key; let request_id: RequestId = request.request_id.into(); @@ -166,9 +172,9 @@ impl TryFrom for .box_err() .context(ConvertTableSchema)?; - let space_id: SpaceId = request.space_id.into(); + let space_id: SpaceId = request.space_id; let table_id: TableId = request.table_id.into(); - let sequence: SequenceNumber = request.sequence.into(); + let sequence: SequenceNumber = request.sequence; let input_ctx: InputContext = request .input_ctx @@ -218,7 +224,7 @@ impl TryFrom for InputContext { fn try_from(value: horaedbproto::compaction_service::InputContext) -> Result { let num_rows_per_row_group: usize = value.num_rows_per_row_group as usize; let merge_iter_options = IterOptions { - batch_size: value.merge_iter_options as usize + batch_size: value.merge_iter_options as usize, }; let need_dedup = value.need_dedup; diff --git a/src/analytic_engine/src/memtable/mod.rs b/src/analytic_engine/src/memtable/mod.rs index ed55f43371..b21c54da27 100644 --- a/src/analytic_engine/src/memtable/mod.rs +++ b/src/analytic_engine/src/memtable/mod.rs @@ -119,11 +119,7 @@ impl From for manifest::LayeredMemtableOptions { fn from(value: LayeredMemtableOptions) -> Self { Self { mutable_segment_switch_threshold: value.mutable_segment_switch_threshold.0, - disable: if value.mutable_segment_switch_threshold.0 == 0 { - true - } else { - false - }, + disable: value.mutable_segment_switch_threshold.0 == 0, } } } diff --git a/src/analytic_engine/src/sst/factory.rs b/src/analytic_engine/src/sst/factory.rs index 3b6832dcdc..51703d771e 100644 --- a/src/analytic_engine/src/sst/factory.rs +++ b/src/analytic_engine/src/sst/factory.rs @@ -25,7 +25,7 @@ use generic_error::{BoxError, GenericError}; use macros::define_result; use object_store::{ObjectStoreRef, Path}; use runtime::Runtime; -use snafu::{Backtrace, ResultExt, Snafu, OptionExt}; +use snafu::{Backtrace, OptionExt, ResultExt, Snafu}; use table_engine::predicate::PredicateRef; use trace_metric::MetricsCollector; @@ -196,9 +196,7 @@ impl TryFrom for SstWriteOpti let column_stats: HashMap = value .column_stats .into_iter() - .map(|(k,v)| (k.into(), ColumnStats { - low_cardinality: v - })) + .map(|(k, v)| (k, ColumnStats { low_cardinality: v })) .collect(); Ok(SstWriteOptions { @@ -206,7 +204,7 @@ impl TryFrom for SstWriteOpti num_rows_per_row_group, compression, max_buffer_size, - column_stats, + column_stats, }) } } diff --git a/src/analytic_engine/src/sst/file.rs b/src/analytic_engine/src/sst/file.rs index b97c6b7cc7..33c4bcb96b 100644 --- a/src/analytic_engine/src/sst/file.rs +++ b/src/analytic_engine/src/sst/file.rs @@ -41,7 +41,7 @@ use macros::define_result; use metric_ext::Meter; use object_store::{ObjectStoreRef, Path}; use runtime::{JoinHandle, Runtime}; -use snafu::{Backtrace, ResultExt, OptionExt, Snafu}; +use snafu::{Backtrace, OptionExt, ResultExt, Snafu}; use table_engine::table::TableId; use tokio::sync::{ mpsc::{self, UnboundedReceiver, UnboundedSender}, @@ -112,10 +112,7 @@ impl TryFrom for Level { type Error = Error; fn try_from(value: u32) -> Result { - let value: u16 = value - .try_into() - .box_err() - .context(ConvertOverflow)?; + let value: u16 = value.try_into().box_err().context(ConvertOverflow)?; Ok(value.into()) } } @@ -571,11 +568,7 @@ impl FilePurgeQueue { impl From for FilePurgeQueue { fn from(value: horaedbproto::compaction_service::FilePurgeQueue) -> Self { let (tx, _rx) = mpsc::unbounded_channel(); - FilePurgeQueue::new( - value.space_id.into(), - value.table_id.into(), - tx - ) + FilePurgeQueue::new(value.space_id, value.table_id.into(), tx) } } diff --git a/src/analytic_engine/src/sst/writer.rs b/src/analytic_engine/src/sst/writer.rs index f8bb9226ab..280d279f5b 100644 --- a/src/analytic_engine/src/sst/writer.rs +++ b/src/analytic_engine/src/sst/writer.rs @@ -125,7 +125,7 @@ impl From for horaedbproto::compaction_service::SstInfo { storage_format: value.storage_format.into(), meta_path: value.meta_path, time_range: Some(value.time_range.into()), - } + } } } diff --git a/src/analytic_engine/src/table_options.rs b/src/analytic_engine/src/table_options.rs index 3382198546..dec42074b3 100644 --- a/src/analytic_engine/src/table_options.rs +++ b/src/analytic_engine/src/table_options.rs @@ -370,7 +370,7 @@ impl From for i32 { fn from(value: StorageFormat) -> Self { match value { StorageFormat::Columnar => 0, - } + } } } @@ -405,7 +405,7 @@ impl TryFrom for StorageFormat { 0 => Self::Columnar, _ => return UnknownStorageFormatType { value }.fail(), }; - Ok(format) + Ok(format) } } diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index 11afabab36..c51c64dfe3 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -24,10 +24,8 @@ use async_trait::async_trait; use common_types::table::ShardId; use compaction_client::{ compaction_impl::{build_compaction_client, CompactionClientConfig}, - types::{ - ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse - }, - CompactionClientRef + types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}, + CompactionClientRef, }; use etcd_client::{Certificate, ConnectOptions, Identity, TlsOptions}; use generic_error::BoxError; @@ -53,8 +51,9 @@ use crate::{ shard_set::{Shard, ShardRef, ShardSet}, topology::ClusterTopology, Cluster, ClusterNodesNotFound, ClusterNodesResp, ClusterType, CompactionClientFailure, - CompactionOffloadNotAllowed, EtcdClientFailureWithCause, InitEtcdClientConfig, InvalidArguments, - MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, TableStatus + CompactionOffloadNotAllowed, EtcdClientFailureWithCause, InitEtcdClientConfig, + InvalidArguments, MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, + TableStatus, }; /// ClusterImpl is an implementation of [`Cluster`] based [`MetaClient`]. @@ -349,7 +348,8 @@ impl Inner { shards.iter().map(|shard| shard.shard_info()).collect() } - /// Get proper remote compaction node for compaction offload with meta client. + /// Get proper remote compaction node for compaction offload with meta + /// client. async fn get_compaction_node(&self) -> Result { unimplemented!() } @@ -367,8 +367,12 @@ impl Inner { .expect("fail to build compaction client") } - async fn compact(&self, req: &ExecuteCompactionTaskRequest) -> Result { - // TODO(leslie): Execute the compaction task locally when fails to build compaction client. + async fn compact( + &self, + req: &ExecuteCompactionTaskRequest, + ) -> Result { + // TODO(leslie): Execute the compaction task locally when fails to build + // compaction client. let compact_resp = self .compaction_client() .await @@ -454,12 +458,15 @@ impl Cluster for ClusterImpl { self.shard_lock_manager.clone() } - async fn compact(&self, req: &ExecuteCompactionTaskRequest) -> Result { + async fn compact( + &self, + req: &ExecuteCompactionTaskRequest, + ) -> Result { ensure!( self.cluster_type() == ClusterType::HoraeDB, CompactionOffloadNotAllowed { cluster_type: self.cluster_type() - } + } ); self.inner.compact(req).await } diff --git a/src/cluster/src/lib.rs b/src/cluster/src/lib.rs index 4b4dfc5eff..56e287454b 100644 --- a/src/cluster/src/lib.rs +++ b/src/cluster/src/lib.rs @@ -211,7 +211,7 @@ pub trait Cluster { async fn start(&self) -> Result<()>; async fn stop(&self) -> Result<()>; - + /// Get cluster type. fn cluster_type(&self) -> ClusterType; @@ -239,5 +239,8 @@ pub trait Cluster { fn shard_lock_manager(&self) -> ShardLockManagerRef; /// Execute compaction task in remote compaction node. - async fn compact(&self, req: &ExecuteCompactionTaskRequest) -> Result; + async fn compact( + &self, + req: &ExecuteCompactionTaskRequest, + ) -> Result; } diff --git a/src/common_types/src/cluster.rs b/src/common_types/src/cluster.rs index 7c790587a1..7f78c1f793 100644 --- a/src/common_types/src/cluster.rs +++ b/src/common_types/src/cluster.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use serde::{Serialize, Deserialize}; +use serde::{Deserialize, Serialize}; /// Type to distinguish different server type in cluster mode. #[derive(Debug, Default, Clone, Serialize, Deserialize, PartialEq, Eq)] diff --git a/src/compaction_client/src/compaction_impl.rs b/src/compaction_client/src/compaction_impl.rs index 5af2d6c45b..1e14b4df91 100644 --- a/src/compaction_client/src/compaction_impl.rs +++ b/src/compaction_client/src/compaction_impl.rs @@ -20,17 +20,17 @@ use std::sync::Arc; use async_trait::async_trait; use generic_error::BoxError; use horaedbproto::{ - common::ResponseHeader, - compaction_service::{self, compaction_service_client::CompactionServiceClient} + common::ResponseHeader, + compaction_service::{self, compaction_service_client::CompactionServiceClient}, }; use logger::info; use serde::{Deserialize, Serialize}; -use snafu::{ResultExt, OptionExt}; +use snafu::{OptionExt, ResultExt}; use time_ext::ReadableDuration; use crate::{ - BadResponse, CompactionClient, CompactionClientRef, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse, - FailConnect, FailExecuteCompactionTask, MissingHeader, Result, + BadResponse, CompactionClient, CompactionClientRef, ExecuteCompactionTaskRequest, + ExecuteCompactionTaskResponse, FailConnect, FailExecuteCompactionTask, MissingHeader, Result, }; type CompactionServiceGrpcClient = CompactionServiceClient; @@ -49,7 +49,8 @@ impl Default for CompactionClientConfig { } } -/// Default compaction client impl, will interact with the remote compaction node. +/// Default compaction client impl, will interact with the remote compaction +/// node. pub struct CompactionClientImpl { client: CompactionServiceGrpcClient, } @@ -57,12 +58,13 @@ pub struct CompactionClientImpl { impl CompactionClientImpl { pub async fn connect(config: CompactionClientConfig) -> Result { let client = { - let endpoint = tonic::transport::Endpoint::from_shared(config.compaction_server_addr.to_string()) - .box_err() - .context(FailConnect { - addr: &config.compaction_server_addr, - })? - .timeout(config.timeout.0); + let endpoint = + tonic::transport::Endpoint::from_shared(config.compaction_server_addr.to_string()) + .box_err() + .context(FailConnect { + addr: &config.compaction_server_addr, + })? + .timeout(config.timeout.0); CompactionServiceGrpcClient::connect(endpoint) .await .box_err() @@ -71,9 +73,7 @@ impl CompactionClientImpl { })? }; - Ok(Self { - client, - }) + Ok(Self { client }) } #[inline] @@ -82,17 +82,18 @@ impl CompactionClientImpl { } } - #[async_trait] impl CompactionClient for CompactionClientImpl { - async fn execute_compaction_task(&self, req: ExecuteCompactionTaskRequest) - -> Result { + async fn execute_compaction_task( + &self, + req: ExecuteCompactionTaskRequest, + ) -> Result { let pb_req = compaction_service::ExecuteCompactionTaskRequest::from(req); // TODO(leslie): Add request header for ExecuteCompactionTaskRequest. info!( - "Compaction client try to execute compaction task in remote compaction node, req:{:?}", + "Compaction client try to execute compaction task in remote compaction node, req:{:?}", pb_req ); @@ -102,7 +103,7 @@ impl CompactionClient for CompactionClientImpl { .await .box_err() .context(FailExecuteCompactionTask)? - .into_inner(); + .into_inner(); info!( "Compaction client finish executing compaction task in remote compaction node, req:{:?}", @@ -114,7 +115,8 @@ impl CompactionClient for CompactionClientImpl { } } -// TODO(leslie): Consider to refactor and reuse the similar function in meta_client. +// TODO(leslie): Consider to refactor and reuse the similar function in +// meta_client. fn check_response_header(header: &Option) -> Result<()> { let header = header.as_ref().context(MissingHeader)?; if header.code == 0 { diff --git a/src/compaction_client/src/lib.rs b/src/compaction_client/src/lib.rs index 64c5614ad8..c8ec47f246 100644 --- a/src/compaction_client/src/lib.rs +++ b/src/compaction_client/src/lib.rs @@ -20,11 +20,11 @@ use std::sync::Arc; use async_trait::async_trait; use generic_error::GenericError; use macros::define_result; -use types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; use snafu::{Backtrace, Snafu}; +use types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; -pub mod types; pub mod compaction_impl; +pub mod types; #[derive(Debug, Snafu)] #[snafu(visibility = "pub")] @@ -62,12 +62,14 @@ pub enum Error { define_result!(Error); -/// CompactionClient is the abstraction of client used for HoraeDB to +/// CompactionClient is the abstraction of client used for HoraeDB to /// communicate with CompactionServer cluster. #[async_trait] pub trait CompactionClient: Send + Sync { - async fn execute_compaction_task(&self, req: ExecuteCompactionTaskRequest) - -> Result; + async fn execute_compaction_task( + &self, + req: ExecuteCompactionTaskRequest, + ) -> Result; } pub type CompactionClientRef = Arc; diff --git a/src/compaction_client/src/types.rs b/src/compaction_client/src/types.rs index a274286613..ca72149677 100644 --- a/src/compaction_client/src/types.rs +++ b/src/compaction_client/src/types.rs @@ -38,6 +38,6 @@ impl TryFrom for ExecuteCompa type Error = Error; fn try_from(_value: compaction_service::ExecuteCompactionTaskResponse) -> Result { - unimplemented!() + unimplemented!() } } diff --git a/src/horaedb/bin/horaedb-server.rs b/src/horaedb/bin/horaedb-server.rs index 402be90146..bf6662085d 100644 --- a/src/horaedb/bin/horaedb-server.rs +++ b/src/horaedb/bin/horaedb-server.rs @@ -98,7 +98,7 @@ fn main() { if let Ok(cluster) = env::var(CLUSTER_NAME) { if let Some(ClusterDeployment::WithMeta(v)) = &mut config.cluster_deployment { v.meta_client.cluster_name = cluster; - } + } } println!("HoraeDB server tries starting with config:{config:?}"); diff --git a/src/horaedb/src/config.rs b/src/horaedb/src/config.rs index 2f15f5269b..e7f19233f0 100644 --- a/src/horaedb/src/config.rs +++ b/src/horaedb/src/config.rs @@ -26,8 +26,8 @@ use size_ext::ReadableSize; #[derive(Clone, Debug, Deserialize, Serialize)] #[serde(default)] pub struct NodeInfo { - /// The address of the horaedb (or compaction server) node. It can be a domain - /// name or an IP address without port followed. + /// The address of the horaedb (or compaction server) node. It can be a + /// domain name or an IP address without port followed. pub addr: String, pub zone: String, pub idc: String, diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index 8f1dd2186c..aaf7f80cdd 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -323,7 +323,7 @@ async fn build_with_meta( meta_impl::build_meta_client(cluster_config.meta_client.clone(), node_meta_info) .await .expect("fail to build meta client"); - + let shard_set = ShardSet::default(); let cluster = { let cluster_impl = ClusterImpl::try_new( @@ -351,7 +351,10 @@ async fn build_with_meta( engine_runtimes: runtimes.clone(), opened_wals: opened_wals.clone(), }; - let TableEngineContext { table_engine, local_compaction_runner } = engine_builder + let TableEngineContext { + table_engine, + local_compaction_runner, + } = engine_builder .build() .await .expect("Failed to setup analytic engine"); @@ -379,7 +382,8 @@ async fn build_with_meta( .router(router) .schema_config_provider(schema_config_provider); if let ClusterType::CompactionServer = cluster_config.cluster_type { - builder = builder.compaction_runner(local_compaction_runner.expect("Empty compaction runner.")); + builder = + builder.compaction_runner(local_compaction_runner.expect("Empty compaction runner.")); } builder } diff --git a/src/meta_client/src/lib.rs b/src/meta_client/src/lib.rs index 19e204182d..a6cb8df6b9 100644 --- a/src/meta_client/src/lib.rs +++ b/src/meta_client/src/lib.rs @@ -94,7 +94,7 @@ pub enum Error { define_result!(Error); -/// MetaClient is the abstraction of client used to communicate with HoraeMeta +/// MetaClient is the abstraction of client used to communicate with HoraeMeta /// cluster. #[async_trait] pub trait MetaClient: Send + Sync { diff --git a/src/router/src/cluster_based.rs b/src/router/src/cluster_based.rs index 4734df983a..01e67deb05 100644 --- a/src/router/src/cluster_based.rs +++ b/src/router/src/cluster_based.rs @@ -203,7 +203,7 @@ mod tests { use cluster::{ shard_lock_manager::ShardLockManagerRef, shard_set::ShardRef, Cluster, ClusterNodesResp, - TableStatus + TableStatus, }; use common_types::{cluster::ClusterType, table::ShardId}; use compaction_client::types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; @@ -296,7 +296,10 @@ mod tests { unimplemented!(); } - async fn compact(&self, _req: &ExecuteCompactionTaskRequest) -> cluster::Result { + async fn compact( + &self, + _req: &ExecuteCompactionTaskRequest, + ) -> cluster::Result { unimplemented!() } } diff --git a/src/server/src/grpc/compaction_service/error.rs b/src/server/src/grpc/compaction_service/error.rs index ac10996439..eadb3f2418 100644 --- a/src/server/src/grpc/compaction_service/error.rs +++ b/src/server/src/grpc/compaction_service/error.rs @@ -16,16 +16,16 @@ // under the License. //! Error definitions for compaction service. - + use generic_error::GenericError; use horaedbproto::common::ResponseHeader; use macros::define_result; use snafu::Snafu; use crate::error_util; - + define_result!(Error); - + #[derive(Snafu, Debug)] #[snafu(visibility(pub))] pub enum Error { diff --git a/src/server/src/grpc/compaction_service/mod.rs b/src/server/src/grpc/compaction_service/mod.rs index 1d6d266e31..c2635d585c 100644 --- a/src/server/src/grpc/compaction_service/mod.rs +++ b/src/server/src/grpc/compaction_service/mod.rs @@ -22,6 +22,7 @@ use std::sync::Arc; use analytic_engine::compaction::runner::{CompactionRunnerRef, CompactionRunnerTask}; use async_trait::async_trait; use cluster::ClusterRef; +use error::{build_err_header, build_ok_header, ErrWithCause, StatusCode}; use generic_error::BoxError; use horaedbproto::compaction_service::{ compaction_service_server::CompactionService, ExecResult, ExecuteCompactionTaskRequest, @@ -31,7 +32,6 @@ use proxy::instance::InstanceRef; use runtime::Runtime; use snafu::ResultExt; use tonic::{Request, Response, Status}; -use error::{build_err_header, build_ok_header, ErrWithCause, StatusCode}; mod error; @@ -75,23 +75,29 @@ impl CompactionService for CompactionServiceImpl { &self, request: Request, ) -> Result, Status> { - let request: Result = request.into_inner().try_into().box_err().context(ErrWithCause { - code: StatusCode::BadRequest, - msg: "fail to convert the execute compaction task request", - }); + let request: Result = request + .into_inner() + .try_into() + .box_err() + .context(ErrWithCause { + code: StatusCode::BadRequest, + msg: "fail to convert the execute compaction task request", + }); let mut resp: ExecuteCompactionTaskResponse = ExecuteCompactionTaskResponse::default(); match request { Ok(task) => { let request_id = task.request_id.clone(); - let res = self.compaction_runner.run(task).await - .box_err().with_context(|| { - ErrWithCause { - code: StatusCode::Internal, - msg: format!("fail to compact task, request:{request_id}") - } - }); - + let res = self + .compaction_runner + .run(task) + .await + .box_err() + .with_context(|| ErrWithCause { + code: StatusCode::Internal, + msg: format!("fail to compact task, request:{request_id}"), + }); + match res { Ok(res) => { resp.header = Some(build_ok_header()); @@ -100,13 +106,13 @@ impl CompactionService for CompactionServiceImpl { sst_info: Some(res.sst_info.into()), sst_meta: Some(res.sst_meta.into()), }); - // TODO(leslie): Add status. + // TODO(leslie): Add status. } Err(e) => { resp.header = Some(build_err_header(e)); } } - }, + } Err(e) => { resp.header = Some(build_err_header(e)); } diff --git a/src/server/src/grpc/mod.rs b/src/server/src/grpc/mod.rs index fb4cdf87e9..0a81916ccf 100644 --- a/src/server/src/grpc/mod.rs +++ b/src/server/src/grpc/mod.rs @@ -326,41 +326,43 @@ impl Builder { let proxy = self.proxy.context(MissingProxy)?; let hotspot_recorder = self.hotspot_recorder.context(MissingHotspotRecorder)?; let mut meta_rpc_server: Option> = None; - let mut compaction_rpc_server: Option> = None; + let mut compaction_rpc_server: Option> = + None; self.cluster - .map(|v| { - let result: Result<()> = (|| { - match v.cluster_type() { - ClusterType::HoraeDB => { - // Support meta rpc service. - let opened_wals = self.opened_wals.context(MissingWals)?; - let builder = meta_event_service::Builder { - cluster: v, - instance: instance.clone(), - runtime: runtimes.meta_runtime.clone(), - opened_wals, - }; - meta_rpc_server = Some(MetaEventServiceServer::new(builder.build())); + .map(|v| { + let result: Result<()> = (|| { + match v.cluster_type() { + ClusterType::HoraeDB => { + // Support meta rpc service. + let opened_wals = self.opened_wals.context(MissingWals)?; + let builder = meta_event_service::Builder { + cluster: v, + instance: instance.clone(), + runtime: runtimes.meta_runtime.clone(), + opened_wals, + }; + meta_rpc_server = Some(MetaEventServiceServer::new(builder.build())); + } + ClusterType::CompactionServer => { + // Support remote rpc service. + let compaction_runner = + self.compaction_runner.context(MissingCompactionRunner)?; + let builder = compaction_service::Builder { + cluster: v, + instance: instance.clone(), + runtime: runtimes.compact_runtime.clone(), + compaction_runner, + }; + compaction_rpc_server = + Some(CompactionServiceServer::new(builder.build())); + } } - ClusterType::CompactionServer => { - // Support remote rpc service. - let compaction_runner = self - .compaction_runner - .context(MissingCompactionRunner)?; - let builder = compaction_service::Builder { - cluster: v, - instance: instance.clone(), - runtime: runtimes.compact_runtime.clone(), - compaction_runner, - }; - compaction_rpc_server = Some(CompactionServiceServer::new(builder.build())); - } - } - Ok(()) - })(); - result - }).transpose()?; + Ok(()) + })(); + result + }) + .transpose()?; let remote_engine_server = { let query_dedup = self diff --git a/src/server/src/server.rs b/src/server/src/server.rs index e49ddcd88b..bca6c8d151 100644 --- a/src/server/src/server.rs +++ b/src/server/src/server.rs @@ -178,7 +178,7 @@ impl Server { .context(OpenLocalTables)?; } - // Run in cluster mode + // Run in cluster mode if let Some(cluster) = &self.cluster { info!("Server start, start cluster"); cluster.start().await.context(StartCluster)?; diff --git a/src/table_engine/src/predicate.rs b/src/table_engine/src/predicate.rs index f444cea534..3a3294fcd9 100644 --- a/src/table_engine/src/predicate.rs +++ b/src/table_engine/src/predicate.rs @@ -135,9 +135,7 @@ impl TryFrom<&Predicate> for horaedbproto::remote_engine::Predicate { impl TryFrom for Predicate { type Error = Error; - fn try_from( - pb: horaedbproto::remote_engine::Predicate, - ) -> Result { + fn try_from(pb: horaedbproto::remote_engine::Predicate) -> Result { let time_range = pb.time_range.context(EmptyTimeRange)?; let mut exprs = Vec::with_capacity(pb.exprs.len()); for pb_expr in pb.exprs { From debabab5fdb4bdc4daf5b1a0975ab8c5599b6d5c Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Wed, 21 Aug 2024 03:08:06 +0800 Subject: [PATCH 23/49] rename ClusterType to NodeType. --- src/cluster/src/cluster_impl.rs | 12 ++++++------ src/cluster/src/config.rs | 4 ++-- src/cluster/src/lib.rs | 12 ++++++------ src/common_types/src/cluster.rs | 4 ++-- src/horaedb/src/setup.rs | 4 ++-- src/router/src/cluster_based.rs | 6 +++--- src/server/src/grpc/mod.rs | 8 ++++---- 7 files changed, 25 insertions(+), 25 deletions(-) diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index c51c64dfe3..0df11335af 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -50,7 +50,7 @@ use crate::{ shard_lock_manager::{self, ShardLockManager, ShardLockManagerRef}, shard_set::{Shard, ShardRef, ShardSet}, topology::ClusterTopology, - Cluster, ClusterNodesNotFound, ClusterNodesResp, ClusterType, CompactionClientFailure, + Cluster, ClusterNodesNotFound, ClusterNodesResp, NodeType, CompactionClientFailure, CompactionOffloadNotAllowed, EtcdClientFailureWithCause, InitEtcdClientConfig, InvalidArguments, MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, TableStatus, @@ -387,7 +387,7 @@ impl Inner { #[async_trait] impl Cluster for ClusterImpl { /// Type of the server in cluster mode. - type ClusterType = ClusterType; + type NodeType = NodeType; async fn start(&self) -> Result<()> { info!("Cluster is starting with config:{:?}", self.config); @@ -420,8 +420,8 @@ impl Cluster for ClusterImpl { Ok(()) } - fn cluster_type(&self) -> ClusterType { - self.config.cluster_type.clone() + fn node_type(&self) -> NodeType { + self.config.node_type.clone() } async fn open_shard(&self, shard_info: &ShardInfo) -> Result { @@ -463,9 +463,9 @@ impl Cluster for ClusterImpl { req: &ExecuteCompactionTaskRequest, ) -> Result { ensure!( - self.cluster_type() == ClusterType::HoraeDB, + self.node_type() == NodeType::HoraeDB, CompactionOffloadNotAllowed { - cluster_type: self.cluster_type() + node_type: self.node_type() } ); self.inner.compact(req).await diff --git a/src/cluster/src/config.rs b/src/cluster/src/config.rs index b59eb0026d..d0b1c694b9 100644 --- a/src/cluster/src/config.rs +++ b/src/cluster/src/config.rs @@ -23,7 +23,7 @@ use serde::{Deserialize, Serialize}; use table_engine::ANALYTIC_ENGINE_TYPE; use time_ext::ReadableDuration; -use crate::ClusterType; +use crate::NodeType; #[derive(Debug, Clone, Deserialize, Serialize)] #[serde(default)] @@ -135,7 +135,7 @@ impl Default for TlsConfig { #[serde(default)] pub struct ClusterConfig { pub cmd_channel_buffer_size: usize, - pub cluster_type: ClusterType, + pub node_type: NodeType, pub meta_client: MetaClientConfig, pub etcd_client: EtcdClientConfig, } diff --git a/src/cluster/src/lib.rs b/src/cluster/src/lib.rs index 56e287454b..978b075696 100644 --- a/src/cluster/src/lib.rs +++ b/src/cluster/src/lib.rs @@ -28,7 +28,7 @@ use std::sync::Arc; use async_trait::async_trait; -use common_types::{cluster::ClusterType, schema::SchemaName}; +use common_types::{cluster::NodeType, schema::SchemaName}; use compaction_client::types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; use generic_error::GenericError; use macros::define_result; @@ -167,10 +167,10 @@ pub enum Error { ClusterNodesNotFound { version: u64, backtrace: Backtrace }, #[snafu(display( - "Not allowed to execute compaction offload in cluster_type:{cluster_type:?}.\nBacktrace:\n{backtrace:?}" + "Not allowed to execute compaction offload in node_type:{node_type:?}.\nBacktrace:\n{backtrace:?}" ))] CompactionOffloadNotAllowed { - cluster_type: ClusterType, + node_type: NodeType, backtrace: Backtrace, }, } @@ -194,7 +194,7 @@ impl From for TableStatus { } } -pub type ClusterRef = Arc + Send + Sync>; +pub type ClusterRef = Arc + Send + Sync>; #[derive(Clone, Debug)] pub struct ClusterNodesResp { @@ -207,13 +207,13 @@ pub struct ClusterNodesResp { /// + (Optional) Executes compaction task remotely. #[async_trait] pub trait Cluster { - type ClusterType: Send + Sync; + type NodeType: Send + Sync; async fn start(&self) -> Result<()>; async fn stop(&self) -> Result<()>; /// Get cluster type. - fn cluster_type(&self) -> ClusterType; + fn node_type(&self) -> NodeType; /// Fetch related information and open shard. async fn open_shard(&self, shard_info: &ShardInfo) -> Result; diff --git a/src/common_types/src/cluster.rs b/src/common_types/src/cluster.rs index 7f78c1f793..ad302023e9 100644 --- a/src/common_types/src/cluster.rs +++ b/src/common_types/src/cluster.rs @@ -17,9 +17,9 @@ use serde::{Deserialize, Serialize}; -/// Type to distinguish different server type in cluster mode. +/// Type to distinguish different node type in cluster mode. #[derive(Debug, Default, Clone, Serialize, Deserialize, PartialEq, Eq)] -pub enum ClusterType { +pub enum NodeType { #[default] HoraeDB, CompactionServer, diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index aaf7f80cdd..03b1d0f65b 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -26,7 +26,7 @@ use analytic_engine::{ use catalog::{manager::ManagerRef, schema::OpenOptions, table_operator::TableOperator}; use catalog_impls::{table_based::TableBasedManager, volatile, CatalogManagerImpl}; use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_set::ShardSet}; -use common_types::cluster::ClusterType; +use common_types::cluster::NodeType; use datafusion::execution::runtime_env::RuntimeConfig as DfRuntimeConfig; use df_operator::registry::{FunctionRegistry, FunctionRegistryImpl}; use interpreters::table_manipulator::{catalog_based, meta_based}; @@ -381,7 +381,7 @@ async fn build_with_meta( .opened_wals(opened_wals) .router(router) .schema_config_provider(schema_config_provider); - if let ClusterType::CompactionServer = cluster_config.cluster_type { + if let NodeType::CompactionServer = cluster_config.node_type { builder = builder.compaction_runner(local_compaction_runner.expect("Empty compaction runner.")); } diff --git a/src/router/src/cluster_based.rs b/src/router/src/cluster_based.rs index 01e67deb05..897369bc71 100644 --- a/src/router/src/cluster_based.rs +++ b/src/router/src/cluster_based.rs @@ -205,7 +205,7 @@ mod tests { shard_lock_manager::ShardLockManagerRef, shard_set::ShardRef, Cluster, ClusterNodesResp, TableStatus, }; - use common_types::{cluster::ClusterType, table::ShardId}; + use common_types::{cluster::NodeType, table::ShardId}; use compaction_client::types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; use horaedbproto::storage::{RequestContext, RouteRequest as RouteRequestPb}; use meta_client::types::{ @@ -219,7 +219,7 @@ mod tests { #[async_trait] impl Cluster for MockClusterImpl { - type ClusterType = ClusterType; + type NodeType = NodeType; async fn start(&self) -> cluster::Result<()> { unimplemented!(); @@ -229,7 +229,7 @@ mod tests { unimplemented!(); } - fn cluster_type(&self) -> ClusterType { + fn node_type(&self) -> NodeType { unimplemented!() } diff --git a/src/server/src/grpc/mod.rs b/src/server/src/grpc/mod.rs index 0a81916ccf..dcee780125 100644 --- a/src/server/src/grpc/mod.rs +++ b/src/server/src/grpc/mod.rs @@ -26,7 +26,7 @@ use std::{ use analytic_engine::compaction::runner::CompactionRunnerRef; use cluster::ClusterRef; -use common_types::{cluster::ClusterType, column_schema}; +use common_types::{cluster::NodeType, column_schema}; use compaction_service::CompactionServiceImpl; use futures::FutureExt; use generic_error::GenericError; @@ -332,8 +332,8 @@ impl Builder { self.cluster .map(|v| { let result: Result<()> = (|| { - match v.cluster_type() { - ClusterType::HoraeDB => { + match v.node_type() { + NodeType::HoraeDB => { // Support meta rpc service. let opened_wals = self.opened_wals.context(MissingWals)?; let builder = meta_event_service::Builder { @@ -344,7 +344,7 @@ impl Builder { }; meta_rpc_server = Some(MetaEventServiceServer::new(builder.build())); } - ClusterType::CompactionServer => { + NodeType::CompactionServer => { // Support remote rpc service. let compaction_runner = self.compaction_runner.context(MissingCompactionRunner)?; From 03af564cea372312b14b78db781cd62252a5ad39 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Wed, 21 Aug 2024 11:17:41 +0800 Subject: [PATCH 24/49] fix style. --- src/cluster/src/cluster_impl.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index 0df11335af..2700e7f3ae 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -50,10 +50,10 @@ use crate::{ shard_lock_manager::{self, ShardLockManager, ShardLockManagerRef}, shard_set::{Shard, ShardRef, ShardSet}, topology::ClusterTopology, - Cluster, ClusterNodesNotFound, ClusterNodesResp, NodeType, CompactionClientFailure, + Cluster, ClusterNodesNotFound, ClusterNodesResp, CompactionClientFailure, CompactionOffloadNotAllowed, EtcdClientFailureWithCause, InitEtcdClientConfig, - InvalidArguments, MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, - TableStatus, + InvalidArguments, MetaClientFailure, NodeType, OpenShard, OpenShardWithCause, Result, + ShardNotFound, TableStatus, }; /// ClusterImpl is an implementation of [`Cluster`] based [`MetaClient`]. From 16d1f3d0d50d86b657ff6c4e3da2001fa3040016 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Mon, 26 Aug 2024 11:47:12 +0800 Subject: [PATCH 25/49] impl default. --- src/compaction_client/src/compaction_impl.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/compaction_client/src/compaction_impl.rs b/src/compaction_client/src/compaction_impl.rs index 1e14b4df91..39424a9bd8 100644 --- a/src/compaction_client/src/compaction_impl.rs +++ b/src/compaction_client/src/compaction_impl.rs @@ -45,7 +45,11 @@ pub struct CompactionClientConfig { impl Default for CompactionClientConfig { fn default() -> Self { - unimplemented!() + Self { + cluster_name: String::new(), + compaction_server_addr: "127.0.0.1:7878".to_string(), + timeout: ReadableDuration::secs(5), + } } } From 32a53afd303a5a62edea4aff77dd52022594cd5b Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 27 Aug 2024 20:45:37 +0800 Subject: [PATCH 26/49] support meta client to fetch compaction node and send heartbeat with node type. --- Cargo.lock | 2 +- src/cluster/src/cluster_impl.rs | 18 +++++++--- src/compaction_client/src/compaction_impl.rs | 2 -- src/horaedb/src/setup.rs | 1 + src/meta_client/src/lib.rs | 14 ++++++-- src/meta_client/src/meta_impl.rs | 38 +++++++++++++++++--- src/meta_client/src/types.rs | 28 +++++++++++++++ 7 files changed, 88 insertions(+), 15 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 1a7200313e..67282f669d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3160,7 +3160,7 @@ dependencies = [ [[package]] name = "horaedbproto" version = "2.0.0" -source = "git+https://github.com/LeslieKid/incubator-horaedb-proto.git?rev=f5acee2e718d541ba04cd682b612a9fc7c93b25e#f5acee2e718d541ba04cd682b612a9fc7c93b25e" +source = "git+https://github.com/LeslieKid/incubator-horaedb-proto.git?rev=1e480ac702d07e18b876dca558d6d32ad731852b#1e480ac702d07e18b876dca558d6d32ad731852b" dependencies = [ "prost 0.11.8", "protoc-bin-vendored", diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index 2700e7f3ae..d6e4b2cba7 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -32,8 +32,8 @@ use generic_error::BoxError; use logger::{error, info, warn}; use meta_client::{ types::{ - GetNodesRequest, GetTablesOfShardsRequest, RouteTablesRequest, RouteTablesResponse, - ShardInfo, + FetchCompactionNodeRequest, GetNodesRequest, GetTablesOfShardsRequest, RouteTablesRequest, + RouteTablesResponse, ShardInfo, }, MetaClientRef, }; @@ -348,10 +348,20 @@ impl Inner { shards.iter().map(|shard| shard.shard_info()).collect() } - /// Get proper remote compaction node for compaction offload with meta + /// Get proper remote compaction node info for compaction offload with meta /// client. async fn get_compaction_node(&self) -> Result { - unimplemented!() + let mut config = CompactionClientConfig::default(); + + let req = FetchCompactionNodeRequest::default(); + let resp = self + .meta_client + .fetch_compaction_node(req) + .await + .context(MetaClientFailure)?; + + config.compaction_server_addr = resp.endpoint; + Ok(config) } /// Return a new compaction client. diff --git a/src/compaction_client/src/compaction_impl.rs b/src/compaction_client/src/compaction_impl.rs index 39424a9bd8..b154b5f646 100644 --- a/src/compaction_client/src/compaction_impl.rs +++ b/src/compaction_client/src/compaction_impl.rs @@ -38,7 +38,6 @@ type CompactionServiceGrpcClient = CompactionServiceClient Self { Self { - cluster_name: String::new(), compaction_server_addr: "127.0.0.1:7878".to_string(), timeout: ReadableDuration::secs(5), } diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index 03b1d0f65b..ae2ff59624 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -314,6 +314,7 @@ async fn build_with_meta( zone: config.node.zone.clone(), idc: config.node.idc.clone(), binary_version: config.node.binary_version.clone(), + node_type: cluster_config.node_type.clone(), }; info!("Build horaedb with node meta info:{node_meta_info:?}"); diff --git a/src/meta_client/src/lib.rs b/src/meta_client/src/lib.rs index a6cb8df6b9..ba93313537 100644 --- a/src/meta_client/src/lib.rs +++ b/src/meta_client/src/lib.rs @@ -23,9 +23,9 @@ use macros::define_result; use snafu::{Backtrace, Snafu}; use types::{ AllocSchemaIdRequest, AllocSchemaIdResponse, CreateTableRequest, CreateTableResponse, - DropTableRequest, DropTableResponse, GetNodesRequest, GetNodesResponse, - GetTablesOfShardsRequest, GetTablesOfShardsResponse, RouteTablesRequest, RouteTablesResponse, - ShardInfo, + DropTableRequest, DropTableResponse, FetchCompactionNodeRequest, FetchCompactionNodeResponse, + GetNodesRequest, GetNodesResponse, GetTablesOfShardsRequest, GetTablesOfShardsResponse, + RouteTablesRequest, RouteTablesResponse, ShardInfo, }; pub mod meta_impl; @@ -76,6 +76,9 @@ pub enum Error { #[snafu(display("Failed to get tables, err:{}", source))] FailGetTables { source: GenericError }, + #[snafu(display("Failed to fetch compaction node, err:{}", source))] + FailFetchCompactionNode { source: GenericError }, + #[snafu(display("Failed to route tables, err:{}", source))] FailRouteTables { source: GenericError }, @@ -113,6 +116,11 @@ pub trait MetaClient: Send + Sync { async fn get_nodes(&self, req: GetNodesRequest) -> Result; + async fn fetch_compaction_node( + &self, + req: FetchCompactionNodeRequest, + ) -> Result; + async fn send_heartbeat(&self, req: Vec) -> Result<()>; } diff --git a/src/meta_client/src/meta_impl.rs b/src/meta_client/src/meta_impl.rs index 5ba98de5fc..74e07d3817 100644 --- a/src/meta_client/src/meta_impl.rs +++ b/src/meta_client/src/meta_impl.rs @@ -31,12 +31,14 @@ use time_ext::ReadableDuration; use crate::{ types::{ AllocSchemaIdRequest, AllocSchemaIdResponse, CreateTableRequest, CreateTableResponse, - DropTableRequest, DropTableResponse, GetNodesRequest, GetNodesResponse, - GetTablesOfShardsRequest, GetTablesOfShardsResponse, NodeInfo, NodeMetaInfo, RequestHeader, - RouteTablesRequest, RouteTablesResponse, ShardInfo, + DropTableRequest, DropTableResponse, FetchCompactionNodeRequest, + FetchCompactionNodeResponse, GetNodesRequest, GetNodesResponse, GetTablesOfShardsRequest, + GetTablesOfShardsResponse, NodeInfo, NodeMetaInfo, RequestHeader, RouteTablesRequest, + RouteTablesResponse, ShardInfo, }, - BadResponse, FailAllocSchemaId, FailConnect, FailCreateTable, FailDropTable, FailGetTables, - FailRouteTables, FailSendHeartbeat, MetaClient, MetaClientRef, MissingHeader, Result, + BadResponse, FailAllocSchemaId, FailConnect, FailCreateTable, FailDropTable, + FailFetchCompactionNode, FailGetTables, FailRouteTables, FailSendHeartbeat, MetaClient, + MetaClientRef, MissingHeader, Result, }; type MetaServiceGrpcClient = MetaRpcServiceClient; @@ -236,6 +238,32 @@ impl MetaClient for MetaClientImpl { GetNodesResponse::try_from(pb_resp) } + async fn fetch_compaction_node( + &self, + req: FetchCompactionNodeRequest, + ) -> Result { + let mut pb_req = meta_service::FetchCompactionNodeRequest::from(req); + pb_req.header = Some(self.request_header().into()); + + debug!("Meta client try to fetch compaction node, req:{:?}", pb_req); + + let pb_resp = self + .client() + .fetch_compaction_node(pb_req) + .await + .box_err() + .context(FailFetchCompactionNode)? + .into_inner(); + + debug!( + "Meta client finish fetching compaction node, resp:{:?}", + pb_resp + ); + + check_response_header(&pb_resp.header)?; + Ok(FetchCompactionNodeResponse::from(pb_resp)) + } + async fn send_heartbeat(&self, shard_infos: Vec) -> Result<()> { let node_info = NodeInfo { node_meta_info: self.node_meta_info.clone(), diff --git a/src/meta_client/src/types.rs b/src/meta_client/src/types.rs index 6a6aba6918..8c7aff3bc7 100644 --- a/src/meta_client/src/types.rs +++ b/src/meta_client/src/types.rs @@ -19,6 +19,7 @@ use std::{collections::HashMap, fmt, sync::Arc}; pub use common_types::table::{ShardId, ShardVersion}; use common_types::{ + cluster::NodeType, schema::{SchemaId, SchemaName}, table::{TableId, TableName}, }; @@ -163,6 +164,7 @@ pub struct NodeMetaInfo { pub zone: String, pub idc: String, pub binary_version: String, + pub node_type: NodeType, } impl NodeMetaInfo { @@ -281,6 +283,11 @@ impl From for meta_service_pb::NodeInfo { binary_version: node_info.node_meta_info.binary_version, shard_infos, lease: 0, + node_type: if node_info.node_meta_info.node_type == NodeType::HoraeDB { + cluster_pb::NodeType::HoraeDb + } else { + cluster_pb::NodeType::CompactionServer + } as i32, } } } @@ -589,3 +596,24 @@ impl TryFrom for GetNodesResponse { }) } } + +#[derive(Debug, Clone, Default)] +pub struct FetchCompactionNodeRequest {} + +pub struct FetchCompactionNodeResponse { + pub endpoint: String, +} + +impl From for meta_service_pb::FetchCompactionNodeRequest { + fn from(_: FetchCompactionNodeRequest) -> Self { + meta_service_pb::FetchCompactionNodeRequest::default() + } +} + +impl From for FetchCompactionNodeResponse { + fn from(value: meta_service_pb::FetchCompactionNodeResponse) -> Self { + Self { + endpoint: value.endpoint, + } + } +} From 56fe903ab2167ab71b4593faa7579cf1b1f62264 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Fri, 30 Aug 2024 21:10:02 +0800 Subject: [PATCH 27/49] feat: analytic engine can support compaction offload --- Cargo.lock | 2 + src/analytic_engine/Cargo.toml | 2 + .../src/compaction/runner/mod.rs | 1 + .../src/compaction/runner/remote_runner.rs | 39 +++++++++++++++ src/analytic_engine/src/context.rs | 4 ++ src/analytic_engine/src/instance/engine.rs | 7 +++ src/analytic_engine/src/instance/open.rs | 48 ++++++++++++++----- src/analytic_engine/src/lib.rs | 4 ++ src/analytic_engine/src/setup.rs | 10 ++++ src/horaedb/src/setup.rs | 4 ++ 10 files changed, 109 insertions(+), 12 deletions(-) create mode 100644 src/analytic_engine/src/compaction/runner/remote_runner.rs diff --git a/Cargo.lock b/Cargo.lock index 67282f669d..e04c0bd77a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -88,8 +88,10 @@ dependencies = [ "atomic_enum", "base64 0.13.1", "bytes_ext", + "cluster", "codec", "common_types", + "compaction_client", "datafusion", "env_logger", "future_ext", diff --git a/src/analytic_engine/Cargo.toml b/src/analytic_engine/Cargo.toml index 8197b4eeb1..631e0af8f2 100644 --- a/src/analytic_engine/Cargo.toml +++ b/src/analytic_engine/Cargo.toml @@ -48,8 +48,10 @@ async-trait = { workspace = true } atomic_enum = { workspace = true } base64 = { workspace = true } bytes_ext = { workspace = true } +cluster = { workspace = true } codec = { workspace = true } common_types = { workspace = true } +compaction_client = { workspace = true } datafusion = { workspace = true } future_ext = { workspace = true } futures = { workspace = true } diff --git a/src/analytic_engine/src/compaction/runner/mod.rs b/src/analytic_engine/src/compaction/runner/mod.rs index 39ef356f36..5312ea67ac 100644 --- a/src/analytic_engine/src/compaction/runner/mod.rs +++ b/src/analytic_engine/src/compaction/runner/mod.rs @@ -16,6 +16,7 @@ // under the License. pub mod local_runner; +pub mod remote_runner; use std::sync::Arc; diff --git a/src/analytic_engine/src/compaction/runner/remote_runner.rs b/src/analytic_engine/src/compaction/runner/remote_runner.rs new file mode 100644 index 0000000000..a6f5c772e0 --- /dev/null +++ b/src/analytic_engine/src/compaction/runner/remote_runner.rs @@ -0,0 +1,39 @@ +// 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. + +use async_trait::async_trait; +use cluster::ClusterRef; + +use crate::{ + compaction::runner::{CompactionRunner, CompactionRunnerResult, CompactionRunnerTask}, + instance::flush_compaction::Result, +}; + +pub struct RemoteCompactionRunner { + pub cluster: ClusterRef, +} + +#[async_trait] +impl CompactionRunner for RemoteCompactionRunner { + async fn run(&self, _task: CompactionRunnerTask) -> Result { + // TODO(leslie): Impl the function. + // 1. Transfer `CompactionRunnerTask` into `ExecuteCompactionTaskRequest`. + // 2. Call `self.cluster.compact(req)`. + // 3. Transfer `ExecuteCompactionTaskResponse` into `CompactionRunnerResult`. + unimplemented!() + } +} diff --git a/src/analytic_engine/src/context.rs b/src/analytic_engine/src/context.rs index 1a3ba8a848..8efeb20e77 100644 --- a/src/analytic_engine/src/context.rs +++ b/src/analytic_engine/src/context.rs @@ -19,6 +19,7 @@ use std::{fmt, sync::Arc}; +use common_types::cluster::NodeType; use table_engine::engine::EngineRuntimes; use crate::{sst::meta_data::cache::MetaCacheRef, Config}; @@ -33,6 +34,9 @@ pub struct OpenContext { /// Sst meta data cache. pub meta_cache: Option, + + /// The type of the node. + pub node_type: NodeType, } impl fmt::Debug for OpenContext { diff --git a/src/analytic_engine/src/instance/engine.rs b/src/analytic_engine/src/instance/engine.rs index abea72a824..1d5ff68ca0 100644 --- a/src/analytic_engine/src/instance/engine.rs +++ b/src/analytic_engine/src/instance/engine.rs @@ -253,6 +253,12 @@ pub enum Error { sequence: SequenceNumber, source: wal::manager::Error, }, + + #[snafu(display( + "Failed to find cluster to construct remote compaction runner.\nBacktrace:\n{}", + backtrace + ))] + ClusterNotExist { backtrace: Backtrace }, } define_result!(Error); @@ -290,6 +296,7 @@ impl From for table_engine::engine::Error { | Error::OpenTablesOfShard { .. } | Error::ReplayWalNoCause { .. } | Error::PurgeWal { .. } + | Error::ClusterNotExist { .. } | Error::ReplayWalWithCause { .. } => Self::Unexpected { source: Box::new(err), }, diff --git a/src/analytic_engine/src/instance/open.rs b/src/analytic_engine/src/instance/open.rs index 525a97defb..302e2e3316 100644 --- a/src/analytic_engine/src/instance/open.rs +++ b/src/analytic_engine/src/instance/open.rs @@ -22,22 +22,26 @@ use std::{ sync::{Arc, RwLock}, }; -use common_types::table::ShardId; +use cluster::ClusterRef; +use common_types::{cluster::NodeType, table::ShardId}; use logger::{error, info}; use object_store::ObjectStoreRef; -use snafu::ResultExt; +use snafu::{OptionExt, ResultExt}; use table_engine::{engine::TableDef, table::TableId}; use wal::manager::WalManagerRef; use crate::{ compaction::{ - runner::{local_runner::LocalCompactionRunner, CompactionRunnerPtr, CompactionRunnerRef}, + runner::{ + local_runner::LocalCompactionRunner, remote_runner::RemoteCompactionRunner, + CompactionRunner, CompactionRunnerPtr, CompactionRunnerRef, + }, scheduler::SchedulerImpl, }, context::OpenContext, engine, instance::{ - engine::{OpenManifest, OpenTablesOfShard, ReadMetaUpdate, Result}, + engine::{ClusterNotExist, OpenManifest, OpenTablesOfShard, ReadMetaUpdate, Result}, flush_compaction::Flusher, mem_collector::MemUsageCollector, wal_replayer::{ReplayMode, WalReplayer}, @@ -68,14 +72,34 @@ impl InstanceContext { wal_manager: WalManagerRef, store_picker: ObjectStorePickerRef, sst_factory: SstFactoryRef, + cluster: Option, ) -> Result { - let compaction_runner = Box::new(LocalCompactionRunner::new( - ctx.runtimes.compact_runtime.clone(), - &ctx.config, - sst_factory.clone(), - store_picker.clone(), - ctx.meta_cache.clone(), - )); + let compaction_runner: CompactionRunnerPtr = match ctx.config.compaction_offload { + true => Box::new(RemoteCompactionRunner { + cluster: cluster.context(ClusterNotExist)?, + }), + false => Box::new(LocalCompactionRunner::new( + ctx.runtimes.compact_runtime.clone(), + &ctx.config, + sst_factory.clone(), + store_picker.clone(), + ctx.meta_cache.clone(), + )), + }; + + let local_compaction_runner: Option> = + if let NodeType::CompactionServer = ctx.node_type { + // The compaction runner for compaction node. + Some(Arc::new(LocalCompactionRunner::new( + ctx.runtimes.compact_runtime.clone(), + &ctx.config, + sst_factory.clone(), + store_picker.clone(), + ctx.meta_cache.clone(), + ))) + } else { + None + }; let instance = Instance::open( ctx, @@ -89,7 +113,7 @@ impl InstanceContext { Ok(Self { instance, - local_compaction_runner: None, + local_compaction_runner, }) } } diff --git a/src/analytic_engine/src/lib.rs b/src/analytic_engine/src/lib.rs index f2cc51ad15..ce36aa03db 100644 --- a/src/analytic_engine/src/lib.rs +++ b/src/analytic_engine/src/lib.rs @@ -71,6 +71,9 @@ pub struct Config { pub compaction: SchedulerConfig, + /// Offload the compaction task to remote nodes or not. + pub compaction_offload: bool, + /// sst meta cache capacity pub sst_meta_cache_cap: Option, /// sst data cache capacity @@ -184,6 +187,7 @@ impl Default for Config { max_replay_tables_per_batch: 64, table_opts: TableOptions::default(), compaction: SchedulerConfig::default(), + compaction_offload: false, sst_meta_cache_cap: Some(1000), sst_data_cache_cap: Some(1000), manifest: ManifestOptions::default(), diff --git a/src/analytic_engine/src/setup.rs b/src/analytic_engine/src/setup.rs index 1e25fb2ed5..06bbf6e35a 100644 --- a/src/analytic_engine/src/setup.rs +++ b/src/analytic_engine/src/setup.rs @@ -19,6 +19,8 @@ use std::{num::NonZeroUsize, path::Path, pin::Pin, sync::Arc}; +use cluster::ClusterRef; +use common_types::cluster::NodeType; use futures::Future; use macros::define_result; use object_store::{ @@ -97,6 +99,8 @@ pub struct EngineBuilder<'a> { pub config: &'a Config, pub engine_runtimes: Arc, pub opened_wals: OpenedWals, + pub cluster: Option, + pub node_type: NodeType, } impl<'a> EngineBuilder<'a> { @@ -117,6 +121,8 @@ impl<'a> EngineBuilder<'a> { self.opened_wals.data_wal, manifest_storages, Arc::new(opened_storages), + self.cluster, + self.node_type, ) .await?; @@ -135,6 +141,8 @@ async fn build_instance_context( wal_manager: WalManagerRef, manifest_storages: ManifestStorages, store_picker: ObjectStorePickerRef, + cluster: Option, + node_type: NodeType, ) -> Result { let meta_cache: Option = config .sst_meta_cache_cap @@ -144,6 +152,7 @@ async fn build_instance_context( config, runtimes: engine_runtimes, meta_cache, + node_type, }; let instance_ctx = InstanceContext::new( @@ -152,6 +161,7 @@ async fn build_instance_context( wal_manager, store_picker, Arc::new(FactoryImpl), + cluster, ) .await .context(OpenInstance)?; diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index ae2ff59624..63ca28c837 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -351,6 +351,8 @@ async fn build_with_meta( config: &config.analytic, engine_runtimes: runtimes.clone(), opened_wals: opened_wals.clone(), + cluster: Some(cluster.clone()), + node_type: cluster_config.node_type.clone(), }; let TableEngineContext { table_engine, @@ -405,6 +407,8 @@ async fn build_without_meta( config: &config.analytic, engine_runtimes: runtimes.clone(), opened_wals: opened_wals.clone(), + cluster: None, + node_type: NodeType::HoraeDB, }; let TableEngineContext { table_engine, .. } = engine_builder .build() From 8eff59d26420c71f4141b8120fb57e69e2384f13 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Sat, 31 Aug 2024 14:33:38 +0800 Subject: [PATCH 28/49] support conversation from compaction runner task to execute compaction task request. --- src/analytic_engine/src/compaction/mod.rs | 24 +++++++++++ .../src/compaction/runner/mod.rs | 36 ++++++++++++++++ src/analytic_engine/src/sst/factory.rs | 18 ++++++++ src/analytic_engine/src/sst/file.rs | 34 +++++++++++++++ src/analytic_engine/src/table_options.rs | 13 +++++- src/cluster/src/cluster_impl.rs | 9 ++-- src/cluster/src/lib.rs | 5 +-- src/compaction_client/src/compaction_impl.rs | 15 +++---- src/compaction_client/src/lib.rs | 6 +-- src/compaction_client/src/types.rs | 43 ------------------- src/router/src/cluster_based.rs | 6 +-- src/table_engine/src/table.rs | 6 +++ 12 files changed, 147 insertions(+), 68 deletions(-) delete mode 100644 src/compaction_client/src/types.rs diff --git a/src/analytic_engine/src/compaction/mod.rs b/src/analytic_engine/src/compaction/mod.rs index dce46d3e9c..8f63c93ece 100644 --- a/src/analytic_engine/src/compaction/mod.rs +++ b/src/analytic_engine/src/compaction/mod.rs @@ -379,6 +379,30 @@ impl TryFrom for Compact } } +impl From for horaedbproto::compaction_service::CompactionInputFiles { + fn from(value: CompactionInputFiles) -> Self { + let mut files = Vec::with_capacity(value.files.len()); + for file in value.files { + let handle = horaedbproto::compaction_service::FileHandle { + meta: Some(file.meta().into()), + purge_queue: Some(horaedbproto::compaction_service::FilePurgeQueue { + space_id: file.space_id(), + table_id: file.table_id().into(), + }), + being_compacted: file.being_compacted(), + metrics: Some(horaedbproto::compaction_service::SstMetrics {}), + }; + files.push(handle); + } + + Self { + level: value.level.as_u32(), + files, + output_level: value.output_level.as_u32(), + } + } +} + #[derive(Debug, Default, Clone)] pub struct ExpiredFiles { /// Level of the expired files. diff --git a/src/analytic_engine/src/compaction/runner/mod.rs b/src/analytic_engine/src/compaction/runner/mod.rs index 5312ea67ac..7e65ea3fb2 100644 --- a/src/analytic_engine/src/compaction/runner/mod.rs +++ b/src/analytic_engine/src/compaction/runner/mod.rs @@ -204,6 +204,22 @@ impl TryFrom } } +// TODO(leslie): Unused now, will be used in remote compaction runner impl. +impl From for horaedbproto::compaction_service::ExecuteCompactionTaskRequest { + fn from(task: CompactionRunnerTask) -> Self { + Self { + task_key: task.task_key, + request_id: task.request_id.into(), + schema: Some((&(task.schema)).into()), + space_id: task.space_id, + table_id: task.table_id.into(), + sequence: task.sequence, + input_ctx: Some(task.input_ctx.into()), + output_ctx: Some(task.output_ctx.into()), + } + } +} + pub struct CompactionRunnerResult { pub output_file_path: Path, pub sst_info: SstInfo, @@ -245,6 +261,17 @@ impl TryFrom for InputContext { } } +impl From for horaedbproto::compaction_service::InputContext { + fn from(value: InputContext) -> Self { + Self { + files: Some(value.files.into()), + num_rows_per_row_group: value.num_rows_per_row_group as u64, + merge_iter_options: value.merge_iter_options.batch_size as u64, + need_dedup: value.need_dedup, + } + } +} + #[derive(Debug, Clone)] pub struct OutputContext { /// Output sst file path @@ -271,3 +298,12 @@ impl TryFrom for OutputContext }) } } + +impl From for horaedbproto::compaction_service::OutputContext { + fn from(value: OutputContext) -> Self { + Self { + file_path: value.file_path.into(), + write_options: Some(value.write_options.into()), + } + } +} diff --git a/src/analytic_engine/src/sst/factory.rs b/src/analytic_engine/src/sst/factory.rs index 51703d771e..1f17b8df1d 100644 --- a/src/analytic_engine/src/sst/factory.rs +++ b/src/analytic_engine/src/sst/factory.rs @@ -209,6 +209,24 @@ impl TryFrom for SstWriteOpti } } +impl From for horaedbproto::compaction_service::SstWriteOptions { + fn from(value: SstWriteOptions) -> Self { + let column_stats = value + .column_stats + .into_iter() + .map(|(k, v)| (k, v.low_cardinality)) + .collect(); + + Self { + storage_format_hint: Some(value.storage_format_hint.into()), + num_rows_per_row_group: value.num_rows_per_row_group as u64, + compression: value.compression.into(), + max_buffer_size: value.max_buffer_size as u64, + column_stats, + } + } +} + impl From<&ColumnStats> for ColumnEncoding { fn from(value: &ColumnStats) -> Self { ColumnEncoding { diff --git a/src/analytic_engine/src/sst/file.rs b/src/analytic_engine/src/sst/file.rs index 33c4bcb96b..a6cc336a31 100644 --- a/src/analytic_engine/src/sst/file.rs +++ b/src/analytic_engine/src/sst/file.rs @@ -219,6 +219,16 @@ impl FileHandle { } } + #[inline] + pub fn space_id(&self) -> SpaceId { + self.inner.purge_queue.space_id() + } + + #[inline] + pub fn table_id(&self) -> TableId { + self.inner.purge_queue.table_id() + } + #[inline] pub fn read_meter(&self) -> Arc { self.inner.metrics.read_meter.clone() @@ -515,6 +525,20 @@ impl TryFrom for FileMeta { } } +impl From for horaedbproto::compaction_service::FileMeta { + fn from(value: FileMeta) -> Self { + Self { + file_id: value.id, + max_seq: value.max_seq, + time_range: Some(value.time_range.into()), + size: value.size, + row_num: value.row_num, + storage_format: value.storage_format.into(), + associated_files: value.associated_files, + } + } +} + // Queue to store files to be deleted for a table. #[derive(Clone)] pub struct FilePurgeQueue { @@ -563,6 +587,16 @@ impl FilePurgeQueue { ); } } + + #[inline] + pub fn space_id(&self) -> SpaceId { + self.inner.space_id + } + + #[inline] + pub fn table_id(&self) -> TableId { + self.inner.table_id + } } impl From for FilePurgeQueue { diff --git a/src/analytic_engine/src/table_options.rs b/src/analytic_engine/src/table_options.rs index dec42074b3..242d5abc1e 100644 --- a/src/analytic_engine/src/table_options.rs +++ b/src/analytic_engine/src/table_options.rs @@ -252,7 +252,7 @@ impl TryFrom for Compression { 0 => Compression::Uncompressed, 1 => Compression::Lz4, 2 => Compression::Snappy, - 4 => Compression::Zstd, + 3 => Compression::Zstd, _ => return UnknownCompressionType { value: compression }.fail(), }; @@ -260,6 +260,17 @@ impl TryFrom for Compression { } } +impl From for i32 { + fn from(value: Compression) -> Self { + match value { + Compression::Uncompressed => 0, + Compression::Lz4 => 1, + Compression::Snappy => 2, + Compression::Zstd => 3, + } + } +} + impl From for ParquetCompression { fn from(compression: Compression) -> Self { match compression { diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index d6e4b2cba7..978dd91c76 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -24,7 +24,6 @@ use async_trait::async_trait; use common_types::table::ShardId; use compaction_client::{ compaction_impl::{build_compaction_client, CompactionClientConfig}, - types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}, CompactionClientRef, }; use etcd_client::{Certificate, ConnectOptions, Identity, TlsOptions}; @@ -379,8 +378,8 @@ impl Inner { async fn compact( &self, - req: &ExecuteCompactionTaskRequest, - ) -> Result { + req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, + ) -> Result { // TODO(leslie): Execute the compaction task locally when fails to build // compaction client. let compact_resp = self @@ -470,8 +469,8 @@ impl Cluster for ClusterImpl { async fn compact( &self, - req: &ExecuteCompactionTaskRequest, - ) -> Result { + req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, + ) -> Result { ensure!( self.node_type() == NodeType::HoraeDB, CompactionOffloadNotAllowed { diff --git a/src/cluster/src/lib.rs b/src/cluster/src/lib.rs index 978b075696..c9aafaaf6e 100644 --- a/src/cluster/src/lib.rs +++ b/src/cluster/src/lib.rs @@ -29,7 +29,6 @@ use std::sync::Arc; use async_trait::async_trait; use common_types::{cluster::NodeType, schema::SchemaName}; -use compaction_client::types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; use generic_error::GenericError; use macros::define_result; use meta_client::types::{ @@ -241,6 +240,6 @@ pub trait Cluster { /// Execute compaction task in remote compaction node. async fn compact( &self, - req: &ExecuteCompactionTaskRequest, - ) -> Result; + req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, + ) -> Result; } diff --git a/src/compaction_client/src/compaction_impl.rs b/src/compaction_client/src/compaction_impl.rs index b154b5f646..88587355d5 100644 --- a/src/compaction_client/src/compaction_impl.rs +++ b/src/compaction_client/src/compaction_impl.rs @@ -20,8 +20,7 @@ use std::sync::Arc; use async_trait::async_trait; use generic_error::BoxError; use horaedbproto::{ - common::ResponseHeader, - compaction_service::{self, compaction_service_client::CompactionServiceClient}, + common::ResponseHeader, compaction_service::compaction_service_client::CompactionServiceClient, }; use logger::info; use serde::{Deserialize, Serialize}; @@ -29,8 +28,8 @@ use snafu::{OptionExt, ResultExt}; use time_ext::ReadableDuration; use crate::{ - BadResponse, CompactionClient, CompactionClientRef, ExecuteCompactionTaskRequest, - ExecuteCompactionTaskResponse, FailConnect, FailExecuteCompactionTask, MissingHeader, Result, + BadResponse, CompactionClient, CompactionClientRef, FailConnect, FailExecuteCompactionTask, + MissingHeader, Result, }; type CompactionServiceGrpcClient = CompactionServiceClient; @@ -88,10 +87,8 @@ impl CompactionClientImpl { impl CompactionClient for CompactionClientImpl { async fn execute_compaction_task( &self, - req: ExecuteCompactionTaskRequest, - ) -> Result { - let pb_req = compaction_service::ExecuteCompactionTaskRequest::from(req); - + pb_req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, + ) -> Result { // TODO(leslie): Add request header for ExecuteCompactionTaskRequest. info!( @@ -113,7 +110,7 @@ impl CompactionClient for CompactionClientImpl { ); check_response_header(&pb_resp.header)?; - ExecuteCompactionTaskResponse::try_from(pb_resp) + Ok(pb_resp) } } diff --git a/src/compaction_client/src/lib.rs b/src/compaction_client/src/lib.rs index c8ec47f246..0a74f66d17 100644 --- a/src/compaction_client/src/lib.rs +++ b/src/compaction_client/src/lib.rs @@ -21,10 +21,8 @@ use async_trait::async_trait; use generic_error::GenericError; use macros::define_result; use snafu::{Backtrace, Snafu}; -use types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; pub mod compaction_impl; -pub mod types; #[derive(Debug, Snafu)] #[snafu(visibility = "pub")] @@ -68,8 +66,8 @@ define_result!(Error); pub trait CompactionClient: Send + Sync { async fn execute_compaction_task( &self, - req: ExecuteCompactionTaskRequest, - ) -> Result; + req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, + ) -> Result; } pub type CompactionClientRef = Arc; diff --git a/src/compaction_client/src/types.rs b/src/compaction_client/src/types.rs deleted file mode 100644 index ca72149677..0000000000 --- a/src/compaction_client/src/types.rs +++ /dev/null @@ -1,43 +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. - -use horaedbproto::compaction_service; -use macros::define_result; - -use crate::Error; - -define_result!(Error); - -#[derive(Debug, Clone)] -pub struct ExecuteCompactionTaskRequest {} - -impl From for compaction_service::ExecuteCompactionTaskRequest { - fn from(_value: ExecuteCompactionTaskRequest) -> Self { - unimplemented!() - } -} - -#[derive(Debug, Clone)] -pub struct ExecuteCompactionTaskResponse {} - -impl TryFrom for ExecuteCompactionTaskResponse { - type Error = Error; - - fn try_from(_value: compaction_service::ExecuteCompactionTaskResponse) -> Result { - unimplemented!() - } -} diff --git a/src/router/src/cluster_based.rs b/src/router/src/cluster_based.rs index 897369bc71..e964178d01 100644 --- a/src/router/src/cluster_based.rs +++ b/src/router/src/cluster_based.rs @@ -206,7 +206,6 @@ mod tests { TableStatus, }; use common_types::{cluster::NodeType, table::ShardId}; - use compaction_client::types::{ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse}; use horaedbproto::storage::{RequestContext, RouteRequest as RouteRequestPb}; use meta_client::types::{ NodeShard, RouteEntry, RouteTablesResponse, ShardInfo, ShardRole::Leader, TableInfo, @@ -298,8 +297,9 @@ mod tests { async fn compact( &self, - _req: &ExecuteCompactionTaskRequest, - ) -> cluster::Result { + _req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, + ) -> cluster::Result + { unimplemented!() } } diff --git a/src/table_engine/src/table.rs b/src/table_engine/src/table.rs index 3c611b4395..6526579ab5 100644 --- a/src/table_engine/src/table.rs +++ b/src/table_engine/src/table.rs @@ -307,6 +307,12 @@ impl From for TableId { } } +impl From for u64 { + fn from(id: TableId) -> Self { + id.0 + } +} + impl fmt::Display for TableId { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { write!(f, "{}", self.0) From 142d8dc3740dbf0319ea9d86fdbda24c53b7241c Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Sat, 31 Aug 2024 18:07:37 +0800 Subject: [PATCH 29/49] update Cargo.toml (tmp) --- Cargo.lock | 2 +- Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index e04c0bd77a..302d9aecf7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3162,7 +3162,7 @@ dependencies = [ [[package]] name = "horaedbproto" version = "2.0.0" -source = "git+https://github.com/LeslieKid/incubator-horaedb-proto.git?rev=1e480ac702d07e18b876dca558d6d32ad731852b#1e480ac702d07e18b876dca558d6d32ad731852b" +source = "git+https://github.com/LeslieKid/incubator-horaedb-proto.git?rev=84f72744efc6e5f2e3a3ad8bbb24b807e383c492#84f72744efc6e5f2e3a3ad8bbb24b807e383c492" dependencies = [ "prost 0.11.8", "protoc-bin-vendored", diff --git a/Cargo.toml b/Cargo.toml index 3102ce6fbb..fbfb4c49cf 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -103,7 +103,7 @@ bytes_ext = { path = "src/components/bytes_ext" } catalog = { path = "src/catalog" } catalog_impls = { path = "src/catalog_impls" } # TODO(leslie): modify it when the related pr in incubator-horaedb-proto is merged. -horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "f5acee2e718d541ba04cd682b612a9fc7c93b25e" } +horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "84f72744efc6e5f2e3a3ad8bbb24b807e383c492" } codec = { path = "src/components/codec" } chrono = "0.4" clap = { version = "4.5.1", features = ["derive"] } From 4b72e3204dc8eb429da8505a66d79d2102568796 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Mon, 2 Sep 2024 20:29:19 +0800 Subject: [PATCH 30/49] fix bug. --- src/analytic_engine/src/tests/util.rs | 2 ++ src/benchmarks/src/util.rs | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/analytic_engine/src/tests/util.rs b/src/analytic_engine/src/tests/util.rs index 7eab3c1be2..d4ad6faa0b 100644 --- a/src/analytic_engine/src/tests/util.rs +++ b/src/analytic_engine/src/tests/util.rs @@ -141,6 +141,8 @@ impl TestContext { config: &self.config, engine_runtimes: self.runtimes.clone(), opened_wals: opened_wals.clone(), + cluster: None, + node_type: common_types::cluster::NodeType::HoraeDB, }; self.opened_wals = Some(opened_wals); diff --git a/src/benchmarks/src/util.rs b/src/benchmarks/src/util.rs index cb6d8de967..0316c1b6df 100644 --- a/src/benchmarks/src/util.rs +++ b/src/benchmarks/src/util.rs @@ -516,6 +516,8 @@ impl TestContext { config: &self.config, engine_runtimes: self.runtimes.clone(), opened_wals: opened_wals.clone(), + cluster: None, + node_type: common_types::cluster::NodeType::HoraeDB, }; self.opened_wals = Some(opened_wals); From b01a4b0f635b35442ee2729efead7ae54ee199d5 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Mon, 2 Sep 2024 20:30:17 +0800 Subject: [PATCH 31/49] update docs for cluster deployment. --- docs/example-cluster-3.toml | 62 ++++++++++++++++++++++++ docs/example-cluster-4.toml | 61 +++++++++++++++++++++++ src/analytic_engine/src/instance/open.rs | 5 ++ 3 files changed, 128 insertions(+) create mode 100644 docs/example-cluster-3.toml create mode 100644 docs/example-cluster-4.toml diff --git a/docs/example-cluster-3.toml b/docs/example-cluster-3.toml new file mode 100644 index 0000000000..ee7aa765f0 --- /dev/null +++ b/docs/example-cluster-3.toml @@ -0,0 +1,62 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[node] +addr = "127.0.0.1" + +[logger] +level = "info" + +[server] +bind_addr = "0.0.0.0" +http_port = 5442 +grpc_port = 8833 +mysql_port = 23307 +deploy_mode = "Cluster" + +[analytic] +compaction_offload = true + +[analytic.storage] +mem_cache_capacity = '1G' +mem_cache_partition_bits = 0 + +[analytic.storage.object_store] +type = "Local" +data_dir = "/tmp/horaedb0" + +[analytic.wal] +type = "RocksDB" +data_dir = "/tmp/horaedb0" + +[cluster_deployment] +mode = "WithMeta" +cmd_channel_buffer_size = 10 + +[cluster_deployment.meta_client] +# Only support "defaultCluster" currently. +cluster_name = "defaultCluster" +meta_addr = "http://127.0.0.1:2379" +lease = "10s" +timeout = "5s" + +[cluster_deployment.etcd_client] +server_addrs = ['127.0.0.1:2379'] + +[limiter] +write_block_list = ['mytable1'] +read_block_list = ['mytable1'] diff --git a/docs/example-cluster-4.toml b/docs/example-cluster-4.toml new file mode 100644 index 0000000000..ac9ae5bf26 --- /dev/null +++ b/docs/example-cluster-4.toml @@ -0,0 +1,61 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[node] +addr = "127.0.0.1" + +[logger] +level = "debug" + +[server] +bind_addr = "0.0.0.0" +http_port = 5443 +grpc_port = 8834 +mysql_port = 33307 +postgresql_port = 25433 +deploy_mode = "Cluster" + +[analytic.storage] +mem_cache_capacity = '1G' +mem_cache_partition_bits = 0 + +[analytic.storage.object_store] +type = "Local" +data_dir = "/tmp/horaedb1" + +[analytic.wal] +type = "RocksDB" +data_dir = "/tmp/horaedb1" + +[cluster_deployment] +mode = "WithMeta" +cmd_channel_buffer_size = 10 +node_type = "CompactionServer" + +[cluster_deployment.meta_client] +# Only support "defaultCluster" currently. +cluster_name = "defaultCluster" +meta_addr = "http://127.0.0.1:2379" +lease = "10s" +timeout = "5s" + +[cluster_deployment.etcd_client] +server_addrs = ['127.0.0.1:2379'] + +[limiter] +write_block_list = ['mytable1'] +read_block_list = ['mytable1'] diff --git a/src/analytic_engine/src/instance/open.rs b/src/analytic_engine/src/instance/open.rs index 302e2e3316..43eb46b771 100644 --- a/src/analytic_engine/src/instance/open.rs +++ b/src/analytic_engine/src/instance/open.rs @@ -74,6 +74,11 @@ impl InstanceContext { sst_factory: SstFactoryRef, cluster: Option, ) -> Result { + info!( + "Construct compaction runner with compaction_offload:{}", + ctx.config.compaction_offload + ); + let compaction_runner: CompactionRunnerPtr = match ctx.config.compaction_offload { true => Box::new(RemoteCompactionRunner { cluster: cluster.context(ClusterNotExist)?, From e78dc847a5ba1faa263b12c461f8eab0c435e402 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 3 Sep 2024 09:54:40 +0800 Subject: [PATCH 32/49] impl RemoteCompactionRunner. --- .../src/compaction/runner/mod.rs | 46 +++++++++++- .../src/compaction/runner/remote_runner.rs | 23 ++++-- .../src/instance/flush_compaction.rs | 6 ++ src/analytic_engine/src/sst/writer.rs | 71 ++++++++++++++++++- src/cluster/src/cluster_impl.rs | 2 +- 5 files changed, 138 insertions(+), 10 deletions(-) diff --git a/src/analytic_engine/src/compaction/runner/mod.rs b/src/analytic_engine/src/compaction/runner/mod.rs index 7e65ea3fb2..4c26e23b95 100644 --- a/src/analytic_engine/src/compaction/runner/mod.rs +++ b/src/analytic_engine/src/compaction/runner/mod.rs @@ -69,6 +69,15 @@ pub enum Error { #[snafu(display("Empty write options.\nBacktrace:\n{}", backtrace))] EmptySstWriteOptions { backtrace: Backtrace }, + #[snafu(display("Sst meta data is empty.\nBacktrace:\n{backtrace}"))] + EmptySstMeta { backtrace: Backtrace }, + + #[snafu(display("Empty sst info.\nBacktrace:\n{}", backtrace))] + EmptySstInfo { backtrace: Backtrace }, + + #[snafu(display("Empty compaction task exec result.\nBacktrace:\n{}", backtrace))] + EmptyExecResult { backtrace: Backtrace }, + #[snafu(display("Failed to convert table schema, err:{}", source))] ConvertTableSchema { source: GenericError }, @@ -83,6 +92,12 @@ pub enum Error { #[snafu(display("Failed to convert write options, err:{}", source))] ConvertSstWriteOptions { source: GenericError }, + + #[snafu(display("Failed to convert sst info, err:{}", source))] + ConvertSstInfo { source: GenericError }, + + #[snafu(display("Failed to convert sst meta, err:{}", source))] + ConvertSstMeta { source: GenericError }, } define_result!(Error); @@ -204,7 +219,6 @@ impl TryFrom } } -// TODO(leslie): Unused now, will be used in remote compaction runner impl. impl From for horaedbproto::compaction_service::ExecuteCompactionTaskRequest { fn from(task: CompactionRunnerTask) -> Self { Self { @@ -226,6 +240,36 @@ pub struct CompactionRunnerResult { pub sst_meta: MetaData, } +impl TryFrom + for CompactionRunnerResult +{ + type Error = Error; + + fn try_from( + resp: horaedbproto::compaction_service::ExecuteCompactionTaskResponse, + ) -> Result { + let res = resp.result.context(EmptyExecResult)?; + let sst_info = res + .sst_info + .context(EmptySstInfo)? + .try_into() + .box_err() + .context(ConvertSstInfo)?; + let sst_meta = res + .sst_meta + .context(EmptySstMeta)? + .try_into() + .box_err() + .context(ConvertSstMeta)?; + + Ok(Self { + output_file_path: res.output_file_path.into(), + sst_info, + sst_meta, + }) + } +} + #[derive(Debug, Clone)] pub struct InputContext { /// Input sst files in this compaction diff --git a/src/analytic_engine/src/compaction/runner/remote_runner.rs b/src/analytic_engine/src/compaction/runner/remote_runner.rs index a6f5c772e0..6cd86b2734 100644 --- a/src/analytic_engine/src/compaction/runner/remote_runner.rs +++ b/src/analytic_engine/src/compaction/runner/remote_runner.rs @@ -17,10 +17,12 @@ use async_trait::async_trait; use cluster::ClusterRef; +use generic_error::BoxError; +use snafu::ResultExt; use crate::{ compaction::runner::{CompactionRunner, CompactionRunnerResult, CompactionRunnerTask}, - instance::flush_compaction::Result, + instance::flush_compaction::{ConvertCompactionTaskResponse, RemoteCompact, Result}, }; pub struct RemoteCompactionRunner { @@ -29,11 +31,18 @@ pub struct RemoteCompactionRunner { #[async_trait] impl CompactionRunner for RemoteCompactionRunner { - async fn run(&self, _task: CompactionRunnerTask) -> Result { - // TODO(leslie): Impl the function. - // 1. Transfer `CompactionRunnerTask` into `ExecuteCompactionTaskRequest`. - // 2. Call `self.cluster.compact(req)`. - // 3. Transfer `ExecuteCompactionTaskResponse` into `CompactionRunnerResult`. - unimplemented!() + async fn run(&self, task: CompactionRunnerTask) -> Result { + let pb_resp = self + .cluster + .compact(task.into()) + .await + .context(RemoteCompact)?; + + let resp = pb_resp + .try_into() + .box_err() + .context(ConvertCompactionTaskResponse)?; + + Ok(resp) } } diff --git a/src/analytic_engine/src/instance/flush_compaction.rs b/src/analytic_engine/src/instance/flush_compaction.rs index da1647eb70..e8f7351a45 100644 --- a/src/analytic_engine/src/instance/flush_compaction.rs +++ b/src/analytic_engine/src/instance/flush_compaction.rs @@ -158,6 +158,12 @@ pub enum Error { #[snafu(display("Failed to alloc file id, err:{}", source))] AllocFileId { source: data::Error }, + + #[snafu(display("Failed to convert compaction task response, err:{}", source))] + ConvertCompactionTaskResponse { source: GenericError }, + + #[snafu(display("Failed to execute compaction task remotely, err:{}", source))] + RemoteCompact { source: cluster::Error }, } define_result!(Error); diff --git a/src/analytic_engine/src/sst/writer.rs b/src/analytic_engine/src/sst/writer.rs index 280d279f5b..577f499332 100644 --- a/src/analytic_engine/src/sst/writer.rs +++ b/src/analytic_engine/src/sst/writer.rs @@ -26,7 +26,8 @@ use common_types::{ SequenceNumber, }; use futures::Stream; -use generic_error::GenericError; +use generic_error::{BoxError, GenericError}; +use snafu::{OptionExt, ResultExt}; use crate::table_options::StorageFormat; @@ -96,6 +97,21 @@ pub mod error { #[snafu(display("Other kind of error, msg:{}.\nBacktrace:\n{}", msg, backtrace))] OtherNoCause { msg: String, backtrace: Backtrace }, + + #[snafu(display("Empty time range.\nBacktrace:\n{}", backtrace))] + EmptyTimeRange { backtrace: Backtrace }, + + #[snafu(display("Empty schema.\nBacktrace:\n{}", backtrace))] + EmptySchema { backtrace: Backtrace }, + + #[snafu(display("Failed to convert time range, err:{}", source))] + ConvertTimeRange { source: GenericError }, + + #[snafu(display("Failed to convert sst info, err:{}", source))] + ConvertSstInfo { source: GenericError }, + + #[snafu(display("Failed to convert schema, err:{}", source))] + ConvertSchema { source: GenericError }, } define_result!(Error); @@ -117,6 +133,32 @@ pub struct SstInfo { pub time_range: TimeRange, } +impl TryFrom for SstInfo { + type Error = Error; + + fn try_from(value: horaedbproto::compaction_service::SstInfo) -> Result { + let storage_format = value + .storage_format + .try_into() + .box_err() + .context(ConvertSstInfo)?; + let time_range = value + .time_range + .context(EmptyTimeRange)? + .try_into() + .box_err() + .context(ConvertTimeRange)?; + + Ok(Self { + file_size: value.file_size as usize, + row_num: value.row_num as usize, + storage_format, + meta_path: value.meta_path, + time_range, + }) + } +} + impl From for horaedbproto::compaction_service::SstInfo { fn from(value: SstInfo) -> Self { Self { @@ -143,6 +185,33 @@ pub struct MetaData { pub schema: Schema, } +impl TryFrom for MetaData { + type Error = Error; + + fn try_from(meta: horaedbproto::compaction_service::MetaData) -> Result { + let time_range = meta + .time_range + .context(EmptyTimeRange)? + .try_into() + .box_err() + .context(ConvertTimeRange)?; + let schema = meta + .schema + .context(EmptySchema)? + .try_into() + .box_err() + .context(ConvertSchema)?; + + Ok(Self { + min_key: Bytes::from(meta.min_key), + max_key: Bytes::from(meta.max_key), + time_range, + max_sequence: meta.max_sequence, + schema, + }) + } +} + impl From for horaedbproto::compaction_service::MetaData { fn from(meta: MetaData) -> Self { Self { diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index 978dd91c76..8647f62943 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -381,7 +381,7 @@ impl Inner { req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, ) -> Result { // TODO(leslie): Execute the compaction task locally when fails to build - // compaction client. + // compaction client or execute compaction task remotely. let compact_resp = self .compaction_client() .await From 1c986270e017cb6e6c16549faae435513f2aa69e Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 10 Sep 2024 14:21:28 +0800 Subject: [PATCH 33/49] refactor: place remote compact logic into RemoteCompactionRunner instead of Cluster. --- Cargo.lock | 1 + src/analytic_engine/Cargo.toml | 1 + .../src/compaction/runner/mod.rs | 1 + .../src/compaction/runner/node_picker.rs | 64 ++++++++++++++++ .../src/compaction/runner/remote_runner.rs | 42 +++++++++-- src/analytic_engine/src/instance/engine.rs | 6 +- .../src/instance/flush_compaction.rs | 12 ++- src/analytic_engine/src/instance/open.rs | 15 ++-- src/analytic_engine/src/setup.rs | 11 +-- src/analytic_engine/src/tests/util.rs | 2 +- src/benchmarks/src/util.rs | 2 +- src/cluster/src/cluster_impl.rs | 73 ++----------------- src/cluster/src/lib.rs | 20 ----- src/horaedb/src/setup.rs | 4 +- src/router/src/cluster_based.rs | 8 -- 15 files changed, 140 insertions(+), 122 deletions(-) create mode 100644 src/analytic_engine/src/compaction/runner/node_picker.rs diff --git a/Cargo.lock b/Cargo.lock index 302d9aecf7..04a23b7c16 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -108,6 +108,7 @@ dependencies = [ "lru 0.7.8", "macros", "message_queue", + "meta_client", "metric_ext", "object_store 2.0.0", "parquet", diff --git a/src/analytic_engine/Cargo.toml b/src/analytic_engine/Cargo.toml index 631e0af8f2..72ee3114d6 100644 --- a/src/analytic_engine/Cargo.toml +++ b/src/analytic_engine/Cargo.toml @@ -67,6 +67,7 @@ logger = { workspace = true } lru = { workspace = true } macros = { workspace = true } message_queue = { workspace = true } +meta_client = { workspace = true } metric_ext = { workspace = true } object_store = { workspace = true } parquet = { workspace = true } diff --git a/src/analytic_engine/src/compaction/runner/mod.rs b/src/analytic_engine/src/compaction/runner/mod.rs index 4c26e23b95..986e7067fa 100644 --- a/src/analytic_engine/src/compaction/runner/mod.rs +++ b/src/analytic_engine/src/compaction/runner/mod.rs @@ -16,6 +16,7 @@ // under the License. pub mod local_runner; +pub mod node_picker; pub mod remote_runner; use std::sync::Arc; diff --git a/src/analytic_engine/src/compaction/runner/node_picker.rs b/src/analytic_engine/src/compaction/runner/node_picker.rs new file mode 100644 index 0000000000..122b67a63f --- /dev/null +++ b/src/analytic_engine/src/compaction/runner/node_picker.rs @@ -0,0 +1,64 @@ +// 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. + +//! Remote compaction node picker. + +use std::sync::Arc; + +use async_trait::async_trait; +use macros::define_result; +use meta_client::{types::FetchCompactionNodeRequest, MetaClientRef}; +use snafu::{Snafu, ResultExt}; + +#[async_trait] +pub trait RemoteCompactionNodePicker: Send + Sync { + /// Get the addr of the remote compaction node. + async fn get_compaction_node(&self) -> Result; +} + +pub type RemoteCompactionNodePickerRef = Arc; + +#[derive(Debug, Snafu)] +pub enum Error { + #[snafu(display("Meta client fetch compaciton node failed, err:{source}."))] + FetchCompactionNodeFailure { source: meta_client::Error }, +} + +define_result!(Error); + +/// RemoteCompactionNodePickerImpl is an implementation of +/// [`RemoteCompactionNodePicker`] based [`MetaClient`]. +pub struct RemoteCompactionNodePickerImpl { + pub meta_client: MetaClientRef, +} + +#[async_trait] +impl RemoteCompactionNodePicker for RemoteCompactionNodePickerImpl { + /// Get proper remote compaction node info for compaction offload with meta + /// client. + async fn get_compaction_node(&self) -> Result { + let req = FetchCompactionNodeRequest::default(); + let resp = self + .meta_client + .fetch_compaction_node(req) + .await + .context(FetchCompactionNodeFailure)?; + + let compaction_node_addr = resp.endpoint; + Ok(compaction_node_addr) + } +} diff --git a/src/analytic_engine/src/compaction/runner/remote_runner.rs b/src/analytic_engine/src/compaction/runner/remote_runner.rs index 6cd86b2734..1b34bc8887 100644 --- a/src/analytic_engine/src/compaction/runner/remote_runner.rs +++ b/src/analytic_engine/src/compaction/runner/remote_runner.rs @@ -16,27 +16,55 @@ // under the License. use async_trait::async_trait; -use cluster::ClusterRef; +use compaction_client::{ + compaction_impl::{build_compaction_client, CompactionClientConfig}, + CompactionClientRef, +}; use generic_error::BoxError; use snafu::ResultExt; +use super::node_picker::RemoteCompactionNodePickerRef; use crate::{ compaction::runner::{CompactionRunner, CompactionRunnerResult, CompactionRunnerTask}, - instance::flush_compaction::{ConvertCompactionTaskResponse, RemoteCompact, Result}, + instance::flush_compaction::{ + BuildCompactionClientFailed, ConvertCompactionTaskResponse, GetCompactionClientFailed, + PickCompactionNodeFailed, RemoteCompactFailed, Result, + }, }; pub struct RemoteCompactionRunner { - pub cluster: ClusterRef, + pub node_picker: RemoteCompactionNodePickerRef, +} + +impl RemoteCompactionRunner { + async fn get_compaction_client(&self) -> Result { + let mut config = CompactionClientConfig::default(); + let node_addr = self + .node_picker + .get_compaction_node() + .await + .context(PickCompactionNodeFailed)?; + config.compaction_server_addr = node_addr; + + let client = build_compaction_client(config) + .await + .context(BuildCompactionClientFailed)?; + Ok(client) + } } #[async_trait] impl CompactionRunner for RemoteCompactionRunner { async fn run(&self, task: CompactionRunnerTask) -> Result { - let pb_resp = self - .cluster - .compact(task.into()) + let client = self + .get_compaction_client() + .await + .box_err() + .context(GetCompactionClientFailed)?; + let pb_resp = client + .execute_compaction_task(task.into()) .await - .context(RemoteCompact)?; + .context(RemoteCompactFailed)?; let resp = pb_resp .try_into() diff --git a/src/analytic_engine/src/instance/engine.rs b/src/analytic_engine/src/instance/engine.rs index 1d5ff68ca0..4054303cee 100644 --- a/src/analytic_engine/src/instance/engine.rs +++ b/src/analytic_engine/src/instance/engine.rs @@ -255,10 +255,10 @@ pub enum Error { }, #[snafu(display( - "Failed to find cluster to construct remote compaction runner.\nBacktrace:\n{}", + "Failed to find meta client to construct remote compaction runner.\nBacktrace:\n{}", backtrace ))] - ClusterNotExist { backtrace: Backtrace }, + MetaClientNotExist { backtrace: Backtrace }, } define_result!(Error); @@ -296,7 +296,7 @@ impl From for table_engine::engine::Error { | Error::OpenTablesOfShard { .. } | Error::ReplayWalNoCause { .. } | Error::PurgeWal { .. } - | Error::ClusterNotExist { .. } + | Error::MetaClientNotExist { .. } | Error::ReplayWalWithCause { .. } => Self::Unexpected { source: Box::new(err), }, diff --git a/src/analytic_engine/src/instance/flush_compaction.rs b/src/analytic_engine/src/instance/flush_compaction.rs index e8f7351a45..2ac5539326 100644 --- a/src/analytic_engine/src/instance/flush_compaction.rs +++ b/src/analytic_engine/src/instance/flush_compaction.rs @@ -41,6 +41,7 @@ use tokio::{sync::oneshot, time::Instant}; use wal::manager::WalLocation; use crate::{ + compaction::runner::node_picker, instance::{ self, reorder_memtable::Reorder, serial_executor::TableFlushScheduler, SpaceStoreRef, }, @@ -162,8 +163,17 @@ pub enum Error { #[snafu(display("Failed to convert compaction task response, err:{}", source))] ConvertCompactionTaskResponse { source: GenericError }, + #[snafu(display("Failed to pick remote compaction node, err:{}", source))] + PickCompactionNodeFailed { source: node_picker::Error }, + + #[snafu(display("Failed to build compaction client, err:{}", source))] + BuildCompactionClientFailed { source: compaction_client::Error }, + + #[snafu(display("Failed to get compaction client, err:{}", source))] + GetCompactionClientFailed { source: GenericError }, + #[snafu(display("Failed to execute compaction task remotely, err:{}", source))] - RemoteCompact { source: cluster::Error }, + RemoteCompactFailed { source: compaction_client::Error }, } define_result!(Error); diff --git a/src/analytic_engine/src/instance/open.rs b/src/analytic_engine/src/instance/open.rs index 43eb46b771..99ca748537 100644 --- a/src/analytic_engine/src/instance/open.rs +++ b/src/analytic_engine/src/instance/open.rs @@ -22,9 +22,9 @@ use std::{ sync::{Arc, RwLock}, }; -use cluster::ClusterRef; use common_types::{cluster::NodeType, table::ShardId}; use logger::{error, info}; +use meta_client::MetaClientRef; use object_store::ObjectStoreRef; use snafu::{OptionExt, ResultExt}; use table_engine::{engine::TableDef, table::TableId}; @@ -33,15 +33,16 @@ use wal::manager::WalManagerRef; use crate::{ compaction::{ runner::{ - local_runner::LocalCompactionRunner, remote_runner::RemoteCompactionRunner, - CompactionRunner, CompactionRunnerPtr, CompactionRunnerRef, + local_runner::LocalCompactionRunner, node_picker::RemoteCompactionNodePickerImpl, + remote_runner::RemoteCompactionRunner, CompactionRunner, CompactionRunnerPtr, + CompactionRunnerRef, }, scheduler::SchedulerImpl, }, context::OpenContext, engine, instance::{ - engine::{ClusterNotExist, OpenManifest, OpenTablesOfShard, ReadMetaUpdate, Result}, + engine::{MetaClientNotExist, OpenManifest, OpenTablesOfShard, ReadMetaUpdate, Result}, flush_compaction::Flusher, mem_collector::MemUsageCollector, wal_replayer::{ReplayMode, WalReplayer}, @@ -72,7 +73,7 @@ impl InstanceContext { wal_manager: WalManagerRef, store_picker: ObjectStorePickerRef, sst_factory: SstFactoryRef, - cluster: Option, + meta_client: Option, ) -> Result { info!( "Construct compaction runner with compaction_offload:{}", @@ -81,7 +82,9 @@ impl InstanceContext { let compaction_runner: CompactionRunnerPtr = match ctx.config.compaction_offload { true => Box::new(RemoteCompactionRunner { - cluster: cluster.context(ClusterNotExist)?, + node_picker: Arc::new(RemoteCompactionNodePickerImpl { + meta_client: meta_client.context(MetaClientNotExist)?, + }), }), false => Box::new(LocalCompactionRunner::new( ctx.runtimes.compact_runtime.clone(), diff --git a/src/analytic_engine/src/setup.rs b/src/analytic_engine/src/setup.rs index 06bbf6e35a..e6bab51727 100644 --- a/src/analytic_engine/src/setup.rs +++ b/src/analytic_engine/src/setup.rs @@ -19,10 +19,10 @@ use std::{num::NonZeroUsize, path::Path, pin::Pin, sync::Arc}; -use cluster::ClusterRef; use common_types::cluster::NodeType; use futures::Future; use macros::define_result; +use meta_client::MetaClientRef; use object_store::{ aliyun, config::{ObjectStoreOptions, StorageOptions}, @@ -99,7 +99,8 @@ pub struct EngineBuilder<'a> { pub config: &'a Config, pub engine_runtimes: Arc, pub opened_wals: OpenedWals, - pub cluster: Option, + // Meta client is needed when compaction offload. + pub meta_client: Option, pub node_type: NodeType, } @@ -121,7 +122,7 @@ impl<'a> EngineBuilder<'a> { self.opened_wals.data_wal, manifest_storages, Arc::new(opened_storages), - self.cluster, + self.meta_client, self.node_type, ) .await?; @@ -141,7 +142,7 @@ async fn build_instance_context( wal_manager: WalManagerRef, manifest_storages: ManifestStorages, store_picker: ObjectStorePickerRef, - cluster: Option, + meta_client: Option, node_type: NodeType, ) -> Result { let meta_cache: Option = config @@ -161,7 +162,7 @@ async fn build_instance_context( wal_manager, store_picker, Arc::new(FactoryImpl), - cluster, + meta_client.clone(), ) .await .context(OpenInstance)?; diff --git a/src/analytic_engine/src/tests/util.rs b/src/analytic_engine/src/tests/util.rs index d4ad6faa0b..34aec3c987 100644 --- a/src/analytic_engine/src/tests/util.rs +++ b/src/analytic_engine/src/tests/util.rs @@ -141,7 +141,7 @@ impl TestContext { config: &self.config, engine_runtimes: self.runtimes.clone(), opened_wals: opened_wals.clone(), - cluster: None, + meta_client: None, node_type: common_types::cluster::NodeType::HoraeDB, }; self.opened_wals = Some(opened_wals); diff --git a/src/benchmarks/src/util.rs b/src/benchmarks/src/util.rs index 0316c1b6df..eaa0b76692 100644 --- a/src/benchmarks/src/util.rs +++ b/src/benchmarks/src/util.rs @@ -516,7 +516,7 @@ impl TestContext { config: &self.config, engine_runtimes: self.runtimes.clone(), opened_wals: opened_wals.clone(), - cluster: None, + meta_client: None, node_type: common_types::cluster::NodeType::HoraeDB, }; self.opened_wals = Some(opened_wals); diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index 8647f62943..74b6a0654d 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -22,17 +22,13 @@ use std::{ use async_trait::async_trait; use common_types::table::ShardId; -use compaction_client::{ - compaction_impl::{build_compaction_client, CompactionClientConfig}, - CompactionClientRef, -}; use etcd_client::{Certificate, ConnectOptions, Identity, TlsOptions}; use generic_error::BoxError; use logger::{error, info, warn}; use meta_client::{ types::{ - FetchCompactionNodeRequest, GetNodesRequest, GetTablesOfShardsRequest, RouteTablesRequest, - RouteTablesResponse, ShardInfo, + GetNodesRequest, GetTablesOfShardsRequest, RouteTablesRequest, RouteTablesResponse, + ShardInfo, }, MetaClientRef, }; @@ -49,10 +45,9 @@ use crate::{ shard_lock_manager::{self, ShardLockManager, ShardLockManagerRef}, shard_set::{Shard, ShardRef, ShardSet}, topology::ClusterTopology, - Cluster, ClusterNodesNotFound, ClusterNodesResp, CompactionClientFailure, - CompactionOffloadNotAllowed, EtcdClientFailureWithCause, InitEtcdClientConfig, - InvalidArguments, MetaClientFailure, NodeType, OpenShard, OpenShardWithCause, Result, - ShardNotFound, TableStatus, + Cluster, ClusterNodesNotFound, ClusterNodesResp, EtcdClientFailureWithCause, + InitEtcdClientConfig, InvalidArguments, MetaClientFailure, NodeType, OpenShard, + OpenShardWithCause, Result, ShardNotFound, TableStatus, }; /// ClusterImpl is an implementation of [`Cluster`] based [`MetaClient`]. @@ -346,51 +341,6 @@ impl Inner { shards.iter().map(|shard| shard.shard_info()).collect() } - - /// Get proper remote compaction node info for compaction offload with meta - /// client. - async fn get_compaction_node(&self) -> Result { - let mut config = CompactionClientConfig::default(); - - let req = FetchCompactionNodeRequest::default(); - let resp = self - .meta_client - .fetch_compaction_node(req) - .await - .context(MetaClientFailure)?; - - config.compaction_server_addr = resp.endpoint; - Ok(config) - } - - /// Return a new compaction client. - async fn compaction_client(&self) -> CompactionClientRef { - // TODO(leslie): impl better error handling with snafu. - let config = self - .get_compaction_node() - .await - .expect("fail to get remote compaction node"); - - build_compaction_client(config) - .await - .expect("fail to build compaction client") - } - - async fn compact( - &self, - req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, - ) -> Result { - // TODO(leslie): Execute the compaction task locally when fails to build - // compaction client or execute compaction task remotely. - let compact_resp = self - .compaction_client() - .await - .execute_compaction_task(req.clone()) - .await - .context(CompactionClientFailure)?; - - Ok(compact_resp) - } } #[async_trait] @@ -466,19 +416,6 @@ impl Cluster for ClusterImpl { fn shard_lock_manager(&self) -> ShardLockManagerRef { self.shard_lock_manager.clone() } - - async fn compact( - &self, - req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, - ) -> Result { - ensure!( - self.node_type() == NodeType::HoraeDB, - CompactionOffloadNotAllowed { - node_type: self.node_type() - } - ); - self.inner.compact(req).await - } } /// Build the connect options for accessing etcd cluster. diff --git a/src/cluster/src/lib.rs b/src/cluster/src/lib.rs index c9aafaaf6e..778b3ab5d7 100644 --- a/src/cluster/src/lib.rs +++ b/src/cluster/src/lib.rs @@ -67,9 +67,6 @@ pub enum Error { #[snafu(display("Meta client execute failed, err:{source}."))] MetaClientFailure { source: meta_client::Error }, - #[snafu(display("Compaction client execute failed, err:{source}."))] - CompactionClientFailure { source: compaction_client::Error }, - #[snafu(display("Failed to init etcd client config, err:{source}.\nBacktrace:\n{backtrace}"))] InitEtcdClientConfig { source: std::io::Error, @@ -164,14 +161,6 @@ pub enum Error { "Cluster nodes are not found in the topology, version:{version}.\nBacktrace:\n{backtrace}", ))] ClusterNodesNotFound { version: u64, backtrace: Backtrace }, - - #[snafu(display( - "Not allowed to execute compaction offload in node_type:{node_type:?}.\nBacktrace:\n{backtrace:?}" - ))] - CompactionOffloadNotAllowed { - node_type: NodeType, - backtrace: Backtrace, - }, } define_result!(Error); @@ -201,9 +190,6 @@ pub struct ClusterNodesResp { pub cluster_nodes: ClusterNodesRef, } -/// Cluster has the following functions: -/// + Manages tables and shard infos in cluster mode. -/// + (Optional) Executes compaction task remotely. #[async_trait] pub trait Cluster { type NodeType: Send + Sync; @@ -236,10 +222,4 @@ pub trait Cluster { async fn route_tables(&self, req: &RouteTablesRequest) -> Result; async fn fetch_nodes(&self) -> Result; fn shard_lock_manager(&self) -> ShardLockManagerRef; - - /// Execute compaction task in remote compaction node. - async fn compact( - &self, - req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, - ) -> Result; } diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index 63ca28c837..273b1df3ac 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -351,7 +351,7 @@ async fn build_with_meta( config: &config.analytic, engine_runtimes: runtimes.clone(), opened_wals: opened_wals.clone(), - cluster: Some(cluster.clone()), + meta_client: Some(meta_client.clone()), node_type: cluster_config.node_type.clone(), }; let TableEngineContext { @@ -407,7 +407,7 @@ async fn build_without_meta( config: &config.analytic, engine_runtimes: runtimes.clone(), opened_wals: opened_wals.clone(), - cluster: None, + meta_client: None, node_type: NodeType::HoraeDB, }; let TableEngineContext { table_engine, .. } = engine_builder diff --git a/src/router/src/cluster_based.rs b/src/router/src/cluster_based.rs index e964178d01..3f9337ca72 100644 --- a/src/router/src/cluster_based.rs +++ b/src/router/src/cluster_based.rs @@ -294,14 +294,6 @@ mod tests { fn shard_lock_manager(&self) -> ShardLockManagerRef { unimplemented!(); } - - async fn compact( - &self, - _req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, - ) -> cluster::Result - { - unimplemented!() - } } #[tokio::test] From 282d886fb0c4dfb5af2294c08506acd7a7be5cec Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Fri, 20 Sep 2024 14:22:39 +0800 Subject: [PATCH 34/49] fix bug. --- Cargo.lock | 2 +- src/analytic_engine/src/compaction/runner/node_picker.rs | 2 +- src/analytic_engine/src/instance/engine.rs | 5 ++--- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a152d84a6e..a317cbdcb7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1599,7 +1599,7 @@ dependencies = [ "logger", "macros", "prost 0.11.8", - "reqwest", + "reqwest 0.12.4", "serde", "serde_json", "snafu 0.6.10", diff --git a/src/analytic_engine/src/compaction/runner/node_picker.rs b/src/analytic_engine/src/compaction/runner/node_picker.rs index 122b67a63f..586e640e73 100644 --- a/src/analytic_engine/src/compaction/runner/node_picker.rs +++ b/src/analytic_engine/src/compaction/runner/node_picker.rs @@ -22,7 +22,7 @@ use std::sync::Arc; use async_trait::async_trait; use macros::define_result; use meta_client::{types::FetchCompactionNodeRequest, MetaClientRef}; -use snafu::{Snafu, ResultExt}; +use snafu::{ResultExt, Snafu}; #[async_trait] pub trait RemoteCompactionNodePicker: Send + Sync { diff --git a/src/analytic_engine/src/instance/engine.rs b/src/analytic_engine/src/instance/engine.rs index 371fcaca15..537b83314f 100644 --- a/src/analytic_engine/src/instance/engine.rs +++ b/src/analytic_engine/src/instance/engine.rs @@ -299,15 +299,14 @@ impl From for table_engine::engine::Error { | Error::DoManifestSnapshot { .. } | Error::OpenManifest { .. } | Error::TableNotExist { .. } + | Error::MetaClientNotExist { .. } | Error::OpenTablesOfShard { .. } | Error::ReplayWalNoCause { .. } | Error::PurgeWal { .. } - | Error::ReplayWalWithCause { .. } => Self::Unexpected { | Error::ReplayWalWithCause { .. } | Error::InvalidTableOptions { .. } => Self::Unexpected { source: Box::new(err), - } - | Error::MetaClientNotExist { .. }, + }, } } } From efe99bd556371b2de632dbbb1e93ebbe6ab73c87 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Wed, 18 Sep 2024 15:10:36 +0800 Subject: [PATCH 35/49] remove unnecessary NodeType in Cluster. --- src/cluster/src/cluster_impl.rs | 3 --- src/cluster/src/lib.rs | 4 +--- src/router/src/cluster_based.rs | 2 -- 3 files changed, 1 insertion(+), 8 deletions(-) diff --git a/src/cluster/src/cluster_impl.rs b/src/cluster/src/cluster_impl.rs index 74b6a0654d..d79eda0485 100644 --- a/src/cluster/src/cluster_impl.rs +++ b/src/cluster/src/cluster_impl.rs @@ -345,9 +345,6 @@ impl Inner { #[async_trait] impl Cluster for ClusterImpl { - /// Type of the server in cluster mode. - type NodeType = NodeType; - async fn start(&self) -> Result<()> { info!("Cluster is starting with config:{:?}", self.config); diff --git a/src/cluster/src/lib.rs b/src/cluster/src/lib.rs index 778b3ab5d7..ddda6c4689 100644 --- a/src/cluster/src/lib.rs +++ b/src/cluster/src/lib.rs @@ -182,7 +182,7 @@ impl From for TableStatus { } } -pub type ClusterRef = Arc + Send + Sync>; +pub type ClusterRef = Arc; #[derive(Clone, Debug)] pub struct ClusterNodesResp { @@ -192,8 +192,6 @@ pub struct ClusterNodesResp { #[async_trait] pub trait Cluster { - type NodeType: Send + Sync; - async fn start(&self) -> Result<()>; async fn stop(&self) -> Result<()>; diff --git a/src/router/src/cluster_based.rs b/src/router/src/cluster_based.rs index 3f9337ca72..bd5efd8b5b 100644 --- a/src/router/src/cluster_based.rs +++ b/src/router/src/cluster_based.rs @@ -218,8 +218,6 @@ mod tests { #[async_trait] impl Cluster for MockClusterImpl { - type NodeType = NodeType; - async fn start(&self) -> cluster::Result<()> { unimplemented!(); } From b7b810d3459b7f90ce97e884e934750bb393b0bf Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Sun, 22 Sep 2024 15:53:27 +0800 Subject: [PATCH 36/49] impl local compaction node picker. --- .../src/compaction/runner/node_picker.rs | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/src/analytic_engine/src/compaction/runner/node_picker.rs b/src/analytic_engine/src/compaction/runner/node_picker.rs index 586e640e73..37fe7fadd4 100644 --- a/src/analytic_engine/src/compaction/runner/node_picker.rs +++ b/src/analytic_engine/src/compaction/runner/node_picker.rs @@ -25,12 +25,12 @@ use meta_client::{types::FetchCompactionNodeRequest, MetaClientRef}; use snafu::{ResultExt, Snafu}; #[async_trait] -pub trait RemoteCompactionNodePicker: Send + Sync { +pub trait CompactionNodePicker: Send + Sync { /// Get the addr of the remote compaction node. async fn get_compaction_node(&self) -> Result; } -pub type RemoteCompactionNodePickerRef = Arc; +pub type RemoteCompactionNodePickerRef = Arc; #[derive(Debug, Snafu)] pub enum Error { @@ -41,13 +41,13 @@ pub enum Error { define_result!(Error); /// RemoteCompactionNodePickerImpl is an implementation of -/// [`RemoteCompactionNodePicker`] based [`MetaClient`]. +/// [`CompactionNodePicker`] based [`MetaClient`]. pub struct RemoteCompactionNodePickerImpl { pub meta_client: MetaClientRef, } #[async_trait] -impl RemoteCompactionNodePicker for RemoteCompactionNodePickerImpl { +impl CompactionNodePicker for RemoteCompactionNodePickerImpl { /// Get proper remote compaction node info for compaction offload with meta /// client. async fn get_compaction_node(&self) -> Result { @@ -62,3 +62,17 @@ impl RemoteCompactionNodePicker for RemoteCompactionNodePickerImpl { Ok(compaction_node_addr) } } + +/// LocalCompactionNodePickerImpl is an implementation of +/// [`CompactionNodePicker`] mainly used for testing. +pub struct LocalCompactionNodePickerImpl { + endpoint: String, +} + +#[async_trait] +impl CompactionNodePicker for LocalCompactionNodePickerImpl { + /// Return the local addr and port of grpc service. + async fn get_compaction_node(&self) -> Result { + Ok(self.endpoint.clone()) + } +} From 5cf33b2efde21d77a38d379f1359ff888eff9f64 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Mon, 23 Sep 2024 15:48:07 +0800 Subject: [PATCH 37/49] use compaction mode instead of compaction offload for analytic engine config. --- .../src/compaction/runner/node_picker.rs | 11 ++++++- src/analytic_engine/src/instance/open.rs | 29 +++++++++++++------ src/analytic_engine/src/lib.rs | 20 +++++++++++-- src/analytic_engine/src/setup.rs | 2 +- 4 files changed, 48 insertions(+), 14 deletions(-) diff --git a/src/analytic_engine/src/compaction/runner/node_picker.rs b/src/analytic_engine/src/compaction/runner/node_picker.rs index 37fe7fadd4..893f544487 100644 --- a/src/analytic_engine/src/compaction/runner/node_picker.rs +++ b/src/analytic_engine/src/compaction/runner/node_picker.rs @@ -22,8 +22,17 @@ use std::sync::Arc; use async_trait::async_trait; use macros::define_result; use meta_client::{types::FetchCompactionNodeRequest, MetaClientRef}; +use serde::{Deserialize, Serialize}; use snafu::{ResultExt, Snafu}; +#[derive(Clone, Debug, Deserialize, Serialize)] +pub enum NodePicker { + // Local node picker that specifies the local endpoint. + // The endpoint in the form `addr:port`. + Local(String), + Remote, +} + #[async_trait] pub trait CompactionNodePicker: Send + Sync { /// Get the addr of the remote compaction node. @@ -66,7 +75,7 @@ impl CompactionNodePicker for RemoteCompactionNodePickerImpl { /// LocalCompactionNodePickerImpl is an implementation of /// [`CompactionNodePicker`] mainly used for testing. pub struct LocalCompactionNodePickerImpl { - endpoint: String, + pub endpoint: String, } #[async_trait] diff --git a/src/analytic_engine/src/instance/open.rs b/src/analytic_engine/src/instance/open.rs index 8edaf3ad02..7ff01fd8d3 100644 --- a/src/analytic_engine/src/instance/open.rs +++ b/src/analytic_engine/src/instance/open.rs @@ -33,9 +33,12 @@ use wal::manager::WalManagerRef; use crate::{ compaction::{ runner::{ - local_runner::LocalCompactionRunner, node_picker::RemoteCompactionNodePickerImpl, - remote_runner::RemoteCompactionRunner, CompactionRunner, CompactionRunnerPtr, - CompactionRunnerRef, + local_runner::LocalCompactionRunner, + node_picker::{ + LocalCompactionNodePickerImpl, NodePicker, RemoteCompactionNodePickerImpl, + }, + remote_runner::RemoteCompactionRunner, + CompactionRunner, CompactionRunnerPtr, CompactionRunnerRef, }, scheduler::SchedulerImpl, }, @@ -57,7 +60,7 @@ use crate::{ }, table::data::{TableCatalogInfo, TableDataRef}, table_meta_set_impl::TableMetaSetImpl, - RecoverMode, + CompactionMode, RecoverMode, }; pub(crate) struct InstanceContext { @@ -76,17 +79,25 @@ impl InstanceContext { meta_client: Option, ) -> Result { info!( - "Construct compaction runner with compaction_offload:{}", - ctx.config.compaction_offload + "Construct compaction runner with compaction_mode:{:?}", + ctx.config.compaction_mode ); - let compaction_runner: CompactionRunnerPtr = match ctx.config.compaction_offload { - true => Box::new(RemoteCompactionRunner { + let compaction_runner: CompactionRunnerPtr = match &ctx.config.compaction_mode { + CompactionMode::Offload(NodePicker::Local(endpoint)) => { + Box::new(RemoteCompactionRunner { + node_picker: Arc::new(LocalCompactionNodePickerImpl { + endpoint: endpoint.clone(), + }), + }) + } + CompactionMode::Offload(NodePicker::Remote) => Box::new(RemoteCompactionRunner { node_picker: Arc::new(RemoteCompactionNodePickerImpl { meta_client: meta_client.context(MetaClientNotExist)?, }), }), - false => Box::new(LocalCompactionRunner::new( + + CompactionMode::Local => Box::new(LocalCompactionRunner::new( ctx.runtimes.compact_runtime.clone(), &ctx.config, sst_factory.clone(), diff --git a/src/analytic_engine/src/lib.rs b/src/analytic_engine/src/lib.rs index 7fbcebf4a9..458cecfa3b 100644 --- a/src/analytic_engine/src/lib.rs +++ b/src/analytic_engine/src/lib.rs @@ -40,6 +40,7 @@ pub mod table_meta_set_impl; #[cfg(any(test, feature = "test"))] pub mod tests; +use compaction::runner::node_picker::NodePicker; use error::ErrorKind; use manifest::details::Options as ManifestOptions; use object_store::config::StorageOptions; @@ -54,6 +55,19 @@ pub use crate::{ table_options::TableOptions, }; +/// The compaction mode decides compaction offload or not. +/// +/// [CompactionMode::Offload] means offload the compaction task +/// to a local or remote node. +/// +/// [CompactionMode::Local] means local compaction, no offloading. +#[derive(Clone, Default, Debug, Deserialize, Serialize)] +pub enum CompactionMode { + #[default] + Local, + Offload(NodePicker), +} + /// Config of analytic engine #[derive(Debug, Clone, Deserialize, Serialize)] #[serde(default)] @@ -77,8 +91,8 @@ pub struct Config { pub compaction: SchedulerConfig, - /// Offload the compaction task to remote nodes or not. - pub compaction_offload: bool, + /// Offload the compaction task or not. + pub compaction_mode: CompactionMode, /// sst meta cache capacity pub sst_meta_cache_cap: Option, @@ -190,7 +204,7 @@ impl Default for Config { table_opts: TableOptions::default(), try_compat_old_layered_memtable_opts: false, compaction: SchedulerConfig::default(), - compaction_offload: false, + compaction_mode: CompactionMode::Local, sst_meta_cache_cap: Some(1000), sst_data_cache_cap: Some(1000), manifest: ManifestOptions::default(), diff --git a/src/analytic_engine/src/setup.rs b/src/analytic_engine/src/setup.rs index b99d782963..e107743db3 100644 --- a/src/analytic_engine/src/setup.rs +++ b/src/analytic_engine/src/setup.rs @@ -98,7 +98,7 @@ pub struct EngineBuilder<'a> { pub config: &'a Config, pub engine_runtimes: Arc, pub opened_wals: OpenedWals, - // Meta client is needed when compaction offload. + // Meta client is needed when compaction offload with remote node picker. pub meta_client: Option, pub node_type: NodeType, } From bf3b67cc35234b56b448dea6fb8c540480a745ac Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 24 Sep 2024 15:52:23 +0800 Subject: [PATCH 38/49] remove unnecessary NodeType. --- src/analytic_engine/src/context.rs | 4 -- src/analytic_engine/src/instance/open.rs | 9 +---- src/analytic_engine/src/setup.rs | 5 --- src/analytic_engine/src/tests/util.rs | 1 - src/benchmarks/src/util.rs | 1 - src/horaedb/src/setup.rs | 9 +---- src/server/src/grpc/mod.rs | 49 +++++++++++------------- 7 files changed, 26 insertions(+), 52 deletions(-) diff --git a/src/analytic_engine/src/context.rs b/src/analytic_engine/src/context.rs index 8efeb20e77..1a3ba8a848 100644 --- a/src/analytic_engine/src/context.rs +++ b/src/analytic_engine/src/context.rs @@ -19,7 +19,6 @@ use std::{fmt, sync::Arc}; -use common_types::cluster::NodeType; use table_engine::engine::EngineRuntimes; use crate::{sst::meta_data::cache::MetaCacheRef, Config}; @@ -34,9 +33,6 @@ pub struct OpenContext { /// Sst meta data cache. pub meta_cache: Option, - - /// The type of the node. - pub node_type: NodeType, } impl fmt::Debug for OpenContext { diff --git a/src/analytic_engine/src/instance/open.rs b/src/analytic_engine/src/instance/open.rs index 7ff01fd8d3..1b6b576f53 100644 --- a/src/analytic_engine/src/instance/open.rs +++ b/src/analytic_engine/src/instance/open.rs @@ -22,7 +22,7 @@ use std::{ sync::{Arc, RwLock}, }; -use common_types::{cluster::NodeType, table::ShardId}; +use common_types::table::ShardId; use logger::{error, info}; use meta_client::MetaClientRef; use object_store::ObjectStoreRef; @@ -107,7 +107,6 @@ impl InstanceContext { }; let local_compaction_runner: Option> = - if let NodeType::CompactionServer = ctx.node_type { // The compaction runner for compaction node. Some(Arc::new(LocalCompactionRunner::new( ctx.runtimes.compact_runtime.clone(), @@ -115,11 +114,7 @@ impl InstanceContext { sst_factory.clone(), store_picker.clone(), ctx.meta_cache.clone(), - ))) - } else { - None - }; - + ))); let instance = Instance::open( ctx, manifest_storages, diff --git a/src/analytic_engine/src/setup.rs b/src/analytic_engine/src/setup.rs index e107743db3..4075e250db 100644 --- a/src/analytic_engine/src/setup.rs +++ b/src/analytic_engine/src/setup.rs @@ -19,7 +19,6 @@ use std::{num::NonZeroUsize, path::Path, pin::Pin, sync::Arc}; -use common_types::cluster::NodeType; use futures::Future; use macros::define_result; use meta_client::MetaClientRef; @@ -100,7 +99,6 @@ pub struct EngineBuilder<'a> { pub opened_wals: OpenedWals, // Meta client is needed when compaction offload with remote node picker. pub meta_client: Option, - pub node_type: NodeType, } impl<'a> EngineBuilder<'a> { @@ -122,7 +120,6 @@ impl<'a> EngineBuilder<'a> { manifest_storages, Arc::new(opened_storages), self.meta_client, - self.node_type, ) .await?; @@ -142,7 +139,6 @@ async fn build_instance_context( manifest_storages: ManifestStorages, store_picker: ObjectStorePickerRef, meta_client: Option, - node_type: NodeType, ) -> Result { let meta_cache: Option = config .sst_meta_cache_cap @@ -152,7 +148,6 @@ async fn build_instance_context( config, runtimes: engine_runtimes, meta_cache, - node_type, }; let instance_ctx = InstanceContext::new( diff --git a/src/analytic_engine/src/tests/util.rs b/src/analytic_engine/src/tests/util.rs index d644862fdd..04bc09f75f 100644 --- a/src/analytic_engine/src/tests/util.rs +++ b/src/analytic_engine/src/tests/util.rs @@ -142,7 +142,6 @@ impl TestContext { engine_runtimes: self.runtimes.clone(), opened_wals: opened_wals.clone(), meta_client: None, - node_type: common_types::cluster::NodeType::HoraeDB, }; self.opened_wals = Some(opened_wals); diff --git a/src/benchmarks/src/util.rs b/src/benchmarks/src/util.rs index 6bd97b5bcd..97c8457be8 100644 --- a/src/benchmarks/src/util.rs +++ b/src/benchmarks/src/util.rs @@ -523,7 +523,6 @@ impl TestContext { engine_runtimes: self.runtimes.clone(), opened_wals: opened_wals.clone(), meta_client: None, - node_type: common_types::cluster::NodeType::HoraeDB, }; self.opened_wals = Some(opened_wals); diff --git a/src/horaedb/src/setup.rs b/src/horaedb/src/setup.rs index 273b1df3ac..33632b5524 100644 --- a/src/horaedb/src/setup.rs +++ b/src/horaedb/src/setup.rs @@ -26,7 +26,6 @@ use analytic_engine::{ use catalog::{manager::ManagerRef, schema::OpenOptions, table_operator::TableOperator}; use catalog_impls::{table_based::TableBasedManager, volatile, CatalogManagerImpl}; use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_set::ShardSet}; -use common_types::cluster::NodeType; use datafusion::execution::runtime_env::RuntimeConfig as DfRuntimeConfig; use df_operator::registry::{FunctionRegistry, FunctionRegistryImpl}; use interpreters::table_manipulator::{catalog_based, meta_based}; @@ -352,7 +351,6 @@ async fn build_with_meta( engine_runtimes: runtimes.clone(), opened_wals: opened_wals.clone(), meta_client: Some(meta_client.clone()), - node_type: cluster_config.node_type.clone(), }; let TableEngineContext { table_engine, @@ -384,10 +382,8 @@ async fn build_with_meta( .opened_wals(opened_wals) .router(router) .schema_config_provider(schema_config_provider); - if let NodeType::CompactionServer = cluster_config.node_type { - builder = - builder.compaction_runner(local_compaction_runner.expect("Empty compaction runner.")); - } + builder = builder.compaction_runner(local_compaction_runner.expect("Empty compaction runner.")); + builder } @@ -408,7 +404,6 @@ async fn build_without_meta( engine_runtimes: runtimes.clone(), opened_wals: opened_wals.clone(), meta_client: None, - node_type: NodeType::HoraeDB, }; let TableEngineContext { table_engine, .. } = engine_builder .build() diff --git a/src/server/src/grpc/mod.rs b/src/server/src/grpc/mod.rs index dcee780125..97d81e6c03 100644 --- a/src/server/src/grpc/mod.rs +++ b/src/server/src/grpc/mod.rs @@ -26,7 +26,7 @@ use std::{ use analytic_engine::compaction::runner::CompactionRunnerRef; use cluster::ClusterRef; -use common_types::{cluster::NodeType, column_schema}; +use common_types::column_schema; use compaction_service::CompactionServiceImpl; use futures::FutureExt; use generic_error::GenericError; @@ -332,32 +332,27 @@ impl Builder { self.cluster .map(|v| { let result: Result<()> = (|| { - match v.node_type() { - NodeType::HoraeDB => { - // Support meta rpc service. - let opened_wals = self.opened_wals.context(MissingWals)?; - let builder = meta_event_service::Builder { - cluster: v, - instance: instance.clone(), - runtime: runtimes.meta_runtime.clone(), - opened_wals, - }; - meta_rpc_server = Some(MetaEventServiceServer::new(builder.build())); - } - NodeType::CompactionServer => { - // Support remote rpc service. - let compaction_runner = - self.compaction_runner.context(MissingCompactionRunner)?; - let builder = compaction_service::Builder { - cluster: v, - instance: instance.clone(), - runtime: runtimes.compact_runtime.clone(), - compaction_runner, - }; - compaction_rpc_server = - Some(CompactionServiceServer::new(builder.build())); - } - } + // Support meta rpc service. + let opened_wals = self.opened_wals.context(MissingWals)?; + let builder = meta_event_service::Builder { + cluster: v.clone(), + instance: instance.clone(), + runtime: runtimes.meta_runtime.clone(), + opened_wals, + }; + meta_rpc_server = Some(MetaEventServiceServer::new(builder.build())); + + // Support remote compaction rpc service. + let compaction_runner = + self.compaction_runner.context(MissingCompactionRunner)?; + let builder = compaction_service::Builder { + cluster: v, + instance: instance.clone(), + runtime: runtimes.compact_runtime.clone(), + compaction_runner, + }; + compaction_rpc_server = Some(CompactionServiceServer::new(builder.build())); + Ok(()) })(); result From 1beaa16138de273388e9576a33e7569c3d08e2b2 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 24 Sep 2024 15:53:12 +0800 Subject: [PATCH 39/49] add serde tag for compaction mode. --- docs/example-cluster-3.toml | 5 +++-- src/analytic_engine/src/compaction/runner/node_picker.rs | 1 + src/analytic_engine/src/lib.rs | 1 + 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/example-cluster-3.toml b/docs/example-cluster-3.toml index ee7aa765f0..54b58909bc 100644 --- a/docs/example-cluster-3.toml +++ b/docs/example-cluster-3.toml @@ -28,8 +28,9 @@ grpc_port = 8833 mysql_port = 23307 deploy_mode = "Cluster" -[analytic] -compaction_offload = true +[analytic.compaction_mode] +compaction_mode = "Offload" +node_picker = "Remote" [analytic.storage] mem_cache_capacity = '1G' diff --git a/src/analytic_engine/src/compaction/runner/node_picker.rs b/src/analytic_engine/src/compaction/runner/node_picker.rs index 893f544487..bf21787c71 100644 --- a/src/analytic_engine/src/compaction/runner/node_picker.rs +++ b/src/analytic_engine/src/compaction/runner/node_picker.rs @@ -26,6 +26,7 @@ use serde::{Deserialize, Serialize}; use snafu::{ResultExt, Snafu}; #[derive(Clone, Debug, Deserialize, Serialize)] +#[serde(tag = "node_picker", content = "endpoint")] pub enum NodePicker { // Local node picker that specifies the local endpoint. // The endpoint in the form `addr:port`. diff --git a/src/analytic_engine/src/lib.rs b/src/analytic_engine/src/lib.rs index 458cecfa3b..687bcf637a 100644 --- a/src/analytic_engine/src/lib.rs +++ b/src/analytic_engine/src/lib.rs @@ -62,6 +62,7 @@ pub use crate::{ /// /// [CompactionMode::Local] means local compaction, no offloading. #[derive(Clone, Default, Debug, Deserialize, Serialize)] +#[serde(tag = "compaction_mode")] pub enum CompactionMode { #[default] Local, From 158a5292812e2959a233e3ad932d287cc1559fbb Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Wed, 25 Sep 2024 16:12:28 +0800 Subject: [PATCH 40/49] fix style. --- Cargo.toml | 1 - src/wal/Cargo.toml | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index ce25a9f094..556ee442f5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -102,7 +102,6 @@ thiserror = "1" bytes_ext = { path = "src/components/bytes_ext" } catalog = { path = "src/catalog" } catalog_impls = { path = "src/catalog_impls" } -# TODO(leslie): modify it when the related pr in incubator-horaedb-proto is merged. horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "84f72744efc6e5f2e3a3ad8bbb24b807e383c492" } codec = { path = "src/components/codec" } chrono = "0.4" diff --git a/src/wal/Cargo.toml b/src/wal/Cargo.toml index 30a5b00461..f691746d88 100644 --- a/src/wal/Cargo.toml +++ b/src/wal/Cargo.toml @@ -27,15 +27,15 @@ workspace = true [package.authors] workspace = true -[package.edition] -workspace = true - [dependencies.rocksdb] git = "https://github.com/tikv/rust-rocksdb.git" rev = "f04f4dd8eacc30e67c24bc2529a6d9c6edb85f8f" features = ["portable"] optional = true +[package.edition] +workspace = true + [features] wal-message-queue = ["dep:message_queue"] wal-table-kv = ["dep:table_kv"] From 86cebbbef1faad1c56b7247b62370d2217b0c7ee Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Fri, 27 Sep 2024 10:32:48 +0800 Subject: [PATCH 41/49] delete part of docs for cluster config. --- docs/example-cluster-3.toml | 63 ------------------------------------- docs/example-cluster-4.toml | 61 ----------------------------------- 2 files changed, 124 deletions(-) delete mode 100644 docs/example-cluster-3.toml delete mode 100644 docs/example-cluster-4.toml diff --git a/docs/example-cluster-3.toml b/docs/example-cluster-3.toml deleted file mode 100644 index 54b58909bc..0000000000 --- a/docs/example-cluster-3.toml +++ /dev/null @@ -1,63 +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. - -[node] -addr = "127.0.0.1" - -[logger] -level = "info" - -[server] -bind_addr = "0.0.0.0" -http_port = 5442 -grpc_port = 8833 -mysql_port = 23307 -deploy_mode = "Cluster" - -[analytic.compaction_mode] -compaction_mode = "Offload" -node_picker = "Remote" - -[analytic.storage] -mem_cache_capacity = '1G' -mem_cache_partition_bits = 0 - -[analytic.storage.object_store] -type = "Local" -data_dir = "/tmp/horaedb0" - -[analytic.wal] -type = "RocksDB" -data_dir = "/tmp/horaedb0" - -[cluster_deployment] -mode = "WithMeta" -cmd_channel_buffer_size = 10 - -[cluster_deployment.meta_client] -# Only support "defaultCluster" currently. -cluster_name = "defaultCluster" -meta_addr = "http://127.0.0.1:2379" -lease = "10s" -timeout = "5s" - -[cluster_deployment.etcd_client] -server_addrs = ['127.0.0.1:2379'] - -[limiter] -write_block_list = ['mytable1'] -read_block_list = ['mytable1'] diff --git a/docs/example-cluster-4.toml b/docs/example-cluster-4.toml deleted file mode 100644 index ac9ae5bf26..0000000000 --- a/docs/example-cluster-4.toml +++ /dev/null @@ -1,61 +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. - -[node] -addr = "127.0.0.1" - -[logger] -level = "debug" - -[server] -bind_addr = "0.0.0.0" -http_port = 5443 -grpc_port = 8834 -mysql_port = 33307 -postgresql_port = 25433 -deploy_mode = "Cluster" - -[analytic.storage] -mem_cache_capacity = '1G' -mem_cache_partition_bits = 0 - -[analytic.storage.object_store] -type = "Local" -data_dir = "/tmp/horaedb1" - -[analytic.wal] -type = "RocksDB" -data_dir = "/tmp/horaedb1" - -[cluster_deployment] -mode = "WithMeta" -cmd_channel_buffer_size = 10 -node_type = "CompactionServer" - -[cluster_deployment.meta_client] -# Only support "defaultCluster" currently. -cluster_name = "defaultCluster" -meta_addr = "http://127.0.0.1:2379" -lease = "10s" -timeout = "5s" - -[cluster_deployment.etcd_client] -server_addrs = ['127.0.0.1:2379'] - -[limiter] -write_block_list = ['mytable1'] -read_block_list = ['mytable1'] From 317d248b3eff52ba04399722751edd10e15a66dc Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Fri, 27 Sep 2024 10:10:41 +0800 Subject: [PATCH 42/49] impl simple error handle for remote compaction runner. --- .../src/compaction/runner/remote_runner.rs | 31 ++++++++++++++----- src/analytic_engine/src/instance/open.rs | 23 +++++++------- src/compaction_client/src/lib.rs | 13 ++------ 3 files changed, 38 insertions(+), 29 deletions(-) diff --git a/src/analytic_engine/src/compaction/runner/remote_runner.rs b/src/analytic_engine/src/compaction/runner/remote_runner.rs index 1b34bc8887..51ee3e1982 100644 --- a/src/analytic_engine/src/compaction/runner/remote_runner.rs +++ b/src/analytic_engine/src/compaction/runner/remote_runner.rs @@ -23,17 +23,20 @@ use compaction_client::{ use generic_error::BoxError; use snafu::ResultExt; -use super::node_picker::RemoteCompactionNodePickerRef; +use super::{local_runner::LocalCompactionRunner, node_picker::RemoteCompactionNodePickerRef}; use crate::{ compaction::runner::{CompactionRunner, CompactionRunnerResult, CompactionRunnerTask}, instance::flush_compaction::{ BuildCompactionClientFailed, ConvertCompactionTaskResponse, GetCompactionClientFailed, - PickCompactionNodeFailed, RemoteCompactFailed, Result, + PickCompactionNodeFailed, Result, }, }; pub struct RemoteCompactionRunner { pub node_picker: RemoteCompactionNodePickerRef, + /// Responsible for executing compaction task locally if fail to remote + /// compact, used for better fault tolerance. + pub local_compaction_runner: LocalCompactionRunner, } impl RemoteCompactionRunner { @@ -51,20 +54,34 @@ impl RemoteCompactionRunner { .context(BuildCompactionClientFailed)?; Ok(client) } + + async fn local_compact(&self, task: CompactionRunnerTask) -> Result { + self.local_compaction_runner.run(task).await + } } #[async_trait] impl CompactionRunner for RemoteCompactionRunner { + /// Run the compaction task either on a remote node or fall back to local + /// compaction. async fn run(&self, task: CompactionRunnerTask) -> Result { let client = self .get_compaction_client() .await .box_err() - .context(GetCompactionClientFailed)?; - let pb_resp = client - .execute_compaction_task(task.into()) - .await - .context(RemoteCompactFailed)?; + .context(GetCompactionClientFailed); + + let pb_resp = match client { + Ok(client) => match client.execute_compaction_task(task.clone().into()).await { + Ok(resp) => resp, + Err(_) => { + return self.local_compact(task).await; + } + }, + Err(_) => { + return self.local_compact(task).await; + } + }; let resp = pb_resp .try_into() diff --git a/src/analytic_engine/src/instance/open.rs b/src/analytic_engine/src/instance/open.rs index 1b6b576f53..0f8bc803c9 100644 --- a/src/analytic_engine/src/instance/open.rs +++ b/src/analytic_engine/src/instance/open.rs @@ -38,7 +38,7 @@ use crate::{ LocalCompactionNodePickerImpl, NodePicker, RemoteCompactionNodePickerImpl, }, remote_runner::RemoteCompactionRunner, - CompactionRunner, CompactionRunnerPtr, CompactionRunnerRef, + CompactionRunnerPtr, CompactionRunnerRef, }, scheduler::SchedulerImpl, }, @@ -83,18 +83,28 @@ impl InstanceContext { ctx.config.compaction_mode ); + let local_compaction_runner = LocalCompactionRunner::new( + ctx.runtimes.compact_runtime.clone(), + &ctx.config, + sst_factory.clone(), + store_picker.clone(), + ctx.meta_cache.clone(), + ); + let compaction_runner: CompactionRunnerPtr = match &ctx.config.compaction_mode { CompactionMode::Offload(NodePicker::Local(endpoint)) => { Box::new(RemoteCompactionRunner { node_picker: Arc::new(LocalCompactionNodePickerImpl { endpoint: endpoint.clone(), }), + local_compaction_runner: local_compaction_runner.clone(), }) } CompactionMode::Offload(NodePicker::Remote) => Box::new(RemoteCompactionRunner { node_picker: Arc::new(RemoteCompactionNodePickerImpl { meta_client: meta_client.context(MetaClientNotExist)?, }), + local_compaction_runner: local_compaction_runner.clone(), }), CompactionMode::Local => Box::new(LocalCompactionRunner::new( @@ -106,15 +116,6 @@ impl InstanceContext { )), }; - let local_compaction_runner: Option> = - // The compaction runner for compaction node. - Some(Arc::new(LocalCompactionRunner::new( - ctx.runtimes.compact_runtime.clone(), - &ctx.config, - sst_factory.clone(), - store_picker.clone(), - ctx.meta_cache.clone(), - ))); let instance = Instance::open( ctx, manifest_storages, @@ -127,7 +128,7 @@ impl InstanceContext { Ok(Self { instance, - local_compaction_runner, + local_compaction_runner: Some(Arc::new(local_compaction_runner)), }) } } diff --git a/src/compaction_client/src/lib.rs b/src/compaction_client/src/lib.rs index 0a74f66d17..b35a67b398 100644 --- a/src/compaction_client/src/lib.rs +++ b/src/compaction_client/src/lib.rs @@ -27,17 +27,8 @@ pub mod compaction_impl; #[derive(Debug, Snafu)] #[snafu(visibility = "pub")] pub enum Error { - #[snafu(display( - "Failed to connect the service endpoint:{}, err:{}\nBacktrace:\n{}", - addr, - source, - backtrace - ))] - FailConnect { - addr: String, - source: GenericError, - backtrace: Backtrace, - }, + #[snafu(display("Failed to connect the service endpoint:{}, err:{}", addr, source,))] + FailConnect { addr: String, source: GenericError }, #[snafu(display("Failed to execute compaction task, err:{}", source))] FailExecuteCompactionTask { source: GenericError }, From 7efa88c9702aa9b4678c58fe7adb4e0ac5ee7acc Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 30 Sep 2024 17:12:21 +0800 Subject: [PATCH 43/49] move compaction client to compaction runner crate. --- Cargo.lock | 30 ++------- Cargo.toml | 2 - src/analytic_engine/Cargo.toml | 5 +- .../src/compaction/runner/mod.rs | 23 +++++++ .../src/compaction/runner/remote_client.rs} | 17 ++++- .../src/compaction/runner/remote_runner.rs | 9 ++- .../src/instance/flush_compaction.rs | 8 ++- src/cluster/Cargo.toml | 1 - src/compaction_client/Cargo.toml | 49 -------------- src/compaction_client/src/lib.rs | 64 ------------------- src/horaedb/Cargo.toml | 1 - src/router/Cargo.toml | 1 - 12 files changed, 55 insertions(+), 155 deletions(-) rename src/{compaction_client/src/compaction_impl.rs => analytic_engine/src/compaction/runner/remote_client.rs} (88%) delete mode 100644 src/compaction_client/Cargo.toml delete mode 100644 src/compaction_client/src/lib.rs diff --git a/Cargo.lock b/Cargo.lock index 40d2c318d9..af4bb6f047 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -92,7 +92,6 @@ dependencies = [ "cluster", "codec", "common_types", - "compaction_client", "datafusion", "env_logger", "future_ext", @@ -119,10 +118,12 @@ dependencies = [ "prost 0.11.8", "rand 0.8.5", "remote_engine_client", + "reqwest 0.12.4", "router", "runtime", "sampling_cache", "serde", + "serde_json", "size_ext", "skiplist", "smallvec", @@ -134,7 +135,9 @@ dependencies = [ "thiserror", "time_ext", "tokio", + "tonic 0.8.3", "trace_metric", + "url", "wal", "xorfilter-rs", ] @@ -1499,7 +1502,6 @@ dependencies = [ "bytes_ext", "catalog", "common_types", - "compaction_client", "etcd-client", "future_ext", "generic_error", @@ -1599,28 +1601,6 @@ dependencies = [ "uuid", ] -[[package]] -name = "compaction_client" -version = "2.0.0" -dependencies = [ - "async-trait", - "common_types", - "futures 0.3.28", - "generic_error", - "horaedbproto 2.0.0", - "logger", - "macros", - "prost 0.11.8", - "reqwest 0.12.4", - "serde", - "serde_json", - "snafu 0.6.10", - "time_ext", - "tokio", - "tonic 0.8.3", - "url", -] - [[package]] name = "concurrent-queue" version = "2.1.0" @@ -3177,7 +3157,6 @@ dependencies = [ "clap", "cluster", "common_types", - "compaction_client", "datafusion", "df_operator", "etcd-client", @@ -6348,7 +6327,6 @@ dependencies = [ "async-trait", "cluster", "common_types", - "compaction_client", "generic_error", "horaedbproto 2.0.0", "logger", diff --git a/Cargo.toml b/Cargo.toml index 556ee442f5..aa96fcc6ea 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -34,7 +34,6 @@ members = [ "src/catalog_impls", "src/cluster", "src/common_types", - "src/compaction_client", "src/components/alloc_tracker", "src/components/arena", "src/components/arrow_ext", @@ -111,7 +110,6 @@ cluster = { path = "src/cluster" } criterion = "0.5" horaedb-client = "1.0.2" common_types = { path = "src/common_types" } -compaction_client = { path = "src/compaction_client" } datafusion = { git = "https://github.com/CeresDB/arrow-datafusion.git", rev = "e21b03154" } datafusion-proto = { git = "https://github.com/CeresDB/arrow-datafusion.git", rev = "e21b03154" } derive_builder = "0.12" diff --git a/src/analytic_engine/Cargo.toml b/src/analytic_engine/Cargo.toml index e742924b12..d6c642eb75 100644 --- a/src/analytic_engine/Cargo.toml +++ b/src/analytic_engine/Cargo.toml @@ -52,7 +52,6 @@ bytes_ext = { workspace = true } cluster = { workspace = true } codec = { workspace = true } common_types = { workspace = true } -compaction_client = { workspace = true } datafusion = { workspace = true } future_ext = { workspace = true } futures = { workspace = true } @@ -76,10 +75,12 @@ parquet_ext = { workspace = true } prometheus = { workspace = true } prost = { workspace = true } remote_engine_client = { workspace = true } +reqwest = { workspace = true } router = { workspace = true } runtime = { workspace = true } sampling_cache = { workspace = true } serde = { workspace = true } +serde_json = { workspace = true } size_ext = { workspace = true } skiplist = { path = "../components/skiplist" } smallvec = { workspace = true } @@ -90,7 +91,9 @@ tempfile = { workspace = true, optional = true } thiserror = { workspace = true } time_ext = { workspace = true } tokio = { workspace = true } +tonic = { workspace = true } trace_metric = { workspace = true } +url = "2.2" wal = { workspace = true } xorfilter-rs = { workspace = true } diff --git a/src/analytic_engine/src/compaction/runner/mod.rs b/src/analytic_engine/src/compaction/runner/mod.rs index 986e7067fa..c8e34484cc 100644 --- a/src/analytic_engine/src/compaction/runner/mod.rs +++ b/src/analytic_engine/src/compaction/runner/mod.rs @@ -17,6 +17,7 @@ pub mod local_runner; pub mod node_picker; +mod remote_client; pub mod remote_runner; use std::sync::Arc; @@ -54,6 +55,7 @@ pub type CompactionRunnerPtr = Box; pub type CompactionRunnerRef = Arc; #[derive(Debug, Snafu)] +#[snafu(visibility = "pub")] pub enum Error { #[snafu(display("Empty table schema.\nBacktrace:\n{}", backtrace))] EmptyTableSchema { backtrace: Backtrace }, @@ -99,6 +101,27 @@ pub enum Error { #[snafu(display("Failed to convert sst meta, err:{}", source))] ConvertSstMeta { source: GenericError }, + + #[snafu(display("Failed to connect the service endpoint:{}, err:{}", addr, source,))] + FailConnect { addr: String, source: GenericError }, + + #[snafu(display("Failed to execute compaction task, err:{}", source))] + FailExecuteCompactionTask { source: GenericError }, + + #[snafu(display("Missing header in rpc response.\nBacktrace:\n{}", backtrace))] + MissingHeader { backtrace: Backtrace }, + + #[snafu(display( + "Bad response, resp code:{}, msg:{}.\nBacktrace:\n{}", + code, + msg, + backtrace + ))] + BadResponse { + code: u32, + msg: String, + backtrace: Backtrace, + }, } define_result!(Error); diff --git a/src/compaction_client/src/compaction_impl.rs b/src/analytic_engine/src/compaction/runner/remote_client.rs similarity index 88% rename from src/compaction_client/src/compaction_impl.rs rename to src/analytic_engine/src/compaction/runner/remote_client.rs index 88587355d5..cf1f69be4a 100644 --- a/src/compaction_client/src/compaction_impl.rs +++ b/src/analytic_engine/src/compaction/runner/remote_client.rs @@ -27,9 +27,8 @@ use serde::{Deserialize, Serialize}; use snafu::{OptionExt, ResultExt}; use time_ext::ReadableDuration; -use crate::{ - BadResponse, CompactionClient, CompactionClientRef, FailConnect, FailExecuteCompactionTask, - MissingHeader, Result, +use crate::compaction::runner::{ + BadResponse, FailConnect, FailExecuteCompactionTask, MissingHeader, Result, }; type CompactionServiceGrpcClient = CompactionServiceClient; @@ -50,6 +49,18 @@ impl Default for CompactionClientConfig { } } +/// CompactionClient is the abstraction of client used for HoraeDB to +/// communicate with CompactionServer cluster. +#[async_trait] +pub trait CompactionClient: Send + Sync { + async fn execute_compaction_task( + &self, + req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, + ) -> Result; +} + +pub type CompactionClientRef = Arc; + /// Default compaction client impl, will interact with the remote compaction /// node. pub struct CompactionClientImpl { diff --git a/src/analytic_engine/src/compaction/runner/remote_runner.rs b/src/analytic_engine/src/compaction/runner/remote_runner.rs index 51ee3e1982..7d487133df 100644 --- a/src/analytic_engine/src/compaction/runner/remote_runner.rs +++ b/src/analytic_engine/src/compaction/runner/remote_runner.rs @@ -16,16 +16,15 @@ // under the License. use async_trait::async_trait; -use compaction_client::{ - compaction_impl::{build_compaction_client, CompactionClientConfig}, - CompactionClientRef, -}; use generic_error::BoxError; use snafu::ResultExt; use super::{local_runner::LocalCompactionRunner, node_picker::RemoteCompactionNodePickerRef}; use crate::{ - compaction::runner::{CompactionRunner, CompactionRunnerResult, CompactionRunnerTask}, + compaction::runner::{ + remote_client::{build_compaction_client, CompactionClientConfig, CompactionClientRef}, + CompactionRunner, CompactionRunnerResult, CompactionRunnerTask, + }, instance::flush_compaction::{ BuildCompactionClientFailed, ConvertCompactionTaskResponse, GetCompactionClientFailed, PickCompactionNodeFailed, Result, diff --git a/src/analytic_engine/src/instance/flush_compaction.rs b/src/analytic_engine/src/instance/flush_compaction.rs index 2ac5539326..9deceff563 100644 --- a/src/analytic_engine/src/instance/flush_compaction.rs +++ b/src/analytic_engine/src/instance/flush_compaction.rs @@ -167,13 +167,17 @@ pub enum Error { PickCompactionNodeFailed { source: node_picker::Error }, #[snafu(display("Failed to build compaction client, err:{}", source))] - BuildCompactionClientFailed { source: compaction_client::Error }, + BuildCompactionClientFailed { + source: crate::compaction::runner::Error, + }, #[snafu(display("Failed to get compaction client, err:{}", source))] GetCompactionClientFailed { source: GenericError }, #[snafu(display("Failed to execute compaction task remotely, err:{}", source))] - RemoteCompactFailed { source: compaction_client::Error }, + RemoteCompactFailed { + source: crate::compaction::runner::Error, + }, } define_result!(Error); diff --git a/src/cluster/Cargo.toml b/src/cluster/Cargo.toml index bd7025ea3e..e48fd847c1 100644 --- a/src/cluster/Cargo.toml +++ b/src/cluster/Cargo.toml @@ -35,7 +35,6 @@ async-trait = { workspace = true } bytes_ext = { workspace = true } catalog = { workspace = true } common_types = { workspace = true } -compaction_client = { workspace = true } etcd-client = { workspace = true } future_ext = { workspace = true } generic_error = { workspace = true } diff --git a/src/compaction_client/Cargo.toml b/src/compaction_client/Cargo.toml deleted file mode 100644 index 36374130ff..0000000000 --- a/src/compaction_client/Cargo.toml +++ /dev/null @@ -1,49 +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] -name = "compaction_client" - -[package.license] -workspace = true - -[package.version] -workspace = true - -[package.authors] -workspace = true - -[package.edition] -workspace = true - -[dependencies] -async-trait = { workspace = true } -common_types = { workspace = true } -futures = { workspace = true } -generic_error = { workspace = true } -horaedbproto = { workspace = true } -logger = { workspace = true } -macros = { workspace = true } -prost = { workspace = true } -reqwest = { workspace = true } -serde = { workspace = true } -serde_json = { workspace = true } -snafu = { workspace = true } -time_ext = { workspace = true } -tokio = { workspace = true } -tonic = { workspace = true } -url = "2.2" diff --git a/src/compaction_client/src/lib.rs b/src/compaction_client/src/lib.rs deleted file mode 100644 index b35a67b398..0000000000 --- a/src/compaction_client/src/lib.rs +++ /dev/null @@ -1,64 +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. - -use std::sync::Arc; - -use async_trait::async_trait; -use generic_error::GenericError; -use macros::define_result; -use snafu::{Backtrace, Snafu}; - -pub mod compaction_impl; - -#[derive(Debug, Snafu)] -#[snafu(visibility = "pub")] -pub enum Error { - #[snafu(display("Failed to connect the service endpoint:{}, err:{}", addr, source,))] - FailConnect { addr: String, source: GenericError }, - - #[snafu(display("Failed to execute compaction task, err:{}", source))] - FailExecuteCompactionTask { source: GenericError }, - - #[snafu(display("Missing header in rpc response.\nBacktrace:\n{}", backtrace))] - MissingHeader { backtrace: Backtrace }, - - #[snafu(display( - "Bad response, resp code:{}, msg:{}.\nBacktrace:\n{}", - code, - msg, - backtrace - ))] - BadResponse { - code: u32, - msg: String, - backtrace: Backtrace, - }, -} - -define_result!(Error); - -/// CompactionClient is the abstraction of client used for HoraeDB to -/// communicate with CompactionServer cluster. -#[async_trait] -pub trait CompactionClient: Send + Sync { - async fn execute_compaction_task( - &self, - req: horaedbproto::compaction_service::ExecuteCompactionTaskRequest, - ) -> Result; -} - -pub type CompactionClientRef = Arc; diff --git a/src/horaedb/Cargo.toml b/src/horaedb/Cargo.toml index 943de17bbe..a056ac6456 100644 --- a/src/horaedb/Cargo.toml +++ b/src/horaedb/Cargo.toml @@ -44,7 +44,6 @@ catalog_impls = { workspace = true } clap = { workspace = true } cluster = { workspace = true } common_types = { workspace = true } -compaction_client = { workspace = true } datafusion = { workspace = true } df_operator = { workspace = true } etcd-client = { workspace = true } diff --git a/src/router/Cargo.toml b/src/router/Cargo.toml index 7183c270e5..7f3d8c23ac 100644 --- a/src/router/Cargo.toml +++ b/src/router/Cargo.toml @@ -34,7 +34,6 @@ workspace = true async-trait = { workspace = true } cluster = { workspace = true } common_types = { workspace = true } -compaction_client = { workspace = true } generic_error = { workspace = true } horaedbproto = { workspace = true } logger = { workspace = true } From 2f362086826510be52f2b6a8dc246ae2f81c3288 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 30 Sep 2024 17:33:28 +0800 Subject: [PATCH 44/49] sort out codes for compation service. --- Cargo.lock | 2 +- Cargo.toml | 2 +- src/meta_client/src/meta_impl.rs | 28 ++++------------------------ src/meta_client/src/types.rs | 19 ------------------- 4 files changed, 6 insertions(+), 45 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index af4bb6f047..68c58c31aa 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3230,7 +3230,7 @@ dependencies = [ [[package]] name = "horaedbproto" version = "2.0.0" -source = "git+https://github.com/LeslieKid/incubator-horaedb-proto.git?rev=84f72744efc6e5f2e3a3ad8bbb24b807e383c492#84f72744efc6e5f2e3a3ad8bbb24b807e383c492" +source = "git+https://github.com/LeslieKid/incubator-horaedb-proto.git?rev=c8a073af23a7e5d29ec88011dd3c5abeffd37d23#c8a073af23a7e5d29ec88011dd3c5abeffd37d23" dependencies = [ "prost 0.11.8", "protoc-bin-vendored", diff --git a/Cargo.toml b/Cargo.toml index aa96fcc6ea..d9fd6e2ecf 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -101,7 +101,7 @@ thiserror = "1" bytes_ext = { path = "src/components/bytes_ext" } catalog = { path = "src/catalog" } catalog_impls = { path = "src/catalog_impls" } -horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "84f72744efc6e5f2e3a3ad8bbb24b807e383c492" } +horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "c8a073af23a7e5d29ec88011dd3c5abeffd37d23" } codec = { path = "src/components/codec" } chrono = "0.4" clap = { version = "4.5.1", features = ["derive"] } diff --git a/src/meta_client/src/meta_impl.rs b/src/meta_client/src/meta_impl.rs index 74e07d3817..ffe32faeb8 100644 --- a/src/meta_client/src/meta_impl.rs +++ b/src/meta_client/src/meta_impl.rs @@ -36,9 +36,8 @@ use crate::{ GetTablesOfShardsResponse, NodeInfo, NodeMetaInfo, RequestHeader, RouteTablesRequest, RouteTablesResponse, ShardInfo, }, - BadResponse, FailAllocSchemaId, FailConnect, FailCreateTable, FailDropTable, - FailFetchCompactionNode, FailGetTables, FailRouteTables, FailSendHeartbeat, MetaClient, - MetaClientRef, MissingHeader, Result, + BadResponse, FailAllocSchemaId, FailConnect, FailCreateTable, FailDropTable, FailGetTables, + FailRouteTables, FailSendHeartbeat, MetaClient, MetaClientRef, MissingHeader, Result, }; type MetaServiceGrpcClient = MetaRpcServiceClient; @@ -240,28 +239,9 @@ impl MetaClient for MetaClientImpl { async fn fetch_compaction_node( &self, - req: FetchCompactionNodeRequest, + _req: FetchCompactionNodeRequest, ) -> Result { - let mut pb_req = meta_service::FetchCompactionNodeRequest::from(req); - pb_req.header = Some(self.request_header().into()); - - debug!("Meta client try to fetch compaction node, req:{:?}", pb_req); - - let pb_resp = self - .client() - .fetch_compaction_node(pb_req) - .await - .box_err() - .context(FailFetchCompactionNode)? - .into_inner(); - - debug!( - "Meta client finish fetching compaction node, resp:{:?}", - pb_resp - ); - - check_response_header(&pb_resp.header)?; - Ok(FetchCompactionNodeResponse::from(pb_resp)) + todo!() } async fn send_heartbeat(&self, shard_infos: Vec) -> Result<()> { diff --git a/src/meta_client/src/types.rs b/src/meta_client/src/types.rs index 8c7aff3bc7..524843620b 100644 --- a/src/meta_client/src/types.rs +++ b/src/meta_client/src/types.rs @@ -283,11 +283,6 @@ impl From for meta_service_pb::NodeInfo { binary_version: node_info.node_meta_info.binary_version, shard_infos, lease: 0, - node_type: if node_info.node_meta_info.node_type == NodeType::HoraeDB { - cluster_pb::NodeType::HoraeDb - } else { - cluster_pb::NodeType::CompactionServer - } as i32, } } } @@ -603,17 +598,3 @@ pub struct FetchCompactionNodeRequest {} pub struct FetchCompactionNodeResponse { pub endpoint: String, } - -impl From for meta_service_pb::FetchCompactionNodeRequest { - fn from(_: FetchCompactionNodeRequest) -> Self { - meta_service_pb::FetchCompactionNodeRequest::default() - } -} - -impl From for FetchCompactionNodeResponse { - fn from(value: meta_service_pb::FetchCompactionNodeResponse) -> Self { - Self { - endpoint: value.endpoint, - } - } -} From d03c4c57329e8964d9b16557701ff525f8f9f782 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 30 Sep 2024 17:48:38 +0800 Subject: [PATCH 45/49] redirect horaedbproto to apache repo. --- Cargo.lock | 2 +- Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 68c58c31aa..e4e5468dac 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3230,7 +3230,7 @@ dependencies = [ [[package]] name = "horaedbproto" version = "2.0.0" -source = "git+https://github.com/LeslieKid/incubator-horaedb-proto.git?rev=c8a073af23a7e5d29ec88011dd3c5abeffd37d23#c8a073af23a7e5d29ec88011dd3c5abeffd37d23" +source = "git+https://github.com/apache/incubator-horaedb-proto.git?rev=fac8564e6e3d50e51daa2af6eb905e747f3191b0#fac8564e6e3d50e51daa2af6eb905e747f3191b0" dependencies = [ "prost 0.11.8", "protoc-bin-vendored", diff --git a/Cargo.toml b/Cargo.toml index d9fd6e2ecf..db0812561c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -101,7 +101,7 @@ thiserror = "1" bytes_ext = { path = "src/components/bytes_ext" } catalog = { path = "src/catalog" } catalog_impls = { path = "src/catalog_impls" } -horaedbproto = { git = "https://github.com/LeslieKid/incubator-horaedb-proto.git", rev = "c8a073af23a7e5d29ec88011dd3c5abeffd37d23" } +horaedbproto = { git = "https://github.com/apache/incubator-horaedb-proto.git", rev = "fac8564e6e3d50e51daa2af6eb905e747f3191b0" } codec = { path = "src/components/codec" } chrono = "0.4" clap = { version = "4.5.1", features = ["derive"] } From 0e6fe29e4944f6b4bdc21c47644aae5e75939bc0 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 1 Oct 2024 19:41:34 +0800 Subject: [PATCH 46/49] remove unnecessary field in compaction service. --- src/server/src/grpc/compaction_service/mod.rs | 10 ---------- src/server/src/grpc/mod.rs | 2 -- 2 files changed, 12 deletions(-) diff --git a/src/server/src/grpc/compaction_service/mod.rs b/src/server/src/grpc/compaction_service/mod.rs index c2635d585c..3954b78a44 100644 --- a/src/server/src/grpc/compaction_service/mod.rs +++ b/src/server/src/grpc/compaction_service/mod.rs @@ -21,14 +21,12 @@ use std::sync::Arc; use analytic_engine::compaction::runner::{CompactionRunnerRef, CompactionRunnerTask}; use async_trait::async_trait; -use cluster::ClusterRef; use error::{build_err_header, build_ok_header, ErrWithCause, StatusCode}; use generic_error::BoxError; use horaedbproto::compaction_service::{ compaction_service_server::CompactionService, ExecResult, ExecuteCompactionTaskRequest, ExecuteCompactionTaskResponse, }; -use proxy::instance::InstanceRef; use runtime::Runtime; use snafu::ResultExt; use tonic::{Request, Response, Status}; @@ -37,8 +35,6 @@ mod error; /// Builder for [CompactionServiceImpl] pub struct Builder { - pub cluster: ClusterRef, - pub instance: InstanceRef, pub runtime: Arc, pub compaction_runner: CompactionRunnerRef, } @@ -46,15 +42,11 @@ pub struct Builder { impl Builder { pub fn build(self) -> CompactionServiceImpl { let Self { - cluster, - instance, runtime, compaction_runner, } = self; CompactionServiceImpl { - cluster, - instance, runtime, compaction_runner, } @@ -63,8 +55,6 @@ impl Builder { #[derive(Clone)] pub struct CompactionServiceImpl { - pub cluster: ClusterRef, - pub instance: InstanceRef, pub runtime: Arc, pub compaction_runner: CompactionRunnerRef, } diff --git a/src/server/src/grpc/mod.rs b/src/server/src/grpc/mod.rs index 97d81e6c03..24a181682b 100644 --- a/src/server/src/grpc/mod.rs +++ b/src/server/src/grpc/mod.rs @@ -346,8 +346,6 @@ impl Builder { let compaction_runner = self.compaction_runner.context(MissingCompactionRunner)?; let builder = compaction_service::Builder { - cluster: v, - instance: instance.clone(), runtime: runtimes.compact_runtime.clone(), compaction_runner, }; From 0b821cc8a13609ec0c6505df69fa51a35a27c372 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 1 Oct 2024 20:50:54 +0800 Subject: [PATCH 47/49] introduce fallback_local_when_failed in remote compaction runner. --- .../src/compaction/runner/remote_runner.rs | 29 +++++++++++++++---- src/analytic_engine/src/instance/open.rs | 3 ++ 2 files changed, 27 insertions(+), 5 deletions(-) diff --git a/src/analytic_engine/src/compaction/runner/remote_runner.rs b/src/analytic_engine/src/compaction/runner/remote_runner.rs index 7d487133df..39c08761e7 100644 --- a/src/analytic_engine/src/compaction/runner/remote_runner.rs +++ b/src/analytic_engine/src/compaction/runner/remote_runner.rs @@ -17,6 +17,7 @@ use async_trait::async_trait; use generic_error::BoxError; +use logger::info; use snafu::ResultExt; use super::{local_runner::LocalCompactionRunner, node_picker::RemoteCompactionNodePickerRef}; @@ -26,15 +27,17 @@ use crate::{ CompactionRunner, CompactionRunnerResult, CompactionRunnerTask, }, instance::flush_compaction::{ - BuildCompactionClientFailed, ConvertCompactionTaskResponse, GetCompactionClientFailed, - PickCompactionNodeFailed, Result, + self, BuildCompactionClientFailed, ConvertCompactionTaskResponse, GetCompactionClientFailed, PickCompactionNodeFailed, Result }, }; pub struct RemoteCompactionRunner { pub node_picker: RemoteCompactionNodePickerRef, + + pub fallback_local_when_failed: bool, /// Responsible for executing compaction task locally if fail to remote - /// compact, used for better fault tolerance. + /// compact when `fallback_local_when_failed` is true, used for better fault + /// tolerance. pub local_compaction_runner: LocalCompactionRunner, } @@ -73,11 +76,27 @@ impl CompactionRunner for RemoteCompactionRunner { let pb_resp = match client { Ok(client) => match client.execute_compaction_task(task.clone().into()).await { Ok(resp) => resp, - Err(_) => { + Err(e) => { + if !self.fallback_local_when_failed { + return Err(flush_compaction::Error::RemoteCompactFailed { source: e } ); + } + + info!( + "The compaction task falls back to local because of error:{}", + e + ); return self.local_compact(task).await; } }, - Err(_) => { + Err(e) => { + if !self.fallback_local_when_failed { + return Err(e); + } + + info!( + "The compaction task falls back to local because of error:{}", + e + ); return self.local_compact(task).await; } }; diff --git a/src/analytic_engine/src/instance/open.rs b/src/analytic_engine/src/instance/open.rs index 0f8bc803c9..97717c5ab0 100644 --- a/src/analytic_engine/src/instance/open.rs +++ b/src/analytic_engine/src/instance/open.rs @@ -97,6 +97,8 @@ impl InstanceContext { node_picker: Arc::new(LocalCompactionNodePickerImpl { endpoint: endpoint.clone(), }), + // This field is set to false here for testing. + fallback_local_when_failed: false, local_compaction_runner: local_compaction_runner.clone(), }) } @@ -104,6 +106,7 @@ impl InstanceContext { node_picker: Arc::new(RemoteCompactionNodePickerImpl { meta_client: meta_client.context(MetaClientNotExist)?, }), + fallback_local_when_failed: true, local_compaction_runner: local_compaction_runner.clone(), }), From b3ef9a5e58d7c14262452ce0ae33ea5b3e90c0c9 Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 1 Oct 2024 21:34:22 +0800 Subject: [PATCH 48/49] make endpoint formatted. --- .../src/compaction/runner/remote_runner.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/analytic_engine/src/compaction/runner/remote_runner.rs b/src/analytic_engine/src/compaction/runner/remote_runner.rs index 39c08761e7..c0ebe5799c 100644 --- a/src/analytic_engine/src/compaction/runner/remote_runner.rs +++ b/src/analytic_engine/src/compaction/runner/remote_runner.rs @@ -44,12 +44,12 @@ pub struct RemoteCompactionRunner { impl RemoteCompactionRunner { async fn get_compaction_client(&self) -> Result { let mut config = CompactionClientConfig::default(); - let node_addr = self + let endpoint = self .node_picker .get_compaction_node() .await .context(PickCompactionNodeFailed)?; - config.compaction_server_addr = node_addr; + config.compaction_server_addr = make_formatted_endpoint(&endpoint); let client = build_compaction_client(config) .await @@ -109,3 +109,7 @@ impl CompactionRunner for RemoteCompactionRunner { Ok(resp) } } + +fn make_formatted_endpoint(endpoint: &str) -> String { + format!("http://{endpoint}") +} From f07dcd27bf6e83c7bfaf838b0f96dfb524c73e0e Mon Sep 17 00:00:00 2001 From: LeslieKid <3530611790@qq.com> Date: Tue, 1 Oct 2024 21:35:56 +0800 Subject: [PATCH 49/49] fix fmt. --- src/analytic_engine/src/compaction/runner/remote_runner.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/analytic_engine/src/compaction/runner/remote_runner.rs b/src/analytic_engine/src/compaction/runner/remote_runner.rs index c0ebe5799c..59a70c2fc2 100644 --- a/src/analytic_engine/src/compaction/runner/remote_runner.rs +++ b/src/analytic_engine/src/compaction/runner/remote_runner.rs @@ -27,7 +27,8 @@ use crate::{ CompactionRunner, CompactionRunnerResult, CompactionRunnerTask, }, instance::flush_compaction::{ - self, BuildCompactionClientFailed, ConvertCompactionTaskResponse, GetCompactionClientFailed, PickCompactionNodeFailed, Result + self, BuildCompactionClientFailed, ConvertCompactionTaskResponse, + GetCompactionClientFailed, PickCompactionNodeFailed, Result, }, }; @@ -78,7 +79,7 @@ impl CompactionRunner for RemoteCompactionRunner { Ok(resp) => resp, Err(e) => { if !self.fallback_local_when_failed { - return Err(flush_compaction::Error::RemoteCompactFailed { source: e } ); + return Err(flush_compaction::Error::RemoteCompactFailed { source: e }); } info!( @@ -91,7 +92,7 @@ impl CompactionRunner for RemoteCompactionRunner { Err(e) => { if !self.fallback_local_when_failed { return Err(e); - } + } info!( "The compaction task falls back to local because of error:{}",