Skip to content

feat(frontend): derive columns from iceberg source automatically #15415

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

Merged
merged 1 commit into from
Mar 4, 2024
Merged
Show file tree
Hide file tree
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
1 change: 1 addition & 0 deletions src/common/src/array/arrow/arrow_impl.rs
Original file line number Diff line number Diff line change
Expand Up @@ -222,6 +222,7 @@ impl From<&arrow_schema::DataType> for DataType {
LargeUtf8 => Self::Jsonb,
Struct(fields) => Self::Struct(fields.into()),
List(field) => Self::List(Box::new(field.data_type().into())),
Decimal128(_, _) => Self::Decimal,
_ => todo!("Unsupported arrow data type: {value:?}"),
}
}
Expand Down
61 changes: 56 additions & 5 deletions src/frontend/src/handler/create_source.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@ use maplit::{convert_args, hashmap};
use pgwire::pg_response::{PgResponse, StatementType};
use risingwave_common::bail_not_implemented;
use risingwave_common::catalog::{
is_column_ids_dedup, ColumnCatalog, ColumnDesc, Schema, TableId, INITIAL_SOURCE_VERSION_ID,
KAFKA_TIMESTAMP_COLUMN_NAME,
is_column_ids_dedup, ColumnCatalog, ColumnDesc, ColumnId, Schema, TableId,
INITIAL_SOURCE_VERSION_ID, KAFKA_TIMESTAMP_COLUMN_NAME,
};
use risingwave_common::types::DataType;
use risingwave_connector::parser::additional_columns::{
Expand All @@ -37,6 +37,7 @@ use risingwave_connector::parser::{
use risingwave_connector::schema::schema_registry::{
name_strategy_from_str, SchemaRegistryAuth, SCHEMA_REGISTRY_PASSWORD, SCHEMA_REGISTRY_USERNAME,
};
use risingwave_connector::sink::iceberg::IcebergConfig;
use risingwave_connector::source::cdc::external::CdcTableType;
use risingwave_connector::source::cdc::{
CDC_SHARING_MODE_KEY, CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY, CDC_TRANSACTIONAL_KEY,
Expand Down Expand Up @@ -67,7 +68,6 @@ use thiserror_ext::AsReport;
use super::RwPgResponse;
use crate::binder::Binder;
use crate::catalog::source_catalog::SourceCatalog;
use crate::catalog::ColumnId;
use crate::error::ErrorCode::{self, InvalidInputSyntax, NotSupported, ProtocolError};
use crate::error::{Result, RwError};
use crate::expr::Expr;
Expand All @@ -76,7 +76,8 @@ use crate::handler::create_table::{
ensure_table_constraints_supported, ColumnIdGenerator,
};
use crate::handler::util::{
connector_need_pk, get_connector, is_cdc_connector, is_kafka_connector, SourceSchemaCompatExt,
connector_need_pk, get_connector, is_cdc_connector, is_iceberg_connector, is_kafka_connector,
SourceSchemaCompatExt,
};
use crate::handler::HandlerArgs;
use crate::optimizer::plan_node::generic::SourceNodeKind;
Expand Down Expand Up @@ -333,7 +334,6 @@ pub(crate) async fn bind_columns_from_source(

let columns = match (&source_schema.format, &source_schema.row_encode) {
(Format::Native, Encode::Native)
| (Format::None, Encode::None)
| (Format::Plain, Encode::Bytes)
| (Format::DebeziumMongo, Encode::Json) => None,
(Format::Plain, Encode::Protobuf) => {
Expand Down Expand Up @@ -444,6 +444,17 @@ pub(crate) async fn bind_columns_from_source(
)
.await?
}
(Format::None, Encode::None) => {
if is_iceberg_connector(with_properties) {
Some(
extract_iceberg_columns(with_properties)
.await
.map_err(|err| ProtocolError(err.to_report_string()))?,
)
} else {
None
}
}
(format, encoding) => {
return Err(RwError::from(ProtocolError(format!(
"Unknown combination {:?} {:?}",
Expand Down Expand Up @@ -1158,6 +1169,46 @@ pub(super) fn check_nexmark_schema(
Ok(())
}

pub async fn extract_iceberg_columns(
with_properties: &HashMap<String, String>,
) -> anyhow::Result<Vec<ColumnCatalog>> {
let props = ConnectorProperties::extract(with_properties.clone(), true)?;
if let ConnectorProperties::Iceberg(properties) = props {
let iceberg_config: IcebergConfig = properties.to_iceberg_config();
let table = iceberg_config.load_table().await?;
let iceberg_schema: arrow_schema::Schema = table
.current_table_metadata()
.current_schema()?
.clone()
.try_into()?;

let columns = iceberg_schema
.fields()
.iter()
.enumerate()
.map(|(i, field)| {
let data_type = field.data_type().clone();
let column_desc = ColumnDesc::named(
field.name(),
ColumnId::new((i as u32).try_into().unwrap()),
data_type.into(),
);
ColumnCatalog {
column_desc,
is_hidden: false,
}
})
.collect();

Ok(columns)
} else {
Err(anyhow!(format!(
"Invalid properties for iceberg source: {:?}",
props
)))
}
}

pub async fn check_iceberg_source(
props: &HashMap<String, String>,
columns: &[ColumnCatalog],
Expand Down
6 changes: 4 additions & 2 deletions src/sqlparser/src/ast/statement.rs
Original file line number Diff line number Diff line change
Expand Up @@ -133,9 +133,10 @@ impl Format {
"PLAIN" => Format::Plain,
"UPSERT" => Format::Upsert,
"NATIVE" => Format::Native, // used internally for schema change
"NONE" => Format::None, // used by iceberg
_ => {
return Err(ParserError::ParserError(
"expected CANAL | PROTOBUF | DEBEZIUM | MAXWELL | PLAIN | NATIVE after FORMAT"
"expected CANAL | PROTOBUF | DEBEZIUM | MAXWELL | PLAIN | NATIVE | NONE after FORMAT"
.to_string(),
))
}
Expand Down Expand Up @@ -186,8 +187,9 @@ impl Encode {
"JSON" => Encode::Json,
"TEMPLATE" => Encode::Template,
"NATIVE" => Encode::Native, // used internally for schema change
"NONE" => Encode::None, // used by iceberg
_ => return Err(ParserError::ParserError(
"expected AVRO | BYTES | CSV | PROTOBUF | JSON | NATIVE | TEMPLATE after Encode"
"expected AVRO | BYTES | CSV | PROTOBUF | JSON | NATIVE | TEMPLATE | NONE after Encode"
.to_string(),
)),
})
Expand Down