Skip to content

Commit

Permalink
Add documentation for streaming usecase (#9070)
Browse files Browse the repository at this point in the history
* Add unbounded table creation.

* Update comment

* Add new streaming test cases

* Minor changes

* Fix doctest

* Update docs/source/user-guide/sql/ddl.md

Co-authored-by: Mehmet Ozan Kabak <[email protected]>

* Comment Updates, minor changes

Co-authored-by: Andrew Lamb <[email protected]>

* Add link of streaming example to readme page

* Fix formatting

---------

Co-authored-by: Mehmet Ozan Kabak <[email protected]>
Co-authored-by: Andrew Lamb <[email protected]>
  • Loading branch information
3 people authored Jan 31, 2024
1 parent d6d35f7 commit 0414b9f
Show file tree
Hide file tree
Showing 4 changed files with 106 additions and 1 deletion.
1 change: 1 addition & 0 deletions datafusion-examples/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ cargo run --example csv_sql

- [`avro_sql.rs`](examples/avro_sql.rs): Build and run a query plan from a SQL statement against a local AVRO file
- [`csv_sql.rs`](examples/csv_sql.rs): Build and run a query plan from a SQL statement against a local CSV file
- [`csv_sql_streaming.rs`](examples/csv_sql_streaming.rs): Build and run a streaming query plan from a SQL statement against a local CSV file
- [`catalog.rs`](examples/external_dependency/catalog.rs): Register the table into a custom catalog
- [`custom_datasource.rs`](examples/custom_datasource.rs): Run queries against a custom datasource (TableProvider)
- [`dataframe.rs`](examples/dataframe.rs): Run a query using a DataFrame against a local parquet file
Expand Down
74 changes: 74 additions & 0 deletions datafusion-examples/examples/csv_sql_streaming.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
// 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 datafusion::common::test_util::datafusion_test_data;
use datafusion::error::Result;
use datafusion::prelude::*;

/// This example demonstrates executing a simple query against an Arrow data source (CSV) and
/// fetching results with streaming aggregation and streaming window
#[tokio::main]
async fn main() -> Result<()> {
// create local execution context
let ctx = SessionContext::new();

let testdata = datafusion_test_data();

// Register a table source and tell DataFusion the file is ordered by `ts ASC`.
// Note it is the responsibility of the user to make sure
// that file indeed satisfies this condition or else incorrect answers may be produced.
let asc = true;
let nulls_first = true;
let sort_expr = vec![col("ts").sort(asc, nulls_first)];
// register csv file with the execution context
ctx.register_csv(
"ordered_table",
&format!("{testdata}/window_1.csv"),
CsvReadOptions::new().file_sort_order(vec![sort_expr]),
)
.await?;

// execute the query
// Following query can be executed with unbounded sources because group by expressions (e.g ts) is
// already ordered at the source.
//
// Unbounded sources means that if the input came from a "never ending" source (such as a FIFO
// file on unix) the query could produce results incrementally as data was read.
let df = ctx
.sql(
"SELECT ts, MIN(inc_col), MAX(inc_col) \
FROM ordered_table \
GROUP BY ts",
)
.await?;

df.show().await?;

// execute the query
// Following query can be executed with unbounded sources because window executor can calculate
// its result in streaming fashion, because its required ordering is already satisfied at the source.
let df = ctx
.sql(
"SELECT ts, SUM(inc_col) OVER(ORDER BY ts ASC) \
FROM ordered_table",
)
.await?;

df.show().await?;

Ok(())
}
20 changes: 20 additions & 0 deletions datafusion/common/src/test_util.rs
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,26 @@ macro_rules! assert_not_contains {
};
}

/// Returns the datafusion test data directory, which is by default rooted at `datafusion/core/tests/data`.
///
/// The default can be overridden by the optional environment
/// variable `DATAFUSION_TEST_DATA`
///
/// panics when the directory can not be found.
///
/// Example:
/// ```
/// let testdata = datafusion_common::test_util::datafusion_test_data();
/// let csvdata = format!("{}/window_1.csv", testdata);
/// assert!(std::path::PathBuf::from(csvdata).exists());
/// ```
pub fn datafusion_test_data() -> String {
match get_data_dir("DATAFUSION_TEST_DATA", "../../datafusion/core/tests/data") {
Ok(pb) => pb.display().to_string(),
Err(err) => panic!("failed to get arrow data dir: {err}"),
}
}

/// Returns the arrow test data directory, which is by default stored
/// in a git submodule rooted at `testing/data`.
///
Expand Down
12 changes: 11 additions & 1 deletion docs/source/user-guide/sql/ddl.md
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ file system or remote object store as a named table which can be queried.
The supported syntax is:

```
CREATE EXTERNAL TABLE
CREATE [UNBOUNDED] EXTERNAL TABLE
[ IF NOT EXISTS ]
<TABLE_NAME>[ (<column_definition>) ]
STORED AS <file_type>
Expand Down Expand Up @@ -147,6 +147,16 @@ WITH HEADER ROW
LOCATION '/path/to/directory/of/files';
```

With `CREATE UNBOUNDED EXTERNAL TABLE` SQL statement. We can create unbounded data sources such as following:

```sql
CREATE UNBOUNDED EXTERNAL TABLE taxi
STORED AS PARQUET
LOCATION '/mnt/nyctaxi/tripdata.parquet';
```

Note that this statement actually reads data from a fixed-size file, so a better example would involve reading from a FIFO file. Nevertheless, once Datafusion sees the `UNBOUNDED` keyword in a data source, it tries to execute queries that refer to this unbounded source in streaming fashion. If this is not possible according to query specifications, plan generation fails stating it is not possible to execute given query in streaming fashion. Note that queries that can run with unbounded sources (i.e. in streaming mode) are a subset of those that can with bounded sources. A query that fails with unbounded source(s) may work with bounded source(s).

When creating an output from a data source that is already ordered by
an expression, you can pre-specify the order of the data using the
`WITH ORDER` clause. This applies even if the expression used for
Expand Down

0 comments on commit 0414b9f

Please sign in to comment.