From 5204ea4495ce77e5bbff00997c8155488867b5cc Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Thu, 17 Oct 2024 17:02:44 +0800 Subject: [PATCH 1/4] feat(sink): introduce batching strategy for file sink (#18472) --- ci/workflows/main-cron.yml | 22 ++ e2e_test/s3/fs_sink_batch.py | 142 +++++++++ e2e_test/sink/append_only_sink.slt | 26 ++ src/connector/src/sink/catalog/desc.rs | 17 + src/connector/src/sink/file_sink/azblob.rs | 13 +- .../src/sink/file_sink/batching_log_sink.rs | 172 ++++++++++ src/connector/src/sink/file_sink/fs.rs | 11 + src/connector/src/sink/file_sink/gcs.rs | 13 +- src/connector/src/sink/file_sink/mod.rs | 1 + .../src/sink/file_sink/opendal_sink.rs | 299 +++++++++++++----- src/connector/src/sink/file_sink/s3.rs | 13 +- src/connector/src/sink/file_sink/webhdfs.rs | 13 +- src/connector/src/with_options.rs | 1 + src/connector/with_options_sink.yaml | 60 ++++ .../src/optimizer/plan_node/stream_sink.rs | 7 +- 15 files changed, 720 insertions(+), 90 deletions(-) create mode 100644 e2e_test/s3/fs_sink_batch.py create mode 100644 src/connector/src/sink/file_sink/batching_log_sink.rs diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 8252e44df305b..e5fba261aa376 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -530,6 +530,28 @@ steps: timeout_in_minutes: 25 retry: *auto-retry + - label: "S3 sink batching test" + key: "s3-sink-batching-test" + command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_sink_batch.py" + if: | + !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-s3-source-tests" + || build.env("CI_STEPS") =~ /(^|,)s3-source-tests?(,|$$)/ + depends_on: build + plugins: + - seek-oss/aws-sm#v2.3.1: + env: + S3_SOURCE_TEST_CONF: ci_s3_source_test_aws + - docker-compose#v5.1.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + environment: + - S3_SOURCE_TEST_CONF + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 25 + retry: *auto-retry + - label: "S3 source batch read on AWS (json parser)" key: "s3-v2-source-batch-read-check-aws-json-parser" command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_source_batch.py -t json" diff --git a/e2e_test/s3/fs_sink_batch.py b/e2e_test/s3/fs_sink_batch.py new file mode 100644 index 0000000000000..125675f61beed --- /dev/null +++ b/e2e_test/s3/fs_sink_batch.py @@ -0,0 +1,142 @@ +import os +import sys +import random +import psycopg2 +import json +import time +import pyarrow as pa +import pyarrow.parquet as pq +import pandas as pd +from datetime import datetime, timezone +from time import sleep +from minio import Minio +from random import uniform + +def do_test(config, file_num, item_num_per_file, prefix): + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + # Open a cursor to execute SQL statements + cur = conn.cursor() + + + # Execute a SELECT statement + cur.execute(f'''CREATE TABLE t (v1 int, v2 int);''') + + print('create sink') + cur.execute(f'''CREATE sink test_file_sink_batching as select + v1, v2 from t WITH ( + connector = 's3', + s3.region_name = 'custom', + s3.bucket_name = 'hummock001', + s3.credentials.access = 'hummockadmin', + s3.credentials.secret = 'hummockadmin', + s3.endpoint_url = 'http://hummock001.127.0.0.1:9301', + s3.path = 'test_sink/', + s3.file_type = 'parquet', + type = 'append-only', + rollover_seconds = 5, + max_row_count = 5, + force_append_only='true' + ) FORMAT PLAIN ENCODE PARQUET(force_append_only='true');''') + + cur.execute(f'''CREATE TABLE test_sink_table( + v1 int, + v2 int, + ) WITH ( + connector = 's3', + match_pattern = 'test_sink/*.parquet', + refresh.interval.sec = 1, + s3.region_name = 'custom', + s3.bucket_name = 'hummock001', + s3.credentials.access = 'hummockadmin', + s3.credentials.secret = 'hummockadmin', + s3.endpoint_url = 'http://hummock001.127.0.0.1:9301', + ) FORMAT PLAIN ENCODE PARQUET;''') + + cur.execute(f'''ALTER SINK test_file_sink_batching SET PARALLELISM = 2;''') + + cur.execute(f'''INSERT INTO t VALUES (10, 10);''') + + + cur.execute(f'select count(*) from test_sink_table') + # no item will be selectedpsq + result = cur.fetchone() + + def _assert_eq(field, got, expect): + assert got == expect, f'{field} assertion failed: got {got}, expect {expect}.' + def _assert_greater(field, got, expect): + assert got > expect, f'{field} assertion failed: got {got}, expect {expect}.' + + _assert_eq('count(*)', result[0], 0) + print('the rollover_seconds has not reached, count(*) = 0') + + + time.sleep(11) + + cur.execute(f'select count(*) from test_sink_table') + result = cur.fetchone() + _assert_eq('count(*)', result[0], 1) + print('the rollover_seconds has reached, count(*) = ', result[0]) + + cur.execute(f''' + INSERT INTO t VALUES (20, 20); + INSERT INTO t VALUES (30, 30); + INSERT INTO t VALUES (40, 40); + INSERT INTO t VALUES (50, 10); + ''') + + cur.execute(f'select count(*) from test_sink_table') + # count(*) = 1 + result = cur.fetchone() + _assert_eq('count(*)', result[0], 1) + print('the max row count has not reached, count(*) = ', result[0]) + + cur.execute(f''' + INSERT INTO t VALUES (60, 20); + INSERT INTO t VALUES (70, 30); + INSERT INTO t VALUES (80, 10); + INSERT INTO t VALUES (90, 20); + INSERT INTO t VALUES (100, 30); + INSERT INTO t VALUES (100, 10); + ''') + + time.sleep(10) + + cur.execute(f'select count(*) from test_sink_table') + result = cur.fetchone() + _assert_greater('count(*)', result[0], 1) + print('the rollover_seconds has reached, count(*) = ', result[0]) + + cur.execute(f'drop sink test_file_sink_batching;') + cur.execute(f'drop table t;') + cur.execute(f'drop table test_sink_table;') + cur.close() + conn.close() + +if __name__ == "__main__": + FILE_NUM = 10 + ITEM_NUM_PER_FILE = 2000 + + config = json.loads(os.environ["S3_SOURCE_TEST_CONF"]) + client = Minio( + "127.0.0.1:9301", + "hummockadmin", + "hummockadmin", + secure=False, + ) + run_id = str(random.randint(1000, 9999)) + _local = lambda idx: f'data_{idx}.parquet' + _s3 = lambda idx: f"{run_id}_data_{idx}.parquet" + + do_test(config, FILE_NUM, ITEM_NUM_PER_FILE, run_id) + + objects = client.list_objects("hummock001", prefix="test_sink/", recursive=True) + + for obj in objects: + client.remove_object("hummock001", obj.object_name) + print(f"Deleted: {obj.object_name}") diff --git a/e2e_test/sink/append_only_sink.slt b/e2e_test/sink/append_only_sink.slt index 23bd1706465e3..486c5af6f94a0 100644 --- a/e2e_test/sink/append_only_sink.slt +++ b/e2e_test/sink/append_only_sink.slt @@ -25,6 +25,32 @@ create sink invalid_force_append_only from t with (connector = 'blackhole', forc statement error unsupported sink type invalid create sink invalid_connector from t with (connector = 'invalid'); +statement ok +set sink_decouple=false; + +statement error +CREATE SINK file_sink +FROM + t +WITH +( + connector = 's3', + s3.region_name = 'test', + s3.bucket_name = 'test', + s3.path = '', + s3.file_type = 'parquet', + type = 'append-only', + force_append_only='true' +) FORMAT PLAIN ENCODE PARQUET(force_append_only='true'); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Sink error + 2: config error + 3: File sink can only be created with sink_decouple enabled. Please run `set sink_decouple = true` first. + + statement ok drop sink s1 diff --git a/src/connector/src/sink/catalog/desc.rs b/src/connector/src/sink/catalog/desc.rs index 22be28c7e2db6..5f097eafa33c9 100644 --- a/src/connector/src/sink/catalog/desc.rs +++ b/src/connector/src/sink/catalog/desc.rs @@ -23,6 +23,11 @@ use risingwave_pb::secret::PbSecretRef; use risingwave_pb::stream_plan::PbSinkDesc; use super::{SinkCatalog, SinkFormatDesc, SinkId, SinkType}; +use crate::sink::file_sink::azblob::AZBLOB_SINK; +use crate::sink::file_sink::fs::FS_SINK; +use crate::sink::file_sink::s3::S3_SINK; +use crate::sink::file_sink::webhdfs::WEBHDFS_SINK; +use crate::sink::CONNECTOR_TYPE_KEY; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct SinkDesc { @@ -140,4 +145,16 @@ impl SinkDesc { secret_refs: self.secret_refs.clone(), } } + + pub fn is_file_sink(&self) -> bool { + self.properties + .get(CONNECTOR_TYPE_KEY) + .map(|s| { + s.eq_ignore_ascii_case(FS_SINK) + || s.eq_ignore_ascii_case(AZBLOB_SINK) + || s.eq_ignore_ascii_case(S3_SINK) + || s.eq_ignore_ascii_case(WEBHDFS_SINK) + }) + .unwrap_or(false) + } } diff --git a/src/connector/src/sink/file_sink/azblob.rs b/src/connector/src/sink/file_sink/azblob.rs index 182f469902ec6..2244e5a37a914 100644 --- a/src/connector/src/sink/file_sink/azblob.rs +++ b/src/connector/src/sink/file_sink/azblob.rs @@ -21,7 +21,7 @@ use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use super::opendal_sink::FileSink; +use super::opendal_sink::{BatchingStrategy, FileSink}; use crate::sink::file_sink::opendal_sink::OpendalSinkBackend; use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; use crate::source::UnknownFields; @@ -46,6 +46,9 @@ pub struct AzblobConfig { #[serde(flatten)] pub common: AzblobCommon, + #[serde(flatten)] + pub batching_strategy: BatchingStrategy, + pub r#type: String, // accept "append-only" #[serde(flatten)] @@ -128,4 +131,12 @@ impl OpendalSinkBackend for AzblobSink { fn get_engine_type() -> super::opendal_sink::EngineType { super::opendal_sink::EngineType::Azblob } + + fn get_batching_strategy(properties: Self::Properties) -> BatchingStrategy { + BatchingStrategy { + max_row_count: properties.batching_strategy.max_row_count, + rollover_seconds: properties.batching_strategy.rollover_seconds, + path_partition_prefix: properties.batching_strategy.path_partition_prefix, + } + } } diff --git a/src/connector/src/sink/file_sink/batching_log_sink.rs b/src/connector/src/sink/file_sink/batching_log_sink.rs new file mode 100644 index 0000000000000..864fa6294c530 --- /dev/null +++ b/src/connector/src/sink/file_sink/batching_log_sink.rs @@ -0,0 +1,172 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed 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 crate::sink::file_sink::opendal_sink::OpenDalSinkWriter; +use crate::sink::log_store::{LogStoreReadItem, TruncateOffset}; +use crate::sink::{LogSinker, Result, SinkLogReader}; + +/// `BatchingLogSinker` is used for a commit-decoupled sink that supports cross-barrier batching. +/// Currently, it is only used for file sinks, so it contains an `OpenDalSinkWriter`. +pub struct BatchingLogSinker { + writer: OpenDalSinkWriter, +} + +impl BatchingLogSinker { + /// Create a log sinker with a file sink writer. + pub fn new(writer: OpenDalSinkWriter) -> Self { + BatchingLogSinker { writer } + } +} + +#[async_trait] +impl LogSinker for BatchingLogSinker { + async fn consume_log_and_sink(self, log_reader: &mut impl SinkLogReader) -> Result { + let mut sink_writer = self.writer; + #[derive(Debug)] + enum LogConsumerState { + /// Mark that the log consumer is not initialized yet + Uninitialized, + + /// Mark that a new epoch has begun, and store the max_uncommitted_epoch for cross-barrier batching. + /// For example, suppose the current order is (chunk1, barrier1, chunk2, barrier2, chunk3), and the batching is not completed until chunk3, + /// that is, barrier2 and its previous chunks are not truncated, the `max_uncommitted_epoch` is barrier2. + /// When we truncate chunk3, we should first truncate barrier2, and then truncate chunk3. + EpochBegun { + curr_epoch: u64, + max_uncommitted_epoch: Option, + }, + + /// Mark that the consumer has just received a barrier + BarrierReceived { prev_epoch: u64 }, + } + + let mut state = LogConsumerState::Uninitialized; + loop { + let (epoch, item): (u64, LogStoreReadItem) = log_reader.next_item().await?; + if let LogStoreReadItem::UpdateVnodeBitmap(_) = &item { + match &state { + LogConsumerState::BarrierReceived { prev_epoch } => { + // we need to force to finish the batch here. Otherwise, there can be data loss because actor can be dropped and rebuilt during scaling. + if sink_writer.try_commit().await? { + // If epoch increased, we first need to truncate the previous epoch. + if epoch > *prev_epoch { + log_reader + .truncate(TruncateOffset::Barrier { epoch: *prev_epoch })?; + } + }; + } + _ => unreachable!( + "update vnode bitmap can be accepted only right after \ + barrier, but current state is {:?}", + state + ), + } + continue; + } + // begin_epoch when not previously began + state = match state { + LogConsumerState::Uninitialized => LogConsumerState::EpochBegun { + curr_epoch: epoch, + max_uncommitted_epoch: None, + }, + LogConsumerState::EpochBegun { + curr_epoch, + max_uncommitted_epoch, + } => { + assert!( + epoch >= curr_epoch, + "new epoch {} should not be below the current epoch {}", + epoch, + curr_epoch + ); + LogConsumerState::EpochBegun { + curr_epoch: epoch, + max_uncommitted_epoch, + } + } + LogConsumerState::BarrierReceived { prev_epoch } => { + assert!( + epoch > prev_epoch, + "new epoch {} should be greater than prev epoch {}", + epoch, + prev_epoch + ); + LogConsumerState::EpochBegun { + curr_epoch: epoch, + max_uncommitted_epoch: Some(prev_epoch), + } + } + }; + match item { + LogStoreReadItem::StreamChunk { chunk, chunk_id } => { + sink_writer.write_batch(chunk).await?; + match sink_writer.try_commit().await { + Err(e) => { + return Err(e); + } + // The file has been successfully written and is now visible to downstream consumers. + // Truncate the file to remove the specified `chunk_id` and any preceding content. + Ok(true) => { + // If epoch increased, we first need to truncate the previous epoch. + if let Some(max_uncommitted_epoch) = match state { + LogConsumerState::EpochBegun { + curr_epoch: _, + max_uncommitted_epoch, + } => max_uncommitted_epoch, + _ => unreachable!("epoch must have begun before handling barrier"), + } { + assert!(epoch > max_uncommitted_epoch); + log_reader.truncate(TruncateOffset::Barrier { + epoch: max_uncommitted_epoch, + })?; + state = LogConsumerState::EpochBegun { + curr_epoch: epoch, + max_uncommitted_epoch: None, + } + }; + + log_reader.truncate(TruncateOffset::Chunk { + epoch: (epoch), + chunk_id: (chunk_id), + })?; + } + // The file has not been written into downstream file system. + Ok(false) => {} + } + } + LogStoreReadItem::Barrier { is_checkpoint: _ } => { + let prev_epoch = match state { + LogConsumerState::EpochBegun { + curr_epoch, + max_uncommitted_epoch: _, + } => curr_epoch, + _ => unreachable!("epoch must have begun before handling barrier"), + }; + + // When the barrier arrives, call the writer's try_finish interface to check if the file write can be completed. + // If it is completed, which means the file is visible in the downstream file system, then truncate the file in the log store; otherwise, do nothing. + // Since the current data must be before the current epoch, we only need to truncate `prev_epoch`. + if sink_writer.try_commit().await? { + log_reader.truncate(TruncateOffset::Barrier { epoch: prev_epoch })?; + }; + + state = LogConsumerState::BarrierReceived { prev_epoch } + } + LogStoreReadItem::UpdateVnodeBitmap(_vnode_bitmap) => {} + } + } + } +} diff --git a/src/connector/src/sink/file_sink/fs.rs b/src/connector/src/sink/file_sink/fs.rs index 5570bab561f20..a2310d71863c7 100644 --- a/src/connector/src/sink/file_sink/fs.rs +++ b/src/connector/src/sink/file_sink/fs.rs @@ -21,6 +21,7 @@ use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; +use super::opendal_sink::BatchingStrategy; use crate::sink::file_sink::opendal_sink::{FileSink, OpendalSinkBackend}; use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; use crate::source::UnknownFields; @@ -37,6 +38,8 @@ pub struct FsCommon { pub struct FsConfig { #[serde(flatten)] pub common: FsCommon, + #[serde(flatten)] + pub batching_strategy: BatchingStrategy, pub r#type: String, // accept "append-only" @@ -97,4 +100,12 @@ impl OpendalSinkBackend for FsSink { fn get_engine_type() -> super::opendal_sink::EngineType { super::opendal_sink::EngineType::Fs } + + fn get_batching_strategy(properties: Self::Properties) -> BatchingStrategy { + BatchingStrategy { + max_row_count: properties.batching_strategy.max_row_count, + rollover_seconds: properties.batching_strategy.rollover_seconds, + path_partition_prefix: properties.batching_strategy.path_partition_prefix, + } + } } diff --git a/src/connector/src/sink/file_sink/gcs.rs b/src/connector/src/sink/file_sink/gcs.rs index bddf962aa3bd7..0278a5a42e44a 100644 --- a/src/connector/src/sink/file_sink/gcs.rs +++ b/src/connector/src/sink/file_sink/gcs.rs @@ -21,7 +21,7 @@ use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use super::opendal_sink::FileSink; +use super::opendal_sink::{BatchingStrategy, FileSink}; use crate::sink::file_sink::opendal_sink::OpendalSinkBackend; use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; use crate::source::UnknownFields; @@ -50,6 +50,9 @@ pub struct GcsConfig { #[serde(flatten)] pub common: GcsCommon, + #[serde(flatten)] + pub batching_strategy: BatchingStrategy, + pub r#type: String, // accept "append-only" #[serde(flatten)] @@ -113,4 +116,12 @@ impl OpendalSinkBackend for GcsSink { fn get_engine_type() -> super::opendal_sink::EngineType { super::opendal_sink::EngineType::Gcs } + + fn get_batching_strategy(properties: Self::Properties) -> BatchingStrategy { + BatchingStrategy { + max_row_count: properties.batching_strategy.max_row_count, + rollover_seconds: properties.batching_strategy.rollover_seconds, + path_partition_prefix: properties.batching_strategy.path_partition_prefix, + } + } } diff --git a/src/connector/src/sink/file_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs index f32c6812f3c00..c336294c402b1 100644 --- a/src/connector/src/sink/file_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. pub mod azblob; +pub mod batching_log_sink; pub mod fs; pub mod gcs; pub mod opendal_sink; diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 1f061de6c398e..2a491e7765fbe 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -16,33 +16,44 @@ use std::collections::{BTreeMap, HashMap}; use std::fmt::Write; use std::marker::PhantomData; use std::sync::Arc; +use std::time::{Duration, SystemTime, UNIX_EPOCH}; use anyhow::anyhow; -use async_trait::async_trait; use bytes::BytesMut; -use opendal::{FuturesAsyncWriter, Operator, Writer, Writer as OpendalWriter}; +use chrono::{TimeZone, Utc}; +use opendal::{FuturesAsyncWriter, Operator, Writer as OpendalWriter}; use parquet::arrow::AsyncArrowWriter; use parquet::file::properties::WriterProperties; use risingwave_common::array::arrow::arrow_schema_iceberg::{self, SchemaRef}; use risingwave_common::array::arrow::IcebergArrowConvert; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::catalog::Schema; +use serde::Deserialize; use serde_json::Value; +use serde_with::{serde_as, DisplayFromStr}; +use strum_macros::{Display, EnumString}; use tokio_util::compat::{Compat, FuturesAsyncWriteCompatExt}; +use with_options::WithOptions; use crate::sink::catalog::SinkEncode; use crate::sink::encoder::{ JsonEncoder, JsonbHandlingMode, RowEncoder, TimeHandlingMode, TimestampHandlingMode, TimestamptzHandlingMode, }; -use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; -use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam, SinkWriter, - SinkWriterMetrics, -}; +use crate::sink::file_sink::batching_log_sink::BatchingLogSinker; +use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam}; use crate::source::TryFromBTreeMap; use crate::with_options::WithOptions; +pub const DEFAULT_ROLLOVER_SECONDS: usize = 10; +pub const DEFAULT_MAX_ROW_COUNR: usize = 10240; + +pub fn default_rollover_seconds() -> usize { + DEFAULT_ROLLOVER_SECONDS +} +pub fn default_max_row_count() -> usize { + DEFAULT_MAX_ROW_COUNR +} /// The `FileSink` struct represents a file sink that uses the `OpendalSinkBackend` trait for its backend implementation. /// /// # Type Parameters @@ -56,6 +67,8 @@ pub struct FileSink { /// The schema describing the structure of the data being written to the file sink. pub(crate) schema: Schema, pub(crate) is_append_only: bool, + /// The batching strategy for sinking data to files. + pub(crate) batching_strategy: BatchingStrategy, /// The description of the sink's format. pub(crate) format_desc: SinkFormatDesc, @@ -87,6 +100,7 @@ pub trait OpendalSinkBackend: Send + Sync + 'static + Clone + PartialEq { fn new_operator(properties: Self::Properties) -> Result; fn get_path(properties: Self::Properties) -> String; fn get_engine_type() -> EngineType; + fn get_batching_strategy(properties: Self::Properties) -> BatchingStrategy; } #[derive(Clone, Debug)] @@ -100,7 +114,7 @@ pub enum EngineType { impl Sink for FileSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = BatchingLogSinker; const SINK_NAME: &'static str = S::SINK_NAME; @@ -132,16 +146,16 @@ impl Sink for FileSink { &self, writer_param: crate::sink::SinkWriterParam, ) -> Result { - Ok(OpenDalSinkWriter::new( + let writer = OpenDalSinkWriter::new( self.op.clone(), &self.path, self.schema.clone(), - self.is_append_only, writer_param.executor_id, self.format_desc.encode.clone(), self.engine_type.clone(), - )? - .into_log_sinker(SinkWriterMetrics::new(&writer_param))) + self.batching_strategy.clone(), + )?; + Ok(BatchingLogSinker::new(writer)) } } @@ -153,6 +167,7 @@ impl TryFrom for FileSink { let config = S::from_btreemap(param.properties)?; let path = S::get_path(config.clone()).to_string(); let op = S::new_operator(config.clone())?; + let batching_strategy = S::get_batching_strategy(config.clone()); let engine_type = S::get_engine_type(); Ok(Self { @@ -160,6 +175,7 @@ impl TryFrom for FileSink { path, schema, is_append_only: param.sink_type.is_append_only(), + batching_strategy, format_desc: param .format_desc .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, @@ -173,13 +189,14 @@ pub struct OpenDalSinkWriter { schema: SchemaRef, operator: Operator, sink_writer: Option, - is_append_only: bool, write_path: String, - epoch: Option, executor_id: u64, encode_type: SinkEncode, row_encoder: JsonEncoder, engine_type: EngineType, + pub(crate) batching_strategy: BatchingStrategy, + current_bached_row_num: usize, + created_time: SystemTime, } /// The `FileWriterEnum` enum represents different types of file writers used for various sink @@ -195,61 +212,139 @@ pub struct OpenDalSinkWriter { /// The choice of writer used during the actual writing process depends on the encode type of the sink. enum FileWriterEnum { ParquetFileWriter(AsyncArrowWriter>), - FileWriter(Writer), + FileWriter(OpendalWriter), } -#[async_trait] -impl SinkWriter for OpenDalSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - // Note: epoch is used to name the output files. - // Todo: after enabling sink decouple, use the new naming convention. - let epoch = self.epoch.ok_or_else(|| { - SinkError::File("epoch has not been initialize, call `begin_epoch`".to_string()) - })?; +/// Public interface exposed to `BatchingLogSinker`, used to write chunk and commit files. +impl OpenDalSinkWriter { + /// This method writes a chunk. + pub async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { if self.sink_writer.is_none() { - self.create_sink_writer(epoch).await?; - } - if self.is_append_only { - self.append_only(chunk).await - } else { - // currently file sink only supports append only mode. - unimplemented!() - } - } - - async fn begin_epoch(&mut self, epoch: u64) -> Result<()> { - self.epoch = Some(epoch); + assert_eq!(self.current_bached_row_num, 0); + self.create_sink_writer().await?; + }; + self.append_only(chunk).await?; Ok(()) } - /// For the file sink, currently, the sink decoupling feature is not enabled. - /// When a checkpoint arrives, the force commit is performed to write the data to the file. - /// In the future if flush and checkpoint is decoupled, we should enable sink decouple accordingly. - async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { - if is_checkpoint && let Some(sink_writer) = self.sink_writer.take() { + /// This method close current writer, finish writing a file and returns whether the commit is successful. + pub async fn commit(&mut self) -> Result { + if let Some(sink_writer) = self.sink_writer.take() { match sink_writer { FileWriterEnum::ParquetFileWriter(w) => { - let _ = w.close().await?; + if w.bytes_written() > 0 { + let metadata = w.close().await?; + tracing::info!( + "writer {:?}_{:?}finish write file, metadata: {:?}", + self.executor_id, + self.created_time + .duration_since(UNIX_EPOCH) + .expect("Time went backwards") + .as_secs(), + metadata + ); + } } FileWriterEnum::FileWriter(mut w) => { - let _ = w.close().await?; + w.close().await?; } }; + self.current_bached_row_num = 0; + return Ok(true); } + Ok(false) + } + + // Try commit if the batching condition is met. + pub async fn try_commit(&mut self) -> Result { + if self.can_commit() { + return self.commit().await; + } + Ok(false) + } +} + +/// Private methods related to batching. +impl OpenDalSinkWriter { + /// Method for judging whether batch condiction is met. + fn can_commit(&self) -> bool { + self.duration_seconds_since_writer_created() >= self.batching_strategy.rollover_seconds + || self.current_bached_row_num >= self.batching_strategy.max_row_count + } + + fn path_partition_prefix(&self, duration: &Duration) -> String { + let datetime = Utc + .timestamp_opt(duration.as_secs() as i64, 0) + .single() + .expect("Failed to convert timestamp to DateTime") + .with_timezone(&Utc); + let path_partition_prefix = self + .batching_strategy + .path_partition_prefix + .as_ref() + .unwrap_or(&PathPartitionPrefix::None); + match path_partition_prefix { + PathPartitionPrefix::None => "".to_string(), + PathPartitionPrefix::Day => datetime.format("%Y-%m-%d/").to_string(), + PathPartitionPrefix::Month => datetime.format("/%Y-%m/").to_string(), + PathPartitionPrefix::Hour => datetime.format("/%Y-%m-%d %H:00/").to_string(), + } + } + + fn duration_seconds_since_writer_created(&self) -> usize { + let now = SystemTime::now(); + now.duration_since(self.created_time) + .expect("Time went backwards") + .as_secs() as usize + } + // Method for writing chunk and update related batching condition. + async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { + match self + .sink_writer + .as_mut() + .ok_or_else(|| SinkError::File("Sink writer is not created.".to_string()))? + { + FileWriterEnum::ParquetFileWriter(w) => { + let batch = + IcebergArrowConvert.to_record_batch(self.schema.clone(), chunk.data_chunk())?; + let batch_row_nums = batch.num_rows(); + w.write(&batch).await?; + self.current_bached_row_num += batch_row_nums; + } + FileWriterEnum::FileWriter(w) => { + let mut chunk_buf = BytesMut::new(); + let batch_row_nums = chunk.data_chunk().capacity(); + // write the json representations of the row(s) in current chunk to `chunk_buf` + for (op, row) in chunk.rows() { + assert_eq!(op, Op::Insert, "expect all `op(s)` to be `Op::Insert`"); + // to prevent temporary string allocation, + // so we directly write to `chunk_buf` implicitly via `write_fmt`. + writeln!( + chunk_buf, + "{}", + Value::Object(self.row_encoder.encode(row)?) + ) + .unwrap(); // write to a `BytesMut` should never fail + } + w.write(chunk_buf.freeze()).await?; + self.current_bached_row_num += batch_row_nums; + } + } Ok(()) } } +/// Init methods. impl OpenDalSinkWriter { pub fn new( operator: Operator, write_path: &str, rw_schema: Schema, - is_append_only: bool, executor_id: u64, encode_type: SinkEncode, engine_type: EngineType, + batching_strategy: BatchingStrategy, ) -> Result { let arrow_schema = convert_rw_schema_to_arrow_schema(rw_schema.clone())?; let row_encoder = JsonEncoder::new( @@ -266,17 +361,18 @@ impl OpenDalSinkWriter { write_path: write_path.to_string(), operator, sink_writer: None, - is_append_only, - epoch: None, executor_id, encode_type, row_encoder, engine_type, + batching_strategy, + current_bached_row_num: 0, + created_time: SystemTime::now(), }) } - async fn create_object_writer(&mut self, epoch: u64) -> Result { + async fn create_object_writer(&mut self) -> Result { // Todo: specify more file suffixes based on encode_type. let suffix = match self.encode_type { SinkEncode::Parquet => "parquet", @@ -284,17 +380,35 @@ impl OpenDalSinkWriter { _ => unimplemented!(), }; - // Note: sink decoupling is not currently supported, which means that output files will not be batched across checkpoints. - // The current implementation writes files every time a checkpoint arrives, so the naming convention is `epoch + executor_id + .suffix`. + let create_time = self + .created_time + .duration_since(UNIX_EPOCH) + .expect("Time went backwards"); + + // With batching in place, the file writing process is decoupled from checkpoints. + // The current file naming convention is as follows: + // 1. A subdirectory is defined based on `path_partition_prefix` (e.g., by day态hour or month or none.). + // 2. The file name includes the `executor_id` and the creation time in seconds since the UNIX epoch. + // If the engine type is `Fs`, the path is automatically handled, and the filename does not include a path prefix. let object_name = match self.engine_type { // For the local fs sink, the data will be automatically written to the defined path. // Therefore, there is no need to specify the path in the file name. EngineType::Fs => { - format!("{}_{}.{}", epoch, self.executor_id, suffix,) + format!( + "{}{}_{}.{}", + self.path_partition_prefix(&create_time), + self.executor_id, + create_time.as_secs(), + suffix + ) } _ => format!( - "{}/{}_{}.{}", - self.write_path, epoch, self.executor_id, suffix, + "{}/{}{}_{}.{}", + self.write_path, + self.path_partition_prefix(&create_time), + self.executor_id, + create_time.as_secs(), + suffix, ), }; @@ -305,8 +419,8 @@ impl OpenDalSinkWriter { .await?) } - async fn create_sink_writer(&mut self, epoch: u64) -> Result<()> { - let object_writer = self.create_object_writer(epoch).await?; + async fn create_sink_writer(&mut self) -> Result<()> { + let object_writer = self.create_object_writer().await?; match self.encode_type { SinkEncode::Parquet => { let props = WriterProperties::builder(); @@ -324,39 +438,9 @@ impl OpenDalSinkWriter { self.sink_writer = Some(FileWriterEnum::FileWriter(object_writer)); } } + self.current_bached_row_num = 0; - Ok(()) - } - - async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { - match self - .sink_writer - .as_mut() - .ok_or_else(|| SinkError::File("Sink writer is not created.".to_string()))? - { - FileWriterEnum::ParquetFileWriter(w) => { - let batch = - IcebergArrowConvert.to_record_batch(self.schema.clone(), chunk.data_chunk())?; - w.write(&batch).await?; - } - FileWriterEnum::FileWriter(w) => { - let mut chunk_buf = BytesMut::new(); - - // write the json representations of the row(s) in current chunk to `chunk_buf` - for (op, row) in chunk.rows() { - assert_eq!(op, Op::Insert, "expect all `op(s)` to be `Op::Insert`"); - // to prevent temporary string allocation, - // so we directly write to `chunk_buf` implicitly via `write_fmt`. - writeln!( - chunk_buf, - "{}", - Value::Object(self.row_encoder.encode(row)?) - ) - .unwrap(); // write to a `BytesMut` should never fail - } - w.write(chunk_buf.freeze()).await?; - } - } + self.created_time = SystemTime::now(); Ok(()) } @@ -381,3 +465,48 @@ fn convert_rw_schema_to_arrow_schema( Ok(arrow_schema_iceberg::Schema::new(arrow_fields)) } + +/// `BatchingStrategy` represents the strategy for batching data before writing to files. +/// +/// This struct contains settings that control how data is collected and +/// partitioned based on specified criteria: +/// +/// - `max_row_count`: Optional maximum number of rows to accumulate before writing. +/// - `rollover_seconds`: Optional time interval (in seconds) to trigger a write, +/// regardless of the number of accumulated rows. +/// - `path_partition_prefix`: Specifies how files are organized into directories +/// based on creation time (e.g., by day, month, or hour). + +#[serde_as] +#[derive(Default, Deserialize, Debug, Clone, WithOptions)] +pub struct BatchingStrategy { + #[serde(default = "default_max_row_count")] + #[serde_as(as = "DisplayFromStr")] + pub max_row_count: usize, + #[serde(default = "default_rollover_seconds")] + #[serde_as(as = "DisplayFromStr")] + pub rollover_seconds: usize, + #[serde(default)] + #[serde_as(as = "Option")] + pub path_partition_prefix: Option, +} + +/// `PathPartitionPrefix` defines the granularity of file partitions based on creation time. +/// +/// Each variant specifies how files are organized into directories: +/// - `None`: No partitioning. +/// - `Day`: Files are written in a directory for each day. +/// - `Month`: Files are written in a directory for each month. +/// - `Hour`: Files are written in a directory for each hour. +#[derive(Default, Debug, Clone, PartialEq, Display, Deserialize, EnumString)] +#[strum(serialize_all = "snake_case")] +pub enum PathPartitionPrefix { + #[default] + None = 0, + #[serde(alias = "day")] + Day = 1, + #[serde(alias = "month")] + Month = 2, + #[serde(alias = "hour")] + Hour = 3, +} diff --git a/src/connector/src/sink/file_sink/s3.rs b/src/connector/src/sink/file_sink/s3.rs index 652057080e9e5..0e11f125097f1 100644 --- a/src/connector/src/sink/file_sink/s3.rs +++ b/src/connector/src/sink/file_sink/s3.rs @@ -21,7 +21,7 @@ use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use super::opendal_sink::FileSink; +use super::opendal_sink::{BatchingStrategy, FileSink}; use crate::sink::file_sink::opendal_sink::OpendalSinkBackend; use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; use crate::source::UnknownFields; @@ -50,6 +50,9 @@ pub struct S3Config { #[serde(flatten)] pub common: S3Common, + #[serde(flatten)] + pub batching_strategy: BatchingStrategy, + pub r#type: String, // accept "append-only" #[serde(flatten)] @@ -139,4 +142,12 @@ impl OpendalSinkBackend for S3Sink { fn get_engine_type() -> super::opendal_sink::EngineType { super::opendal_sink::EngineType::S3 } + + fn get_batching_strategy(properties: Self::Properties) -> BatchingStrategy { + BatchingStrategy { + max_row_count: properties.batching_strategy.max_row_count, + rollover_seconds: properties.batching_strategy.rollover_seconds, + path_partition_prefix: properties.batching_strategy.path_partition_prefix, + } + } } diff --git a/src/connector/src/sink/file_sink/webhdfs.rs b/src/connector/src/sink/file_sink/webhdfs.rs index 31ed904e9083c..fbef81dd107e5 100644 --- a/src/connector/src/sink/file_sink/webhdfs.rs +++ b/src/connector/src/sink/file_sink/webhdfs.rs @@ -21,7 +21,7 @@ use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use super::opendal_sink::FileSink; +use super::opendal_sink::{BatchingStrategy, FileSink}; use crate::sink::file_sink::opendal_sink::OpendalSinkBackend; use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; use crate::source::UnknownFields; @@ -40,6 +40,9 @@ pub struct WebhdfsConfig { #[serde(flatten)] pub common: WebhdfsCommon, + #[serde(flatten)] + pub batching_strategy: BatchingStrategy, + pub r#type: String, // accept "append-only" #[serde(flatten)] @@ -103,4 +106,12 @@ impl OpendalSinkBackend for WebhdfsSink { fn get_engine_type() -> super::opendal_sink::EngineType { super::opendal_sink::EngineType::Webhdfs } + + fn get_batching_strategy(properties: Self::Properties) -> BatchingStrategy { + BatchingStrategy { + max_row_count: properties.batching_strategy.max_row_count, + rollover_seconds: properties.batching_strategy.rollover_seconds, + path_partition_prefix: properties.batching_strategy.path_partition_prefix, + } + } } diff --git a/src/connector/src/with_options.rs b/src/connector/src/with_options.rs index 7476f5a10789c..37cca7ec09bed 100644 --- a/src/connector/src/with_options.rs +++ b/src/connector/src/with_options.rs @@ -67,6 +67,7 @@ impl WithOptions for i64 {} impl WithOptions for f64 {} impl WithOptions for std::time::Duration {} impl WithOptions for crate::connector_common::MqttQualityOfService {} +impl WithOptions for crate::sink::file_sink::opendal_sink::PathPartitionPrefix {} impl WithOptions for crate::sink::kafka::CompressionCodec {} impl WithOptions for crate::source::filesystem::file_common::CompressionFormat {} impl WithOptions for nexmark::config::RateShape {} diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index a92c3eaf844dd..eb03288dfbcca 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -21,6 +21,18 @@ AzblobConfig: - name: azblob.endpoint_url field_type: String required: true + - name: max_row_count + field_type: usize + required: false + default: DEFAULT_MAX_ROW_COUNR + - name: rollover_seconds + field_type: usize + required: false + default: DEFAULT_ROLLOVER_SECONDS + - name: path_partition_prefix + field_type: PathPartitionPrefix + required: false + default: Default::default - name: r#type field_type: String required: true @@ -279,6 +291,18 @@ FsConfig: field_type: String comments: The directory where the sink file is located. required: true + - name: max_row_count + field_type: usize + required: false + default: DEFAULT_MAX_ROW_COUNR + - name: rollover_seconds + field_type: usize + required: false + default: DEFAULT_ROLLOVER_SECONDS + - name: path_partition_prefix + field_type: PathPartitionPrefix + required: false + default: Default::default - name: r#type field_type: String required: true @@ -300,6 +324,18 @@ GcsConfig: field_type: String comments: The directory where the sink file is located required: true + - name: max_row_count + field_type: usize + required: false + default: DEFAULT_MAX_ROW_COUNR + - name: rollover_seconds + field_type: usize + required: false + default: DEFAULT_ROLLOVER_SECONDS + - name: path_partition_prefix + field_type: PathPartitionPrefix + required: false + default: Default::default - name: r#type field_type: String required: true @@ -963,6 +999,18 @@ S3Config: field_type: String required: false default: Default::default + - name: max_row_count + field_type: usize + required: false + default: DEFAULT_MAX_ROW_COUNR + - name: rollover_seconds + field_type: usize + required: false + default: DEFAULT_ROLLOVER_SECONDS + - name: path_partition_prefix + field_type: PathPartitionPrefix + required: false + default: Default::default - name: r#type field_type: String required: true @@ -1094,6 +1142,18 @@ WebhdfsConfig: field_type: String comments: The directory where the sink file is located. required: true + - name: max_row_count + field_type: usize + required: false + default: DEFAULT_MAX_ROW_COUNR + - name: rollover_seconds + field_type: usize + required: false + default: DEFAULT_ROLLOVER_SECONDS + - name: path_partition_prefix + field_type: PathPartitionPrefix + required: false + default: Default::default - name: r#type field_type: String required: true diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 69e2086b9dd3b..39cf84fe13705 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -259,7 +259,12 @@ impl StreamSink { ); } }; - + // For file sink, it must have sink_decouple turned on. + if !sink_decouple && sink.is_file_sink() { + return Err( + SinkError::Config(anyhow!("File sink can only be created with sink_decouple enabled. Please run `set sink_decouple = true` first.")).into(), + ); + } let log_store_type = if sink_decouple { SinkLogStoreType::KvLogStore } else { From c56294f16d30d9bdb0b8d4ea9901dc95319edf1d Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Thu, 17 Oct 2024 17:03:12 +0800 Subject: [PATCH 2/4] chore(ci): eliminate the progress output of wget (#18965) --- ci/scripts/backwards-compat-test.sh | 2 +- ci/scripts/connector-node-integration-test.sh | 4 ++-- ci/scripts/e2e-cassandra-sink-test.sh | 2 +- ci/scripts/e2e-deltalake-sink-rust-test.sh | 2 +- ci/scripts/e2e-iceberg-sink-test.sh | 2 +- ci/scripts/e2e-mongodb-sink-test.sh | 4 ++-- ci/scripts/e2e-source-test.sh | 4 ++-- ci/scripts/release.sh | 2 +- 8 files changed, 11 insertions(+), 11 deletions(-) diff --git a/ci/scripts/backwards-compat-test.sh b/ci/scripts/backwards-compat-test.sh index 7739273daa268..d6b64d91e6e39 100755 --- a/ci/scripts/backwards-compat-test.sh +++ b/ci/scripts/backwards-compat-test.sh @@ -96,7 +96,7 @@ setup_old_cluster() { echo "--- Get RisingWave binary for $OLD_VERSION" OLD_URL=https://github.com/risingwavelabs/risingwave/releases/download/v${OLD_VERSION}/risingwave-v${OLD_VERSION}-x86_64-unknown-linux.tar.gz set +e - wget "$OLD_URL" + wget --no-verbose "$OLD_URL" if [[ "$?" -ne 0 ]]; then set -e echo "Failed to download ${OLD_VERSION} from github releases, build from source later during \`risedev d\`" diff --git a/ci/scripts/connector-node-integration-test.sh b/ci/scripts/connector-node-integration-test.sh index eb568031b0095..2989fe9eb36af 100755 --- a/ci/scripts/connector-node-integration-test.sh +++ b/ci/scripts/connector-node-integration-test.sh @@ -55,12 +55,12 @@ sudo -u postgres psql -d test -c "CREATE TABLE test (id serial PRIMARY KEY, name echo "--- starting minio" echo "setting up minio" -wget https://dl.minio.io/server/minio/release/linux-amd64/minio > /dev/null +wget --no-verbose https://dl.minio.io/server/minio/release/linux-amd64/minio > /dev/null chmod +x minio sudo ./minio server /tmp/minio & # wait for minio to start sleep 3 -wget https://dl.minio.io/client/mc/release/linux-amd64/mc > /dev/null +wget --no-verbose https://dl.minio.io/client/mc/release/linux-amd64/mc > /dev/null chmod +x mc MC_PATH=${PWD}/mc ${MC_PATH} config host add minio http://127.0.0.1:9000 minioadmin minioadmin diff --git a/ci/scripts/e2e-cassandra-sink-test.sh b/ci/scripts/e2e-cassandra-sink-test.sh index 678b97aac4b54..4c46e48c42b38 100755 --- a/ci/scripts/e2e-cassandra-sink-test.sh +++ b/ci/scripts/e2e-cassandra-sink-test.sh @@ -37,7 +37,7 @@ risedev ci-start ci-sink-test sleep 40 echo "--- install cassandra" -wget $(get_latest_cassandra_download_url) -O cassandra_latest.tar.gz +wget --no-verbose $(get_latest_cassandra_download_url) -O cassandra_latest.tar.gz tar xfvz cassandra_latest.tar.gz export LATEST_CASSANDRA_VERSION=$(get_latest_cassandra_version) export CASSANDRA_DIR="./apache-cassandra-${LATEST_CASSANDRA_VERSION}" diff --git a/ci/scripts/e2e-deltalake-sink-rust-test.sh b/ci/scripts/e2e-deltalake-sink-rust-test.sh index 416d176c1e2d5..4863dd8c3f874 100755 --- a/ci/scripts/e2e-deltalake-sink-rust-test.sh +++ b/ci/scripts/e2e-deltalake-sink-rust-test.sh @@ -38,7 +38,7 @@ sleep 1 # prepare minio deltalake sink echo "--- preparing deltalake" .risingwave/bin/mcli -C .risingwave/config/mcli mb hummock-minio/deltalake -wget https://rw-ci-deps-dist.s3.amazonaws.com/spark-3.3.1-bin-hadoop3.tgz +wget --no-verbose https://rw-ci-deps-dist.s3.amazonaws.com/spark-3.3.1-bin-hadoop3.tgz tar -xf spark-3.3.1-bin-hadoop3.tgz --no-same-owner DEPENDENCIES=io.delta:delta-core_2.12:2.2.0,org.apache.hadoop:hadoop-aws:3.3.2 spark-3.3.1-bin-hadoop3/bin/spark-sql --packages $DEPENDENCIES \ diff --git a/ci/scripts/e2e-iceberg-sink-test.sh b/ci/scripts/e2e-iceberg-sink-test.sh index d22e576694304..66e35dcf2d994 100755 --- a/ci/scripts/e2e-iceberg-sink-test.sh +++ b/ci/scripts/e2e-iceberg-sink-test.sh @@ -39,7 +39,7 @@ sleep 1 # prepare minio iceberg sink echo "--- preparing iceberg" .risingwave/bin/mcli -C .risingwave/config/mcli mb hummock-minio/iceberg -wget https://rw-ci-deps-dist.s3.amazonaws.com/spark-3.3.1-bin-hadoop3.tgz +wget --no-verbose https://rw-ci-deps-dist.s3.amazonaws.com/spark-3.3.1-bin-hadoop3.tgz tar -xf spark-3.3.1-bin-hadoop3.tgz --no-same-owner DEPENDENCIES=org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.3.1,org.apache.hadoop:hadoop-aws:3.3.2 spark-3.3.1-bin-hadoop3/bin/spark-sql --packages $DEPENDENCIES \ diff --git a/ci/scripts/e2e-mongodb-sink-test.sh b/ci/scripts/e2e-mongodb-sink-test.sh index 6ec6e97cf0fe1..aea6ee0ae634f 100755 --- a/ci/scripts/e2e-mongodb-sink-test.sh +++ b/ci/scripts/e2e-mongodb-sink-test.sh @@ -28,8 +28,8 @@ cargo make ci-start ci-sink-test sleep 1 # install the mongo shell -wget http://archive.ubuntu.com/ubuntu/pool/main/o/openssl/libssl1.1_1.1.1f-1ubuntu2_amd64.deb -wget https://repo.mongodb.org/apt/ubuntu/dists/focal/mongodb-org/4.4/multiverse/binary-amd64/mongodb-org-shell_4.4.28_amd64.deb +wget --no-verbose http://archive.ubuntu.com/ubuntu/pool/main/o/openssl/libssl1.1_1.1.1f-1ubuntu2_amd64.deb +wget --no-verbose https://repo.mongodb.org/apt/ubuntu/dists/focal/mongodb-org/4.4/multiverse/binary-amd64/mongodb-org-shell_4.4.28_amd64.deb dpkg -i libssl1.1_1.1.1f-1ubuntu2_amd64.deb dpkg -i mongodb-org-shell_4.4.28_amd64.deb diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index b86de52c6014c..31ded0b59d4c8 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -72,8 +72,8 @@ sleep 2 echo "--- mongodb cdc test" # install the mongo shell -wget http://archive.ubuntu.com/ubuntu/pool/main/o/openssl/libssl1.1_1.1.1f-1ubuntu2_amd64.deb -wget https://repo.mongodb.org/apt/ubuntu/dists/focal/mongodb-org/4.4/multiverse/binary-amd64/mongodb-org-shell_4.4.28_amd64.deb +wget --no-verbose http://archive.ubuntu.com/ubuntu/pool/main/o/openssl/libssl1.1_1.1.1f-1ubuntu2_amd64.deb +wget --no-verbose https://repo.mongodb.org/apt/ubuntu/dists/focal/mongodb-org/4.4/multiverse/binary-amd64/mongodb-org-shell_4.4.28_amd64.deb dpkg -i libssl1.1_1.1.1f-1ubuntu2_amd64.deb dpkg -i mongodb-org-shell_4.4.28_amd64.deb diff --git a/ci/scripts/release.sh b/ci/scripts/release.sh index c5013314fc543..a887f1717b6dd 100755 --- a/ci/scripts/release.sh +++ b/ci/scripts/release.sh @@ -26,7 +26,7 @@ dnf install -y perl-core wget python3 python3-devel cyrus-sasl-devel rsync opens echo "--- Install java and maven" dnf install -y java-17-openjdk java-17-openjdk-devel pip3 install toml-cli -wget https://rw-ci-deps-dist.s3.amazonaws.com/apache-maven-3.9.3-bin.tar.gz && tar -zxvf apache-maven-3.9.3-bin.tar.gz +wget --no-verbose https://rw-ci-deps-dist.s3.amazonaws.com/apache-maven-3.9.3-bin.tar.gz && tar -zxvf apache-maven-3.9.3-bin.tar.gz export PATH="${REPO_ROOT}/apache-maven-3.9.3/bin:$PATH" mvn -v From 9a6a7f9052d5679165ff57cc01417c742c95351c Mon Sep 17 00:00:00 2001 From: August Date: Thu, 17 Oct 2024 17:30:34 +0800 Subject: [PATCH 3/4] chore: cleanup v2 naming for sql metastore (#18941) --- .licenserc.yaml | 2 +- Cargo.lock | 26 +- Cargo.toml | 8 +- src/ctl/Cargo.toml | 2 +- src/ctl/src/cmd_impl/meta/reschedule.rs | 2 +- src/meta/Cargo.toml | 2 +- src/meta/{model_v2 => model}/Cargo.toml | 2 +- .../{model_v2 => model}/migration/Cargo.toml | 0 .../{model_v2 => model}/migration/README.md | 0 .../{model_v2 => model}/migration/src/lib.rs | 0 .../migration/src/m20230908_072257_init.rs | 0 .../migration/src/m20231008_020431_hummock.rs | 0 .../src/m20240304_074901_subscription.rs | 0 ...10_082733_with_version_column_migration.rs | 0 .../src/m20240410_154406_session_params.rs | 0 ...062305_subscription_internal_table_name.rs | 0 .../src/m20240418_142249_function_runtime.rs | 0 ...240506_112555_subscription_partial_ckpt.rs | 0 .../migration/src/m20240525_090457_secret.rs | 0 ...20240617_070131_index_column_properties.rs | 0 ...m20240617_071625_sink_into_table_column.rs | 0 ...40618_072634_function_compressed_binary.rs | 0 .../m20240630_131430_remove_parallel_unit.rs | 0 .../m20240701_060504_hummock_time_travel.rs | 0 .../m20240702_080451_system_param_value.rs | 0 .../src/m20240702_084927_unnecessary_fk.rs | 0 .../m20240726_063833_auto_schema_change.rs | 0 ...143329_add_rate_limit_to_source_catalog.rs | 0 ..._081248_add_time_travel_per_table_epoch.rs | 0 .../m20240911_083152_variable_vnode_count.rs | 0 .../{model_v2 => model}/migration/src/main.rs | 0 src/meta/{model_v2 => model}/src/README.md | 18 +- src/meta/{model_v2 => model}/src/actor.rs | 0 .../src/actor_dispatcher.rs | 0 .../src/catalog_version.rs | 0 src/meta/{model_v2 => model}/src/cluster.rs | 0 .../src/compaction_config.rs | 0 .../src/compaction_status.rs | 0 .../src/compaction_task.rs | 0 .../{model_v2 => model}/src/connection.rs | 0 src/meta/{model_v2 => model}/src/database.rs | 0 src/meta/{model_v2 => model}/src/fragment.rs | 0 src/meta/{model_v2 => model}/src/function.rs | 0 .../src/hummock_epoch_to_version.rs | 0 .../src/hummock_pinned_snapshot.rs | 0 .../src/hummock_pinned_version.rs | 0 .../src/hummock_sequence.rs | 0 .../src/hummock_sstable_info.rs | 0 .../src/hummock_time_travel_delta.rs | 0 .../src/hummock_time_travel_version.rs | 0 .../src/hummock_version_delta.rs | 0 .../src/hummock_version_stats.rs | 0 src/meta/{model_v2 => model}/src/index.rs | 0 src/meta/{model_v2 => model}/src/lib.rs | 0 src/meta/{model_v2 => model}/src/object.rs | 0 .../src/object_dependency.rs | 0 src/meta/{model_v2 => model}/src/prelude.rs | 0 src/meta/{model_v2 => model}/src/schema.rs | 0 src/meta/{model_v2 => model}/src/secret.rs | 0 .../src/serde_seaql_migration.rs | 0 .../src/session_parameter.rs | 0 src/meta/{model_v2 => model}/src/sink.rs | 0 src/meta/{model_v2 => model}/src/source.rs | 0 .../{model_v2 => model}/src/streaming_job.rs | 0 .../{model_v2 => model}/src/subscription.rs | 0 .../src/system_parameter.rs | 0 src/meta/{model_v2 => model}/src/table.rs | 0 src/meta/{model_v2 => model}/src/user.rs | 0 .../{model_v2 => model}/src/user_privilege.rs | 0 src/meta/{model_v2 => model}/src/view.rs | 0 src/meta/{model_v2 => model}/src/worker.rs | 0 .../src/worker_property.rs | 0 src/meta/service/Cargo.toml | 2 +- src/meta/service/src/cloud_service.rs | 2 +- src/meta/service/src/cluster_limit_service.rs | 2 +- src/meta/service/src/cluster_service.rs | 2 +- src/meta/service/src/scale_service.rs | 2 +- src/meta/service/src/stream_service.rs | 2 +- src/meta/service/src/telemetry_service.rs | 2 +- src/meta/service/src/user_service.rs | 2 +- src/meta/src/backup_restore/backup_manager.rs | 4 +- ...builder_v2.rs => meta_snapshot_builder.rs} | 6 +- src/meta/src/backup_restore/mod.rs | 2 +- src/meta/src/barrier/command.rs | 4 +- .../barrier/creating_job/barrier_control.rs | 2 +- src/meta/src/barrier/creating_job/mod.rs | 2 +- src/meta/src/barrier/creating_job/status.rs | 2 +- src/meta/src/barrier/info.rs | 2 +- src/meta/src/barrier/mod.rs | 2 +- src/meta/src/barrier/progress.rs | 2 +- src/meta/src/barrier/recovery.rs | 2 +- src/meta/src/barrier/rpc.rs | 2 +- src/meta/src/controller/catalog.rs | 8 +- src/meta/src/controller/cluster.rs | 6 +- src/meta/src/controller/fragment.rs | 16 +- src/meta/src/controller/id.rs | 4 +- src/meta/src/controller/mod.rs | 2 +- src/meta/src/controller/scale.rs | 16 +- src/meta/src/controller/session_params.rs | 4 +- src/meta/src/controller/streaming_job.rs | 19 +- src/meta/src/controller/system_param.rs | 4 +- src/meta/src/controller/user.rs | 8 +- src/meta/src/controller/utils.rs | 12 +- src/meta/src/dashboard/mod.rs | 2 +- src/meta/src/error.rs | 2 +- src/meta/src/hummock/compactor_manager.rs | 2 +- .../compaction/compaction_group_manager.rs | 2 +- src/meta/src/hummock/manager/gc.rs | 4 +- src/meta/src/hummock/manager/mod.rs | 2 +- src/meta/src/hummock/manager/sequence.rs | 6 +- src/meta/src/hummock/manager/tests.rs | 2 +- src/meta/src/hummock/manager/time_travel.rs | 20 +- src/meta/src/hummock/model/ext/hummock.rs | 12 +- src/meta/src/hummock/test_utils.rs | 2 +- src/meta/src/manager/diagnose.rs | 2 +- src/meta/src/manager/env.rs | 2 +- src/meta/src/manager/metadata.rs | 2 +- src/meta/src/manager/mod.rs | 2 +- src/meta/src/manager/notification_version.rs | 6 +- src/meta/src/model/stream.rs | 2 +- src/meta/src/rpc/ddl_controller.rs | 563 ++++++++++++++++- src/meta/src/rpc/ddl_controller_v2.rs | 580 ------------------ src/meta/src/rpc/metrics.rs | 6 +- src/meta/src/rpc/mod.rs | 1 - src/meta/src/stream/scale.rs | 12 +- src/meta/src/stream/source_manager.rs | 6 +- src/meta/src/stream/stream_graph/actor.rs | 2 +- src/meta/src/stream/stream_graph/fragment.rs | 2 +- src/meta/src/stream/stream_graph/schedule.rs | 2 +- src/meta/src/stream/stream_manager.rs | 4 +- src/prost/build.rs | 2 +- src/storage/backup/Cargo.toml | 2 +- src/storage/backup/src/meta_snapshot_v2.rs | 58 +- 133 files changed, 745 insertions(+), 776 deletions(-) rename src/meta/{model_v2 => model}/Cargo.toml (94%) rename src/meta/{model_v2 => model}/migration/Cargo.toml (100%) rename src/meta/{model_v2 => model}/migration/README.md (100%) rename src/meta/{model_v2 => model}/migration/src/lib.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20230908_072257_init.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20231008_020431_hummock.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240304_074901_subscription.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240410_082733_with_version_column_migration.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240410_154406_session_params.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240417_062305_subscription_internal_table_name.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240418_142249_function_runtime.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240506_112555_subscription_partial_ckpt.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240525_090457_secret.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240617_070131_index_column_properties.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240617_071625_sink_into_table_column.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240618_072634_function_compressed_binary.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240630_131430_remove_parallel_unit.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240701_060504_hummock_time_travel.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240702_080451_system_param_value.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240702_084927_unnecessary_fk.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240726_063833_auto_schema_change.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240806_143329_add_rate_limit_to_source_catalog.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240820_081248_add_time_travel_per_table_epoch.rs (100%) rename src/meta/{model_v2 => model}/migration/src/m20240911_083152_variable_vnode_count.rs (100%) rename src/meta/{model_v2 => model}/migration/src/main.rs (100%) rename src/meta/{model_v2 => model}/src/README.md (71%) rename src/meta/{model_v2 => model}/src/actor.rs (100%) rename src/meta/{model_v2 => model}/src/actor_dispatcher.rs (100%) rename src/meta/{model_v2 => model}/src/catalog_version.rs (100%) rename src/meta/{model_v2 => model}/src/cluster.rs (100%) rename src/meta/{model_v2 => model}/src/compaction_config.rs (100%) rename src/meta/{model_v2 => model}/src/compaction_status.rs (100%) rename src/meta/{model_v2 => model}/src/compaction_task.rs (100%) rename src/meta/{model_v2 => model}/src/connection.rs (100%) rename src/meta/{model_v2 => model}/src/database.rs (100%) rename src/meta/{model_v2 => model}/src/fragment.rs (100%) rename src/meta/{model_v2 => model}/src/function.rs (100%) rename src/meta/{model_v2 => model}/src/hummock_epoch_to_version.rs (100%) rename src/meta/{model_v2 => model}/src/hummock_pinned_snapshot.rs (100%) rename src/meta/{model_v2 => model}/src/hummock_pinned_version.rs (100%) rename src/meta/{model_v2 => model}/src/hummock_sequence.rs (100%) rename src/meta/{model_v2 => model}/src/hummock_sstable_info.rs (100%) rename src/meta/{model_v2 => model}/src/hummock_time_travel_delta.rs (100%) rename src/meta/{model_v2 => model}/src/hummock_time_travel_version.rs (100%) rename src/meta/{model_v2 => model}/src/hummock_version_delta.rs (100%) rename src/meta/{model_v2 => model}/src/hummock_version_stats.rs (100%) rename src/meta/{model_v2 => model}/src/index.rs (100%) rename src/meta/{model_v2 => model}/src/lib.rs (100%) rename src/meta/{model_v2 => model}/src/object.rs (100%) rename src/meta/{model_v2 => model}/src/object_dependency.rs (100%) rename src/meta/{model_v2 => model}/src/prelude.rs (100%) rename src/meta/{model_v2 => model}/src/schema.rs (100%) rename src/meta/{model_v2 => model}/src/secret.rs (100%) rename src/meta/{model_v2 => model}/src/serde_seaql_migration.rs (100%) rename src/meta/{model_v2 => model}/src/session_parameter.rs (100%) rename src/meta/{model_v2 => model}/src/sink.rs (100%) rename src/meta/{model_v2 => model}/src/source.rs (100%) rename src/meta/{model_v2 => model}/src/streaming_job.rs (100%) rename src/meta/{model_v2 => model}/src/subscription.rs (100%) rename src/meta/{model_v2 => model}/src/system_parameter.rs (100%) rename src/meta/{model_v2 => model}/src/table.rs (100%) rename src/meta/{model_v2 => model}/src/user.rs (100%) rename src/meta/{model_v2 => model}/src/user_privilege.rs (100%) rename src/meta/{model_v2 => model}/src/view.rs (100%) rename src/meta/{model_v2 => model}/src/worker.rs (100%) rename src/meta/{model_v2 => model}/src/worker_property.rs (100%) rename src/meta/src/backup_restore/{meta_snapshot_builder_v2.rs => meta_snapshot_builder.rs} (95%) delete mode 100644 src/meta/src/rpc/ddl_controller_v2.rs diff --git a/.licenserc.yaml b/.licenserc.yaml index 93449f89340cb..6d63d62f092fb 100644 --- a/.licenserc.yaml +++ b/.licenserc.yaml @@ -18,7 +18,7 @@ header: - "src/sqlparser/**/*.rs" - "java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/*.java" - "java/connector-node/risingwave-sink-iceberg/src/main/java/org/apache/iceberg/**/*.java" - - "src/meta/model_v2/migration/**/*.rs" + - "src/meta/model/migration/**/*.rs" - "lints/ui/**" comment: on-failure diff --git a/Cargo.lock b/Cargo.lock index f803ba579ff4e..a56d01707a499 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10412,7 +10412,7 @@ dependencies = [ "prost 0.13.1", "risingwave_common", "risingwave_hummock_sdk", - "risingwave_meta_model_v2", + "risingwave_meta_model", "risingwave_object_store", "risingwave_pb", "serde", @@ -11097,8 +11097,8 @@ dependencies = [ "risingwave_frontend", "risingwave_hummock_sdk", "risingwave_meta", + "risingwave_meta_model", "risingwave_meta_model_migration", - "risingwave_meta_model_v2", "risingwave_object_store", "risingwave_pb", "risingwave_rpc_client", @@ -11578,8 +11578,8 @@ dependencies = [ "risingwave_hummock_sdk", "risingwave_license", "risingwave_meta_dashboard", + "risingwave_meta_model", "risingwave_meta_model_migration", - "risingwave_meta_model_v2", "risingwave_object_store", "risingwave_pb", "risingwave_rpc_client", @@ -11629,28 +11629,28 @@ dependencies = [ ] [[package]] -name = "risingwave_meta_model_migration" +name = "risingwave_meta_model" version = "2.2.0-alpha" dependencies = [ - "async-std", + "prost 0.13.1", + "risingwave_common", + "risingwave_hummock_sdk", + "risingwave_pb", "sea-orm", - "sea-orm-migration", "serde", "serde_json", - "uuid", ] [[package]] -name = "risingwave_meta_model_v2" +name = "risingwave_meta_model_migration" version = "2.2.0-alpha" dependencies = [ - "prost 0.13.1", - "risingwave_common", - "risingwave_hummock_sdk", - "risingwave_pb", + "async-std", "sea-orm", + "sea-orm-migration", "serde", "serde_json", + "uuid", ] [[package]] @@ -11704,7 +11704,7 @@ dependencies = [ "risingwave_connector", "risingwave_hummock_sdk", "risingwave_meta", - "risingwave_meta_model_v2", + "risingwave_meta_model", "risingwave_pb", "sea-orm", "serde_json", diff --git a/Cargo.toml b/Cargo.toml index 34d0567d3d07f..78f02d46d60c2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -29,8 +29,8 @@ members = [ "src/license", "src/meta", "src/meta/dashboard", - "src/meta/model_v2", - "src/meta/model_v2/migration", + "src/meta/model", + "src/meta/model/migration", "src/meta/node", "src/meta/service", "src/object_store", @@ -228,8 +228,8 @@ risingwave_mem_table_spill_test = { path = "./src/stream/spill_test" } risingwave_meta = { path = "./src/meta" } risingwave_meta_dashboard = { path = "./src/meta/dashboard" } risingwave_meta_service = { path = "./src/meta/service" } -risingwave_meta_model_migration = { path = "src/meta/model_v2/migration" } -risingwave_meta_model_v2 = { path = "./src/meta/model_v2" } +risingwave_meta_model = { path = "src/meta/model" } +risingwave_meta_model_migration = { path = "src/meta/model/migration" } risingwave_meta_node = { path = "./src/meta/node" } risingwave_object_store = { path = "./src/object_store" } risingwave_pb = { path = "./src/prost" } diff --git a/src/ctl/Cargo.toml b/src/ctl/Cargo.toml index d36a4fa2b93b7..7b0fc940de946 100644 --- a/src/ctl/Cargo.toml +++ b/src/ctl/Cargo.toml @@ -32,8 +32,8 @@ risingwave_connector = { workspace = true } risingwave_frontend = { workspace = true } risingwave_hummock_sdk = { workspace = true } risingwave_meta = { workspace = true } +risingwave_meta_model = { workspace = true } risingwave_meta_model_migration = { workspace = true } -risingwave_meta_model_v2 = { workspace = true } risingwave_object_store = { workspace = true } risingwave_pb = { workspace = true } risingwave_rpc_client = { workspace = true } diff --git a/src/ctl/src/cmd_impl/meta/reschedule.rs b/src/ctl/src/cmd_impl/meta/reschedule.rs index 2c46ceee8d6be..1b042b55d1fd1 100644 --- a/src/ctl/src/cmd_impl/meta/reschedule.rs +++ b/src/ctl/src/cmd_impl/meta/reschedule.rs @@ -19,7 +19,7 @@ use anyhow::{anyhow, Result}; use inquire::Confirm; use itertools::Itertools; use regex::Regex; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::common::WorkerNode; use risingwave_pb::meta::{GetClusterInfoResponse, PbWorkerReschedule}; use serde::{Deserialize, Serialize}; diff --git a/src/meta/Cargo.toml b/src/meta/Cargo.toml index 28a281ba5b191..4773a88802184 100644 --- a/src/meta/Cargo.toml +++ b/src/meta/Cargo.toml @@ -58,8 +58,8 @@ risingwave_connector = { workspace = true } risingwave_hummock_sdk = { workspace = true } risingwave_license = { workspace = true } risingwave_meta_dashboard = { workspace = true } +risingwave_meta_model = { workspace = true } risingwave_meta_model_migration = { workspace = true } -risingwave_meta_model_v2 = { workspace = true } risingwave_object_store = { workspace = true } risingwave_pb = { workspace = true } risingwave_rpc_client = { workspace = true } diff --git a/src/meta/model_v2/Cargo.toml b/src/meta/model/Cargo.toml similarity index 94% rename from src/meta/model_v2/Cargo.toml rename to src/meta/model/Cargo.toml index 942b6adffa070..991becc820642 100644 --- a/src/meta/model_v2/Cargo.toml +++ b/src/meta/model/Cargo.toml @@ -1,5 +1,5 @@ [package] -name = "risingwave_meta_model_v2" +name = "risingwave_meta_model" version = { workspace = true } edition = { workspace = true } homepage = { workspace = true } diff --git a/src/meta/model_v2/migration/Cargo.toml b/src/meta/model/migration/Cargo.toml similarity index 100% rename from src/meta/model_v2/migration/Cargo.toml rename to src/meta/model/migration/Cargo.toml diff --git a/src/meta/model_v2/migration/README.md b/src/meta/model/migration/README.md similarity index 100% rename from src/meta/model_v2/migration/README.md rename to src/meta/model/migration/README.md diff --git a/src/meta/model_v2/migration/src/lib.rs b/src/meta/model/migration/src/lib.rs similarity index 100% rename from src/meta/model_v2/migration/src/lib.rs rename to src/meta/model/migration/src/lib.rs diff --git a/src/meta/model_v2/migration/src/m20230908_072257_init.rs b/src/meta/model/migration/src/m20230908_072257_init.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20230908_072257_init.rs rename to src/meta/model/migration/src/m20230908_072257_init.rs diff --git a/src/meta/model_v2/migration/src/m20231008_020431_hummock.rs b/src/meta/model/migration/src/m20231008_020431_hummock.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20231008_020431_hummock.rs rename to src/meta/model/migration/src/m20231008_020431_hummock.rs diff --git a/src/meta/model_v2/migration/src/m20240304_074901_subscription.rs b/src/meta/model/migration/src/m20240304_074901_subscription.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240304_074901_subscription.rs rename to src/meta/model/migration/src/m20240304_074901_subscription.rs diff --git a/src/meta/model_v2/migration/src/m20240410_082733_with_version_column_migration.rs b/src/meta/model/migration/src/m20240410_082733_with_version_column_migration.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240410_082733_with_version_column_migration.rs rename to src/meta/model/migration/src/m20240410_082733_with_version_column_migration.rs diff --git a/src/meta/model_v2/migration/src/m20240410_154406_session_params.rs b/src/meta/model/migration/src/m20240410_154406_session_params.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240410_154406_session_params.rs rename to src/meta/model/migration/src/m20240410_154406_session_params.rs diff --git a/src/meta/model_v2/migration/src/m20240417_062305_subscription_internal_table_name.rs b/src/meta/model/migration/src/m20240417_062305_subscription_internal_table_name.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240417_062305_subscription_internal_table_name.rs rename to src/meta/model/migration/src/m20240417_062305_subscription_internal_table_name.rs diff --git a/src/meta/model_v2/migration/src/m20240418_142249_function_runtime.rs b/src/meta/model/migration/src/m20240418_142249_function_runtime.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240418_142249_function_runtime.rs rename to src/meta/model/migration/src/m20240418_142249_function_runtime.rs diff --git a/src/meta/model_v2/migration/src/m20240506_112555_subscription_partial_ckpt.rs b/src/meta/model/migration/src/m20240506_112555_subscription_partial_ckpt.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240506_112555_subscription_partial_ckpt.rs rename to src/meta/model/migration/src/m20240506_112555_subscription_partial_ckpt.rs diff --git a/src/meta/model_v2/migration/src/m20240525_090457_secret.rs b/src/meta/model/migration/src/m20240525_090457_secret.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240525_090457_secret.rs rename to src/meta/model/migration/src/m20240525_090457_secret.rs diff --git a/src/meta/model_v2/migration/src/m20240617_070131_index_column_properties.rs b/src/meta/model/migration/src/m20240617_070131_index_column_properties.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240617_070131_index_column_properties.rs rename to src/meta/model/migration/src/m20240617_070131_index_column_properties.rs diff --git a/src/meta/model_v2/migration/src/m20240617_071625_sink_into_table_column.rs b/src/meta/model/migration/src/m20240617_071625_sink_into_table_column.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240617_071625_sink_into_table_column.rs rename to src/meta/model/migration/src/m20240617_071625_sink_into_table_column.rs diff --git a/src/meta/model_v2/migration/src/m20240618_072634_function_compressed_binary.rs b/src/meta/model/migration/src/m20240618_072634_function_compressed_binary.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240618_072634_function_compressed_binary.rs rename to src/meta/model/migration/src/m20240618_072634_function_compressed_binary.rs diff --git a/src/meta/model_v2/migration/src/m20240630_131430_remove_parallel_unit.rs b/src/meta/model/migration/src/m20240630_131430_remove_parallel_unit.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240630_131430_remove_parallel_unit.rs rename to src/meta/model/migration/src/m20240630_131430_remove_parallel_unit.rs diff --git a/src/meta/model_v2/migration/src/m20240701_060504_hummock_time_travel.rs b/src/meta/model/migration/src/m20240701_060504_hummock_time_travel.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240701_060504_hummock_time_travel.rs rename to src/meta/model/migration/src/m20240701_060504_hummock_time_travel.rs diff --git a/src/meta/model_v2/migration/src/m20240702_080451_system_param_value.rs b/src/meta/model/migration/src/m20240702_080451_system_param_value.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240702_080451_system_param_value.rs rename to src/meta/model/migration/src/m20240702_080451_system_param_value.rs diff --git a/src/meta/model_v2/migration/src/m20240702_084927_unnecessary_fk.rs b/src/meta/model/migration/src/m20240702_084927_unnecessary_fk.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240702_084927_unnecessary_fk.rs rename to src/meta/model/migration/src/m20240702_084927_unnecessary_fk.rs diff --git a/src/meta/model_v2/migration/src/m20240726_063833_auto_schema_change.rs b/src/meta/model/migration/src/m20240726_063833_auto_schema_change.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240726_063833_auto_schema_change.rs rename to src/meta/model/migration/src/m20240726_063833_auto_schema_change.rs diff --git a/src/meta/model_v2/migration/src/m20240806_143329_add_rate_limit_to_source_catalog.rs b/src/meta/model/migration/src/m20240806_143329_add_rate_limit_to_source_catalog.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240806_143329_add_rate_limit_to_source_catalog.rs rename to src/meta/model/migration/src/m20240806_143329_add_rate_limit_to_source_catalog.rs diff --git a/src/meta/model_v2/migration/src/m20240820_081248_add_time_travel_per_table_epoch.rs b/src/meta/model/migration/src/m20240820_081248_add_time_travel_per_table_epoch.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240820_081248_add_time_travel_per_table_epoch.rs rename to src/meta/model/migration/src/m20240820_081248_add_time_travel_per_table_epoch.rs diff --git a/src/meta/model_v2/migration/src/m20240911_083152_variable_vnode_count.rs b/src/meta/model/migration/src/m20240911_083152_variable_vnode_count.rs similarity index 100% rename from src/meta/model_v2/migration/src/m20240911_083152_variable_vnode_count.rs rename to src/meta/model/migration/src/m20240911_083152_variable_vnode_count.rs diff --git a/src/meta/model_v2/migration/src/main.rs b/src/meta/model/migration/src/main.rs similarity index 100% rename from src/meta/model_v2/migration/src/main.rs rename to src/meta/model/migration/src/main.rs diff --git a/src/meta/model_v2/src/README.md b/src/meta/model/src/README.md similarity index 71% rename from src/meta/model_v2/src/README.md rename to src/meta/model/src/README.md index 48095d3e6d67f..539093816119c 100644 --- a/src/meta/model_v2/src/README.md +++ b/src/meta/model/src/README.md @@ -1,15 +1,18 @@ # How to define changes between versions and generate migration and model files -- Generate a new migration file and apply it to the database, check [migration](../migration/README.md) for more details. Let's take a local PG database as an example(`postgres://postgres:@localhost:5432/postgres`): +- Generate a new migration file and apply it to the database, check [migration](../migration/README.md) for more + details. Let's take a local PG database as an example(`postgres://postgres:@localhost:5432/postgres`): ```sh export DATABASE_URL=postgres://postgres:@localhost:5432/postgres; cargo run -- generate MIGRATION_NAME cargo run -- up ``` - - Define tables, indexes, foreign keys in the file. The new generated file will include a sample migration script, - you can replace it with your own migration scripts, like defining or changing tables, indexes, foreign keys and other - dml operation to do data correctness etc. Check [writing-migration](https://www.sea-ql.org/SeaORM/docs/migration/writing-migration/) - for more details. + - Define tables, indexes, foreign keys in the file. The new generated file will include a sample migration script, + you can replace it with your own migration scripts, like defining or changing tables, indexes, foreign keys and + other + dml operation to do data correctness etc. + Check [writing-migration](https://www.sea-ql.org/SeaORM/docs/migration/writing-migration/) + for more details. ```rust #[async_trait::async_trait] impl MigrationTrait for Migration { @@ -24,11 +27,12 @@ } } ``` -- Apply migration, and generate model files for new tables and indexes from the database, so you don't need to write them manually, +- Apply migration, and generate model files for new tables and indexes from the database, so you don't need to write + them manually, ```sh cargo run -- up sea-orm-cli generate entity -u postgres://postgres:@localhost:5432/postgres -s public -o {target_dir} - cp {target_dir}/xxx.rs src/meta/src/model_v2/ + cp {target_dir}/xxx.rs src/meta/src/model/ ``` - Defines enum and array types in the model files, since they're basically only supported in PG, and we need to define them in the model files manually. For example: diff --git a/src/meta/model_v2/src/actor.rs b/src/meta/model/src/actor.rs similarity index 100% rename from src/meta/model_v2/src/actor.rs rename to src/meta/model/src/actor.rs diff --git a/src/meta/model_v2/src/actor_dispatcher.rs b/src/meta/model/src/actor_dispatcher.rs similarity index 100% rename from src/meta/model_v2/src/actor_dispatcher.rs rename to src/meta/model/src/actor_dispatcher.rs diff --git a/src/meta/model_v2/src/catalog_version.rs b/src/meta/model/src/catalog_version.rs similarity index 100% rename from src/meta/model_v2/src/catalog_version.rs rename to src/meta/model/src/catalog_version.rs diff --git a/src/meta/model_v2/src/cluster.rs b/src/meta/model/src/cluster.rs similarity index 100% rename from src/meta/model_v2/src/cluster.rs rename to src/meta/model/src/cluster.rs diff --git a/src/meta/model_v2/src/compaction_config.rs b/src/meta/model/src/compaction_config.rs similarity index 100% rename from src/meta/model_v2/src/compaction_config.rs rename to src/meta/model/src/compaction_config.rs diff --git a/src/meta/model_v2/src/compaction_status.rs b/src/meta/model/src/compaction_status.rs similarity index 100% rename from src/meta/model_v2/src/compaction_status.rs rename to src/meta/model/src/compaction_status.rs diff --git a/src/meta/model_v2/src/compaction_task.rs b/src/meta/model/src/compaction_task.rs similarity index 100% rename from src/meta/model_v2/src/compaction_task.rs rename to src/meta/model/src/compaction_task.rs diff --git a/src/meta/model_v2/src/connection.rs b/src/meta/model/src/connection.rs similarity index 100% rename from src/meta/model_v2/src/connection.rs rename to src/meta/model/src/connection.rs diff --git a/src/meta/model_v2/src/database.rs b/src/meta/model/src/database.rs similarity index 100% rename from src/meta/model_v2/src/database.rs rename to src/meta/model/src/database.rs diff --git a/src/meta/model_v2/src/fragment.rs b/src/meta/model/src/fragment.rs similarity index 100% rename from src/meta/model_v2/src/fragment.rs rename to src/meta/model/src/fragment.rs diff --git a/src/meta/model_v2/src/function.rs b/src/meta/model/src/function.rs similarity index 100% rename from src/meta/model_v2/src/function.rs rename to src/meta/model/src/function.rs diff --git a/src/meta/model_v2/src/hummock_epoch_to_version.rs b/src/meta/model/src/hummock_epoch_to_version.rs similarity index 100% rename from src/meta/model_v2/src/hummock_epoch_to_version.rs rename to src/meta/model/src/hummock_epoch_to_version.rs diff --git a/src/meta/model_v2/src/hummock_pinned_snapshot.rs b/src/meta/model/src/hummock_pinned_snapshot.rs similarity index 100% rename from src/meta/model_v2/src/hummock_pinned_snapshot.rs rename to src/meta/model/src/hummock_pinned_snapshot.rs diff --git a/src/meta/model_v2/src/hummock_pinned_version.rs b/src/meta/model/src/hummock_pinned_version.rs similarity index 100% rename from src/meta/model_v2/src/hummock_pinned_version.rs rename to src/meta/model/src/hummock_pinned_version.rs diff --git a/src/meta/model_v2/src/hummock_sequence.rs b/src/meta/model/src/hummock_sequence.rs similarity index 100% rename from src/meta/model_v2/src/hummock_sequence.rs rename to src/meta/model/src/hummock_sequence.rs diff --git a/src/meta/model_v2/src/hummock_sstable_info.rs b/src/meta/model/src/hummock_sstable_info.rs similarity index 100% rename from src/meta/model_v2/src/hummock_sstable_info.rs rename to src/meta/model/src/hummock_sstable_info.rs diff --git a/src/meta/model_v2/src/hummock_time_travel_delta.rs b/src/meta/model/src/hummock_time_travel_delta.rs similarity index 100% rename from src/meta/model_v2/src/hummock_time_travel_delta.rs rename to src/meta/model/src/hummock_time_travel_delta.rs diff --git a/src/meta/model_v2/src/hummock_time_travel_version.rs b/src/meta/model/src/hummock_time_travel_version.rs similarity index 100% rename from src/meta/model_v2/src/hummock_time_travel_version.rs rename to src/meta/model/src/hummock_time_travel_version.rs diff --git a/src/meta/model_v2/src/hummock_version_delta.rs b/src/meta/model/src/hummock_version_delta.rs similarity index 100% rename from src/meta/model_v2/src/hummock_version_delta.rs rename to src/meta/model/src/hummock_version_delta.rs diff --git a/src/meta/model_v2/src/hummock_version_stats.rs b/src/meta/model/src/hummock_version_stats.rs similarity index 100% rename from src/meta/model_v2/src/hummock_version_stats.rs rename to src/meta/model/src/hummock_version_stats.rs diff --git a/src/meta/model_v2/src/index.rs b/src/meta/model/src/index.rs similarity index 100% rename from src/meta/model_v2/src/index.rs rename to src/meta/model/src/index.rs diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model/src/lib.rs similarity index 100% rename from src/meta/model_v2/src/lib.rs rename to src/meta/model/src/lib.rs diff --git a/src/meta/model_v2/src/object.rs b/src/meta/model/src/object.rs similarity index 100% rename from src/meta/model_v2/src/object.rs rename to src/meta/model/src/object.rs diff --git a/src/meta/model_v2/src/object_dependency.rs b/src/meta/model/src/object_dependency.rs similarity index 100% rename from src/meta/model_v2/src/object_dependency.rs rename to src/meta/model/src/object_dependency.rs diff --git a/src/meta/model_v2/src/prelude.rs b/src/meta/model/src/prelude.rs similarity index 100% rename from src/meta/model_v2/src/prelude.rs rename to src/meta/model/src/prelude.rs diff --git a/src/meta/model_v2/src/schema.rs b/src/meta/model/src/schema.rs similarity index 100% rename from src/meta/model_v2/src/schema.rs rename to src/meta/model/src/schema.rs diff --git a/src/meta/model_v2/src/secret.rs b/src/meta/model/src/secret.rs similarity index 100% rename from src/meta/model_v2/src/secret.rs rename to src/meta/model/src/secret.rs diff --git a/src/meta/model_v2/src/serde_seaql_migration.rs b/src/meta/model/src/serde_seaql_migration.rs similarity index 100% rename from src/meta/model_v2/src/serde_seaql_migration.rs rename to src/meta/model/src/serde_seaql_migration.rs diff --git a/src/meta/model_v2/src/session_parameter.rs b/src/meta/model/src/session_parameter.rs similarity index 100% rename from src/meta/model_v2/src/session_parameter.rs rename to src/meta/model/src/session_parameter.rs diff --git a/src/meta/model_v2/src/sink.rs b/src/meta/model/src/sink.rs similarity index 100% rename from src/meta/model_v2/src/sink.rs rename to src/meta/model/src/sink.rs diff --git a/src/meta/model_v2/src/source.rs b/src/meta/model/src/source.rs similarity index 100% rename from src/meta/model_v2/src/source.rs rename to src/meta/model/src/source.rs diff --git a/src/meta/model_v2/src/streaming_job.rs b/src/meta/model/src/streaming_job.rs similarity index 100% rename from src/meta/model_v2/src/streaming_job.rs rename to src/meta/model/src/streaming_job.rs diff --git a/src/meta/model_v2/src/subscription.rs b/src/meta/model/src/subscription.rs similarity index 100% rename from src/meta/model_v2/src/subscription.rs rename to src/meta/model/src/subscription.rs diff --git a/src/meta/model_v2/src/system_parameter.rs b/src/meta/model/src/system_parameter.rs similarity index 100% rename from src/meta/model_v2/src/system_parameter.rs rename to src/meta/model/src/system_parameter.rs diff --git a/src/meta/model_v2/src/table.rs b/src/meta/model/src/table.rs similarity index 100% rename from src/meta/model_v2/src/table.rs rename to src/meta/model/src/table.rs diff --git a/src/meta/model_v2/src/user.rs b/src/meta/model/src/user.rs similarity index 100% rename from src/meta/model_v2/src/user.rs rename to src/meta/model/src/user.rs diff --git a/src/meta/model_v2/src/user_privilege.rs b/src/meta/model/src/user_privilege.rs similarity index 100% rename from src/meta/model_v2/src/user_privilege.rs rename to src/meta/model/src/user_privilege.rs diff --git a/src/meta/model_v2/src/view.rs b/src/meta/model/src/view.rs similarity index 100% rename from src/meta/model_v2/src/view.rs rename to src/meta/model/src/view.rs diff --git a/src/meta/model_v2/src/worker.rs b/src/meta/model/src/worker.rs similarity index 100% rename from src/meta/model_v2/src/worker.rs rename to src/meta/model/src/worker.rs diff --git a/src/meta/model_v2/src/worker_property.rs b/src/meta/model/src/worker_property.rs similarity index 100% rename from src/meta/model_v2/src/worker_property.rs rename to src/meta/model/src/worker_property.rs diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index 69986f8570234..53c3708da0e12 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -26,7 +26,7 @@ risingwave_common = { workspace = true } risingwave_connector = { workspace = true } risingwave_hummock_sdk = { workspace = true } risingwave_meta = { workspace = true } -risingwave_meta_model_v2 = { workspace = true } +risingwave_meta_model = { workspace = true } risingwave_pb = { workspace = true } sea-orm = { workspace = true } serde_json = "1" diff --git a/src/meta/service/src/cloud_service.rs b/src/meta/service/src/cloud_service.rs index c609d202b0d66..e913b91826b6f 100644 --- a/src/meta/service/src/cloud_service.rs +++ b/src/meta/service/src/cloud_service.rs @@ -24,7 +24,7 @@ use risingwave_connector::source::{ }; use risingwave_connector::{dispatch_source_prop, WithOptionsSecResolved}; use risingwave_meta::manager::MetadataManager; -use risingwave_meta_model_v2::ConnectionId; +use risingwave_meta_model::ConnectionId; use risingwave_pb::catalog::connection::Info::PrivateLinkService; use risingwave_pb::cloud_service::cloud_service_server::CloudService; use risingwave_pb::cloud_service::rw_cloud_validate_source_response::{Error, ErrorType}; diff --git a/src/meta/service/src/cluster_limit_service.rs b/src/meta/service/src/cluster_limit_service.rs index 20a1ed6d6ba53..83aae536e7e56 100644 --- a/src/meta/service/src/cluster_limit_service.rs +++ b/src/meta/service/src/cluster_limit_service.rs @@ -19,7 +19,7 @@ use risingwave_common::util::cluster_limit::{ }; use risingwave_meta::manager::{MetaSrvEnv, MetadataManager}; use risingwave_meta::MetaResult; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::common::worker_node::State; use risingwave_pb::common::WorkerType; use risingwave_pb::meta::cluster_limit_service_server::ClusterLimitService; diff --git a/src/meta/service/src/cluster_service.rs b/src/meta/service/src/cluster_service.rs index 5d008af75a6df..e346ebd446ea9 100644 --- a/src/meta/service/src/cluster_service.rs +++ b/src/meta/service/src/cluster_service.rs @@ -14,7 +14,7 @@ use risingwave_meta::barrier::BarrierManagerRef; use risingwave_meta::manager::MetadataManager; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::common::worker_node::State; use risingwave_pb::common::HostAddress; use risingwave_pb::meta::cluster_service_server::ClusterService; diff --git a/src/meta/service/src/scale_service.rs b/src/meta/service/src/scale_service.rs index a40dd259b102f..889df5b33904c 100644 --- a/src/meta/service/src/scale_service.rs +++ b/src/meta/service/src/scale_service.rs @@ -16,7 +16,7 @@ use risingwave_common::catalog::TableId; use risingwave_meta::manager::MetadataManager; use risingwave_meta::model::TableParallelism; use risingwave_meta::stream::{RescheduleOptions, ScaleControllerRef, WorkerReschedule}; -use risingwave_meta_model_v2::FragmentId; +use risingwave_meta_model::FragmentId; use risingwave_pb::common::WorkerType; use risingwave_pb::meta::scale_service_server::ScaleService; use risingwave_pb::meta::{ diff --git a/src/meta/service/src/stream_service.rs b/src/meta/service/src/stream_service.rs index 283d000f24c70..19fafb5de74f4 100644 --- a/src/meta/service/src/stream_service.rs +++ b/src/meta/service/src/stream_service.rs @@ -21,7 +21,7 @@ use risingwave_meta::manager::{LocalNotification, MetadataManager}; use risingwave_meta::model; use risingwave_meta::model::ActorId; use risingwave_meta::stream::{SourceManagerRunningInfo, ThrottleConfig}; -use risingwave_meta_model_v2::{SourceId, StreamingParallelism}; +use risingwave_meta_model::{SourceId, StreamingParallelism}; use risingwave_pb::meta::cancel_creating_jobs_request::Jobs; use risingwave_pb::meta::list_actor_splits_response::FragmentType; use risingwave_pb::meta::list_table_fragments_response::{ diff --git a/src/meta/service/src/telemetry_service.rs b/src/meta/service/src/telemetry_service.rs index 32cd653e830af..be76bc05bcf04 100644 --- a/src/meta/service/src/telemetry_service.rs +++ b/src/meta/service/src/telemetry_service.rs @@ -13,7 +13,7 @@ // limitations under the License. use risingwave_meta::controller::SqlMetaStore; -use risingwave_meta_model_v2::prelude::Cluster; +use risingwave_meta_model::prelude::Cluster; use risingwave_pb::meta::telemetry_info_service_server::TelemetryInfoService; use risingwave_pb::meta::{GetTelemetryInfoRequest, TelemetryInfoResponse}; use sea_orm::EntityTrait; diff --git a/src/meta/service/src/user_service.rs b/src/meta/service/src/user_service.rs index 8e9571aea62ea..319bbbebbd04a 100644 --- a/src/meta/service/src/user_service.rs +++ b/src/meta/service/src/user_service.rs @@ -14,7 +14,7 @@ use itertools::Itertools; use risingwave_meta::manager::MetadataManager; -use risingwave_meta_model_v2::UserId; +use risingwave_meta_model::UserId; use risingwave_pb::user::grant_privilege::Object; use risingwave_pb::user::update_user_request::UpdateField; use risingwave_pb::user::user_service_server::UserService; diff --git a/src/meta/src/backup_restore/backup_manager.rs b/src/meta/src/backup_restore/backup_manager.rs index 0cdf660544c89..414c97bab1fd7 100644 --- a/src/meta/src/backup_restore/backup_manager.rs +++ b/src/meta/src/backup_restore/backup_manager.rs @@ -31,7 +31,7 @@ use risingwave_pb::meta::subscribe_response::{Info, Operation}; use thiserror_ext::AsReport; use tokio::task::JoinHandle; -use crate::backup_restore::meta_snapshot_builder_v2; +use crate::backup_restore::meta_snapshot_builder; use crate::backup_restore::metrics::BackupManagerMetrics; use crate::hummock::sequence::next_meta_backup_id; use crate::hummock::{HummockManagerRef, HummockVersionSafePoint}; @@ -353,7 +353,7 @@ impl BackupWorker { }; let meta_store = backup_manager_clone.env.meta_store(); let mut snapshot_builder = - meta_snapshot_builder_v2::MetaSnapshotV2Builder::new(meta_store); + meta_snapshot_builder::MetaSnapshotV2Builder::new(meta_store); // Reuse job id as snapshot id. snapshot_builder .build(job_id, hummock_version_builder) diff --git a/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs b/src/meta/src/backup_restore/meta_snapshot_builder.rs similarity index 95% rename from src/meta/src/backup_restore/meta_snapshot_builder_v2.rs rename to src/meta/src/backup_restore/meta_snapshot_builder.rs index 9e4ad6a0c05a1..ab29020f2de51 100644 --- a/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs +++ b/src/meta/src/backup_restore/meta_snapshot_builder.rs @@ -19,7 +19,7 @@ use risingwave_backup::error::{BackupError, BackupResult}; use risingwave_backup::meta_snapshot_v2::{MetaSnapshotV2, MetadataV2}; use risingwave_backup::MetaSnapshotId; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; -use risingwave_meta_model_v2 as model_v2; +use risingwave_meta_model as model; use risingwave_pb::hummock::PbHummockVersionDelta; use sea_orm::{DbErr, EntityTrait, QueryOrder, TransactionTrait}; @@ -83,8 +83,8 @@ impl MetaSnapshotV2Builder { ) .await .map_err(map_db_err)?; - let version_deltas = model_v2::prelude::HummockVersionDelta::find() - .order_by_asc(model_v2::hummock_version_delta::Column::Id) + let version_deltas = model::prelude::HummockVersionDelta::find() + .order_by_asc(model::hummock_version_delta::Column::Id) .all(&txn) .await .map_err(map_db_err)? diff --git a/src/meta/src/backup_restore/mod.rs b/src/meta/src/backup_restore/mod.rs index dad089dbc8551..c493d98d06fdf 100644 --- a/src/meta/src/backup_restore/mod.rs +++ b/src/meta/src/backup_restore/mod.rs @@ -15,7 +15,7 @@ mod backup_manager; pub use backup_manager::*; mod error; -mod meta_snapshot_builder_v2; +mod meta_snapshot_builder; mod metrics; mod restore; mod restore_impl; diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 6190350627c3c..a5b7efe63098d 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -22,7 +22,7 @@ use risingwave_common::hash::ActorMapping; use risingwave_common::types::Timestamptz; use risingwave_common::util::epoch::Epoch; use risingwave_connector::source::SplitImpl; -use risingwave_meta_model_v2::{ObjectId, WorkerId}; +use risingwave_meta_model::{ObjectId, WorkerId}; use risingwave_pb::catalog::{CreateType, Table}; use risingwave_pb::common::PbWorkerNode; use risingwave_pb::meta::table_fragments::PbActorStatus; @@ -1116,7 +1116,7 @@ impl CommandContext { // Apply the split changes in source manager. self.barrier_manager_context .source_manager - .drop_source_fragments(std::slice::from_ref(old_table_fragments)) + .drop_source_fragments_vec(std::slice::from_ref(old_table_fragments)) .await; let source_fragments = new_table_fragments.stream_source_fragments(); // XXX: is it possible to have backfill fragments here? diff --git a/src/meta/src/barrier/creating_job/barrier_control.rs b/src/meta/src/barrier/creating_job/barrier_control.rs index b0aca04645003..e1d965c969372 100644 --- a/src/meta/src/barrier/creating_job/barrier_control.rs +++ b/src/meta/src/barrier/creating_job/barrier_control.rs @@ -20,7 +20,7 @@ use std::time::Instant; use prometheus::HistogramTimer; use risingwave_common::catalog::TableId; use risingwave_common::metrics::{LabelGuardedHistogram, LabelGuardedIntGauge}; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::stream_service::BarrierCompleteResponse; use tracing::debug; diff --git a/src/meta/src/barrier/creating_job/mod.rs b/src/meta/src/barrier/creating_job/mod.rs index 08b82148676f8..42e416d737881 100644 --- a/src/meta/src/barrier/creating_job/mod.rs +++ b/src/meta/src/barrier/creating_job/mod.rs @@ -24,7 +24,7 @@ use std::time::Duration; use prometheus::HistogramTimer; use risingwave_common::metrics::{LabelGuardedHistogram, LabelGuardedIntGauge}; use risingwave_common::util::epoch::Epoch; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::common::WorkerNode; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; diff --git a/src/meta/src/barrier/creating_job/status.rs b/src/meta/src/barrier/creating_job/status.rs index aaae86d0a2144..bfc5b4bbe7faf 100644 --- a/src/meta/src/barrier/creating_job/status.rs +++ b/src/meta/src/barrier/creating_job/status.rs @@ -17,7 +17,7 @@ use std::mem::take; use std::sync::Arc; use risingwave_common::util::epoch::Epoch; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::StreamActor; diff --git a/src/meta/src/barrier/info.rs b/src/meta/src/barrier/info.rs index 2fdfd40b091e6..b02240f402ba0 100644 --- a/src/meta/src/barrier/info.rs +++ b/src/meta/src/barrier/info.rs @@ -15,7 +15,7 @@ use std::collections::{HashMap, HashSet}; use risingwave_common::catalog::TableId; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::common::WorkerNode; use tracing::warn; diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index dfe51da9a64ac..119aed6f0158d 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -36,7 +36,7 @@ use risingwave_hummock_sdk::table_watermark::{ merge_multiple_new_table_watermarks, TableWatermarks, }; use risingwave_hummock_sdk::{HummockSstableObjectId, LocalSstableInfo}; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::meta::{PausedReason, PbRecoveryStatus}; diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 5f00ea489a279..e28d41ef09b5e 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -18,7 +18,7 @@ use std::mem::take; use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::Epoch; -use risingwave_meta_model_v2::ObjectId; +use risingwave_meta_model::ObjectId; use risingwave_pb::catalog::CreateType; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index fcfe914fbda52..498950a98014f 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -21,7 +21,7 @@ use risingwave_common::catalog::TableId; use risingwave_common::config::DefaultParallelism; use risingwave_common::hash::WorkerSlotId; use risingwave_common::util::epoch::Epoch; -use risingwave_meta_model_v2::{StreamingParallelism, WorkerId}; +use risingwave_meta_model::{StreamingParallelism, WorkerId}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::{PausedReason, Recovery}; use risingwave_pb::stream_plan::barrier_mutation::Mutation; diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index 320e09dc5eb1e..6e608489177be 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -26,7 +26,7 @@ use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::util::tracing::TracingContext; use risingwave_hummock_sdk::HummockVersionId; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::common::{ActorInfo, WorkerNode}; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::{Barrier, BarrierMutation, StreamActor, SubscriptionUpstreamInfo}; diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index e90bfb3b12700..08824459e916e 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -25,10 +25,10 @@ use risingwave_common::util::stream_graph_visitor::visit_stream_node_cont_mut; use risingwave_common::{bail, current_cluster_version}; use risingwave_connector::source::cdc::build_cdc_table_id; use risingwave_connector::source::UPSTREAM_SOURCE_KEY; -use risingwave_meta_model_v2::object::ObjectType; -use risingwave_meta_model_v2::prelude::*; -use risingwave_meta_model_v2::table::TableType; -use risingwave_meta_model_v2::{ +use risingwave_meta_model::object::ObjectType; +use risingwave_meta_model::prelude::*; +use risingwave_meta_model::table::TableType; +use risingwave_meta_model::{ actor, connection, database, fragment, function, index, object, object_dependency, schema, secret, sink, source, streaming_job, subscription, table, user_privilege, view, ActorId, ActorUpstreamActors, ColumnCatalogArray, ConnectionId, CreateType, DatabaseId, FragmentId, diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index 52de9c812c32d..8d58ba8fd6c1b 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -26,9 +26,9 @@ use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use risingwave_common::RW_VERSION; use risingwave_license::LicenseManager; -use risingwave_meta_model_v2::prelude::{Worker, WorkerProperty}; -use risingwave_meta_model_v2::worker::{WorkerStatus, WorkerType}; -use risingwave_meta_model_v2::{worker, worker_property, TransactionId, WorkerId}; +use risingwave_meta_model::prelude::{Worker, WorkerProperty}; +use risingwave_meta_model::worker::{WorkerStatus, WorkerType}; +use risingwave_meta_model::{worker, worker_property, TransactionId, WorkerId}; use risingwave_pb::common::worker_node::{PbProperty, PbResource, PbState}; use risingwave_pb::common::{HostAddress, PbHostAddress, PbWorkerNode, PbWorkerType, WorkerNode}; use risingwave_pb::meta::add_worker_node_request::Property as AddNodeProperty; diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 18fe9365bdde4..e118ad3c9683c 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -22,15 +22,15 @@ use risingwave_common::bail; use risingwave_common::hash::{VnodeCountCompat, WorkerSlotId}; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_common::util::worker_util::WorkerNodeId; -use risingwave_meta_model_migration::{Alias, SelectStatement}; -use risingwave_meta_model_v2::actor::ActorStatus; -use risingwave_meta_model_v2::fragment::DistributionType; -use risingwave_meta_model_v2::prelude::{Actor, ActorDispatcher, Fragment, Sink, StreamingJob}; -use risingwave_meta_model_v2::{ +use risingwave_meta_model::actor::ActorStatus; +use risingwave_meta_model::fragment::DistributionType; +use risingwave_meta_model::prelude::{Actor, ActorDispatcher, Fragment, Sink, StreamingJob}; +use risingwave_meta_model::{ actor, actor_dispatcher, fragment, sink, streaming_job, ActorId, ActorUpstreamActors, ConnectorSplits, ExprContext, FragmentId, I32Array, JobStatus, ObjectId, SinkId, SourceId, StreamNode, StreamingParallelism, TableId, VnodeBitmap, WorkerId, }; +use risingwave_meta_model_migration::{Alias, SelectStatement}; use risingwave_pb::common::PbActorLocation; use risingwave_pb::meta::subscribe_response::{ Info as NotificationInfo, Operation as NotificationOperation, @@ -1500,9 +1500,9 @@ mod tests { use risingwave_common::hash::{ActorMapping, VirtualNode}; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_common::util::stream_graph_visitor::visit_stream_node; - use risingwave_meta_model_v2::actor::ActorStatus; - use risingwave_meta_model_v2::fragment::DistributionType; - use risingwave_meta_model_v2::{ + use risingwave_meta_model::actor::ActorStatus; + use risingwave_meta_model::fragment::DistributionType; + use risingwave_meta_model::{ actor, actor_dispatcher, fragment, ActorId, ActorUpstreamActors, ConnectorSplits, ExprContext, FragmentId, I32Array, ObjectId, StreamNode, TableId, VnodeBitmap, }; diff --git a/src/meta/src/controller/id.rs b/src/meta/src/controller/id.rs index 34cc51b326087..5318512afe6c2 100644 --- a/src/meta/src/controller/id.rs +++ b/src/meta/src/controller/id.rs @@ -15,8 +15,8 @@ use std::sync::atomic::{AtomicU64, Ordering}; use std::sync::Arc; -use risingwave_meta_model_v2::prelude::{Actor, Fragment}; -use risingwave_meta_model_v2::{actor, fragment}; +use risingwave_meta_model::prelude::{Actor, Fragment}; +use risingwave_meta_model::{actor, fragment}; use sea_orm::sea_query::{Expr, Func}; use sea_orm::{DatabaseConnection, EntityTrait, QuerySelect}; diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 4c4588711bd76..cd0bb8530acbb 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -16,7 +16,7 @@ use std::collections::BTreeMap; use anyhow::anyhow; use risingwave_common::util::epoch::Epoch; -use risingwave_meta_model_v2::{ +use risingwave_meta_model::{ connection, database, function, index, object, schema, secret, sink, source, subscription, table, view, }; diff --git a/src/meta/src/controller/scale.rs b/src/meta/src/controller/scale.rs index 4c7fcf482cbda..65dd58ff1d34e 100644 --- a/src/meta/src/controller/scale.rs +++ b/src/meta/src/controller/scale.rs @@ -19,18 +19,18 @@ use itertools::Itertools; use risingwave_common::bitmap::Bitmap; use risingwave_common::hash; use risingwave_connector::source::{SplitImpl, SplitMetaData}; +use risingwave_meta_model::actor::ActorStatus; +use risingwave_meta_model::actor_dispatcher::DispatcherType; +use risingwave_meta_model::fragment::DistributionType; +use risingwave_meta_model::prelude::{Actor, ActorDispatcher, Fragment, StreamingJob}; +use risingwave_meta_model::{ + actor, actor_dispatcher, fragment, streaming_job, ActorId, ActorMapping, ActorUpstreamActors, + ConnectorSplits, FragmentId, I32Array, ObjectId, VnodeBitmap, +}; use risingwave_meta_model_migration::{ Alias, CommonTableExpression, Expr, IntoColumnRef, QueryStatementBuilder, SelectStatement, UnionType, WithClause, WithQuery, }; -use risingwave_meta_model_v2::actor::ActorStatus; -use risingwave_meta_model_v2::actor_dispatcher::DispatcherType; -use risingwave_meta_model_v2::fragment::DistributionType; -use risingwave_meta_model_v2::prelude::{Actor, ActorDispatcher, Fragment, StreamingJob}; -use risingwave_meta_model_v2::{ - actor, actor_dispatcher, fragment, streaming_job, ActorId, ActorMapping, ActorUpstreamActors, - ConnectorSplits, FragmentId, I32Array, ObjectId, VnodeBitmap, -}; use sea_orm::{ ColumnTrait, ConnectionTrait, DbErr, DerivePartialModel, EntityTrait, FromQueryResult, JoinType, QueryFilter, QuerySelect, RelationTrait, Statement, TransactionTrait, diff --git a/src/meta/src/controller/session_params.rs b/src/meta/src/controller/session_params.rs index 81ad467d573ef..ad3b93f12bd05 100644 --- a/src/meta/src/controller/session_params.rs +++ b/src/meta/src/controller/session_params.rs @@ -16,8 +16,8 @@ use std::sync::Arc; use itertools::Itertools; use risingwave_common::session_config::{SessionConfig, SessionConfigError}; -use risingwave_meta_model_v2::prelude::SessionParameter; -use risingwave_meta_model_v2::session_parameter; +use risingwave_meta_model::prelude::SessionParameter; +use risingwave_meta_model::session_parameter; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::SetSessionParamRequest; use sea_orm::ActiveValue::Set; diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index 7d742f5aa64fc..e8adc309d10e2 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -19,15 +19,15 @@ use itertools::Itertools; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_common::{bail, current_cluster_version}; -use risingwave_meta_model_v2::actor::ActorStatus; -use risingwave_meta_model_v2::actor_dispatcher::DispatcherType; -use risingwave_meta_model_v2::object::ObjectType; -use risingwave_meta_model_v2::prelude::{ +use risingwave_meta_model::actor::ActorStatus; +use risingwave_meta_model::actor_dispatcher::DispatcherType; +use risingwave_meta_model::object::ObjectType; +use risingwave_meta_model::prelude::{ Actor, ActorDispatcher, Fragment, Index, Object, ObjectDependency, Sink, Source, StreamingJob as StreamingJobModel, Table, }; -use risingwave_meta_model_v2::table::TableType; -use risingwave_meta_model_v2::{ +use risingwave_meta_model::table::TableType; +use risingwave_meta_model::{ actor, actor_dispatcher, fragment, index, object, object_dependency, sink, source, streaming_job, table, ActorId, ActorUpstreamActors, ColumnCatalogArray, CreateType, DatabaseId, ExprNodeArray, FragmentId, I32Array, IndexId, JobStatus, ObjectId, SchemaId, SinkId, SourceId, @@ -1648,10 +1648,9 @@ impl CatalogController { // Only hash dispatcher needs mapping if dispatcher.dispatcher_type.as_ref() == &DispatcherType::Hash { - dispatcher.hash_mapping = - Set(upstream_dispatcher_mapping.as_ref().map(|m| { - risingwave_meta_model_v2::ActorMapping::from(&m.to_protobuf()) - })); + dispatcher.hash_mapping = Set(upstream_dispatcher_mapping + .as_ref() + .map(|m| risingwave_meta_model::ActorMapping::from(&m.to_protobuf()))); } let mut new_downstream_actor_ids = diff --git a/src/meta/src/controller/system_param.rs b/src/meta/src/controller/system_param.rs index fdeee871a898b..4bb36c8e1962c 100644 --- a/src/meta/src/controller/system_param.rs +++ b/src/meta/src/controller/system_param.rs @@ -22,8 +22,8 @@ use risingwave_common::system_param::{ check_missing_params, default, derive_missing_fields, set_system_param, }; use risingwave_common::{for_all_params, key_of}; -use risingwave_meta_model_v2::prelude::SystemParameter; -use risingwave_meta_model_v2::system_parameter; +use risingwave_meta_model::prelude::SystemParameter; +use risingwave_meta_model::system_parameter; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::PbSystemParams; use sea_orm::ActiveValue::Set; diff --git a/src/meta/src/controller/user.rs b/src/meta/src/controller/user.rs index 744f3cf112f6e..3a8d728c84fd9 100644 --- a/src/meta/src/controller/user.rs +++ b/src/meta/src/controller/user.rs @@ -16,9 +16,9 @@ use std::collections::{HashMap, HashSet}; use itertools::Itertools; use risingwave_common::catalog::{DEFAULT_SUPER_USER, DEFAULT_SUPER_USER_FOR_PG}; -use risingwave_meta_model_v2::prelude::{Object, User, UserPrivilege}; -use risingwave_meta_model_v2::user_privilege::Action; -use risingwave_meta_model_v2::{object, user, user_privilege, AuthInfo, PrivilegeId, UserId}; +use risingwave_meta_model::prelude::{Object, User, UserPrivilege}; +use risingwave_meta_model::user_privilege::Action; +use risingwave_meta_model::{object, user, user_privilege, AuthInfo, PrivilegeId, UserId}; use risingwave_pb::meta::subscribe_response::{ Info as NotificationInfo, Operation as NotificationOperation, }; @@ -469,7 +469,7 @@ impl CatalogController { #[cfg(test)] mod tests { - use risingwave_meta_model_v2::DatabaseId; + use risingwave_meta_model::DatabaseId; use risingwave_pb::user::grant_privilege::{PbAction, PbActionWithGrantOption, PbObject}; use super::*; diff --git a/src/meta/src/controller/utils.rs b/src/meta/src/controller/utils.rs index c9f31decc94d8..a9650388b8db8 100644 --- a/src/meta/src/controller/utils.rs +++ b/src/meta/src/controller/utils.rs @@ -19,17 +19,17 @@ use itertools::Itertools; use risingwave_common::bitmap::Bitmap; use risingwave_common::hash; use risingwave_common::hash::{ActorMapping, WorkerSlotId, WorkerSlotMapping}; -use risingwave_meta_model_migration::WithQuery; -use risingwave_meta_model_v2::actor::ActorStatus; -use risingwave_meta_model_v2::fragment::DistributionType; -use risingwave_meta_model_v2::object::ObjectType; -use risingwave_meta_model_v2::prelude::*; -use risingwave_meta_model_v2::{ +use risingwave_meta_model::actor::ActorStatus; +use risingwave_meta_model::fragment::DistributionType; +use risingwave_meta_model::object::ObjectType; +use risingwave_meta_model::prelude::*; +use risingwave_meta_model::{ actor, actor_dispatcher, connection, database, fragment, function, index, object, object_dependency, schema, secret, sink, source, subscription, table, user, user_privilege, view, ActorId, ConnectorSplits, DataTypeArray, DatabaseId, FragmentId, I32Array, ObjectId, PrivilegeId, SchemaId, SourceId, StreamNode, UserId, VnodeBitmap, WorkerId, }; +use risingwave_meta_model_migration::WithQuery; use risingwave_pb::catalog::{ PbConnection, PbFunction, PbIndex, PbSecret, PbSink, PbSource, PbSubscription, PbTable, PbView, }; diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index beb794d228aed..bec4b82f59c96 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -57,7 +57,7 @@ pub(super) mod handlers { use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common_heap_profiling::COLLAPSED_SUFFIX; - use risingwave_meta_model_v2::WorkerId; + use risingwave_meta_model::WorkerId; use risingwave_pb::catalog::table::TableType; use risingwave_pb::catalog::{PbDatabase, PbSchema, Sink, Source, Subscription, Table, View}; use risingwave_pb::common::{WorkerNode, WorkerType}; diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index 78051e28e7cbd..8bfe188d4a3fa 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -16,7 +16,7 @@ use risingwave_common::error::BoxedError; use risingwave_common::session_config::SessionConfigError; use risingwave_connector::error::ConnectorError; use risingwave_connector::sink::SinkError; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::PbFieldNotFound; use risingwave_rpc_client::error::{RpcError, ToTonicStatus}; diff --git a/src/meta/src/hummock/compactor_manager.rs b/src/meta/src/hummock/compactor_manager.rs index f71f6da0345ae..c2b5f54c1b512 100644 --- a/src/meta/src/hummock/compactor_manager.rs +++ b/src/meta/src/hummock/compactor_manager.rs @@ -135,7 +135,7 @@ pub struct CompactorManagerInner { impl CompactorManagerInner { pub async fn with_meta(env: MetaSrvEnv) -> MetaResult { - use risingwave_meta_model_v2::compaction_task; + use risingwave_meta_model::compaction_task; use sea_orm::EntityTrait; // Retrieve the existing task assignments from metastore. let task_assignment: Vec = compaction_task::Entity::find() diff --git a/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs index a1a85f57a466c..77822a0d6f3cc 100644 --- a/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs @@ -25,7 +25,7 @@ use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ use risingwave_hummock_sdk::compaction_group::{StateTableId, StaticCompactionGroupId}; use risingwave_hummock_sdk::version::GroupDelta; use risingwave_hummock_sdk::CompactionGroupId; -use risingwave_meta_model_v2::compaction_config; +use risingwave_meta_model::compaction_config; use risingwave_pb::hummock::rise_ctl_update_compaction_config_request::mutable_config::MutableConfig; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ diff --git a/src/meta/src/hummock/manager/gc.rs b/src/meta/src/hummock/manager/gc.rs index bb8ad52875b74..f84404d322c77 100644 --- a/src/meta/src/hummock/manager/gc.rs +++ b/src/meta/src/hummock/manager/gc.rs @@ -22,9 +22,9 @@ use futures::future::try_join_all; use itertools::Itertools; use parking_lot::Mutex; use risingwave_hummock_sdk::HummockSstableObjectId; +use risingwave_meta_model::hummock_sequence; +use risingwave_meta_model::hummock_sequence::HUMMOCK_NOW; use risingwave_meta_model_migration::OnConflict; -use risingwave_meta_model_v2::hummock_sequence; -use risingwave_meta_model_v2::hummock_sequence::HUMMOCK_NOW; use risingwave_pb::hummock::subscribe_compaction_event_response::Event as ResponseEvent; use risingwave_pb::hummock::FullScanTask; use risingwave_pb::stream_service::GetMinUncommittedSstIdRequest; diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index e1f23ba540158..8aef8a5e45ba8 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -26,7 +26,7 @@ use risingwave_hummock_sdk::{ version_archive_dir, version_checkpoint_path, CompactionGroupId, HummockCompactionTaskId, HummockContextId, HummockVersionId, }; -use risingwave_meta_model_v2::{ +use risingwave_meta_model::{ compaction_status, compaction_task, hummock_pinned_version, hummock_version_delta, hummock_version_stats, }; diff --git a/src/meta/src/hummock/manager/sequence.rs b/src/meta/src/hummock/manager/sequence.rs index cbd6bbf362b0f..5e852ae155e66 100644 --- a/src/meta/src/hummock/manager/sequence.rs +++ b/src/meta/src/hummock/manager/sequence.rs @@ -17,11 +17,11 @@ use std::fmt::Display; use std::sync::LazyLock; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; -use risingwave_meta_model_v2::hummock_sequence; -use risingwave_meta_model_v2::hummock_sequence::{ +use risingwave_meta_model::hummock_sequence; +use risingwave_meta_model::hummock_sequence::{ COMPACTION_GROUP_ID, COMPACTION_TASK_ID, META_BACKUP_ID, SSTABLE_OBJECT_ID, }; -use risingwave_meta_model_v2::prelude::HummockSequence; +use risingwave_meta_model::prelude::HummockSequence; use sea_orm::{ActiveModelTrait, ActiveValue, DatabaseConnection, EntityTrait, TransactionTrait}; use tokio::sync::Mutex; diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index ca86622977188..8077998aa1ee0 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -121,7 +121,7 @@ fn get_compaction_group_object_ids( } async fn list_pinned_version_from_meta_store(env: &MetaSrvEnv) -> Vec { - use risingwave_meta_model_v2::hummock_pinned_version; + use risingwave_meta_model::hummock_pinned_version; use sea_orm::EntityTrait; hummock_pinned_version::Entity::find() .all(&env.meta_store_ref().conn) diff --git a/src/meta/src/hummock/manager/time_travel.rs b/src/meta/src/hummock/manager/time_travel.rs index 4b15309abfb98..d342ec5b7bc61 100644 --- a/src/meta/src/hummock/manager/time_travel.rs +++ b/src/meta/src/hummock/manager/time_travel.rs @@ -28,8 +28,8 @@ use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{ CompactionGroupId, HummockEpoch, HummockSstableId, HummockSstableObjectId, }; -use risingwave_meta_model_v2::hummock_sstable_info::SstableInfoV2Backend; -use risingwave_meta_model_v2::{ +use risingwave_meta_model::hummock_sstable_info::SstableInfoV2Backend; +use risingwave_meta_model::{ hummock_epoch_to_version, hummock_sstable_info, hummock_time_travel_delta, hummock_time_travel_version, }; @@ -82,7 +82,7 @@ impl HummockManager { let version_watermark = hummock_epoch_to_version::Entity::find() .filter( hummock_epoch_to_version::Column::Epoch - .lt(risingwave_meta_model_v2::Epoch::try_from(epoch_watermark).unwrap()), + .lt(risingwave_meta_model::Epoch::try_from(epoch_watermark).unwrap()), ) .order_by_desc(hummock_epoch_to_version::Column::Epoch) .order_by_asc(hummock_epoch_to_version::Column::VersionId) @@ -95,7 +95,7 @@ impl HummockManager { let res = hummock_epoch_to_version::Entity::delete_many() .filter( hummock_epoch_to_version::Column::Epoch - .lt(risingwave_meta_model_v2::Epoch::try_from(epoch_watermark).unwrap()), + .lt(risingwave_meta_model::Epoch::try_from(epoch_watermark).unwrap()), ) .exec(&txn) .await?; @@ -122,7 +122,7 @@ impl HummockManager { earliest_valid_version.get_sst_ids(), ) }; - let version_ids_to_delete: Vec = + let version_ids_to_delete: Vec = hummock_time_travel_version::Entity::find() .select_only() .column(hummock_time_travel_version::Column::VersionId) @@ -134,7 +134,7 @@ impl HummockManager { .into_tuple() .all(&txn) .await?; - let delta_ids_to_delete: Vec = + let delta_ids_to_delete: Vec = hummock_time_travel_delta::Entity::find() .select_only() .column(hummock_time_travel_delta::Column::VersionId) @@ -234,7 +234,7 @@ impl HummockManager { pub(crate) async fn all_object_ids_in_time_travel( &self, ) -> Result> { - let object_ids: Vec = + let object_ids: Vec = hummock_sstable_info::Entity::find() .select_only() .column(hummock_sstable_info::Column::ObjectId) @@ -274,7 +274,7 @@ impl HummockManager { ) .filter( hummock_epoch_to_version::Column::Epoch - .lte(risingwave_meta_model_v2::Epoch::try_from(query_epoch).unwrap()), + .lte(risingwave_meta_model::Epoch::try_from(query_epoch).unwrap()), ) .order_by_desc(hummock_epoch_to_version::Column::Epoch) .one(&sql_store.conn) @@ -428,7 +428,7 @@ impl HummockManager { ) .await?; let m = hummock_time_travel_version::ActiveModel { - version_id: Set(risingwave_meta_model_v2::HummockVersionId::try_from( + version_id: Set(risingwave_meta_model::HummockVersionId::try_from( version.id.to_u64(), ) .unwrap()), @@ -456,7 +456,7 @@ impl HummockManager { // Ignore delta which adds no data. if written > 0 { let m = hummock_time_travel_delta::ActiveModel { - version_id: Set(risingwave_meta_model_v2::HummockVersionId::try_from( + version_id: Set(risingwave_meta_model::HummockVersionId::try_from( delta.id.to_u64(), ) .unwrap()), diff --git a/src/meta/src/hummock/model/ext/hummock.rs b/src/meta/src/hummock/model/ext/hummock.rs index b66672a2057cb..9a625670efbc8 100644 --- a/src/meta/src/hummock/model/ext/hummock.rs +++ b/src/meta/src/hummock/model/ext/hummock.rs @@ -14,12 +14,12 @@ use itertools::Itertools; use risingwave_hummock_sdk::version::HummockVersionDelta; -use risingwave_meta_model_v2::compaction_config::CompactionConfig; -use risingwave_meta_model_v2::compaction_status::LevelHandlers; -use risingwave_meta_model_v2::compaction_task::CompactionTask; -use risingwave_meta_model_v2::hummock_version_delta::FullVersionDelta; -use risingwave_meta_model_v2::hummock_version_stats::TableStats; -use risingwave_meta_model_v2::{ +use risingwave_meta_model::compaction_config::CompactionConfig; +use risingwave_meta_model::compaction_status::LevelHandlers; +use risingwave_meta_model::compaction_task::CompactionTask; +use risingwave_meta_model::hummock_version_delta::FullVersionDelta; +use risingwave_meta_model::hummock_version_stats::TableStats; +use risingwave_meta_model::{ compaction_config, compaction_status, compaction_task, hummock_pinned_snapshot, hummock_pinned_version, hummock_version_delta, hummock_version_stats, CompactionGroupId, CompactionTaskId, HummockVersionId, WorkerId, diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 2fc54c06e3cb8..006b31475461d 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -31,7 +31,7 @@ use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionStateTableIn use risingwave_hummock_sdk::{ CompactionGroupId, HummockEpoch, HummockSstableObjectId, LocalSstableInfo, SyncResult, }; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::common::{HostAddress, WorkerType}; use risingwave_pb::hummock::compact_task::TaskStatus; use risingwave_pb::hummock::CompactionConfig; diff --git a/src/meta/src/manager/diagnose.rs b/src/meta/src/manager/diagnose.rs index 2022c6ae9764d..de8f983056a6a 100644 --- a/src/meta/src/manager/diagnose.rs +++ b/src/meta/src/manager/diagnose.rs @@ -22,7 +22,7 @@ use prometheus_http_query::response::Data::Vector; use risingwave_common::types::Timestamptz; use risingwave_common::util::StackTraceResponseExt; use risingwave_hummock_sdk::level::Level; -use risingwave_meta_model_v2::table::TableType; +use risingwave_meta_model::table::TableType; use risingwave_pb::common::WorkerType; use risingwave_pb::meta::event_log::Event; use risingwave_pb::meta::EventLog; diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 380ab94bd9210..3a7f75098373c 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -20,8 +20,8 @@ use risingwave_common::config::{CompactionConfig, DefaultParallelism, ObjectStor use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::{bail, system_param}; +use risingwave_meta_model::prelude::Cluster; use risingwave_meta_model_migration::{MigrationStatus, Migrator, MigratorTrait}; -use risingwave_meta_model_v2::prelude::Cluster; use risingwave_pb::meta::SystemParams; use risingwave_rpc_client::{ FrontendClientPool, FrontendClientPoolRef, StreamClientPool, StreamClientPoolRef, diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index cf506e2da1edc..48dfbfaf55c0a 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -19,7 +19,7 @@ use std::time::Duration; use anyhow::anyhow; use futures::future::{select, Either}; use risingwave_common::catalog::{TableId, TableOption}; -use risingwave_meta_model_v2::{ObjectId, SourceId, WorkerId}; +use risingwave_meta_model::{ObjectId, SourceId, WorkerId}; use risingwave_pb::catalog::{PbSink, PbSource, PbTable}; use risingwave_pb::common::worker_node::{PbResource, State}; use risingwave_pb::common::{HostAddress, PbWorkerNode, PbWorkerType, WorkerNode, WorkerType}; diff --git a/src/meta/src/manager/mod.rs b/src/meta/src/manager/mod.rs index 399ee27a198b3..b49ce350c5501 100644 --- a/src/meta/src/manager/mod.rs +++ b/src/meta/src/manager/mod.rs @@ -31,7 +31,7 @@ pub use event_log::EventLogManagerRef; pub use idle::*; pub use metadata::*; pub use notification::{LocalNotification, MessageStatus, NotificationManagerRef, *}; -pub use risingwave_meta_model_v2::prelude; +pub use risingwave_meta_model::prelude; use risingwave_pb::catalog::{PbSink, PbSource}; use risingwave_pb::common::PbHostAddress; pub use streaming_job::*; diff --git a/src/meta/src/manager/notification_version.rs b/src/meta/src/manager/notification_version.rs index 42738f207406a..ee4771d792416 100644 --- a/src/meta/src/manager/notification_version.rs +++ b/src/meta/src/manager/notification_version.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_meta_model_v2::catalog_version; -use risingwave_meta_model_v2::catalog_version::VersionCategory; -use risingwave_meta_model_v2::prelude::CatalogVersion; +use risingwave_meta_model::catalog_version; +use risingwave_meta_model::catalog_version::VersionCategory; +use risingwave_meta_model::prelude::CatalogVersion; use sea_orm::ActiveValue::Set; use sea_orm::{ActiveModelTrait, DatabaseConnection, EntityTrait, TransactionTrait}; diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index d773d2a5de935..7eb921e0befa4 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -19,7 +19,7 @@ use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::hash::{VirtualNode, WorkerSlotId}; use risingwave_connector::source::SplitImpl; -use risingwave_meta_model_v2::{SourceId, WorkerId}; +use risingwave_meta_model::{SourceId, WorkerId}; use risingwave_pb::common::PbActorLocation; use risingwave_pb::meta::table_fragments::actor_status::ActorState; use risingwave_pb::meta::table_fragments::{ActorStatus, Fragment, State}; diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 0cc4e82969c22..995643215d317 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -38,8 +38,8 @@ use risingwave_connector::source::{ UPSTREAM_SOURCE_KEY, }; use risingwave_connector::{dispatch_source_prop, WithOptionsSecResolved}; -use risingwave_meta_model_v2::object::ObjectType; -use risingwave_meta_model_v2::{ +use risingwave_meta_model::object::ObjectType; +use risingwave_meta_model::{ ConnectionId, DatabaseId, FunctionId, IndexId, ObjectId, SchemaId, SecretId, SinkId, SourceId, SubscriptionId, TableId, UserId, ViewId, }; @@ -48,8 +48,8 @@ use risingwave_pb::catalog::connection::PrivateLinkService; use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ - Comment, Connection, CreateType, Database, Function, PbSink, PbSource, PbTable, Schema, Secret, - Sink, Source, Subscription, Table, View, + connection, Comment, Connection, CreateType, Database, Function, PbSink, PbSource, PbTable, + Schema, Secret, Sink, Source, Subscription, Table, View, }; use risingwave_pb::ddl_service::alter_owner_request::Object; use risingwave_pb::ddl_service::{ @@ -59,6 +59,7 @@ use risingwave_pb::meta::table_fragments::fragment::FragmentDistributionType; use risingwave_pb::meta::table_fragments::PbFragment; use risingwave_pb::meta::PbTableParallelism; use risingwave_pb::stream_plan::stream_node::NodeBody; +use risingwave_pb::stream_plan::update_mutation::PbMergeUpdate; use risingwave_pb::stream_plan::{ Dispatcher, DispatcherType, FragmentTypeFlag, MergeNode, PbStreamFragmentGraph, StreamFragmentGraph as StreamFragmentGraphProto, @@ -70,15 +71,17 @@ use tracing::log::warn; use tracing::Instrument; use crate::barrier::BarrierManagerRef; +use crate::controller::catalog::ReleaseContext; use crate::controller::cluster::StreamingClusterInfo; use crate::error::{bail_invalid_parameter, bail_unavailable}; use crate::manager::{ DdlType, LocalNotification, MetaSrvEnv, MetadataManager, NotificationVersion, StreamingJob, + IGNORED_NOTIFICATION_VERSION, }; use crate::model::{FragmentId, StreamContext, TableFragments, TableParallelism}; use crate::rpc::cloud_provider::AwsEc2Client; use crate::stream::{ - create_source_worker_handle, ActorGraphBuildResult, ActorGraphBuilder, + create_source_worker_handle, validate_sink, ActorGraphBuildResult, ActorGraphBuilder, CompleteStreamFragmentGraph, CreateStreamingJobContext, CreateStreamingJobOption, GlobalStreamManagerRef, ReplaceTableContext, SourceManagerRef, StreamFragmentGraph, }; @@ -304,7 +307,7 @@ impl DdlController { _create_type, affected_table_replace_info, ) => { - ctrl.create_streaming_job_v2( + ctrl.create_streaming_job( stream_job, fragment_graph, affected_table_replace_info, @@ -320,7 +323,7 @@ impl DdlController { fragment_graph, col_index_mapping, }) => { - ctrl.replace_table_v2(streaming_job, fragment_graph, col_index_mapping) + ctrl.replace_table(streaming_job, fragment_graph, col_index_mapping) .await } DdlCommand::AlterName(relation, name) => ctrl.alter_name(relation, &name).await, @@ -424,7 +427,7 @@ impl DdlController { .await } - /// Shared source is handled in [`Self::create_streaming_job_v2`] + /// Shared source is handled in [`Self::create_streaming_job`] async fn create_source_without_streaming_job( &self, source: Source, @@ -924,6 +927,550 @@ impl DdlController { .push(upstream_fragment_id); } + /// For [`CreateType::Foreground`], the function will only return after backfilling finishes + /// ([`crate::manager::MetadataManager::wait_streaming_job_finished`]). + pub async fn create_streaming_job( + &self, + mut streaming_job: StreamingJob, + mut fragment_graph: StreamFragmentGraphProto, + affected_table_replace_info: Option, + ) -> MetaResult { + let ctx = StreamContext::from_protobuf(fragment_graph.get_ctx().unwrap()); + self.metadata_manager + .catalog_controller + .create_job_catalog( + &mut streaming_job, + &ctx, + &fragment_graph.parallelism, + fragment_graph.max_parallelism as _, + ) + .await?; + let job_id = streaming_job.id(); + + match &mut streaming_job { + StreamingJob::Table(src, table, job_type) => { + // If we're creating a table with connector, we should additionally fill its ID first. + fill_table_stream_graph_info(src, table, *job_type, &mut fragment_graph); + } + StreamingJob::Source(src) => { + // set the inner source id of source node. + for fragment in fragment_graph.fragments.values_mut() { + visit_fragment(fragment, |node_body| { + if let NodeBody::Source(source_node) = node_body { + source_node.source_inner.as_mut().unwrap().source_id = src.id; + } + }); + } + } + _ => {} + } + + tracing::debug!( + id = job_id, + definition = streaming_job.definition(), + create_type = streaming_job.create_type().as_str_name(), + "starting streaming job", + ); + let _permit = self + .creating_streaming_job_permits + .semaphore + .acquire() + .await + .unwrap(); + let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await; + + let id = streaming_job.id(); + let name = streaming_job.name(); + let definition = streaming_job.definition(); + let source_id = match &streaming_job { + StreamingJob::Table(Some(src), _, _) | StreamingJob::Source(src) => Some(src.id), + _ => None, + }; + + // create streaming job. + match self + .create_streaming_job_inner( + ctx, + streaming_job, + fragment_graph, + affected_table_replace_info, + ) + .await + { + Ok(version) => Ok(version), + Err(err) => { + tracing::error!(id = job_id, error = %err.as_report(), "failed to create streaming job"); + let event = risingwave_pb::meta::event_log::EventCreateStreamJobFail { + id, + name, + definition, + error: err.as_report().to_string(), + }; + self.env.event_log_manager_ref().add_event_logs(vec![ + risingwave_pb::meta::event_log::Event::CreateStreamJobFail(event), + ]); + let aborted = self + .metadata_manager + .catalog_controller + .try_abort_creating_streaming_job(job_id as _, false) + .await?; + if aborted { + tracing::warn!(id = job_id, "aborted streaming job"); + if let Some(source_id) = source_id { + self.source_manager + .unregister_sources(vec![source_id as SourceId]) + .await; + } + } + Err(err) + } + } + } + + async fn create_streaming_job_inner( + &self, + ctx: StreamContext, + mut streaming_job: StreamingJob, + fragment_graph: StreamFragmentGraphProto, + affected_table_replace_info: Option, + ) -> MetaResult { + let mut fragment_graph = + StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?; + streaming_job.set_table_fragment_id(fragment_graph.table_fragment_id()); + streaming_job.set_dml_fragment_id(fragment_graph.dml_fragment_id()); + + // create internal table catalogs and refill table id. + let internal_tables = fragment_graph.internal_tables().into_values().collect_vec(); + let table_id_map = self + .metadata_manager + .catalog_controller + .create_internal_table_catalog(&streaming_job, internal_tables) + .await?; + fragment_graph.refill_internal_table_ids(table_id_map); + + let affected_table_replace_info = match affected_table_replace_info { + Some(replace_table_info) => { + let ReplaceTableInfo { + mut streaming_job, + fragment_graph, + .. + } = replace_table_info; + + let fragment_graph = + StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?; + streaming_job.set_table_fragment_id(fragment_graph.table_fragment_id()); + streaming_job.set_dml_fragment_id(fragment_graph.dml_fragment_id()); + let streaming_job = streaming_job; + + Some((streaming_job, fragment_graph)) + } + None => None, + }; + + // create fragment and actor catalogs. + tracing::debug!(id = streaming_job.id(), "building streaming job"); + let (ctx, table_fragments) = self + .build_stream_job( + ctx, + streaming_job, + fragment_graph, + affected_table_replace_info, + ) + .await?; + + let streaming_job = &ctx.streaming_job; + + match streaming_job { + StreamingJob::Table(None, table, TableJobType::SharedCdcSource) => { + Self::validate_cdc_table(table, &table_fragments).await?; + } + StreamingJob::Table(Some(source), ..) => { + // Register the source on the connector node. + self.source_manager.register_source(source).await?; + } + StreamingJob::Sink(sink, _) => { + // Validate the sink on the connector node. + validate_sink(sink).await?; + } + StreamingJob::Source(source) => { + // Register the source on the connector node. + self.source_manager.register_source(source).await?; + } + _ => {} + } + + self.metadata_manager + .catalog_controller + .prepare_streaming_job(table_fragments.to_protobuf(), streaming_job, false) + .await?; + + // create streaming jobs. + let stream_job_id = streaming_job.id(); + match (streaming_job.create_type(), &streaming_job) { + (CreateType::Unspecified, _) + | (CreateType::Foreground, _) + // FIXME(kwannoel): Unify background stream's creation path with MV below. + | (CreateType::Background, StreamingJob::Sink(_, _)) => { + let version = self.stream_manager + .create_streaming_job(table_fragments, ctx) + .await?; + Ok(version) + } + (CreateType::Background, _) => { + let ctrl = self.clone(); + let fut = async move { + let _ = ctrl + .stream_manager + .create_streaming_job(table_fragments, ctx) + .await.inspect_err(|err| { + tracing::error!(id = stream_job_id, error = ?err.as_report(), "failed to create background streaming job"); + }); + }; + tokio::spawn(fut); + Ok(IGNORED_NOTIFICATION_VERSION) + } + } + } + + pub async fn drop_object( + &self, + object_type: ObjectType, + object_id: ObjectId, + drop_mode: DropMode, + target_replace_info: Option, + ) -> MetaResult { + let (release_ctx, mut version) = match object_type { + ObjectType::Database => { + self.metadata_manager + .catalog_controller + .drop_database(object_id) + .await? + } + ObjectType::Schema => { + return self + .metadata_manager + .catalog_controller + .drop_schema(object_id, drop_mode) + .await; + } + ObjectType::Function => { + return self + .metadata_manager + .catalog_controller + .drop_function(object_id) + .await; + } + ObjectType::Connection => { + let (version, conn) = self + .metadata_manager + .catalog_controller + .drop_connection(object_id) + .await?; + if let Some(connection::Info::PrivateLinkService(svc)) = &conn.info { + self.delete_vpc_endpoint(svc).await?; + } + return Ok(version); + } + _ => { + self.metadata_manager + .catalog_controller + .drop_relation(object_type, object_id, drop_mode) + .await? + } + }; + + if let Some(replace_table_info) = target_replace_info { + let stream_ctx = + StreamContext::from_protobuf(replace_table_info.fragment_graph.get_ctx().unwrap()); + + let ReplaceTableInfo { + mut streaming_job, + fragment_graph, + .. + } = replace_table_info; + + let sink_id = if let ObjectType::Sink = object_type { + object_id as _ + } else { + panic!("additional replace table event only occurs when dropping sink into table") + }; + + let fragment_graph = + StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?; + streaming_job.set_table_fragment_id(fragment_graph.table_fragment_id()); + streaming_job.set_dml_fragment_id(fragment_graph.dml_fragment_id()); + let streaming_job = streaming_job; + + let table = streaming_job.table().unwrap(); + + tracing::debug!(id = streaming_job.id(), "replacing table for dropped sink"); + let dummy_id = self + .metadata_manager + .catalog_controller + .create_job_catalog_for_replace( + &streaming_job, + &stream_ctx, + table.get_version()?, + &fragment_graph.specified_parallelism(), + fragment_graph.max_parallelism(), + ) + .await? as u32; + + let (ctx, table_fragments) = self + .inject_replace_table_job_for_table_sink( + dummy_id, + &self.metadata_manager, + stream_ctx, + None, + None, + Some(sink_id), + &streaming_job, + fragment_graph, + ) + .await?; + + let result: MetaResult> = try { + let merge_updates = ctx.merge_updates.clone(); + + self.metadata_manager + .catalog_controller + .prepare_streaming_job(table_fragments.to_protobuf(), &streaming_job, true) + .await?; + + self.stream_manager + .replace_table(table_fragments, ctx) + .await?; + + merge_updates + }; + + version = match result { + Ok(merge_updates) => { + let version = self + .metadata_manager + .catalog_controller + .finish_replace_streaming_job( + dummy_id as _, + streaming_job, + merge_updates, + None, + None, + Some(sink_id), + vec![], + ) + .await?; + Ok(version) + } + Err(err) => { + tracing::error!(id = object_id, error = ?err.as_report(), "failed to replace table"); + let _ = self.metadata_manager + .catalog_controller + .try_abort_replacing_streaming_job(dummy_id as _) + .await + .inspect_err(|err| { + tracing::error!(id = object_id, error = ?err.as_report(), "failed to abort replacing table"); + }); + Err(err) + } + }?; + } + + let ReleaseContext { + streaming_job_ids, + state_table_ids, + source_ids, + connections, + source_fragments, + removed_actors, + removed_fragments, + } = release_ctx; + + // delete vpc endpoints. + for conn in connections { + let _ = self + .delete_vpc_endpoint(&conn.to_protobuf()) + .await + .inspect_err(|err| { + tracing::warn!(err = ?err.as_report(), "failed to delete vpc endpoint"); + }); + } + + // unregister sources. + self.source_manager + .unregister_sources(source_ids.into_iter().map(|id| id as _).collect()) + .await; + + // unregister fragments and actors from source manager. + self.source_manager + .drop_source_fragments( + source_fragments + .into_iter() + .map(|(source_id, fragments)| { + ( + source_id, + fragments.into_iter().map(|id| id as u32).collect(), + ) + }) + .collect(), + removed_actors.iter().map(|id| *id as _).collect(), + ) + .await; + + // drop streaming jobs. + self.stream_manager + .drop_streaming_jobs( + removed_actors.into_iter().map(|id| id as _).collect(), + streaming_job_ids, + state_table_ids, + removed_fragments.iter().map(|id| *id as _).collect(), + ) + .await; + + Ok(version) + } + + /// This is used for `ALTER TABLE ADD/DROP COLUMN`. + pub async fn replace_table( + &self, + mut streaming_job: StreamingJob, + fragment_graph: StreamFragmentGraphProto, + table_col_index_mapping: Option, + ) -> MetaResult { + let job_id = streaming_job.id(); + + let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await; + let ctx = StreamContext::from_protobuf(fragment_graph.get_ctx().unwrap()); + + // 1. build fragment graph. + let fragment_graph = StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?; + streaming_job.set_table_fragment_id(fragment_graph.table_fragment_id()); + streaming_job.set_dml_fragment_id(fragment_graph.dml_fragment_id()); + let streaming_job = streaming_job; + + let StreamingJob::Table(_, table, ..) = &streaming_job else { + unreachable!("unexpected job: {streaming_job:?}") + }; + let dummy_id = self + .metadata_manager + .catalog_controller + .create_job_catalog_for_replace( + &streaming_job, + &ctx, + table.get_version()?, + &fragment_graph.specified_parallelism(), + fragment_graph.max_parallelism(), + ) + .await?; + + tracing::debug!(id = streaming_job.id(), "building replace streaming job"); + let mut updated_sink_catalogs = vec![]; + + let result: MetaResult> = try { + let (mut ctx, mut table_fragments) = self + .build_replace_table( + ctx, + &streaming_job, + fragment_graph, + table_col_index_mapping.clone(), + dummy_id as _, + ) + .await?; + + let mut union_fragment_id = None; + + for (fragment_id, fragment) in &mut table_fragments.fragments { + for actor in &mut fragment.actors { + if let Some(node) = &mut actor.nodes { + visit_stream_node(node, |body| { + if let NodeBody::Union(_) = body { + if let Some(union_fragment_id) = union_fragment_id.as_mut() { + // The union fragment should be unique. + assert_eq!(*union_fragment_id, *fragment_id); + } else { + union_fragment_id = Some(*fragment_id); + } + } + }) + }; + } + } + + let target_fragment_id = + union_fragment_id.expect("fragment of placeholder merger not found"); + + let catalogs = self + .metadata_manager + .get_sink_catalog_by_ids(&table.incoming_sinks) + .await?; + + for sink in catalogs { + let sink_id = &sink.id; + + let sink_table_fragments = self + .metadata_manager + .get_job_fragments_by_id(&risingwave_common::catalog::TableId::new(*sink_id)) + .await?; + + let sink_fragment = sink_table_fragments.sink_fragment().unwrap(); + + Self::inject_replace_table_plan_for_sink( + Some(*sink_id), + &sink_fragment, + table, + &mut ctx, + &mut table_fragments, + target_fragment_id, + Some(&sink.unique_identity()), + ); + + if sink.original_target_columns.is_empty() { + updated_sink_catalogs.push(sink.id as _); + } + } + + let merge_updates = ctx.merge_updates.clone(); + + self.metadata_manager + .catalog_controller + .prepare_streaming_job(table_fragments.to_protobuf(), &streaming_job, true) + .await?; + + self.stream_manager + .replace_table(table_fragments, ctx) + .await?; + merge_updates + }; + + match result { + Ok(merge_updates) => { + let version = self + .metadata_manager + .catalog_controller + .finish_replace_streaming_job( + dummy_id, + streaming_job, + merge_updates, + table_col_index_mapping, + None, + None, + updated_sink_catalogs, + ) + .await?; + Ok(version) + } + Err(err) => { + tracing::error!(id = job_id, error = ?err.as_report(), "failed to replace table"); + let _ = self.metadata_manager + .catalog_controller + .try_abort_replacing_streaming_job(dummy_id) + .await.inspect_err(|err| { + tracing::error!(id = job_id, error = ?err.as_report(), "failed to abort replacing table"); + }); + Err(err) + } + } + } + async fn drop_streaming_job( &self, job_id: StreamingJobId, diff --git a/src/meta/src/rpc/ddl_controller_v2.rs b/src/meta/src/rpc/ddl_controller_v2.rs deleted file mode 100644 index 5ab5c0d3ad00a..0000000000000 --- a/src/meta/src/rpc/ddl_controller_v2.rs +++ /dev/null @@ -1,580 +0,0 @@ -// Copyright 2024 RisingWave Labs -// -// Licensed 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 itertools::Itertools; -use risingwave_common::util::column_index_mapping::ColIndexMapping; -use risingwave_common::util::stream_graph_visitor::{visit_fragment, visit_stream_node}; -use risingwave_meta_model_v2::object::ObjectType; -use risingwave_meta_model_v2::{ObjectId, SourceId}; -use risingwave_pb::catalog::{connection, CreateType}; -use risingwave_pb::ddl_service::TableJobType; -use risingwave_pb::stream_plan::stream_node::NodeBody; -use risingwave_pb::stream_plan::update_mutation::PbMergeUpdate; -use risingwave_pb::stream_plan::StreamFragmentGraph as StreamFragmentGraphProto; -use thiserror_ext::AsReport; - -use crate::controller::catalog::ReleaseContext; -use crate::manager::{NotificationVersion, StreamingJob, IGNORED_NOTIFICATION_VERSION}; -use crate::model::StreamContext; -use crate::rpc::ddl_controller::{ - fill_table_stream_graph_info, DdlController, DropMode, ReplaceTableInfo, -}; -use crate::stream::{validate_sink, StreamFragmentGraph}; -use crate::MetaResult; - -impl DdlController { - /// For [`CreateType::Foreground`], the function will only return after backfilling finishes - /// ([`crate::manager::MetadataManager::wait_streaming_job_finished`]). - pub async fn create_streaming_job_v2( - &self, - mut streaming_job: StreamingJob, - mut fragment_graph: StreamFragmentGraphProto, - affected_table_replace_info: Option, - ) -> MetaResult { - let ctx = StreamContext::from_protobuf(fragment_graph.get_ctx().unwrap()); - self.metadata_manager - .catalog_controller - .create_job_catalog( - &mut streaming_job, - &ctx, - &fragment_graph.parallelism, - fragment_graph.max_parallelism as _, - ) - .await?; - let job_id = streaming_job.id(); - - match &mut streaming_job { - StreamingJob::Table(src, table, job_type) => { - // If we're creating a table with connector, we should additionally fill its ID first. - fill_table_stream_graph_info(src, table, *job_type, &mut fragment_graph); - } - StreamingJob::Source(src) => { - // set the inner source id of source node. - for fragment in fragment_graph.fragments.values_mut() { - visit_fragment(fragment, |node_body| { - if let NodeBody::Source(source_node) = node_body { - source_node.source_inner.as_mut().unwrap().source_id = src.id; - } - }); - } - } - _ => {} - } - - tracing::debug!( - id = job_id, - definition = streaming_job.definition(), - create_type = streaming_job.create_type().as_str_name(), - "starting streaming job", - ); - let _permit = self - .creating_streaming_job_permits - .semaphore - .acquire() - .await - .unwrap(); - let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await; - - let id = streaming_job.id(); - let name = streaming_job.name(); - let definition = streaming_job.definition(); - let source_id = match &streaming_job { - StreamingJob::Table(Some(src), _, _) | StreamingJob::Source(src) => Some(src.id), - _ => None, - }; - - // create streaming job. - match self - .create_streaming_job_inner_v2( - ctx, - streaming_job, - fragment_graph, - affected_table_replace_info, - ) - .await - { - Ok(version) => Ok(version), - Err(err) => { - tracing::error!(id = job_id, error = %err.as_report(), "failed to create streaming job"); - let event = risingwave_pb::meta::event_log::EventCreateStreamJobFail { - id, - name, - definition, - error: err.as_report().to_string(), - }; - self.env.event_log_manager_ref().add_event_logs(vec![ - risingwave_pb::meta::event_log::Event::CreateStreamJobFail(event), - ]); - let aborted = self - .metadata_manager - .catalog_controller - .try_abort_creating_streaming_job(job_id as _, false) - .await?; - if aborted { - tracing::warn!(id = job_id, "aborted streaming job"); - if let Some(source_id) = source_id { - self.source_manager - .unregister_sources(vec![source_id as SourceId]) - .await; - } - } - Err(err) - } - } - } - - async fn create_streaming_job_inner_v2( - &self, - ctx: StreamContext, - mut streaming_job: StreamingJob, - fragment_graph: StreamFragmentGraphProto, - affected_table_replace_info: Option, - ) -> MetaResult { - let mut fragment_graph = - StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?; - streaming_job.set_table_fragment_id(fragment_graph.table_fragment_id()); - streaming_job.set_dml_fragment_id(fragment_graph.dml_fragment_id()); - - // create internal table catalogs and refill table id. - let internal_tables = fragment_graph.internal_tables().into_values().collect_vec(); - let table_id_map = self - .metadata_manager - .catalog_controller - .create_internal_table_catalog(&streaming_job, internal_tables) - .await?; - fragment_graph.refill_internal_table_ids(table_id_map); - - let affected_table_replace_info = match affected_table_replace_info { - Some(replace_table_info) => { - let ReplaceTableInfo { - mut streaming_job, - fragment_graph, - .. - } = replace_table_info; - - let fragment_graph = - StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?; - streaming_job.set_table_fragment_id(fragment_graph.table_fragment_id()); - streaming_job.set_dml_fragment_id(fragment_graph.dml_fragment_id()); - let streaming_job = streaming_job; - - Some((streaming_job, fragment_graph)) - } - None => None, - }; - - // create fragment and actor catalogs. - tracing::debug!(id = streaming_job.id(), "building streaming job"); - let (ctx, table_fragments) = self - .build_stream_job( - ctx, - streaming_job, - fragment_graph, - affected_table_replace_info, - ) - .await?; - - let streaming_job = &ctx.streaming_job; - - match streaming_job { - StreamingJob::Table(None, table, TableJobType::SharedCdcSource) => { - Self::validate_cdc_table(table, &table_fragments).await?; - } - StreamingJob::Table(Some(source), ..) => { - // Register the source on the connector node. - self.source_manager.register_source(source).await?; - } - StreamingJob::Sink(sink, _) => { - // Validate the sink on the connector node. - validate_sink(sink).await?; - } - StreamingJob::Source(source) => { - // Register the source on the connector node. - self.source_manager.register_source(source).await?; - } - _ => {} - } - - self.metadata_manager - .catalog_controller - .prepare_streaming_job(table_fragments.to_protobuf(), streaming_job, false) - .await?; - - // create streaming jobs. - let stream_job_id = streaming_job.id(); - match (streaming_job.create_type(), &streaming_job) { - (CreateType::Unspecified, _) - | (CreateType::Foreground, _) - // FIXME(kwannoel): Unify background stream's creation path with MV below. - | (CreateType::Background, StreamingJob::Sink(_, _)) => { - let version = self.stream_manager - .create_streaming_job(table_fragments, ctx) - .await?; - Ok(version) - } - (CreateType::Background, _) => { - let ctrl = self.clone(); - let fut = async move { - let _ = ctrl - .stream_manager - .create_streaming_job(table_fragments, ctx) - .await.inspect_err(|err| { - tracing::error!(id = stream_job_id, error = ?err.as_report(), "failed to create background streaming job"); - }); - }; - tokio::spawn(fut); - Ok(IGNORED_NOTIFICATION_VERSION) - } - } - } - - pub async fn drop_object( - &self, - object_type: ObjectType, - object_id: ObjectId, - drop_mode: DropMode, - target_replace_info: Option, - ) -> MetaResult { - let (release_ctx, mut version) = match object_type { - ObjectType::Database => { - self.metadata_manager - .catalog_controller - .drop_database(object_id) - .await? - } - ObjectType::Schema => { - return self - .metadata_manager - .catalog_controller - .drop_schema(object_id, drop_mode) - .await; - } - ObjectType::Function => { - return self - .metadata_manager - .catalog_controller - .drop_function(object_id) - .await; - } - ObjectType::Connection => { - let (version, conn) = self - .metadata_manager - .catalog_controller - .drop_connection(object_id) - .await?; - if let Some(connection::Info::PrivateLinkService(svc)) = &conn.info { - self.delete_vpc_endpoint(svc).await?; - } - return Ok(version); - } - _ => { - self.metadata_manager - .catalog_controller - .drop_relation(object_type, object_id, drop_mode) - .await? - } - }; - - if let Some(replace_table_info) = target_replace_info { - let stream_ctx = - StreamContext::from_protobuf(replace_table_info.fragment_graph.get_ctx().unwrap()); - - let ReplaceTableInfo { - mut streaming_job, - fragment_graph, - .. - } = replace_table_info; - - let sink_id = if let ObjectType::Sink = object_type { - object_id as _ - } else { - panic!("additional replace table event only occurs when dropping sink into table") - }; - - let fragment_graph = - StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?; - streaming_job.set_table_fragment_id(fragment_graph.table_fragment_id()); - streaming_job.set_dml_fragment_id(fragment_graph.dml_fragment_id()); - let streaming_job = streaming_job; - - let table = streaming_job.table().unwrap(); - - tracing::debug!(id = streaming_job.id(), "replacing table for dropped sink"); - let dummy_id = self - .metadata_manager - .catalog_controller - .create_job_catalog_for_replace( - &streaming_job, - &stream_ctx, - table.get_version()?, - &fragment_graph.specified_parallelism(), - fragment_graph.max_parallelism(), - ) - .await? as u32; - - let (ctx, table_fragments) = self - .inject_replace_table_job_for_table_sink( - dummy_id, - &self.metadata_manager, - stream_ctx, - None, - None, - Some(sink_id), - &streaming_job, - fragment_graph, - ) - .await?; - - let result: MetaResult> = try { - let merge_updates = ctx.merge_updates.clone(); - - self.metadata_manager - .catalog_controller - .prepare_streaming_job(table_fragments.to_protobuf(), &streaming_job, true) - .await?; - - self.stream_manager - .replace_table(table_fragments, ctx) - .await?; - - merge_updates - }; - - version = match result { - Ok(merge_updates) => { - let version = self - .metadata_manager - .catalog_controller - .finish_replace_streaming_job( - dummy_id as _, - streaming_job, - merge_updates, - None, - None, - Some(sink_id), - vec![], - ) - .await?; - Ok(version) - } - Err(err) => { - tracing::error!(id = object_id, error = ?err.as_report(), "failed to replace table"); - let _ = self.metadata_manager - .catalog_controller - .try_abort_replacing_streaming_job(dummy_id as _) - .await - .inspect_err(|err| { - tracing::error!(id = object_id, error = ?err.as_report(), "failed to abort replacing table"); - }); - Err(err) - } - }?; - } - - let ReleaseContext { - streaming_job_ids, - state_table_ids, - source_ids, - connections, - source_fragments, - removed_actors, - removed_fragments, - } = release_ctx; - - // delete vpc endpoints. - for conn in connections { - let _ = self - .delete_vpc_endpoint(&conn.to_protobuf()) - .await - .inspect_err(|err| { - tracing::warn!(err = ?err.as_report(), "failed to delete vpc endpoint"); - }); - } - - // unregister sources. - self.source_manager - .unregister_sources(source_ids.into_iter().map(|id| id as _).collect()) - .await; - - // unregister fragments and actors from source manager. - self.source_manager - .drop_source_fragments_v2( - source_fragments - .into_iter() - .map(|(source_id, fragments)| { - ( - source_id, - fragments.into_iter().map(|id| id as u32).collect(), - ) - }) - .collect(), - removed_actors.iter().map(|id| *id as _).collect(), - ) - .await; - - // drop streaming jobs. - self.stream_manager - .drop_streaming_jobs( - removed_actors.into_iter().map(|id| id as _).collect(), - streaming_job_ids, - state_table_ids, - removed_fragments.iter().map(|id| *id as _).collect(), - ) - .await; - - Ok(version) - } - - /// This is used for `ALTER TABLE ADD/DROP COLUMN`. - pub async fn replace_table_v2( - &self, - mut streaming_job: StreamingJob, - fragment_graph: StreamFragmentGraphProto, - table_col_index_mapping: Option, - ) -> MetaResult { - let job_id = streaming_job.id(); - - let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await; - let ctx = StreamContext::from_protobuf(fragment_graph.get_ctx().unwrap()); - - // 1. build fragment graph. - let fragment_graph = StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?; - streaming_job.set_table_fragment_id(fragment_graph.table_fragment_id()); - streaming_job.set_dml_fragment_id(fragment_graph.dml_fragment_id()); - let streaming_job = streaming_job; - - let StreamingJob::Table(_, table, ..) = &streaming_job else { - unreachable!("unexpected job: {streaming_job:?}") - }; - let dummy_id = self - .metadata_manager - .catalog_controller - .create_job_catalog_for_replace( - &streaming_job, - &ctx, - table.get_version()?, - &fragment_graph.specified_parallelism(), - fragment_graph.max_parallelism(), - ) - .await?; - - tracing::debug!(id = streaming_job.id(), "building replace streaming job"); - let mut updated_sink_catalogs = vec![]; - - let result: MetaResult> = try { - let (mut ctx, mut table_fragments) = self - .build_replace_table( - ctx, - &streaming_job, - fragment_graph, - table_col_index_mapping.clone(), - dummy_id as _, - ) - .await?; - - let mut union_fragment_id = None; - - for (fragment_id, fragment) in &mut table_fragments.fragments { - for actor in &mut fragment.actors { - if let Some(node) = &mut actor.nodes { - visit_stream_node(node, |body| { - if let NodeBody::Union(_) = body { - if let Some(union_fragment_id) = union_fragment_id.as_mut() { - // The union fragment should be unique. - assert_eq!(*union_fragment_id, *fragment_id); - } else { - union_fragment_id = Some(*fragment_id); - } - } - }) - }; - } - } - - let target_fragment_id = - union_fragment_id.expect("fragment of placeholder merger not found"); - - let catalogs = self - .metadata_manager - .get_sink_catalog_by_ids(&table.incoming_sinks) - .await?; - - for sink in catalogs { - let sink_id = &sink.id; - - let sink_table_fragments = self - .metadata_manager - .get_job_fragments_by_id(&risingwave_common::catalog::TableId::new(*sink_id)) - .await?; - - let sink_fragment = sink_table_fragments.sink_fragment().unwrap(); - - Self::inject_replace_table_plan_for_sink( - Some(*sink_id), - &sink_fragment, - table, - &mut ctx, - &mut table_fragments, - target_fragment_id, - Some(&sink.unique_identity()), - ); - - if sink.original_target_columns.is_empty() { - updated_sink_catalogs.push(sink.id as _); - } - } - - let merge_updates = ctx.merge_updates.clone(); - - self.metadata_manager - .catalog_controller - .prepare_streaming_job(table_fragments.to_protobuf(), &streaming_job, true) - .await?; - - self.stream_manager - .replace_table(table_fragments, ctx) - .await?; - merge_updates - }; - - match result { - Ok(merge_updates) => { - let version = self - .metadata_manager - .catalog_controller - .finish_replace_streaming_job( - dummy_id, - streaming_job, - merge_updates, - table_col_index_mapping, - None, - None, - updated_sink_catalogs, - ) - .await?; - Ok(version) - } - Err(err) => { - tracing::error!(id = job_id, error = ?err.as_report(), "failed to replace table"); - let _ = self.metadata_manager - .catalog_controller - .try_abort_replacing_streaming_job(dummy_id) - .await.inspect_err(|err| { - tracing::error!(id = job_id, error = ?err.as_report(), "failed to abort replacing table"); - }); - Err(err) - } - } - } -} diff --git a/src/meta/src/rpc/metrics.rs b/src/meta/src/rpc/metrics.rs index ccceddfb0799c..2d5020edb04be 100644 --- a/src/meta/src/rpc/metrics.rs +++ b/src/meta/src/rpc/metrics.rs @@ -34,7 +34,7 @@ use risingwave_common::{ register_guarded_int_gauge_vec_with_registry, }; use risingwave_connector::source::monitor::EnumeratorMetrics as SourceEnumeratorMetrics; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_object_store::object::object_metrics::{ ObjectStoreMetrics, GLOBAL_OBJECT_STORE_METRICS, }; @@ -896,7 +896,7 @@ pub fn start_worker_info_monitor( (join_handle, shutdown_tx) } -pub async fn refresh_fragment_info_metrics_v2( +pub async fn refresh_fragment_info_metrics( catalog_controller: &CatalogControllerRef, cluster_controller: &ClusterControllerRef, hummock_manager: &HummockManagerRef, @@ -1038,7 +1038,7 @@ pub fn start_fragment_info_monitor( } } - refresh_fragment_info_metrics_v2( + refresh_fragment_info_metrics( &metadata_manager.catalog_controller, &metadata_manager.cluster_controller, &hummock_manager, diff --git a/src/meta/src/rpc/mod.rs b/src/meta/src/rpc/mod.rs index 09fbf7e12f48e..8b256d1b2145e 100644 --- a/src/meta/src/rpc/mod.rs +++ b/src/meta/src/rpc/mod.rs @@ -14,7 +14,6 @@ pub mod cloud_provider; pub mod ddl_controller; -mod ddl_controller_v2; pub mod election; pub mod intercept; pub mod metrics; diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index aba20f1e30642..2dbd6364d4e2c 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -30,7 +30,7 @@ use risingwave_common::bitmap::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::TableId; use risingwave_common::hash::ActorMapping; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_meta_model_v2::{actor, fragment, ObjectId, StreamingParallelism, WorkerId}; +use risingwave_meta_model::{actor, fragment, ObjectId, StreamingParallelism, WorkerId}; use risingwave_pb::common::{PbActorLocation, WorkerNode, WorkerType}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::table_fragments::actor_status::ActorState; @@ -498,7 +498,7 @@ impl ScaleController { fragment_state: &mut HashMap, fragment_to_table: &mut HashMap, mgr: &MetadataManager, - fragment_ids: Vec, + fragment_ids: Vec, ) -> Result<(), MetaError> { let RescheduleWorkingSet { fragments, @@ -513,7 +513,7 @@ impl ScaleController { .await?; let mut fragment_actors: HashMap< - risingwave_meta_model_v2::FragmentId, + risingwave_meta_model::FragmentId, Vec, > = HashMap::new(); @@ -1843,7 +1843,7 @@ impl ScaleController { // index for fragment_id -> [actor_id] let mut fragment_actor_id_map = HashMap::new(); - async fn build_index_v2( + async fn build_index( no_shuffle_source_fragment_ids: &mut HashSet, no_shuffle_target_fragment_ids: &mut HashSet, fragment_distribution_map: &mut HashMap< @@ -1870,7 +1870,7 @@ impl ScaleController { for (fragment_id, downstreams) in fragment_downstreams { for (downstream_fragment_id, dispatcher_type) in downstreams { - if let risingwave_meta_model_v2::actor_dispatcher::DispatcherType::NoShuffle = + if let risingwave_meta_model::actor_dispatcher::DispatcherType::NoShuffle = dispatcher_type { no_shuffle_source_fragment_ids.insert(fragment_id as FragmentId); @@ -1910,7 +1910,7 @@ impl ScaleController { .map(|id| *id as ObjectId) .collect(); - build_index_v2( + build_index( &mut no_shuffle_source_fragment_ids, &mut no_shuffle_target_fragment_ids, &mut fragment_distribution_map, diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 6f6a9950bb363..c5bcc0c179ba3 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -29,7 +29,7 @@ use risingwave_connector::source::{ SplitEnumerator, SplitId, SplitImpl, SplitMetaData, }; use risingwave_connector::{dispatch_source_prop, WithOptionsSecResolved}; -use risingwave_meta_model_v2::SourceId; +use risingwave_meta_model::SourceId; use risingwave_pb::catalog::Source; use risingwave_pb::source::{ConnectorSplit, ConnectorSplits}; use risingwave_pb::stream_plan::Dispatcher; @@ -752,7 +752,7 @@ impl SourceManager { }) } - pub async fn drop_source_fragments_v2( + pub async fn drop_source_fragments( &self, source_fragments: HashMap>, removed_actors: HashSet, @@ -762,7 +762,7 @@ impl SourceManager { } /// For dropping MV. - pub async fn drop_source_fragments(&self, table_fragments: &[TableFragments]) { + pub async fn drop_source_fragments_vec(&self, table_fragments: &[TableFragments]) { let mut core = self.core.lock().await; // Extract the fragments that include source operators. diff --git a/src/meta/src/stream/stream_graph/actor.rs b/src/meta/src/stream/stream_graph/actor.rs index 099571bab2cea..3446a2661d962 100644 --- a/src/meta/src/stream/stream_graph/actor.rs +++ b/src/meta/src/stream/stream_graph/actor.rs @@ -23,7 +23,7 @@ use risingwave_common::bitmap::Bitmap; use risingwave_common::hash::{ActorId, ActorMapping, WorkerSlotId}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::stream_graph_visitor::visit_tables; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::meta::table_fragments::Fragment; use risingwave_pb::plan_common::ExprContext; use risingwave_pb::stream_plan::stream_node::NodeBody; diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 6b78d67448d74..aaa30ee0d57c5 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -27,7 +27,7 @@ use risingwave_common::catalog::{ use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::stream_graph_visitor; use risingwave_common::util::stream_graph_visitor::visit_stream_node_cont; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::catalog::Table; use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::meta::table_fragments::Fragment; diff --git a/src/meta/src/stream/stream_graph/schedule.rs b/src/meta/src/stream/stream_graph/schedule.rs index 23cb4489f8bce..5d465b19d195d 100644 --- a/src/meta/src/stream/stream_graph/schedule.rs +++ b/src/meta/src/stream/stream_graph/schedule.rs @@ -27,7 +27,7 @@ use itertools::Itertools; use risingwave_common::bitmap::Bitmap; use risingwave_common::hash::{ActorMapping, VirtualNode, WorkerSlotId, WorkerSlotMapping}; use risingwave_common::{bail, hash}; -use risingwave_meta_model_v2::WorkerId; +use risingwave_meta_model::WorkerId; use risingwave_pb::common::{ActorInfo, WorkerNode}; use risingwave_pb::meta::table_fragments::fragment::{ FragmentDistributionType, PbFragmentDistributionType, diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 218fc09b5189a..21a642f1be1c7 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -19,7 +19,7 @@ use futures::future::join_all; use itertools::Itertools; use risingwave_common::bail; use risingwave_common::catalog::TableId; -use risingwave_meta_model_v2::{ObjectId, WorkerId}; +use risingwave_meta_model::{ObjectId, WorkerId}; use risingwave_pb::catalog::{CreateType, Subscription, Table}; use risingwave_pb::stream_plan::update_mutation::MergeUpdate; use risingwave_pb::stream_plan::Dispatcher; @@ -489,7 +489,7 @@ impl GlobalStreamManager { &self, removed_actors: Vec, streaming_job_ids: Vec, - state_table_ids: Vec, + state_table_ids: Vec, fragment_ids: HashSet, ) { if !removed_actors.is_empty() diff --git a/src/prost/build.rs b/src/prost/build.rs index ee04705ef19e5..194d565e31508 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -170,7 +170,7 @@ fn main() -> Result<(), Box> { .type_attribute("common.ColumnOrder", "#[derive(Eq, Hash)]") .type_attribute("common.OrderType", "#[derive(Eq, Hash)]") .type_attribute("common.Buffer", "#[derive(Eq)]") - // Eq is required to derive `FromJsonQueryResult` for models in risingwave_meta_model_v2. + // Eq is required to derive `FromJsonQueryResult` for models in risingwave_meta_model. .type_attribute("hummock.TableStats", "#[derive(Eq)]") .type_attribute("hummock.SstableInfo", "#[derive(Eq)]") .type_attribute("hummock.KeyRange", "#[derive(Eq)]") diff --git a/src/storage/backup/Cargo.toml b/src/storage/backup/Cargo.toml index ef994a38c4a15..3fd7d7ecde5ce 100644 --- a/src/storage/backup/Cargo.toml +++ b/src/storage/backup/Cargo.toml @@ -23,7 +23,7 @@ parking_lot = { workspace = true } prost = { workspace = true } risingwave_common = { workspace = true } risingwave_hummock_sdk = { workspace = true } -risingwave_meta_model_v2 = { workspace = true } +risingwave_meta_model = { workspace = true } risingwave_object_store = { workspace = true } risingwave_pb = { workspace = true } serde = { version = "1", features = ["derive"] } diff --git a/src/storage/backup/src/meta_snapshot_v2.rs b/src/storage/backup/src/meta_snapshot_v2.rs index 6afb90b1258dd..4fea523916fff 100644 --- a/src/storage/backup/src/meta_snapshot_v2.rs +++ b/src/storage/backup/src/meta_snapshot_v2.rs @@ -33,35 +33,35 @@ impl From for BackupError { macro_rules! for_all_metadata_models_v2 { ($macro:ident) => { $macro! { - {seaql_migrations, risingwave_meta_model_v2::serde_seaql_migration}, - {version_stats, risingwave_meta_model_v2::hummock_version_stats}, - {compaction_configs, risingwave_meta_model_v2::compaction_config}, - {actors, risingwave_meta_model_v2::actor}, - {clusters, risingwave_meta_model_v2::cluster}, - {actor_dispatchers, risingwave_meta_model_v2::actor_dispatcher}, - {catalog_versions, risingwave_meta_model_v2::catalog_version}, - {connections, risingwave_meta_model_v2::connection}, - {databases, risingwave_meta_model_v2::database}, - {fragments, risingwave_meta_model_v2::fragment}, - {functions, risingwave_meta_model_v2::function}, - {indexes, risingwave_meta_model_v2::index}, - {objects, risingwave_meta_model_v2::object}, - {object_dependencies, risingwave_meta_model_v2::object_dependency}, - {schemas, risingwave_meta_model_v2::schema}, - {sinks, risingwave_meta_model_v2::sink}, - {sources, risingwave_meta_model_v2::source}, - {streaming_jobs, risingwave_meta_model_v2::streaming_job}, - {subscriptions, risingwave_meta_model_v2::subscription}, - {system_parameters, risingwave_meta_model_v2::system_parameter}, - {tables, risingwave_meta_model_v2::table}, - {users, risingwave_meta_model_v2::user}, - {user_privileges, risingwave_meta_model_v2::user_privilege}, - {views, risingwave_meta_model_v2::view}, - {workers, risingwave_meta_model_v2::worker}, - {worker_properties, risingwave_meta_model_v2::worker_property}, - {hummock_sequences, risingwave_meta_model_v2::hummock_sequence}, - {session_parameters, risingwave_meta_model_v2::session_parameter}, - {secrets, risingwave_meta_model_v2::secret} + {seaql_migrations, risingwave_meta_model::serde_seaql_migration}, + {version_stats, risingwave_meta_model::hummock_version_stats}, + {compaction_configs, risingwave_meta_model::compaction_config}, + {actors, risingwave_meta_model::actor}, + {clusters, risingwave_meta_model::cluster}, + {actor_dispatchers, risingwave_meta_model::actor_dispatcher}, + {catalog_versions, risingwave_meta_model::catalog_version}, + {connections, risingwave_meta_model::connection}, + {databases, risingwave_meta_model::database}, + {fragments, risingwave_meta_model::fragment}, + {functions, risingwave_meta_model::function}, + {indexes, risingwave_meta_model::index}, + {objects, risingwave_meta_model::object}, + {object_dependencies, risingwave_meta_model::object_dependency}, + {schemas, risingwave_meta_model::schema}, + {sinks, risingwave_meta_model::sink}, + {sources, risingwave_meta_model::source}, + {streaming_jobs, risingwave_meta_model::streaming_job}, + {subscriptions, risingwave_meta_model::subscription}, + {system_parameters, risingwave_meta_model::system_parameter}, + {tables, risingwave_meta_model::table}, + {users, risingwave_meta_model::user}, + {user_privileges, risingwave_meta_model::user_privilege}, + {views, risingwave_meta_model::view}, + {workers, risingwave_meta_model::worker}, + {worker_properties, risingwave_meta_model::worker_property}, + {hummock_sequences, risingwave_meta_model::hummock_sequence}, + {session_parameters, risingwave_meta_model::session_parameter}, + {secrets, risingwave_meta_model::secret} } }; } From fb1edb60c7a1aaed794aad8fd1d93ad4255e04f8 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Thu, 17 Oct 2024 17:42:30 +0800 Subject: [PATCH 4/4] chore(test): longer streaming_upload_attempt_timeout_ms for backfill test only (#18960) --- src/config/ci-backfill.toml | 2 +- src/config/ci-longer-streaming-upload-timeout.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/config/ci-backfill.toml b/src/config/ci-backfill.toml index 55b6149df3ebb..7d0660e8377fe 100644 --- a/src/config/ci-backfill.toml +++ b/src/config/ci-backfill.toml @@ -5,7 +5,7 @@ telemetry_enabled = false stream_chunk_size = 10 [storage.object_store.retry] -streaming_upload_attempt_timeout_ms = 20000 +streaming_upload_attempt_timeout_ms = 30000 [system] # NOTE(kwannoel): If can't reproduce it, set to a lower number. diff --git a/src/config/ci-longer-streaming-upload-timeout.toml b/src/config/ci-longer-streaming-upload-timeout.toml index 90bd3790527d2..5510ffc51c17e 100644 --- a/src/config/ci-longer-streaming-upload-timeout.toml +++ b/src/config/ci-longer-streaming-upload-timeout.toml @@ -17,7 +17,7 @@ stream_exchange_concurrent_barriers = 10 imm_merge_threshold = 2 [storage.object_store.retry] -streaming_upload_attempt_timeout_ms = 20000 +streaming_upload_attempt_timeout_ms = 30000 upload_retry_attempts = 5 read_attempt_timeout_ms = 16000 read_retry_attempts = 6