datafusion_functions/datetime/mod.rs
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
// 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.
//! date & time DataFusion functions
use std::sync::Arc;
use datafusion_expr::ScalarUDF;
pub mod common;
pub mod current_date;
pub mod current_time;
pub mod date_bin;
pub mod date_part;
pub mod date_trunc;
pub mod from_unixtime;
pub mod make_date;
pub mod now;
pub mod to_char;
pub mod to_date;
pub mod to_local_time;
pub mod to_timestamp;
pub mod to_unixtime;
// create UDFs
make_udf_function!(current_date::CurrentDateFunc, CURRENT_DATE, current_date);
make_udf_function!(current_time::CurrentTimeFunc, CURRENT_TIME, current_time);
make_udf_function!(date_bin::DateBinFunc, DATE_BIN, date_bin);
make_udf_function!(date_part::DatePartFunc, DATE_PART, date_part);
make_udf_function!(date_trunc::DateTruncFunc, DATE_TRUNC, date_trunc);
make_udf_function!(make_date::MakeDateFunc, MAKE_DATE, make_date);
make_udf_function!(
from_unixtime::FromUnixtimeFunc,
FROM_UNIXTIME,
from_unixtime
);
make_udf_function!(now::NowFunc, NOW, now);
make_udf_function!(to_char::ToCharFunc, TO_CHAR, to_char);
make_udf_function!(to_date::ToDateFunc, TO_DATE, to_date);
make_udf_function!(to_local_time::ToLocalTimeFunc, TO_LOCAL_TIME, to_local_time);
make_udf_function!(to_unixtime::ToUnixtimeFunc, TO_UNIXTIME, to_unixtime);
make_udf_function!(to_timestamp::ToTimestampFunc, TO_TIMESTAMP, to_timestamp);
make_udf_function!(
to_timestamp::ToTimestampSecondsFunc,
TO_TIMESTAMP_SECONDS,
to_timestamp_seconds
);
make_udf_function!(
to_timestamp::ToTimestampMillisFunc,
TO_TIMESTAMP_MILLIS,
to_timestamp_millis
);
make_udf_function!(
to_timestamp::ToTimestampMicrosFunc,
TO_TIMESTAMP_MICROS,
to_timestamp_micros
);
make_udf_function!(
to_timestamp::ToTimestampNanosFunc,
TO_TIMESTAMP_NANOS,
to_timestamp_nanos
);
// we cannot currently use the export_functions macro since it doesn't handle
// functions with varargs currently
pub mod expr_fn {
use datafusion_expr::Expr;
export_functions!((
current_date,
"returns current UTC date as a Date32 value",
),(
current_time,
"returns current UTC time as a Time64 value",
),(
from_unixtime,
"converts an integer to RFC3339 timestamp format string",
unixtime
),(
date_bin,
"coerces an arbitrary timestamp to the start of the nearest specified interval",
stride source origin
),(
date_part,
"extracts a subfield from the date",
part date
),(
date_trunc,
"truncates the date to a specified level of precision",
part date
),(
make_date,
"make a date from year, month and day component parts",
year month day
),(
now,
"returns the current timestamp in nanoseconds, using the same value for all instances of now() in same statement",
),
(
to_local_time,
"converts a timezone-aware timestamp to local time (with no offset or timezone information), i.e. strips off the timezone from the timestamp",
args,
),
(
to_unixtime,
"converts a string and optional formats to a Unixtime",
args,
),(
to_timestamp,
"converts a string and optional formats to a `Timestamp(Nanoseconds, None)`",
args,
),(
to_timestamp_seconds,
"converts a string and optional formats to a `Timestamp(Seconds, None)`",
args,
),(
to_timestamp_millis,
"converts a string and optional formats to a `Timestamp(Milliseconds, None)`",
args,
),(
to_timestamp_micros,
"converts a string and optional formats to a `Timestamp(Microseconds, None)`",
args,
),(
to_timestamp_nanos,
"converts a string and optional formats to a `Timestamp(Nanoseconds, None)`",
args,
));
/// Returns a string representation of a date, time, timestamp or duration based
/// on a Chrono pattern.
///
/// The syntax for the patterns can be found at
/// <https://docs.rs/chrono/latest/chrono/format/strftime/index.html>
///
/// # Examples
///
/// ```ignore
/// # use chrono::prelude::*;
/// # use datafusion::prelude::*;
/// # use datafusion::error::Result;
/// # use datafusion_common::ScalarValue::TimestampNanosecond;
/// # use std::sync::Arc;
/// # use arrow_array::{Date32Array, RecordBatch, StringArray};
/// # use arrow_schema::{DataType, Field, Schema};
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// let schema = Arc::new(Schema::new(vec![
/// Field::new("values", DataType::Date32, false),
/// Field::new("patterns", DataType::Utf8, false),
/// ]));
///
/// let batch = RecordBatch::try_new(
/// schema,
/// vec![
/// Arc::new(Date32Array::from(vec![
/// 18506,
/// 18507,
/// 18508,
/// 18509,
/// ])),
/// Arc::new(StringArray::from(vec![
/// "%Y-%m-%d",
/// "%Y:%m:%d",
/// "%Y%m%d",
/// "%d-%m-%Y",
/// ])),
/// ],
/// )?;
///
/// let ctx = SessionContext::new();
/// ctx.register_batch("t", batch)?;
/// let df = ctx.table("t").await?;
///
/// // use the to_char function to convert col 'values',
/// // to strings using patterns in col 'patterns'
/// let df = df.with_column(
/// "date_str",
/// to_char(col("values"), col("patterns"))
/// )?;
/// // Note that providing a scalar value for the pattern
/// // is more performant
/// let df = df.with_column(
/// "date_str2",
/// to_char(col("values"), lit("%d-%m-%Y"))
/// )?;
/// // literals can be used as well with dataframe calls
/// let timestamp = "2026-07-08T09:10:11"
/// .parse::<NaiveDateTime>()
/// .unwrap()
/// .with_nanosecond(56789)
/// .unwrap()
/// .timestamp_nanos_opt()
/// .unwrap();
/// let df = df.with_column(
/// "timestamp_str",
/// to_char(lit(TimestampNanosecond(Some(timestamp), None)), lit("%d-%m-%Y %H:%M:%S"))
/// )?;
///
/// df.show().await?;
///
/// # Ok(())
/// # }
/// ```
pub fn to_char(datetime: Expr, format: Expr) -> Expr {
super::to_char().call(vec![datetime, format])
}
/// ```ignore
/// # use std::sync::Arc;
///
/// # use datafusion_common::Result;
///
/// # #[tokio::main]
/// # async fn main() -> Result<()> {
/// # use arrow::array::StringArray;
/// # use arrow::datatypes::{DataType, Field, Schema};
/// # use arrow::record_batch::RecordBatch;
/// # use datafusion_expr::col;
/// # use datafusion::prelude::*;
/// # use datafusion_functions::expr_fn::to_date;
///
/// // define a schema.
/// let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, false)]));
///
/// // define data.
/// let batch = RecordBatch::try_new(
/// schema,
/// vec![Arc::new(StringArray::from(vec![
/// "2020-09-08T13:42:29Z",
/// "2020-09-08T13:42:29.190855-05:00",
/// "2020-08-09 12:13:29",
/// "2020-01-02",
/// ]))],
/// )?;
///
/// // declare a new context. In spark API, this corresponds to a new spark SQLsession
/// let ctx = SessionContext::new();
///
/// // declare a table in memory. In spark API, this corresponds to createDataFrame(...).
/// ctx.register_batch("t", batch)?;
/// let df = ctx.table("t").await?;
///
/// // use to_date function to convert col 'a' to timestamp type using the default parsing
/// let df = df.with_column("a", to_date(vec![col("a")]))?;
///
/// let df = df.select_columns(&["a"])?;
///
/// // print the results
/// df.show().await?;
///
/// # Ok(())
/// # }
/// ```
pub fn to_date(args: Vec<Expr>) -> Expr {
super::to_date().call(args)
}
}
/// Returns all DataFusion functions defined in this package
pub fn functions() -> Vec<Arc<ScalarUDF>> {
vec![
current_date(),
current_time(),
date_bin(),
date_part(),
date_trunc(),
from_unixtime(),
make_date(),
now(),
to_char(),
to_date(),
to_local_time(),
to_unixtime(),
to_timestamp(),
to_timestamp_seconds(),
to_timestamp_millis(),
to_timestamp_micros(),
to_timestamp_nanos(),
]
}