-
Notifications
You must be signed in to change notification settings - Fork 215
/
Copy pathscan.rs
140 lines (122 loc) · 4.25 KB
/
scan.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you 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::any::Any;
use std::pin::Pin;
use std::sync::Arc;
use datafusion::arrow::array::RecordBatch;
use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef;
use datafusion::error::Result as DFResult;
use datafusion::execution::{SendableRecordBatchStream, TaskContext};
use datafusion::physical_expr::EquivalenceProperties;
use datafusion::physical_plan::stream::RecordBatchStreamAdapter;
use datafusion::physical_plan::{
DisplayAs, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties,
};
use futures::{Stream, TryStreamExt};
use iceberg::table::Table;
use crate::to_datafusion_error;
/// Manages the scanning process of an Iceberg [`Table`], encapsulating the
/// necessary details and computed properties required for execution planning.
#[derive(Debug)]
pub(crate) struct IcebergTableScan {
/// A table in the catalog.
table: Table,
/// A reference-counted arrow `Schema`.
schema: ArrowSchemaRef,
/// Stores certain, often expensive to compute,
/// plan properties used in query optimization.
plan_properties: PlanProperties,
}
impl IcebergTableScan {
/// Creates a new [`IcebergTableScan`] object.
pub(crate) fn new(table: Table, schema: ArrowSchemaRef) -> Self {
let plan_properties = Self::compute_properties(schema.clone());
Self {
table,
schema,
plan_properties,
}
}
/// Computes [`PlanProperties`] used in query optimization.
fn compute_properties(schema: ArrowSchemaRef) -> PlanProperties {
// TODO:
// This is more or less a placeholder, to be replaced
// once we support output-partitioning
PlanProperties::new(
EquivalenceProperties::new(schema),
Partitioning::UnknownPartitioning(1),
ExecutionMode::Bounded,
)
}
}
impl ExecutionPlan for IcebergTableScan {
fn name(&self) -> &str {
"IcebergTableScan"
}
fn as_any(&self) -> &dyn Any {
self
}
fn children(&self) -> Vec<&Arc<(dyn ExecutionPlan + 'static)>> {
vec![]
}
fn with_new_children(
self: Arc<Self>,
_children: Vec<Arc<dyn ExecutionPlan>>,
) -> DFResult<Arc<dyn ExecutionPlan>> {
Ok(self)
}
fn properties(&self) -> &PlanProperties {
&self.plan_properties
}
fn execute(
&self,
_partition: usize,
_context: Arc<TaskContext>,
) -> DFResult<SendableRecordBatchStream> {
let fut = get_batch_stream(self.table.clone());
let stream = futures::stream::once(fut).try_flatten();
Ok(Box::pin(RecordBatchStreamAdapter::new(
self.schema.clone(),
stream,
)))
}
}
impl DisplayAs for IcebergTableScan {
fn fmt_as(
&self,
_t: datafusion::physical_plan::DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
write!(f, "IcebergTableScan")
}
}
/// Asynchronously retrieves a stream of [`RecordBatch`] instances
/// from a given table.
///
/// This function initializes a [`TableScan`], builds it,
/// and then converts it into a stream of Arrow [`RecordBatch`]es.
async fn get_batch_stream(
table: Table,
) -> DFResult<Pin<Box<dyn Stream<Item = DFResult<RecordBatch>> + Send>>> {
let table_scan = table.scan().build().map_err(to_datafusion_error)?;
let stream = table_scan
.to_arrow()
.await
.map_err(to_datafusion_error)?
.map_err(to_datafusion_error);
Ok(Box::pin(stream))
}