Skip to main content

datafusion_functions/datetime/
now.rs

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
18use arrow::datatypes::DataType::Timestamp;
19use arrow::datatypes::TimeUnit::Nanosecond;
20use arrow::datatypes::{DataType, Field, FieldRef};
21use std::sync::Arc;
22
23use datafusion_common::config::ConfigOptions;
24use datafusion_common::{Result, ScalarValue, internal_err};
25use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyContext};
26use datafusion_expr::{
27    ColumnarValue, Documentation, Expr, ReturnFieldArgs, ScalarFunctionArgs, ScalarUDF,
28    ScalarUDFImpl, Signature, Volatility,
29};
30use datafusion_macros::user_doc;
31
32#[user_doc(
33    doc_section(label = "Time and Date Functions"),
34    description = r#"
35Returns the current timestamp in the system configured timezone (None by default).
36
37The `now()` return value is determined at query time and will return the same timestamp, no matter when in the query plan the function executes.
38"#,
39    syntax_example = "now()",
40    sql_example = r#"```sql
41> SELECT now();
42+----------------------------------+
43| now()                            |
44+----------------------------------+
45| 2024-12-23T06:30:00.123456789    |
46+----------------------------------+
47
48-- The timezone of the returned timestamp depends on the session time zone
49> SET datafusion.execution.time_zone = 'America/New_York';
50> SELECT now();
51+--------------------------------------+
52| now()                                |
53+--------------------------------------+
54| 2024-12-23T01:30:00.123456789-05:00  |
55+--------------------------------------+
56```"#
57)]
58#[derive(Debug, PartialEq, Eq, Hash)]
59pub struct NowFunc {
60    signature: Signature,
61    aliases: Vec<String>,
62    timezone: Option<Arc<str>>,
63}
64
65impl Default for NowFunc {
66    fn default() -> Self {
67        Self::new_with_config(&ConfigOptions::default())
68    }
69}
70
71impl NowFunc {
72    #[deprecated(since = "50.2.0", note = "use `new_with_config` instead")]
73    /// Deprecated constructor retained for backwards compatibility.
74    ///
75    /// Prefer [`NowFunc::new_with_config`] which allows specifying the
76    /// timezone via [`ConfigOptions`]. This helper now mirrors the
77    /// canonical default offset (None) provided by `ConfigOptions::default()`.
78    pub fn new() -> Self {
79        Self::new_with_config(&ConfigOptions::default())
80    }
81
82    pub fn new_with_config(config: &ConfigOptions) -> Self {
83        Self {
84            signature: Signature::nullary(Volatility::Stable),
85            aliases: vec!["current_timestamp".to_string()],
86            timezone: config
87                .execution
88                .time_zone
89                .as_ref()
90                .map(|tz| Arc::from(tz.as_str())),
91        }
92    }
93}
94
95/// Create an implementation of `now()` that always returns the
96/// specified timestamp.
97///
98/// The semantics of `now()` require it to return the same value
99/// wherever it appears within a single statement. This value is
100/// chosen during planning time.
101impl ScalarUDFImpl for NowFunc {
102    fn name(&self) -> &str {
103        "now"
104    }
105
106    fn signature(&self) -> &Signature {
107        &self.signature
108    }
109
110    fn with_updated_config(&self, config: &ConfigOptions) -> Option<ScalarUDF> {
111        Some(Self::new_with_config(config).into())
112    }
113
114    fn return_field_from_args(&self, _args: ReturnFieldArgs) -> Result<FieldRef> {
115        Ok(Field::new(
116            self.name(),
117            Timestamp(Nanosecond, self.timezone.clone()),
118            false,
119        )
120        .into())
121    }
122
123    fn return_type(&self, _arg_types: &[DataType]) -> Result<DataType> {
124        internal_err!("return_field_from_args should be called instead")
125    }
126
127    fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result<ColumnarValue> {
128        internal_err!("invoke should not be called on a simplified now() function")
129    }
130
131    fn simplify(
132        &self,
133        args: Vec<Expr>,
134        info: &SimplifyContext,
135    ) -> Result<ExprSimplifyResult> {
136        let Some(now_ts) = info.query_execution_start_time() else {
137            return Ok(ExprSimplifyResult::Original(args));
138        };
139
140        Ok(ExprSimplifyResult::Simplified(Expr::Literal(
141            ScalarValue::TimestampNanosecond(
142                now_ts.timestamp_nanos_opt(),
143                self.timezone.clone(),
144            ),
145            None,
146        )))
147    }
148
149    fn aliases(&self) -> &[String] {
150        &self.aliases
151    }
152
153    fn documentation(&self) -> Option<&Documentation> {
154        self.doc()
155    }
156}
157
158#[cfg(test)]
159mod tests {
160    use super::*;
161
162    #[expect(deprecated)]
163    #[test]
164    fn now_func_default_matches_config() {
165        let default_config = ConfigOptions::default();
166
167        let legacy_now = NowFunc::new();
168        let configured_now = NowFunc::new_with_config(&default_config);
169
170        let empty_fields: [FieldRef; 0] = [];
171        let empty_scalars: [Option<&ScalarValue>; 0] = [];
172
173        let legacy_field = legacy_now
174            .return_field_from_args(ReturnFieldArgs {
175                arg_fields: &empty_fields,
176                scalar_arguments: &empty_scalars,
177            })
178            .expect("legacy now() return field");
179
180        let configured_field = configured_now
181            .return_field_from_args(ReturnFieldArgs {
182                arg_fields: &empty_fields,
183                scalar_arguments: &empty_scalars,
184            })
185            .expect("configured now() return field");
186
187        assert_eq!(legacy_field.as_ref(), configured_field.as_ref());
188
189        let legacy_scalar =
190            ScalarValue::TimestampNanosecond(None, legacy_now.timezone.clone());
191        let configured_scalar =
192            ScalarValue::TimestampNanosecond(None, configured_now.timezone.clone());
193
194        assert_eq!(legacy_scalar, configured_scalar);
195        assert_eq!(None, legacy_now.timezone.as_deref());
196    }
197}