1use core::num::NonZeroU64;
16use std::collections::{BTreeMap, HashMap};
17use std::sync::Arc;
18
19use anyhow::{Context, anyhow};
20use async_trait::async_trait;
21use deltalake::DeltaTable;
22use deltalake::aws::storage::s3_constants::{
23 AWS_ACCESS_KEY_ID, AWS_ALLOW_HTTP, AWS_ENDPOINT_URL, AWS_REGION, AWS_S3_ALLOW_UNSAFE_RENAME,
24 AWS_SECRET_ACCESS_KEY,
25};
26use deltalake::kernel::transaction::CommitBuilder;
27use deltalake::kernel::{Action, Add, DataType as DeltaLakeDataType, PrimitiveType, StructType};
28use deltalake::protocol::{DeltaOperation, SaveMode};
29use deltalake::writer::{DeltaWriter, RecordBatchWriter};
30use phf::{Set, phf_set};
31use risingwave_common::array::StreamChunk;
32use risingwave_common::array::arrow::DeltaLakeConvert;
33use risingwave_common::bail;
34use risingwave_common::catalog::{Field, Schema};
35use risingwave_common::types::DataType;
36use risingwave_common::util::iter_util::ZipEqDebug;
37use risingwave_pb::connector_service::SinkMetadata;
38use risingwave_pb::connector_service::sink_metadata::Metadata::Serialized;
39use risingwave_pb::connector_service::sink_metadata::SerializedMetadata;
40use serde::{Deserialize, Serialize};
41use serde_with::{DisplayFromStr, serde_as};
42use tokio::sync::mpsc::UnboundedSender;
43use with_options::WithOptions;
44
45use crate::connector_common::{AwsAuthProps, IcebergSinkCompactionUpdate};
46use crate::enforce_secret::{EnforceSecret, EnforceSecretError};
47use crate::sink::coordinate::CoordinatedLogSinker;
48use crate::sink::decouple_checkpoint_log_sink::default_commit_checkpoint_interval;
49use crate::sink::writer::SinkWriter;
50use crate::sink::{
51 Result, SINK_TYPE_APPEND_ONLY, SINK_USER_FORCE_APPEND_ONLY_OPTION,
52 SinglePhaseCommitCoordinator, Sink, SinkCommitCoordinator, SinkError, SinkParam,
53 SinkWriterParam,
54};
55
56pub const DEFAULT_REGION: &str = "us-east-1";
57pub const GCS_SERVICE_ACCOUNT: &str = "service_account_key";
58
59pub const DELTALAKE_SINK: &str = "deltalake";
60
61#[serde_as]
62#[derive(Deserialize, Debug, Clone, WithOptions)]
63pub struct DeltaLakeCommon {
64 #[serde(rename = "location")]
65 pub location: String,
66 #[serde(flatten)]
67 pub aws_auth_props: AwsAuthProps,
68
69 #[serde(rename = "gcs.service.account")]
70 pub gcs_service_account: Option<String>,
71 #[serde(default = "default_commit_checkpoint_interval")]
73 #[serde_as(as = "DisplayFromStr")]
74 #[with_option(allow_alter_on_fly)]
75 pub commit_checkpoint_interval: u64,
76}
77
78impl EnforceSecret for DeltaLakeCommon {
79 const ENFORCE_SECRET_PROPERTIES: Set<&'static str> = phf_set! {
80 "gcs.service.account",
81 };
82
83 fn enforce_one(prop: &str) -> crate::error::ConnectorResult<()> {
84 AwsAuthProps::enforce_one(prop)?;
85 if Self::ENFORCE_SECRET_PROPERTIES.contains(prop) {
86 return Err(EnforceSecretError {
87 key: prop.to_owned(),
88 }
89 .into());
90 }
91
92 Ok(())
93 }
94}
95
96impl DeltaLakeCommon {
97 pub async fn create_deltalake_client(&self) -> Result<DeltaTable> {
98 let table = match Self::get_table_url(&self.location)? {
99 DeltaTableUrl::S3(s3_path) => {
100 let storage_options = self.build_delta_lake_config_for_aws().await?;
101 deltalake::aws::register_handlers(None);
102 deltalake::open_table_with_storage_options(&s3_path, storage_options).await?
103 }
104 DeltaTableUrl::Local(local_path) => deltalake::open_table(local_path).await?,
105 DeltaTableUrl::Gcs(gcs_path) => {
106 let mut storage_options = HashMap::new();
107 storage_options.insert(
108 GCS_SERVICE_ACCOUNT.to_owned(),
109 self.gcs_service_account.clone().ok_or_else(|| {
110 SinkError::Config(anyhow!(
111 "gcs.service.account is required with Google Cloud Storage (GCS)"
112 ))
113 })?,
114 );
115 deltalake::gcp::register_handlers(None);
116 deltalake::open_table_with_storage_options(gcs_path.clone(), storage_options)
117 .await?
118 }
119 };
120 Ok(table)
121 }
122
123 fn get_table_url(path: &str) -> Result<DeltaTableUrl> {
124 if path.starts_with("s3://") || path.starts_with("s3a://") {
125 Ok(DeltaTableUrl::S3(path.to_owned()))
126 } else if path.starts_with("gs://") {
127 Ok(DeltaTableUrl::Gcs(path.to_owned()))
128 } else if let Some(path) = path.strip_prefix("file://") {
129 Ok(DeltaTableUrl::Local(path.to_owned()))
130 } else {
131 Err(SinkError::DeltaLake(anyhow!(
132 "path should start with 's3://','s3a://'(s3) ,gs://(gcs) or file://(local)"
133 )))
134 }
135 }
136
137 async fn build_delta_lake_config_for_aws(&self) -> Result<HashMap<String, String>> {
138 let mut storage_options = HashMap::new();
139 storage_options.insert(AWS_ALLOW_HTTP.to_owned(), "true".to_owned());
140 storage_options.insert(AWS_S3_ALLOW_UNSAFE_RENAME.to_owned(), "true".to_owned());
141 let sdk_config = self.aws_auth_props.build_config().await?;
142 let credentials = sdk_config
143 .credentials_provider()
144 .ok_or_else(|| {
145 SinkError::Config(anyhow!(
146 "s3.access.key and s3.secret.key is required with aws s3"
147 ))
148 })?
149 .as_ref()
150 .provide_credentials()
151 .await
152 .map_err(|e| SinkError::Config(e.into()))?;
153 let region = sdk_config.region();
154 let endpoint = sdk_config.endpoint_url();
155 storage_options.insert(
156 AWS_ACCESS_KEY_ID.to_owned(),
157 credentials.access_key_id().to_owned(),
158 );
159 storage_options.insert(
160 AWS_SECRET_ACCESS_KEY.to_owned(),
161 credentials.secret_access_key().to_owned(),
162 );
163 if endpoint.is_none() && region.is_none() {
164 return Err(SinkError::Config(anyhow!(
165 "s3.endpoint and s3.region need to be filled with at least one"
166 )));
167 }
168 storage_options.insert(
169 AWS_REGION.to_owned(),
170 region
171 .map(|r| r.as_ref().to_owned())
172 .unwrap_or_else(|| DEFAULT_REGION.to_owned()),
173 );
174 if let Some(s3_endpoint) = endpoint {
175 storage_options.insert(AWS_ENDPOINT_URL.to_owned(), s3_endpoint.to_owned());
176 }
177 Ok(storage_options)
178 }
179}
180
181enum DeltaTableUrl {
182 S3(String),
183 Local(String),
184 Gcs(String),
185}
186
187#[serde_as]
188#[derive(Clone, Debug, Deserialize, WithOptions)]
189pub struct DeltaLakeConfig {
190 #[serde(flatten)]
191 pub common: DeltaLakeCommon,
192
193 pub r#type: String,
194}
195
196impl EnforceSecret for DeltaLakeConfig {
197 fn enforce_one(prop: &str) -> crate::error::ConnectorResult<()> {
198 DeltaLakeCommon::enforce_one(prop)
199 }
200}
201
202impl DeltaLakeConfig {
203 pub fn from_btreemap(properties: BTreeMap<String, String>) -> Result<Self> {
204 let config = serde_json::from_value::<DeltaLakeConfig>(
205 serde_json::to_value(properties).map_err(|e| SinkError::DeltaLake(e.into()))?,
206 )
207 .map_err(|e| SinkError::Config(anyhow!(e)))?;
208 Ok(config)
209 }
210}
211
212#[derive(Debug)]
213pub struct DeltaLakeSink {
214 pub config: DeltaLakeConfig,
215 param: SinkParam,
216}
217
218impl EnforceSecret for DeltaLakeSink {
219 fn enforce_secret<'a>(
220 prop_iter: impl Iterator<Item = &'a str>,
221 ) -> crate::error::ConnectorResult<()> {
222 for prop in prop_iter {
223 DeltaLakeCommon::enforce_one(prop)?;
224 }
225 Ok(())
226 }
227}
228
229impl DeltaLakeSink {
230 pub fn new(config: DeltaLakeConfig, param: SinkParam) -> Result<Self> {
231 Ok(Self { config, param })
232 }
233}
234
235fn check_field_type(rw_data_type: &DataType, dl_data_type: &DeltaLakeDataType) -> Result<bool> {
236 let result = match rw_data_type {
237 DataType::Boolean => {
238 matches!(
239 dl_data_type,
240 DeltaLakeDataType::Primitive(PrimitiveType::Boolean)
241 )
242 }
243 DataType::Int16 => {
244 matches!(
245 dl_data_type,
246 DeltaLakeDataType::Primitive(PrimitiveType::Short)
247 )
248 }
249 DataType::Int32 => {
250 matches!(
251 dl_data_type,
252 DeltaLakeDataType::Primitive(PrimitiveType::Integer)
253 )
254 }
255 DataType::Int64 => {
256 matches!(
257 dl_data_type,
258 DeltaLakeDataType::Primitive(PrimitiveType::Long)
259 )
260 }
261 DataType::Float32 => {
262 matches!(
263 dl_data_type,
264 DeltaLakeDataType::Primitive(PrimitiveType::Float)
265 )
266 }
267 DataType::Float64 => {
268 matches!(
269 dl_data_type,
270 DeltaLakeDataType::Primitive(PrimitiveType::Double)
271 )
272 }
273 DataType::Decimal => {
274 matches!(
275 dl_data_type,
276 DeltaLakeDataType::Primitive(PrimitiveType::Decimal(_))
277 )
278 }
279 DataType::Date => {
280 matches!(
281 dl_data_type,
282 DeltaLakeDataType::Primitive(PrimitiveType::Date)
283 )
284 }
285 DataType::Varchar => {
286 matches!(
287 dl_data_type,
288 DeltaLakeDataType::Primitive(PrimitiveType::String)
289 )
290 }
291 DataType::Timestamptz => {
292 matches!(
293 dl_data_type,
294 DeltaLakeDataType::Primitive(PrimitiveType::Timestamp)
295 )
296 }
297 DataType::Struct(rw_struct) => {
298 if let DeltaLakeDataType::Struct(dl_struct) = dl_data_type {
299 let mut result = true;
300 for ((rw_name, rw_type), dl_field) in
301 rw_struct.iter().zip_eq_debug(dl_struct.fields())
302 {
303 result = check_field_type(rw_type, dl_field.data_type())?
304 && result
305 && rw_name.eq(dl_field.name());
306 }
307 result
308 } else {
309 false
310 }
311 }
312 DataType::List(rw_list) => {
313 if let DeltaLakeDataType::Array(dl_list) = dl_data_type {
314 check_field_type(rw_list.elem(), dl_list.element_type())?
315 } else {
316 false
317 }
318 }
319 _ => {
320 return Err(SinkError::DeltaLake(anyhow!(
321 "Type {:?} is not supported for DeltaLake sink.",
322 rw_data_type.to_owned()
323 )));
324 }
325 };
326 Ok(result)
327}
328
329impl Sink for DeltaLakeSink {
330 type LogSinker = CoordinatedLogSinker<DeltaLakeSinkWriter>;
331
332 const SINK_NAME: &'static str = DELTALAKE_SINK;
333
334 async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result<Self::LogSinker> {
335 let inner = DeltaLakeSinkWriter::new(
336 self.config.clone(),
337 self.param.schema().clone(),
338 self.param.downstream_pk_or_empty(),
339 )
340 .await?;
341
342 let commit_checkpoint_interval =
343 NonZeroU64::new(self.config.common.commit_checkpoint_interval).expect(
344 "commit_checkpoint_interval should be greater than 0, and it should be checked in config validation",
345 );
346
347 let writer = CoordinatedLogSinker::new(
348 &writer_param,
349 self.param.clone(),
350 inner,
351 commit_checkpoint_interval,
352 )
353 .await?;
354
355 Ok(writer)
356 }
357
358 fn validate_alter_config(config: &BTreeMap<String, String>) -> Result<()> {
359 DeltaLakeConfig::from_btreemap(config.clone())?;
360 Ok(())
361 }
362
363 async fn validate(&self) -> Result<()> {
364 if self.config.r#type != SINK_TYPE_APPEND_ONLY
365 && self.config.r#type != SINK_USER_FORCE_APPEND_ONLY_OPTION
366 {
367 return Err(SinkError::Config(anyhow!(
368 "only append-only delta lake sink is supported",
369 )));
370 }
371 let table = self.config.common.create_deltalake_client().await?;
372 let deltalake_fields: HashMap<&String, &DeltaLakeDataType> = table
373 .get_schema()?
374 .fields()
375 .map(|f| (f.name(), f.data_type()))
376 .collect();
377 if deltalake_fields.len() != self.param.schema().fields().len() {
378 return Err(SinkError::DeltaLake(anyhow!(
379 "Columns mismatch. RisingWave schema has {} fields, DeltaLake schema has {} fields",
380 self.param.schema().fields().len(),
381 deltalake_fields.len()
382 )));
383 }
384 for field in self.param.schema().fields() {
385 if !deltalake_fields.contains_key(&field.name) {
386 return Err(SinkError::DeltaLake(anyhow!(
387 "column {} not found in deltalake table",
388 field.name
389 )));
390 }
391 let deltalake_field_type = deltalake_fields.get(&field.name).ok_or_else(|| {
392 SinkError::DeltaLake(anyhow!("cannot find field type for {}", field.name))
393 })?;
394 if !check_field_type(&field.data_type, deltalake_field_type)? {
395 return Err(SinkError::DeltaLake(anyhow!(
396 "column '{}' type mismatch: deltalake type is {:?}, RisingWave type is {:?}",
397 field.name,
398 deltalake_field_type,
399 field.data_type
400 )));
401 }
402 }
403 if self.config.common.commit_checkpoint_interval == 0 {
404 return Err(SinkError::Config(anyhow!(
405 "`commit_checkpoint_interval` must be greater than 0"
406 )));
407 }
408 Ok(())
409 }
410
411 fn is_coordinated_sink(&self) -> bool {
412 true
413 }
414
415 async fn new_coordinator(
416 &self,
417 _iceberg_compact_stat_sender: Option<UnboundedSender<IcebergSinkCompactionUpdate>>,
418 ) -> Result<SinkCommitCoordinator> {
419 let coordinator = DeltaLakeSinkCommitter {
420 table: self.config.common.create_deltalake_client().await?,
421 };
422 Ok(SinkCommitCoordinator::SinglePhase(Box::new(coordinator)))
423 }
424}
425
426impl TryFrom<SinkParam> for DeltaLakeSink {
427 type Error = SinkError;
428
429 fn try_from(param: SinkParam) -> std::result::Result<Self, Self::Error> {
430 let config = DeltaLakeConfig::from_btreemap(param.properties.clone())?;
431 DeltaLakeSink::new(config, param)
432 }
433}
434
435pub struct DeltaLakeSinkWriter {
436 pub config: DeltaLakeConfig,
437 #[expect(dead_code)]
438 schema: Schema,
439 #[expect(dead_code)]
440 pk_indices: Vec<usize>,
441 writer: RecordBatchWriter,
442 dl_schema: Arc<deltalake::arrow::datatypes::Schema>,
443 #[expect(dead_code)]
444 dl_table: DeltaTable,
445}
446
447impl DeltaLakeSinkWriter {
448 pub async fn new(
449 config: DeltaLakeConfig,
450 schema: Schema,
451 pk_indices: Vec<usize>,
452 ) -> Result<Self> {
453 let dl_table = config.common.create_deltalake_client().await?;
454 let writer = RecordBatchWriter::for_table(&dl_table)?;
455 let dl_schema: Arc<deltalake::arrow::datatypes::Schema> =
456 Arc::new(convert_schema(dl_table.get_schema()?)?);
457
458 Ok(Self {
459 config,
460 schema,
461 pk_indices,
462 writer,
463 dl_schema,
464 dl_table,
465 })
466 }
467
468 async fn write(&mut self, chunk: StreamChunk) -> Result<()> {
469 let a = DeltaLakeConvert
470 .to_record_batch(self.dl_schema.clone(), &chunk)
471 .context("convert record batch error")
472 .map_err(SinkError::DeltaLake)?;
473 self.writer.write(a).await?;
474 Ok(())
475 }
476}
477
478fn convert_schema(schema: &StructType) -> Result<deltalake::arrow::datatypes::Schema> {
479 let mut builder = deltalake::arrow::datatypes::SchemaBuilder::new();
480 for field in schema.fields() {
481 let arrow_field_type = deltalake::arrow::datatypes::DataType::try_from(field.data_type())
482 .with_context(|| {
483 format!(
484 "Failed to convert DeltaLake data type {:?} to Arrow data type for field '{}'",
485 field.data_type(),
486 field.name()
487 )
488 })
489 .map_err(SinkError::DeltaLake)?;
490 let dl_field = deltalake::arrow::datatypes::Field::new(
491 field.name(),
492 arrow_field_type,
493 field.is_nullable(),
494 );
495 builder.push(dl_field);
496 }
497 Ok(builder.finish())
498}
499
500#[async_trait]
501impl SinkWriter for DeltaLakeSinkWriter {
502 type CommitMetadata = Option<SinkMetadata>;
503
504 async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> {
505 self.write(chunk).await
506 }
507
508 async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> {
509 Ok(())
510 }
511
512 async fn abort(&mut self) -> Result<()> {
513 Ok(())
514 }
515
516 async fn barrier(&mut self, is_checkpoint: bool) -> Result<Option<SinkMetadata>> {
517 if !is_checkpoint {
518 return Ok(None);
519 }
520
521 let adds = self.writer.flush().await?;
522 Ok(Some(SinkMetadata::try_from(&DeltaLakeWriteResult {
523 adds,
524 })?))
525 }
526}
527
528pub struct DeltaLakeSinkCommitter {
529 table: DeltaTable,
530}
531
532#[async_trait::async_trait]
533impl SinglePhaseCommitCoordinator for DeltaLakeSinkCommitter {
534 async fn init(&mut self) -> Result<()> {
535 tracing::info!("DeltaLake commit coordinator inited.");
536 Ok(())
537 }
538
539 async fn commit(
540 &mut self,
541 epoch: u64,
542 metadata: Vec<SinkMetadata>,
543 add_columns: Option<Vec<Field>>,
544 ) -> Result<()> {
545 tracing::info!("Starting DeltaLake commit in epoch {epoch}.");
546 if let Some(add_columns) = add_columns {
547 return Err(anyhow!(
548 "Delta lake sink not support add columns, but got: {:?}",
549 add_columns
550 )
551 .into());
552 }
553
554 let deltalake_write_result = metadata
555 .iter()
556 .map(DeltaLakeWriteResult::try_from)
557 .collect::<Result<Vec<DeltaLakeWriteResult>>>()?;
558 let write_adds: Vec<Action> = deltalake_write_result
559 .into_iter()
560 .flat_map(|v| v.adds.into_iter())
561 .map(Action::Add)
562 .collect();
563
564 if write_adds.is_empty() {
565 return Ok(());
566 }
567 let partition_cols = self.table.metadata()?.partition_columns.clone();
568 let partition_by = if !partition_cols.is_empty() {
569 Some(partition_cols)
570 } else {
571 None
572 };
573 let operation = DeltaOperation::Write {
574 mode: SaveMode::Append,
575 partition_by,
576 predicate: None,
577 };
578 let version = CommitBuilder::default()
579 .with_actions(write_adds)
580 .build(
581 Some(self.table.snapshot()?),
582 self.table.log_store().clone(),
583 operation,
584 )
585 .await?
586 .version();
587 self.table.update().await?;
588 tracing::info!(
589 "Succeeded to commit to DeltaLake table in epoch {epoch}, version {version}."
590 );
591 Ok(())
592 }
593}
594
595#[derive(Serialize, Deserialize)]
596struct DeltaLakeWriteResult {
597 adds: Vec<Add>,
598}
599
600impl<'a> TryFrom<&'a DeltaLakeWriteResult> for SinkMetadata {
601 type Error = SinkError;
602
603 fn try_from(value: &'a DeltaLakeWriteResult) -> std::result::Result<Self, Self::Error> {
604 let metadata =
605 serde_json::to_vec(&value.adds).context("cannot serialize deltalake sink metadata")?;
606 Ok(SinkMetadata {
607 metadata: Some(Serialized(SerializedMetadata { metadata })),
608 })
609 }
610}
611
612impl DeltaLakeWriteResult {
613 fn try_from(value: &SinkMetadata) -> Result<Self> {
614 if let Some(Serialized(v)) = &value.metadata {
615 let adds = serde_json::from_slice::<Vec<Add>>(&v.metadata)
616 .context("Can't deserialize deltalake sink metadata")?;
617 Ok(DeltaLakeWriteResult { adds })
618 } else {
619 bail!("Can't create deltalake sink write result from empty data!")
620 }
621 }
622}
623
624impl From<::deltalake::DeltaTableError> for SinkError {
625 fn from(value: ::deltalake::DeltaTableError) -> Self {
626 SinkError::DeltaLake(anyhow!(value))
627 }
628}
629
630#[cfg(all(test, not(madsim)))]
631mod tests {
632 use deltalake::kernel::DataType as SchemaDataType;
633 use deltalake::operations::create::CreateBuilder;
634 use maplit::btreemap;
635 use risingwave_common::array::{Array, I32Array, Op, StreamChunk, Utf8Array};
636 use risingwave_common::catalog::{Field, Schema};
637 use risingwave_common::types::DataType;
638
639 use super::{DeltaLakeConfig, DeltaLakeSinkCommitter, DeltaLakeSinkWriter};
640 use crate::sink::SinglePhaseCommitCoordinator;
641 use crate::sink::writer::SinkWriter;
642
643 #[tokio::test]
644 async fn test_deltalake() {
645 let dir = tempfile::tempdir().unwrap();
646 let path = dir.path().to_str().unwrap();
647 CreateBuilder::new()
648 .with_location(path)
649 .with_column(
650 "id",
651 SchemaDataType::Primitive(deltalake::kernel::PrimitiveType::Integer),
652 false,
653 Default::default(),
654 )
655 .with_column(
656 "name",
657 SchemaDataType::Primitive(deltalake::kernel::PrimitiveType::String),
658 false,
659 Default::default(),
660 )
661 .await
662 .unwrap();
663
664 let properties = btreemap! {
665 "connector".to_owned() => "deltalake".to_owned(),
666 "force_append_only".to_owned() => "true".to_owned(),
667 "type".to_owned() => "append-only".to_owned(),
668 "location".to_owned() => format!("file://{}", path),
669 };
670
671 let schema = Schema::new(vec![
672 Field {
673 data_type: DataType::Int32,
674 name: "id".into(),
675 },
676 Field {
677 data_type: DataType::Varchar,
678 name: "name".into(),
679 },
680 ]);
681
682 let deltalake_config = DeltaLakeConfig::from_btreemap(properties).unwrap();
683 let deltalake_table = deltalake_config
684 .common
685 .create_deltalake_client()
686 .await
687 .unwrap();
688
689 let mut deltalake_writer = DeltaLakeSinkWriter::new(deltalake_config, schema, vec![0])
690 .await
691 .unwrap();
692 let chunk = StreamChunk::new(
693 vec![Op::Insert, Op::Insert, Op::Insert],
694 vec![
695 I32Array::from_iter(vec![1, 2, 3]).into_ref(),
696 Utf8Array::from_iter(vec!["Alice", "Bob", "Clare"]).into_ref(),
697 ],
698 );
699 deltalake_writer.write(chunk).await.unwrap();
700 let mut committer = DeltaLakeSinkCommitter {
701 table: deltalake_table,
702 };
703 let metadata = deltalake_writer.barrier(true).await.unwrap().unwrap();
704 committer.commit(1, vec![metadata], None).await.unwrap();
705
706 let ctx = deltalake::datafusion::prelude::SessionContext::new();
715 let table = deltalake::open_table(path).await.unwrap();
716 ctx.register_table("demo", std::sync::Arc::new(table))
717 .unwrap();
718
719 let batches = ctx
720 .sql("SELECT * FROM demo")
721 .await
722 .unwrap()
723 .collect()
724 .await
725 .unwrap();
726 assert_eq!(3, batches.get(0).unwrap().column(0).len());
727 assert_eq!(3, batches.get(0).unwrap().column(1).len());
728 }
729}