SQL: Implement DATE_PART function ()

DATE_PART(<datetime unit>, <date/datetime>) is a function that allows
the user to extract the specified unit from a date/datetime field
similar to the EXTRACT (<datetime unit> FROM <date/datetime>) but
with different names and aliases for the units and it also provides more
options like `DATE_PART('tzoffset', datetimeField)`.

Implemented following the SQL server's spec: https://docs.microsoft.com/en-us/sql/t-sql/functions/datepart-transact-sql?view=sql-server-2017
with the difference that the <datetime unit> argument is either a
literal single quoted string or gets a value from a table field, whereas
in SQL server keywords are used (unquoted identifiers) and it's not
possible to use a value coming for a table column.

Closes: 
(cherry picked from commit ead743d3579eb753fd314d4a58fae205e465d72e)
This commit is contained in:
Marios Trivyzas 2019-10-01 15:59:25 +03:00
parent 4335e07716
commit f792dbf239
23 changed files with 976 additions and 197 deletions

@ -248,6 +248,97 @@ include-tagged::{sql-specs}/docs/docs.csv-spec[filterNow]
Currently, using a _precision_ greater than 3 doesn't make any difference to the output of the
function as the maximum number of second fractional digits returned is 3 (milliseconds).
[[sql-functions-datetime-part]]
==== `DATE_PART/DATEPART`
.Synopsis:
[source, sql]
--------------------------------------------------
DATE_PART(
string_exp, <1>
datetime_exp) <2>
--------------------------------------------------
*Input*:
<1> string expression denoting the unit to extract from the date/datetime
<2> date/datetime expression
*Output*: integer
.Description:
Extract the specified unit from a date/datetime. If any of the two arguments is `null` a `null` is returned.
It's similar to <<sql-functions-datetime-extract>> but with different names and aliases for the units and
provides more options (e.g.: `TZOFFSET`).
[cols="^,^"]
|===
2+h|Datetime units to extract
s|unit
s|abbreviations
| year | years, yy, yyyy
| quarter | quarters, qq, q
| month | months, mm, m
| dayofyear | dy, y
| day | days, dd, d
| week | weeks, wk, ww
| weekday | weekdays, dw
| hour | hours, hh
| minute | minutes, mi, n
| second | seconds, ss, s
| millisecond | milliseconds, ms
| microsecond | microseconds, mcs
| nanosecond | nanoseconds, ns
| tzoffset | tz
|===
[source, sql]
--------------------------------------------------
include-tagged::{sql-specs}/docs/docs.csv-spec[datePartDateTimeYears]
--------------------------------------------------
[source, sql]
--------------------------------------------------
include-tagged::{sql-specs}/docs/docs.csv-spec[datePartDateTimeMinutes]
--------------------------------------------------
[source, sql]
--------------------------------------------------
include-tagged::{sql-specs}/docs/docs.csv-spec[datePartDateQuarter]
--------------------------------------------------
[source, sql]
--------------------------------------------------
include-tagged::{sql-specs}/docs/docs.csv-spec[datePartDateMonth]
--------------------------------------------------
[NOTE]
For `week` and `weekday` the unit is extracted using the non-ISO calculation, which means
that a given week is considered to start from Sunday, not Monday.
[source, sql]
--------------------------------------------------
include-tagged::{sql-specs}/docs/docs.csv-spec[datePartDateTimeWeek]
--------------------------------------------------
[NOTE]
The `tzoffset` returns the total number of minutes (signed) that represent the time zone's offset.
[source, sql]
--------------------------------------------------
include-tagged::{sql-specs}/docs/docs.csv-spec[datePartDateTimeTzOffsetPlus]
--------------------------------------------------
[source, sql]
--------------------------------------------------
include-tagged::{sql-specs}/docs/docs.csv-spec[datePartDateTimeTzOffsetMinus]
--------------------------------------------------
[[sql-functions-datetime-trunc]]
==== `DATE_TRUNC/DATETRUNC`

@ -51,6 +51,7 @@
** <<sql-functions-current-date>>
** <<sql-functions-current-time>>
** <<sql-functions-current-timestamp>>
** <<sql-functions-datetime-part>>
** <<sql-functions-datetime-trunc>>
** <<sql-functions-datetime-day>>
** <<sql-functions-datetime-dow>>

@ -41,7 +41,9 @@ CURRENT_DATE |SCALAR
CURRENT_TIME |SCALAR
CURRENT_TIMESTAMP|SCALAR
CURTIME |SCALAR
DATEPART |SCALAR
DATETRUNC |SCALAR
DATE_PART |SCALAR
DATE_TRUNC |SCALAR
DAY |SCALAR
DAYNAME |SCALAR

@ -260,7 +260,7 @@ SELECT count(*) as count, DATE_TRUNC('decade', hire_date) dt FROM test_emp GROUP
dateTruncHaving
schema::gender:s|dt:ts
SELECT gender, max(hire_date) dt FROM test_emp GROUP BY gender HAVING DATE_TRUNC('year', max(hire_date)) >= '1997-01-01T00:00:00.000Z'::timestamp ORDER BY 1;
SELECT gender, max(hire_date) AS dt FROM test_emp GROUP BY gender HAVING DATE_TRUNC('year', max(hire_date)) >= '1997-01-01T00:00:00.000Z'::timestamp ORDER BY 1;
gender | dt
--------+-------------------------
@ -268,6 +268,104 @@ null | 1999-04-30 00:00:00.000Z
F | 1997-05-19 00:00:00.000Z
;
selectDatePartWithDate
SELECT DATE_PART('year', '2019-09-04'::date) as dp_years, DATE_PART('quarter', '2019-09-04'::date) as dp_quarter, DATE_PART('month', '2019-09-04'::date) as dp_month,
DATE_PART('dayofyear', '2019-09-04'::date) as dp_doy, DATE_PART('day', '2019-09-04'::date) as dp_day, DATE_PART('week', '2019-09-04'::date) as dp_week,
DATE_PART('weekday', '2019-09-04'::date) as dp_weekday, DATE_PART('hour', '2019-09-04'::date) as dp_hour, DATE_PART('minute', '2019-09-04'::date) as dp_minute,
DATE_PART('second', '2019-09-04'::date) as dp_second, DATE_PART('millisecond', '2019-09-04'::date) as dp_millis, DATE_PART('mcs', '2019-09-04'::date) as dp_micros,
DATE_PART('ns', '2019-09-04'::date) as dp_nanos, DATE_PART('tz', '2019-09-04'::date) as dp_tzoffset;
dp_years | dp_quarter | dp_month | dp_doy | dp_day | dp_week | dp_weekday | dp_hour | dp_minute | dp_second | dp_millis | dp_micros | dp_nanos | dp_tzoffset
---------+------------+----------+--------+--------+---------+------------+---------+-----------+-----------+-----------+-----------+-----------+------------
2019 | 3 | 9 |247 | 4 | 36 | 4 | 0 | 0 | 0 | 0 | 0 | 0 | 0
;
selectDatePartWithDateTime
SELECT DATE_PART('year', '2019-09-04T11:22:33.123Z'::datetime) as dp_years, DATE_PART('quarter', '2019-09-04T11:22:33.123Z'::datetime) as dp_quarter, DATE_PART('month', '2019-09-04T11:22:33.123Z'::datetime) as dp_month,
DATE_PART('dayofyear', '2019-09-04T11:22:33.123Z'::datetime) as dp_doy, DATE_PART('day', '2019-09-04T11:22:33.123Z'::datetime) as dp_day, DATE_PART('week', '2019-09-04T11:22:33.123Z'::datetime) as dp_week,
DATE_PART('weekday', '2019-09-04T11:22:33.123Z'::datetime) as dp_weekday, DATE_PART('hour', '2019-09-04T11:22:33.123Z'::datetime) as dp_hour, DATE_PART('minute', '2019-09-04T11:22:33.123Z'::datetime) as dp_minute,
DATE_PART('second', '2019-09-04T11:22:33.123Z'::datetime) as dp_second, DATE_PART('millisecond', '2019-09-04T11:22:33.123Z'::datetime) as dp_millis, DATE_PART('mcs', '2019-09-04T11:22:33.123Z'::datetime) as dp_micros,
DATE_PART('ns', '2019-09-04T11:22:33.123Z'::datetime) as dp_nanos, DATE_PART('tz', '2019-09-04T11:22:33.123Z'::datetime) as dp_tzoffset;
dp_years | dp_quarter | dp_month | dp_doy | dp_day | dp_week | dp_weekday | dp_hour | dp_minute | dp_second | dp_millis | dp_micros | dp_nanos | dp_tzoffset
---------+------------+----------+--------+--------+---------+------------+---------+-----------+-----------+-----------+-----------+-----------+------------
2019 | 3 | 9 |247 | 4 | 36 | 4 | 11 | 22 | 33 | 123 | 123000 | 123000000 | 0
;
selectDatePartWithNullTruncateField
SELECT DATE_PART(null, birth_date) AS dp FROM test_emp LIMIT 5;
dp:i
------
null
null
null
null
null
;
selectDatePartWithComplexExpressions
SELECT gender, birth_date, DATE_PART(CASE WHEN gender = 'M' THEN CONCAT(gender, 'onths') WHEN gender = 'F' THEN 'year' ELSE 'quarter' END,
birth_date + INTERVAL 10 month) AS dp FROM test_emp WHERE dp > 10 ORDER BY emp_no LIMIT 5;
gender | birth_date | dp
--------+--------------------------+------
F | 1964-06-02 00:00:00.000Z | 1965
M | 1955-01-21 00:00:00.000Z | 11
F | 1953-04-20 00:00:00.000Z | 1954
F | 1957-05-23 00:00:00.000Z | 1958
M | 1958-02-19 00:00:00.000Z | 12
;
datePartOrderBy
schema::emp_no:i|hire_date:ts|dp:i
SELECT emp_no, hire_date, DATE_PART('month', hire_date) as dp FROM test_emp ORDER BY dp DESC NULLS LAST, emp_no LIMIT 5;
emp_no | hire_date | dp
--------+--------------------------+----
10004 | 1986-12-01 00:00:00.000Z | 12
10012 | 1992-12-18 00:00:00.000Z | 12
10023 | 1989-12-17 00:00:00.000Z | 12
10037 | 1990-12-05 00:00:00.000Z | 12
10050 | 1990-12-25 00:00:00.000z | 12
;
datePartFilter
schema::emp_no:i|hire_date:ts|dp:i
SELECT emp_no, hire_date, DATE_PART('day', hire_date) as dp FROM test_emp WHERE DATE_PART('day', hire_date) > 27 order by emp_no;
emp_no | hire_date | dp
--------+--------------------------+----
10003 | 1986-08-28 00:00:00.000Z | 28
10019 | 1999-04-30 00:00:00.000Z | 30
10047 | 1989-03-31 00:00:00.000Z | 31
10062 | 1991-08-30 00:00:00.000Z | 30
10081 | 1986-10-30 00:00:00.000Z | 30
10083 | 1987-03-31 00:00:00.000Z | 31
;
datePartGroupBy
schema::count:l|dp:i
SELECT count(*) as count, DATE_PART('quarter', hire_date) dp FROM test_emp GROUP BY dp ORDER BY 2;
count | dp
-------+----
30 | 1
19 | 2
27 | 3
24 | 4
;
datePartHaving
schema::gender:s|dt:ts
SELECT gender, max(hire_date) AS dt FROM test_emp GROUP BY gender HAVING DATE_PART('year', max(hire_date)) < 1999 ORDER BY 1;
gender | dt
--------+------------------------
F | 1997-05-19 00:00:00.000Z
M | 1996-11-05 00:00:00.000Z
;
//
// Aggregate
//

@ -237,7 +237,9 @@ CURRENT_DATE |SCALAR
CURRENT_TIME |SCALAR
CURRENT_TIMESTAMP|SCALAR
CURTIME |SCALAR
DATEPART |SCALAR
DATETRUNC |SCALAR
DATE_PART |SCALAR
DATE_TRUNC |SCALAR
DAY |SCALAR
DAYNAME |SCALAR
@ -2415,6 +2417,80 @@ SELECT DAY_OF_MONTH(CAST('2018-02-19T10:23:27Z' AS TIMESTAMP)) AS day;
// end::dayOfMonth
;
datePartDateTimeYears
// tag::datePartDateTimeYears
SELECT DATE_PART('year', '2019-09-22T11:22:33.123Z'::datetime) AS "years";
years
----------
2019
// end::datePartDateTimeYears
;
datePartDateTimeWeek
// tag::datePartDateTimeWeek
SELECT DATE_PART('week', '2019-09-22T11:22:33.123Z'::datetime) AS week;
week
----------
39
// end::datePartDateTimeWeek
;
datePartDateTimeMinutes
// tag::datePartDateTimeMinutes
SELECT DATE_PART('mi', '2019-09-04T11:22:33.123Z'::datetime) AS mins;
mins
-----------
22
// end::datePartDateTimeMinutes
;
datePartDateTimeTzOffsetPlus-Ignore
// The timezone is currently ignored when casting a string to datetime
// Awaits fix: https://github.com/elastic/elasticsearch/issues/40692
// tag::datePartDateTimeTzOffsetPlus
SELECT DATE_PART('tzoffset', '2019-09-04T11:22:33.123+05:15'::datetime) AS tz_mins;
tz_mins
--------------
315
// end::datePartDateTimeTzOffsetPlus
;
datePartDateTimeTzOffsetMinus-Ignore
// The timezone is currently ignored when casting a string to datetime
// Awaits fix: https://github.com/elastic/elasticsearch/issues/40692
// tag::datePartDateTimeTzOffsetMinus
SELECT DATE_PART('tzoffset', '2019-09-04T11:22:33.123-03:49'::datetime) AS tz_mins;
tz_mins
--------------
-229
// end::datePartDateTimeTzOffsetMinus
;
datePartDateQuarter
// tag::datePartDateQuarter
SELECT DATE_PART('quarters', CAST('2019-09-24' AS DATE)) AS quarter;
quarter
-------------
3
// end::datePartDateQuarter
;
datePartDateMonth
// tag::datePartDateMonth
SELECT DATE_PART('month', CAST('2019-09-24' AS DATE)) AS month;
month
-------------
9
// end::datePartDateMonth
;
truncateDateTimeMillennium
// tag::truncateDateTimeMillennium
SELECT DATE_TRUNC('millennium', '2019-09-04T11:22:33.123Z'::datetime) AS millennium;

@ -31,6 +31,7 @@ import org.elasticsearch.xpack.sql.expression.function.scalar.User;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.CurrentDate;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.CurrentDateTime;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.CurrentTime;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DatePart;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTrunc;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DayName;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DayOfMonth;
@ -194,6 +195,7 @@ public class FunctionRegistry {
def(DayOfMonth.class, DayOfMonth::new, "DAY_OF_MONTH", "DAYOFMONTH", "DAY", "DOM"),
def(DayOfWeek.class, DayOfWeek::new, "DAY_OF_WEEK", "DAYOFWEEK", "DOW"),
def(DayOfYear.class, DayOfYear::new, "DAY_OF_YEAR", "DAYOFYEAR", "DOY"),
def(DatePart.class, DatePart::new, "DATEPART", "DATE_PART"),
def(DateTrunc.class, DateTrunc::new, "DATETRUNC", "DATE_TRUNC"),
def(HourOfDay.class, HourOfDay::new, "HOUR_OF_DAY", "HOUR"),
def(IsoDayOfWeek.class, IsoDayOfWeek::new, "ISO_DAY_OF_WEEK", "ISODAYOFWEEK", "ISODOW", "IDOW"),

@ -7,6 +7,7 @@ package org.elasticsearch.xpack.sql.expression.function.scalar;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry.Entry;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DatePartProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTimeProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTruncProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.NamedDateTimeProcessor;
@ -89,6 +90,7 @@ public final class Processors {
entries.add(new Entry(Processor.class, NamedDateTimeProcessor.NAME, NamedDateTimeProcessor::new));
entries.add(new Entry(Processor.class, NonIsoDateTimeProcessor.NAME, NonIsoDateTimeProcessor::new));
entries.add(new Entry(Processor.class, QuarterProcessor.NAME, QuarterProcessor::new));
entries.add(new Entry(Processor.class, DatePartProcessor.NAME, DatePartProcessor::new));
entries.add(new Entry(Processor.class, DateTruncProcessor.NAME, DateTruncProcessor::new));
// math
entries.add(new Entry(Processor.class, BinaryMathProcessor.NAME, BinaryMathProcessor::new));

@ -0,0 +1,121 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.sql.expression.function.scalar.datetime;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.expression.Expressions;
import org.elasticsearch.xpack.sql.expression.Nullability;
import org.elasticsearch.xpack.sql.expression.function.scalar.BinaryScalarFunction;
import org.elasticsearch.xpack.sql.expression.gen.pipeline.Pipe;
import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate;
import org.elasticsearch.xpack.sql.tree.Source;
import java.time.ZoneId;
import java.util.List;
import java.util.Objects;
import static org.elasticsearch.common.logging.LoggerMessageFormat.format;
import static org.elasticsearch.xpack.sql.expression.TypeResolutions.isDate;
import static org.elasticsearch.xpack.sql.expression.TypeResolutions.isString;
import static org.elasticsearch.xpack.sql.expression.function.scalar.datetime.BinaryDateTimeProcessor.BinaryDateOperation;
import static org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder.paramsBuilder;
public abstract class BinaryDateTimeFunction extends BinaryScalarFunction {
private final ZoneId zoneId;
private final BinaryDateOperation operation;
public BinaryDateTimeFunction(Source source, Expression datePart, Expression timestamp, ZoneId zoneId,
BinaryDateOperation operation) {
super(source, datePart, timestamp);
this.zoneId = zoneId;
this.operation = operation;
}
@Override
protected TypeResolution resolveType() {
TypeResolution resolution = isString(left(), sourceText(), Expressions.ParamOrdinal.FIRST);
if (resolution.unresolved()) {
return resolution;
}
if (left().foldable()) {
String datePartValue = (String) left().fold();
if (datePartValue != null && resolveDateTimeField(datePartValue) == false) {
List<String> similar = findSimilarDateTimeFields(datePartValue);
if (similar.isEmpty()) {
return new TypeResolution(format(null, "first argument of [{}] must be one of {} or their aliases, found value [{}]",
sourceText(),
validDateTimeFieldValues(),
Expressions.name(left())));
} else {
return new TypeResolution(format(null, "Unknown value [{}] for first argument of [{}]; did you mean {}?",
Expressions.name(left()),
sourceText(),
similar));
}
}
}
resolution = isDate(right(), sourceText(), Expressions.ParamOrdinal.SECOND);
if (resolution.unresolved()) {
return resolution;
}
return TypeResolution.TYPE_RESOLVED;
}
public ZoneId zoneId() {
return zoneId;
}
protected abstract boolean resolveDateTimeField(String dateTimeField);
protected abstract List<String> findSimilarDateTimeFields(String dateTimeField);
protected abstract List<String> validDateTimeFieldValues();
@Override
protected Pipe makePipe() {
return new BinaryDateTimePipe(source(), this, Expressions.pipe(left()), Expressions.pipe(right()), zoneId, operation);
}
@Override
public Nullability nullable() {
return Nullability.TRUE;
}
@Override
protected ScriptTemplate asScriptFrom(ScriptTemplate leftScript, ScriptTemplate rightScript) {
return new ScriptTemplate(
formatTemplate("{sql}." + scriptMethodName() +
"(" + leftScript.template() + "," + rightScript.template()+ ",{})"),
paramsBuilder()
.script(leftScript.params())
.script(rightScript.params())
.variable(zoneId.getId())
.build(),
dataType());
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), zoneId, operation);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
if (!super.equals(o)) {
return false;
}
BinaryDateTimeFunction that = (BinaryDateTimeFunction) o;
return zoneId.equals(that.zoneId) && operation == that.operation;
}
}

@ -8,38 +8,51 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.datetime;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.expression.gen.pipeline.BinaryPipe;
import org.elasticsearch.xpack.sql.expression.gen.pipeline.Pipe;
import org.elasticsearch.xpack.sql.expression.gen.processor.Processor;
import org.elasticsearch.xpack.sql.tree.NodeInfo;
import org.elasticsearch.xpack.sql.tree.Source;
import java.time.ZoneId;
import java.util.Objects;
public class DateTruncPipe extends BinaryPipe {
public class BinaryDateTimePipe extends BinaryPipe {
private final ZoneId zoneId;
private final BinaryDateTimeProcessor.BinaryDateOperation operation;
public DateTruncPipe(Source source, Expression expression, Pipe left, Pipe right, ZoneId zoneId) {
public BinaryDateTimePipe(Source source, Expression expression, Pipe left, Pipe right, ZoneId zoneId,
BinaryDateTimeProcessor.BinaryDateOperation operation) {
super(source, expression, left, right);
this.zoneId = zoneId;
this.operation = operation;
}
ZoneId zoneId() {
return zoneId;
}
BinaryDateTimeProcessor.BinaryDateOperation operation() {
return operation;
}
@Override
protected NodeInfo<DateTruncPipe> info() {
return NodeInfo.create(this, DateTruncPipe::new, expression(), left(), right(), zoneId);
protected NodeInfo<BinaryDateTimePipe> info() {
return NodeInfo.create(this, BinaryDateTimePipe::new, expression(), left(), right(), zoneId, operation);
}
@Override
protected BinaryPipe replaceChildren(Pipe left, Pipe right) {
return new DateTruncPipe(source(), expression(), left, right, zoneId);
return new BinaryDateTimePipe(source(), expression(), left, right, zoneId, operation);
}
@Override
public DateTruncProcessor asProcessor() {
return new DateTruncProcessor(left().asProcessor(), right().asProcessor(), zoneId);
public Processor asProcessor() {
return BinaryDateTimeProcessor.asProcessor(operation, left().asProcessor(), right().asProcessor(), zoneId);
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), zoneId, operation);
}
@Override
@ -53,12 +66,8 @@ public class DateTruncPipe extends BinaryPipe {
if (!super.equals(o)) {
return false;
}
DateTruncPipe that = (DateTruncPipe) o;
return zoneId.equals(that.zoneId);
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), zoneId);
BinaryDateTimePipe that = (BinaryDateTimePipe) o;
return Objects.equals(zoneId, that.zoneId) &&
operation == that.operation;
}
}

@ -0,0 +1,75 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.sql.expression.function.scalar.datetime;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.xpack.sql.common.io.SqlStreamInput;
import org.elasticsearch.xpack.sql.expression.gen.processor.BinaryProcessor;
import org.elasticsearch.xpack.sql.expression.gen.processor.Processor;
import java.io.IOException;
import java.time.ZoneId;
import java.util.Objects;
import static org.elasticsearch.xpack.sql.expression.function.scalar.datetime.BinaryDateTimeProcessor.BinaryDateOperation.TRUNC;
public abstract class BinaryDateTimeProcessor extends BinaryProcessor {
// TODO: Remove and in favour of inheritance (subclasses which implement abstract methods)
public enum BinaryDateOperation {
TRUNC,
PART;
}
private final ZoneId zoneId;
public BinaryDateTimeProcessor(Processor source1, Processor source2, ZoneId zoneId) {
super(source1, source2);
this.zoneId = zoneId;
}
public BinaryDateTimeProcessor(StreamInput in) throws IOException {
super(in);
zoneId = SqlStreamInput.asSqlStream(in).zoneId();
}
@Override
protected void doWrite(StreamOutput out) {
}
ZoneId zoneId() {
return zoneId;
}
@Override
protected abstract Object doProcess(Object left, Object right);
public static BinaryDateTimeProcessor asProcessor(BinaryDateOperation operation, Processor left, Processor right, ZoneId zoneId) {
if (operation == TRUNC) {
return new DateTruncProcessor(left, right, zoneId);
} else {
return new DatePartProcessor(left, right, zoneId);
}
}
@Override
public int hashCode() {
return Objects.hash(zoneId);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
BinaryDateTimeProcessor that = (BinaryDateTimeProcessor) o;
return zoneId.equals(that.zoneId);
}
}

@ -0,0 +1,128 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.sql.expression.function.scalar.datetime;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.expression.Nullability;
import org.elasticsearch.xpack.sql.expression.function.scalar.BinaryScalarFunction;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTimeProcessor.DateTimeExtractor;
import org.elasticsearch.xpack.sql.tree.NodeInfo;
import org.elasticsearch.xpack.sql.tree.Source;
import org.elasticsearch.xpack.sql.type.DataType;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.time.temporal.ChronoField;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import static org.elasticsearch.xpack.sql.expression.function.scalar.datetime.NonIsoDateTimeProcessor.NonIsoDateTimeExtractor;
public class DatePart extends BinaryDateTimeFunction {
public enum Part implements DateTimeField {
YEAR(DateTimeExtractor.YEAR::extract, "years", "yyyy", "yy"),
QUARTER(QuarterProcessor::quarter, "quarters", "qq", "q"),
MONTH(DateTimeExtractor.MONTH_OF_YEAR::extract, "months", "mm", "m"),
DAYOFYEAR(DateTimeExtractor.DAY_OF_YEAR::extract, "dy", "y"),
DAY(DateTimeExtractor.DAY_OF_MONTH::extract, "days", "dd", "d"),
WEEK(NonIsoDateTimeExtractor.WEEK_OF_YEAR::extract, "weeks", "wk", "ww"),
WEEKDAY(NonIsoDateTimeExtractor.DAY_OF_WEEK::extract, "weekdays", "dw"),
HOUR(DateTimeExtractor.HOUR_OF_DAY::extract, "hours", "hh"),
MINUTE(DateTimeExtractor.MINUTE_OF_HOUR::extract, "minutes", "mi", "n"),
SECOND(DateTimeExtractor.SECOND_OF_MINUTE::extract, "seconds", "ss", "s"),
MILLISECOND(dt -> dt.get(ChronoField.MILLI_OF_SECOND), "milliseconds", "ms"),
MICROSECOND(dt -> dt.get(ChronoField.MICRO_OF_SECOND), "microseconds", "mcs"),
NANOSECOND(ZonedDateTime::getNano, "nanoseconds", "ns"),
TZOFFSET(dt -> dt.getOffset().getTotalSeconds() / 60, "tz");
private static final Map<String, Part> NAME_TO_PART;
private static final List<String> VALID_VALUES;
static {
NAME_TO_PART = DateTimeField.initializeResolutionMap(values());
VALID_VALUES = DateTimeField.initializeValidValues(values());
}
private Function<ZonedDateTime, Integer> extractFunction;
private Set<String> aliases;
Part(Function<ZonedDateTime, Integer> extractFunction, String... aliases) {
this.extractFunction = extractFunction;
this.aliases = new HashSet<>(Arrays.asList(aliases));
}
@Override
public Iterable<String> aliases() {
return aliases;
}
public static List<String> findSimilar(String match) {
return DateTimeField.findSimilar(NAME_TO_PART.keySet(), match);
}
public static Part resolve(String truncateTo) {
return DateTimeField.resolveMatch(NAME_TO_PART, truncateTo);
}
public Integer extract(ZonedDateTime dateTime) {
return extractFunction.apply(dateTime);
}
}
public DatePart(Source source, Expression truncateTo, Expression timestamp, ZoneId zoneId) {
super(source, truncateTo, timestamp, zoneId, BinaryDateTimeProcessor.BinaryDateOperation.PART);
}
@Override
public DataType dataType() {
return DataType.INTEGER;
}
@Override
protected BinaryScalarFunction replaceChildren(Expression newTruncateTo, Expression newTimestamp) {
return new DatePart(source(), newTruncateTo, newTimestamp, zoneId());
}
@Override
protected NodeInfo<? extends Expression> info() {
return NodeInfo.create(this, DatePart::new, left(), right(), zoneId());
}
@Override
public Nullability nullable() {
return Nullability.TRUE;
}
@Override
protected boolean resolveDateTimeField(String dateTimeField) {
return Part.resolve(dateTimeField) != null;
}
@Override
protected List<String> findSimilarDateTimeFields(String dateTimeField) {
return Part.findSimilar(dateTimeField);
}
@Override
protected String scriptMethodName() {
return "datePart";
}
@Override
public Object fold() {
return DatePartProcessor.process(left().fold(), right().fold(), zoneId());
}
@Override
protected List<String> validDateTimeFieldValues() {
return Part.VALID_VALUES;
}
}

@ -0,0 +1,69 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.sql.expression.function.scalar.datetime;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.expression.gen.processor.Processor;
import java.io.IOException;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.util.List;
import static org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DatePart.Part;
public class DatePartProcessor extends BinaryDateTimeProcessor {
public static final String NAME = "dtpart";
public DatePartProcessor(Processor source1, Processor source2, ZoneId zoneId) {
super(source1, source2, zoneId);
}
public DatePartProcessor(StreamInput in) throws IOException {
super(in);
}
@Override
public String getWriteableName() {
return NAME;
}
@Override
protected Object doProcess(Object left, Object right) {
return process(left, right, zoneId());
}
/**
* Used in Painless scripting
*/
public static Object process(Object source1, Object source2, ZoneId zoneId) {
if (source1 == null || source2 == null) {
return null;
}
if (source1 instanceof String == false) {
throw new SqlIllegalArgumentException("A string is required; received [{}]", source1);
}
Part datePartField = Part.resolve((String) source1);
if (datePartField == null) {
List<String> similar = Part.findSimilar((String) source1);
if (similar.isEmpty()) {
throw new SqlIllegalArgumentException("A value of {} or their aliases is required; received [{}]",
Part.values(), source1);
} else {
throw new SqlIllegalArgumentException("Received value [{}] is not valid date part for extraction; " +
"did you mean {}?", source1, similar);
}
}
if (source2 instanceof ZonedDateTime == false) {
throw new SqlIllegalArgumentException("A date/datetime is required; received [{}]", source2);
}
return datePartField.extract(((ZonedDateTime) source2).withZoneSameInstant(zoneId));
}
}

@ -0,0 +1,49 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.sql.expression.function.scalar.datetime;
import org.elasticsearch.xpack.sql.util.StringUtils;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.stream.Collectors;
public interface DateTimeField {
static <D extends DateTimeField> Map<String, D> initializeResolutionMap(D[] values) {
Map<String, D> nameToPart = new HashMap<>();
for (D datePart : values) {
String lowerCaseName = datePart.name().toLowerCase(Locale.ROOT);
nameToPart.put(lowerCaseName, datePart);
for (String alias : datePart.aliases()) {
nameToPart.put(alias, datePart);
}
}
return Collections.unmodifiableMap(nameToPart);
}
static <D extends DateTimeField> List<String> initializeValidValues(D[] values) {
return Arrays.stream(values).map(D::name).collect(Collectors.toList());
}
static <D extends DateTimeField> D resolveMatch(Map<String, D> resolutionMap, String possibleMatch) {
return resolutionMap.get(possibleMatch.toLowerCase(Locale.ROOT));
}
static List<String> findSimilar(Iterable<String> similars, String match) {
return StringUtils.findSimilar(match, similars);
}
String name();
Iterable<String> aliases();
}

@ -6,39 +6,28 @@
package org.elasticsearch.xpack.sql.expression.function.scalar.datetime;
import org.elasticsearch.xpack.sql.expression.Expression;
import org.elasticsearch.xpack.sql.expression.Expressions;
import org.elasticsearch.xpack.sql.expression.Nullability;
import org.elasticsearch.xpack.sql.expression.function.scalar.BinaryScalarFunction;
import org.elasticsearch.xpack.sql.expression.gen.pipeline.Pipe;
import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate;
import org.elasticsearch.xpack.sql.tree.NodeInfo;
import org.elasticsearch.xpack.sql.tree.Source;
import org.elasticsearch.xpack.sql.type.DataType;
import org.elasticsearch.xpack.sql.util.StringUtils;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.time.temporal.ChronoField;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.Function;
import static org.elasticsearch.common.logging.LoggerMessageFormat.format;
import static org.elasticsearch.xpack.sql.expression.TypeResolutions.isDate;
import static org.elasticsearch.xpack.sql.expression.TypeResolutions.isString;
import static org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTruncProcessor.process;
import static org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder.paramsBuilder;
import static org.elasticsearch.xpack.sql.expression.function.scalar.datetime.BinaryDateTimeProcessor.BinaryDateOperation.TRUNC;
public class DateTrunc extends BinaryScalarFunction {
public class DateTrunc extends BinaryDateTimeFunction {
public enum Part {
public enum Part implements DateTimeField {
MILLENNIUM(dt -> {
int year = dt.getYear();
@ -113,34 +102,32 @@ public class DateTrunc extends BinaryScalarFunction {
NANOSECOND(dt -> dt, "nanoseconds", "ns");
private static final Map<String, Part> NAME_TO_PART;
private static final List<String> VALID_VALUES;
static {
NAME_TO_PART = new HashMap<>();
for (Part datePart : Part.values()) {
String lowerCaseName = datePart.name().toLowerCase(Locale.ROOT);
NAME_TO_PART.put(lowerCaseName, datePart);
for (String alias : datePart.aliases) {
NAME_TO_PART.put(alias, datePart);
}
}
NAME_TO_PART = DateTimeField.initializeResolutionMap(values());
VALID_VALUES = DateTimeField.initializeValidValues(values());
}
private Set<String> aliases;
private Function<ZonedDateTime, ZonedDateTime> truncateFunction;
private Set<String> aliases;
Part(Function<ZonedDateTime, ZonedDateTime> truncateFunction, String... aliases) {
this.truncateFunction = truncateFunction;
this.aliases = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(aliases)));
}
public static Part resolveTruncate(String truncateTo) {
return NAME_TO_PART.get(truncateTo.toLowerCase(Locale.ROOT));
@Override
public Iterable<String> aliases() {
return aliases;
}
public static List<String> findSimilar(String match) {
return StringUtils.findSimilar(match, NAME_TO_PART.keySet());
return DateTimeField.findSimilar(NAME_TO_PART.keySet(), match);
}
public static Part resolve(String truncateTo) {
return DateTimeField.resolveMatch(NAME_TO_PART, truncateTo);
}
public ZonedDateTime truncate(ZonedDateTime dateTime) {
@ -148,11 +135,8 @@ public class DateTrunc extends BinaryScalarFunction {
}
}
private final ZoneId zoneId;
public DateTrunc(Source source, Expression truncateTo, Expression timestamp, ZoneId zoneId) {
super(source, truncateTo, timestamp);
this.zoneId = zoneId;
super(source, truncateTo, timestamp, zoneId, TRUNC);
}
@Override
@ -160,50 +144,14 @@ public class DateTrunc extends BinaryScalarFunction {
return DataType.DATETIME;
}
@Override
protected TypeResolution resolveType() {
TypeResolution resolution = isString(left(), sourceText(), Expressions.ParamOrdinal.FIRST);
if (resolution.unresolved()) {
return resolution;
}
if (left().foldable()) {
String truncateToValue = (String) left().fold();
if (truncateToValue != null && Part.resolveTruncate(truncateToValue) == null) {
List<String> similar = Part.findSimilar(truncateToValue);
if (similar.isEmpty()) {
return new TypeResolution(format(null, "first argument of [{}] must be one of {} or their aliases, found value [{}]",
sourceText(),
Part.values(),
Expressions.name(left())));
} else {
return new TypeResolution(format(null, "Unknown value [{}] for first argument of [{}]; did you mean {}?",
Expressions.name(left()),
sourceText(),
similar));
}
}
}
resolution = isDate(right(), sourceText(), Expressions.ParamOrdinal.SECOND);
if (resolution.unresolved()) {
return resolution;
}
return TypeResolution.TYPE_RESOLVED;
}
@Override
protected BinaryScalarFunction replaceChildren(Expression newTruncateTo, Expression newTimestamp) {
return new DateTrunc(source(), newTruncateTo, newTimestamp, zoneId);
return new DateTrunc(source(), newTruncateTo, newTimestamp, zoneId());
}
@Override
protected NodeInfo<? extends Expression> info() {
return NodeInfo.create(this, DateTrunc::new, left(), right(), zoneId);
}
@Override
protected Pipe makePipe() {
return new DateTruncPipe(source(), this, Expressions.pipe(left()), Expressions.pipe(right()), zoneId);
return NodeInfo.create(this, DateTrunc::new, left(), right(), zoneId());
}
@Override
@ -211,40 +159,28 @@ public class DateTrunc extends BinaryScalarFunction {
return Nullability.TRUE;
}
@Override
protected boolean resolveDateTimeField(String dateTimeField) {
return Part.resolve(dateTimeField) != null;
}
@Override
protected List<String> findSimilarDateTimeFields(String dateTimeField) {
return Part.findSimilar(dateTimeField);
}
@Override
protected String scriptMethodName() {
return "dateTrunc";
}
@Override
public Object fold() {
return process(left().fold(), right().fold(), zoneId);
return DateTruncProcessor.process(left().fold(), right().fold(), zoneId());
}
@Override
protected ScriptTemplate asScriptFrom(ScriptTemplate leftScript, ScriptTemplate rightScript) {
return new ScriptTemplate(
formatTemplate("{sql}.dateTrunc(" + leftScript.template() + "," + rightScript.template()+ ",{})"),
paramsBuilder()
.script(leftScript.params())
.script(rightScript.params())
.variable(zoneId.getId())
.build(),
dataType());
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), zoneId);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
if (!super.equals(o)) {
return false;
}
DateTrunc dateTrunc = (DateTrunc) o;
return Objects.equals(zoneId, dateTrunc.zoneId);
protected List<String> validDateTimeFieldValues() {
return Part.VALID_VALUES;
}
}

@ -6,34 +6,26 @@
package org.elasticsearch.xpack.sql.expression.function.scalar.datetime;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.common.io.SqlStreamInput;
import org.elasticsearch.xpack.sql.expression.gen.processor.BinaryProcessor;
import org.elasticsearch.xpack.sql.expression.gen.processor.Processor;
import java.io.IOException;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.util.List;
import java.util.Objects;
import static org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTrunc.Part;
public class DateTruncProcessor extends BinaryProcessor {
public class DateTruncProcessor extends BinaryDateTimeProcessor {
public static final String NAME = "dtrunc";
private final ZoneId zoneId;
public DateTruncProcessor(Processor source1, Processor source2, ZoneId zoneId) {
super(source1, source2);
this.zoneId = zoneId;
super(source1, source2, zoneId);
}
public DateTruncProcessor(StreamInput in) throws IOException {
super(in);
zoneId = SqlStreamInput.asSqlStream(in).zoneId();
}
@Override
@ -41,66 +33,37 @@ public class DateTruncProcessor extends BinaryProcessor {
return NAME;
}
@Override
protected void doWrite(StreamOutput out) {
}
ZoneId zoneId() {
return zoneId;
}
@Override
protected Object doProcess(Object left, Object right) {
return process(left, right, zoneId);
return process(left, right, zoneId());
}
/**
* Used in Painless scripting
*/
public static Object process(Object source1, Object source2, String zoneId) {
return process(source1, source2, ZoneId.of(zoneId));
}
static Object process(Object source1, Object source2, ZoneId zoneId) {
public static Object process(Object source1, Object source2, ZoneId zoneId) {
if (source1 == null || source2 == null) {
return null;
}
if (!(source1 instanceof String)) {
if (source1 instanceof String == false) {
throw new SqlIllegalArgumentException("A string is required; received [{}]", source1);
}
Part truncateDateField = Part.resolveTruncate((String) source1);
Part truncateDateField = Part.resolve((String) source1);
if (truncateDateField == null) {
List<String> similar = Part.findSimilar((String) source1);
if (similar.isEmpty()) {
throw new SqlIllegalArgumentException("A value of {} or their aliases is required; received [{}]",
Part.values(), source1);
} else {
throw new SqlIllegalArgumentException("Received value [{}] is not valid date part for truncation; " + "" +
throw new SqlIllegalArgumentException("Received value [{}] is not valid date part for truncation; " +
"did you mean {}?", source1, similar);
}
}
if (!(source2 instanceof ZonedDateTime)) {
throw new SqlIllegalArgumentException("A datetime/date is required; received [{}]", source2);
if (source2 instanceof ZonedDateTime == false) {
throw new SqlIllegalArgumentException("A date/datetime is required; received [{}]", source2);
}
return truncateDateField.truncate(((ZonedDateTime) source2).withZoneSameInstant(zoneId));
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DateTruncProcessor that = (DateTruncProcessor) o;
return zoneId.equals(that.zoneId);
}
@Override
public int hashCode() {
return Objects.hash(zoneId);
}
}

@ -9,6 +9,7 @@ import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.index.fielddata.ScriptDocValues;
import org.elasticsearch.script.JodaCompatibleZonedDateTime;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DatePartProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTimeFunction;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTruncProcessor;
import org.elasticsearch.xpack.sql.expression.function.scalar.datetime.NamedDateTimeProcessor.NameExtractor;
@ -51,6 +52,7 @@ import org.elasticsearch.xpack.sql.util.StringUtils;
import java.time.Duration;
import java.time.OffsetTime;
import java.time.Period;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.util.List;
import java.util.Map;
@ -371,7 +373,11 @@ public final class InternalSqlScriptUtils {
}
public static ZonedDateTime dateTrunc(String truncateTo, Object dateTime, String tzId) {
return (ZonedDateTime) DateTruncProcessor.process(truncateTo, asDateTime(dateTime) ,tzId);
return (ZonedDateTime) DateTruncProcessor.process(truncateTo, asDateTime(dateTime) , ZoneId.of(tzId));
}
public static Integer datePart(String dateField, Object dateTime, String tzId) {
return (Integer) DatePartProcessor.process(dateField, asDateTime(dateTime) , ZoneId.of(tzId));
}
public static ZonedDateTime asDateTime(Object dateTime) {

@ -148,6 +148,4 @@ public final class DateUtils {
nano = nano - nano % (int) Math.pow(10, (9 - precision));
return nano;
}
}

@ -116,6 +116,7 @@ class org.elasticsearch.xpack.sql.expression.function.scalar.whitelist.InternalS
Integer quarter(Object, String)
Integer weekOfYear(Object, String)
ZonedDateTime dateTrunc(String, Object, String)
Integer datePart(String, Object, String)
IntervalDayTime intervalDayTime(String, String)
IntervalYearMonth intervalYearMonth(String, String)
ZonedDateTime asDateTime(Object)

@ -227,6 +227,33 @@ public class VerifierErrorMessagesTests extends ESTestCase {
accept("SELECT DATE_TRUNC('days', date) FROM test");
accept("SELECT DATE_TRUNC('dd', date) FROM test");
accept("SELECT DATE_TRUNC('d', date) FROM test");
}
public void testDatePartInvalidArgs() {
assertEquals("1:8: first argument of [DATE_PART(int, date)] must be [string], found value [int] type [integer]",
error("SELECT DATE_PART(int, date) FROM test"));
assertEquals("1:8: second argument of [DATE_PART(keyword, keyword)] must be [date or datetime], found value [keyword] " +
"type [keyword]", error("SELECT DATE_PART(keyword, keyword) FROM test"));
assertEquals("1:8: first argument of [DATE_PART('invalid', keyword)] must be one of [YEAR, QUARTER, MONTH, DAYOFYEAR, " +
"DAY, WEEK, WEEKDAY, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, NANOSECOND, TZOFFSET] " +
"or their aliases, found value ['invalid']",
error("SELECT DATE_PART('invalid', keyword) FROM test"));
assertEquals("1:8: Unknown value ['tzofset'] for first argument of [DATE_PART('tzofset', keyword)]; " +
"did you mean [tzoffset]?",
error("SELECT DATE_PART('tzofset', keyword) FROM test"));
assertEquals("1:8: Unknown value ['dz'] for first argument of [DATE_PART('dz', keyword)]; " +
"did you mean [dd, tz, dw, dy, d]?",
error("SELECT DATE_PART('dz', keyword) FROM test"));
}
public void testDatePartValidArgs() {
accept("SELECT DATE_PART('weekday', date) FROM test");
accept("SELECT DATE_PART('dw', date) FROM test");
accept("SELECT DATE_PART('tz', date) FROM test");
accept("SELECT DATE_PART('dayofyear', date) FROM test");
accept("SELECT DATE_PART('dy', date) FROM test");
accept("SELECT DATE_PART('ms', date) FROM test");
}
public void testValidDateTimeFunctionsOnTime() {
@ -897,4 +924,4 @@ public class VerifierErrorMessagesTests extends ESTestCase {
assertEquals("1:81: Literal ['bla'] of type [keyword] does not match type [boolean] of PIVOT column [bool]",
error("SELECT * FROM (SELECT int, keyword, bool FROM test) " + "PIVOT(AVG(int) FOR bool IN ('bla', true))"));
}
}
}

@ -25,19 +25,19 @@ import static org.elasticsearch.xpack.sql.expression.Expressions.pipe;
import static org.elasticsearch.xpack.sql.expression.function.scalar.FunctionTestUtils.randomStringLiteral;
import static org.elasticsearch.xpack.sql.tree.SourceTests.randomSource;
public class DateTruncPipeTests extends AbstractNodeTestCase<DateTruncPipe, Pipe> {
public class BinaryDateTimePipeTests extends AbstractNodeTestCase<BinaryDateTimePipe, Pipe> {
@Override
protected DateTruncPipe randomInstance() {
protected BinaryDateTimePipe randomInstance() {
return randomDateTruncPipe();
}
private Expression randomDateTruncPipeExpression() {
return randomDateTruncPipe().expression();
}
public static DateTruncPipe randomDateTruncPipe() {
return (DateTruncPipe) new DateTrunc(
public static BinaryDateTimePipe randomDateTruncPipe() {
return (BinaryDateTimePipe) new DateTrunc(
randomSource(),
randomStringLiteral(),
randomStringLiteral(),
@ -49,50 +49,52 @@ public class DateTruncPipeTests extends AbstractNodeTestCase<DateTruncPipe, Pipe
public void testTransform() {
// test transforming only the properties (source, expression),
// skipping the children (the two parameters of the binary function) which are tested separately
DateTruncPipe b1 = randomInstance();
BinaryDateTimePipe b1 = randomInstance();
Expression newExpression = randomValueOtherThan(b1.expression(), this::randomDateTruncPipeExpression);
DateTruncPipe newB = new DateTruncPipe(
BinaryDateTimePipe newB = new BinaryDateTimePipe(
b1.source(),
newExpression,
b1.left(),
b1.right(),
b1.zoneId());
b1.zoneId(),
b1.operation());
assertEquals(newB, b1.transformPropertiesOnly(v -> Objects.equals(v, b1.expression()) ? newExpression : v, Expression.class));
DateTruncPipe b2 = randomInstance();
BinaryDateTimePipe b2 = randomInstance();
Source newLoc = randomValueOtherThan(b2.source(), SourceTests::randomSource);
newB = new DateTruncPipe(
newB = new BinaryDateTimePipe(
newLoc,
b2.expression(),
b2.left(),
b2.right(),
b2.zoneId());
b2.zoneId(),
b2.operation());
assertEquals(newB,
b2.transformPropertiesOnly(v -> Objects.equals(v, b2.source()) ? newLoc : v, Source.class));
}
@Override
public void testReplaceChildren() {
DateTruncPipe b = randomInstance();
BinaryDateTimePipe b = randomInstance();
Pipe newLeft = pipe(((Expression) randomValueOtherThan(b.left(), FunctionTestUtils::randomStringLiteral)));
Pipe newRight = pipe(((Expression) randomValueOtherThan(b.right(), FunctionTestUtils::randomDatetimeLiteral)));
ZoneId newZoneId = randomValueOtherThan(b.zoneId(), ESTestCase::randomZone);
DateTruncPipe newB =
new DateTruncPipe(b.source(), b.expression(), b.left(), b.right(), newZoneId);
BinaryDateTimePipe newB = new BinaryDateTimePipe(
b.source(), b.expression(), b.left(), b.right(), newZoneId, randomFrom(BinaryDateTimeProcessor.BinaryDateOperation.values()));
BinaryPipe transformed = newB.replaceChildren(newLeft, b.right());
assertEquals(transformed.left(), newLeft);
assertEquals(transformed.source(), b.source());
assertEquals(transformed.expression(), b.expression());
assertEquals(transformed.right(), b.right());
transformed = newB.replaceChildren(b.left(), newRight);
assertEquals(transformed.left(), b.left());
assertEquals(transformed.source(), b.source());
assertEquals(transformed.expression(), b.expression());
assertEquals(transformed.right(), newRight);
transformed = newB.replaceChildren(newLeft, newRight);
assertEquals(transformed.left(), newLeft);
assertEquals(transformed.source(), b.source());
@ -101,33 +103,37 @@ public class DateTruncPipeTests extends AbstractNodeTestCase<DateTruncPipe, Pipe
}
@Override
protected DateTruncPipe mutate(DateTruncPipe instance) {
List<Function<DateTruncPipe, DateTruncPipe>> randoms = new ArrayList<>();
randoms.add(f -> new DateTruncPipe(f.source(),
protected BinaryDateTimePipe mutate(BinaryDateTimePipe instance) {
List<Function<BinaryDateTimePipe, BinaryDateTimePipe>> randoms = new ArrayList<>();
randoms.add(f -> new BinaryDateTimePipe(f.source(),
f.expression(),
pipe(((Expression) randomValueOtherThan(f.left(), FunctionTestUtils::randomStringLiteral))),
f.right(),
randomValueOtherThan(f.zoneId(), ESTestCase::randomZone)));
randoms.add(f -> new DateTruncPipe(f.source(),
randomValueOtherThan(f.zoneId(), ESTestCase::randomZone),
randomFrom(BinaryDateTimeProcessor.BinaryDateOperation.values())));
randoms.add(f -> new BinaryDateTimePipe(f.source(),
f.expression(),
f.left(),
pipe(((Expression) randomValueOtherThan(f.right(), FunctionTestUtils::randomDatetimeLiteral))),
randomValueOtherThan(f.zoneId(), ESTestCase::randomZone)));
randoms.add(f -> new DateTruncPipe(f.source(),
randomValueOtherThan(f.zoneId(), ESTestCase::randomZone),
randomFrom(BinaryDateTimeProcessor.BinaryDateOperation.values())));
randoms.add(f -> new BinaryDateTimePipe(f.source(),
f.expression(),
pipe(((Expression) randomValueOtherThan(f.left(), FunctionTestUtils::randomStringLiteral))),
pipe(((Expression) randomValueOtherThan(f.right(), FunctionTestUtils::randomDatetimeLiteral))),
randomValueOtherThan(f.zoneId(), ESTestCase::randomZone)));
randomValueOtherThan(f.zoneId(), ESTestCase::randomZone),
randomFrom(BinaryDateTimeProcessor.BinaryDateOperation.values())));
return randomFrom(randoms).apply(instance);
}
@Override
protected DateTruncPipe copy(DateTruncPipe instance) {
return new DateTruncPipe(instance.source(),
protected BinaryDateTimePipe copy(BinaryDateTimePipe instance) {
return new BinaryDateTimePipe(instance.source(),
instance.expression(),
instance.left(),
instance.right(),
instance.zoneId());
instance.zoneId(),
instance.operation());
}
}

@ -0,0 +1,103 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License;
* you may not use this file except in compliance with the Elastic License.
*/
package org.elasticsearch.xpack.sql.expression.function.scalar.datetime;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.xpack.sql.AbstractSqlWireSerializingTestCase;
import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
import org.elasticsearch.xpack.sql.expression.Literal;
import org.elasticsearch.xpack.sql.expression.gen.processor.ConstantProcessor;
import org.elasticsearch.xpack.sql.tree.Source;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import static org.elasticsearch.xpack.sql.expression.Literal.NULL;
import static org.elasticsearch.xpack.sql.expression.function.scalar.FunctionTestUtils.l;
import static org.elasticsearch.xpack.sql.expression.function.scalar.FunctionTestUtils.randomDatetimeLiteral;
import static org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTimeTestUtils.dateTime;
public class DatePartProcessorTests extends AbstractSqlWireSerializingTestCase<DatePartProcessor> {
public static DatePartProcessor randomDatePartProcessor() {
return new DatePartProcessor(
new ConstantProcessor(randomRealisticUnicodeOfLengthBetween(0, 128)),
new ConstantProcessor(ZonedDateTime.now()),
randomZone());
}
@Override
protected DatePartProcessor createTestInstance() {
return randomDatePartProcessor();
}
@Override
protected Reader<DatePartProcessor> instanceReader() {
return DatePartProcessor::new;
}
@Override
protected ZoneId instanceZoneId(DatePartProcessor instance) {
return instance.zoneId();
}
@Override
protected DatePartProcessor mutateInstance(DatePartProcessor instance) {
return new DatePartProcessor(
new ConstantProcessor(ESTestCase.randomRealisticUnicodeOfLength(128)),
new ConstantProcessor(ZonedDateTime.now()),
randomValueOtherThan(instance.zoneId(), ESTestCase::randomZone));
}
public void testInvalidInputs() {
SqlIllegalArgumentException siae = expectThrows(SqlIllegalArgumentException.class,
() -> new DatePart(Source.EMPTY, l(5), randomDatetimeLiteral(), randomZone()).makePipe().asProcessor().process(null));
assertEquals("A string is required; received [5]", siae.getMessage());
siae = expectThrows(SqlIllegalArgumentException.class,
() -> new DatePart(Source.EMPTY, l("days"), l("foo"), randomZone()).makePipe().asProcessor().process(null));
assertEquals("A date/datetime is required; received [foo]", siae.getMessage());
siae = expectThrows(SqlIllegalArgumentException.class,
() -> new DatePart(Source.EMPTY, l("invalid"), randomDatetimeLiteral(), randomZone()).makePipe().asProcessor().process(null));
assertEquals("A value of [YEAR, QUARTER, MONTH, DAYOFYEAR, DAY, WEEK, WEEKDAY, HOUR, MINUTE, SECOND, MILLISECOND, " +
"MICROSECOND, NANOSECOND, TZOFFSET] or their aliases is required; received [invalid]",
siae.getMessage());
siae = expectThrows(SqlIllegalArgumentException.class,
() -> new DatePart(Source.EMPTY, l("dayfyear"), randomDatetimeLiteral(), randomZone()).makePipe().asProcessor().process(null));
assertEquals("Received value [dayfyear] is not valid date part for extraction; did you mean [dayofyear, year]?",
siae.getMessage());
}
public void testWithNulls() {
assertNull(new DatePart(Source.EMPTY, NULL, randomDatetimeLiteral(), randomZone()).makePipe().asProcessor().process(null));
assertNull(new DatePart(Source.EMPTY, l("days"), NULL, randomZone()).makePipe().asProcessor().process(null));
assertNull(new DatePart(Source.EMPTY, NULL, NULL, randomZone()).makePipe().asProcessor().process(null));
}
public void testTruncation() {
ZoneId zoneId = ZoneId.of("+05:10");
Literal dateTime = l(dateTime(2007, 10, 30, 12, 15, 32, 123456789));
assertEquals(2007, new DatePart(Source.EMPTY, l("years"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(4, new DatePart(Source.EMPTY, l("quarters"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(10, new DatePart(Source.EMPTY, l("month"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(303, new DatePart(Source.EMPTY, l("dayofyear"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(30, new DatePart(Source.EMPTY, l("day"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(44, new DatePart(Source.EMPTY, l("week"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(3, new DatePart(Source.EMPTY, l("weekday"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(17, new DatePart(Source.EMPTY, l("hour"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(25, new DatePart(Source.EMPTY, l("minutes"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(32, new DatePart(Source.EMPTY, l("ss"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(123, new DatePart(Source.EMPTY, l("ms"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(123456, new DatePart(Source.EMPTY, l("microsecond"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(123456789, new DatePart(Source.EMPTY, l("ns"), dateTime, zoneId).makePipe().asProcessor().process(null));
assertEquals(310, new DatePart(Source.EMPTY, l("tzoffset"), dateTime, zoneId).makePipe().asProcessor().process(null));
}
}

@ -63,7 +63,7 @@ public class DateTruncProcessorTests extends AbstractSqlWireSerializingTestCase<
siae = expectThrows(SqlIllegalArgumentException.class,
() -> new DateTrunc(Source.EMPTY, l("days"), l("foo"), randomZone()).makePipe().asProcessor().process(null));
assertEquals("A datetime/date is required; received [foo]", siae.getMessage());
assertEquals("A date/datetime is required; received [foo]", siae.getMessage());
siae = expectThrows(SqlIllegalArgumentException.class,
() -> new DateTrunc(Source.EMPTY, l("invalid"), randomDatetimeLiteral(), randomZone()).makePipe().asProcessor().process(null));

@ -309,6 +309,22 @@ public class QueryTranslatorTests extends ESTestCase {
assertEquals("[{v=month}, {v=date}, {v=Z}, {v=2018-09-04T00:00:00.000Z}]", sc.script().params().toString());
}
public void testTranslateDatePart_WhereClause_Painless() {
LogicalPlan p = plan("SELECT int FROM test WHERE DATE_PART('month', date) > '2018-09-04'::date");
assertTrue(p instanceof Project);
assertTrue(p.children().get(0) instanceof Filter);
Expression condition = ((Filter) p.children().get(0)).condition();
assertFalse(condition.foldable());
QueryTranslation translation = QueryTranslator.toQuery(condition, false);
assertNull(translation.aggFilter);
assertTrue(translation.query instanceof ScriptQuery);
ScriptQuery sc = (ScriptQuery) translation.query;
assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(InternalSqlScriptUtils.datePart(" +
"params.v0,InternalSqlScriptUtils.docValue(doc,params.v1),params.v2),InternalSqlScriptUtils.asDateTime(params.v3)))",
sc.script().toString());
assertEquals("[{v=month}, {v=date}, {v=Z}, {v=2018-09-04T00:00:00.000Z}]", sc.script().params().toString());
}
public void testLikeOnInexact() {
LogicalPlan p = plan("SELECT * FROM test WHERE some.string LIKE '%a%'");
assertTrue(p instanceof Project);