-
Notifications
You must be signed in to change notification settings - Fork 72
Expand file tree
/
Copy pathmod.rs
More file actions
359 lines (322 loc) · 13.1 KB
/
mod.rs
File metadata and controls
359 lines (322 loc) · 13.1 KB
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
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
use crate::arrow_record_batch_gen::*;
use datafusion::arrow::array::RecordBatch;
use datafusion::arrow::datatypes::SchemaRef;
use datafusion::catalog::TableProviderFactory;
use datafusion::common::{Constraints, ToDFSchema};
use datafusion::datasource::memory::MemorySourceConfig;
use datafusion::execution::context::SessionContext;
use datafusion::logical_expr::dml::InsertOp;
use datafusion::logical_expr::CreateExternalTable;
use datafusion::physical_plan::collect;
use datafusion_table_providers::duckdb::DuckDBTableProviderFactory;
use rstest::rstest;
use std::collections::HashMap;
use std::sync::Arc;
async fn arrow_duckdb_round_trip(
arrow_record: RecordBatch,
_source_schema: SchemaRef,
table_name: &str,
) {
let factory = DuckDBTableProviderFactory::new(duckdb::AccessMode::ReadWrite);
let ctx = SessionContext::new();
let cmd = CreateExternalTable {
schema: Arc::new(arrow_record.schema().to_dfschema().expect("to df schema")),
name: table_name.into(),
location: "".to_string(),
file_type: "".to_string(),
table_partition_cols: vec![],
if_not_exists: false,
or_replace: false,
definition: None,
order_exprs: vec![],
unbounded: false,
options: HashMap::new(),
constraints: Constraints::default(),
column_defaults: HashMap::new(),
temporary: false,
};
let table_provider = factory
.create(&ctx.state(), &cmd)
.await
.expect("table provider created");
let ctx = SessionContext::new();
let mem_exec = MemorySourceConfig::try_new_exec(
&[vec![arrow_record.clone()]],
arrow_record.schema(),
None,
)
.expect("memory exec created");
let insert_plan = table_provider
.insert_into(&ctx.state(), mem_exec, InsertOp::Append)
.await
.expect("insert plan created");
let _ = collect(insert_plan, ctx.task_ctx())
.await
.expect("insert done");
ctx.register_table(table_name, table_provider)
.expect("Table should be registered");
let sql = format!("SELECT * FROM {table_name}");
let df = ctx
.sql(&sql)
.await
.expect("DataFrame should be created from query");
let record_batch = df.collect().await.expect("RecordBatch should be collected");
tracing::debug!("Original Arrow Record Batch: {:?}", arrow_record.columns());
tracing::debug!(
"Duckdb returned Record Batch: {:?}",
record_batch[0].columns()
);
// Check results
assert_eq!(record_batch.len(), 1);
assert_eq!(record_batch[0].num_rows(), arrow_record.num_rows());
assert_eq!(record_batch[0].num_columns(), arrow_record.num_columns());
assert_eq!(record_batch[0], arrow_record);
}
#[rstest]
#[case::binary(get_arrow_binary_record_batch(), "binary")]
#[case::int(get_arrow_int_record_batch(), "int")]
#[case::float(get_arrow_float_record_batch(), "float")]
#[case::utf8(get_arrow_utf8_record_batch(), "utf8")]
#[case::time(get_arrow_time_record_batch(), "time")]
#[case::timestamp(get_arrow_timestamp_record_batch(), "timestamp")]
#[case::date(get_arrow_date_record_batch(), "date")]
#[case::struct_type(get_arrow_struct_record_batch(), "struct")]
#[ignore] // DuckDB does not support Decimal256 / duckdb_arrow_scan failed to register view
#[case::decimal(get_arrow_decimal_record_batch(), "decimal")]
#[ignore]
// Interval(DayTime) is not supported: / "Conversion Error: Could not convert Interval to Microsecond"
#[case::interval(get_arrow_interval_record_batch(), "interval")]
#[ignore] // TimeUnit::Nanosecond is not correctly supported; written values are zeros
#[case::duration(get_arrow_duration_record_batch(), "duration")]
#[case::list(get_arrow_list_record_batch(), "list")]
#[case::null(get_arrow_null_record_batch(), "null")]
#[case::list_of_structs(get_arrow_list_of_structs_record_batch(), "list_of_structs")]
#[case::list_of_fixed_size_lists(
get_arrow_list_of_fixed_size_lists_record_batch(),
"list_of_fixed_size_lists"
)]
#[case::list_of_lists(get_arrow_list_of_lists_record_batch(), "list_of_lists")]
#[case::map(get_arrow_map_record_batch(), "map")]
#[case::dictionary(get_arrow_dictionary_array_record_batch(), "dictionary")]
#[test_log::test(tokio::test)]
async fn test_arrow_duckdb_roundtrip(
#[case] arrow_result: (RecordBatch, SchemaRef),
#[case] table_name: &str,
) {
arrow_duckdb_round_trip(
arrow_result.0,
arrow_result.1,
&format!("{table_name}_types"),
)
.await;
}
mod sort_limit_pushdown {
use super::*;
use datafusion::arrow::array::{Int32Array, StringArray};
use datafusion::arrow::datatypes::{DataType, Field, Schema};
async fn setup_table(ctx: &SessionContext, name: &str) {
let schema = Arc::new(Schema::new(vec![
Field::new("id", DataType::Int32, false),
Field::new("label", DataType::Utf8, false),
]));
// 20 rows: id = 1..=20
let ids: Vec<i32> = (1..=20).collect();
let labels: Vec<String> = ids.iter().map(|i| format!("row-{i:02}")).collect();
let batch = RecordBatch::try_new(
Arc::clone(&schema),
vec![
Arc::new(Int32Array::from(ids)),
Arc::new(StringArray::from(labels)),
],
)
.unwrap();
let factory = DuckDBTableProviderFactory::new(duckdb::AccessMode::ReadWrite);
let cmd = CreateExternalTable {
schema: Arc::new(batch.schema().to_dfschema().unwrap()),
name: name.into(),
location: String::new(),
file_type: String::new(),
table_partition_cols: vec![],
if_not_exists: false,
or_replace: false,
definition: None,
order_exprs: vec![],
unbounded: false,
options: HashMap::new(),
constraints: Constraints::default(),
column_defaults: HashMap::new(),
temporary: false,
};
let table = factory.create(&ctx.state(), &cmd).await.unwrap();
let mem =
MemorySourceConfig::try_new_exec(&[vec![batch.clone()]], batch.schema(), None).unwrap();
let insert = table
.insert_into(&ctx.state(), mem, InsertOp::Append)
.await
.unwrap();
let _ = collect(insert, ctx.task_ctx()).await.unwrap();
ctx.register_table(name, table).unwrap();
}
#[test_log::test(tokio::test)]
async fn order_by_limit_returns_exactly_n_rows() {
let ctx = SessionContext::new();
setup_table(&ctx, "sort_limit_test").await;
let df = ctx
.sql("SELECT id FROM sort_limit_test ORDER BY id DESC LIMIT 5")
.await
.unwrap();
let batches = df.collect().await.unwrap();
let total: usize = batches.iter().map(|b| b.num_rows()).sum();
assert_eq!(total, 5, "LIMIT 5 must return exactly 5 rows");
let col = batches[0]
.column(0)
.as_any()
.downcast_ref::<Int32Array>()
.unwrap();
let got: Vec<i32> = (0..col.len()).map(|i| col.value(i)).collect();
assert_eq!(got, vec![20, 19, 18, 17, 16], "top-5 DESC rows");
}
#[test_log::test(tokio::test)]
async fn order_by_limit_with_filter() {
let ctx = SessionContext::new();
setup_table(&ctx, "sort_limit_filter_test").await;
let df = ctx
.sql("SELECT id FROM sort_limit_filter_test WHERE id > 10 ORDER BY id ASC LIMIT 3")
.await
.unwrap();
let batches = df.collect().await.unwrap();
let total: usize = batches.iter().map(|b| b.num_rows()).sum();
assert_eq!(total, 3);
let col = batches[0]
.column(0)
.as_any()
.downcast_ref::<Int32Array>()
.unwrap();
let got: Vec<i32> = (0..col.len()).map(|i| col.value(i)).collect();
assert_eq!(got, vec![11, 12, 13]);
}
#[test_log::test(tokio::test)]
async fn limit_without_order_by() {
let ctx = SessionContext::new();
setup_table(&ctx, "limit_only_test").await;
let df = ctx
.sql("SELECT id FROM limit_only_test LIMIT 7")
.await
.unwrap();
let batches = df.collect().await.unwrap();
let total: usize = batches.iter().map(|b| b.num_rows()).sum();
assert_eq!(total, 7, "LIMIT without ORDER BY must still cap rows");
}
}
mod multipart_table_reference {
use datafusion::arrow::array::{Int32Array, StringArray};
use datafusion::execution::context::SessionContext;
use datafusion::sql::TableReference;
use datafusion_table_providers::duckdb::DuckDBTableFactory;
use datafusion_table_providers::sql::db_connection_pool::dbconnection::duckdbconn::DuckDbConnection;
use datafusion_table_providers::sql::db_connection_pool::dbconnection::DbConnection;
use datafusion_table_providers::sql::db_connection_pool::duckdbpool::DuckDbConnectionPool;
use std::sync::Arc;
/// Tests that a `DuckDBTable` created with a full (catalog.schema.table) reference
/// generates correct SQL that DuckDB can execute through the non-federated scan path.
#[test_log::test(tokio::test)]
async fn scan_with_full_table_reference() {
let pool = Arc::new(DuckDbConnectionPool::new_memory().expect("pool created"));
// Create a schema and table in DuckDB directly
{
let mut conn = Arc::clone(&pool).connect_sync().expect("connection");
let duckdb_conn = conn
.as_any_mut()
.downcast_mut::<DuckDbConnection>()
.expect("downcast to DuckDbConnection");
duckdb_conn
.conn
.execute_batch(
"CREATE SCHEMA test_schema;
CREATE TABLE test_schema.numbers (id INTEGER, label VARCHAR);
INSERT INTO test_schema.numbers VALUES (1, 'one'), (2, 'two'), (3, 'three');",
)
.expect("setup SQL");
}
// Create table provider with a full reference: memory.test_schema.numbers
let factory = DuckDBTableFactory::new(Arc::clone(&pool));
// DuckDB's in-memory database has the default catalog name "memory".
let table_ref = TableReference::full("memory", "test_schema", "numbers");
let provider = factory
.table_provider(table_ref)
.await
.expect("table_provider with full ref");
// Register in DataFusion and query
let ctx = SessionContext::new();
ctx.register_table("numbers", provider)
.expect("register table");
let batches = ctx
.sql("SELECT id, label FROM numbers ORDER BY id")
.await
.expect("query")
.collect()
.await
.expect("collect");
assert_eq!(batches.len(), 1);
assert_eq!(batches[0].num_rows(), 3);
let ids = batches[0]
.column(0)
.as_any()
.downcast_ref::<Int32Array>()
.expect("id column");
assert_eq!(ids.values().to_vec(), vec![1, 2, 3]);
let labels = batches[0]
.column(1)
.as_any()
.downcast_ref::<StringArray>()
.expect("label column");
assert_eq!(labels.value(0), "one");
assert_eq!(labels.value(1), "two");
assert_eq!(labels.value(2), "three");
}
/// Tests that a `DuckDBTable` created with a partial (schema.table) reference
/// generates correct SQL through the non-federated scan path.
#[test_log::test(tokio::test)]
async fn scan_with_partial_table_reference() {
let pool = Arc::new(DuckDbConnectionPool::new_memory().expect("pool created"));
{
let mut conn = Arc::clone(&pool).connect_sync().expect("connection");
let duckdb_conn = conn
.as_any_mut()
.downcast_mut::<DuckDbConnection>()
.expect("downcast to DuckDbConnection");
duckdb_conn
.conn
.execute_batch(
"CREATE SCHEMA another_schema;
CREATE TABLE another_schema.items (val INTEGER);
INSERT INTO another_schema.items VALUES (10), (20);",
)
.expect("setup SQL");
}
let factory = DuckDBTableFactory::new(Arc::clone(&pool));
let table_ref = TableReference::partial("another_schema", "items");
let provider = factory
.table_provider(table_ref)
.await
.expect("table_provider with partial ref");
let ctx = SessionContext::new();
ctx.register_table("items", provider)
.expect("register table");
let batches = ctx
.sql("SELECT val FROM items ORDER BY val")
.await
.expect("query")
.collect()
.await
.expect("collect");
assert_eq!(batches.len(), 1);
assert_eq!(batches[0].num_rows(), 2);
let vals = batches[0]
.column(0)
.as_any()
.downcast_ref::<Int32Array>()
.expect("val column");
assert_eq!(vals.values().to_vec(), vec![10, 20]);
}
}