Skip to content

Commit 9d47dca

Browse files
Move from_unixtime, now, current_date, current_time functions to datafusion-functions (#9537)
* Move date_part, date_trunc, date_bin functions to datafusion-functions * I do not understand why the logical plan changed but updating the explain text to reflect the change. The physical plan is unchanged. * Fix fmt * Improvements to remove datafusion-functions dependency from sq and physical-expr * WIP * Fix function arguments for date_bin, date_trunc and date_part. * WIP * Fix projection change. Add new test date_bin monotonicity * Move now, current_date and current_time functions to datafusion-functions * Force exact version of chrono * Merge updates. * Updates for chrono changes * Merge fixes * Removed make_now from incorrect merge. * fmt fix. * Updates after correcting merge conflicts. * Only move the tests using now() function from optimizer_integration.rs to the core/tests folder, leave the rest in place. --------- Co-authored-by: Mustafa Akur <[email protected]>
1 parent a43938d commit 9d47dca

File tree

21 files changed

+614
-276
lines changed

21 files changed

+614
-276
lines changed

datafusion-cli/Cargo.lock

Lines changed: 2 additions & 2 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.
Lines changed: 198 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,198 @@
1+
// Licensed to the Apache Software Foundation (ASF) under one
2+
// or more contributor license agreements. See the NOTICE file
3+
// distributed with this work for additional information
4+
// regarding copyright ownership. The ASF licenses this file
5+
// to you under the Apache License, Version 2.0 (the
6+
// "License"); you may not use this file except in compliance
7+
// with the License. You may obtain a copy of the License at
8+
//
9+
// http://www.apache.org/licenses/LICENSE-2.0
10+
//
11+
// Unless required by applicable law or agreed to in writing,
12+
// software distributed under the License is distributed on an
13+
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14+
// KIND, either express or implied. See the License for the
15+
// specific language governing permissions and limitations
16+
// under the License.
17+
18+
use std::any::Any;
19+
use std::collections::HashMap;
20+
use std::sync::Arc;
21+
22+
use arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit};
23+
use datafusion_common::config::ConfigOptions;
24+
use datafusion_common::{plan_err, Result};
25+
use datafusion_expr::{AggregateUDF, LogicalPlan, ScalarUDF, TableSource, WindowUDF};
26+
use datafusion_optimizer::analyzer::Analyzer;
27+
use datafusion_optimizer::optimizer::Optimizer;
28+
use datafusion_optimizer::{OptimizerConfig, OptimizerContext};
29+
use datafusion_sql::planner::{ContextProvider, SqlToRel};
30+
use datafusion_sql::sqlparser::ast::Statement;
31+
use datafusion_sql::sqlparser::dialect::GenericDialect;
32+
use datafusion_sql::sqlparser::parser::Parser;
33+
use datafusion_sql::TableReference;
34+
35+
use chrono::DateTime;
36+
use datafusion_functions::datetime;
37+
38+
#[cfg(test)]
39+
#[ctor::ctor]
40+
fn init() {
41+
// enable logging so RUST_LOG works
42+
let _ = env_logger::try_init();
43+
}
44+
45+
#[test]
46+
fn timestamp_nano_ts_none_predicates() -> Result<()> {
47+
let sql = "SELECT col_int32
48+
FROM test
49+
WHERE col_ts_nano_none < (now() - interval '1 hour')";
50+
let plan = test_sql(sql)?;
51+
// a scan should have the now()... predicate folded to a single
52+
// constant and compared to the column without a cast so it can be
53+
// pushed down / pruned
54+
let expected =
55+
"Projection: test.col_int32\
56+
\n Filter: test.col_ts_nano_none < TimestampNanosecond(1666612093000000000, None)\
57+
\n TableScan: test projection=[col_int32, col_ts_nano_none]";
58+
assert_eq!(expected, format!("{plan:?}"));
59+
Ok(())
60+
}
61+
62+
#[test]
63+
fn timestamp_nano_ts_utc_predicates() {
64+
let sql = "SELECT col_int32
65+
FROM test
66+
WHERE col_ts_nano_utc < (now() - interval '1 hour')";
67+
let plan = test_sql(sql).unwrap();
68+
// a scan should have the now()... predicate folded to a single
69+
// constant and compared to the column without a cast so it can be
70+
// pushed down / pruned
71+
let expected =
72+
"Projection: test.col_int32\n Filter: test.col_ts_nano_utc < TimestampNanosecond(1666612093000000000, Some(\"+00:00\"))\
73+
\n TableScan: test projection=[col_int32, col_ts_nano_utc]";
74+
assert_eq!(expected, format!("{plan:?}"));
75+
}
76+
77+
fn test_sql(sql: &str) -> Result<LogicalPlan> {
78+
// parse the SQL
79+
let dialect = GenericDialect {}; // or AnsiDialect, or your own dialect ...
80+
let ast: Vec<Statement> = Parser::parse_sql(&dialect, sql).unwrap();
81+
let statement = &ast[0];
82+
83+
// create a logical query plan
84+
let now_udf = datetime::functions()
85+
.iter()
86+
.find(|f| f.name() == "now")
87+
.unwrap()
88+
.to_owned();
89+
let context_provider = MyContextProvider::default().with_udf(now_udf);
90+
let sql_to_rel = SqlToRel::new(&context_provider);
91+
let plan = sql_to_rel.sql_statement_to_plan(statement.clone()).unwrap();
92+
93+
// hard code the return value of now()
94+
let now_time = DateTime::from_timestamp(1666615693, 0).unwrap();
95+
let config = OptimizerContext::new()
96+
.with_skip_failing_rules(false)
97+
.with_query_execution_start_time(now_time);
98+
let analyzer = Analyzer::new();
99+
let optimizer = Optimizer::new();
100+
// analyze and optimize the logical plan
101+
let plan = analyzer.execute_and_check(&plan, config.options(), |_, _| {})?;
102+
optimizer.optimize(&plan, &config, |_, _| {})
103+
}
104+
105+
#[derive(Default)]
106+
struct MyContextProvider {
107+
options: ConfigOptions,
108+
udfs: HashMap<String, Arc<ScalarUDF>>,
109+
}
110+
111+
impl MyContextProvider {
112+
fn with_udf(mut self, udf: Arc<ScalarUDF>) -> Self {
113+
self.udfs.insert(udf.name().to_string(), udf);
114+
self
115+
}
116+
}
117+
118+
impl ContextProvider for MyContextProvider {
119+
fn get_table_source(&self, name: TableReference) -> Result<Arc<dyn TableSource>> {
120+
let table_name = name.table();
121+
if table_name.starts_with("test") {
122+
let schema = Schema::new_with_metadata(
123+
vec![
124+
Field::new("col_int32", DataType::Int32, true),
125+
Field::new("col_uint32", DataType::UInt32, true),
126+
Field::new("col_utf8", DataType::Utf8, true),
127+
Field::new("col_date32", DataType::Date32, true),
128+
Field::new("col_date64", DataType::Date64, true),
129+
// timestamp with no timezone
130+
Field::new(
131+
"col_ts_nano_none",
132+
DataType::Timestamp(TimeUnit::Nanosecond, None),
133+
true,
134+
),
135+
// timestamp with UTC timezone
136+
Field::new(
137+
"col_ts_nano_utc",
138+
DataType::Timestamp(TimeUnit::Nanosecond, Some("+00:00".into())),
139+
true,
140+
),
141+
],
142+
HashMap::new(),
143+
);
144+
145+
Ok(Arc::new(MyTableSource {
146+
schema: Arc::new(schema),
147+
}))
148+
} else {
149+
plan_err!("table does not exist")
150+
}
151+
}
152+
153+
fn get_function_meta(&self, name: &str) -> Option<Arc<ScalarUDF>> {
154+
self.udfs.get(name).cloned()
155+
}
156+
157+
fn get_aggregate_meta(&self, _name: &str) -> Option<Arc<AggregateUDF>> {
158+
None
159+
}
160+
161+
fn get_variable_type(&self, _variable_names: &[String]) -> Option<DataType> {
162+
None
163+
}
164+
165+
fn get_window_meta(&self, _name: &str) -> Option<Arc<WindowUDF>> {
166+
None
167+
}
168+
169+
fn options(&self) -> &ConfigOptions {
170+
&self.options
171+
}
172+
173+
fn udfs_names(&self) -> Vec<String> {
174+
Vec::new()
175+
}
176+
177+
fn udafs_names(&self) -> Vec<String> {
178+
Vec::new()
179+
}
180+
181+
fn udwfs_names(&self) -> Vec<String> {
182+
Vec::new()
183+
}
184+
}
185+
186+
struct MyTableSource {
187+
schema: SchemaRef,
188+
}
189+
190+
impl TableSource for MyTableSource {
191+
fn as_any(&self) -> &dyn Any {
192+
self
193+
}
194+
195+
fn schema(&self) -> SchemaRef {
196+
self.schema.clone()
197+
}
198+
}

datafusion/core/tests/simplification.rs

Lines changed: 26 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -185,10 +185,6 @@ fn make_udf_add(volatility: Volatility) -> Arc<ScalarUDF> {
185185
))
186186
}
187187

188-
fn now_expr() -> Expr {
189-
call_fn("now", vec![]).unwrap()
190-
}
191-
192188
fn cast_to_int64_expr(expr: Expr) -> Expr {
193189
Expr::Cast(Cast::new(expr.into(), DataType::Int64))
194190
}
@@ -255,7 +251,7 @@ fn now_less_than_timestamp() -> Result<()> {
255251
// cast(now() as int) < cast(to_timestamp(...) as int) + 50000_i64
256252
let plan = LogicalPlanBuilder::from(table_scan)
257253
.filter(
258-
cast_to_int64_expr(now_expr())
254+
cast_to_int64_expr(now())
259255
.lt(cast_to_int64_expr(to_timestamp_expr(ts_string)) + lit(50000_i64)),
260256
)?
261257
.build()?;
@@ -368,14 +364,14 @@ fn test_const_evaluator_now() {
368364
let time = chrono::Utc.timestamp_nanos(ts_nanos);
369365
let ts_string = "2020-09-08T12:05:00+00:00";
370366
// now() --> ts
371-
test_evaluate_with_start_time(now_expr(), lit_timestamp_nano(ts_nanos), &time);
367+
test_evaluate_with_start_time(now(), lit_timestamp_nano(ts_nanos), &time);
372368

373369
// CAST(now() as int64) + 100_i64 --> ts + 100_i64
374-
let expr = cast_to_int64_expr(now_expr()) + lit(100_i64);
370+
let expr = cast_to_int64_expr(now()) + lit(100_i64);
375371
test_evaluate_with_start_time(expr, lit(ts_nanos + 100), &time);
376372

377373
// CAST(now() as int64) < cast(to_timestamp(...) as int64) + 50000_i64 ---> true
378-
let expr = cast_to_int64_expr(now_expr())
374+
let expr = cast_to_int64_expr(now())
379375
.lt(cast_to_int64_expr(to_timestamp_expr(ts_string)) + lit(50000i64));
380376
test_evaluate_with_start_time(expr, lit(true), &time);
381377
}
@@ -413,3 +409,25 @@ fn test_evaluator_udfs() {
413409
));
414410
test_evaluate(expr, expected_expr);
415411
}
412+
413+
#[test]
414+
fn multiple_now() -> Result<()> {
415+
let table_scan = test_table_scan();
416+
let time = Utc::now();
417+
let proj = vec![now(), now().alias("t2")];
418+
let plan = LogicalPlanBuilder::from(table_scan)
419+
.project(proj)?
420+
.build()?;
421+
422+
// expect the same timestamp appears in both exprs
423+
let actual = get_optimized_plan_formatted(&plan, &time);
424+
let expected = format!(
425+
"Projection: TimestampNanosecond({}, Some(\"+00:00\")) AS now(), TimestampNanosecond({}, Some(\"+00:00\")) AS t2\
426+
\n TableScan: test",
427+
time.timestamp_nanos_opt().unwrap(),
428+
time.timestamp_nanos_opt().unwrap()
429+
);
430+
431+
assert_eq!(expected, actual);
432+
Ok(())
433+
}

datafusion/expr/src/built_in_function.rs

Lines changed: 0 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -204,14 +204,6 @@ pub enum BuiltinScalarFunction {
204204
Substr,
205205
/// to_hex
206206
ToHex,
207-
/// from_unixtime
208-
FromUnixtime,
209-
///now
210-
Now,
211-
///current_date
212-
CurrentDate,
213-
/// current_time
214-
CurrentTime,
215207
/// make_date
216208
MakeDate,
217209
/// translate
@@ -369,17 +361,11 @@ impl BuiltinScalarFunction {
369361
BuiltinScalarFunction::Translate => Volatility::Immutable,
370362
BuiltinScalarFunction::Trim => Volatility::Immutable,
371363
BuiltinScalarFunction::Upper => Volatility::Immutable,
372-
BuiltinScalarFunction::FromUnixtime => Volatility::Immutable,
373364
BuiltinScalarFunction::OverLay => Volatility::Immutable,
374365
BuiltinScalarFunction::Levenshtein => Volatility::Immutable,
375366
BuiltinScalarFunction::SubstrIndex => Volatility::Immutable,
376367
BuiltinScalarFunction::FindInSet => Volatility::Immutable,
377368

378-
// Stable builtin functions
379-
BuiltinScalarFunction::Now => Volatility::Stable,
380-
BuiltinScalarFunction::CurrentDate => Volatility::Stable,
381-
BuiltinScalarFunction::CurrentTime => Volatility::Stable,
382-
383369
// Volatile builtin functions
384370
BuiltinScalarFunction::Random => Volatility::Volatile,
385371
BuiltinScalarFunction::Uuid => Volatility::Volatile,
@@ -396,7 +382,6 @@ impl BuiltinScalarFunction {
396382
/// 2. Deduce the output `DataType` based on the provided `input_expr_types`.
397383
pub fn return_type(self, input_expr_types: &[DataType]) -> Result<DataType> {
398384
use DataType::*;
399-
use TimeUnit::*;
400385

401386
// Note that this function *must* return the same type that the respective physical expression returns
402387
// or the execution panics.
@@ -544,12 +529,6 @@ impl BuiltinScalarFunction {
544529
utf8_to_int_type(&input_expr_types[0], "find_in_set")
545530
}
546531
BuiltinScalarFunction::ToChar => Ok(Utf8),
547-
BuiltinScalarFunction::FromUnixtime => Ok(Timestamp(Second, None)),
548-
BuiltinScalarFunction::Now => {
549-
Ok(Timestamp(Nanosecond, Some("+00:00".into())))
550-
}
551-
BuiltinScalarFunction::CurrentDate => Ok(Date32),
552-
BuiltinScalarFunction::CurrentTime => Ok(Time64(Nanosecond)),
553532
BuiltinScalarFunction::MakeDate => Ok(Date32),
554533
BuiltinScalarFunction::Translate => {
555534
utf8_to_str_type(&input_expr_types[0], "translate")
@@ -757,9 +736,6 @@ impl BuiltinScalarFunction {
757736
],
758737
self.volatility(),
759738
),
760-
BuiltinScalarFunction::FromUnixtime => {
761-
Signature::uniform(1, vec![Int64], self.volatility())
762-
}
763739
BuiltinScalarFunction::Digest => Signature::one_of(
764740
vec![
765741
Exact(vec![Utf8, Utf8]),
@@ -904,11 +880,6 @@ impl BuiltinScalarFunction {
904880
// will be as good as the number of digits in the number
905881
Signature::uniform(1, vec![Float64, Float32], self.volatility())
906882
}
907-
BuiltinScalarFunction::Now
908-
| BuiltinScalarFunction::CurrentDate
909-
| BuiltinScalarFunction::CurrentTime => {
910-
Signature::uniform(0, vec![], self.volatility())
911-
}
912883
BuiltinScalarFunction::MakeDate => Signature::uniform(
913884
3,
914885
vec![Int32, Int64, UInt32, UInt64, Utf8],
@@ -1032,12 +1003,8 @@ impl BuiltinScalarFunction {
10321003
BuiltinScalarFunction::FindInSet => &["find_in_set"],
10331004

10341005
// time/date functions
1035-
BuiltinScalarFunction::Now => &["now"],
1036-
BuiltinScalarFunction::CurrentDate => &["current_date", "today"],
1037-
BuiltinScalarFunction::CurrentTime => &["current_time"],
10381006
BuiltinScalarFunction::MakeDate => &["make_date"],
10391007
BuiltinScalarFunction::ToChar => &["to_char", "date_format"],
1040-
BuiltinScalarFunction::FromUnixtime => &["from_unixtime"],
10411008

10421009
// hashing functions
10431010
BuiltinScalarFunction::Digest => &["digest"],

datafusion/expr/src/expr_fn.rs

Lines changed: 0 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -788,15 +788,6 @@ scalar_expr!(
788788
datetime format,
789789
"converts a date, time, timestamp or duration to a string based on the provided format"
790790
);
791-
scalar_expr!(
792-
FromUnixtime,
793-
from_unixtime,
794-
unixtime,
795-
"returns the unix time in format"
796-
);
797-
scalar_expr!(CurrentDate, current_date, ,"returns current UTC date as a [`DataType::Date32`] value");
798-
scalar_expr!(Now, now, ,"returns current timestamp in nanoseconds, using the same value for all instances of now() in same statement");
799-
scalar_expr!(CurrentTime, current_time, , "returns current UTC time as a [`DataType::Time64`] value");
800791
scalar_expr!(MakeDate, make_date, year month day, "make a date from year, month and day component parts");
801792
scalar_expr!(Nanvl, nanvl, x y, "returns x if x is not NaN otherwise returns y");
802793
scalar_expr!(
@@ -1258,8 +1249,6 @@ mod test {
12581249
test_scalar_expr!(Trim, trim, string);
12591250
test_scalar_expr!(Upper, upper, string);
12601251

1261-
test_scalar_expr!(FromUnixtime, from_unixtime, unixtime);
1262-
12631252
test_scalar_expr!(ArrayPopFront, array_pop_front, array);
12641253
test_scalar_expr!(ArrayPopBack, array_pop_back, array);
12651254
test_scalar_expr!(ArrayPosition, array_position, array, element, index);

datafusion/expr/src/signature.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,7 @@ pub enum Volatility {
4444
Immutable,
4545
/// A stable function may return different values given the same input across different
4646
/// queries but must return the same value for a given input within a query. An example of
47-
/// this is [super::BuiltinScalarFunction::Now]. DataFusion
47+
/// this is the `Now` function. DataFusion
4848
/// will attempt to inline `Stable` functions during planning, when possible.
4949
/// For query `select col1, now() from t1`, it might take a while to execute but
5050
/// `now()` column will be the same for each output row, which is evaluated

0 commit comments

Comments
 (0)