Skip to content

Commit 944f7f2

Browse files
cj-zhukovSergey Zhukovalamb
authored
Run the examples in the new format (#18946)
## Which issue does this PR close? <!-- We generally require a GitHub issue to be filed for all bug fixes and enhancements and this helps us generate change logs for our releases. You can link an issue to this PR using the GitHub syntax. For example `Closes #123` indicates that this PR will close issue #123. --> - Closes ##18512. ## Rationale for this change <!-- Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed. Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes. --> ## What changes are included in this PR? <!-- There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR. --> ## Are these changes tested? <!-- We typically require tests for all PRs in order to: 1. Prevent the code from being accidentally broken by subsequent changes 2. Serve as another way to document the expected behavior of the code If tests are not included in your PR, please explain why (for example, are they covered by existing tests)? --> ## Are there any user-facing changes? <!-- If there are user-facing changes then we may require documentation to be updated before approving the PR. --> <!-- If there are any breaking changes to public APIs, please add the `api change` label. --> --------- Co-authored-by: Sergey Zhukov <szhukov@aligntech.com> Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
1 parent c479dee commit 944f7f2

File tree

20 files changed

+636
-216
lines changed

20 files changed

+636
-216
lines changed

ci/scripts/rust_example.sh

Lines changed: 22 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -25,12 +25,26 @@ export CARGO_PROFILE_CI_STRIP=true
2525
cd datafusion-examples/examples/
2626
cargo build --profile ci --examples
2727

28-
files=$(ls .)
29-
for filename in $files
30-
do
31-
example_name=`basename $filename ".rs"`
32-
# Skip tests that rely on external storage and flight
33-
if [ ! -d $filename ]; then
34-
cargo run --profile ci --example $example_name
35-
fi
28+
SKIP_LIST=("external_dependency" "flight" "ffi")
29+
30+
skip_example() {
31+
local name="$1"
32+
for skip in "${SKIP_LIST[@]}"; do
33+
if [ "$name" = "$skip" ]; then
34+
return 0
35+
fi
36+
done
37+
return 1
38+
}
39+
40+
for dir in */; do
41+
example_name=$(basename "$dir")
42+
43+
if skip_example "$example_name"; then
44+
echo "Skipping $example_name"
45+
continue
46+
fi
47+
48+
echo "Running example group: $example_name"
49+
cargo run --profile ci --example "$example_name" -- all
3650
done

datafusion-examples/examples/builtin_functions/main.rs

Lines changed: 35 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,11 @@
2121
//!
2222
//! ## Usage
2323
//! ```bash
24-
//! cargo run --example builtin_functions -- [date_time|function_factory|regexp]
24+
//! cargo run --example builtin_functions -- [all|date_time|function_factory|regexp]
2525
//! ```
2626
//!
2727
//! Each subcommand runs a corresponding example:
28+
//! - `all` — run all examples included in this module
2829
//! - `date_time` — examples of date-time related functions and queries
2930
//! - `function_factory` — register `CREATE FUNCTION` handler to implement SQL macros
3031
//! - `regexp` — examples of using regular expression functions
@@ -38,6 +39,7 @@ use std::str::FromStr;
3839
use datafusion::error::{DataFusionError, Result};
3940

4041
enum ExampleKind {
42+
All,
4143
DateTime,
4244
FunctionFactory,
4345
Regexp,
@@ -46,6 +48,7 @@ enum ExampleKind {
4648
impl AsRef<str> for ExampleKind {
4749
fn as_ref(&self) -> &str {
4850
match self {
51+
Self::All => "all",
4952
Self::DateTime => "date_time",
5053
Self::FunctionFactory => "function_factory",
5154
Self::Regexp => "regexp",
@@ -58,6 +61,7 @@ impl FromStr for ExampleKind {
5861

5962
fn from_str(s: &str) -> Result<Self> {
6063
match s {
64+
"all" => Ok(Self::All),
6165
"date_time" => Ok(Self::DateTime),
6266
"function_factory" => Ok(Self::FunctionFactory),
6367
"regexp" => Ok(Self::Regexp),
@@ -67,12 +71,38 @@ impl FromStr for ExampleKind {
6771
}
6872

6973
impl ExampleKind {
70-
const ALL: [Self; 3] = [Self::DateTime, Self::FunctionFactory, Self::Regexp];
74+
const ALL_VARIANTS: [Self; 4] = [
75+
Self::All,
76+
Self::DateTime,
77+
Self::FunctionFactory,
78+
Self::Regexp,
79+
];
80+
81+
const RUNNABLE_VARIANTS: [Self; 3] =
82+
[Self::DateTime, Self::FunctionFactory, Self::Regexp];
7183

7284
const EXAMPLE_NAME: &str = "builtin_functions";
7385

7486
fn variants() -> Vec<&'static str> {
75-
Self::ALL.iter().map(|x| x.as_ref()).collect()
87+
Self::ALL_VARIANTS
88+
.iter()
89+
.map(|example| example.as_ref())
90+
.collect()
91+
}
92+
93+
async fn run(&self) -> Result<()> {
94+
match self {
95+
ExampleKind::All => {
96+
for example in ExampleKind::RUNNABLE_VARIANTS {
97+
println!("Running example: {}", example.as_ref());
98+
Box::pin(example.run()).await?;
99+
}
100+
}
101+
ExampleKind::DateTime => date_time::date_time().await?,
102+
ExampleKind::FunctionFactory => function_factory::function_factory().await?,
103+
ExampleKind::Regexp => regexp::regexp().await?,
104+
}
105+
Ok(())
76106
}
77107
}
78108

@@ -89,11 +119,6 @@ async fn main() -> Result<()> {
89119
DataFusionError::Execution("Missing argument".to_string())
90120
})?;
91121

92-
match arg.parse::<ExampleKind>()? {
93-
ExampleKind::DateTime => date_time::date_time().await?,
94-
ExampleKind::FunctionFactory => function_factory::function_factory().await?,
95-
ExampleKind::Regexp => regexp::regexp().await?,
96-
}
97-
98-
Ok(())
122+
let example = arg.parse::<ExampleKind>()?;
123+
example.run().await
99124
}

datafusion-examples/examples/builtin_functions/regexp.rs

Lines changed: 27 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,12 @@
1818

1919
//! See `main.rs` for how to run it.
2020
21+
use std::{fs::File, io::Write};
22+
2123
use datafusion::common::{assert_batches_eq, assert_contains};
2224
use datafusion::error::Result;
2325
use datafusion::prelude::*;
26+
use tempfile::tempdir;
2427

2528
/// This example demonstrates how to use the regexp_* functions
2629
///
@@ -32,12 +35,30 @@ use datafusion::prelude::*;
3235
/// https://docs.rs/regex/latest/regex/#grouping-and-flags
3336
pub async fn regexp() -> Result<()> {
3437
let ctx = SessionContext::new();
35-
ctx.register_csv(
36-
"examples",
37-
"datafusion/physical-expr/tests/data/regex.csv",
38-
CsvReadOptions::new(),
39-
)
40-
.await?;
38+
// content from file 'datafusion/physical-expr/tests/data/regex.csv'
39+
let csv_data = r#"values,patterns,replacement,flags
40+
abc,^(a),bb\1bb,i
41+
ABC,^(A).*,B,i
42+
aBc,(b|d),e,i
43+
AbC,(B|D),e,
44+
aBC,^(b|c),d,
45+
4000,\b4([1-9]\d\d|\d[1-9]\d|\d\d[1-9])\b,xyz,
46+
4010,\b4([1-9]\d\d|\d[1-9]\d|\d\d[1-9])\b,xyz,
47+
Düsseldorf,[\p{Letter}-]+,München,
48+
Москва,[\p{L}-]+,Moscow,
49+
Köln,[a-zA-Z]ö[a-zA-Z]{2},Koln,
50+
اليوم,^\p{Arabic}+$,Today,"#;
51+
let dir = tempdir()?;
52+
let file_path = dir.path().join("regex.csv");
53+
{
54+
let mut file = File::create(&file_path)?;
55+
// write CSV data
56+
file.write_all(csv_data.as_bytes())?;
57+
} // scope closes the file
58+
let file_path = file_path.to_str().unwrap();
59+
60+
ctx.register_csv("examples", file_path, CsvReadOptions::new())
61+
.await?;
4162

4263
//
4364
//

datafusion-examples/examples/custom_data_source/custom_file_casts.rs

Lines changed: 14 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ use datafusion::datasource::listing::{
3232
use datafusion::execution::context::SessionContext;
3333
use datafusion::execution::object_store::ObjectStoreUrl;
3434
use datafusion::parquet::arrow::ArrowWriter;
35-
use datafusion::physical_expr::expressions::CastExpr;
35+
use datafusion::physical_expr::expressions::{CastColumnExpr, CastExpr};
3636
use datafusion::physical_expr::PhysicalExpr;
3737
use datafusion::prelude::SessionConfig;
3838
use datafusion_physical_expr_adapter::{
@@ -192,6 +192,19 @@ impl PhysicalExprAdapter for CustomCastsPhysicalExprAdapter {
192192
);
193193
}
194194
}
195+
if let Some(cast) = expr.as_any().downcast_ref::<CastColumnExpr>() {
196+
let input_data_type =
197+
cast.expr().data_type(&self.physical_file_schema)?;
198+
let output_data_type = cast.data_type(&self.physical_file_schema)?;
199+
if !CastExpr::check_bigger_cast(
200+
cast.target_field().data_type(),
201+
&input_data_type,
202+
) {
203+
return not_impl_err!(
204+
"Unsupported CAST from {input_data_type} to {output_data_type}"
205+
);
206+
}
207+
}
195208
Ok(Transformed::no(expr))
196209
})
197210
.data()

datafusion-examples/examples/custom_data_source/main.rs

Lines changed: 58 additions & 22 deletions
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,11 @@
2121
//!
2222
//! ## Usage
2323
//! ```bash
24-
//! cargo run --example custom_data_source -- [csv_json_opener|csv_sql_streaming|custom_datasource|custom_file_casts|custom_file_format|default_column_values|file_stream_provider]
24+
//! cargo run --example custom_data_source -- [all|csv_json_opener|csv_sql_streaming|custom_datasource|custom_file_casts|custom_file_format|default_column_values|file_stream_provider]
2525
//! ```
2626
//!
2727
//! Each subcommand runs a corresponding example:
28+
//! - `all` — run all examples included in this module
2829
//! - `csv_json_opener` — use low level FileOpener APIs to read CSV/JSON into Arrow RecordBatches
2930
//! - `csv_sql_streaming` — build and run a streaming query plan from a SQL statement against a local CSV file
3031
//! - `custom_datasource` — run queries against a custom datasource (TableProvider)
@@ -46,25 +47,27 @@ use std::str::FromStr;
4647
use datafusion::error::{DataFusionError, Result};
4748

4849
enum ExampleKind {
50+
All,
4951
CsvJsonOpener,
5052
CsvSqlStreaming,
5153
CustomDatasource,
5254
CustomFileCasts,
5355
CustomFileFormat,
5456
DefaultColumnValues,
55-
FileFtreamProvider,
57+
FileStreamProvider,
5658
}
5759

5860
impl AsRef<str> for ExampleKind {
5961
fn as_ref(&self) -> &str {
6062
match self {
63+
Self::All => "all",
6164
Self::CsvJsonOpener => "csv_json_opener",
6265
Self::CsvSqlStreaming => "csv_sql_streaming",
6366
Self::CustomDatasource => "custom_datasource",
6467
Self::CustomFileCasts => "custom_file_casts",
6568
Self::CustomFileFormat => "custom_file_format",
6669
Self::DefaultColumnValues => "default_column_values",
67-
Self::FileFtreamProvider => "file_stream_provider",
70+
Self::FileStreamProvider => "file_stream_provider",
6871
}
6972
}
7073
}
@@ -74,33 +77,79 @@ impl FromStr for ExampleKind {
7477

7578
fn from_str(s: &str) -> Result<Self> {
7679
match s {
80+
"all" => Ok(Self::All),
7781
"csv_json_opener" => Ok(Self::CsvJsonOpener),
7882
"csv_sql_streaming" => Ok(Self::CsvSqlStreaming),
7983
"custom_datasource" => Ok(Self::CustomDatasource),
8084
"custom_file_casts" => Ok(Self::CustomFileCasts),
8185
"custom_file_format" => Ok(Self::CustomFileFormat),
8286
"default_column_values" => Ok(Self::DefaultColumnValues),
83-
"file_stream_provider" => Ok(Self::FileFtreamProvider),
87+
"file_stream_provider" => Ok(Self::FileStreamProvider),
8488
_ => Err(DataFusionError::Execution(format!("Unknown example: {s}"))),
8589
}
8690
}
8791
}
8892

8993
impl ExampleKind {
90-
const ALL: [Self; 7] = [
94+
const ALL_VARIANTS: [Self; 8] = [
95+
Self::All,
9196
Self::CsvJsonOpener,
9297
Self::CsvSqlStreaming,
9398
Self::CustomDatasource,
9499
Self::CustomFileCasts,
95100
Self::CustomFileFormat,
96101
Self::DefaultColumnValues,
97-
Self::FileFtreamProvider,
102+
Self::FileStreamProvider,
103+
];
104+
105+
const RUNNABLE_VARIANTS: [Self; 7] = [
106+
Self::CsvJsonOpener,
107+
Self::CsvSqlStreaming,
108+
Self::CustomDatasource,
109+
Self::CustomFileCasts,
110+
Self::CustomFileFormat,
111+
Self::DefaultColumnValues,
112+
Self::FileStreamProvider,
98113
];
99114

100115
const EXAMPLE_NAME: &str = "custom_data_source";
101116

102117
fn variants() -> Vec<&'static str> {
103-
Self::ALL.iter().map(|x| x.as_ref()).collect()
118+
Self::ALL_VARIANTS
119+
.iter()
120+
.map(|example| example.as_ref())
121+
.collect()
122+
}
123+
124+
async fn run(&self) -> Result<()> {
125+
match self {
126+
ExampleKind::All => {
127+
for example in ExampleKind::RUNNABLE_VARIANTS {
128+
println!("Running example: {}", example.as_ref());
129+
Box::pin(example.run()).await?;
130+
}
131+
}
132+
ExampleKind::CsvJsonOpener => csv_json_opener::csv_json_opener().await?,
133+
ExampleKind::CsvSqlStreaming => {
134+
csv_sql_streaming::csv_sql_streaming().await?
135+
}
136+
ExampleKind::CustomDatasource => {
137+
custom_datasource::custom_datasource().await?
138+
}
139+
ExampleKind::CustomFileCasts => {
140+
custom_file_casts::custom_file_casts().await?
141+
}
142+
ExampleKind::CustomFileFormat => {
143+
custom_file_format::custom_file_format().await?
144+
}
145+
ExampleKind::DefaultColumnValues => {
146+
default_column_values::default_column_values().await?
147+
}
148+
ExampleKind::FileStreamProvider => {
149+
file_stream_provider::file_stream_provider().await?
150+
}
151+
}
152+
Ok(())
104153
}
105154
}
106155

@@ -117,19 +166,6 @@ async fn main() -> Result<()> {
117166
DataFusionError::Execution("Missing argument".to_string())
118167
})?;
119168

120-
match arg.parse::<ExampleKind>()? {
121-
ExampleKind::CsvJsonOpener => csv_json_opener::csv_json_opener().await?,
122-
ExampleKind::CsvSqlStreaming => csv_sql_streaming::csv_sql_streaming().await?,
123-
ExampleKind::CustomDatasource => custom_datasource::custom_datasource().await?,
124-
ExampleKind::CustomFileCasts => custom_file_casts::custom_file_casts().await?,
125-
ExampleKind::CustomFileFormat => custom_file_format::custom_file_format().await?,
126-
ExampleKind::DefaultColumnValues => {
127-
default_column_values::default_column_values().await?
128-
}
129-
ExampleKind::FileFtreamProvider => {
130-
file_stream_provider::file_stream_provider().await?
131-
}
132-
}
133-
134-
Ok(())
169+
let example = arg.parse::<ExampleKind>()?;
170+
example.run().await
135171
}

0 commit comments

Comments
 (0)