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(batch,meta): support iceberg snapshots sys table #16175

Merged
merged 5 commits into from
Apr 9, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
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 Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

10 changes: 9 additions & 1 deletion src/connector/src/source/iceberg/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -210,7 +210,15 @@ impl IcebergSplitEnumerator {
},
};
let mut files = vec![];
for file in table.current_data_files().await? {
for file in table
.data_files_of_snapshot(
table
.current_table_metadata()
.snapshot(snapshot_id)
.expect("snapshot must exists"),
)
.await?
{
if file.content != DataContentType::Data {
bail!("Reading iceberg table with delete files is unsupported. Please try to compact the table first.");
}
Expand Down
1 change: 1 addition & 0 deletions src/frontend/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ futures-async-stream = { workspace = true }
iana-time-zone = "0.1"
icelake = { workspace = true }
itertools = "0.12"
jsonbb = "0.1.2"
linkme = { version = "0.3", features = ["used_linker"] }
maplit = "1"
md5 = "0.7.0"
Expand Down
1 change: 1 addition & 0 deletions src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ mod rw_hummock_pinned_snapshots;
mod rw_hummock_pinned_versions;
mod rw_hummock_version;
mod rw_hummock_version_deltas;
mod rw_iceberg_snapshots;
mod rw_indexes;
mod rw_internal_tables;
mod rw_materialized_views;
Expand Down
Copy link
Member

Choose a reason for hiding this comment

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

Is it possible to add some tests in E2E tests?

Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
// 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 std::ops::Deref;

use icelake::Table;
use jsonbb::{Value, ValueRef};
use risingwave_common::types::{Fields, JsonbVal, Timestamptz};
use risingwave_connector::sink::iceberg::IcebergConfig;
use risingwave_connector::source::ConnectorProperties;
use risingwave_connector::WithPropertiesExt;
use risingwave_frontend_macro::system_catalog;

use crate::catalog::system_catalog::SysCatalogReaderImpl;
use crate::error::Result;

#[derive(Fields)]
struct RwIcebergSnapshots {
#[primary_key]
source_id: i32,
schema_name: String,
source_name: String,
sequence_number: i64,
snapshot_id: i64,
timestamp_ms: Option<Timestamptz>,
manifest_list: String,
summary: JsonbVal,
}

#[system_catalog(table, "rw_catalog.rw_iceberg_snapshots")]
async fn read(reader: &SysCatalogReaderImpl) -> Result<Vec<RwIcebergSnapshots>> {
let iceberg_sources = {
let catalog_reader = reader.catalog_reader.read_guard();
let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?;

let mut iceberg_sources = vec![];
for schema in schemas {
for source in schema.iter_source() {
if source.with_properties.is_iceberg_connector() {
iceberg_sources.push((schema.name.clone(), source.deref().clone()))
}
}
}
iceberg_sources
Copy link
Member

Choose a reason for hiding this comment

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

It doesn't consider permissions, but sounds okay to me

};

let mut result = vec![];
for (schema_name, source) in iceberg_sources {
let source_props: HashMap<String, String> =
HashMap::from_iter(source.with_properties.clone());
let config = ConnectorProperties::extract(source_props, false)?;
if let ConnectorProperties::Iceberg(iceberg_properties) = config {
let iceberg_config: IcebergConfig = iceberg_properties.to_iceberg_config();
let table: Table = iceberg_config.load_table().await?;
if let Some(snapshots) = &table.current_table_metadata().snapshots {
result.extend(snapshots.iter().map(|snapshot| {
RwIcebergSnapshots {
source_id: source.id as i32,
schema_name: schema_name.clone(),
source_name: source.name.clone(),
sequence_number: snapshot.sequence_number,
snapshot_id: snapshot.snapshot_id,
timestamp_ms: Timestamptz::from_millis(snapshot.timestamp_ms),
manifest_list: snapshot.manifest_list.clone(),
summary: Value::object(
snapshot
.summary
.iter()
.map(|(k, v)| (k.as_str(), ValueRef::String(v))),
)
.into(),
}
}));
}
}
}
Ok(result)
}
6 changes: 5 additions & 1 deletion src/frontend/src/scheduler/plan_fragmenter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -342,7 +342,11 @@ impl SourceScanInfo {
}
Some(AsOf::TimestampString(ts)) => Some(
speedate::DateTime::parse_str_rfc3339(&ts)
.map(|t| IcebergTimeTravelInfo::TimestampMs(t.timestamp_tz() * 1000))
.map(|t| {
IcebergTimeTravelInfo::TimestampMs(
t.timestamp_tz() * 1000 + t.time.microsecond as i64 / 1000,
)
})
.map_err(|_e| anyhow!("fail to parse timestamp"))?,
),
Some(AsOf::ProcessTime) => unreachable!(),
Expand Down
Loading