Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat(frontend): support create iceberg source #14971

Merged
merged 18 commits into from
Feb 23, 2024
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions proto/plan_common.proto
Original file line number Diff line number Diff line change
@@ -135,6 +135,7 @@ enum FormatType {
FORMAT_TYPE_CANAL = 5;
FORMAT_TYPE_UPSERT = 6;
FORMAT_TYPE_PLAIN = 7;
FORMAT_TYPE_NONE = 8;
}

enum EncodeType {
@@ -146,6 +147,7 @@ enum EncodeType {
ENCODE_TYPE_JSON = 5;
ENCODE_TYPE_BYTES = 6;
ENCODE_TYPE_TEMPLATE = 7;
ENCODE_TYPE_NONE = 8;
}

enum RowFormatType {
3 changes: 2 additions & 1 deletion src/connector/src/macros.rs
Original file line number Diff line number Diff line change
@@ -36,7 +36,8 @@ macro_rules! for_all_classified_sources {
{ Gcs, $crate::source::filesystem::opendal_source::GcsProperties , $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalGcs> },
{ OpendalS3, $crate::source::filesystem::opendal_source::OpendalS3Properties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalS3> },
{ PosixFs, $crate::source::filesystem::opendal_source::PosixFsProperties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalPosixFs> },
{ Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit}
{ Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit},
{ Iceberg, $crate::source::iceberg::IcebergProperties, $crate::source::iceberg::IcebergSplit}
}
$(
,$extra_args
9 changes: 7 additions & 2 deletions src/connector/src/sink/catalog/mod.rs
Original file line number Diff line number Diff line change
@@ -205,7 +205,12 @@ impl TryFrom<PbSinkFormatDesc> for SinkFormatDesc {
F::Plain => SinkFormat::AppendOnly,
F::Upsert => SinkFormat::Upsert,
F::Debezium => SinkFormat::Debezium,
f @ (F::Unspecified | F::Native | F::DebeziumMongo | F::Maxwell | F::Canal) => {
f @ (F::Unspecified
| F::Native
| F::DebeziumMongo
| F::Maxwell
| F::Canal
| F::None) => {
tabVersion marked this conversation as resolved.
Show resolved Hide resolved
return Err(SinkError::Config(anyhow!(
"sink format unsupported: {}",
f.as_str_name()
@@ -217,7 +222,7 @@ impl TryFrom<PbSinkFormatDesc> for SinkFormatDesc {
E::Protobuf => SinkEncode::Protobuf,
E::Template => SinkEncode::Template,
E::Avro => SinkEncode::Avro,
e @ (E::Unspecified | E::Native | E::Csv | E::Bytes) => {
e @ (E::Unspecified | E::Native | E::Csv | E::Bytes | E::None) => {
return Err(SinkError::Config(anyhow!(
"sink encode unsupported: {}",
e.as_str_name()
2 changes: 1 addition & 1 deletion src/connector/src/sink/iceberg/mod.rs
Original file line number Diff line number Diff line change
@@ -926,7 +926,7 @@ impl SinkCommitCoordinator for IcebergSinkCommitter {
}

/// Try to match our schema with iceberg schema.
fn try_matches_arrow_schema(rw_schema: &Schema, arrow_schema: &ArrowSchema) -> Result<()> {
pub fn try_matches_arrow_schema(rw_schema: &Schema, arrow_schema: &ArrowSchema) -> Result<()> {
if rw_schema.fields.len() != arrow_schema.fields().len() {
return Err(SinkError::Iceberg(anyhow!(
"Schema length not match, ours is {}, and iceberg is {}",
2 changes: 1 addition & 1 deletion src/connector/src/source/base.rs
Original file line number Diff line number Diff line change
@@ -150,7 +150,7 @@ pub struct SourceEnumeratorContext {
pub connector_client: Option<ConnectorClient>,
}

#[derive(Clone, Copy, Debug, Default)]
#[derive(Clone, Debug, Default)]
pub struct SourceEnumeratorInfo {
pub source_id: u32,
}
128 changes: 128 additions & 0 deletions src/connector/src/source/iceberg/mod.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,128 @@
// 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 std::collections::HashMap;

use async_trait::async_trait;
use risingwave_common::types::JsonbVal;
use serde::{Deserialize, Serialize};

use crate::parser::ParserConfig;
use crate::source::{
BoxChunkSourceStream, Column, SourceContextRef, SourceEnumeratorContextRef, SourceProperties,
SplitEnumerator, SplitId, SplitMetaData, SplitReader, UnknownFields,
};

pub const ICEBERG_CONNECTOR: &str = "iceberg";

#[derive(Clone, Debug, Deserialize, PartialEq, with_options::WithOptions)]
pub struct IcebergProperties {
#[serde(rename = "catalog.type")]
pub catalog_type: String,
#[serde(rename = "s3.region")]
pub region_name: String,
#[serde(rename = "s3.endpoint", default)]
pub endpoint: String,
#[serde(rename = "s3.access.key", default)]
pub s3_access: String,
#[serde(rename = "s3.secret.key", default)]
pub s3_secret: String,
#[serde(rename = "warehouse.path")]
pub warehouse_path: String,
#[serde(rename = "database.name")]
pub database_name: String,
#[serde(rename = "table.name")]
pub table_name: String,

#[serde(flatten)]
pub unknown_fields: HashMap<String, String>,
}

impl SourceProperties for IcebergProperties {
type Split = IcebergSplit;
type SplitEnumerator = IcebergSplitEnumerator;
type SplitReader = IcebergFileReader;

const SOURCE_NAME: &'static str = ICEBERG_CONNECTOR;
}

impl UnknownFields for IcebergProperties {
fn unknown_fields(&self) -> HashMap<String, String> {
self.unknown_fields.clone()
}
}

#[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct IcebergSplit {}

impl SplitMetaData for IcebergSplit {
fn id(&self) -> SplitId {
unimplemented!()
}

fn restore_from_json(_value: JsonbVal) -> anyhow::Result<Self> {
unimplemented!()
}

fn encode_to_json(&self) -> JsonbVal {
unimplemented!()
}

fn update_with_offset(&mut self, _start_offset: String) -> anyhow::Result<()> {
unimplemented!()
}
Comment on lines +70 to +84
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

are these going to be impl? or left them here on purpose

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Some of them would be implemented for batch-read in a later PR. But most of them related to streaming are left unimplemented on purpose.

}

#[derive(Debug, Clone)]
pub struct IcebergSplitEnumerator {}

#[async_trait]
impl SplitEnumerator for IcebergSplitEnumerator {
type Properties = IcebergProperties;
type Split = IcebergSplit;

async fn new(
_properties: Self::Properties,
_context: SourceEnumeratorContextRef,
) -> anyhow::Result<Self> {
Ok(Self {})
}

async fn list_splits(&mut self) -> anyhow::Result<Vec<Self::Split>> {
Ok(vec![])
}
}

#[derive(Debug)]
pub struct IcebergFileReader {}

#[async_trait]
impl SplitReader for IcebergFileReader {
type Properties = IcebergProperties;
type Split = IcebergSplit;

async fn new(
_props: IcebergProperties,
_splits: Vec<IcebergSplit>,
_parser_config: ParserConfig,
_source_ctx: SourceContextRef,
_columns: Option<Vec<Column>>,
) -> anyhow::Result<Self> {
unimplemented!()
}

fn into_stream(self) -> BoxChunkSourceStream {
unimplemented!()
}
}
1 change: 1 addition & 0 deletions src/connector/src/source/mod.rs
Original file line number Diff line number Diff line change
@@ -31,6 +31,7 @@ pub use kafka::KAFKA_CONNECTOR;
pub use kinesis::KINESIS_CONNECTOR;
pub use nats::NATS_CONNECTOR;
mod common;
pub mod iceberg;
mod manager;
pub mod reader;
pub mod test_source;
29 changes: 29 additions & 0 deletions src/connector/with_options_source.yaml
Original file line number Diff line number Diff line change
@@ -33,6 +33,35 @@ GcsProperties:
field_type: String
required: false
default: Default::default
IcebergProperties:
fields:
- name: catalog.type
field_type: String
required: true
- name: s3.region
field_type: String
required: true
- name: s3.endpoint
field_type: String
required: false
default: Default::default
- name: s3.access.key
field_type: String
required: false
default: Default::default
- name: s3.secret.key
field_type: String
required: false
default: Default::default
- name: warehouse.path
field_type: String
required: true
- name: database.name
field_type: String
required: true
- name: table.name
field_type: String
required: true
KafkaProperties:
fields:
- name: bytes.per.second
2 changes: 2 additions & 0 deletions src/frontend/src/handler/alter_source_with_sr.rs
Original file line number Diff line number Diff line change
@@ -42,6 +42,7 @@ fn format_type_to_format(from: FormatType) -> Option<Format> {
FormatType::Canal => Format::Canal,
FormatType::Upsert => Format::Upsert,
FormatType::Plain => Format::Plain,
FormatType::None => Format::None,
})
}

@@ -55,6 +56,7 @@ fn encode_type_to_encode(from: EncodeType) -> Option<Encode> {
EncodeType::Json => Encode::Json,
EncodeType::Bytes => Encode::Bytes,
EncodeType::Template => Encode::Template,
EncodeType::None => Encode::None,
})
}

4 changes: 2 additions & 2 deletions src/frontend/src/handler/create_sink.rs
Original file line number Diff line number Diff line change
@@ -693,7 +693,7 @@ fn bind_sink_format_desc(value: ConnectorSchema) -> Result<SinkFormatDesc> {
F::Plain => SinkFormat::AppendOnly,
F::Upsert => SinkFormat::Upsert,
F::Debezium => SinkFormat::Debezium,
f @ (F::Native | F::DebeziumMongo | F::Maxwell | F::Canal) => {
f @ (F::Native | F::DebeziumMongo | F::Maxwell | F::Canal | F::None) => {
return Err(ErrorCode::BindError(format!("sink format unsupported: {f}")).into());
}
};
@@ -702,7 +702,7 @@ fn bind_sink_format_desc(value: ConnectorSchema) -> Result<SinkFormatDesc> {
E::Protobuf => SinkEncode::Protobuf,
E::Avro => SinkEncode::Avro,
E::Template => SinkEncode::Template,
e @ (E::Native | E::Csv | E::Bytes) => {
e @ (E::Native | E::Csv | E::Bytes | E::None) => {
return Err(ErrorCode::BindError(format!("sink encode unsupported: {e}")).into());
}
};
Loading